From a8d7bf7d8a90d602f8f8b30520e4c532837962df Mon Sep 17 00:00:00 2001 From: Kevin Liu Date: Thu, 30 Nov 2023 01:23:24 -0500 Subject: [PATCH 001/350] Add example Trino routine for from_date_string --- docs/src/main/sphinx/routines/examples.md | 44 +++++++++++++++++++++++ 1 file changed, 44 insertions(+) diff --git a/docs/src/main/sphinx/routines/examples.md b/docs/src/main/sphinx/routines/examples.md index 7ed95d24d7986..4e5c9d70dac2c 100644 --- a/docs/src/main/sphinx/routines/examples.md +++ b/docs/src/main/sphinx/routines/examples.md @@ -369,3 +369,47 @@ BEGIN RETURN r; END ``` + +## Date string parsing example + +This example routine parses a date string of type `VARCHAR` into `TIMESTAMP WITH +TIME ZONE`. Date strings are commonly represented by ISO 8601 standard, such as +`2023-12-01`, `2023-12-01T23`. Date strings are also often represented in the +`YYYYmmdd` and `YYYYmmddHH` format, such as `20230101` and `2023010123`. Hive +tables can use this format to represent day and hourly partitions, for example +`/day=20230101`, `/hour=2023010123`. + +This routine parses date strings in a best-effort fashion and can be used as a +replacement for date string manipulation functions such as `date`, `date_parse`, +`from_iso8601_date`, and `from_iso8601_timestamp`. + +Note that the routine defaults the time value to `00:00:00.000` and the time +zone to the session time zone. + + +```sql +FUNCTION from_date_string(date_string VARCHAR) +RETURNS TIMESTAMP WITH TIME ZONE +BEGIN + IF date_string like '%-%' THEN -- ISO 8601 + RETURN from_iso8601_timestamp(date_string); + ELSEIF length(date_string) = 8 THEN -- YYYYmmdd + RETURN date_parse(date_string, '%Y%m%d'); + ELSEIF length(date_string) = 10 THEN -- YYYYmmddHH + RETURN date_parse(date_string, '%Y%m%d%H'); + END IF; + RETURN NULL; +END +``` + +Following are a couple of example invocations with result and explanation: + +```sql +SELECT from_date_string('2023-01-01'); -- 2023-01-01 00:00:00.000 UTC (using the ISO 8601 format) +SELECT from_date_string('2023-01-01T23'); -- 2023-01-01 23:00:00.000 UTC (using the ISO 8601 format) +SELECT from_date_string('2023-01-01T23:23:23'); -- 2023-01-01 23:23:23.000 UTC (using the ISO 8601 format) +SELECT from_date_string('20230101'); -- 2023-01-01 00:00:00.000 UTC (using the YYYYmmdd format) +SELECT from_date_string('2023010123'); -- 2023-01-01 23:00:00.000 UTC (using the YYYYmmddHH format) +SELECT from_date_string(NULL); -- NULL (handles NULL string) +SELECT from_date_string('abc'); -- NULL (not matched to any format) +``` From c0dbc49d8438561a106904ce936848bf65437540 Mon Sep 17 00:00:00 2001 From: James Petty Date: Tue, 5 Dec 2023 13:58:57 -0500 Subject: [PATCH 002/350] Avoid concatenation in checkArgument messages --- .../execution/scheduler/PipelinedQueryScheduler.java | 2 +- .../java/io/trino/metadata/GlobalFunctionCatalog.java | 4 ++-- .../java/io/trino/metadata/InternalFunctionBundle.java | 2 +- .../io/trino/metadata/LanguageFunctionManager.java | 6 +++--- .../src/main/java/io/trino/operator/TaskContext.java | 2 +- .../window/matcher/IrRowPatternToProgramRewriter.java | 5 ++--- .../io/trino/sql/planner/ExpressionInterpreter.java | 2 +- .../io/trino/sql/planner/LocalExecutionPlanner.java | 2 +- ...sformCorrelatedGlobalAggregationWithProjection.java | 2 +- ...rmCorrelatedGlobalAggregationWithoutProjection.java | 2 +- .../rule/TransformCorrelatedScalarSubquery.java | 2 +- .../sql/planner/rowpattern/LogicalIndexPointer.java | 2 +- .../src/main/java/io/trino/util/StatementUtils.java | 4 ++-- .../src/test/java/io/trino/block/BlockAssertions.java | 2 +- .../src/main/java/io/trino/sql/tree/FunctionCall.java | 2 +- .../src/main/java/io/trino/sql/tree/PlanSiblings.java | 2 +- .../src/main/java/io/trino/sql/tree/SkipTo.java | 4 ++-- .../main/java/io/trino/sql/tree/WindowOperation.java | 2 +- .../main/java/io/trino/orc/reader/ColumnReaders.java | 2 +- .../io/trino/parquet/reader/NestedColumnReader.java | 4 ++-- .../io/trino/parquet/reader/flat/FlatColumnReader.java | 2 +- .../reader/flat/FlatDefinitionLevelDecoder.java | 2 +- .../io/trino/parquet/writer/ParquetTypeVisitor.java | 10 +++++----- .../java/io/trino/plugin/mongodb/MongoPageSource.java | 2 +- .../java/io/trino/testing/statistics/StatsContext.java | 2 +- 25 files changed, 36 insertions(+), 37 deletions(-) diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/PipelinedQueryScheduler.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/PipelinedQueryScheduler.java index c370e7bca4da6..d5169126d8ee5 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/PipelinedQueryScheduler.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/PipelinedQueryScheduler.java @@ -585,7 +585,7 @@ private static Map createOutputBuf private static PipelinedOutputBufferManager createSingleStreamOutputBuffer(SqlStage stage) { PartitioningHandle partitioningHandle = stage.getFragment().getOutputPartitioningScheme().getPartitioning().getHandle(); - checkArgument(partitioningHandle.isSingleNode(), "partitioning is expected to be single node: " + partitioningHandle); + checkArgument(partitioningHandle.isSingleNode(), "partitioning is expected to be single node: %s", partitioningHandle); return new PartitionedPipelinedOutputBufferManager(partitioningHandle, 1); } diff --git a/core/trino-main/src/main/java/io/trino/metadata/GlobalFunctionCatalog.java b/core/trino-main/src/main/java/io/trino/metadata/GlobalFunctionCatalog.java index 0b8e28702a7cd..81056644011f9 100644 --- a/core/trino-main/src/main/java/io/trino/metadata/GlobalFunctionCatalog.java +++ b/core/trino-main/src/main/java/io/trino/metadata/GlobalFunctionCatalog.java @@ -272,14 +272,14 @@ public Collection get(String functionName) public FunctionMetadata get(FunctionId functionId) { FunctionMetadata functionMetadata = functionsById.get(functionId); - checkArgument(functionMetadata != null, "Unknown function implementation: " + functionId); + checkArgument(functionMetadata != null, "Unknown function implementation: %s", functionId); return functionMetadata; } public FunctionBundle getFunctionBundle(FunctionId functionId) { FunctionBundle functionBundle = functionBundlesById.get(functionId); - checkArgument(functionBundle != null, "Unknown function implementation: " + functionId); + checkArgument(functionBundle != null, "Unknown function implementation: %s", functionId); return functionBundle; } } diff --git a/core/trino-main/src/main/java/io/trino/metadata/InternalFunctionBundle.java b/core/trino-main/src/main/java/io/trino/metadata/InternalFunctionBundle.java index 20a94c3362fcf..54a8b6c65f502 100644 --- a/core/trino-main/src/main/java/io/trino/metadata/InternalFunctionBundle.java +++ b/core/trino-main/src/main/java/io/trino/metadata/InternalFunctionBundle.java @@ -184,7 +184,7 @@ private WindowFunctionSupplier specializeWindow(FunctionId functionId, BoundSign private SqlFunction getSqlFunction(FunctionId functionId) { SqlFunction function = functions.get(functionId); - checkArgument(function != null, "Unknown function implementation: " + functionId); + checkArgument(function != null, "Unknown function implementation: %s", functionId); return function; } diff --git a/core/trino-main/src/main/java/io/trino/metadata/LanguageFunctionManager.java b/core/trino-main/src/main/java/io/trino/metadata/LanguageFunctionManager.java index e4ae4ecfd77da..a415c339a0b9c 100644 --- a/core/trino-main/src/main/java/io/trino/metadata/LanguageFunctionManager.java +++ b/core/trino-main/src/main/java/io/trino/metadata/LanguageFunctionManager.java @@ -269,7 +269,7 @@ public synchronized List getFunctions(CatalogHandle catalogHan public FunctionDependencyDeclaration getDependencies(FunctionId functionId, AccessControl accessControl) { LanguageFunctionImplementation function = implementationsById.get(functionId); - checkArgument(function != null, "Unknown function implementation: " + functionId); + checkArgument(function != null, "Unknown function implementation: %s", functionId); return function.getFunctionDependencies(accessControl); } @@ -285,7 +285,7 @@ public Optional specialize(ResolvedFunction resolv public FunctionMetadata getFunctionMetadata(FunctionId functionId) { LanguageFunctionImplementation function = implementationsById.get(functionId); - checkArgument(function != null, "Unknown function implementation: " + functionId); + checkArgument(function != null, "Unknown function implementation: %s", functionId); return function.getFunctionMetadata(); } @@ -293,7 +293,7 @@ public void registerResolvedFunction(ResolvedFunction resolvedFunction) { FunctionId functionId = resolvedFunction.getFunctionId(); LanguageFunctionImplementation function = implementationsById.get(functionId); - checkArgument(function != null, "Unknown function implementation: " + functionId); + checkArgument(function != null, "Unknown function implementation: %s", functionId); implementationsByResolvedFunction.put(resolvedFunction, function); } diff --git a/core/trino-main/src/main/java/io/trino/operator/TaskContext.java b/core/trino-main/src/main/java/io/trino/operator/TaskContext.java index f0cfa007d5e1a..3cc4a93e0b2a7 100644 --- a/core/trino-main/src/main/java/io/trino/operator/TaskContext.java +++ b/core/trino-main/src/main/java/io/trino/operator/TaskContext.java @@ -379,7 +379,7 @@ public void setMaxWriterCount(int maxWriterCount) checkArgument(maxWriterCount > 0, "maxWriterCount must be > 0"); int oldMaxWriterCount = this.maxWriterCount.getAndSet(maxWriterCount); - checkArgument(oldMaxWriterCount == -1 || oldMaxWriterCount == maxWriterCount, "maxWriterCount already set to " + oldMaxWriterCount); + checkArgument(oldMaxWriterCount == -1 || oldMaxWriterCount == maxWriterCount, "maxWriterCount already set to %s", oldMaxWriterCount); } public Optional getMaxWriterCount() diff --git a/core/trino-main/src/main/java/io/trino/operator/window/matcher/IrRowPatternToProgramRewriter.java b/core/trino-main/src/main/java/io/trino/operator/window/matcher/IrRowPatternToProgramRewriter.java index b6cb2d8de1351..b7e16be58da0d 100644 --- a/core/trino-main/src/main/java/io/trino/operator/window/matcher/IrRowPatternToProgramRewriter.java +++ b/core/trino-main/src/main/java/io/trino/operator/window/matcher/IrRowPatternToProgramRewriter.java @@ -33,7 +33,6 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.collect.Collections2.orderedPermutations; import static com.google.common.collect.ImmutableList.toImmutableList; -import static java.lang.String.format; import static java.util.Objects.requireNonNull; public class IrRowPatternToProgramRewriter @@ -200,7 +199,7 @@ protected Void visitIrQuantified(IrQuantified node, Void context) private void loopingQuantified(IrRowPattern pattern, boolean greedy, int min) { - checkArgument(min >= 0, "invalid min value: " + min); + checkArgument(min >= 0, "invalid min value: %s", min); if (min == 0) { int startSplitPosition = instructions.size(); @@ -243,7 +242,7 @@ private void loop(int loopingPosition, boolean greedy) private void rangeQuantified(IrRowPattern pattern, boolean greedy, int min, int max) { - checkArgument(min <= max, format("invalid range: (%s, %s)", min, max)); + checkArgument(min <= max, "invalid range: (%s, %s)", min, max); for (int i = 0; i < min; i++) { process(pattern); diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/ExpressionInterpreter.java b/core/trino-main/src/main/java/io/trino/sql/planner/ExpressionInterpreter.java index ebfb26d7209ee..4161e9ce3c3d2 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/ExpressionInterpreter.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/ExpressionInterpreter.java @@ -364,7 +364,7 @@ public Object visitFieldReference(FieldReference node, Object context) @Override protected Object visitDereferenceExpression(DereferenceExpression node, Object context) { - checkArgument(!isQualifiedAllFieldsReference(node), "unexpected expression: all fields labeled reference " + node); + checkArgument(!isQualifiedAllFieldsReference(node), "unexpected expression: all fields labeled reference %s", node); Identifier fieldIdentifier = node.getField().orElseThrow(); Type type = type(node.getBase()); diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/LocalExecutionPlanner.java b/core/trino-main/src/main/java/io/trino/sql/planner/LocalExecutionPlanner.java index fe5bab2061189..c333b05615e6a 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/LocalExecutionPlanner.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/LocalExecutionPlanner.java @@ -906,7 +906,7 @@ public PhysicalOperation visitRemoteSource(RemoteSourceNode node, LocalExecution private PhysicalOperation createMergeSource(RemoteSourceNode node, LocalExecutionPlanContext context) { checkArgument(node.getOrderingScheme().isPresent(), "orderingScheme is absent"); - checkArgument(node.getRetryPolicy() == RetryPolicy.NONE, "unexpected retry policy: " + node.getRetryPolicy()); + checkArgument(node.getRetryPolicy() == RetryPolicy.NONE, "unexpected retry policy: %s", node.getRetryPolicy()); // merging remote source must have a single driver context.setDriverInstanceCount(1); diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedGlobalAggregationWithProjection.java b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedGlobalAggregationWithProjection.java index 6966153344fb4..274626878ed71 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedGlobalAggregationWithProjection.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedGlobalAggregationWithProjection.java @@ -150,7 +150,7 @@ public Pattern getPattern() @Override public Result apply(CorrelatedJoinNode correlatedJoinNode, Captures captures, Context context) { - checkArgument(correlatedJoinNode.getType() == INNER || correlatedJoinNode.getType() == LEFT, "unexpected correlated join type: " + correlatedJoinNode.getType()); + checkArgument(correlatedJoinNode.getType() == INNER || correlatedJoinNode.getType() == LEFT, "unexpected correlated join type: %s", correlatedJoinNode.getType()); // if there is another aggregation below the AggregationNode, handle both PlanNode source = captures.get(SOURCE); diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedGlobalAggregationWithoutProjection.java b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedGlobalAggregationWithoutProjection.java index b8420729e1284..9a3c32e538cba 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedGlobalAggregationWithoutProjection.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedGlobalAggregationWithoutProjection.java @@ -143,7 +143,7 @@ public Pattern getPattern() @Override public Result apply(CorrelatedJoinNode correlatedJoinNode, Captures captures, Context context) { - checkArgument(correlatedJoinNode.getType() == INNER || correlatedJoinNode.getType() == LEFT, "unexpected correlated join type: " + correlatedJoinNode.getType()); + checkArgument(correlatedJoinNode.getType() == INNER || correlatedJoinNode.getType() == LEFT, "unexpected correlated join type: %s", correlatedJoinNode.getType()); PlanNode source = captures.get(SOURCE); diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedScalarSubquery.java b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedScalarSubquery.java index 401219173f837..f42c8dcdf04b3 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedScalarSubquery.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedScalarSubquery.java @@ -103,7 +103,7 @@ public Pattern getPattern() public Result apply(CorrelatedJoinNode correlatedJoinNode, Captures captures, Context context) { // lateral references are only allowed for INNER or LEFT correlated join - checkArgument(correlatedJoinNode.getType() == INNER || correlatedJoinNode.getType() == LEFT, "unexpected correlated join type: " + correlatedJoinNode.getType()); + checkArgument(correlatedJoinNode.getType() == INNER || correlatedJoinNode.getType() == LEFT, "unexpected correlated join type: %s", correlatedJoinNode.getType()); PlanNode subquery = context.getLookup().resolve(correlatedJoinNode.getSubquery()); if (!searchFrom(subquery, context.getLookup()) diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/rowpattern/LogicalIndexPointer.java b/core/trino-main/src/main/java/io/trino/sql/planner/rowpattern/LogicalIndexPointer.java index 57335836a4475..1edec2145f7dc 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/rowpattern/LogicalIndexPointer.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/rowpattern/LogicalIndexPointer.java @@ -53,7 +53,7 @@ public LogicalIndexPointer(Set labels, boolean last, boolean running, i this.labels = requireNonNull(labels, "labels is null"); this.last = last; this.running = running; - checkArgument(logicalOffset >= 0, "logical offset must be >= 0, actual: " + logicalOffset); + checkArgument(logicalOffset >= 0, "logical offset must be >= 0, actual: %s", logicalOffset); this.logicalOffset = logicalOffset; this.physicalOffset = physicalOffset; } diff --git a/core/trino-main/src/main/java/io/trino/util/StatementUtils.java b/core/trino-main/src/main/java/io/trino/util/StatementUtils.java index cf68c4ab700b8..d89628f67e4e9 100644 --- a/core/trino-main/src/main/java/io/trino/util/StatementUtils.java +++ b/core/trino-main/src/main/java/io/trino/util/StatementUtils.java @@ -317,10 +317,10 @@ private StatementTypeInfo(Class statementType, this.queryType = requireNonNull(queryType, "queryType is null"); this.taskType = requireNonNull(taskType, "taskType is null"); if (queryType == DATA_DEFINITION) { - checkArgument(taskType.isPresent(), "taskType is required for " + DATA_DEFINITION); + checkArgument(taskType.isPresent(), "taskType is required for %s", DATA_DEFINITION); } else { - checkArgument(taskType.isEmpty(), "taskType is not allowed for " + queryType); + checkArgument(taskType.isEmpty(), "taskType is not allowed for %s", queryType); } } diff --git a/core/trino-main/src/test/java/io/trino/block/BlockAssertions.java b/core/trino-main/src/test/java/io/trino/block/BlockAssertions.java index b6cffe2973545..038e82963e82c 100644 --- a/core/trino-main/src/test/java/io/trino/block/BlockAssertions.java +++ b/core/trino-main/src/test/java/io/trino/block/BlockAssertions.java @@ -139,7 +139,7 @@ public static Block createRandomDictionaryBlock(Block dictionary, int positionCo public static RunLengthEncodedBlock createRandomRleBlock(Block block, int positionCount) { - checkArgument(block.getPositionCount() >= 2, format("block positions %d is less 2", block.getPositionCount())); + checkArgument(block.getPositionCount() >= 2, "block positions %s is less than 2", block.getPositionCount()); return (RunLengthEncodedBlock) RunLengthEncodedBlock.create(block.getSingleValueBlock(random().nextInt(block.getPositionCount())), positionCount); } diff --git a/core/trino-parser/src/main/java/io/trino/sql/tree/FunctionCall.java b/core/trino-parser/src/main/java/io/trino/sql/tree/FunctionCall.java index a24d0787a0195..8103fde855d1f 100644 --- a/core/trino-parser/src/main/java/io/trino/sql/tree/FunctionCall.java +++ b/core/trino-parser/src/main/java/io/trino/sql/tree/FunctionCall.java @@ -58,7 +58,7 @@ public FunctionCall( super(location); requireNonNull(name, "name is null"); requireNonNull(window, "window is null"); - window.ifPresent(node -> checkArgument(node instanceof WindowReference || node instanceof WindowSpecification, "unexpected window: " + node.getClass().getSimpleName())); + window.ifPresent(node -> checkArgument(node instanceof WindowReference || node instanceof WindowSpecification, "unexpected window: %s", node.getClass().getSimpleName())); requireNonNull(filter, "filter is null"); requireNonNull(orderBy, "orderBy is null"); requireNonNull(nullTreatment, "nullTreatment is null"); diff --git a/core/trino-parser/src/main/java/io/trino/sql/tree/PlanSiblings.java b/core/trino-parser/src/main/java/io/trino/sql/tree/PlanSiblings.java index 844eb4ebb8dcb..46b7a91de48ed 100644 --- a/core/trino-parser/src/main/java/io/trino/sql/tree/PlanSiblings.java +++ b/core/trino-parser/src/main/java/io/trino/sql/tree/PlanSiblings.java @@ -33,7 +33,7 @@ public PlanSiblings(NodeLocation location, SiblingsPlanType type, List= 2, "sibling plan must contain at least two siblings, actual: " + siblings.size()); + checkArgument(siblings.size() >= 2, "sibling plan must contain at least two siblings, actual: %s", siblings.size()); } public SiblingsPlanType getType() diff --git a/core/trino-parser/src/main/java/io/trino/sql/tree/SkipTo.java b/core/trino-parser/src/main/java/io/trino/sql/tree/SkipTo.java index 4b4e8c2bdf19c..1742e476fc8ed 100644 --- a/core/trino-parser/src/main/java/io/trino/sql/tree/SkipTo.java +++ b/core/trino-parser/src/main/java/io/trino/sql/tree/SkipTo.java @@ -107,8 +107,8 @@ private SkipTo(Optional location, Position position, Optional location, Identifier name, Window super(location); requireNonNull(name, "name is null"); requireNonNull(window, "window is null"); - checkArgument(window instanceof WindowReference || window instanceof WindowSpecification, "unexpected window: " + window.getClass().getSimpleName()); + checkArgument(window instanceof WindowReference || window instanceof WindowSpecification, "unexpected window: %s", window.getClass().getSimpleName()); this.name = name; this.window = window; diff --git a/lib/trino-orc/src/main/java/io/trino/orc/reader/ColumnReaders.java b/lib/trino-orc/src/main/java/io/trino/orc/reader/ColumnReaders.java index 27550fe1ce6c9..8d4b0260d8ca6 100644 --- a/lib/trino-orc/src/main/java/io/trino/orc/reader/ColumnReaders.java +++ b/lib/trino-orc/src/main/java/io/trino/orc/reader/ColumnReaders.java @@ -54,7 +54,7 @@ public static ColumnReader createColumnReader( return new TimeColumnReader(type, column, memoryContext.newLocalMemoryContext(ColumnReaders.class.getSimpleName())); } if (type instanceof UuidType) { - checkArgument(column.getColumnType() == BINARY, "UUID type can only be read from BINARY column but got " + column); + checkArgument(column.getColumnType() == BINARY, "UUID type can only be read from BINARY column but got %s", column); checkArgument( "UUID".equals(column.getAttributes().get(ICEBERG_BINARY_TYPE)), "Expected ORC column for UUID data to be annotated with %s=UUID: %s", diff --git a/lib/trino-parquet/src/main/java/io/trino/parquet/reader/NestedColumnReader.java b/lib/trino-parquet/src/main/java/io/trino/parquet/reader/NestedColumnReader.java index 6870ce3122a9b..2ceff847f3264 100644 --- a/lib/trino-parquet/src/main/java/io/trino/parquet/reader/NestedColumnReader.java +++ b/lib/trino-parquet/src/main/java/io/trino/parquet/reader/NestedColumnReader.java @@ -507,8 +507,8 @@ private void readFlatPageV1(DataPageV1 page) int maxDefinitionLevel = field.getDefinitionLevel(); int maxRepetitionLevel = field.getRepetitionLevel(); - checkArgument(maxDefinitionLevel == 0 || definitionEncoding == RLE, "Invalid definition level encoding: " + definitionEncoding); - checkArgument(maxRepetitionLevel == 0 || repetitionEncoding == RLE, "Invalid repetition level encoding: " + definitionEncoding); + checkArgument(maxDefinitionLevel == 0 || definitionEncoding == RLE, "Invalid definition level encoding: %s", definitionEncoding); + checkArgument(maxRepetitionLevel == 0 || repetitionEncoding == RLE, "Invalid repetition level encoding: %s", repetitionEncoding); repetitionLevelDecoder = levelsDecoderProvider.create(maxRepetitionLevel); if (maxRepetitionLevel > 0) { diff --git a/lib/trino-parquet/src/main/java/io/trino/parquet/reader/flat/FlatColumnReader.java b/lib/trino-parquet/src/main/java/io/trino/parquet/reader/flat/FlatColumnReader.java index 45347567ead26..5d2b190b620e7 100644 --- a/lib/trino-parquet/src/main/java/io/trino/parquet/reader/flat/FlatColumnReader.java +++ b/lib/trino-parquet/src/main/java/io/trino/parquet/reader/flat/FlatColumnReader.java @@ -280,7 +280,7 @@ private void readFlatPageV1(DataPageV1 page) Slice buffer = page.getSlice(); ParquetEncoding definitionEncoding = page.getDefinitionLevelEncoding(); - checkArgument(isNonNull() || definitionEncoding == RLE, "Invalid definition level encoding: " + definitionEncoding); + checkArgument(isNonNull() || definitionEncoding == RLE, "Invalid definition level encoding: %s", definitionEncoding); int alreadyRead = 0; if (definitionEncoding == RLE) { // Definition levels are skipped from file when the max definition level is 0 as the bit-width required to store them is 0. diff --git a/lib/trino-parquet/src/main/java/io/trino/parquet/reader/flat/FlatDefinitionLevelDecoder.java b/lib/trino-parquet/src/main/java/io/trino/parquet/reader/flat/FlatDefinitionLevelDecoder.java index de629401fb4af..84fcc6ac85ea9 100644 --- a/lib/trino-parquet/src/main/java/io/trino/parquet/reader/flat/FlatDefinitionLevelDecoder.java +++ b/lib/trino-parquet/src/main/java/io/trino/parquet/reader/flat/FlatDefinitionLevelDecoder.java @@ -39,7 +39,7 @@ interface DefinitionLevelDecoderProvider static FlatDefinitionLevelDecoder getFlatDefinitionLevelDecoder(int maxDefinitionLevel) { - checkArgument(maxDefinitionLevel >= 0 && maxDefinitionLevel <= 1, "Invalid max definition level: " + maxDefinitionLevel); + checkArgument(maxDefinitionLevel >= 0 && maxDefinitionLevel <= 1, "Invalid max definition level: %s", maxDefinitionLevel); if (maxDefinitionLevel == 0) { return new ZeroDefinitionLevelDecoder(); } diff --git a/lib/trino-parquet/src/main/java/io/trino/parquet/writer/ParquetTypeVisitor.java b/lib/trino-parquet/src/main/java/io/trino/parquet/writer/ParquetTypeVisitor.java index d494770fe64a2..13edc85f59972 100644 --- a/lib/trino-parquet/src/main/java/io/trino/parquet/writer/ParquetTypeVisitor.java +++ b/lib/trino-parquet/src/main/java/io/trino/parquet/writer/ParquetTypeVisitor.java @@ -44,15 +44,15 @@ public static T visit(Type type, ParquetTypeVisitor visitor) LogicalTypeAnnotation annotation = group.getLogicalTypeAnnotation(); if (LogicalTypeAnnotation.listType().equals(annotation)) { checkArgument(!group.isRepetition(REPEATED), - "Invalid list: top-level group is repeated: " + group); + "Invalid list: top-level group is repeated: %s", group); checkArgument(group.getFieldCount() == 1, - "Invalid list: does not contain single repeated field: " + group); + "Invalid list: does not contain single repeated field: %s", group); GroupType repeatedElement = group.getFields().get(0).asGroupType(); checkArgument(repeatedElement.isRepetition(REPEATED), "Invalid list: inner group is not repeated"); checkArgument(repeatedElement.getFieldCount() <= 1, - "Invalid list: repeated group is not a single field: " + group); + "Invalid list: repeated group is not a single field: %s", group); visitor.fieldNames.push(repeatedElement.getName()); try { @@ -69,9 +69,9 @@ public static T visit(Type type, ParquetTypeVisitor visitor) } if (LogicalTypeAnnotation.mapType().equals(annotation)) { checkArgument(!group.isRepetition(REPEATED), - "Invalid map: top-level group is repeated: " + group); + "Invalid map: top-level group is repeated: %s", group); checkArgument(group.getFieldCount() == 1, - "Invalid map: does not contain single repeated field: " + group); + "Invalid map: does not contain single repeated field: %s", group); GroupType repeatedKeyValue = group.getType(0).asGroupType(); checkArgument(repeatedKeyValue.isRepetition(REPEATED), diff --git a/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/MongoPageSource.java b/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/MongoPageSource.java index 130f4d344ca8f..3bdf2d61f9f64 100644 --- a/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/MongoPageSource.java +++ b/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/MongoPageSource.java @@ -411,7 +411,7 @@ private static Object getDbRefValue(DBRef dbRefValue, MongoColumnHandle columnHa if (columnHandle.getType() instanceof RowType) { return dbRefValue; } - checkArgument(columnHandle.isDbRefField(), "columnHandle is not a dbRef field: " + columnHandle); + checkArgument(columnHandle.isDbRefField(), "columnHandle is not a dbRef field: %s", columnHandle); List dereferenceNames = columnHandle.getDereferenceNames(); checkState(!dereferenceNames.isEmpty(), "dereferenceNames is empty"); String leafColumnName = dereferenceNames.get(dereferenceNames.size() - 1); diff --git a/testing/trino-testing/src/main/java/io/trino/testing/statistics/StatsContext.java b/testing/trino-testing/src/main/java/io/trino/testing/statistics/StatsContext.java index bb8b3bb8c7e94..f86ed2fe9599b 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/statistics/StatsContext.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/statistics/StatsContext.java @@ -36,7 +36,7 @@ public StatsContext(Map columnSymbols, TypeProvider types) public Symbol getSymbolForColumn(String columnName) { - checkArgument(columnSymbols.containsKey(columnName), "no symbol found for column '" + columnName + "'"); + checkArgument(columnSymbols.containsKey(columnName), "no symbol found for column '%s'", columnName); return columnSymbols.get(columnName); } From 661ef1a4d70b1be7a98597106f69d362f6dc6d63 Mon Sep 17 00:00:00 2001 From: Dain Sundstrom Date: Sat, 2 Dec 2023 00:28:01 -0800 Subject: [PATCH 003/350] Change TrinoFileSystem delete file to not fail when file is missing --- .../filesystem/azure/AzureFileSystem.java | 4 ++++ .../io/trino/filesystem/azure/AzureUtils.java | 22 +++++++++++-------- .../trino/filesystem/gcs/GcsFileSystem.java | 8 +++---- .../s3/AbstractTestS3FileSystem.java | 6 ----- .../io/trino/filesystem/TrinoFileSystem.java | 6 ++--- .../filesystem/local/LocalFileSystem.java | 3 +++ .../filesystem/memory/MemoryFileSystem.java | 5 +---- .../AbstractTestTrinoFileSystem.java | 17 ++------------ .../trino/filesystem/hdfs/HdfsFileSystem.java | 5 ++--- .../hdfs/TestHdfsFileSystemS3Mock.java | 6 ----- 10 files changed, 32 insertions(+), 50 deletions(-) diff --git a/lib/trino-filesystem-azure/src/main/java/io/trino/filesystem/azure/AzureFileSystem.java b/lib/trino-filesystem-azure/src/main/java/io/trino/filesystem/azure/AzureFileSystem.java index 702b68a38a3c9..40c75f9424601 100644 --- a/lib/trino-filesystem-azure/src/main/java/io/trino/filesystem/azure/AzureFileSystem.java +++ b/lib/trino-filesystem-azure/src/main/java/io/trino/filesystem/azure/AzureFileSystem.java @@ -53,6 +53,7 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.collect.ImmutableSet.toImmutableSet; import static io.trino.filesystem.azure.AzureUtils.handleAzureException; +import static io.trino.filesystem.azure.AzureUtils.isFileNotFoundException; import static java.lang.Math.toIntExact; import static java.util.Objects.requireNonNull; import static java.util.UUID.randomUUID; @@ -123,6 +124,9 @@ public void deleteFile(Location location) client.delete(); } catch (RuntimeException e) { + if (isFileNotFoundException(e)) { + return; + } throw handleAzureException(e, "deleting file", azureLocation); } } diff --git a/lib/trino-filesystem-azure/src/main/java/io/trino/filesystem/azure/AzureUtils.java b/lib/trino-filesystem-azure/src/main/java/io/trino/filesystem/azure/AzureUtils.java index ef0c01c299f54..2117af82671f7 100644 --- a/lib/trino-filesystem-azure/src/main/java/io/trino/filesystem/azure/AzureUtils.java +++ b/lib/trino-filesystem-azure/src/main/java/io/trino/filesystem/azure/AzureUtils.java @@ -28,15 +28,8 @@ private AzureUtils() {} public static IOException handleAzureException(RuntimeException exception, String action, AzureLocation location) throws IOException { - if (exception instanceof BlobStorageException blobStorageException) { - if (BlobErrorCode.BLOB_NOT_FOUND.equals(blobStorageException.getErrorCode())) { - throw withCause(new FileNotFoundException(location.toString()), exception); - } - } - if (exception instanceof DataLakeStorageException dataLakeStorageException) { - if ("PathNotFound".equals(dataLakeStorageException.getErrorCode())) { - throw withCause(new FileNotFoundException(location.toString()), exception); - } + if (isFileNotFoundException(exception)) { + throw withCause(new FileNotFoundException(location.toString()), exception); } if (exception instanceof AzureException) { throw new IOException("Azure service error %s file: %s".formatted(action, location), exception); @@ -44,6 +37,17 @@ public static IOException handleAzureException(RuntimeException exception, Strin throw new IOException("Error %s file: %s".formatted(action, location), exception); } + public static boolean isFileNotFoundException(RuntimeException exception) + { + if (exception instanceof BlobStorageException blobStorageException) { + return BlobErrorCode.BLOB_NOT_FOUND.equals(blobStorageException.getErrorCode()); + } + if (exception instanceof DataLakeStorageException dataLakeStorageException) { + return "PathNotFound" .equals(dataLakeStorageException.getErrorCode()); + } + return false; + } + private static T withCause(T throwable, Throwable cause) { throwable.initCause(cause); diff --git a/lib/trino-filesystem-gcs/src/main/java/io/trino/filesystem/gcs/GcsFileSystem.java b/lib/trino-filesystem-gcs/src/main/java/io/trino/filesystem/gcs/GcsFileSystem.java index 3b7f2383cf4c4..c971925b34914 100644 --- a/lib/trino-filesystem-gcs/src/main/java/io/trino/filesystem/gcs/GcsFileSystem.java +++ b/lib/trino-filesystem-gcs/src/main/java/io/trino/filesystem/gcs/GcsFileSystem.java @@ -45,7 +45,7 @@ import static com.google.cloud.storage.Storage.BlobListOption.pageSize; import static com.google.common.collect.Iterables.partition; import static io.airlift.concurrent.MoreFutures.getFutureValue; -import static io.trino.filesystem.gcs.GcsUtils.getBlobOrThrow; +import static io.trino.filesystem.gcs.GcsUtils.getBlob; import static io.trino.filesystem.gcs.GcsUtils.handleGcsException; import static java.util.Objects.requireNonNull; @@ -99,8 +99,7 @@ public void deleteFile(Location location) { GcsLocation gcsLocation = new GcsLocation(location); checkIsValidFile(gcsLocation); - Blob blob = getBlobOrThrow(storage, gcsLocation); - blob.delete(); + getBlob(storage, gcsLocation).ifPresent(Blob::delete); } @Override @@ -112,7 +111,8 @@ public void deleteFiles(Collection locations) for (List locationBatch : partition(locations, batchSize)) { StorageBatch batch = storage.batch(); for (Location location : locationBatch) { - batch.delete(getBlobOrThrow(storage, new GcsLocation(location)).getBlobId()); + getBlob(storage, new GcsLocation(location)) + .ifPresent(blob -> batch.delete(blob.getBlobId())); } batchFutures.add(executorService.submit(batch::submit)); } diff --git a/lib/trino-filesystem-s3/src/test/java/io/trino/filesystem/s3/AbstractTestS3FileSystem.java b/lib/trino-filesystem-s3/src/test/java/io/trino/filesystem/s3/AbstractTestS3FileSystem.java index b755f5a403dc8..9f68d3b5efe84 100644 --- a/lib/trino-filesystem-s3/src/test/java/io/trino/filesystem/s3/AbstractTestS3FileSystem.java +++ b/lib/trino-filesystem-s3/src/test/java/io/trino/filesystem/s3/AbstractTestS3FileSystem.java @@ -93,12 +93,6 @@ protected final boolean supportsRenameFile() return false; } - @Override - protected final boolean deleteFileFailsIfNotExists() - { - return false; - } - @Override protected final void verifyFileSystemIsEmpty() { diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/TrinoFileSystem.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/TrinoFileSystem.java index 9692154a39d02..f4633964fa450 100644 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/TrinoFileSystem.java +++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/TrinoFileSystem.java @@ -78,10 +78,10 @@ public interface TrinoFileSystem /** * Deletes the specified file. The file location path cannot be empty, and must not end with - * a slash or whitespace. If the file is a director, an exception is raised. + * a slash or whitespace. If the file is a director, an exception is raised. If the file does + * not exist, this method is a noop. * * @throws IllegalArgumentException if location is not valid for this file system - * @throws IOException if the file does not exist (optional) or was not deleted */ void deleteFile(Location location) throws IOException; @@ -90,9 +90,9 @@ void deleteFile(Location location) * Delete specified files. This operation is not required to be atomic, so if an error * occurs, all, some, or, none of the files may be deleted. This operation may be faster than simply * looping over the locations as some file systems support batch delete operations natively. + * If a file does not exist, it is ignored. * * @throws IllegalArgumentException if location is not valid for this file system - * @throws IOException if a file does not exist (optional) or was not deleted */ default void deleteFiles(Collection locations) throws IOException diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalFileSystem.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalFileSystem.java index a6fa596124c5a..9d071b7c2d0c1 100644 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalFileSystem.java +++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalFileSystem.java @@ -23,6 +23,7 @@ import java.io.IOException; import java.nio.file.FileVisitResult; import java.nio.file.Files; +import java.nio.file.NoSuchFileException; import java.nio.file.Path; import java.nio.file.SimpleFileVisitor; import java.nio.file.attribute.BasicFileAttributes; @@ -75,6 +76,8 @@ public void deleteFile(Location location) try { Files.delete(filePath); } + catch (NoSuchFileException ignored) { + } catch (IOException e) { throw handleException(location, e); } diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryFileSystem.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryFileSystem.java index 1b07a632fa695..33f5135ce32b1 100644 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryFileSystem.java +++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryFileSystem.java @@ -23,7 +23,6 @@ import io.trino.filesystem.TrinoOutputFile; import io.trino.filesystem.memory.MemoryOutputFile.OutputBlob; -import java.io.FileNotFoundException; import java.io.IOException; import java.nio.file.FileAlreadyExistsException; import java.util.Iterator; @@ -96,9 +95,7 @@ public void overwriteBlob(Slice data) public void deleteFile(Location location) throws IOException { - if (blobs.remove(toBlobKey(location)) == null) { - throw new FileNotFoundException(location.toString()); - } + blobs.remove(toBlobKey(location)); } @Override diff --git a/lib/trino-filesystem/src/test/java/io/trino/filesystem/AbstractTestTrinoFileSystem.java b/lib/trino-filesystem/src/test/java/io/trino/filesystem/AbstractTestTrinoFileSystem.java index b11890c86b0f9..4f18850ad0633 100644 --- a/lib/trino-filesystem/src/test/java/io/trino/filesystem/AbstractTestTrinoFileSystem.java +++ b/lib/trino-filesystem/src/test/java/io/trino/filesystem/AbstractTestTrinoFileSystem.java @@ -73,11 +73,6 @@ protected boolean supportsRenameFile() return true; } - protected boolean deleteFileFailsIfNotExists() - { - return true; - } - protected boolean normalizesListFilesResult() { return false; @@ -671,16 +666,8 @@ void testDeleteFile() .hasMessageContaining(createLocation("foo/").toString()); try (TempBlob tempBlob = randomBlobLocation("delete")) { - if (deleteFileFailsIfNotExists()) { - // deleting a non-existent file is an error - assertThatThrownBy(() -> getFileSystem().deleteFile(tempBlob.location())) - .isInstanceOf(FileNotFoundException.class) - .hasMessageContaining(tempBlob.location().toString()); - } - else { - // deleting a non-existent file is a no-op - getFileSystem().deleteFile(tempBlob.location()); - } + // deleting a non-existent file is a no-op + getFileSystem().deleteFile(tempBlob.location()); tempBlob.createOrOverwrite("delete me"); diff --git a/lib/trino-hdfs/src/main/java/io/trino/filesystem/hdfs/HdfsFileSystem.java b/lib/trino-hdfs/src/main/java/io/trino/filesystem/hdfs/HdfsFileSystem.java index e1ae9400f4614..c7f8d83fd759a 100644 --- a/lib/trino-hdfs/src/main/java/io/trino/filesystem/hdfs/HdfsFileSystem.java +++ b/lib/trino-hdfs/src/main/java/io/trino/filesystem/hdfs/HdfsFileSystem.java @@ -114,9 +114,8 @@ public void deleteFile(Location location) } return null; } - catch (FileNotFoundException e) { - stats.getDeleteFileCalls().recordException(e); - throw new FileNotFoundException(location.toString()); + catch (FileNotFoundException ignored) { + return null; } catch (IOException e) { stats.getDeleteFileCalls().recordException(e); diff --git a/lib/trino-hdfs/src/test/java/io/trino/filesystem/hdfs/TestHdfsFileSystemS3Mock.java b/lib/trino-hdfs/src/test/java/io/trino/filesystem/hdfs/TestHdfsFileSystemS3Mock.java index 45f3d49f372e8..f477a75e793ca 100644 --- a/lib/trino-hdfs/src/test/java/io/trino/filesystem/hdfs/TestHdfsFileSystemS3Mock.java +++ b/lib/trino-hdfs/src/test/java/io/trino/filesystem/hdfs/TestHdfsFileSystemS3Mock.java @@ -114,12 +114,6 @@ protected final boolean supportsCreateExclusive() return false; } - @Override - protected final boolean deleteFileFailsIfNotExists() - { - return false; - } - @Override protected boolean normalizesListFilesResult() { From cb5c6a79951028b0c37bc45976a608a04d310c5a Mon Sep 17 00:00:00 2001 From: Dain Sundstrom Date: Thu, 30 Nov 2023 13:10:28 -0800 Subject: [PATCH 004/350] Remove unused TestingExtendedHiveMetastore --- .../hudi/TestingExtendedHiveMetastore.java | 48 ------------------- 1 file changed, 48 deletions(-) delete mode 100644 plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestingExtendedHiveMetastore.java diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestingExtendedHiveMetastore.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestingExtendedHiveMetastore.java deleted file mode 100644 index fbe97020b0fe6..0000000000000 --- a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestingExtendedHiveMetastore.java +++ /dev/null @@ -1,48 +0,0 @@ -/* - * 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 io.trino.plugin.hive.metastore.Table; -import io.trino.plugin.hive.metastore.UnimplementedHiveMetastore; -import io.trino.spi.predicate.TupleDomain; - -import java.util.List; -import java.util.Optional; - -import static java.util.Objects.requireNonNull; - -public class TestingExtendedHiveMetastore - extends UnimplementedHiveMetastore -{ - private final Table table; - private final List partitions; - - public TestingExtendedHiveMetastore(Table table, List partitions) - { - this.table = requireNonNull(table, "table is null"); - this.partitions = requireNonNull(partitions, "partitions is null"); - } - - @Override - public Optional getTable(String databaseName, String tableName) - { - return Optional.of(table); - } - - @Override - public Optional> getPartitionNamesByFilter(String databaseName, String tableName, List columnNames, TupleDomain partitionKeysFilter) - { - return Optional.of(partitions); - } -} From a5fe7b896e8fa53351a9ab73df22a814453866cd Mon Sep 17 00:00:00 2001 From: Dain Sundstrom Date: Tue, 7 Nov 2023 22:00:15 -0800 Subject: [PATCH 005/350] Use simpler TestingHivePlugin constructor --- .../hive/TestHiveConcurrentModificationGlueMetastore.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveConcurrentModificationGlueMetastore.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveConcurrentModificationGlueMetastore.java index c082dfd97b7f1..ad60fe9a950c4 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveConcurrentModificationGlueMetastore.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveConcurrentModificationGlueMetastore.java @@ -31,7 +31,6 @@ import java.io.IOException; import java.lang.reflect.InvocationTargetException; import java.nio.file.Path; -import java.util.Optional; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -39,7 +38,6 @@ import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; import static com.google.common.reflect.Reflection.newProxy; import static com.google.common.util.concurrent.MoreExecutors.directExecutor; -import static com.google.inject.util.Modules.EMPTY_MODULE; import static io.trino.plugin.hive.HiveErrorCode.HIVE_METASTORE_ERROR; import static io.trino.plugin.hive.HiveTestUtils.HDFS_FILE_SYSTEM_FACTORY; import static io.trino.plugin.hive.metastore.glue.TestingGlueHiveMetastore.createTestingAsyncGlueClient; @@ -102,7 +100,7 @@ protected QueryRunner createQueryRunner() stats, table -> true); - queryRunner.installPlugin(new TestingHivePlugin(Optional.of(metastore), Optional.empty(), EMPTY_MODULE, Optional.empty())); + queryRunner.installPlugin(new TestingHivePlugin(metastore)); queryRunner.createCatalog(CATALOG_NAME, "hive"); queryRunner.execute("CREATE SCHEMA " + SCHEMA); return queryRunner; From fb1b5d676f562007680493702db47272cd70fc6c Mon Sep 17 00:00:00 2001 From: Dain Sundstrom Date: Wed, 8 Nov 2023 21:15:55 -0800 Subject: [PATCH 006/350] Implement LocalFileSystem createTemporaryDirectory --- .../filesystem/local/LocalFileSystem.java | 19 ++++++++++++++++++- 1 file changed, 18 insertions(+), 1 deletion(-) diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalFileSystem.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalFileSystem.java index 9d071b7c2d0c1..7c3c7dba27ed6 100644 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalFileSystem.java +++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalFileSystem.java @@ -35,6 +35,7 @@ import static com.google.common.collect.ImmutableSet.toImmutableSet; import static io.trino.filesystem.local.LocalUtils.handleException; import static java.nio.file.LinkOption.NOFOLLOW_LINKS; +import static java.util.UUID.randomUUID; /** * A hierarchical file system for testing. @@ -226,7 +227,23 @@ public Set listDirectories(Location location) public Optional createTemporaryDirectory(Location targetPath, String temporaryPrefix, String relativePrefix) throws IOException { - throw new IOException("Local file system does not support creating temporary directories"); + // allow for absolute or relative temporary prefix + Location temporary; + if (temporaryPrefix.startsWith("/")) { + String prefix = temporaryPrefix; + while (prefix.startsWith("/")) { + prefix = prefix.substring(1); + } + temporary = Location.of("local:///").appendPath(prefix); + } + else { + temporary = targetPath.appendPath(temporaryPrefix); + } + + temporary = temporary.appendPath(randomUUID().toString()); + + createDirectory(temporary); + return Optional.of(temporary); } private Path toFilePath(Location location) From c9917f74867473b7a590d49fa404dea9c2efa2ac Mon Sep 17 00:00:00 2001 From: Dain Sundstrom Date: Sat, 2 Dec 2023 00:38:20 -0800 Subject: [PATCH 007/350] Remove unnecessary TestHudiPlugin --- .../io/trino/plugin/hudi/TestHudiPlugin.java | 129 ------------------ 1 file changed, 129 deletions(-) delete mode 100644 plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiPlugin.java diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiPlugin.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiPlugin.java deleted file mode 100644 index 71ec2e585916a..0000000000000 --- a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiPlugin.java +++ /dev/null @@ -1,129 +0,0 @@ -/* - * 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.ImmutableMap; -import io.airlift.bootstrap.ApplicationConfigurationException; -import io.trino.plugin.hive.HiveConfig; -import io.trino.spi.Plugin; -import io.trino.spi.connector.ConnectorFactory; -import io.trino.testing.TestingConnectorContext; -import org.junit.jupiter.api.Test; - -import java.util.Map; -import java.util.Optional; - -import static com.google.common.collect.Iterables.getOnlyElement; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -public class TestHudiPlugin -{ - @Test - public void testCreateConnector() - { - ConnectorFactory factory = getConnectorFactory(); - factory.create( - "test", - Map.of( - "hive.metastore.uri", "thrift://foo:1234", - "bootstrap.quiet", "true"), - new TestingConnectorContext()) - .shutdown(); - } - - @Test - public void testCreateTestingConnector() - { - Plugin plugin = new TestingHudiPlugin(Optional.empty()); - ConnectorFactory factory = getOnlyElement(plugin.getConnectorFactories()); - factory.create( - "test", - Map.of( - "hive.metastore.uri", "thrift://foo:1234", - "bootstrap.quiet", "true"), - new TestingConnectorContext()) - .shutdown(); - } - - @Test - public void testTestingFileMetastore() - { - ConnectorFactory factory = getConnectorFactory(); - factory.create( - "test", - ImmutableMap.of( - "hive.metastore", "file", - "hive.metastore.catalog.dir", "/tmp", - "bootstrap.quiet", "true"), - new TestingConnectorContext()) - .shutdown(); - } - - @Test - public void testThriftMetastore() - { - ConnectorFactory factory = getConnectorFactory(); - factory.create( - "test", - Map.of( - "hive.metastore", "thrift", - "hive.metastore.uri", "thrift://foo:1234", - "bootstrap.quiet", "true"), - new TestingConnectorContext()) - .shutdown(); - } - - @Test - public void testGlueMetastore() - { - ConnectorFactory factory = getConnectorFactory(); - factory.create( - "test", - Map.of( - "hive.metastore", "glue", - "hive.metastore.glue.region", "us-east-2", - "bootstrap.quiet", "true"), - new TestingConnectorContext()) - .shutdown(); - - assertThatThrownBy(() -> factory.create( - "test", - Map.of( - "hive.metastore", "glue", - "hive.metastore.uri", "thrift://foo:1234", - "bootstrap.quiet", "true"), - new TestingConnectorContext())) - .isInstanceOf(ApplicationConfigurationException.class) - .hasMessageContaining("Error: Configuration property 'hive.metastore.uri' was not used"); - } - - @Test - public void testHiveConfigIsNotBound() - { - ConnectorFactory factory = getConnectorFactory(); - assertThatThrownBy(() -> factory.create("test", - Map.of( - "hive.metastore.uri", "thrift://foo:1234", - // Try setting any property provided by HiveConfig class - HiveConfig.CONFIGURATION_HIVE_PARTITION_PROJECTION_ENABLED, "true", - "bootstrap.quiet", "true"), - new TestingConnectorContext())) - .hasMessageContaining("Error: Configuration property 'hive.partition-projection-enabled' was not used"); - } - - private static ConnectorFactory getConnectorFactory() - { - return getOnlyElement(new HudiPlugin().getConnectorFactories()); - } -} From b35ccb203e58d4b3ab92ace07f0a4fd02c38d4be Mon Sep 17 00:00:00 2001 From: Dain Sundstrom Date: Tue, 28 Nov 2023 23:11:26 -0800 Subject: [PATCH 008/350] Simplify metastore usage in Hudi tests --- .../io/trino/plugin/hudi/HudiConnector.java | 11 +++ .../plugin/hudi/HudiConnectorFactory.java | 2 +- .../hudi/InternalHudiConnectorFactory.java | 13 +-- .../io/trino/plugin/hudi/HudiQueryRunner.java | 40 +++----- .../trino/plugin/hudi/S3HudiQueryRunner.java | 72 +++----------- .../trino/plugin/hudi/TestHudiSmokeTest.java | 21 ++-- .../hudi/TestingHudiConnectorFactory.java | 28 ++++-- .../trino/plugin/hudi/TestingHudiPlugin.java | 11 +-- .../hudi/testing/HudiTablesInitializer.java | 12 +-- .../ResourceHudiTablesInitializer.java | 65 ++++++------ .../testing/TpchHudiTablesInitializer.java | 98 +++++++++++-------- testing/trino-faulttolerant-tests/pom.xml | 6 ++ 12 files changed, 187 insertions(+), 192 deletions(-) diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConnector.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConnector.java index a4da3a1eae4f1..18bb5a85308e6 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConnector.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConnector.java @@ -13,8 +13,10 @@ */ package io.trino.plugin.hudi; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; +import com.google.inject.Injector; import io.airlift.bootstrap.LifeCycleManager; import io.trino.plugin.base.classloader.ClassLoaderSafeConnectorMetadata; import io.trino.plugin.base.session.SessionPropertiesProvider; @@ -42,6 +44,7 @@ public class HudiConnector implements Connector { + private final Injector injector; private final LifeCycleManager lifeCycleManager; private final HudiTransactionManager transactionManager; private final ConnectorSplitManager splitManager; @@ -52,6 +55,7 @@ public class HudiConnector private final List> tableProperties; public HudiConnector( + Injector injector, LifeCycleManager lifeCycleManager, HudiTransactionManager transactionManager, ConnectorSplitManager splitManager, @@ -61,6 +65,7 @@ public HudiConnector( Set sessionPropertiesProviders, List> tableProperties) { + this.injector = requireNonNull(injector, "injector is null"); this.lifeCycleManager = requireNonNull(lifeCycleManager, "lifeCycleManager is null"); this.transactionManager = requireNonNull(transactionManager, "transactionManager is null"); this.splitManager = requireNonNull(splitManager, "splitManager is null"); @@ -144,4 +149,10 @@ public final void shutdown() { lifeCycleManager.stop(); } + + @VisibleForTesting + public Injector getInjector() + { + return injector; + } } diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConnectorFactory.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConnectorFactory.java index 21c7a6c03e3e7..e084e2ceadacf 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConnectorFactory.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiConnectorFactory.java @@ -36,6 +36,6 @@ public String getName() public Connector create(String catalogName, Map config, ConnectorContext context) { checkStrictSpiVersionMatch(context, this); - return createConnector(catalogName, config, context, Optional.empty(), Optional.empty()); + return createConnector(catalogName, config, context, Optional.empty()); } } diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/InternalHudiConnectorFactory.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/InternalHudiConnectorFactory.java index 9a09752ce57f0..01dd53080eabe 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/InternalHudiConnectorFactory.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/InternalHudiConnectorFactory.java @@ -24,7 +24,6 @@ import io.airlift.json.JsonModule; import io.opentelemetry.api.OpenTelemetry; import io.opentelemetry.api.trace.Tracer; -import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.filesystem.manager.FileSystemModule; import io.trino.plugin.base.CatalogName; import io.trino.plugin.base.classloader.ClassLoaderSafeConnectorPageSourceProvider; @@ -33,7 +32,6 @@ import io.trino.plugin.base.jmx.MBeanServerModule; import io.trino.plugin.base.session.SessionPropertiesProvider; import io.trino.plugin.hive.NodeVersion; -import io.trino.plugin.hive.metastore.HiveMetastore; import io.trino.plugin.hive.metastore.HiveMetastoreModule; import io.trino.spi.NodeManager; import io.trino.spi.classloader.ThreadContextClassLoader; @@ -57,8 +55,7 @@ public static Connector createConnector( String catalogName, Map config, ConnectorContext context, - Optional metastore, - Optional fileSystemFactory) + Optional module) { ClassLoader classLoader = InternalHudiConnectorFactory.class.getClassLoader(); try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { @@ -67,12 +64,11 @@ public static Connector createConnector( new MBeanModule(), new JsonModule(), new HudiModule(), - new HiveMetastoreModule(metastore), - fileSystemFactory - .map(factory -> (Module) binder -> binder.bind(TrinoFileSystemFactory.class).toInstance(factory)) - .orElseGet(() -> new FileSystemModule(catalogName, context.getNodeManager(), context.getOpenTelemetry())), + new HiveMetastoreModule(Optional.empty()), + new FileSystemModule(catalogName, context.getNodeManager(), context.getOpenTelemetry()), new MBeanServerModule(), binder -> { + module.ifPresent(binder::install); binder.bind(OpenTelemetry.class).toInstance(context.getOpenTelemetry()); binder.bind(Tracer.class).toInstance(context.getTracer()); binder.bind(NodeVersion.class).toInstance(new NodeVersion(context.getNodeManager().getCurrentNode().getVersion())); @@ -95,6 +91,7 @@ public static Connector createConnector( HudiTableProperties hudiTableProperties = injector.getInstance(HudiTableProperties.class); return new HudiConnector( + injector, lifeCycleManager, transactionManager, new ClassLoaderSafeConnectorSplitManager(splitManager, classLoader), diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/HudiQueryRunner.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/HudiQueryRunner.java index 61b3f6b65ed92..a732f4b203920 100644 --- a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/HudiQueryRunner.java +++ b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/HudiQueryRunner.java @@ -18,21 +18,17 @@ import io.airlift.log.Logger; import io.airlift.log.Logging; import io.trino.Session; -import io.trino.plugin.hive.SchemaAlreadyExistsException; +import io.trino.filesystem.Location; import io.trino.plugin.hive.metastore.Database; -import io.trino.plugin.hive.metastore.HiveMetastore; +import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.hudi.testing.HudiTablesInitializer; import io.trino.plugin.hudi.testing.ResourceHudiTablesInitializer; import io.trino.spi.security.PrincipalType; import io.trino.testing.DistributedQueryRunner; -import java.io.File; -import java.nio.file.Path; import java.util.Map; import java.util.Optional; -import static io.trino.plugin.hive.HiveTestUtils.HDFS_ENVIRONMENT; -import static io.trino.plugin.hive.metastore.file.TestingFileHiveMetastore.createTestingFileHiveMetastore; import static io.trino.testing.TestingSession.testSessionBuilder; public final class HudiQueryRunner @@ -57,28 +53,20 @@ public static DistributedQueryRunner createHudiQueryRunner( .setExtraProperties(extraProperties) .build(); - Path coordinatorBaseDir = queryRunner.getCoordinator().getBaseDataDir(); - File catalogDir = coordinatorBaseDir.resolve("catalog").toFile(); - HiveMetastore metastore = createTestingFileHiveMetastore(catalogDir); - - // create testing database - Database database = Database.builder() - .setDatabaseName(SCHEMA_NAME) - .setOwnerName(Optional.of("public")) - .setOwnerType(Optional.of(PrincipalType.ROLE)) - .build(); - try { - metastore.createDatabase(database); - } - catch (SchemaAlreadyExistsException e) { - // do nothing if database already exists - } - - queryRunner.installPlugin(new TestingHudiPlugin(Optional.of(metastore))); + queryRunner.installPlugin(new TestingHudiPlugin(queryRunner.getCoordinator().getBaseDataDir().resolve("hudi_data"))); queryRunner.createCatalog("hudi", "hudi", connectorProperties); - String dataDir = coordinatorBaseDir.resolve("data").toString(); - dataLoader.initializeTables(queryRunner, metastore, SCHEMA_NAME, dataDir, HDFS_ENVIRONMENT); + // Hudi connector does not support creating schema or any other write operations + ((HudiConnector) queryRunner.getCoordinator().getConnector("hudi")).getInjector() + .getInstance(HiveMetastoreFactory.class) + .createMetastore(Optional.empty()) + .createDatabase(Database.builder() + .setDatabaseName(SCHEMA_NAME) + .setOwnerName(Optional.of("public")) + .setOwnerType(Optional.of(PrincipalType.ROLE)) + .build()); + + dataLoader.initializeTables(queryRunner, Location.of("local:///"), SCHEMA_NAME); return queryRunner; } diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/S3HudiQueryRunner.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/S3HudiQueryRunner.java index 6605ef23a31fd..27d1233298245 100644 --- a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/S3HudiQueryRunner.java +++ b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/S3HudiQueryRunner.java @@ -14,23 +14,13 @@ package io.trino.plugin.hudi; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; import io.airlift.log.Logger; import io.airlift.log.Logging; import io.trino.Session; -import io.trino.filesystem.hdfs.HdfsFileSystemFactory; -import io.trino.hdfs.DynamicHdfsConfiguration; -import io.trino.hdfs.HdfsConfig; -import io.trino.hdfs.HdfsConfigurationInitializer; -import io.trino.hdfs.HdfsEnvironment; -import io.trino.hdfs.authentication.NoHdfsAuthentication; -import io.trino.hdfs.s3.HiveS3Config; -import io.trino.hdfs.s3.TrinoS3ConfigurationInitializer; -import io.trino.plugin.hive.SchemaAlreadyExistsException; +import io.trino.filesystem.Location; import io.trino.plugin.hive.containers.HiveMinioDataLake; import io.trino.plugin.hive.metastore.Database; -import io.trino.plugin.hive.metastore.HiveMetastore; -import io.trino.plugin.hive.metastore.thrift.BridgingHiveMetastore; +import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.hudi.testing.HudiTablesInitializer; import io.trino.plugin.hudi.testing.TpchHudiTablesInitializer; import io.trino.spi.security.PrincipalType; @@ -40,9 +30,6 @@ import java.util.Map; import java.util.Optional; -import static io.trino.plugin.hive.HiveTestUtils.HDFS_FILE_SYSTEM_STATS; -import static io.trino.plugin.hive.HiveTestUtils.SOCKS_PROXY; -import static io.trino.plugin.hive.TestingThriftHiveMetastoreBuilder.testingThriftHiveMetastoreBuilder; import static io.trino.testing.TestingSession.testSessionBuilder; import static io.trino.testing.containers.Minio.MINIO_ACCESS_KEY; import static io.trino.testing.containers.Minio.MINIO_REGION; @@ -62,30 +49,10 @@ public static DistributedQueryRunner create( HiveMinioDataLake hiveMinioDataLake) throws Exception { - String basePath = "s3a://" + hiveMinioDataLake.getBucketName() + "/" + TPCH_SCHEMA; - HdfsEnvironment hdfsEnvironment = getHdfsEnvironment(hiveMinioDataLake); - - HiveMetastore metastore = new BridgingHiveMetastore( - testingThriftHiveMetastoreBuilder() - .metastoreClient(hiveMinioDataLake.getHiveHadoop().getHiveMetastoreEndpoint()) - .fileSystemFactory(new HdfsFileSystemFactory(hdfsEnvironment, HDFS_FILE_SYSTEM_STATS)) - .build()); - Database database = Database.builder() - .setDatabaseName(TPCH_SCHEMA) - .setOwnerName(Optional.of("public")) - .setOwnerType(Optional.of(PrincipalType.ROLE)) - .build(); - try { - metastore.createDatabase(database); - } - catch (SchemaAlreadyExistsException e) { - // do nothing if database already exists - } - DistributedQueryRunner queryRunner = DistributedQueryRunner.builder(createSession()) .setExtraProperties(extraProperties) .build(); - queryRunner.installPlugin(new TestingHudiPlugin(Optional.of(metastore))); + queryRunner.installPlugin(new TestingHudiPlugin(queryRunner.getCoordinator().getBaseDataDir().resolve("hudi_data"))); queryRunner.createCatalog( "hudi", "hudi", @@ -100,7 +67,18 @@ public static DistributedQueryRunner create( .putAll(connectorProperties) .buildOrThrow()); - dataLoader.initializeTables(queryRunner, metastore, TPCH_SCHEMA, basePath, hdfsEnvironment); + // Hudi connector does not support creating schema or any other write operations + ((HudiConnector) queryRunner.getCoordinator().getConnector("hudi")).getInjector() + .getInstance(HiveMetastoreFactory.class) + .createMetastore(Optional.empty()) + .createDatabase(Database.builder() + .setDatabaseName(TPCH_SCHEMA) + .setOwnerName(Optional.of("public")) + .setOwnerType(Optional.of(PrincipalType.ROLE)) + .build()); + + dataLoader.initializeTables(queryRunner, Location.of("s3://" + hiveMinioDataLake.getBucketName() + "/"), TPCH_SCHEMA); + return queryRunner; } @@ -112,26 +90,6 @@ private static Session createSession() .build(); } - private static HdfsEnvironment getHdfsEnvironment(HiveMinioDataLake hiveMinioDataLake) - { - DynamicHdfsConfiguration dynamicHdfsConfiguration = new DynamicHdfsConfiguration( - new HdfsConfigurationInitializer( - new HdfsConfig() - .setSocksProxy(SOCKS_PROXY.orElse(null)), - ImmutableSet.of( - new TrinoS3ConfigurationInitializer(new HiveS3Config() - .setS3AwsAccessKey(MINIO_ACCESS_KEY) - .setS3AwsSecretKey(MINIO_SECRET_KEY) - .setS3Endpoint(hiveMinioDataLake.getMinio().getMinioAddress()) - .setS3PathStyleAccess(true)))), - ImmutableSet.of()); - - return new HdfsEnvironment( - dynamicHdfsConfiguration, - new HdfsConfig(), - new NoHdfsAuthentication()); - } - public static void main(String[] args) throws Exception { 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 71a687134880e..5dee9b2d7f8e0 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 @@ -14,13 +14,15 @@ package io.trino.plugin.hudi; import com.google.common.collect.ImmutableMap; +import io.trino.filesystem.Location; +import io.trino.filesystem.TrinoFileSystemFactory; +import io.trino.filesystem.TrinoInputFile; import io.trino.plugin.hudi.testing.ResourceHudiTablesInitializer; +import io.trino.spi.security.ConnectorIdentity; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.QueryRunner; import org.junit.jupiter.api.Test; -import java.nio.file.Files; -import java.nio.file.Path; import java.time.ZonedDateTime; import static io.trino.plugin.hudi.HudiQueryRunner.createHudiQueryRunner; @@ -132,9 +134,10 @@ public void testMetaColumns() @Test public void testPathColumn() + throws Exception { String path = (String) computeScalar("SELECT \"$path\" FROM " + HUDI_COW_PT_TBL + " WHERE id = 1"); - assertThat(toPath(path)).exists(); + assertThat(toInputFile(path).exists()).isTrue(); } @Test @@ -143,7 +146,7 @@ public void testFileSizeColumn() { String path = (String) computeScalar("SELECT \"$path\" FROM " + HUDI_COW_PT_TBL + " WHERE id = 1"); long fileSize = (long) computeScalar("SELECT \"$file_size\" FROM " + HUDI_COW_PT_TBL + " WHERE id = 1"); - assertThat(fileSize).isEqualTo(Files.size(toPath(path))); + assertThat(fileSize).isEqualTo(toInputFile(path).length()); } @Test @@ -153,7 +156,7 @@ public void testFileModifiedColumn() String path = (String) computeScalar("SELECT \"$path\" FROM " + HUDI_COW_PT_TBL + " WHERE id = 1"); ZonedDateTime fileModifiedTime = (ZonedDateTime) computeScalar("SELECT \"$file_modified_time\" FROM " + HUDI_COW_PT_TBL + " WHERE id = 1"); assertThat(fileModifiedTime.toInstant().toEpochMilli()) - .isEqualTo(Files.getLastModifiedTime(toPath(path)).toInstant().toEpochMilli()); + .isEqualTo(toInputFile(path).lastModified().toEpochMilli()); } @Test @@ -165,9 +168,11 @@ public void testPartitionColumn() assertQueryFails("SELECT \"$partition\" FROM " + HUDI_NON_PART_COW, ".* Column '\\$partition' cannot be resolved"); } - private static Path toPath(String path) + private TrinoInputFile toInputFile(String path) { - // Remove leading 'file:' because path column returns 'file:/path-to-file' in case of local file system - return Path.of(path.replaceFirst("^file:", "")); + return ((HudiConnector) getDistributedQueryRunner().getCoordinator().getConnector("hudi")).getInjector() + .getInstance(TrinoFileSystemFactory.class) + .create(ConnectorIdentity.ofUser("test")) + .newInputFile(Location.of(path)); } } diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestingHudiConnectorFactory.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestingHudiConnectorFactory.java index d221ada7e851a..73e2108084465 100644 --- a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestingHudiConnectorFactory.java +++ b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestingHudiConnectorFactory.java @@ -13,25 +13,31 @@ */ package io.trino.plugin.hudi; -import io.trino.plugin.hive.metastore.HiveMetastore; +import com.google.common.collect.ImmutableMap; +import io.trino.filesystem.TrinoFileSystemFactory; +import io.trino.filesystem.local.LocalFileSystemFactory; +import io.trino.plugin.hive.metastore.file.FileHiveMetastoreConfig; import io.trino.spi.connector.Connector; import io.trino.spi.connector.ConnectorContext; import io.trino.spi.connector.ConnectorFactory; +import java.nio.file.Path; import java.util.Map; import java.util.Optional; +import static com.google.inject.multibindings.MapBinder.newMapBinder; +import static io.airlift.configuration.ConfigBinder.configBinder; import static io.trino.plugin.hudi.InternalHudiConnectorFactory.createConnector; -import static java.util.Objects.requireNonNull; public class TestingHudiConnectorFactory implements ConnectorFactory { - private final Optional metastore; + private final Path localFileSystemRootPath; - public TestingHudiConnectorFactory(Optional metastore) + public TestingHudiConnectorFactory(Path localFileSystemRootPath) { - this.metastore = requireNonNull(metastore, "metastore is null"); + localFileSystemRootPath.toFile().mkdirs(); + this.localFileSystemRootPath = localFileSystemRootPath; } @Override @@ -43,6 +49,16 @@ public String getName() @Override public Connector create(String catalogName, Map config, ConnectorContext context) { - return createConnector(catalogName, config, context, metastore, Optional.empty()); + ImmutableMap.Builder configBuilder = ImmutableMap.builder() + .putAll(config) + .put("bootstrap.quiet", "true"); + if (!config.containsKey("hive.metastore")) { + configBuilder.put("hive.metastore", "file"); + } + return createConnector(catalogName, configBuilder.buildOrThrow(), context, Optional.of(binder -> { + newMapBinder(binder, String.class, TrinoFileSystemFactory.class) + .addBinding("local").toInstance(new LocalFileSystemFactory(localFileSystemRootPath)); + configBinder(binder).bindConfigDefaults(FileHiveMetastoreConfig.class, metastoreConfig -> metastoreConfig.setCatalogDirectory("local:///managed/")); + })); } } diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestingHudiPlugin.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestingHudiPlugin.java index 42788856d895f..02744377e2dad 100644 --- a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestingHudiPlugin.java +++ b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestingHudiPlugin.java @@ -14,11 +14,10 @@ package io.trino.plugin.hudi; import com.google.common.collect.ImmutableList; -import io.trino.plugin.hive.metastore.HiveMetastore; import io.trino.spi.connector.ConnectorFactory; +import java.nio.file.Path; import java.util.List; -import java.util.Optional; import static com.google.common.base.Verify.verify; import static java.util.Objects.requireNonNull; @@ -26,11 +25,11 @@ public class TestingHudiPlugin extends HudiPlugin { - private final Optional metastore; + private final Path localFileSystemRootPath; - public TestingHudiPlugin(Optional metastore) + public TestingHudiPlugin(Path localFileSystemRootPath) { - this.metastore = requireNonNull(metastore, "metastore is null"); + this.localFileSystemRootPath = requireNonNull(localFileSystemRootPath, "localFileSystemRootPath is null"); } @Override @@ -39,6 +38,6 @@ public Iterable getConnectorFactories() List connectorFactories = ImmutableList.copyOf(super.getConnectorFactories()); verify(connectorFactories.size() == 1, "Unexpected connector factories: %s", connectorFactories); - return ImmutableList.of(new TestingHudiConnectorFactory(metastore)); + return ImmutableList.of(new TestingHudiConnectorFactory(localFileSystemRootPath)); } } diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/HudiTablesInitializer.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/HudiTablesInitializer.java index 770ed52f0406b..b9a64da14a2d2 100644 --- a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/HudiTablesInitializer.java +++ b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/testing/HudiTablesInitializer.java @@ -13,17 +13,11 @@ */ package io.trino.plugin.hudi.testing; -import io.trino.hdfs.HdfsEnvironment; -import io.trino.plugin.hive.metastore.HiveMetastore; -import io.trino.testing.QueryRunner; +import io.trino.filesystem.Location; +import io.trino.testing.DistributedQueryRunner; public interface HudiTablesInitializer { - void initializeTables( - QueryRunner queryRunner, - HiveMetastore metastore, - String schemaName, - String dataDir, - HdfsEnvironment hdfsEnvironment) + void initializeTables(DistributedQueryRunner queryRunner, Location externalLocation, String schemaName) throws Exception; } 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 3c89326d69aea..b55cfe52c265a 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 @@ -15,21 +15,26 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import io.airlift.log.Logger; -import io.trino.hdfs.HdfsEnvironment; +import io.trino.filesystem.Location; +import io.trino.filesystem.TrinoFileSystem; +import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.plugin.hive.HiveType; import io.trino.plugin.hive.PartitionStatistics; import io.trino.plugin.hive.metastore.Column; import io.trino.plugin.hive.metastore.HiveMetastore; +import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.metastore.Partition; import io.trino.plugin.hive.metastore.PartitionWithStatistics; import io.trino.plugin.hive.metastore.PrincipalPrivileges; import io.trino.plugin.hive.metastore.StorageFormat; import io.trino.plugin.hive.metastore.Table; -import io.trino.testing.QueryRunner; +import io.trino.plugin.hudi.HudiConnector; +import io.trino.spi.security.ConnectorIdentity; +import io.trino.testing.DistributedQueryRunner; import java.io.File; import java.io.IOException; +import java.io.OutputStream; import java.nio.file.Files; import java.nio.file.Path; import java.util.ArrayList; @@ -56,27 +61,23 @@ public class ResourceHudiTablesInitializer implements HudiTablesInitializer { - public ResourceHudiTablesInitializer() {} - @Override - public void initializeTables( - QueryRunner queryRunner, - HiveMetastore metastore, - String schemaName, - String dataDir, - HdfsEnvironment environment) + public void initializeTables(DistributedQueryRunner queryRunner, Location externalLocation, String schemaName) throws Exception { - Path basePath = Path.of(dataDir); - copyDir(new File(getResource("hudi-testing-data").toURI()).toPath(), basePath); - Logger.get(getClass()).info("Prepared table data in %s", basePath); + TrinoFileSystem fileSystem = ((HudiConnector) queryRunner.getCoordinator().getConnector("hudi")).getInjector() + .getInstance(TrinoFileSystemFactory.class) + .create(ConnectorIdentity.ofUser("test")); + Location baseLocation = externalLocation.appendSuffix(schemaName); + copyDir(new File(getResource("hudi-testing-data").toURI()).toPath(), fileSystem, baseLocation); for (TestingTable table : TestingTable.values()) { String tableName = table.getTableName(); + Location tablePath = baseLocation.appendPath(tableName); createTable( - metastore, + queryRunner, schemaName, - basePath.resolve(tableName), + tablePath, tableName, table.getDataColumns(), table.getPartitionColumns(), @@ -85,9 +86,9 @@ public void initializeTables( } private void createTable( - HiveMetastore metastore, + DistributedQueryRunner queryRunner, String schemaName, - Path tablePath, + Location tablePath, String tableName, List dataColumns, List partitionColumns, @@ -108,8 +109,11 @@ private void createTable( .setParameters(ImmutableMap.of("serialization.format", "1", "EXTERNAL", "TRUE")) .withStorage(storageBuilder -> storageBuilder .setStorageFormat(storageFormat) - .setLocation("file://" + tablePath)) + .setLocation(tablePath.toString())) .build(); + HiveMetastore metastore = ((HudiConnector) queryRunner.getCoordinator().getConnector("hudi")).getInjector() + .getInstance(HiveMetastoreFactory.class) + .createMetastore(Optional.empty()); metastore.createTable(table, PrincipalPrivileges.NO_PRIVILEGES); List partitionsToAdd = new ArrayList<>(); @@ -120,7 +124,7 @@ private void createTable( .setValues(extractPartitionValues(partitionName)) .withStorage(storageBuilder -> storageBuilder .setStorageFormat(storageFormat) - .setLocation("file://" + tablePath.resolve(partitionPath))) + .setLocation(tablePath.appendPath(partitionPath).toString())) .setColumns(dataColumns) .build(); partitionsToAdd.add(new PartitionWithStatistics(partition, partitionName, PartitionStatistics.empty())); @@ -133,20 +137,25 @@ private static Column column(String name, HiveType type) return new Column(name, type, Optional.empty(), Map.of()); } - private static void copyDir(Path srcDir, Path dstDir) + public static void copyDir(Path sourceDirectory, TrinoFileSystem fileSystem, Location destinationDirectory) throws IOException { - try (Stream paths = Files.walk(srcDir)) { + try (Stream paths = Files.walk(sourceDirectory)) { for (Iterator iterator = paths.iterator(); iterator.hasNext(); ) { Path path = iterator.next(); - Path relativePath = srcDir.relativize(path); if (path.toFile().isDirectory()) { - Files.createDirectories(dstDir.resolve(relativePath)); + continue; } - else { - Path dstFile = dstDir.resolve(relativePath); - Files.createDirectories(dstFile.getParent()); - Files.copy(path, dstFile); + + // hudi blows up if crc files are present + if (path.toString().endsWith(".crc")) { + continue; + } + + Location location = destinationDirectory.appendPath(sourceDirectory.relativize(path).toString()); + fileSystem.createDirectory(location.parentDirectory()); + try (OutputStream out = fileSystem.newOutputFile(location).create()) { + Files.copy(path, out); } } } 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 7807fdf129456..64a44e0a8eefd 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 @@ -16,18 +16,25 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import io.airlift.log.Logger; +import io.trino.filesystem.Location; +import io.trino.filesystem.TrinoFileSystem; +import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.hdfs.HdfsContext; import io.trino.hdfs.HdfsEnvironment; import io.trino.plugin.hive.HiveType; import io.trino.plugin.hive.metastore.Column; import io.trino.plugin.hive.metastore.HiveMetastore; +import io.trino.plugin.hive.metastore.HiveMetastoreFactory; +import io.trino.plugin.hive.metastore.PrincipalPrivileges; import io.trino.plugin.hive.metastore.StorageFormat; import io.trino.plugin.hive.metastore.Table; +import io.trino.plugin.hudi.HudiConnector; import io.trino.plugin.tpch.TpchPlugin; import io.trino.spi.connector.CatalogSchemaName; +import io.trino.spi.security.ConnectorIdentity; +import io.trino.testing.DistributedQueryRunner; import io.trino.testing.MaterializedResult; import io.trino.testing.MaterializedRow; -import io.trino.testing.QueryRunner; import io.trino.tpch.TpchColumn; import io.trino.tpch.TpchColumnType; import io.trino.tpch.TpchColumnTypes; @@ -69,21 +76,24 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Verify.verify; +import static com.google.common.io.MoreFiles.deleteRecursively; +import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; +import static io.trino.plugin.hive.HiveTestUtils.HDFS_ENVIRONMENT; import static io.trino.plugin.hive.HiveType.HIVE_DATE; import static io.trino.plugin.hive.HiveType.HIVE_DOUBLE; import static io.trino.plugin.hive.HiveType.HIVE_INT; import static io.trino.plugin.hive.HiveType.HIVE_LONG; import static io.trino.plugin.hive.HiveType.HIVE_STRING; import static io.trino.plugin.hive.TableType.EXTERNAL_TABLE; -import static io.trino.plugin.hive.metastore.PrincipalPrivileges.NO_PRIVILEGES; import static io.trino.plugin.hive.util.HiveClassNames.HUDI_PARQUET_INPUT_FORMAT; import static io.trino.plugin.hive.util.HiveClassNames.MAPRED_PARQUET_OUTPUT_FORMAT_CLASS; import static io.trino.plugin.hive.util.HiveClassNames.PARQUET_HIVE_SERDE_CLASS; import static io.trino.testing.TestingConnectorSession.SESSION; import static java.lang.String.format; +import static java.nio.file.Files.createTempDirectory; import static java.util.Collections.unmodifiableList; import static java.util.Objects.requireNonNull; -import static java.util.stream.Collectors.toUnmodifiableList; +import static org.apache.hudi.common.model.HoodieTableType.COPY_ON_WRITE; public class TpchHudiTablesInitializer implements HudiTablesInitializer @@ -110,29 +120,41 @@ public TpchHudiTablesInitializer(HoodieTableType tableType, List> t } @Override - public void initializeTables( - QueryRunner queryRunner, - HiveMetastore metastore, - String schemaName, - String dataDir, - HdfsEnvironment hdfsEnvironment) + public void initializeTables(DistributedQueryRunner queryRunner, Location externalLocation, String schemaName) + throws Exception { queryRunner.installPlugin(new TpchPlugin()); queryRunner.createCatalog(TPCH_TINY.getCatalogName(), "tpch", ImmutableMap.of()); - for (TpchTable table : tpchTables) { - load(table, queryRunner, metastore, schemaName, dataDir, hdfsEnvironment); + TrinoFileSystem fileSystem = ((HudiConnector) queryRunner.getCoordinator().getConnector("hudi")).getInjector() + .getInstance(TrinoFileSystemFactory.class) + .create(ConnectorIdentity.ofUser("test")); + HiveMetastore metastore = ((HudiConnector) queryRunner.getCoordinator().getConnector("hudi")).getInjector() + .getInstance(HiveMetastoreFactory.class) + .createMetastore(Optional.empty()); + + Location dataLocation = externalLocation.appendPath("tpch"); + + java.nio.file.Path tempDir = createTempDirectory("test"); + try { + for (TpchTable tpchTable : tpchTables) { + java.nio.file.Path tempTableDir = tempDir.resolve(tpchTable.getTableName()); + load(tpchTable, queryRunner, tempTableDir); + + Location tableLocation = dataLocation.appendPath(tpchTable.getTableName()); + ResourceHudiTablesInitializer.copyDir(tempTableDir, fileSystem, tableLocation); + + Table table = createTableDefinition(schemaName, tpchTable, tableLocation); + metastore.createTable(table, PrincipalPrivileges.NO_PRIVILEGES); + } + } + finally { + deleteRecursively(tempDir, ALLOW_INSECURE); } } - private void load( - TpchTable tpchTables, - QueryRunner queryRunner, - HiveMetastore metastore, - String schemaName, - String basePath, - HdfsEnvironment hdfsEnvironment) + public void load(TpchTable tpchTables, DistributedQueryRunner queryRunner, java.nio.file.Path tableDirectory) { - try (HoodieJavaWriteClient writeClient = createWriteClient(tpchTables, basePath, hdfsEnvironment)) { + try (HoodieJavaWriteClient writeClient = createWriteClient(tpchTables, HDFS_ENVIRONMENT, new Path(tableDirectory.toUri()))) { RecordConverter recordConverter = createRecordConverter(tpchTables); @Language("SQL") String sql = generateScanSql(TPCH_TINY, tpchTables); @@ -148,11 +170,9 @@ private void load( writeClient.startCommitWithTime(timestamp); writeClient.insert(records, timestamp); } - - metastore.createTable(createMetastoreTable(schemaName, tpchTables, basePath), NO_PRIVILEGES); } - private String generateScanSql(CatalogSchemaName catalogSchemaName, TpchTable table) + private static String generateScanSql(CatalogSchemaName catalogSchemaName, TpchTable table) { StringBuilder builder = new StringBuilder(); builder.append("SELECT "); @@ -166,12 +186,11 @@ private String generateScanSql(CatalogSchemaName catalogSchemaName, TpchTable return builder.toString(); } - private Table createMetastoreTable(String schemaName, TpchTable table, String basePath) + private static Table createTableDefinition(String schemaName, TpchTable table, Location location) { - String tablePath = getTablePath(table, basePath); List columns = Stream.of(HUDI_META_COLUMNS, createMetastoreColumns(table)) .flatMap(Collection::stream) - .collect(toUnmodifiableList()); + .toList(); StorageFormat storageFormat = StorageFormat.create( PARQUET_HIVE_SERDE_CLASS, HUDI_PARQUET_INPUT_FORMAT, @@ -186,38 +205,36 @@ private Table createMetastoreTable(String schemaName, TpchTable table, String .setParameters(ImmutableMap.of("serialization.format", "1", "EXTERNAL", "TRUE")) .withStorage(storageBuilder -> storageBuilder .setStorageFormat(storageFormat) - .setLocation(tablePath)) + .setLocation(location.toString())) .build(); } - private HoodieJavaWriteClient createWriteClient(TpchTable table, String basePath, HdfsEnvironment hdfsEnvironment) + private static HoodieJavaWriteClient createWriteClient(TpchTable table, HdfsEnvironment hdfsEnvironment, Path tablePath) { - String tableName = table.getTableName(); - String tablePath = getTablePath(table, basePath); Schema schema = createAvroSchema(table); - Configuration conf = hdfsEnvironment.getConfiguration(CONTEXT, new Path(tablePath)); + Configuration conf = hdfsEnvironment.getConfiguration(CONTEXT, tablePath); try { HoodieTableMetaClient.withPropertyBuilder() - .setTableType(tableType) - .setTableName(tableName) + .setTableType(COPY_ON_WRITE) + .setTableName(table.getTableName()) .setBootstrapIndexClass(NoOpBootstrapIndex.class.getName()) .setPayloadClassName(HoodieAvroPayload.class.getName()) .setRecordKeyFields(FIELD_UUID) - .initTable(conf, tablePath); + .initTable(conf, tablePath.toString()); } catch (IOException e) { - throw new RuntimeException("Could not init table " + tableName, e); + throw new RuntimeException("Could not init table " + table.getTableName(), e); } HoodieIndexConfig indexConfig = HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build(); HoodieArchivalConfig archivalConfig = HoodieArchivalConfig.newBuilder().archiveCommitsWith(20, 30).build(); HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder() - .withPath(tablePath) + .withPath(tablePath.toString()) .withSchema(schema.toString()) .withParallelism(2, 2) .withDeleteParallelism(2) - .forTable(tableName) + .forTable(table.getTableName()) .withIndexConfig(indexConfig) .withArchivalConfig(archivalConfig) .withEmbeddedTimelineServerEnabled(false) @@ -226,11 +243,6 @@ private HoodieJavaWriteClient createWriteClient(TpchTable return new HoodieJavaWriteClient<>(new HoodieJavaEngineContext(conf), cfg); } - private String getTablePath(TpchTable table, String basePath) - { - return basePath + "/" + table.getTableName(); - } - private static RecordConverter createRecordConverter(TpchTable table) { Schema schema = createAvroSchema(table); @@ -239,11 +251,11 @@ private static RecordConverter createRecordConverter(TpchTable table) int numberOfColumns = columns.size(); List columnNames = columns.stream() .map(TpchColumn::getSimplifiedColumnName) - .collect(toUnmodifiableList()); + .toList(); List> columnConverters = columns.stream() .map(TpchColumn::getType) .map(TpchHudiTablesInitializer::avroEncoderOf) - .collect(toUnmodifiableList()); + .toList(); return row -> { checkArgument(row.size() == numberOfColumns); diff --git a/testing/trino-faulttolerant-tests/pom.xml b/testing/trino-faulttolerant-tests/pom.xml index bce534c1e5e3e..266fb41468e8e 100644 --- a/testing/trino-faulttolerant-tests/pom.xml +++ b/testing/trino-faulttolerant-tests/pom.xml @@ -185,6 +185,12 @@ test + + io.trino + trino-filesystem + test + + io.trino trino-hdfs From 2a104052987c70170fbc9c2075819beb96e6ae15 Mon Sep 17 00:00:00 2001 From: Dain Sundstrom Date: Sat, 11 Nov 2023 13:10:07 -0800 Subject: [PATCH 009/350] Remove complex unnecessary tests These complex test simply check that a runtime exception is ignored after table creation is complete. --- .../TestSemiTransactionalHiveMetastore.java | 194 ------------------ .../cache/TestCachingHiveMetastore.java | 171 --------------- ...cebergFileMetastoreCreateTableFailure.java | 140 ------------- ...MetastoreTableOperationsInsertFailure.java | 125 ----------- .../TestIcebergGlueCreateTableFailure.java | 192 ----------------- 5 files changed, 822 deletions(-) delete mode 100644 plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/TestSemiTransactionalHiveMetastore.java delete mode 100644 plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestIcebergFileMetastoreCreateTableFailure.java delete mode 100644 plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestIcebergFileMetastoreTableOperationsInsertFailure.java delete mode 100644 plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCreateTableFailure.java diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/TestSemiTransactionalHiveMetastore.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/TestSemiTransactionalHiveMetastore.java deleted file mode 100644 index b8eccab2fad74..0000000000000 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/TestSemiTransactionalHiveMetastore.java +++ /dev/null @@ -1,194 +0,0 @@ -/* - * 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.hive.metastore; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import io.trino.filesystem.Location; -import io.trino.plugin.hive.HiveBucketProperty; -import io.trino.plugin.hive.HiveMetastoreClosure; -import io.trino.plugin.hive.HiveType; -import io.trino.plugin.hive.PartitionStatistics; -import io.trino.plugin.hive.acid.AcidTransaction; -import io.trino.plugin.hive.fs.FileSystemDirectoryLister; -import org.junit.jupiter.api.Test; - -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.OptionalLong; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.Executor; -import java.util.concurrent.TimeUnit; -import java.util.function.Function; -import java.util.stream.IntStream; - -import static com.google.common.util.concurrent.MoreExecutors.directExecutor; -import static io.trino.plugin.hive.HiveBasicStatistics.createEmptyStatistics; -import static io.trino.plugin.hive.HiveTestUtils.HDFS_FILE_SYSTEM_FACTORY; -import static io.trino.plugin.hive.acid.AcidOperation.INSERT; -import static io.trino.plugin.hive.util.HiveBucketing.BucketingVersion.BUCKETING_V1; -import static io.trino.testing.TestingConnectorSession.SESSION; -import static io.trino.type.InternalTypeManager.TESTING_TYPE_MANAGER; -import static java.util.concurrent.Executors.newFixedThreadPool; -import static java.util.concurrent.Executors.newScheduledThreadPool; -import static org.assertj.core.api.Assertions.assertThat; - -// countDownLatch field is shared between tests -public class TestSemiTransactionalHiveMetastore -{ - private static final Column TABLE_COLUMN = new Column( - "column", - HiveType.HIVE_INT, - Optional.of("comment"), - Map.of()); - private static final Storage TABLE_STORAGE = new Storage( - StorageFormat.create("serde", "input", "output"), - Optional.of("/test"), - Optional.of(new HiveBucketProperty(ImmutableList.of("column"), BUCKETING_V1, 10, ImmutableList.of(new SortingColumn("column", SortingColumn.Order.ASCENDING)))), - true, - ImmutableMap.of("param", "value2")); - - private CountDownLatch countDownLatch; - - @Test - public void testParallelPartitionDrops() - { - int partitionsToDrop = 5; - IntStream dropThreadsConfig = IntStream.of(1, 2); - dropThreadsConfig.forEach(dropThreads -> { - countDownLatch = new CountDownLatch(dropThreads); - SemiTransactionalHiveMetastore semiTransactionalHiveMetastore = getSemiTransactionalHiveMetastoreWithDropExecutor(newFixedThreadPool(dropThreads)); - IntStream.range(0, partitionsToDrop).forEach(i -> semiTransactionalHiveMetastore.dropPartition(SESSION, - "test", - "test", - ImmutableList.of(String.valueOf(i)), - true)); - semiTransactionalHiveMetastore.commit(); - }); - } - - private SemiTransactionalHiveMetastore getSemiTransactionalHiveMetastoreWithDropExecutor(Executor dropExecutor) - { - return new SemiTransactionalHiveMetastore( - HDFS_FILE_SYSTEM_FACTORY, - new HiveMetastoreClosure(new TestingHiveMetastore(), TESTING_TYPE_MANAGER, false), - directExecutor(), - dropExecutor, - directExecutor(), - false, - false, - true, - Optional.empty(), - newScheduledThreadPool(1), - new FileSystemDirectoryLister()); - } - - @Test - public void testParallelUpdateStatisticsOperations() - { - int tablesToUpdate = 5; - IntStream updateThreadsConfig = IntStream.of(1, 2); - updateThreadsConfig.forEach(updateThreads -> { - countDownLatch = new CountDownLatch(updateThreads); - SemiTransactionalHiveMetastore semiTransactionalHiveMetastore; - if (updateThreads == 1) { - semiTransactionalHiveMetastore = getSemiTransactionalHiveMetastoreWithUpdateExecutor(directExecutor()); - } - else { - semiTransactionalHiveMetastore = getSemiTransactionalHiveMetastoreWithUpdateExecutor(newFixedThreadPool(updateThreads)); - } - IntStream.range(0, tablesToUpdate).forEach(i -> semiTransactionalHiveMetastore.finishChangingExistingTable(INSERT, SESSION, - "database", - "table_" + i, - Location.of(TABLE_STORAGE.getLocation()), - ImmutableList.of(), - PartitionStatistics.empty(), - false)); - semiTransactionalHiveMetastore.commit(); - }); - } - - private SemiTransactionalHiveMetastore getSemiTransactionalHiveMetastoreWithUpdateExecutor(Executor updateExecutor) - { - return new SemiTransactionalHiveMetastore( - HDFS_FILE_SYSTEM_FACTORY, - new HiveMetastoreClosure(new TestingHiveMetastore(), TESTING_TYPE_MANAGER, false), - directExecutor(), - directExecutor(), - updateExecutor, - false, - false, - true, - Optional.empty(), - newScheduledThreadPool(1), - new FileSystemDirectoryLister()); - } - - private class TestingHiveMetastore - extends UnimplementedHiveMetastore - { - @Override - public Optional
getTable(String databaseName, String tableName) - { - if (databaseName.equals("database")) { - return Optional.of(new Table( - "database", - tableName, - Optional.of("owner"), - "table_type", - TABLE_STORAGE, - ImmutableList.of(TABLE_COLUMN), - ImmutableList.of(TABLE_COLUMN), - ImmutableMap.of("param", "value3"), - Optional.of("original_text"), - Optional.of("expanded_text"), - OptionalLong.empty())); - } - return Optional.empty(); - } - - @Override - public PartitionStatistics getTableStatistics(Table table) - { - return new PartitionStatistics(createEmptyStatistics(), ImmutableMap.of()); - } - - @Override - public void dropPartition(String databaseName, String tableName, List parts, boolean deleteData) - { - assertCountDownLatch(); - } - - @Override - public void updateTableStatistics(String databaseName, - String tableName, - AcidTransaction transaction, - Function update) - { - assertCountDownLatch(); - } - - private void assertCountDownLatch() - { - try { - countDownLatch.countDown(); - assertThat(countDownLatch.await(10, TimeUnit.SECONDS)).isTrue(); //all other threads launched should count down within 10 seconds - } - catch (InterruptedException ie) { - Thread.currentThread().interrupt(); - } - } - } -} diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastore.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastore.java index dc176ec0a00ac..ddffedb8a9aa9 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastore.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastore.java @@ -36,7 +36,6 @@ import io.trino.plugin.hive.metastore.Partition; import io.trino.plugin.hive.metastore.PrincipalPrivileges; import io.trino.plugin.hive.metastore.Table; -import io.trino.plugin.hive.metastore.UnimplementedHiveMetastore; import io.trino.plugin.hive.metastore.thrift.BridgingHiveMetastore; import io.trino.plugin.hive.metastore.thrift.MockThriftMetastoreClient; import io.trino.plugin.hive.metastore.thrift.ThriftHiveMetastore; @@ -57,15 +56,10 @@ import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.parallel.Execution; -import java.util.ArrayList; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.OptionalLong; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CountDownLatch; import java.util.concurrent.CyclicBarrier; import java.util.concurrent.Executor; @@ -81,18 +75,14 @@ import static io.airlift.slice.Slices.utf8Slice; import static io.trino.plugin.hive.HiveColumnHandle.ColumnType.PARTITION_KEY; import static io.trino.plugin.hive.HiveColumnHandle.createBaseColumn; -import static io.trino.plugin.hive.HiveStorageFormat.TEXTFILE; import static io.trino.plugin.hive.HiveType.HIVE_LONG; import static io.trino.plugin.hive.HiveType.HIVE_STRING; -import static io.trino.plugin.hive.HiveType.toHiveType; -import static io.trino.plugin.hive.TableType.EXTERNAL_TABLE; import static io.trino.plugin.hive.TableType.VIRTUAL_VIEW; import static io.trino.plugin.hive.TestingThriftHiveMetastoreBuilder.testingThriftHiveMetastoreBuilder; import static io.trino.plugin.hive.metastore.HiveColumnStatistics.createIntegerColumnStatistics; import static io.trino.plugin.hive.metastore.MetastoreUtil.computePartitionKeyFilter; import static io.trino.plugin.hive.metastore.MetastoreUtil.makePartitionName; import static io.trino.plugin.hive.metastore.StorageFormat.VIEW_STORAGE_FORMAT; -import static io.trino.plugin.hive.metastore.StorageFormat.fromHiveStorageFormat; import static io.trino.plugin.hive.metastore.cache.CachingHiveMetastore.createPerTransactionCache; import static io.trino.plugin.hive.metastore.thrift.MockThriftMetastoreClient.BAD_DATABASE; import static io.trino.plugin.hive.metastore.thrift.MockThriftMetastoreClient.BAD_PARTITION; @@ -932,167 +922,6 @@ public void testCachingHiveMetastoreCreationViaMemoize() assertThat(metastore.getDatabaseNamesStats().getRequestCount()).isEqualTo(0); } - @Test - @Timeout(60) - public void testLoadAfterInvalidate() - throws Exception - { - testLoadAfterInvalidate(true); - testLoadAfterInvalidate(false); - } - - private void testLoadAfterInvalidate(boolean invalidateAll) - throws Exception - { - // State - CopyOnWriteArrayList tableColumns = new CopyOnWriteArrayList<>(); - ConcurrentMap tablePartitionsByName = new ConcurrentHashMap<>(); - Map tableParameters = new ConcurrentHashMap<>(); - tableParameters.put("frequent-changing-table-parameter", "parameter initial value"); - - // Initialize data - String databaseName = "my_database"; - String tableName = "my_table_name"; - - tableColumns.add(new Column("value", toHiveType(VARCHAR), Optional.empty() /* comment */, Map.of())); - tableColumns.add(new Column("pk", toHiveType(VARCHAR), Optional.empty() /* comment */, Map.of())); - - List partitionNames = new ArrayList<>(); - for (int i = 0; i < 10; i++) { - String partitionName = "pk=" + i; - tablePartitionsByName.put( - partitionName, - Partition.builder() - .setDatabaseName(databaseName) - .setTableName(tableName) - .setColumns(ImmutableList.copyOf(tableColumns)) - .setValues(List.of(Integer.toString(i))) - .withStorage(storage -> storage.setStorageFormat(fromHiveStorageFormat(TEXTFILE))) - .setParameters(Map.of("frequent-changing-partition-parameter", "parameter initial value")) - .build()); - partitionNames.add(partitionName); - } - - // Mock metastore - CountDownLatch getTableEnteredLatch = new CountDownLatch(1); - CountDownLatch getTableReturnLatch = new CountDownLatch(1); - CountDownLatch getTableFinishedLatch = new CountDownLatch(1); - CountDownLatch getPartitionsByNamesEnteredLatch = new CountDownLatch(1); - CountDownLatch getPartitionsByNamesReturnLatch = new CountDownLatch(1); - CountDownLatch getPartitionsByNamesFinishedLatch = new CountDownLatch(1); - - HiveMetastore mockMetastore = new UnimplementedHiveMetastore() - { - @Override - public Optional
getTable(String databaseName, String tableName) - { - Optional
table = Optional.of(Table.builder() - .setDatabaseName(databaseName) - .setTableName(tableName) - .setTableType(EXTERNAL_TABLE.name()) - .setDataColumns(tableColumns) - .setParameters(ImmutableMap.copyOf(tableParameters)) - // Required by 'Table', but not used by view translation. - .withStorage(storage -> storage.setStorageFormat(fromHiveStorageFormat(TEXTFILE))) - .setOwner(Optional.empty()) - .build()); - - getTableEnteredLatch.countDown(); // 1 - await(getTableReturnLatch, 10, SECONDS); // 2 - - return table; - } - - @Override - public Map> getPartitionsByNames(Table table, List partitionNames) - { - Map> result = new HashMap<>(); - for (String partitionName : partitionNames) { - result.put(partitionName, Optional.ofNullable(tablePartitionsByName.get(partitionName))); - } - - getPartitionsByNamesEnteredLatch.countDown(); // loader#1 - await(getPartitionsByNamesReturnLatch, 10, SECONDS); // loader#2 - - return result; - } - }; - - // Caching metastore - metastore = createCachingHiveMetastore(mockMetastore, CACHE_TTL, true, true, executor); - - // The test. Main thread does modifications and verifies subsequent load sees them. Background thread loads the state into the cache. - ExecutorService executor = Executors.newFixedThreadPool(1); - try { - Future future = executor.submit(() -> { - try { - Table table; - - table = metastore.getTable(databaseName, tableName).orElseThrow(); - getTableFinishedLatch.countDown(); // 3 - - metastore.getPartitionsByNames(table, partitionNames); - getPartitionsByNamesFinishedLatch.countDown(); // 6 - - return null; - } - catch (Throwable e) { - log.error(e); - throw e; - } - }); - - await(getTableEnteredLatch, 10, SECONDS); // 21 - tableParameters.put("frequent-changing-table-parameter", "main-thread-put-xyz"); - if (invalidateAll) { - metastore.flushCache(); - } - else { - metastore.invalidateTable(databaseName, tableName); - } - getTableReturnLatch.countDown(); // 2 - await(getTableFinishedLatch, 10, SECONDS); // 3 - Table table = metastore.getTable(databaseName, tableName).orElseThrow(); - assertThat(table.getParameters()) - .isEqualTo(Map.of("frequent-changing-table-parameter", "main-thread-put-xyz")); - - await(getPartitionsByNamesEnteredLatch, 10, SECONDS); // 4 - String partitionName = partitionNames.get(2); - Map newPartitionParameters = Map.of("frequent-changing-partition-parameter", "main-thread-put-alice"); - tablePartitionsByName.put(partitionName, - Partition.builder(tablePartitionsByName.get(partitionName)) - .setParameters(newPartitionParameters) - .build()); - if (invalidateAll) { - metastore.flushCache(); - } - else { - metastore.invalidateTable(databaseName, tableName); - } - getPartitionsByNamesReturnLatch.countDown(); // 5 - await(getPartitionsByNamesFinishedLatch, 10, SECONDS); // 6 - Map> loadedPartitions = metastore.getPartitionsByNames(table, partitionNames); - assertThat(loadedPartitions.get(partitionName)) - .isNotNull() - .isPresent() - .hasValueSatisfying(partition -> assertThat(partition.getParameters()).isEqualTo(newPartitionParameters)); - - // verify no failure in the background thread - future.get(10, SECONDS); - } - finally { - getTableEnteredLatch.countDown(); - getTableReturnLatch.countDown(); - getTableFinishedLatch.countDown(); - getPartitionsByNamesEnteredLatch.countDown(); - getPartitionsByNamesReturnLatch.countDown(); - getPartitionsByNamesFinishedLatch.countDown(); - - executor.shutdownNow(); - assertThat(executor.awaitTermination(10, SECONDS)).isTrue(); - } - } - @Test public void testDropTable() { diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestIcebergFileMetastoreCreateTableFailure.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestIcebergFileMetastoreCreateTableFailure.java deleted file mode 100644 index 373b44231e546..0000000000000 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestIcebergFileMetastoreCreateTableFailure.java +++ /dev/null @@ -1,140 +0,0 @@ -/* - * 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.iceberg.catalog.file; - -import io.trino.Session; -import io.trino.plugin.hive.NodeVersion; -import io.trino.plugin.hive.metastore.HiveMetastore; -import io.trino.plugin.hive.metastore.HiveMetastoreConfig; -import io.trino.plugin.hive.metastore.PrincipalPrivileges; -import io.trino.plugin.hive.metastore.Table; -import io.trino.plugin.hive.metastore.file.FileHiveMetastore; -import io.trino.plugin.hive.metastore.file.FileHiveMetastoreConfig; -import io.trino.plugin.iceberg.TestingIcebergPlugin; -import io.trino.spi.connector.SchemaNotFoundException; -import io.trino.testing.AbstractTestQueryFramework; -import io.trino.testing.DistributedQueryRunner; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; -import org.junit.jupiter.api.parallel.Execution; - -import java.nio.file.Files; -import java.nio.file.Path; -import java.util.Optional; -import java.util.concurrent.atomic.AtomicReference; - -import static com.google.common.io.MoreFiles.deleteRecursively; -import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; -import static com.google.inject.util.Modules.EMPTY_MODULE; -import static io.trino.plugin.hive.HiveTestUtils.HDFS_FILE_SYSTEM_FACTORY; -import static io.trino.testing.TestingNames.randomNameSuffix; -import static io.trino.testing.TestingSession.testSessionBuilder; -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; - -@TestInstance(PER_CLASS) -@Execution(SAME_THREAD) // testException is shared mutable state -public class TestIcebergFileMetastoreCreateTableFailure - extends AbstractTestQueryFramework -{ - private static final String ICEBERG_CATALOG = "iceberg"; - private static final String SCHEMA_NAME = "test_schema"; - - private static final String METADATA_GLOB = "glob:**.metadata.json"; - - private Path dataDirectory; - private HiveMetastore metastore; - private final AtomicReference testException = new AtomicReference<>(); - - @Override - protected DistributedQueryRunner createQueryRunner() - throws Exception - { - this.dataDirectory = Files.createTempDirectory("test_iceberg_create_table_failure"); - // Using FileHiveMetastore as approximation of HMS - this.metastore = new FileHiveMetastore( - new NodeVersion("testversion"), - HDFS_FILE_SYSTEM_FACTORY, - new HiveMetastoreConfig().isHideDeltaLakeTables(), - new FileHiveMetastoreConfig() - .setCatalogDirectory(dataDirectory.toString())) - { - @Override - public synchronized void createTable(Table table, PrincipalPrivileges principalPrivileges) - { - throw testException.get(); - } - }; - - Session session = testSessionBuilder() - .setCatalog(ICEBERG_CATALOG) - .setSchema(SCHEMA_NAME) - .build(); - - DistributedQueryRunner queryRunner = DistributedQueryRunner.builder(session).build(); - queryRunner.installPlugin(new TestingIcebergPlugin(Optional.of(new TestingIcebergFileMetastoreCatalogModule(metastore)), Optional.empty(), EMPTY_MODULE)); - queryRunner.createCatalog(ICEBERG_CATALOG, "iceberg"); - queryRunner.execute("CREATE SCHEMA " + SCHEMA_NAME); - - return queryRunner; - } - - @AfterAll - public void cleanup() - throws Exception - { - if (metastore != null) { - metastore.dropDatabase(SCHEMA_NAME, true); - } - if (dataDirectory != null) { - deleteRecursively(dataDirectory, ALLOW_INSECURE); - } - } - - @Test - public void testCreateTableFailureMetadataCleanedUp() - { - String exceptionMessage = "Test-simulated metastore schema not found exception"; - testException.set(new SchemaNotFoundException("simulated_test_schema", exceptionMessage)); - testCreateTableFailure(exceptionMessage, false); - } - - @Test - public void testCreateTableFailureMetadataNotCleanedUp() - { - String exceptionMessage = "Test-simulated metastore runtime exception"; - testException.set(new RuntimeException(exceptionMessage)); - testCreateTableFailure(exceptionMessage, true); - } - - protected void testCreateTableFailure(String expectedExceptionMessage, boolean shouldMetadataFileExist) - { - String tableName = "test_create_failure_" + randomNameSuffix(); - String tableLocation = Path.of(dataDirectory.toString(), tableName).toString(); - assertThatThrownBy(() -> getQueryRunner().execute("CREATE TABLE " + tableName + " (a varchar) WITH (location = '" + tableLocation + "')")) - .hasMessageContaining(expectedExceptionMessage); - - Path metadataDirectory = Path.of(tableLocation, "metadata"); - if (shouldMetadataFileExist) { - assertThat(metadataDirectory).as("Metadata file should exist").isDirectoryContaining(METADATA_GLOB); - } - else { - // file cleanup is more conservative since https://github.com/apache/iceberg/pull/8599 - assertThat(metadataDirectory).as("Metadata file should not exist").isDirectoryNotContaining(METADATA_GLOB); - } - } -} diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestIcebergFileMetastoreTableOperationsInsertFailure.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestIcebergFileMetastoreTableOperationsInsertFailure.java deleted file mode 100644 index 8bd537a466e9c..0000000000000 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestIcebergFileMetastoreTableOperationsInsertFailure.java +++ /dev/null @@ -1,125 +0,0 @@ -/* - * 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.iceberg.catalog.file; - -import com.google.common.collect.ImmutableMap; -import io.trino.Session; -import io.trino.metadata.InternalFunctionBundle; -import io.trino.plugin.hive.NodeVersion; -import io.trino.plugin.hive.metastore.Database; -import io.trino.plugin.hive.metastore.HiveMetastore; -import io.trino.plugin.hive.metastore.HiveMetastoreConfig; -import io.trino.plugin.hive.metastore.PrincipalPrivileges; -import io.trino.plugin.hive.metastore.Table; -import io.trino.plugin.hive.metastore.file.FileHiveMetastore; -import io.trino.plugin.hive.metastore.file.FileHiveMetastoreConfig; -import io.trino.plugin.iceberg.IcebergPlugin; -import io.trino.plugin.iceberg.TestingIcebergConnectorFactory; -import io.trino.spi.security.PrincipalType; -import io.trino.testing.AbstractTestQueryFramework; -import io.trino.testing.LocalQueryRunner; -import org.apache.iceberg.exceptions.CommitStateUnknownException; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; - -import java.io.File; -import java.nio.file.Files; -import java.util.Optional; - -import static com.google.common.io.MoreFiles.deleteRecursively; -import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; -import static com.google.inject.util.Modules.EMPTY_MODULE; -import static io.trino.plugin.hive.HiveTestUtils.HDFS_FILE_SYSTEM_FACTORY; -import static io.trino.testing.TestingSession.testSessionBuilder; -import static java.lang.String.format; -import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; - -@TestInstance(PER_CLASS) -public class TestIcebergFileMetastoreTableOperationsInsertFailure - extends AbstractTestQueryFramework -{ - private static final String ICEBERG_CATALOG = "iceberg"; - private static final String SCHEMA_NAME = "test_schema"; - private File baseDir; - - @Override - protected LocalQueryRunner createQueryRunner() - throws Exception - { - Session session = testSessionBuilder() - .setCatalog(ICEBERG_CATALOG) - .setSchema(SCHEMA_NAME) - .build(); - - baseDir = Files.createTempDirectory(null).toFile(); - - HiveMetastore metastore = new FileHiveMetastore( - new NodeVersion("testversion"), - HDFS_FILE_SYSTEM_FACTORY, - new HiveMetastoreConfig().isHideDeltaLakeTables(), - new FileHiveMetastoreConfig() - .setCatalogDirectory(baseDir.toURI().toString()) - .setMetastoreUser("test")) - { - @Override - public synchronized void replaceTable(String databaseName, String tableName, Table newTable, PrincipalPrivileges principalPrivileges) - { - super.replaceTable(databaseName, tableName, newTable, principalPrivileges); - throw new RuntimeException("Test-simulated metastore timeout exception"); - } - }; - LocalQueryRunner queryRunner = LocalQueryRunner.create(session); - - InternalFunctionBundle.InternalFunctionBundleBuilder functions = InternalFunctionBundle.builder(); - new IcebergPlugin().getFunctions().forEach(functions::functions); - queryRunner.addFunctions(functions.build()); - - queryRunner.createCatalog( - ICEBERG_CATALOG, - new TestingIcebergConnectorFactory(Optional.of(new TestingIcebergFileMetastoreCatalogModule(metastore)), Optional.empty(), EMPTY_MODULE), - ImmutableMap.of()); - - Database database = Database.builder() - .setDatabaseName(SCHEMA_NAME) - .setOwnerName(Optional.of("public")) - .setOwnerType(Optional.of(PrincipalType.ROLE)) - .build(); - metastore.createDatabase(database); - - return queryRunner; - } - - @AfterAll - public void cleanup() - throws Exception - { - if (baseDir != null) { - deleteRecursively(baseDir.toPath(), ALLOW_INSECURE); - } - } - - @Test - public void testInsertFailureDoesNotCorruptTheTableMetadata() - { - String tableName = "test_insert_failure"; - - getQueryRunner().execute(format("CREATE TABLE %s (a_varchar) AS VALUES ('Trino')", tableName)); - assertThatThrownBy(() -> getQueryRunner().execute("INSERT INTO " + tableName + " VALUES 'rocks'")) - .isInstanceOf(CommitStateUnknownException.class) - .hasMessageContaining("Test-simulated metastore timeout exception"); - assertQuery("SELECT * FROM " + tableName, "VALUES 'Trino', 'rocks'"); - } -} diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCreateTableFailure.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCreateTableFailure.java deleted file mode 100644 index 6ee8c7a30fd13..0000000000000 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCreateTableFailure.java +++ /dev/null @@ -1,192 +0,0 @@ -/* - * 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.iceberg.catalog.glue; - -import com.amazonaws.services.glue.AWSGlueAsync; -import com.amazonaws.services.glue.model.InvalidInputException; -import com.amazonaws.services.glue.model.OperationTimeoutException; -import com.google.common.collect.ImmutableMap; -import io.airlift.log.Logger; -import io.trino.Session; -import io.trino.filesystem.FileEntry; -import io.trino.filesystem.FileIterator; -import io.trino.filesystem.Location; -import io.trino.filesystem.TrinoFileSystem; -import io.trino.metadata.InternalFunctionBundle; -import io.trino.plugin.hive.metastore.Database; -import io.trino.plugin.hive.metastore.glue.GlueHiveMetastore; -import io.trino.plugin.iceberg.IcebergPlugin; -import io.trino.plugin.iceberg.TestingIcebergConnectorFactory; -import io.trino.spi.security.PrincipalType; -import io.trino.testing.AbstractTestQueryFramework; -import io.trino.testing.LocalQueryRunner; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; - -import java.lang.reflect.InvocationTargetException; -import java.nio.file.Files; -import java.nio.file.Path; -import java.util.Optional; -import java.util.concurrent.atomic.AtomicReference; - -import static com.google.common.io.MoreFiles.deleteRecursively; -import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; -import static com.google.common.reflect.Reflection.newProxy; -import static com.google.inject.util.Modules.EMPTY_MODULE; -import static io.trino.plugin.hive.HiveTestUtils.HDFS_FILE_SYSTEM_FACTORY; -import static io.trino.plugin.hive.metastore.glue.TestingGlueHiveMetastore.createTestingGlueHiveMetastore; -import static io.trino.testing.TestingConnectorSession.SESSION; -import static io.trino.testing.TestingNames.randomNameSuffix; -import static io.trino.testing.TestingSession.testSessionBuilder; -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; - -/* - * The test currently uses AWS Default Credential Provider Chain, - * See https://docs.aws.amazon.com/sdk-for-java/v1/developer-guide/credentials.html#credentials-default - * on ways to set your AWS credentials which will be needed to run this test. - */ -@TestInstance(PER_CLASS) -public class TestIcebergGlueCreateTableFailure - extends AbstractTestQueryFramework -{ - private static final Logger LOG = Logger.get(TestIcebergGlueCreateTableFailure.class); - - private static final String ICEBERG_CATALOG = "iceberg"; - - private final String schemaName = "test_iceberg_glue_" + randomNameSuffix(); - - private Path dataDirectory; - private TrinoFileSystem fileSystem; - private GlueHiveMetastore glueHiveMetastore; - private final AtomicReference testException = new AtomicReference<>(); - - @Override - protected LocalQueryRunner createQueryRunner() - throws Exception - { - Session session = testSessionBuilder() - .setCatalog(ICEBERG_CATALOG) - .setSchema(schemaName) - .build(); - LocalQueryRunner queryRunner = LocalQueryRunner.create(session); - - AWSGlueAsyncAdapterProvider awsGlueAsyncAdapterProvider = delegate -> newProxy(AWSGlueAsync.class, (proxy, method, methodArgs) -> { - Object result; - if (method.getName().equals("createTable")) { - throw testException.get(); - } - try { - result = method.invoke(delegate, methodArgs); - } - catch (InvocationTargetException e) { - throw e.getCause(); - } - return result; - }); - - InternalFunctionBundle.InternalFunctionBundleBuilder functions = InternalFunctionBundle.builder(); - new IcebergPlugin().getFunctions().forEach(functions::functions); - queryRunner.addFunctions(functions.build()); - - queryRunner.createCatalog( - ICEBERG_CATALOG, - new TestingIcebergConnectorFactory(Optional.of(new TestingIcebergGlueCatalogModule(awsGlueAsyncAdapterProvider)), Optional.empty(), EMPTY_MODULE), - ImmutableMap.of()); - - dataDirectory = Files.createTempDirectory("test_iceberg_create_table_failure"); - dataDirectory.toFile().deleteOnExit(); - - glueHiveMetastore = createTestingGlueHiveMetastore(dataDirectory); - fileSystem = HDFS_FILE_SYSTEM_FACTORY.create(SESSION); - - Database database = Database.builder() - .setDatabaseName(schemaName) - .setOwnerName(Optional.of("public")) - .setOwnerType(Optional.of(PrincipalType.ROLE)) - .setLocation(Optional.of(dataDirectory.toString())) - .build(); - glueHiveMetastore.createDatabase(database); - - return queryRunner; - } - - @AfterAll - public void cleanup() - { - try { - if (glueHiveMetastore != null) { - glueHiveMetastore.dropDatabase(schemaName, false); - } - if (dataDirectory != null) { - deleteRecursively(dataDirectory, ALLOW_INSECURE); - } - } - catch (Exception e) { - LOG.error(e, "Failed to clean up Glue database: %s", schemaName); - } - } - - @Test - public void testCreateTableFailureMetadataCleanedUp() - throws Exception - { - final String exceptionMessage = "Test-simulated metastore invalid input exception"; - testException.set(new InvalidInputException(exceptionMessage)); - testCreateTableFailure(exceptionMessage, false); - } - - @Test - public void testCreateTableFailureMetadataNotCleanedUp() - throws Exception - { - final String exceptionMessage = "Test-simulated metastore operation timeout exception"; - testException.set(new OperationTimeoutException(exceptionMessage)); - testCreateTableFailure(exceptionMessage, true); - } - - private void testCreateTableFailure(String expectedExceptionMessage, boolean shouldMetadataFileExist) - throws Exception - { - String tableName = "test_create_failure_" + randomNameSuffix(); - assertThatThrownBy(() -> getQueryRunner().execute("CREATE TABLE " + tableName + " (a_varchar) AS VALUES ('Trino')")) - .hasMessageContaining(expectedExceptionMessage); - - assertMetadataLocation(tableName, shouldMetadataFileExist); - } - - protected void assertMetadataLocation(String tableName, boolean shouldMetadataFileExist) - throws Exception - { - FileIterator fileIterator = fileSystem.listFiles(Location.of(dataDirectory.toString())); - String tableLocationPrefix = Path.of(dataDirectory.toString(), tableName).toString(); - boolean metadataFileFound = false; - while (fileIterator.hasNext()) { - FileEntry fileEntry = fileIterator.next(); - String location = fileEntry.location().toString(); - if (location.startsWith(tableLocationPrefix) && location.endsWith(".metadata.json")) { - metadataFileFound = true; - break; - } - } - if (shouldMetadataFileExist) { - assertThat(metadataFileFound).as("Metadata file should exist").isTrue(); - } - else { - assertThat(metadataFileFound).as("Metadata file should not exist").isFalse(); - } - } -} From ba0941cb5edda40515f88753a0115ef3518102ed Mon Sep 17 00:00:00 2001 From: Dain Sundstrom Date: Sat, 2 Dec 2023 00:43:35 -0800 Subject: [PATCH 010/350] Remove unnecessary TestDeltaLakePlugin --- .../plugin/deltalake/TestDeltaLakePlugin.java | 220 ------------------ 1 file changed, 220 deletions(-) delete mode 100644 plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakePlugin.java diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakePlugin.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakePlugin.java deleted file mode 100644 index cf64442880cc0..0000000000000 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakePlugin.java +++ /dev/null @@ -1,220 +0,0 @@ -/* - * 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.deltalake; - -import com.google.common.collect.ImmutableMap; -import io.airlift.bootstrap.ApplicationConfigurationException; -import io.trino.plugin.hive.HiveConfig; -import io.trino.spi.Plugin; -import io.trino.spi.connector.Connector; -import io.trino.spi.connector.ConnectorFactory; -import io.trino.testing.TestingConnectorContext; -import org.junit.jupiter.api.Test; - -import java.io.File; -import java.nio.file.Files; - -import static com.google.common.collect.Iterables.getOnlyElement; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -public class TestDeltaLakePlugin -{ - @Test - public void testCreateConnector() - { - ConnectorFactory factory = getConnectorFactory(); - factory.create( - "test", - ImmutableMap.of( - "hive.metastore.uri", "thrift://foo:1234", - "bootstrap.quiet", "true"), - new TestingConnectorContext()) - .shutdown(); - } - - @Test - public void testCreateTestingConnector() - { - Plugin plugin = new TestingDeltaLakePlugin(); - ConnectorFactory factory = getOnlyElement(plugin.getConnectorFactories()); - factory.create( - "test", - ImmutableMap.of( - "hive.metastore.uri", "thrift://foo:1234", - "bootstrap.quiet", "true"), - new TestingConnectorContext()) - .shutdown(); - } - - @Test - public void testTestingFileMetastore() - { - ConnectorFactory factory = getConnectorFactory(); - factory.create( - "test", - ImmutableMap.of( - "hive.metastore", "file", - "hive.metastore.catalog.dir", "/tmp", - "bootstrap.quiet", "true"), - new TestingConnectorContext()) - .shutdown(); - } - - @Test - public void testThriftMetastore() - { - ConnectorFactory factory = getConnectorFactory(); - factory.create( - "test", - ImmutableMap.of( - "hive.metastore", "thrift", - "hive.metastore.uri", "thrift://foo:1234", - "bootstrap.quiet", "true"), - new TestingConnectorContext()) - .shutdown(); - - assertThatThrownBy(() -> factory.create( - "test", - ImmutableMap.of( - "hive.metastore", "thrift", - "hive.metastore.uri", "thrift://foo:1234", - "delta.hide-non-delta-lake-tables", "true", - "bootstrap.quiet", "true"), - new TestingConnectorContext())) - .isInstanceOf(ApplicationConfigurationException.class) - // TODO support delta.hide-non-delta-lake-tables with thrift metastore - .hasMessageContaining("Error: Configuration property 'delta.hide-non-delta-lake-tables' was not used"); - } - - @Test - public void testGlueMetastore() - { - ConnectorFactory factory = getConnectorFactory(); - factory.create( - "test", - ImmutableMap.of( - "hive.metastore", "glue", - "hive.metastore.glue.region", "us-east-2", - "bootstrap.quiet", "true"), - new TestingConnectorContext()) - .shutdown(); - - assertThatThrownBy(() -> factory.create( - "test", - ImmutableMap.of( - "hive.metastore", "glue", - "hive.metastore.uri", "thrift://foo:1234", - "bootstrap.quiet", "true"), - new TestingConnectorContext())) - .isInstanceOf(ApplicationConfigurationException.class) - .hasMessageContaining("Error: Configuration property 'hive.metastore.uri' was not used"); - } - - @Test - public void testNoCaching() - { - ConnectorFactory factory = getConnectorFactory(); - factory.create("test", - ImmutableMap.of( - "hive.metastore.uri", "thrift://foo:1234", - "delta.metadata.cache-ttl", "0s", - "bootstrap.quiet", "true"), - new TestingConnectorContext()) - .shutdown(); - } - - @Test - public void testNoActiveDataFilesCaching() - { - ConnectorFactory factory = getConnectorFactory(); - factory.create("test", - ImmutableMap.of( - "hive.metastore.uri", "thrift://foo:1234", - "delta.metadata.live-files.cache-ttl", "0s", - "bootstrap.quiet", "true"), - new TestingConnectorContext()) - .shutdown(); - } - - @Test - public void testHiveConfigIsNotBound() - { - ConnectorFactory factory = getConnectorFactory(); - assertThatThrownBy(() -> factory.create("test", - ImmutableMap.of( - "hive.metastore.uri", "thrift://foo:1234", - // Try setting any property provided by HiveConfig class - HiveConfig.CONFIGURATION_HIVE_PARTITION_PROJECTION_ENABLED, "true", - "bootstrap.quiet", "true"), - new TestingConnectorContext())) - .hasMessageContaining("Error: Configuration property 'hive.partition-projection-enabled' was not used"); - } - - @Test - public void testReadOnlyAllAccessControl() - { - ConnectorFactory factory = getConnectorFactory(); - factory.create( - "test", - ImmutableMap.builder() - .put("hive.metastore.uri", "thrift://foo:1234") - .put("delta.security", "read-only") - .put("bootstrap.quiet", "true") - .buildOrThrow(), - new TestingConnectorContext()) - .shutdown(); - } - - @Test - public void testSystemAccessControl() - { - ConnectorFactory factory = getConnectorFactory(); - Connector connector = factory.create( - "test", - ImmutableMap.builder() - .put("hive.metastore.uri", "thrift://foo:1234") - .put("delta.security", "system") - .put("bootstrap.quiet", "true") - .buildOrThrow(), - new TestingConnectorContext()); - assertThatThrownBy(connector::getAccessControl).isInstanceOf(UnsupportedOperationException.class); - connector.shutdown(); - } - - @Test - public void testFileBasedAccessControl() - throws Exception - { - ConnectorFactory factory = getConnectorFactory(); - File tempFile = File.createTempFile("test-delta-lake-plugin-access-control", ".json"); - tempFile.deleteOnExit(); - Files.writeString(tempFile.toPath(), "{}"); - - factory.create( - "test", - ImmutableMap.builder() - .put("hive.metastore.uri", "thrift://foo:1234") - .put("delta.security", "file") - .put("security.config-file", tempFile.getAbsolutePath()) - .put("bootstrap.quiet", "true") - .buildOrThrow(), - new TestingConnectorContext()) - .shutdown(); - } - - private static ConnectorFactory getConnectorFactory() - { - return getOnlyElement(new DeltaLakePlugin().getConnectorFactories()); - } -} From ed67a55284372bb15ea21fae4257bf5dd78e055b Mon Sep 17 00:00:00 2001 From: Dain Sundstrom Date: Fri, 10 Nov 2023 19:44:55 -0800 Subject: [PATCH 011/350] Simplify metastore usage in Hive, Iceberg, and Delta tests --- .../io/trino/testing/LocalQueryRunner.java | 8 +++ .../BaseDeltaLakeConnectorSmokeTest.java | 3 +- ...seDeltaLakeRegisterTableProcedureTest.java | 40 ++++++----- ...BaseDeltaLakeSharedMetastoreViewsTest.java | 6 +- .../BaseDeltaLakeTableWithCustomLocation.java | 1 - .../deltalake/DeltaLakeQueryRunner.java | 2 +- .../deltalake/TestDeltaLakeConnectorTest.java | 12 ++-- .../TestDeltaLakeFileOperations.java | 3 +- ...DeltaLakePerTransactionMetastoreCache.java | 32 +++------ .../TestDeltaLakeProjectionPushdownPlans.java | 35 +++++----- ...redFileMetastoreWithTableRedirections.java | 4 +- ...redGlueMetastoreWithTableRedirections.java | 2 +- ...DeltaLakeSharedHiveMetastoreWithViews.java | 3 +- ...eWithCustomLocationUsingGlueMetastore.java | 5 +- ...eWithCustomLocationUsingHiveMetastore.java | 14 ++-- .../deltalake/TestingDeltaLakePlugin.java | 17 ++++- .../deltalake/TestingDeltaLakeUtils.java | 6 ++ ...estDeltaLakeMetastoreAccessOperations.java | 8 +-- ...keConcurrentModificationGlueMetastore.java | 2 +- .../glue/TestDeltaLakeViewsGlueMetastore.java | 2 +- plugin/trino-geospatial/pom.xml | 6 ++ .../plugin/geospatial/TestSpatialJoins.java | 22 +----- .../plugin/hive/BaseHiveConnectorTest.java | 49 ++++++++++---- .../io/trino/plugin/hive/HiveQueryRunner.java | 21 +++--- ...veConcurrentModificationGlueMetastore.java | 2 +- .../hive/TestHiveCreateExternalTable.java | 22 ++---- .../plugin/hive/TestParquetPageSkipping.java | 63 +++++++++++++----- .../io/trino/plugin/hive/TestRegexTable.java | 20 +++--- .../hive/TestingHiveConnectorFactory.java | 41 +++++++----- .../trino/plugin/hive/TestingHivePlugin.java | 17 +++-- .../trino/plugin/hive/TestingHiveUtils.java | 45 +++++++++++++ .../fs/BaseCachingDirectoryListerTest.java | 30 ++++----- ...stCachingHiveMetastoreWithQueryRunner.java | 22 +++--- .../file/TestingFileHiveMetastore.java | 8 ++- .../TestHiveMetastoreAccessOperations.java | 2 +- ...astoreMetadataQueriesAccessOperations.java | 2 +- .../TestConnectorPushdownRulesWithHive.java | 25 +++---- .../plugin/hive/optimizer/TestHivePlans.java | 24 +++---- ...stHiveProjectionPushdownIntoTableScan.java | 20 +++--- .../hive/orc/TestHiveOrcWithShortZoneId.java | 33 +++++++-- .../TestReadingTimeLogicalAnnotation.java | 28 ++++++-- .../hive/s3/TestHiveS3MinioQueries.java | 18 +---- .../s3/TestS3FileSystemAccessOperations.java | 23 +------ ...45-40c6-98ad-b56c8ee94ac6 => data.parquet} | Bin ...230725_103128_00063_6ramm => data.parquet} | Bin ...230725_074756_00016_6ramm => data.parquet} | Bin ...f7-4d05-afff-ba6b72bfb531 => data.parquet} | Bin .../{issue-16801.parquet => data.parquet} | Bin ...e-4b22-9846-54395fde4199-c000 => data.orc} | Bin .../iceberg/BaseIcebergConnectorTest.java | 9 ++- .../BaseIcebergMaterializedViewTest.java | 12 +++- .../plugin/iceberg/IcebergQueryRunner.java | 12 +--- .../plugin/iceberg/IcebergTestUtils.java | 7 -- .../TestIcebergConnectorSmokeTest.java | 41 +++++------- .../iceberg/TestIcebergFileOperations.java | 28 ++++---- ...stIcebergGetTableStatisticsOperations.java | 21 +++--- .../iceberg/TestIcebergMaterializedView.java | 19 +++--- .../iceberg/TestIcebergMergeAppend.java | 12 ++-- .../iceberg/TestIcebergMetadataListing.java | 18 ++--- .../TestIcebergMetastoreAccessOperations.java | 6 +- .../iceberg/TestIcebergMigrateProcedure.java | 6 +- .../TestIcebergOrcMetricsCollection.java | 12 ++-- .../TestIcebergParquetConnectorTest.java | 3 +- .../TestIcebergParquetWithBloomFilters.java | 4 +- .../TestIcebergProjectionPushdownPlans.java | 11 +-- .../TestIcebergRegisterTableProcedure.java | 33 +++++++-- .../iceberg/TestIcebergSplitSource.java | 8 ++- .../TestIcebergTableWithCustomLocation.java | 32 +++------ .../TestIcebergTableWithExternalLocation.java | 32 ++++----- .../trino/plugin/iceberg/TestIcebergV2.java | 39 +++-------- .../TestMetadataQueryOptimization.java | 11 +-- .../iceberg/TestSharedHiveMetastore.java | 15 ++--- .../TestingIcebergConnectorFactory.java | 33 ++++++++- .../plugin/iceberg/TestingIcebergPlugin.java | 14 +++- .../iceberg/catalog/BaseTrinoCatalogTest.java | 17 ++--- ...TestTrinoHiveCatalogWithFileMetastore.java | 11 +-- ...estIcebergGlueCatalogAccessOperations.java | 1 + ...ebergGlueTableOperationsInsertFailure.java | 8 +-- .../catalog/glue/TestSharedGlueMetastore.java | 2 +- ...toreTableOperationsReleaseLockFailure.java | 2 +- ...TestConnectorPushdownRulesWithIceberg.java | 29 ++++---- ...etFaultTolerantExecutionConnectorTest.java | 3 +- 82 files changed, 699 insertions(+), 560 deletions(-) create mode 100644 plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestingHiveUtils.java rename plugin/trino-hive/src/test/resources/parquet_page_skipping/column_name_with_dot/{20230725_101306_00056_6ramm_28cb680f-d745-40c6-98ad-b56c8ee94ac6 => data.parquet} (100%) rename plugin/trino-hive/src/test/resources/parquet_page_skipping/lineitem_sorted_by_suppkey/{000000_0_a94130b9-2234-4000-9162-4114aefcd919_20230725_103128_00063_6ramm => data.parquet} (100%) rename plugin/trino-hive/src/test/resources/parquet_page_skipping/orders_sorted_by_totalprice/{000000_0_ca5374d9-007e-4bbd-8717-bac6677b6ee7_20230725_074756_00016_6ramm => data.parquet} (100%) rename plugin/trino-hive/src/test/resources/parquet_page_skipping/random/{20230725_092119_00042_6ramm_25f11bb4-b7f7-4d05-afff-ba6b72bfb531 => data.parquet} (100%) rename plugin/trino-hive/src/test/resources/parquet_page_skipping/unsupported_column_index/{issue-16801.parquet => data.parquet} (100%) rename plugin/trino-hive/src/test/resources/with_short_zone_id/data/{part-00000-cab83205-643e-4b22-9846-54395fde4199-c000 => data.orc} (100%) diff --git a/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java b/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java index cf607bef71191..289a7fff840cc 100644 --- a/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java +++ b/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java @@ -139,6 +139,7 @@ import io.trino.spi.PageSorter; import io.trino.spi.Plugin; import io.trino.spi.connector.CatalogHandle; +import io.trino.spi.connector.Connector; import io.trino.spi.connector.ConnectorFactory; import io.trino.spi.exchange.ExchangeManager; import io.trino.spi.session.PropertyMetadata; @@ -786,6 +787,13 @@ public CatalogManager getCatalogManager() return catalogManager; } + public Connector getConnector(String catalogName) + { + return catalogManager + .getConnectorServices(getCatalogHandle(catalogName)) + .getConnector(); + } + public LocalQueryRunner printPlan() { printPlan = true; diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeConnectorSmokeTest.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeConnectorSmokeTest.java index 20b28c4392006..ea8c1f301b675 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeConnectorSmokeTest.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeConnectorSmokeTest.java @@ -188,12 +188,13 @@ protected QueryRunner createQueryRunner() registerTableFromResources(table.tableName(), table.resourcePath(), queryRunner); }); - queryRunner.installPlugin(new TestingHivePlugin()); + queryRunner.installPlugin(new TestingHivePlugin(queryRunner.getCoordinator().getBaseDataDir().resolve("hive_data"))); queryRunner.createCatalog( "hive", "hive", ImmutableMap.builder() + .put("hive.metastore", "thrift") .put("hive.metastore.uri", "thrift://" + hiveHadoop.getHiveMetastoreEndpoint()) .put("hive.allow-drop-table", "true") .putAll(hiveStorageConfiguration()) diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeRegisterTableProcedureTest.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeRegisterTableProcedureTest.java index 8de5cbe0da438..a7773413ee122 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeRegisterTableProcedureTest.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeRegisterTableProcedureTest.java @@ -15,8 +15,12 @@ import com.google.common.collect.ImmutableMap; import io.trino.Session; +import io.trino.filesystem.Location; +import io.trino.filesystem.TrinoFileSystem; +import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.plugin.deltalake.metastore.TestingDeltaLakeMetastoreModule; import io.trino.plugin.hive.metastore.HiveMetastore; +import io.trino.spi.security.ConnectorIdentity; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.DistributedQueryRunner; import io.trino.testing.QueryRunner; @@ -24,9 +28,7 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.TestInstance; -import java.io.File; import java.io.IOException; -import java.net.URI; import java.nio.file.Path; import java.util.Map; import java.util.Optional; @@ -34,11 +36,11 @@ import java.util.regex.Pattern; import static com.google.common.base.Verify.verify; -import static com.google.common.io.MoreFiles.deleteDirectoryContents; import static com.google.common.io.MoreFiles.deleteRecursively; import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; import static com.google.inject.util.Modules.EMPTY_MODULE; import static io.trino.plugin.deltalake.DeltaLakeConnectorFactory.CONNECTOR_NAME; +import static io.trino.plugin.deltalake.DeltaLakeQueryRunner.DELTA_CATALOG; import static io.trino.plugin.deltalake.transactionlog.TransactionLogUtil.getTransactionLogDir; import static io.trino.plugin.deltalake.transactionlog.TransactionLogUtil.getTransactionLogJsonEntryPath; import static io.trino.testing.TestingNames.randomNameSuffix; @@ -51,7 +53,6 @@ public abstract class BaseDeltaLakeRegisterTableProcedureTest extends AbstractTestQueryFramework { - protected static final String CATALOG_NAME = "delta_lake"; protected static final String SCHEMA = "test_delta_lake_register_table_" + randomNameSuffix(); private Path dataDirectory; @@ -62,7 +63,7 @@ protected QueryRunner createQueryRunner() throws Exception { Session session = testSessionBuilder() - .setCatalog(CATALOG_NAME) + .setCatalog(DELTA_CATALOG) .setSchema(SCHEMA) .build(); DistributedQueryRunner queryRunner = DistributedQueryRunner.builder(session).build(); @@ -70,14 +71,14 @@ protected QueryRunner createQueryRunner() this.dataDirectory = queryRunner.getCoordinator().getBaseDataDir().resolve("delta_lake_data"); this.metastore = createTestMetastore(dataDirectory); - queryRunner.installPlugin(new TestingDeltaLakePlugin(Optional.of(new TestingDeltaLakeMetastoreModule(metastore)), Optional.empty(), EMPTY_MODULE)); + queryRunner.installPlugin(new TestingDeltaLakePlugin(dataDirectory, Optional.of(new TestingDeltaLakeMetastoreModule(metastore)), Optional.empty(), EMPTY_MODULE)); Map connectorProperties = ImmutableMap.builder() .put("delta.unique-table-location", "true") .put("delta.register-table-procedure.enabled", "true") .buildOrThrow(); - queryRunner.createCatalog(CATALOG_NAME, CONNECTOR_NAME, connectorProperties); + queryRunner.createCatalog(DELTA_CATALOG, CONNECTOR_NAME, connectorProperties); queryRunner.execute("CREATE SCHEMA " + SCHEMA); return queryRunner; @@ -170,7 +171,7 @@ public void testRegisterTableWithDifferentTableName() metastore.dropTable(SCHEMA, tableName, false); String tableNameNew = "test_register_table_with_different_table_name_new_" + randomNameSuffix(); - assertQuerySucceeds(format("CALL %s.system.register_table('%s', '%s', '%s')", CATALOG_NAME, SCHEMA, tableNameNew, tableLocation)); + assertQuerySucceeds(format("CALL %s.system.register_table('%s', '%s', '%s')", DELTA_CATALOG, SCHEMA, tableNameNew, tableLocation)); String showCreateTableNew = (String) computeScalar("SHOW CREATE TABLE " + tableNameNew); assertThat(showCreateTableOld).isEqualTo(showCreateTableNew.replaceFirst(tableNameNew, tableName)); @@ -183,7 +184,7 @@ public void testRegisterTableWithDifferentTableName() public void testRegisterTableWithTrailingSpaceInLocation() { String tableName = "test_register_table_with_trailing_space_" + randomNameSuffix(); - String tableLocationWithTrailingSpace = dataDirectory.toUri() + "/" + tableName + " "; + String tableLocationWithTrailingSpace = "local:///" + tableName + " "; assertQuerySucceeds(format("CREATE TABLE %s WITH (location = '%s') AS SELECT 1 AS a, 'INDIA' AS b, true AS c", tableName, tableLocationWithTrailingSpace)); assertQuery("SELECT * FROM " + tableName, "VALUES (1, 'INDIA', true)"); @@ -228,14 +229,18 @@ public void testRegisterTableWithInvalidDeltaTable() String tableNameNew = "test_register_table_with_no_transaction_log_new_" + randomNameSuffix(); // Delete files under transaction log directory and put an invalid log file to verify register_table call fails - String transactionLogDir = URI.create(getTransactionLogDir(tableLocation)).getPath(); - deleteDirectoryContents(Path.of(transactionLogDir), ALLOW_INSECURE); - new File("/" + getTransactionLogJsonEntryPath(transactionLogDir, 0).path()).createNewFile(); + DistributedQueryRunner queryRunner = (DistributedQueryRunner) getQueryRunner(); + TrinoFileSystem fileSystem = TestingDeltaLakeUtils.getConnectorService(queryRunner, TrinoFileSystemFactory.class) + .create(ConnectorIdentity.ofUser("test")); + fileSystem.deleteDirectory(Location.of(tableLocation)); + fileSystem.newOutputFile(getTransactionLogJsonEntryPath(getTransactionLogDir(tableLocation), 0)) + .create() + .close(); assertQueryFails(format("CALL system.register_table('%s', '%s', '%s')", SCHEMA, tableNameNew, tableLocation), ".*Metadata not found in transaction log for (.*)"); - deleteRecursively(Path.of(URI.create(tableLocation).getPath()), ALLOW_INSECURE); + fileSystem.deleteDirectory(Location.of(tableLocation)); metastore.dropTable(SCHEMA, tableName, false); } @@ -252,12 +257,15 @@ public void testRegisterTableWithNoTransactionLog() String tableNameNew = "test_register_table_with_no_transaction_log_new_" + randomNameSuffix(); // Delete files under transaction log directory to verify register_table call fails - deleteDirectoryContents(Path.of(URI.create(getTransactionLogDir(tableLocation)).getPath()), ALLOW_INSECURE); + DistributedQueryRunner queryRunner = (DistributedQueryRunner) getQueryRunner(); + TrinoFileSystem fileSystem = TestingDeltaLakeUtils.getConnectorService(queryRunner, TrinoFileSystemFactory.class) + .create(ConnectorIdentity.ofUser("test")); + fileSystem.deleteDirectory(Location.of(tableLocation)); assertQueryFails(format("CALL system.register_table('%s', '%s', '%s')", SCHEMA, tableNameNew, tableLocation), ".*No transaction log found in location (.*)"); - deleteRecursively(Path.of(URI.create(tableLocation).getPath()), ALLOW_INSECURE); + fileSystem.deleteDirectory(Location.of(tableLocation)); metastore.dropTable(SCHEMA, tableName, false); } @@ -363,7 +371,7 @@ private String getTableComment(String tableName) { return (String) computeScalar(format( "SELECT comment FROM system.metadata.table_comments WHERE catalog_name = '%s' AND schema_name = '%s' AND table_name = '%s'", - CATALOG_NAME, + DELTA_CATALOG, SCHEMA, tableName)); } diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeSharedMetastoreViewsTest.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeSharedMetastoreViewsTest.java index 209c8c0629ac6..d2b416aefb037 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeSharedMetastoreViewsTest.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeSharedMetastoreViewsTest.java @@ -62,13 +62,13 @@ protected QueryRunner createQueryRunner() .build(); DistributedQueryRunner queryRunner = DistributedQueryRunner.builder(session).build(); - this.dataDirectory = queryRunner.getCoordinator().getBaseDataDir().resolve("delta_lake_data"); + this.dataDirectory = queryRunner.getCoordinator().getBaseDataDir().resolve("shared_data"); this.metastore = createTestMetastore(dataDirectory); - queryRunner.installPlugin(new TestingDeltaLakePlugin(Optional.of(new TestingDeltaLakeMetastoreModule(metastore)), Optional.empty(), EMPTY_MODULE)); + queryRunner.installPlugin(new TestingDeltaLakePlugin(dataDirectory, Optional.of(new TestingDeltaLakeMetastoreModule(metastore)), Optional.empty(), EMPTY_MODULE)); queryRunner.createCatalog(DELTA_CATALOG_NAME, "delta_lake"); - queryRunner.installPlugin(new TestingHivePlugin(metastore)); + queryRunner.installPlugin(new TestingHivePlugin(dataDirectory, metastore)); ImmutableMap hiveProperties = ImmutableMap.builder() .put("hive.allow-drop-table", "true") diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeTableWithCustomLocation.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeTableWithCustomLocation.java index 8c5f0130ad56e..f339634512297 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeTableWithCustomLocation.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeTableWithCustomLocation.java @@ -36,7 +36,6 @@ public abstract class BaseDeltaLakeTableWithCustomLocation extends AbstractTestQueryFramework { protected static final String SCHEMA = "test_tables_with_custom_location" + randomNameSuffix(); - protected static final String CATALOG_NAME = "delta_with_custom_location"; protected File metastoreDir; protected HiveMetastore metastore; diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/DeltaLakeQueryRunner.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/DeltaLakeQueryRunner.java index 552b30658d629..8afdc0eca0c11 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/DeltaLakeQueryRunner.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/DeltaLakeQueryRunner.java @@ -114,7 +114,7 @@ public DistributedQueryRunner build() queryRunner.installPlugin(new TpcdsPlugin()); queryRunner.createCatalog("tpcds", "tpcds"); - queryRunner.installPlugin(new TestingDeltaLakePlugin()); + queryRunner.installPlugin(new TestingDeltaLakePlugin(queryRunner.getCoordinator().getBaseDataDir().resolve("delta_lake_data"))); queryRunner.createCatalog(catalogName, CONNECTOR_NAME, deltaProperties.buildOrThrow()); return queryRunner; diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeConnectorTest.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeConnectorTest.java index fc51ec0c76394..3b0b4f0233afd 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeConnectorTest.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeConnectorTest.java @@ -22,6 +22,7 @@ import io.trino.execution.QueryInfo; import io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.ColumnMappingMode; import io.trino.plugin.hive.metastore.HiveMetastore; +import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.tpch.TpchPlugin; import io.trino.sql.planner.plan.FilterNode; import io.trino.sql.planner.plan.TableDeleteNode; @@ -41,7 +42,6 @@ import org.intellij.lang.annotations.Language; import org.junit.jupiter.api.Test; -import java.nio.file.Path; import java.time.ZonedDateTime; import java.util.List; import java.util.Map; @@ -60,7 +60,6 @@ import static io.trino.plugin.deltalake.DeltaLakeMetadata.CHANGE_DATA_FEED_COLUMN_NAMES; import static io.trino.plugin.deltalake.DeltaLakeQueryRunner.DELTA_CATALOG; import static io.trino.plugin.deltalake.transactionlog.TransactionLogUtil.TRANSACTION_LOG_DIRECTORY; -import static io.trino.plugin.hive.metastore.file.TestingFileHiveMetastore.createTestingFileHiveMetastore; import static io.trino.plugin.tpch.TpchMetadata.TINY_SCHEMA_NAME; import static io.trino.spi.type.TimeZoneKey.getTimeZoneKey; import static io.trino.spi.type.VarcharType.VARCHAR; @@ -89,7 +88,6 @@ public class TestDeltaLakeConnectorTest protected final String bucketName = "test-bucket-" + randomNameSuffix(); protected MinioClient minioClient; - protected HiveMetastore metastore; @Override protected QueryRunner createQueryRunner() @@ -105,8 +103,6 @@ protected QueryRunner createQueryRunner() .setSchema(SCHEMA) .build()) .build(); - Path metastoreDirectory = queryRunner.getCoordinator().getBaseDataDir().resolve("file-metastore"); - metastore = createTestingFileHiveMetastore(metastoreDirectory.toFile()); try { queryRunner.installPlugin(new TpchPlugin()); queryRunner.createCatalog("tpch", "tpch"); @@ -114,7 +110,7 @@ protected QueryRunner createQueryRunner() queryRunner.installPlugin(new DeltaLakePlugin()); queryRunner.createCatalog(DELTA_CATALOG, DeltaLakeConnectorFactory.CONNECTOR_NAME, ImmutableMap.builder() .put("hive.metastore", "file") - .put("hive.metastore.catalog.dir", metastoreDirectory.toString()) + .put("hive.metastore.catalog.dir", queryRunner.getCoordinator().getBaseDataDir().resolve("file-metastore").toString()) .put("hive.metastore.disable-location-checks", "true") .put("hive.s3.aws-access-key", MINIO_ACCESS_KEY) .put("hive.s3.aws-secret-key", MINIO_SECRET_KEY) @@ -3390,6 +3386,10 @@ public void testTrinoCacheInvalidatedOnCreateTable() assertUpdate("CREATE TABLE " + tableName + "(id, boolean, tinyint) WITH (location = '" + tableLocation + "') AS " + initialValues, 5); assertThat(query("SELECT * FROM " + tableName)).matches(initialValues); + DistributedQueryRunner queryRunner = (DistributedQueryRunner) getQueryRunner(); + HiveMetastore metastore = TestingDeltaLakeUtils.getConnectorService(queryRunner, HiveMetastoreFactory.class) + .createMetastore(Optional.empty()); + metastore.dropTable(SCHEMA, tableName, false); for (String file : minioClient.listObjects(bucketName, SCHEMA + "/" + tableName)) { minioClient.removeObject(bucketName, file); diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeFileOperations.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeFileOperations.java index 6470b9978c2f8..bf2467effc310 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeFileOperations.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeFileOperations.java @@ -93,7 +93,8 @@ protected DistributedQueryRunner createQueryRunner() queryRunner.installPlugin(new TpchPlugin()); queryRunner.createCatalog("tpch", "tpch"); - queryRunner.installPlugin(new TestingDeltaLakePlugin(Optional.empty(), Optional.of(trackingFileSystemFactory), EMPTY_MODULE)); + Path dataDirectory = queryRunner.getCoordinator().getBaseDataDir().resolve("delta_lake_data"); + queryRunner.installPlugin(new TestingDeltaLakePlugin(dataDirectory, Optional.empty(), Optional.of(trackingFileSystemFactory), EMPTY_MODULE)); queryRunner.createCatalog( "delta_lake", "delta_lake", diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakePerTransactionMetastoreCache.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakePerTransactionMetastoreCache.java index 9103e73a1f4c5..de8ba048e2a40 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakePerTransactionMetastoreCache.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakePerTransactionMetastoreCache.java @@ -16,7 +16,6 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMultiset; import com.google.common.collect.Multiset; -import com.google.common.reflect.ClassPath; import com.google.inject.Binder; import com.google.inject.Key; import io.airlift.configuration.AbstractConfigurationAwareModule; @@ -27,25 +26,19 @@ import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.metastore.RawHiveMetastoreFactory; import io.trino.plugin.hive.metastore.file.FileHiveMetastore; +import io.trino.plugin.tpch.TpchPlugin; import io.trino.testing.DistributedQueryRunner; import io.trino.testing.QueryRunner; -import io.trino.tpch.TpchEntity; -import io.trino.tpch.TpchTable; import org.intellij.lang.annotations.Language; import org.junit.jupiter.api.Test; import java.nio.file.Path; -import java.util.List; import java.util.Optional; import static io.trino.plugin.deltalake.DeltaLakeQueryRunner.DELTA_CATALOG; import static io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Method.GET_TABLE; import static io.trino.plugin.hive.metastore.file.TestingFileHiveMetastore.createTestingFileHiveMetastore; -import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.testing.TestingSession.testSessionBuilder; -import static java.lang.String.format; -import static java.nio.file.Files.createDirectories; -import static java.nio.file.Files.write; import static java.util.Objects.requireNonNull; public class TestDeltaLakePerTransactionMetastoreCache @@ -62,9 +55,10 @@ private DistributedQueryRunner createQueryRunner(boolean enablePerTransactionHiv DistributedQueryRunner queryRunner = DistributedQueryRunner.builder(session).build(); try { - FileHiveMetastore fileMetastore = createTestingFileHiveMetastore(queryRunner.getCoordinator().getBaseDataDir().resolve("file-metastore").toFile()); + Path dataDirectory = queryRunner.getCoordinator().getBaseDataDir().resolve("file-metastore"); + FileHiveMetastore fileMetastore = createTestingFileHiveMetastore(dataDirectory.toFile()); metastore = new CountingAccessHiveMetastore(fileMetastore); - queryRunner.installPlugin(new TestingDeltaLakePlugin(Optional.empty(), Optional.empty(), new CountingAccessMetastoreModule(metastore))); + queryRunner.installPlugin(new TestingDeltaLakePlugin(dataDirectory, Optional.empty(), Optional.empty(), new CountingAccessMetastoreModule(metastore))); ImmutableMap.Builder deltaLakeProperties = ImmutableMap.builder(); deltaLakeProperties.put("hive.metastore", "test"); // use test value so we do not get clash with default bindings) @@ -77,21 +71,11 @@ private DistributedQueryRunner createQueryRunner(boolean enablePerTransactionHiv queryRunner.createCatalog(DELTA_CATALOG, "delta_lake", deltaLakeProperties.buildOrThrow()); queryRunner.execute("CREATE SCHEMA " + session.getSchema().orElseThrow()); - for (TpchTable table : List.of(TpchTable.NATION, TpchTable.REGION)) { - String tableName = table.getTableName(); - String resourcePath = "io/trino/plugin/deltalake/testing/resources/databricks73/" + tableName + "/"; - Path tableDirectory = queryRunner.getCoordinator().getBaseDataDir().resolve("%s-%s".formatted(tableName, randomNameSuffix())); + queryRunner.installPlugin(new TpchPlugin()); + queryRunner.createCatalog("tpch", "tpch"); - for (ClassPath.ResourceInfo resourceInfo : ClassPath.from(getClass().getClassLoader()).getResources()) { - if (resourceInfo.getResourceName().startsWith(resourcePath)) { - Path targetFile = tableDirectory.resolve(resourceInfo.getResourceName().substring(resourcePath.length())); - createDirectories(targetFile.getParent()); - write(targetFile, resourceInfo.asByteSource().read()); - } - } - - queryRunner.execute(format("CALL system.register_table(CURRENT_SCHEMA, '%s', '%s')", tableName, tableDirectory)); - } + queryRunner.execute("CREATE TABLE nation AS SELECT * FROM tpch.tiny.nation"); + queryRunner.execute("CREATE TABLE region AS SELECT * FROM tpch.tiny.region"); } catch (Throwable e) { Closables.closeAllSuppress(e, queryRunner); diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeProjectionPushdownPlans.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeProjectionPushdownPlans.java index 07448e7d968f4..ac8c42a1b250c 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeProjectionPushdownPlans.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeProjectionPushdownPlans.java @@ -19,9 +19,9 @@ import io.trino.Session; import io.trino.metadata.QualifiedObjectName; import io.trino.metadata.TableHandle; -import io.trino.plugin.deltalake.metastore.TestingDeltaLakeMetastoreModule; import io.trino.plugin.hive.metastore.Database; import io.trino.plugin.hive.metastore.HiveMetastore; +import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.spi.connector.ColumnHandle; import io.trino.spi.predicate.Domain; import io.trino.spi.predicate.TupleDomain; @@ -32,10 +32,10 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.TestInstance; -import java.io.File; import java.io.IOException; import java.io.UncheckedIOException; import java.nio.file.Files; +import java.nio.file.Path; import java.util.List; import java.util.Map; import java.util.Optional; @@ -44,8 +44,7 @@ import static com.google.common.base.Predicates.equalTo; import static com.google.common.io.MoreFiles.deleteRecursively; import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; -import static com.google.inject.util.Modules.EMPTY_MODULE; -import static io.trino.plugin.hive.metastore.file.TestingFileHiveMetastore.createTestingFileHiveMetastore; +import static io.trino.plugin.deltalake.DeltaLakeQueryRunner.DELTA_CATALOG; import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.sql.planner.assertions.PlanMatchPattern.any; import static io.trino.sql.planner.assertions.PlanMatchPattern.anyTree; @@ -65,25 +64,33 @@ public class TestDeltaLakeProjectionPushdownPlans extends BasePushdownPlanTest { - private static final String CATALOG = "delta"; private static final String SCHEMA = "test_schema"; - private File baseDir; + private Path baseDir; @Override protected LocalQueryRunner createLocalQueryRunner() { Session session = testSessionBuilder() - .setCatalog(CATALOG) + .setCatalog(DELTA_CATALOG) .setSchema(SCHEMA) .build(); try { - baseDir = Files.createTempDirectory("delta_lake_projection_pushdown").toFile(); + baseDir = Files.createTempDirectory("delta_lake_projection_pushdown"); } catch (IOException e) { throw new UncheckedIOException(e); } - HiveMetastore metastore = createTestingFileHiveMetastore(baseDir); + + LocalQueryRunner queryRunner = LocalQueryRunner.create(session); + queryRunner.installPlugin(new TestingDeltaLakePlugin(baseDir)); + queryRunner.createCatalog(DELTA_CATALOG, "delta_lake", ImmutableMap.builder() + .put("hive.metastore", "file") + .put("hive.metastore.catalog.dir", baseDir.toString()) + .buildOrThrow()); + + HiveMetastore metastore = TestingDeltaLakeUtils.getConnectorService(queryRunner, HiveMetastoreFactory.class) + .createMetastore(Optional.empty()); Database database = Database.builder() .setDatabaseName(SCHEMA) .setOwnerName(Optional.of("public")) @@ -92,10 +99,6 @@ protected LocalQueryRunner createLocalQueryRunner() metastore.createDatabase(database); - LocalQueryRunner queryRunner = LocalQueryRunner.create(session); - queryRunner.installPlugin(new TestingDeltaLakePlugin(Optional.of(new TestingDeltaLakeMetastoreModule(metastore)), Optional.empty(), EMPTY_MODULE)); - queryRunner.createCatalog(CATALOG, "delta_lake", ImmutableMap.of()); - return queryRunner; } @@ -104,7 +107,7 @@ public void cleanup() throws Exception { if (baseDir != null) { - deleteRecursively(baseDir.toPath(), ALLOW_INSECURE); + deleteRecursively(baseDir, ALLOW_INSECURE); } } @@ -114,7 +117,7 @@ public void testPushdownDisabled() String testTable = "test_pushdown_disabled_" + randomNameSuffix(); Session session = Session.builder(getQueryRunner().getDefaultSession()) - .setCatalogSessionProperty(CATALOG, "projection_pushdown_enabled", "false") + .setCatalogSessionProperty(DELTA_CATALOG, "projection_pushdown_enabled", "false") .build(); getQueryRunner().execute(format( @@ -134,7 +137,7 @@ public void testPushdownDisabled() public void testDereferencePushdown() { String testTable = "test_simple_projection_pushdown" + randomNameSuffix(); - QualifiedObjectName completeTableName = new QualifiedObjectName(CATALOG, SCHEMA, testTable); + QualifiedObjectName completeTableName = new QualifiedObjectName(DELTA_CATALOG, SCHEMA, testTable); getQueryRunner().execute(format( "CREATE TABLE %s (col0, col1) WITH (partitioned_by = ARRAY['col1']) AS" + diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSharedFileMetastoreWithTableRedirections.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSharedFileMetastoreWithTableRedirections.java index 442456778740e..4db9ee5d73058 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSharedFileMetastoreWithTableRedirections.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSharedFileMetastoreWithTableRedirections.java @@ -47,7 +47,7 @@ protected QueryRunner createQueryRunner() DistributedQueryRunner queryRunner = DistributedQueryRunner.builder(deltaLakeSession).build(); dataDirectory = queryRunner.getCoordinator().getBaseDataDir().resolve("data"); - queryRunner.installPlugin(new TestingDeltaLakePlugin()); + queryRunner.installPlugin(new TestingDeltaLakePlugin(dataDirectory)); Map deltaLakeProperties = ImmutableMap.builder() .put("hive.metastore", "file") .put("hive.metastore.catalog.dir", dataDirectory.toString()) @@ -58,7 +58,7 @@ protected QueryRunner createQueryRunner() queryRunner.createCatalog("delta_with_redirections", CONNECTOR_NAME, deltaLakeProperties); queryRunner.execute("CREATE SCHEMA " + schema); - queryRunner.installPlugin(new TestingHivePlugin()); + queryRunner.installPlugin(new TestingHivePlugin(dataDirectory)); queryRunner.createCatalog( "hive_with_redirections", diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSharedGlueMetastoreWithTableRedirections.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSharedGlueMetastoreWithTableRedirections.java index 83f43ff8940a8..9e6527a08ace1 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSharedGlueMetastoreWithTableRedirections.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSharedGlueMetastoreWithTableRedirections.java @@ -70,7 +70,7 @@ protected QueryRunner createQueryRunner() .buildOrThrow()); this.glueMetastore = createTestingGlueHiveMetastore(dataDirectory); - queryRunner.installPlugin(new TestingHivePlugin(glueMetastore)); + queryRunner.installPlugin(new TestingHivePlugin(queryRunner.getCoordinator().getBaseDataDir().resolve("hive_data"), glueMetastore)); queryRunner.createCatalog( "hive_with_redirections", "hive", diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSharedHiveMetastoreWithViews.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSharedHiveMetastoreWithViews.java index c7b62bd865e30..e84783eb94a81 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSharedHiveMetastoreWithViews.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSharedHiveMetastoreWithViews.java @@ -56,7 +56,7 @@ protected QueryRunner createQueryRunner() hiveMinioDataLake.getHiveHadoop()); queryRunner.execute("CREATE SCHEMA " + schema + " WITH (location = 's3://" + bucketName + "/" + schema + "')"); - queryRunner.installPlugin(new TestingHivePlugin()); + queryRunner.installPlugin(new TestingHivePlugin(queryRunner.getCoordinator().getBaseDataDir().resolve("hive_data"))); Map s3Properties = ImmutableMap.builder() .put("hive.s3.aws-access-key", MINIO_ACCESS_KEY) .put("hive.s3.aws-secret-key", MINIO_SECRET_KEY) @@ -67,6 +67,7 @@ protected QueryRunner createQueryRunner() "hive", "hive", ImmutableMap.builder() + .put("hive.metastore", "thrift") .put("hive.metastore.uri", "thrift://" + hiveMinioDataLake.getHiveHadoop().getHiveMetastoreEndpoint()) .put("hive.allow-drop-table", "true") .putAll(s3Properties) diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeTableWithCustomLocationUsingGlueMetastore.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeTableWithCustomLocationUsingGlueMetastore.java index 2f4f64b448e76..7d392a008ad44 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeTableWithCustomLocationUsingGlueMetastore.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeTableWithCustomLocationUsingGlueMetastore.java @@ -26,6 +26,7 @@ import static com.google.common.io.MoreFiles.deleteRecursively; import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; import static io.trino.plugin.deltalake.DeltaLakeConnectorFactory.CONNECTOR_NAME; +import static io.trino.plugin.deltalake.DeltaLakeQueryRunner.DELTA_CATALOG; import static io.trino.plugin.hive.metastore.glue.TestingGlueHiveMetastore.createTestingGlueHiveMetastore; import static io.trino.testing.TestingSession.testSessionBuilder; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; @@ -41,7 +42,7 @@ protected QueryRunner createQueryRunner() throws Exception { Session deltaLakeSession = testSessionBuilder() - .setCatalog(CATALOG_NAME) + .setCatalog(DELTA_CATALOG) .setSchema(SCHEMA) .build(); @@ -52,7 +53,7 @@ protected QueryRunner createQueryRunner() queryRunner.installPlugin(new DeltaLakePlugin()); queryRunner.createCatalog( - CATALOG_NAME, + DELTA_CATALOG, CONNECTOR_NAME, ImmutableMap.builder() .put("hive.metastore", "glue") diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeTableWithCustomLocationUsingHiveMetastore.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeTableWithCustomLocationUsingHiveMetastore.java index eea54a4d91bcc..55d3549b25a9f 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeTableWithCustomLocationUsingHiveMetastore.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeTableWithCustomLocationUsingHiveMetastore.java @@ -14,15 +14,17 @@ package io.trino.plugin.deltalake; import io.trino.Session; +import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.testing.DistributedQueryRunner; import io.trino.testing.QueryRunner; import java.nio.file.Files; import java.util.HashMap; import java.util.Map; +import java.util.Optional; import static io.trino.plugin.deltalake.DeltaLakeConnectorFactory.CONNECTOR_NAME; -import static io.trino.plugin.hive.metastore.file.TestingFileHiveMetastore.createTestingFileHiveMetastore; +import static io.trino.plugin.deltalake.DeltaLakeQueryRunner.DELTA_CATALOG; import static io.trino.testing.TestingSession.testSessionBuilder; public class TestDeltaLakeTableWithCustomLocationUsingHiveMetastore @@ -33,7 +35,7 @@ protected QueryRunner createQueryRunner() throws Exception { Session session = testSessionBuilder() - .setCatalog(CATALOG_NAME) + .setCatalog(DELTA_CATALOG) .setSchema(SCHEMA) .build(); @@ -42,13 +44,15 @@ protected QueryRunner createQueryRunner() Map connectorProperties = new HashMap<>(); metastoreDir = Files.createTempDirectory("test_delta_lake").toFile(); - metastore = createTestingFileHiveMetastore(metastoreDir); connectorProperties.putIfAbsent("delta.unique-table-location", "true"); connectorProperties.putIfAbsent("hive.metastore", "file"); connectorProperties.putIfAbsent("hive.metastore.catalog.dir", metastoreDir.getPath()); - queryRunner.installPlugin(new TestingDeltaLakePlugin()); - queryRunner.createCatalog(CATALOG_NAME, CONNECTOR_NAME, connectorProperties); + queryRunner.installPlugin(new TestingDeltaLakePlugin(metastoreDir.toPath())); + queryRunner.createCatalog(DELTA_CATALOG, CONNECTOR_NAME, connectorProperties); + + metastore = TestingDeltaLakeUtils.getConnectorService(queryRunner, HiveMetastoreFactory.class) + .createMetastore(Optional.empty()); queryRunner.execute("CREATE SCHEMA " + SCHEMA); diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestingDeltaLakePlugin.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestingDeltaLakePlugin.java index 3fc363f27b55f..f75748e2e42c1 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestingDeltaLakePlugin.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestingDeltaLakePlugin.java @@ -17,31 +17,38 @@ import com.google.inject.Module; import io.airlift.configuration.AbstractConfigurationAwareModule; import io.trino.filesystem.TrinoFileSystemFactory; +import io.trino.filesystem.local.LocalFileSystemFactory; +import io.trino.plugin.hive.metastore.file.FileHiveMetastoreConfig; import io.trino.spi.connector.Connector; import io.trino.spi.connector.ConnectorContext; import io.trino.spi.connector.ConnectorFactory; +import java.nio.file.Path; import java.util.List; import java.util.Map; import java.util.Optional; +import static com.google.inject.multibindings.MapBinder.newMapBinder; import static com.google.inject.util.Modules.EMPTY_MODULE; +import static io.airlift.configuration.ConfigBinder.configBinder; import static java.util.Objects.requireNonNull; public class TestingDeltaLakePlugin extends DeltaLakePlugin { + private final Path localFileSystemRootPath; private final Optional metastoreModule; private final Optional fileSystemFactory; private final Module additionalModule; - public TestingDeltaLakePlugin() + public TestingDeltaLakePlugin(Path localFileSystemRootPath) { - this(Optional.empty(), Optional.empty(), EMPTY_MODULE); + this(localFileSystemRootPath, Optional.empty(), Optional.empty(), EMPTY_MODULE); } - public TestingDeltaLakePlugin(Optional metastoreModule, Optional fileSystemFactory, Module additionalModule) + public TestingDeltaLakePlugin(Path localFileSystemRootPath, Optional metastoreModule, Optional fileSystemFactory, Module additionalModule) { + this.localFileSystemRootPath = requireNonNull(localFileSystemRootPath, "localFileSystemRootPath is null"); this.metastoreModule = requireNonNull(metastoreModule, "metastoreModule is null"); this.fileSystemFactory = requireNonNull(fileSystemFactory, "fileSystemFactory is null"); this.additionalModule = requireNonNull(additionalModule, "additionalModule is null"); @@ -61,6 +68,7 @@ public String getName() @Override public Connector create(String catalogName, Map config, ConnectorContext context) { + localFileSystemRootPath.toFile().mkdirs(); return InternalDeltaLakeConnectorFactory.createConnector( catalogName, config, @@ -74,6 +82,9 @@ protected void setup(Binder binder) { install(additionalModule); install(new TestingDeltaLakeExtensionsModule()); + newMapBinder(binder, String.class, TrinoFileSystemFactory.class) + .addBinding("local").toInstance(new LocalFileSystemFactory(localFileSystemRootPath)); + configBinder(binder).bindConfigDefaults(FileHiveMetastoreConfig.class, config -> config.setCatalogDirectory("local:///")); } }); } diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestingDeltaLakeUtils.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestingDeltaLakeUtils.java index 251985eb1afe0..c9fe01e010d42 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestingDeltaLakeUtils.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestingDeltaLakeUtils.java @@ -20,6 +20,7 @@ import io.trino.plugin.deltalake.transactionlog.TransactionLogAccess; import io.trino.spi.connector.SchemaTableName; import io.trino.testing.DistributedQueryRunner; +import io.trino.testing.LocalQueryRunner; import java.io.IOException; import java.nio.file.Files; @@ -35,6 +36,11 @@ public final class TestingDeltaLakeUtils { private TestingDeltaLakeUtils() {} + public static T getConnectorService(LocalQueryRunner queryRunner, Class clazz) + { + return ((DeltaLakeConnector) queryRunner.getConnector(DELTA_CATALOG)).getInjector().getInstance(clazz); + } + public static T getConnectorService(DistributedQueryRunner queryRunner, Class clazz) { return ((DeltaLakeConnector) queryRunner.getCoordinator().getConnector(DELTA_CATALOG)).getInjector().getInstance(clazz); diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/TestDeltaLakeMetastoreAccessOperations.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/TestDeltaLakeMetastoreAccessOperations.java index dda0004e52361..b29b715bd254b 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/TestDeltaLakeMetastoreAccessOperations.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/TestDeltaLakeMetastoreAccessOperations.java @@ -32,7 +32,7 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.parallel.Execution; -import java.io.File; +import java.nio.file.Path; import java.util.Optional; import static io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Method.CREATE_TABLE; @@ -63,10 +63,10 @@ protected DistributedQueryRunner createQueryRunner() { DistributedQueryRunner queryRunner = DistributedQueryRunner.builder(TEST_SESSION).build(); - File baseDir = queryRunner.getCoordinator().getBaseDataDir().resolve("delta_lake").toFile(); - metastore = new CountingAccessHiveMetastore(createTestingFileHiveMetastore(baseDir)); + Path baseDir = queryRunner.getCoordinator().getBaseDataDir().resolve("delta_lake"); + metastore = new CountingAccessHiveMetastore(createTestingFileHiveMetastore(baseDir.toFile())); - queryRunner.installPlugin(new TestingDeltaLakePlugin(Optional.empty(), Optional.empty(), new CountingAccessMetastoreModule(metastore))); + queryRunner.installPlugin(new TestingDeltaLakePlugin(baseDir, Optional.empty(), Optional.empty(), new CountingAccessMetastoreModule(metastore))); ImmutableMap.Builder deltaLakeProperties = ImmutableMap.builder(); deltaLakeProperties.put("hive.metastore", "test"); // use test value so we do not get clash with default bindings) queryRunner.createCatalog("delta", "delta_lake", deltaLakeProperties.buildOrThrow()); diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeConcurrentModificationGlueMetastore.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeConcurrentModificationGlueMetastore.java index 9fb30ddd5957d..04092fe237db9 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeConcurrentModificationGlueMetastore.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeConcurrentModificationGlueMetastore.java @@ -101,7 +101,7 @@ protected QueryRunner createQueryRunner() stats, table -> true); - queryRunner.installPlugin(new TestingDeltaLakePlugin(Optional.of(new TestingDeltaLakeMetastoreModule(metastore)), Optional.empty(), EMPTY_MODULE)); + queryRunner.installPlugin(new TestingDeltaLakePlugin(dataDirectory, Optional.of(new TestingDeltaLakeMetastoreModule(metastore)), Optional.empty(), EMPTY_MODULE)); queryRunner.createCatalog(CATALOG_NAME, "delta_lake"); queryRunner.execute("CREATE SCHEMA " + SCHEMA); return queryRunner; diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeViewsGlueMetastore.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeViewsGlueMetastore.java index cce31746b617a..9fee324abe606 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeViewsGlueMetastore.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeViewsGlueMetastore.java @@ -67,7 +67,7 @@ protected QueryRunner createQueryRunner() dataDirectory = queryRunner.getCoordinator().getBaseDataDir().resolve("data_delta_lake_views"); metastore = createTestMetastore(dataDirectory); - queryRunner.installPlugin(new TestingDeltaLakePlugin(Optional.of(new TestingDeltaLakeMetastoreModule(metastore)), Optional.empty(), EMPTY_MODULE)); + queryRunner.installPlugin(new TestingDeltaLakePlugin(dataDirectory, Optional.of(new TestingDeltaLakeMetastoreModule(metastore)), Optional.empty(), EMPTY_MODULE)); queryRunner.createCatalog(CATALOG_NAME, "delta_lake"); queryRunner.execute("CREATE SCHEMA " + SCHEMA); diff --git a/plugin/trino-geospatial/pom.xml b/plugin/trino-geospatial/pom.xml index b1a115e8085a9..d11409deb6d2e 100644 --- a/plugin/trino-geospatial/pom.xml +++ b/plugin/trino-geospatial/pom.xml @@ -83,6 +83,12 @@ runtime + + com.google.inject + guice + runtime + + it.unimi.dsi fastutil diff --git a/plugin/trino-geospatial/src/test/java/io/trino/plugin/geospatial/TestSpatialJoins.java b/plugin/trino-geospatial/src/test/java/io/trino/plugin/geospatial/TestSpatialJoins.java index 8693030d20abc..10ef1b5104c68 100644 --- a/plugin/trino-geospatial/src/test/java/io/trino/plugin/geospatial/TestSpatialJoins.java +++ b/plugin/trino-geospatial/src/test/java/io/trino/plugin/geospatial/TestSpatialJoins.java @@ -15,19 +15,12 @@ import io.trino.Session; import io.trino.plugin.hive.TestingHivePlugin; -import io.trino.plugin.hive.metastore.Database; -import io.trino.plugin.hive.metastore.HiveMetastore; -import io.trino.spi.security.PrincipalType; import io.trino.sql.query.QueryAssertions; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.DistributedQueryRunner; import org.junit.jupiter.api.Test; -import java.io.File; -import java.util.Optional; - import static io.trino.SystemSessionProperties.SPATIAL_PARTITIONING_TABLE_NAME; -import static io.trino.plugin.hive.metastore.file.TestingFileHiveMetastore.createTestingFileHiveMetastore; import static io.trino.testing.TestingSession.testSessionBuilder; import static java.lang.String.format; import static org.assertj.core.api.Assertions.assertThat; @@ -74,19 +67,10 @@ protected DistributedQueryRunner createQueryRunner() DistributedQueryRunner queryRunner = DistributedQueryRunner.builder(session).build(); queryRunner.installPlugin(new GeoPlugin()); - File baseDir = queryRunner.getCoordinator().getBaseDataDir().resolve("hive_data").toFile(); - - HiveMetastore metastore = createTestingFileHiveMetastore(baseDir); - - metastore.createDatabase( - Database.builder() - .setDatabaseName("default") - .setOwnerName(Optional.of("public")) - .setOwnerType(Optional.of(PrincipalType.ROLE)) - .build()); - queryRunner.installPlugin(new TestingHivePlugin(metastore)); - + queryRunner.installPlugin(new TestingHivePlugin(queryRunner.getCoordinator().getBaseDataDir().resolve("hive_data"))); queryRunner.createCatalog("hive", "hive"); + queryRunner.execute("CREATE SCHEMA hive.default"); + return queryRunner; } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java index 39a6d4e505720..27d90ae44a4b0 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java @@ -16,6 +16,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import com.google.common.io.Resources; import io.airlift.json.JsonCodec; import io.airlift.json.JsonCodecFactory; import io.airlift.json.ObjectMapperProvider; @@ -24,6 +25,8 @@ import io.trino.cost.StatsAndCosts; import io.trino.execution.QueryInfo; import io.trino.filesystem.Location; +import io.trino.filesystem.TrinoFileSystem; +import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.metadata.FunctionManager; import io.trino.metadata.Metadata; import io.trino.metadata.QualifiedObjectName; @@ -39,6 +42,7 @@ import io.trino.spi.connector.CatalogSchemaTableName; import io.trino.spi.connector.ColumnMetadata; import io.trino.spi.connector.Constraint; +import io.trino.spi.security.ConnectorIdentity; import io.trino.spi.security.Identity; import io.trino.spi.security.SelectedRole; import io.trino.spi.type.DateType; @@ -71,7 +75,9 @@ import java.io.File; import java.io.IOException; +import java.io.OutputStream; import java.math.BigDecimal; +import java.net.URL; import java.time.Instant; import java.time.LocalDate; import java.time.LocalDateTime; @@ -88,6 +94,7 @@ import java.util.OptionalLong; import java.util.Set; import java.util.StringJoiner; +import java.util.UUID; import java.util.function.BiConsumer; import java.util.function.Consumer; import java.util.function.Function; @@ -171,7 +178,6 @@ import static io.trino.testing.TestingAccessControlManager.privilege; import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.testing.TestingSession.testSessionBuilder; -import static io.trino.testing.containers.TestContainers.getPathFromClassPathResource; import static io.trino.transaction.TransactionBuilder.transaction; import static io.trino.type.InternalTypeManager.TESTING_TYPE_MANAGER; import static java.lang.String.format; @@ -187,7 +193,6 @@ import static java.util.stream.Collectors.toSet; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.assertj.core.api.Fail.fail; import static org.assertj.core.data.Offset.offset; import static org.junit.jupiter.api.Assumptions.abort; @@ -4411,13 +4416,17 @@ private void testCreateExternalTable( List tableProperties) throws Exception { - java.nio.file.Path tempDir = createTempDirectory(null); - File dataFile = tempDir.resolve("test.txt").toFile(); - writeString(dataFile.toPath(), fileContents); + TrinoFileSystem fileSystem = getTrinoFileSystem(); + Location tempDir = Location.of("local:///temp_" + UUID.randomUUID()); + fileSystem.createDirectory(tempDir); + Location dataFile = tempDir.appendPath("text.text"); + try (OutputStream out = fileSystem.newOutputFile(dataFile).create()) { + out.write(fileContents.getBytes(UTF_8)); + } // Table properties StringJoiner propertiesSql = new StringJoiner(",\n "); - propertiesSql.add(format("external_location = '%s'", tempDir.toUri().toASCIIString())); + propertiesSql.add(format("external_location = '%s'", tempDir)); propertiesSql.add("format = 'TEXTFILE'"); tableProperties.forEach(propertiesSql::add); @@ -4440,8 +4449,8 @@ private void testCreateExternalTable( assertQuery(format("SELECT col1, col2 from %s", tableName), expectedResults); assertUpdate(format("DROP TABLE %s", tableName)); - assertThat(dataFile).exists(); // file should still exist after drop - deleteRecursively(tempDir, ALLOW_INSECURE); + assertThat(fileSystem.newInputFile(dataFile).exists()).isTrue(); // file should still exist after drop + fileSystem.deleteDirectory(tempDir); } @Test @@ -8664,8 +8673,7 @@ public void testTimestampWithTimeZone() String tableLocation = getTableLocation("test_timestamptz_base"); // TIMESTAMP WITH LOCAL TIME ZONE is not mapped to any Trino type, so we need to create the metastore entry manually - HiveMetastore metastore = ((HiveConnector) getDistributedQueryRunner().getCoordinator().getConnector(catalog)) - .getInjector().getInstance(HiveMetastoreFactory.class) + HiveMetastore metastore = TestingHiveUtils.getConnectorService(getDistributedQueryRunner(), HiveMetastoreFactory.class) .createMetastore(Optional.of(getSession().getIdentity().toConnectorIdentity(catalog))); metastore.createTable( new Table( @@ -9003,16 +9011,26 @@ public void testCollidingMixedCaseProperty() @Test public void testSelectWithShortZoneId() + throws IOException { - String resourceLocation = getPathFromClassPathResource("with_short_zone_id/data"); + URL resourceLocation = Resources.getResource("with_short_zone_id/data/data.orc"); + + TrinoFileSystem fileSystem = getTrinoFileSystem(); + Location tempDir = Location.of("local:///temp_" + UUID.randomUUID()); + fileSystem.createDirectory(tempDir); + Location dataFile = tempDir.appendPath("data.orc"); + try (OutputStream out = fileSystem.newOutputFile(dataFile).create()) { + Resources.copy(resourceLocation, out); + } try (TestTable testTable = new TestTable( getQueryRunner()::execute, "test_select_with_short_zone_id_", - "(id INT, firstName VARCHAR, lastName VARCHAR) WITH (external_location = '%s')".formatted(resourceLocation))) { + "(id INT, firstName VARCHAR, lastName VARCHAR) WITH (external_location = '%s')".formatted(tempDir))) { assertThatThrownBy(() -> query("SELECT * FROM %s".formatted(testTable.getName()))) .hasMessageMatching(".*Failed to read ORC file: .*") .hasStackTraceContaining("Unknown time-zone ID: EST"); + } } @@ -9103,7 +9121,7 @@ private static void testWithStorageFormat(TestingHiveStorageFormat storageFormat test.accept(session, storageFormat.getFormat()); } catch (Exception | AssertionError e) { - fail(format("Failure for format %s with properties %s", storageFormat.getFormat(), session.getCatalogProperties()), e); + throw new AssertionError(format("Failure for format %s with properties %s", storageFormat.getFormat(), session.getCatalogProperties()), e); } } @@ -9228,6 +9246,11 @@ private String getTableLocation(String tableName) return (String) computeScalar("SELECT DISTINCT regexp_replace(\"$path\", '/[^/]*$', '') FROM " + tableName); } + private TrinoFileSystem getTrinoFileSystem() + { + return TestingHiveUtils.getConnectorService(getQueryRunner(), TrinoFileSystemFactory.class).create(ConnectorIdentity.ofUser("test")); + } + @Override protected boolean supportsPhysicalPushdown() { diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveQueryRunner.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveQueryRunner.java index 3910565ffa73e..cbccbdfb70b98 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveQueryRunner.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveQueryRunner.java @@ -25,6 +25,7 @@ import io.trino.plugin.hive.fs.DirectoryLister; import io.trino.plugin.hive.metastore.Database; import io.trino.plugin.hive.metastore.HiveMetastore; +import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.tpcds.TpcdsPlugin; import io.trino.plugin.tpch.ColumnNaming; import io.trino.plugin.tpch.DecimalTypeMapping; @@ -38,7 +39,6 @@ import org.intellij.lang.annotations.Language; import org.joda.time.DateTimeZone; -import java.io.File; import java.nio.file.Path; import java.nio.file.Paths; import java.util.HashMap; @@ -50,7 +50,6 @@ import static com.google.inject.util.Modules.EMPTY_MODULE; import static io.airlift.log.Level.WARN; import static io.airlift.units.Duration.nanosSince; -import static io.trino.plugin.hive.metastore.file.TestingFileHiveMetastore.createTestingFileHiveMetastore; import static io.trino.plugin.hive.security.HiveSecurityModule.ALLOW_ALL; import static io.trino.plugin.hive.security.HiveSecurityModule.SQL_STANDARD; import static io.trino.plugin.tpch.ColumnNaming.SIMPLIFIED; @@ -106,10 +105,7 @@ public static class Builder> private List> initialTables = ImmutableList.of(); private Optional initialSchemasLocationBase = Optional.empty(); private Function initialTablesSessionMutator = Function.identity(); - private Function metastore = queryRunner -> { - File baseDir = queryRunner.getCoordinator().getBaseDataDir().resolve("hive_data").toFile(); - return createTestingFileHiveMetastore(baseDir); - }; + private Optional> metastore = Optional.empty(); private Optional openTelemetry = Optional.empty(); private Module module = EMPTY_MODULE; private Optional directoryLister = Optional.empty(); @@ -175,7 +171,7 @@ public SELF setInitialTablesSessionMutator(Function initialTab @CanIgnoreReturnValue public SELF setMetastore(Function metastore) { - this.metastore = requireNonNull(metastore, "metastore is null"); + this.metastore = Optional.of(metastore); return self(); } @@ -254,8 +250,9 @@ public DistributedQueryRunner build() queryRunner.createCatalog("tpcds", "tpcds"); } - HiveMetastore metastore = this.metastore.apply(queryRunner); - queryRunner.installPlugin(new TestingHivePlugin(Optional.of(metastore), openTelemetry, module, directoryLister)); + Optional metastore = this.metastore.map(factory -> factory.apply(queryRunner)); + Path dataDir = queryRunner.getCoordinator().getBaseDataDir().resolve("hive_data"); + queryRunner.installPlugin(new TestingHivePlugin(dataDir, metastore, openTelemetry, module, directoryLister)); Map hiveProperties = new HashMap<>(); if (!skipTimezoneSetup) { @@ -285,7 +282,7 @@ public DistributedQueryRunner build() queryRunner.createCatalog(HIVE_CATALOG, "hive", hiveProperties); if (createTpchSchemas) { - populateData(queryRunner, metastore); + populateData(queryRunner); } return queryRunner; @@ -296,8 +293,10 @@ public DistributedQueryRunner build() } } - private void populateData(DistributedQueryRunner queryRunner, HiveMetastore metastore) + private void populateData(DistributedQueryRunner queryRunner) { + HiveMetastore metastore = TestingHiveUtils.getConnectorService(queryRunner, HiveMetastoreFactory.class) + .createMetastore(Optional.empty()); if (metastore.getDatabase(TPCH_SCHEMA).isEmpty()) { metastore.createDatabase(createDatabaseMetastoreObject(TPCH_SCHEMA, initialSchemasLocationBase)); Session session = initialTablesSessionMutator.apply(queryRunner.getDefaultSession()); diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveConcurrentModificationGlueMetastore.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveConcurrentModificationGlueMetastore.java index ad60fe9a950c4..fb08016d7200a 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveConcurrentModificationGlueMetastore.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveConcurrentModificationGlueMetastore.java @@ -100,7 +100,7 @@ protected QueryRunner createQueryRunner() stats, table -> true); - queryRunner.installPlugin(new TestingHivePlugin(metastore)); + queryRunner.installPlugin(new TestingHivePlugin(queryRunner.getCoordinator().getBaseDataDir().resolve("hive_data"), metastore)); queryRunner.createCatalog(CATALOG_NAME, "hive"); queryRunner.execute("CREATE SCHEMA " + SCHEMA); return queryRunner; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveCreateExternalTable.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveCreateExternalTable.java index 6d0831f018b05..f689230fdf48a 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveCreateExternalTable.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveCreateExternalTable.java @@ -15,6 +15,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import io.trino.filesystem.Location; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.MaterializedResult; import io.trino.testing.QueryRunner; @@ -24,9 +25,8 @@ import java.io.File; import java.io.IOException; import java.nio.file.Path; +import java.util.UUID; -import static com.google.common.io.MoreFiles.deleteRecursively; -import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; import static io.trino.testing.QueryAssertions.assertEqualsIgnoreOrder; import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.tpch.TpchTable.CUSTOMER; @@ -50,16 +50,13 @@ protected QueryRunner createQueryRunner() @Test public void testCreateExternalTableWithData() - throws IOException { - Path tempDir = createTempDirectory(null); - String tableLocation = tempDir.resolve("data").toUri().toString(); - + Location tempDir = Location.of("local:///temp_" + UUID.randomUUID()); @Language("SQL") String createTableSql = format("" + "CREATE TABLE test_create_external " + "WITH (external_location = '%s') AS " + "SELECT * FROM tpch.tiny.nation", - tableLocation); + tempDir); assertUpdate(createTableSql, 25); @@ -69,10 +66,9 @@ public void testCreateExternalTableWithData() MaterializedResult result = computeActual("SELECT DISTINCT regexp_replace(\"$path\", '/[^/]*$', '/') FROM test_create_external"); String tablePath = (String) result.getOnlyValue(); - assertThat(tablePath).startsWith(tableLocation); + assertThat(tablePath).startsWith(tempDir.toString()); assertUpdate("DROP TABLE test_create_external"); - deleteRecursively(tempDir, ALLOW_INSECURE); } @Test @@ -92,11 +88,8 @@ public void testCreateExternalTableAsWithExistingDirectory() @Test public void testCreateExternalTableOnNonExistingPath() - throws Exception { - java.nio.file.Path tempDir = createTempDirectory(null); - // delete dir, trino should recreate it - deleteRecursively(tempDir, ALLOW_INSECURE); + Location tempDir = Location.of("local:///temp_" + UUID.randomUUID()); String tableName = "test_create_external_non_exists_" + randomNameSuffix(); @Language("SQL") String createTableSql = format("" + @@ -111,13 +104,12 @@ public void testCreateExternalTableOnNonExistingPath() getSession().getCatalog().get(), getSession().getSchema().get(), tableName, - tempDir.toUri().toASCIIString()); + tempDir); assertUpdate(createTableSql); String actual = (String) computeScalar("SHOW CREATE TABLE " + tableName); assertThat(actual).isEqualTo(createTableSql); assertUpdate("DROP TABLE " + tableName); - deleteRecursively(tempDir, ALLOW_INSECURE); } @Test diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestParquetPageSkipping.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestParquetPageSkipping.java index c3e41ac1881d3..40fc1166921a5 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestParquetPageSkipping.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestParquetPageSkipping.java @@ -17,10 +17,14 @@ import com.google.common.io.Resources; import io.trino.Session; import io.trino.execution.QueryStats; +import io.trino.filesystem.Location; +import io.trino.filesystem.TrinoFileSystem; +import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.operator.OperatorStats; import io.trino.spi.QueryId; import io.trino.spi.metrics.Count; import io.trino.spi.metrics.Metric; +import io.trino.spi.security.ConnectorIdentity; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.DistributedQueryRunner; import io.trino.testing.MaterializedResult; @@ -29,9 +33,11 @@ import org.intellij.lang.annotations.Language; import org.junit.jupiter.api.Test; -import java.io.File; -import java.net.URISyntaxException; +import java.io.IOException; +import java.io.OutputStream; +import java.net.URL; import java.util.Map; +import java.util.UUID; import static com.google.common.collect.MoreCollectors.onlyElement; import static io.trino.parquet.reader.ParquetReader.COLUMN_INDEX_ROWS_FILTERED; @@ -43,24 +49,32 @@ public class TestParquetPageSkipping extends AbstractTestQueryFramework { + private TrinoFileSystem fileSystem; + @Override protected QueryRunner createQueryRunner() throws Exception { - return HiveQueryRunner.builder() + DistributedQueryRunner queryRunner = HiveQueryRunner.builder() .setHiveProperties( ImmutableMap.of( "parquet.use-column-index", "true", "parquet.max-buffer-size", "1MB")) .build(); + + fileSystem = TestingHiveUtils.getConnectorService(queryRunner, TrinoFileSystemFactory.class) + .create(ConnectorIdentity.ofUser("test")); + + return queryRunner; } @Test public void testRowGroupPruningFromPageIndexes() throws Exception { + Location dataFile = copyInDataFile("parquet_page_skipping/orders_sorted_by_totalprice/data.parquet"); + String tableName = "test_row_group_pruning_" + randomNameSuffix(); - File parquetFile = new File(Resources.getResource("parquet_page_skipping/orders_sorted_by_totalprice").toURI()); assertUpdate( """ CREATE TABLE %s ( @@ -77,7 +91,7 @@ comment varchar(79), WITH ( format = 'PARQUET', external_location = '%s') - """.formatted(tableName, parquetFile.getAbsolutePath())); + """.formatted(tableName, dataFile.parentDirectory())); int rowCount = assertColumnIndexResults("SELECT * FROM " + tableName + " WHERE totalprice BETWEEN 100000 AND 131280 AND clerk = 'Clerk#000000624'"); assertThat(rowCount).isGreaterThan(0); @@ -90,14 +104,14 @@ comment varchar(79), @Test public void testPageSkippingWithNonSequentialOffsets() - throws URISyntaxException + throws IOException { + Location dataFile = copyInDataFile("parquet_page_skipping/random/data.parquet"); String tableName = "test_random_" + randomNameSuffix(); - File parquetFile = new File(Resources.getResource("parquet_page_skipping/random").toURI()); assertUpdate(format( "CREATE TABLE %s (col double) WITH (format = 'PARQUET', external_location = '%s')", tableName, - parquetFile.getAbsolutePath())); + dataFile.parentDirectory())); // These queries select a subset of pages which are stored at non-sequential offsets // This reproduces the issue identified in https://github.com/trinodb/trino/issues/9097 for (double i = 0; i < 1; i += 0.1) { @@ -108,17 +122,18 @@ public void testPageSkippingWithNonSequentialOffsets() @Test public void testFilteringOnColumnNameWithDot() - throws URISyntaxException + throws IOException { + Location dataFile = copyInDataFile("parquet_page_skipping/column_name_with_dot/data.parquet"); + String nameInSql = "\"a.dot\""; String tableName = "test_column_name_with_dot_" + randomNameSuffix(); - File parquetFile = new File(Resources.getResource("parquet_page_skipping/column_name_with_dot").toURI()); assertUpdate(format( "CREATE TABLE %s (key varchar(50), %s varchar(50)) WITH (format = 'PARQUET', external_location = '%s')", tableName, nameInSql, - parquetFile.getAbsolutePath())); + dataFile.parentDirectory())); assertQuery("SELECT key FROM " + tableName + " WHERE " + nameInSql + " IS NULL", "VALUES ('null value')"); assertQuery("SELECT key FROM " + tableName + " WHERE " + nameInSql + " = 'abc'", "VALUES ('sample value')"); @@ -128,16 +143,16 @@ public void testFilteringOnColumnNameWithDot() @Test public void testUnsupportedColumnIndex() - throws URISyntaxException + throws IOException { String tableName = "test_unsupported_column_index_" + randomNameSuffix(); // Test for https://github.com/trinodb/trino/issues/16801 - File parquetFile = new File(Resources.getResource("parquet_page_skipping/unsupported_column_index").toURI()); + Location dataFile = copyInDataFile("parquet_page_skipping/unsupported_column_index/data.parquet"); assertUpdate(format( "CREATE TABLE %s (stime timestamp(3), btime timestamp(3), detail varchar) WITH (format = 'PARQUET', external_location = '%s')", tableName, - parquetFile.getAbsolutePath())); + dataFile.parentDirectory())); assertQuery( "SELECT * FROM " + tableName + " WHERE btime >= timestamp '2023-03-27 13:30:00'", @@ -201,15 +216,15 @@ private void testPageSkipping(String sortByColumn, String sortByColumnType, Obje @Test public void testFilteringWithColumnIndex() - throws URISyntaxException + throws IOException { + Location dataFile = copyInDataFile("parquet_page_skipping/lineitem_sorted_by_suppkey/data.parquet"); String tableName = "test_page_filtering_" + randomNameSuffix(); - File parquetFile = new File(Resources.getResource("parquet_page_skipping/lineitem_sorted_by_suppkey").toURI()); assertUpdate(format( "CREATE TABLE %s (suppkey bigint, extendedprice decimal(12, 2), shipmode varchar(10), comment varchar(44)) " + "WITH (format = 'PARQUET', external_location = '%s')", tableName, - parquetFile.getAbsolutePath())); + dataFile.parentDirectory())); verifyFilteringWithColumnIndex("SELECT * FROM " + tableName + " WHERE suppkey = 10"); verifyFilteringWithColumnIndex("SELECT * FROM " + tableName + " WHERE suppkey BETWEEN 25 AND 35"); @@ -337,4 +352,18 @@ private void buildSortedTables(String tableName, String sortByColumnName, String format("INSERT INTO %s SELECT *, ARRAY[rand(), rand(), rand()] FROM tpch.tiny.orders", tableName), 15000); } + + private Location copyInDataFile(String resourceFileName) + throws IOException + { + URL resourceLocation = Resources.getResource(resourceFileName); + + Location tempDir = Location.of("local:///temp_" + UUID.randomUUID()); + fileSystem.createDirectory(tempDir); + Location dataFile = tempDir.appendPath("data.parquet"); + try (OutputStream out = fileSystem.newOutputFile(dataFile).create()) { + Resources.copy(resourceLocation, out); + } + return dataFile; + } } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestRegexTable.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestRegexTable.java index 8895127f71a1c..6945670d586f5 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestRegexTable.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestRegexTable.java @@ -14,6 +14,7 @@ package io.trino.plugin.hive; import com.google.common.collect.ImmutableMap; +import io.trino.filesystem.Location; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.MaterializedResult; import io.trino.testing.QueryRunner; @@ -21,12 +22,9 @@ import org.junit.jupiter.api.Test; import java.io.IOException; -import java.nio.file.Path; +import java.util.UUID; -import static com.google.common.io.MoreFiles.deleteRecursively; -import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; import static io.trino.testing.QueryAssertions.assertEqualsIgnoreOrder; -import static java.nio.file.Files.createTempDirectory; public class TestRegexTable extends AbstractTestQueryFramework @@ -44,8 +42,7 @@ protected QueryRunner createQueryRunner() public void testCreateExternalTableWithData() throws IOException { - Path tempDir = createTempDirectory(null); - Path tableLocation = tempDir.resolve("data"); + Location tempDir = Location.of("local:///temp_" + UUID.randomUUID()); // REGEX format is read-only, so create data files using the text file format @Language("SQL") String createTableSql = """ @@ -55,13 +52,15 @@ public void testCreateExternalTableWithData() textfile_field_separator = 'x', external_location = '%s') AS SELECT nationkey, name FROM tpch.tiny.nation - """.formatted(tableLocation.toUri().toASCIIString()); + """.formatted(tempDir); assertUpdate(createTableSql, 25); MaterializedResult expected = computeActual("SELECT nationkey, name FROM tpch.tiny.nation"); MaterializedResult actual = computeActual("SELECT nationkey, name FROM test_regex_data"); assertEqualsIgnoreOrder(actual.getMaterializedRows(), expected.getMaterializedRows()); + MaterializedResult x = computeActual("SELECT \"$path\" FROM test_regex_data"); + // REGEX table over the text file created data createTableSql = """ CREATE TABLE test_regex ( @@ -71,7 +70,7 @@ CREATE TABLE test_regex ( format = 'regex', regex = '(\\d+)x(.+)', external_location = '%s') - """.formatted(tableLocation.toUri().toASCIIString()); + """.formatted(tempDir); assertUpdate(createTableSql); actual = computeActual("SELECT nationkey, name FROM test_regex"); @@ -91,7 +90,7 @@ CREATE TABLE test_regex ( regex = '(\\d+)X(.+)', regex_case_insensitive = true, external_location = '%s') - """.formatted(tableLocation.toUri().toASCIIString()); + """.formatted(tempDir); assertUpdate(createTableSql); actual = computeActual("SELECT nationkey, name FROM test_regex"); assertEqualsIgnoreOrder(actual.getMaterializedRows(), expected.getMaterializedRows()); @@ -106,14 +105,13 @@ CREATE TABLE test_regex ( format = 'regex', regex = '(\\d+)X(.+)', external_location = '%s') - """.formatted(tableLocation.toUri().toASCIIString()); + """.formatted(tempDir); assertUpdate(createTableSql); // when the pattern does not match all columns are null assertQueryReturnsEmptyResult("SELECT nationkey, name FROM test_regex WHERE nationkey IS NOT NULL AND name IS NOT NULL"); assertUpdate("DROP TABLE test_regex"); assertUpdate("DROP TABLE test_regex_data"); - deleteRecursively(tempDir, ALLOW_INSECURE); } @Test diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestingHiveConnectorFactory.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestingHiveConnectorFactory.java index ae78ec5fc4a54..c0dcb429488ce 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestingHiveConnectorFactory.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestingHiveConnectorFactory.java @@ -16,16 +16,22 @@ import com.google.common.collect.ImmutableMap; import com.google.inject.Module; import io.opentelemetry.api.OpenTelemetry; +import io.trino.filesystem.TrinoFileSystemFactory; +import io.trino.filesystem.local.LocalFileSystemFactory; import io.trino.plugin.hive.fs.DirectoryLister; import io.trino.plugin.hive.metastore.HiveMetastore; +import io.trino.plugin.hive.metastore.file.FileHiveMetastoreConfig; import io.trino.spi.connector.Connector; import io.trino.spi.connector.ConnectorContext; import io.trino.spi.connector.ConnectorFactory; +import java.nio.file.Path; import java.util.Map; import java.util.Optional; +import static com.google.inject.multibindings.MapBinder.newMapBinder; import static com.google.inject.util.Modules.EMPTY_MODULE; +import static io.airlift.configuration.ConfigBinder.configBinder; import static io.trino.plugin.hive.InternalHiveConnectorFactory.createConnector; import static java.util.Objects.requireNonNull; @@ -37,12 +43,14 @@ public class TestingHiveConnectorFactory private final Module module; private final Optional directoryLister; - public TestingHiveConnectorFactory(HiveMetastore metastore) + public TestingHiveConnectorFactory(Path localFileSystemRootPath) { - this(Optional.of(metastore), Optional.empty(), EMPTY_MODULE, Optional.empty()); + this(localFileSystemRootPath, Optional.empty(), Optional.empty(), EMPTY_MODULE, Optional.empty()); } + @Deprecated public TestingHiveConnectorFactory( + Path localFileSystemRootPath, Optional metastore, Optional openTelemetry, Module module, @@ -50,7 +58,15 @@ public TestingHiveConnectorFactory( { this.metastore = requireNonNull(metastore, "metastore is null"); this.openTelemetry = requireNonNull(openTelemetry, "openTelemetry is null"); - this.module = requireNonNull(module, "module is null"); + + localFileSystemRootPath.toFile().mkdirs(); + this.module = binder -> { + binder.install(module); + newMapBinder(binder, String.class, TrinoFileSystemFactory.class) + .addBinding("local").toInstance(new LocalFileSystemFactory(localFileSystemRootPath)); + configBinder(binder).bindConfigDefaults(FileHiveMetastoreConfig.class, config -> config.setCatalogDirectory("local:///")); + }; + this.directoryLister = requireNonNull(directoryLister, "directoryLister is null"); } @@ -63,17 +79,12 @@ public String getName() @Override public Connector create(String catalogName, Map config, ConnectorContext context) { - return createConnector( - catalogName, - ImmutableMap.builder() - .putAll(config) - .put("bootstrap.quiet", "true") - .buildOrThrow(), - context, - module, - metastore, - Optional.empty(), - openTelemetry, - directoryLister); + ImmutableMap.Builder configBuilder = ImmutableMap.builder() + .putAll(config) + .put("bootstrap.quiet", "true"); + if (metastore.isEmpty() && !config.containsKey("hive.metastore")) { + configBuilder.put("hive.metastore", "file"); + } + return createConnector(catalogName, configBuilder.buildOrThrow(), context, module, metastore, Optional.empty(), openTelemetry, directoryLister); } } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestingHivePlugin.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestingHivePlugin.java index 13975b1995b33..202c5ef949a08 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestingHivePlugin.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestingHivePlugin.java @@ -21,6 +21,7 @@ import io.trino.spi.Plugin; import io.trino.spi.connector.ConnectorFactory; +import java.nio.file.Path; import java.util.Optional; import static com.google.inject.util.Modules.EMPTY_MODULE; @@ -29,23 +30,27 @@ public class TestingHivePlugin implements Plugin { + private final Path localFileSystemRootPath; private final Optional metastore; private final Optional openTelemetry; private final Module module; private final Optional directoryLister; - public TestingHivePlugin() + public TestingHivePlugin(Path localFileSystemRootPath) { - this(Optional.empty(), Optional.empty(), EMPTY_MODULE, Optional.empty()); + this(localFileSystemRootPath, Optional.empty(), Optional.empty(), EMPTY_MODULE, Optional.empty()); } - public TestingHivePlugin(HiveMetastore metastore) + @Deprecated + public TestingHivePlugin(Path localFileSystemRootPath, HiveMetastore metastore) { - this(Optional.of(metastore), Optional.empty(), EMPTY_MODULE, Optional.empty()); + this(localFileSystemRootPath, Optional.of(metastore), Optional.empty(), EMPTY_MODULE, Optional.empty()); } - public TestingHivePlugin(Optional metastore, Optional openTelemetry, Module module, Optional directoryLister) + @Deprecated + public TestingHivePlugin(Path localFileSystemRootPath, Optional metastore, Optional openTelemetry, Module module, Optional directoryLister) { + this.localFileSystemRootPath = requireNonNull(localFileSystemRootPath, "localFileSystemRootPath is null"); this.metastore = requireNonNull(metastore, "metastore is null"); this.openTelemetry = requireNonNull(openTelemetry, "openTelemetry is null"); this.module = requireNonNull(module, "module is null"); @@ -55,6 +60,6 @@ public TestingHivePlugin(Optional metastore, Optional getConnectorFactories() { - return ImmutableList.of(new TestingHiveConnectorFactory(metastore, openTelemetry, module, directoryLister)); + return ImmutableList.of(new TestingHiveConnectorFactory(localFileSystemRootPath, metastore, openTelemetry, module, directoryLister)); } } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestingHiveUtils.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestingHiveUtils.java new file mode 100644 index 0000000000000..768395d856c83 --- /dev/null +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestingHiveUtils.java @@ -0,0 +1,45 @@ +/* + * 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.hive; + +import com.google.inject.Injector; +import com.google.inject.Key; +import io.trino.testing.DistributedQueryRunner; +import io.trino.testing.LocalQueryRunner; +import io.trino.testing.QueryRunner; + +import static io.trino.plugin.hive.HiveQueryRunner.HIVE_CATALOG; + +public final class TestingHiveUtils +{ + private TestingHiveUtils() {} + + public static T getConnectorService(QueryRunner queryRunner, Class clazz) + { + return getConnectorInjector(queryRunner).getInstance(clazz); + } + + public static T getConnectorService(QueryRunner queryRunner, Key key) + { + return getConnectorInjector(queryRunner).getInstance(key); + } + + private static Injector getConnectorInjector(QueryRunner queryRunner) + { + if (queryRunner instanceof DistributedQueryRunner) { + return ((HiveConnector) ((DistributedQueryRunner) queryRunner).getCoordinator().getConnector(HIVE_CATALOG)).getInjector(); + } + return ((HiveConnector) ((LocalQueryRunner) queryRunner).getConnector(HIVE_CATALOG)).getInjector(); + } +} diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/fs/BaseCachingDirectoryListerTest.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/fs/BaseCachingDirectoryListerTest.java index f305d7a5e5146..d1f2a84e22e11 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/fs/BaseCachingDirectoryListerTest.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/fs/BaseCachingDirectoryListerTest.java @@ -17,33 +17,31 @@ import com.google.common.collect.ImmutableMap; import io.trino.filesystem.Location; import io.trino.plugin.hive.HiveQueryRunner; +import io.trino.plugin.hive.TestingHiveUtils; +import io.trino.plugin.hive.metastore.HiveMetastore; +import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.metastore.PrincipalPrivileges; import io.trino.plugin.hive.metastore.Table; -import io.trino.plugin.hive.metastore.file.FileHiveMetastore; import io.trino.testing.AbstractTestQueryFramework; +import io.trino.testing.DistributedQueryRunner; import io.trino.testing.MaterializedRow; import io.trino.testing.QueryRunner; import org.junit.jupiter.api.Test; -import java.nio.file.Path; import java.util.List; import java.util.Map; import java.util.NoSuchElementException; import java.util.Optional; -import static com.google.common.io.MoreFiles.deleteRecursively; -import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; import static io.trino.plugin.hive.HiveQueryRunner.TPCH_SCHEMA; -import static io.trino.plugin.hive.metastore.file.TestingFileHiveMetastore.createTestingFileHiveMetastore; import static java.lang.String.format; -import static java.nio.file.Files.createTempDirectory; import static org.assertj.core.api.Assertions.assertThat; public abstract class BaseCachingDirectoryListerTest extends AbstractTestQueryFramework { private C directoryLister; - private FileHiveMetastore fileHiveMetastore; + private HiveMetastore metastore; @Override protected QueryRunner createQueryRunner() @@ -55,14 +53,16 @@ protected QueryRunner createQueryRunner() protected QueryRunner createQueryRunner(Map properties) throws Exception { - Path temporaryMetastoreDirectory = createTempDirectory(null); - closeAfterClass(() -> deleteRecursively(temporaryMetastoreDirectory, ALLOW_INSECURE)); directoryLister = createDirectoryLister(); - return HiveQueryRunner.builder() + DistributedQueryRunner queryRunner = HiveQueryRunner.builder() .setHiveProperties(properties) - .setMetastore(distributedQueryRunner -> fileHiveMetastore = createTestingFileHiveMetastore(temporaryMetastoreDirectory.toFile())) .setDirectoryLister(directoryLister) .build(); + + metastore = TestingHiveUtils.getConnectorService(queryRunner, HiveMetastoreFactory.class) + .createMetastore(Optional.empty()); + + return queryRunner; } protected abstract C createDirectoryLister(); @@ -335,17 +335,17 @@ public void testDropPartitionedTable() protected Optional
getTable(String schemaName, String tableName) { - return fileHiveMetastore.getTable(schemaName, tableName); + return metastore.getTable(schemaName, tableName); } protected void createTable(Table table, PrincipalPrivileges principalPrivileges) { - fileHiveMetastore.createTable(table, principalPrivileges); + metastore.createTable(table, principalPrivileges); } protected void dropTable(String schemaName, String tableName, boolean deleteData) { - fileHiveMetastore.dropTable(schemaName, tableName, deleteData); + metastore.dropTable(schemaName, tableName, deleteData); } protected String getTableLocation(String schemaName, String tableName) @@ -360,7 +360,7 @@ protected String getPartitionLocation(String schemaName, String tableName, List< Table table = getTable(schemaName, tableName) .orElseThrow(() -> new NoSuchElementException(format("The table %s.%s could not be found", schemaName, tableName))); - return fileHiveMetastore.getPartition(table, partitionValues) + return metastore.getPartition(table, partitionValues) .map(partition -> partition.getStorage().getLocation()) .orElseThrow(() -> new NoSuchElementException(format("The partition %s from the table %s.%s could not be found", partitionValues, schemaName, tableName))); } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastoreWithQueryRunner.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastoreWithQueryRunner.java index a5d520820803c..f035968afb16c 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastoreWithQueryRunner.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastoreWithQueryRunner.java @@ -16,9 +16,13 @@ import com.google.common.base.Joiner; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.inject.Key; import io.trino.Session; import io.trino.plugin.hive.HiveQueryRunner; -import io.trino.plugin.hive.metastore.file.FileHiveMetastore; +import io.trino.plugin.hive.TestingHiveUtils; +import io.trino.plugin.hive.metastore.HiveMetastore; +import io.trino.plugin.hive.metastore.HiveMetastoreFactory; +import io.trino.plugin.hive.metastore.RawHiveMetastoreFactory; import io.trino.spi.security.Identity; import io.trino.spi.security.SelectedRole; import io.trino.testing.AbstractTestQueryFramework; @@ -27,18 +31,13 @@ import org.testng.annotations.DataProvider; import org.testng.annotations.Test; -import java.nio.file.Path; import java.util.List; import java.util.Optional; import static com.google.common.base.Verify.verify; import static com.google.common.collect.Lists.cartesianProduct; -import static com.google.common.io.MoreFiles.deleteRecursively; -import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; -import static io.trino.plugin.hive.metastore.file.TestingFileHiveMetastore.createTestingFileHiveMetastore; import static io.trino.spi.security.SelectedRole.Type.ROLE; import static io.trino.testing.TestingSession.testSessionBuilder; -import static java.nio.file.Files.createTempDirectory; import static java.util.Collections.nCopies; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -54,28 +53,27 @@ public class TestCachingHiveMetastoreWithQueryRunner private static final String ALICE_NAME = "alice"; private static final Session ALICE = getTestSession(new Identity.Builder(ALICE_NAME).build()); - private FileHiveMetastore fileHiveMetastore; + private HiveMetastore rawMetastore; @Override protected QueryRunner createQueryRunner() throws Exception { - Path temporaryMetastoreDirectory = createTempDirectory(null); - closeAfterClass(() -> deleteRecursively(temporaryMetastoreDirectory, ALLOW_INSECURE)); - DistributedQueryRunner queryRunner = HiveQueryRunner.builder(ADMIN) .setNodeCount(3) // Required by testPartitionAppend test. // Coordinator needs to be excluded from workers to deterministically reproduce the original problem // https://github.com/trinodb/trino/pull/6853 .setCoordinatorProperties(ImmutableMap.of("node-scheduler.include-coordinator", "false")) - .setMetastore(distributedQueryRunner -> fileHiveMetastore = createTestingFileHiveMetastore(temporaryMetastoreDirectory.toFile())) .setHiveProperties(ImmutableMap.of( "hive.security", "sql-standard", "hive.metastore-cache-ttl", "60m", "hive.metastore-refresh-interval", "10m")) .build(); + rawMetastore = TestingHiveUtils.getConnectorService(queryRunner, Key.get(HiveMetastoreFactory.class, RawHiveMetastoreFactory.class)) + .createMetastore(Optional.empty()); + queryRunner.execute(ADMIN, "CREATE SCHEMA " + SCHEMA); queryRunner.execute("CREATE TABLE test (test INT)"); @@ -128,7 +126,7 @@ public void testFlushHiveMetastoreCacheProcedureCallable() getQueryRunner().execute("SELECT initial FROM cached"); // Rename column name in Metastore outside Trino - fileHiveMetastore.renameColumn("test", "cached", "initial", "renamed"); + rawMetastore.renameColumn("test", "cached", "initial", "renamed"); String renamedColumnQuery = "SELECT renamed FROM cached"; // Should fail as Trino has old metadata cached diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/file/TestingFileHiveMetastore.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/file/TestingFileHiveMetastore.java index e97982ff74840..9efac5c589845 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/file/TestingFileHiveMetastore.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/file/TestingFileHiveMetastore.java @@ -15,20 +15,22 @@ import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystemFactory; +import io.trino.filesystem.local.LocalFileSystemFactory; import io.trino.plugin.hive.NodeVersion; import io.trino.plugin.hive.metastore.HiveMetastoreConfig; import java.io.File; -import static io.trino.plugin.hive.HiveTestUtils.HDFS_FILE_SYSTEM_FACTORY; - public final class TestingFileHiveMetastore { private TestingFileHiveMetastore() {} public static FileHiveMetastore createTestingFileHiveMetastore(File catalogDirectory) { - return createTestingFileHiveMetastore(HDFS_FILE_SYSTEM_FACTORY, Location.of(catalogDirectory.toURI().toString())); + catalogDirectory.mkdirs(); + return createTestingFileHiveMetastore( + new LocalFileSystemFactory(catalogDirectory.toPath()), + Location.of("local:///")); } public static FileHiveMetastore createTestingFileHiveMetastore(TrinoFileSystemFactory fileSystemFactory, Location catalogDirectory) diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreAccessOperations.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreAccessOperations.java index 35dce20c9f3e9..b24a9f15e267b 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreAccessOperations.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreAccessOperations.java @@ -62,7 +62,7 @@ protected QueryRunner createQueryRunner() File baseDir = queryRunner.getCoordinator().getBaseDataDir().resolve("hive").toFile(); metastore = new CountingAccessHiveMetastore(createTestingFileHiveMetastore(baseDir)); - queryRunner.installPlugin(new TestingHivePlugin(metastore)); + queryRunner.installPlugin(new TestingHivePlugin(baseDir.toPath(), metastore)); queryRunner.createCatalog("hive", "hive", ImmutableMap.of()); queryRunner.execute("CREATE SCHEMA test_schema"); diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreMetadataQueriesAccessOperations.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreMetadataQueriesAccessOperations.java index 348f971f32296..1e12b9b8659f4 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreMetadataQueriesAccessOperations.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreMetadataQueriesAccessOperations.java @@ -84,7 +84,7 @@ protected QueryRunner createQueryRunner() mockMetastore = new MockHiveMetastore(); metastore = new CountingAccessHiveMetastore(mockMetastore); - queryRunner.installPlugin(new TestingHivePlugin(metastore)); + queryRunner.installPlugin(new TestingHivePlugin(queryRunner.getCoordinator().getBaseDataDir().resolve("hive_data"), metastore)); queryRunner.createCatalog("hive", "hive", ImmutableMap.of()); return queryRunner; } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/optimizer/TestConnectorPushdownRulesWithHive.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/optimizer/TestConnectorPushdownRulesWithHive.java index f58af76061358..8a0a58523560d 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/optimizer/TestConnectorPushdownRulesWithHive.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/optimizer/TestConnectorPushdownRulesWithHive.java @@ -24,8 +24,10 @@ import io.trino.plugin.hive.HiveTableHandle; import io.trino.plugin.hive.HiveTransactionHandle; import io.trino.plugin.hive.TestingHiveConnectorFactory; +import io.trino.plugin.hive.TestingHiveUtils; import io.trino.plugin.hive.metastore.Database; import io.trino.plugin.hive.metastore.HiveMetastore; +import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.spi.connector.CatalogHandle; import io.trino.spi.predicate.Domain; import io.trino.spi.predicate.TupleDomain; @@ -59,9 +61,9 @@ import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; import static io.trino.plugin.hive.HiveColumnHandle.ColumnType.REGULAR; import static io.trino.plugin.hive.HiveColumnHandle.createBaseColumn; +import static io.trino.plugin.hive.HiveQueryRunner.HIVE_CATALOG; import static io.trino.plugin.hive.HiveType.HIVE_INT; import static io.trino.plugin.hive.HiveType.toHiveType; -import static io.trino.plugin.hive.metastore.file.TestingFileHiveMetastore.createTestingFileHiveMetastore; import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.spi.type.IntegerType.INTEGER; import static io.trino.spi.type.RowType.field; @@ -72,7 +74,6 @@ import static io.trino.sql.planner.assertions.PlanMatchPattern.tableScan; import static io.trino.sql.tree.ArithmeticBinaryExpression.Operator.ADD; import static io.trino.sql.tree.ArithmeticUnaryExpression.Sign.MINUS; -import static io.trino.testing.TestingHandles.TEST_CATALOG_NAME; import static io.trino.testing.TestingSession.testSessionBuilder; import static java.lang.String.format; import static java.util.Arrays.asList; @@ -89,7 +90,7 @@ public class TestConnectorPushdownRulesWithHive private CatalogHandle catalogHandle; private static final Session HIVE_SESSION = testSessionBuilder() - .setCatalog(TEST_CATALOG_NAME) + .setCatalog(HIVE_CATALOG) .setSchema(SCHEMA_NAME) .build(); @@ -103,18 +104,18 @@ protected Optional createLocalQueryRunner() throw new UncheckedIOException(e); } - metastore = createTestingFileHiveMetastore(baseDir); - Database database = Database.builder() + LocalQueryRunner queryRunner = LocalQueryRunner.create(HIVE_SESSION); + queryRunner.createCatalog(HIVE_CATALOG, new TestingHiveConnectorFactory(baseDir.toPath()), ImmutableMap.of()); + catalogHandle = queryRunner.getCatalogHandle(HIVE_CATALOG); + + metastore = TestingHiveUtils.getConnectorService(queryRunner, HiveMetastoreFactory.class) + .createMetastore(Optional.empty()); + + metastore.createDatabase(Database.builder() .setDatabaseName(SCHEMA_NAME) .setOwnerName(Optional.of("public")) .setOwnerType(Optional.of(PrincipalType.ROLE)) - .build(); - - metastore.createDatabase(database); - - LocalQueryRunner queryRunner = LocalQueryRunner.create(HIVE_SESSION); - queryRunner.createCatalog(TEST_CATALOG_NAME, new TestingHiveConnectorFactory(metastore), ImmutableMap.of()); - catalogHandle = queryRunner.getCatalogHandle(TEST_CATALOG_NAME); + .build()); return Optional.of(queryRunner); } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/optimizer/TestHivePlans.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/optimizer/TestHivePlans.java index 54c72c6597fbb..b7ea19adbf174 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/optimizer/TestHivePlans.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/optimizer/TestHivePlans.java @@ -15,8 +15,10 @@ import io.trino.Session; import io.trino.plugin.hive.TestingHiveConnectorFactory; +import io.trino.plugin.hive.TestingHiveUtils; import io.trino.plugin.hive.metastore.Database; import io.trino.plugin.hive.metastore.HiveMetastore; +import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.spi.security.PrincipalType; import io.trino.sql.planner.OptimizerConfig.JoinDistributionType; import io.trino.sql.planner.OptimizerConfig.JoinReorderingStrategy; @@ -38,7 +40,6 @@ import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; import static io.trino.SystemSessionProperties.JOIN_DISTRIBUTION_TYPE; import static io.trino.SystemSessionProperties.JOIN_REORDERING_STRATEGY; -import static io.trino.plugin.hive.metastore.file.TestingFileHiveMetastore.createTestingFileHiveMetastore; import static io.trino.sql.planner.assertions.PlanMatchPattern.exchange; import static io.trino.sql.planner.assertions.PlanMatchPattern.filter; import static io.trino.sql.planner.assertions.PlanMatchPattern.join; @@ -73,22 +74,19 @@ protected LocalQueryRunner createLocalQueryRunner() catch (IOException e) { throw new UncheckedIOException(e); } - HiveMetastore metastore = createTestingFileHiveMetastore(baseDir); - Database database = Database.builder() + + LocalQueryRunner queryRunner = LocalQueryRunner.create(HIVE_SESSION); + queryRunner.createCatalog(HIVE_CATALOG_NAME, new TestingHiveConnectorFactory(baseDir.toPath()), Map.of("hive.max-partitions-for-eager-load", "5")); + + HiveMetastore metastore = TestingHiveUtils.getConnectorService(queryRunner, HiveMetastoreFactory.class) + .createMetastore(Optional.empty()); + + metastore.createDatabase(Database.builder() .setDatabaseName(SCHEMA_NAME) .setOwnerName(Optional.of("public")) .setOwnerType(Optional.of(PrincipalType.ROLE)) - .build(); + .build()); - metastore.createDatabase(database); - - return createQueryRunner(HIVE_SESSION, metastore); - } - - protected LocalQueryRunner createQueryRunner(Session session, HiveMetastore metastore) - { - LocalQueryRunner queryRunner = LocalQueryRunner.create(session); - queryRunner.createCatalog(HIVE_CATALOG_NAME, new TestingHiveConnectorFactory(metastore), Map.of("hive.max-partitions-for-eager-load", "5")); return queryRunner; } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/optimizer/TestHiveProjectionPushdownIntoTableScan.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/optimizer/TestHiveProjectionPushdownIntoTableScan.java index 1de3d1f5d714c..e32faaa3a1c1c 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/optimizer/TestHiveProjectionPushdownIntoTableScan.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/optimizer/TestHiveProjectionPushdownIntoTableScan.java @@ -22,8 +22,10 @@ import io.trino.plugin.hive.HiveColumnHandle; import io.trino.plugin.hive.HiveTableHandle; import io.trino.plugin.hive.TestingHiveConnectorFactory; +import io.trino.plugin.hive.TestingHiveUtils; import io.trino.plugin.hive.metastore.Database; import io.trino.plugin.hive.metastore.HiveMetastore; +import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.spi.connector.ColumnHandle; import io.trino.spi.predicate.Domain; import io.trino.spi.predicate.TupleDomain; @@ -43,7 +45,6 @@ import static com.google.common.io.MoreFiles.deleteRecursively; import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; import static io.trino.plugin.hive.TestHiveReaderProjectionsUtil.createProjectedColumnHandle; -import static io.trino.plugin.hive.metastore.file.TestingFileHiveMetastore.createTestingFileHiveMetastore; import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.sql.planner.assertions.PlanMatchPattern.any; import static io.trino.sql.planner.assertions.PlanMatchPattern.anyTree; @@ -79,17 +80,18 @@ protected LocalQueryRunner createLocalQueryRunner() catch (IOException e) { throw new UncheckedIOException(e); } - HiveMetastore metastore = createTestingFileHiveMetastore(baseDir); - Database database = Database.builder() + + LocalQueryRunner queryRunner = LocalQueryRunner.create(HIVE_SESSION); + queryRunner.createCatalog(HIVE_CATALOG_NAME, new TestingHiveConnectorFactory(baseDir.toPath()), ImmutableMap.of()); + + HiveMetastore metastore = TestingHiveUtils.getConnectorService(queryRunner, HiveMetastoreFactory.class) + .createMetastore(Optional.empty()); + + metastore.createDatabase(Database.builder() .setDatabaseName(SCHEMA_NAME) .setOwnerName(Optional.of("public")) .setOwnerType(Optional.of(PrincipalType.ROLE)) - .build(); - - metastore.createDatabase(database); - - LocalQueryRunner queryRunner = LocalQueryRunner.create(HIVE_SESSION); - queryRunner.createCatalog(HIVE_CATALOG_NAME, new TestingHiveConnectorFactory(metastore), ImmutableMap.of()); + .build()); return queryRunner; } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/orc/TestHiveOrcWithShortZoneId.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/orc/TestHiveOrcWithShortZoneId.java index 0e07eb2ecc2d9..0d4a88097c054 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/orc/TestHiveOrcWithShortZoneId.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/orc/TestHiveOrcWithShortZoneId.java @@ -14,28 +14,49 @@ package io.trino.plugin.hive.orc; import com.google.common.collect.ImmutableList; +import com.google.common.io.Resources; +import io.trino.filesystem.Location; +import io.trino.filesystem.TrinoFileSystem; +import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.plugin.hive.HiveQueryRunner; +import io.trino.plugin.hive.TestingHiveUtils; +import io.trino.spi.security.ConnectorIdentity; import io.trino.testing.AbstractTestQueryFramework; +import io.trino.testing.DistributedQueryRunner; import io.trino.testing.QueryRunner; import io.trino.testing.sql.TestTable; import org.junit.jupiter.api.Test; -import static io.trino.testing.containers.TestContainers.getPathFromClassPathResource; +import java.io.OutputStream; +import java.net.URL; +import java.util.UUID; public class TestHiveOrcWithShortZoneId extends AbstractTestQueryFramework { - private String resourceLocation; + private Location dataFile; @Override protected QueryRunner createQueryRunner() throws Exception { - // See README.md to know how resource is generated - resourceLocation = getPathFromClassPathResource("with_short_zone_id/data"); - return HiveQueryRunner.builder() + DistributedQueryRunner queryRunner = HiveQueryRunner.builder() .addHiveProperty("hive.orc.read-legacy-short-zone-id", "true") .build(); + + URL resourceLocation = Resources.getResource("with_short_zone_id/data/data.orc"); + + TrinoFileSystem fileSystem = TestingHiveUtils.getConnectorService(queryRunner, TrinoFileSystemFactory.class) + .create(ConnectorIdentity.ofUser("test")); + + Location tempDir = Location.of("local:///temp_" + UUID.randomUUID()); + fileSystem.createDirectory(tempDir); + dataFile = tempDir.appendPath("data.orc"); + try (OutputStream out = fileSystem.newOutputFile(dataFile).create()) { + Resources.copy(resourceLocation, out); + } + + return queryRunner; } @Test @@ -45,7 +66,7 @@ public void testSelectWithShortZoneId() try (TestTable testTable = new TestTable( getQueryRunner()::execute, "test_select_with_short_zone_id_", - "(id INT, firstName VARCHAR, lastName VARCHAR) WITH (external_location = '%s')".formatted(resourceLocation))) { + "(id INT, firstName VARCHAR, lastName VARCHAR) WITH (external_location = '%s')".formatted(dataFile.parentDirectory()))) { assertQuery("SELECT * FROM " + testTable.getName(), "VALUES (1, 'John', 'Doe')"); } } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestReadingTimeLogicalAnnotation.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestReadingTimeLogicalAnnotation.java index b229f111478d5..2810a7161ec21 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestReadingTimeLogicalAnnotation.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestReadingTimeLogicalAnnotation.java @@ -14,16 +14,22 @@ package io.trino.plugin.hive.parquet; import com.google.common.io.Resources; +import io.trino.filesystem.Location; +import io.trino.filesystem.TrinoFileSystem; +import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.plugin.hive.HiveQueryRunner; +import io.trino.plugin.hive.TestingHiveUtils; +import io.trino.spi.security.ConnectorIdentity; import io.trino.sql.query.QueryAssertions; import io.trino.testing.DistributedQueryRunner; import org.junit.jupiter.api.Test; -import java.io.File; +import java.io.OutputStream; +import java.net.URL; +import java.util.UUID; import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.testing.MaterializedResult.resultBuilder; -import static java.lang.String.format; import static org.assertj.core.api.Assertions.assertThat; public class TestReadingTimeLogicalAnnotation @@ -32,17 +38,27 @@ public class TestReadingTimeLogicalAnnotation public void testReadingTimeLogicalAnnotationAsBigInt() throws Exception { - File parquetFile = new File(Resources.getResource("parquet_file_with_time_logical_annotation").toURI()); try (DistributedQueryRunner queryRunner = HiveQueryRunner.builder().build(); QueryAssertions assertions = new QueryAssertions(queryRunner)) { - queryRunner.execute(format(""" + URL resourceLocation = Resources.getResource("parquet_file_with_time_logical_annotation/time-micros.parquet"); + + TrinoFileSystem fileSystem = TestingHiveUtils.getConnectorService(queryRunner, TrinoFileSystemFactory.class) + .create(ConnectorIdentity.ofUser("test")); + + Location tempDir = Location.of("local:///temp_" + UUID.randomUUID()); + fileSystem.createDirectory(tempDir); + Location dataFile = tempDir.appendPath("data.parquet"); + try (OutputStream out = fileSystem.newOutputFile(dataFile).create()) { + Resources.copy(resourceLocation, out); + } + + queryRunner.execute(""" CREATE TABLE table_with_time_logical_annotation ( "opens" row(member0 bigint, member_1 varchar)) WITH ( external_location = '%s', format = 'PARQUET') - """, - parquetFile.getAbsolutePath())); + """.formatted(dataFile.parentDirectory())); assertThat(assertions.query("SELECT opens.member0 FROM table_with_time_logical_annotation GROUP BY 1 ORDER BY 1 LIMIT 5")) .matches(resultBuilder(queryRunner.getDefaultSession(), BIGINT) diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/TestHiveS3MinioQueries.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/TestHiveS3MinioQueries.java index ecac36e437fb8..0f40e6abb7b02 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/TestHiveS3MinioQueries.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/TestHiveS3MinioQueries.java @@ -15,21 +15,15 @@ import com.google.common.collect.ImmutableMap; import io.trino.plugin.hive.HiveQueryRunner; -import io.trino.plugin.hive.NodeVersion; -import io.trino.plugin.hive.metastore.HiveMetastoreConfig; -import io.trino.plugin.hive.metastore.file.FileHiveMetastore; -import io.trino.plugin.hive.metastore.file.FileHiveMetastoreConfig; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.QueryRunner; import io.trino.testing.containers.Minio; import org.junit.jupiter.api.Test; -import java.io.File; import java.util.regex.Matcher; import java.util.regex.Pattern; import static com.google.common.base.Verify.verify; -import static io.trino.plugin.hive.HiveTestUtils.HDFS_FILE_SYSTEM_FACTORY; import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.testing.containers.Minio.MINIO_ACCESS_KEY; import static io.trino.testing.containers.Minio.MINIO_SECRET_KEY; @@ -49,18 +43,8 @@ protected QueryRunner createQueryRunner() minio.start(); return HiveQueryRunner.builder() - .setMetastore(queryRunner -> { - File baseDir = queryRunner.getCoordinator().getBaseDataDir().resolve("hive_data").toFile(); - return new FileHiveMetastore( - new NodeVersion("testversion"), - HDFS_FILE_SYSTEM_FACTORY, - new HiveMetastoreConfig().isHideDeltaLakeTables(), - new FileHiveMetastoreConfig() - .setCatalogDirectory(baseDir.toURI().toString()) - .setDisableLocationChecks(true) // matches Glue behavior - .setMetastoreUser("test")); - }) .setHiveProperties(ImmutableMap.builder() + .put("hive.metastore.disable-location-checks", "true") .put("hive.s3.aws-access-key", MINIO_ACCESS_KEY) .put("hive.s3.aws-secret-key", MINIO_SECRET_KEY) .put("hive.s3.endpoint", minio.getMinioAddress()) diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/TestS3FileSystemAccessOperations.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/TestS3FileSystemAccessOperations.java index a57b47e67f5b3..b044fe9e7b260 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/TestS3FileSystemAccessOperations.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/TestS3FileSystemAccessOperations.java @@ -25,14 +25,7 @@ import io.opentelemetry.sdk.trace.data.SpanData; import io.opentelemetry.sdk.trace.export.SimpleSpanProcessor; import io.trino.Session; -import io.trino.filesystem.s3.S3FileSystemConfig; -import io.trino.filesystem.s3.S3FileSystemFactory; import io.trino.plugin.hive.HiveQueryRunner; -import io.trino.plugin.hive.NodeVersion; -import io.trino.plugin.hive.metastore.HiveMetastoreConfig; -import io.trino.plugin.hive.metastore.file.FileHiveMetastore; -import io.trino.plugin.hive.metastore.file.FileHiveMetastoreConfig; -import io.trino.plugin.hive.metastore.tracing.TracingHiveMetastore; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.DistributedQueryRunner; import io.trino.testing.QueryRunner; @@ -88,22 +81,8 @@ protected QueryRunner createQueryRunner() .build(); return HiveQueryRunner.builder() - .setMetastore(ignored -> new TracingHiveMetastore( - openTelemetry.getTracer("test"), - new FileHiveMetastore( - new NodeVersion("testversion"), - new S3FileSystemFactory(openTelemetry, new S3FileSystemConfig() - .setAwsAccessKey(MINIO_ACCESS_KEY) - .setAwsSecretKey(MINIO_SECRET_KEY) - .setRegion(MINIO_REGION) - .setEndpoint(minio.getMinioAddress()) - .setPathStyleAccess(true)), - new HiveMetastoreConfig().isHideDeltaLakeTables(), - new FileHiveMetastoreConfig() - .setCatalogDirectory("s3://%s/catalog".formatted(BUCKET)) - .setDisableLocationChecks(true) // matches Glue behavior - .setMetastoreUser("test")))) .setHiveProperties(ImmutableMap.builder() + .put("hive.metastore.disable-location-checks", "true") .put("fs.hadoop.enabled", "false") .put("fs.native-s3.enabled", "true") .put("s3.aws-access-key", MINIO_ACCESS_KEY) diff --git a/plugin/trino-hive/src/test/resources/parquet_page_skipping/column_name_with_dot/20230725_101306_00056_6ramm_28cb680f-d745-40c6-98ad-b56c8ee94ac6 b/plugin/trino-hive/src/test/resources/parquet_page_skipping/column_name_with_dot/data.parquet similarity index 100% rename from plugin/trino-hive/src/test/resources/parquet_page_skipping/column_name_with_dot/20230725_101306_00056_6ramm_28cb680f-d745-40c6-98ad-b56c8ee94ac6 rename to plugin/trino-hive/src/test/resources/parquet_page_skipping/column_name_with_dot/data.parquet diff --git a/plugin/trino-hive/src/test/resources/parquet_page_skipping/lineitem_sorted_by_suppkey/000000_0_a94130b9-2234-4000-9162-4114aefcd919_20230725_103128_00063_6ramm b/plugin/trino-hive/src/test/resources/parquet_page_skipping/lineitem_sorted_by_suppkey/data.parquet similarity index 100% rename from plugin/trino-hive/src/test/resources/parquet_page_skipping/lineitem_sorted_by_suppkey/000000_0_a94130b9-2234-4000-9162-4114aefcd919_20230725_103128_00063_6ramm rename to plugin/trino-hive/src/test/resources/parquet_page_skipping/lineitem_sorted_by_suppkey/data.parquet diff --git a/plugin/trino-hive/src/test/resources/parquet_page_skipping/orders_sorted_by_totalprice/000000_0_ca5374d9-007e-4bbd-8717-bac6677b6ee7_20230725_074756_00016_6ramm b/plugin/trino-hive/src/test/resources/parquet_page_skipping/orders_sorted_by_totalprice/data.parquet similarity index 100% rename from plugin/trino-hive/src/test/resources/parquet_page_skipping/orders_sorted_by_totalprice/000000_0_ca5374d9-007e-4bbd-8717-bac6677b6ee7_20230725_074756_00016_6ramm rename to plugin/trino-hive/src/test/resources/parquet_page_skipping/orders_sorted_by_totalprice/data.parquet diff --git a/plugin/trino-hive/src/test/resources/parquet_page_skipping/random/20230725_092119_00042_6ramm_25f11bb4-b7f7-4d05-afff-ba6b72bfb531 b/plugin/trino-hive/src/test/resources/parquet_page_skipping/random/data.parquet similarity index 100% rename from plugin/trino-hive/src/test/resources/parquet_page_skipping/random/20230725_092119_00042_6ramm_25f11bb4-b7f7-4d05-afff-ba6b72bfb531 rename to plugin/trino-hive/src/test/resources/parquet_page_skipping/random/data.parquet diff --git a/plugin/trino-hive/src/test/resources/parquet_page_skipping/unsupported_column_index/issue-16801.parquet b/plugin/trino-hive/src/test/resources/parquet_page_skipping/unsupported_column_index/data.parquet similarity index 100% rename from plugin/trino-hive/src/test/resources/parquet_page_skipping/unsupported_column_index/issue-16801.parquet rename to plugin/trino-hive/src/test/resources/parquet_page_skipping/unsupported_column_index/data.parquet diff --git a/plugin/trino-hive/src/test/resources/with_short_zone_id/data/part-00000-cab83205-643e-4b22-9846-54395fde4199-c000 b/plugin/trino-hive/src/test/resources/with_short_zone_id/data/data.orc similarity index 100% rename from plugin/trino-hive/src/test/resources/with_short_zone_id/data/part-00000-cab83205-643e-4b22-9846-54395fde4199-c000 rename to plugin/trino-hive/src/test/resources/with_short_zone_id/data/data.orc diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java index 958a82ea15c0b..6fe1ac5518f8e 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java @@ -105,7 +105,6 @@ import static io.trino.SystemSessionProperties.TASK_MAX_WRITER_COUNT; import static io.trino.SystemSessionProperties.TASK_MIN_WRITER_COUNT; import static io.trino.SystemSessionProperties.USE_PREFERRED_WRITE_PARTITIONING; -import static io.trino.plugin.hive.metastore.file.TestingFileHiveMetastore.createTestingFileHiveMetastore; import static io.trino.plugin.iceberg.IcebergFileFormat.AVRO; import static io.trino.plugin.iceberg.IcebergFileFormat.ORC; import static io.trino.plugin.iceberg.IcebergFileFormat.PARQUET; @@ -306,7 +305,7 @@ public void testShowCreateSchema() .matches("CREATE SCHEMA iceberg.tpch\n" + "AUTHORIZATION USER user\n" + "WITH \\(\n" + - "\\s+location = '.*/iceberg_data/tpch'\n" + + "\\s+location = '.*/tpch'\n" + "\\)"); } @@ -345,7 +344,7 @@ public void testShowCreateTable() "WITH (\n" + " format = '" + format.name() + "',\n" + " format_version = 2,\n" + - " location = '\\E.*/iceberg_data/tpch/orders-.*\\Q'\n" + + " location = '\\E.*/tpch/orders-.*\\Q'\n" + ")\\E"); } @@ -4195,7 +4194,7 @@ public void testIncorrectIcebergFileSizes() // Using Iceberg provided file size fails the query assertQueryFails( "SELECT * FROM test_iceberg_file_size", - "(Malformed ORC file\\. Invalid file metadata.*)|(.*Error opening Iceberg split.* Incorrect file size \\(%s\\) for file .*)".formatted(alteredValue)); + "(Malformed ORC file\\. Invalid file metadata.*)|(.*Malformed Parquet file.*)"); dropTable("test_iceberg_file_size"); } @@ -6961,7 +6960,7 @@ public void testDropCorruptedTableWithHiveRedirection() "iceberg.catalog.type", "TESTING_FILE_METASTORE", "hive.metastore.catalog.dir", dataDirectory.getPath())); - queryRunner.installPlugin(new TestingHivePlugin(createTestingFileHiveMetastore(dataDirectory))); + queryRunner.installPlugin(new TestingHivePlugin(dataDirectory.toPath())); queryRunner.createCatalog( hiveRedirectionCatalog, "hive", diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergMaterializedViewTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergMaterializedViewTest.java index 9f3820d214109..6250594e79092 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergMaterializedViewTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergMaterializedViewTest.java @@ -16,11 +16,14 @@ import com.google.common.collect.ImmutableSet; import io.trino.Session; import io.trino.filesystem.Location; -import io.trino.filesystem.local.LocalFileSystem; +import io.trino.filesystem.TrinoFileSystem; +import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.plugin.iceberg.fileio.ForwardingFileIo; import io.trino.spi.QueryId; +import io.trino.spi.security.ConnectorIdentity; import io.trino.sql.tree.ExplainType; import io.trino.testing.AbstractTestQueryFramework; +import io.trino.testing.DistributedQueryRunner; import io.trino.testing.MaterializedRow; import org.apache.iceberg.PartitionField; import org.apache.iceberg.TableMetadata; @@ -31,7 +34,6 @@ import org.junit.jupiter.api.TestInstance; import org.junit.jupiter.api.parallel.Execution; -import java.nio.file.Path; import java.util.Optional; import java.util.Set; @@ -745,8 +747,12 @@ protected String getColumnComment(String tableName, String columnName) private TableMetadata getStorageTableMetadata(String materializedViewName) { + DistributedQueryRunner queryRunner = (DistributedQueryRunner) getQueryRunner(); + TrinoFileSystem fileSystemFactory = ((IcebergConnector) queryRunner.getCoordinator().getConnector("iceberg")).getInjector() + .getInstance(TrinoFileSystemFactory.class) + .create(ConnectorIdentity.ofUser("test")); Location metadataLocation = Location.of(getStorageMetadataLocation(materializedViewName)); - return TableMetadataParser.read(new ForwardingFileIo(new LocalFileSystem(Path.of(metadataLocation.parentDirectory().toString()))), "local:///" + metadataLocation); + return TableMetadataParser.read(new ForwardingFileIo(fileSystemFactory), metadataLocation.toString()); } private long getLatestSnapshotId(String tableName) diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/IcebergQueryRunner.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/IcebergQueryRunner.java index b38adf6aea7f8..14ef2d855e0e4 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/IcebergQueryRunner.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/IcebergQueryRunner.java @@ -36,7 +36,6 @@ import java.nio.file.attribute.FileAttribute; import java.nio.file.attribute.PosixFilePermission; import java.nio.file.attribute.PosixFilePermissions; -import java.util.HashMap; import java.util.Map; import java.util.Optional; import java.util.Set; @@ -138,16 +137,9 @@ public DistributedQueryRunner build() queryRunner.installPlugin(new TpchPlugin()); queryRunner.createCatalog("tpch", "tpch"); - queryRunner.installPlugin(new IcebergPlugin()); - Map icebergProperties = new HashMap<>(this.icebergProperties.buildOrThrow()); - String catalogType = icebergProperties.get("iceberg.catalog.type"); Path dataDir = metastoreDirectory.map(File::toPath).orElseGet(() -> queryRunner.getCoordinator().getBaseDataDir().resolve("iceberg_data")); - if (catalogType == null) { - icebergProperties.put("iceberg.catalog.type", "TESTING_FILE_METASTORE"); - icebergProperties.put("hive.metastore.catalog.dir", dataDir.toString()); - } - - queryRunner.createCatalog(ICEBERG_CATALOG, "iceberg", icebergProperties); + queryRunner.installPlugin(new TestingIcebergPlugin(dataDir)); + queryRunner.createCatalog(ICEBERG_CATALOG, "iceberg", icebergProperties.buildOrThrow()); schemaInitializer.orElseGet(() -> SchemaInitializer.builder().build()).accept(queryRunner); return queryRunner; diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/IcebergTestUtils.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/IcebergTestUtils.java index 39199092f2c35..a4716d6b5701c 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/IcebergTestUtils.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/IcebergTestUtils.java @@ -19,7 +19,6 @@ import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.filesystem.TrinoInputFile; -import io.trino.filesystem.local.LocalInputFile; import io.trino.orc.OrcDataSource; import io.trino.orc.OrcReader; import io.trino.orc.OrcReaderOptions; @@ -35,7 +34,6 @@ import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; import org.apache.parquet.hadoop.metadata.ParquetMetadata; -import java.io.File; import java.io.IOException; import java.io.UncheckedIOException; import java.util.List; @@ -113,11 +111,6 @@ private static boolean checkOrcFileSorting(Supplier dataSourceSup } } - public static boolean checkParquetFileSorting(String path, String sortColumnName) - { - return checkParquetFileSorting(new LocalInputFile(new File(path)), sortColumnName); - } - @SuppressWarnings({"unchecked", "rawtypes"}) public static boolean checkParquetFileSorting(TrinoInputFile inputFile, String sortColumnName) { diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergConnectorSmokeTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergConnectorSmokeTest.java index a397934fee860..ca81ed98fe685 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergConnectorSmokeTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergConnectorSmokeTest.java @@ -16,21 +16,17 @@ import com.google.common.collect.ImmutableMap; import io.trino.filesystem.Location; import io.trino.plugin.hive.metastore.HiveMetastore; +import io.trino.plugin.hive.metastore.HiveMetastoreFactory; +import io.trino.testing.DistributedQueryRunner; import io.trino.testing.QueryRunner; -import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.TestInstance; -import java.io.File; import java.io.IOException; import java.io.UncheckedIOException; -import java.nio.file.Files; -import java.nio.file.Path; +import java.util.Optional; -import static com.google.common.io.MoreFiles.deleteRecursively; -import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; -import static io.trino.plugin.hive.metastore.file.TestingFileHiveMetastore.createTestingFileHiveMetastore; +import static io.trino.plugin.iceberg.IcebergQueryRunner.ICEBERG_CATALOG; import static io.trino.plugin.iceberg.IcebergTestUtils.checkOrcFileSorting; -import static java.lang.String.format; import static org.apache.iceberg.FileFormat.ORC; import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; @@ -42,7 +38,6 @@ public class TestIcebergConnectorSmokeTest extends BaseIcebergConnectorSmokeTest { private HiveMetastore metastore; - private File metastoreDir; public TestIcebergConnectorSmokeTest() { @@ -53,24 +48,17 @@ public TestIcebergConnectorSmokeTest() protected QueryRunner createQueryRunner() throws Exception { - this.metastoreDir = Files.createTempDirectory("test_iceberg_table_smoke_test").toFile(); - this.metastoreDir.deleteOnExit(); - this.metastore = createTestingFileHiveMetastore(metastoreDir); - return IcebergQueryRunner.builder() + DistributedQueryRunner queryRunner = IcebergQueryRunner.builder() .setInitialTables(REQUIRED_TPCH_TABLES) - .setMetastoreDirectory(metastoreDir) .setIcebergProperties(ImmutableMap.of( "iceberg.file-format", format.name(), "iceberg.register-table-procedure.enabled", "true", "iceberg.writer-sort-buffer-size", "1MB")) .build(); - } - - @AfterAll - public void tearDown() - throws IOException - { - deleteRecursively(metastoreDir.toPath(), ALLOW_INSECURE); + metastore = ((IcebergConnector) queryRunner.getCoordinator().getConnector(ICEBERG_CATALOG)).getInjector() + .getInstance(HiveMetastoreFactory.class) + .createMetastore(Optional.empty()); + return queryRunner; } @Override @@ -91,20 +79,25 @@ protected String getMetadataLocation(String tableName) @Override protected String schemaPath() { - return format("%s/%s", metastoreDir, getSession().getSchema().orElseThrow()); + return "local:///%s".formatted(getSession().getSchema().orElseThrow()); } @Override protected boolean locationExists(String location) { - return Files.exists(Path.of(location)); + try { + return fileSystem.newInputFile(Location.of(location)).exists(); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } } @Override protected void deleteDirectory(String location) { try { - deleteRecursively(Path.of(location), ALLOW_INSECURE); + fileSystem.deleteDirectory(Location.of(location)); } catch (IOException e) { throw new UncheckedIOException(e); diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergFileOperations.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergFileOperations.java index 8b2b79cc6de2d..9e1235f2a3d38 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergFileOperations.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergFileOperations.java @@ -21,9 +21,7 @@ import io.trino.SystemSessionProperties; import io.trino.filesystem.TrackingFileSystemFactory; import io.trino.filesystem.TrackingFileSystemFactory.OperationType; -import io.trino.filesystem.hdfs.HdfsFileSystemFactory; -import io.trino.plugin.hive.metastore.HiveMetastore; -import io.trino.plugin.iceberg.catalog.file.TestingIcebergFileMetastoreCatalogModule; +import io.trino.filesystem.local.LocalFileSystemFactory; import io.trino.plugin.tpch.TpchPlugin; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.DistributedQueryRunner; @@ -35,7 +33,7 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.MethodSource; -import java.io.File; +import java.nio.file.Path; import java.util.Optional; import java.util.function.Predicate; @@ -47,14 +45,12 @@ import static io.trino.filesystem.TrackingFileSystemFactory.OperationType.INPUT_FILE_NEW_STREAM; import static io.trino.filesystem.TrackingFileSystemFactory.OperationType.OUTPUT_FILE_CREATE; import static io.trino.filesystem.TrackingFileSystemFactory.OperationType.OUTPUT_FILE_CREATE_OR_OVERWRITE; -import static io.trino.plugin.hive.HiveTestUtils.HDFS_ENVIRONMENT; -import static io.trino.plugin.hive.HiveTestUtils.HDFS_FILE_SYSTEM_STATS; -import static io.trino.plugin.hive.metastore.file.TestingFileHiveMetastore.createTestingFileHiveMetastore; import static io.trino.plugin.iceberg.IcebergQueryRunner.ICEBERG_CATALOG; import static io.trino.plugin.iceberg.IcebergSessionProperties.COLLECT_EXTENDED_STATISTICS_ON_WRITE; import static io.trino.plugin.iceberg.TestIcebergFileOperations.FileType.DATA; import static io.trino.plugin.iceberg.TestIcebergFileOperations.FileType.MANIFEST; import static io.trino.plugin.iceberg.TestIcebergFileOperations.FileType.METADATA_JSON; +import static io.trino.plugin.iceberg.TestIcebergFileOperations.FileType.METASTORE; import static io.trino.plugin.iceberg.TestIcebergFileOperations.FileType.SNAPSHOT; import static io.trino.plugin.iceberg.TestIcebergFileOperations.FileType.STATS; import static io.trino.plugin.iceberg.TestIcebergFileOperations.FileType.fromFilePath; @@ -97,12 +93,12 @@ protected DistributedQueryRunner createQueryRunner() .addCoordinatorProperty("optimizer.experimental-max-prefetched-information-schema-prefixes", Integer.toString(MAX_PREFIXES_COUNT)) .build(); - File baseDir = queryRunner.getCoordinator().getBaseDataDir().resolve("iceberg_data").toFile(); - HiveMetastore metastore = createTestingFileHiveMetastore(baseDir); - - trackingFileSystemFactory = new TrackingFileSystemFactory(new HdfsFileSystemFactory(HDFS_ENVIRONMENT, HDFS_FILE_SYSTEM_STATS)); + Path dataDirectory = queryRunner.getCoordinator().getBaseDataDir().resolve("iceberg_data"); + dataDirectory.toFile().mkdirs(); + trackingFileSystemFactory = new TrackingFileSystemFactory(new LocalFileSystemFactory(dataDirectory)); queryRunner.installPlugin(new TestingIcebergPlugin( - Optional.of(new TestingIcebergFileMetastoreCatalogModule(metastore)), + dataDirectory, + Optional.empty(), Optional.of(trackingFileSystemFactory), binder -> { newOptionalBinder(binder, Key.get(boolean.class, AsyncIcebergSplitProducer.class)) @@ -868,14 +864,14 @@ enum Scope @Override public boolean test(FileOperation fileOperation) { - return fileOperation.fileType() != DATA; + return fileOperation.fileType() != DATA && fileOperation.fileType() != METASTORE; } }, ALL_FILES { @Override public boolean test(FileOperation fileOperation) { - return true; + return fileOperation.fileType() != METASTORE; } }, } @@ -887,6 +883,7 @@ enum FileType MANIFEST, STATS, DATA, + METASTORE, /**/; public static FileType fromFilePath(String path) @@ -906,6 +903,9 @@ public static FileType fromFilePath(String path) if (path.contains("/data/") && (path.endsWith(".orc") || path.endsWith(".parquet"))) { return DATA; } + if (path.endsWith(".trinoSchema") || path.contains("/.trinoPermissions/")) { + return METASTORE; + } throw new IllegalArgumentException("File not recognized: " + path); } } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergGetTableStatisticsOperations.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergGetTableStatisticsOperations.java index 46260b4121039..b31ca0a442a16 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergGetTableStatisticsOperations.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergGetTableStatisticsOperations.java @@ -21,7 +21,7 @@ import io.trino.metadata.InternalFunctionBundle; import io.trino.plugin.hive.metastore.Database; import io.trino.plugin.hive.metastore.HiveMetastore; -import io.trino.plugin.iceberg.catalog.file.TestingIcebergFileMetastoreCatalogModule; +import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.tpch.TpchPlugin; import io.trino.spi.security.PrincipalType; import io.trino.testing.AbstractTestQueryFramework; @@ -34,17 +34,16 @@ import org.junit.jupiter.api.TestInstance; import org.junit.jupiter.api.parallel.Execution; -import java.io.File; import java.io.IOException; import java.nio.file.Files; +import java.nio.file.Path; import java.util.Optional; import static com.google.common.io.MoreFiles.deleteRecursively; import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; -import static com.google.inject.util.Modules.EMPTY_MODULE; import static io.trino.execution.querystats.PlanOptimizersStatsCollector.createPlanOptimizersStatsCollector; import static io.trino.execution.warnings.WarningCollector.NOOP; -import static io.trino.plugin.hive.metastore.file.TestingFileHiveMetastore.createTestingFileHiveMetastore; +import static io.trino.plugin.iceberg.IcebergQueryRunner.ICEBERG_CATALOG; import static io.trino.sql.planner.LogicalPlanner.Stage.OPTIMIZED_AND_VALIDATED; import static io.trino.testing.TestingSession.testSessionBuilder; import static org.assertj.core.api.Assertions.assertThat; @@ -60,7 +59,7 @@ public class TestIcebergGetTableStatisticsOperations { private LocalQueryRunner localQueryRunner; private InMemorySpanExporter spanExporter; - private File metastoreDir; + private Path metastoreDir; @Override protected QueryRunner createQueryRunner() @@ -82,12 +81,16 @@ protected QueryRunner createQueryRunner() new IcebergPlugin().getFunctions().forEach(functions::functions); localQueryRunner.addFunctions(functions.build()); - metastoreDir = Files.createTempDirectory("test_iceberg_get_table_statistics_operations").toFile(); - HiveMetastore metastore = createTestingFileHiveMetastore(metastoreDir); + metastoreDir = Files.createTempDirectory("test_iceberg_get_table_statistics_operations"); localQueryRunner.createCatalog( "iceberg", - new TestingIcebergConnectorFactory(Optional.of(new TestingIcebergFileMetastoreCatalogModule(metastore)), Optional.empty(), EMPTY_MODULE), + new TestingIcebergConnectorFactory(metastoreDir), ImmutableMap.of()); + + HiveMetastore metastore = ((IcebergConnector) localQueryRunner.getConnector(ICEBERG_CATALOG)).getInjector() + .getInstance(HiveMetastoreFactory.class) + .createMetastore(Optional.empty()); + Database database = Database.builder() .setDatabaseName("tiny") .setOwnerName(Optional.of("public")) @@ -106,7 +109,7 @@ protected QueryRunner createQueryRunner() public void tearDown() throws IOException { - deleteRecursively(metastoreDir.toPath(), ALLOW_INSECURE); + deleteRecursively(metastoreDir, ALLOW_INSECURE); localQueryRunner.close(); } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMaterializedView.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMaterializedView.java index 7b8a2aa6dd714..ba916ffaca313 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMaterializedView.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMaterializedView.java @@ -15,18 +15,17 @@ import io.trino.Session; import io.trino.plugin.hive.metastore.HiveMetastore; +import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.metastore.Table; import io.trino.sql.tree.ExplainType; import io.trino.testing.DistributedQueryRunner; import org.junit.jupiter.api.Test; -import java.io.File; -import java.nio.file.Files; -import java.nio.file.Path; import java.util.Map; +import java.util.Optional; import static io.trino.plugin.base.util.Closables.closeAllSuppress; -import static io.trino.plugin.hive.metastore.file.TestingFileHiveMetastore.createTestingFileHiveMetastore; +import static io.trino.plugin.iceberg.IcebergQueryRunner.ICEBERG_CATALOG; import static org.apache.iceberg.BaseMetastoreTableOperations.METADATA_LOCATION_PROP; import static org.assertj.core.api.Assertions.assertThat; @@ -34,21 +33,19 @@ public class TestIcebergMaterializedView extends BaseIcebergMaterializedViewTest { private Session secondIceberg; - private String fileMetastoreDirectory; private HiveMetastore metastore; @Override protected DistributedQueryRunner createQueryRunner() throws Exception { - File metastoreDir = Files.createTempDirectory("test_iceberg_table_smoke_test").toFile(); - metastoreDir.deleteOnExit(); - this.fileMetastoreDirectory = metastoreDir.getAbsolutePath(); - this.metastore = createTestingFileHiveMetastore(metastoreDir); DistributedQueryRunner queryRunner = IcebergQueryRunner.builder() - .setMetastoreDirectory(metastoreDir) .build(); try { + metastore = ((IcebergConnector) queryRunner.getCoordinator().getConnector(ICEBERG_CATALOG)).getInjector() + .getInstance(HiveMetastoreFactory.class) + .createMetastore(Optional.empty()); + queryRunner.createCatalog("iceberg2", "iceberg", Map.of( "iceberg.catalog.type", "TESTING_FILE_METASTORE", "hive.metastore.catalog.dir", queryRunner.getCoordinator().getBaseDataDir().resolve("iceberg2-catalog").toString(), @@ -70,7 +67,7 @@ protected DistributedQueryRunner createQueryRunner() @Override protected String getSchemaDirectory() { - return Path.of(fileMetastoreDirectory, "tpch").toString(); + return "local:///tpch"; } @Override diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMergeAppend.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMergeAppend.java index 33e15cedb9a85..355cb21ee25c0 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMergeAppend.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMergeAppend.java @@ -16,6 +16,8 @@ import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.plugin.base.CatalogName; import io.trino.plugin.hive.TrinoViewHiveMetastore; +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.plugin.iceberg.catalog.IcebergTableOperationsProvider; import io.trino.plugin.iceberg.catalog.TrinoCatalog; @@ -31,10 +33,10 @@ import org.apache.iceberg.Table; import org.junit.jupiter.api.Test; -import java.io.File; +import java.util.Optional; import static io.trino.plugin.hive.metastore.cache.CachingHiveMetastore.createPerTransactionCache; -import static io.trino.plugin.hive.metastore.file.TestingFileHiveMetastore.createTestingFileHiveMetastore; +import static io.trino.plugin.iceberg.IcebergQueryRunner.ICEBERG_CATALOG; import static io.trino.plugin.iceberg.IcebergTestUtils.getFileSystemFactory; import static org.assertj.core.api.Assertions.assertThat; @@ -49,8 +51,10 @@ protected QueryRunner createQueryRunner() throws Exception { DistributedQueryRunner queryRunner = IcebergQueryRunner.createIcebergQueryRunner(); - File baseDir = queryRunner.getCoordinator().getBaseDataDir().resolve("iceberg_data").toFile(); - CachingHiveMetastore cachingHiveMetastore = createPerTransactionCache(createTestingFileHiveMetastore(baseDir), 1000); + HiveMetastore metastore = ((IcebergConnector) queryRunner.getCoordinator().getConnector(ICEBERG_CATALOG)).getInjector() + .getInstance(HiveMetastoreFactory.class) + .createMetastore(Optional.empty()); + CachingHiveMetastore cachingHiveMetastore = createPerTransactionCache(metastore, 1000); TrinoFileSystemFactory fileSystemFactory = getFileSystemFactory(queryRunner); tableOperationsProvider = new FileMetastoreTableOperationsProvider(fileSystemFactory); trinoCatalog = new TrinoHiveCatalog( diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetadataListing.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetadataListing.java index d05e92225477b..8c3831435bde8 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetadataListing.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetadataListing.java @@ -18,8 +18,9 @@ import io.trino.metadata.MaterializedViewDefinition; import io.trino.metadata.QualifiedObjectName; import io.trino.plugin.hive.TestingHivePlugin; -import io.trino.plugin.hive.metastore.file.FileHiveMetastore; -import io.trino.plugin.iceberg.catalog.file.TestingIcebergFileMetastoreCatalogModule; +import io.trino.plugin.hive.TestingHiveUtils; +import io.trino.plugin.hive.metastore.HiveMetastore; +import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.spi.connector.SchemaTableName; import io.trino.spi.security.Identity; import io.trino.spi.security.SelectedRole; @@ -35,8 +36,6 @@ import java.io.File; import java.util.Optional; -import static com.google.inject.util.Modules.EMPTY_MODULE; -import static io.trino.plugin.hive.metastore.file.TestingFileHiveMetastore.createTestingFileHiveMetastore; import static io.trino.spi.security.SelectedRole.Type.ROLE; import static io.trino.testing.TestingSession.testSessionBuilder; import static org.assertj.core.api.Assertions.assertThat; @@ -46,7 +45,7 @@ public class TestIcebergMetadataListing extends AbstractTestQueryFramework { - private FileHiveMetastore metastore; + private HiveMetastore metastore; private SchemaTableName storageTable; @Override @@ -62,13 +61,14 @@ protected DistributedQueryRunner createQueryRunner() File baseDir = queryRunner.getCoordinator().getBaseDataDir().resolve("iceberg_data").toFile(); - metastore = createTestingFileHiveMetastore(baseDir); - - queryRunner.installPlugin(new TestingIcebergPlugin(Optional.of(new TestingIcebergFileMetastoreCatalogModule(metastore)), Optional.empty(), EMPTY_MODULE)); + queryRunner.installPlugin(new TestingIcebergPlugin(baseDir.toPath())); queryRunner.createCatalog("iceberg", "iceberg"); - queryRunner.installPlugin(new TestingHivePlugin(metastore)); + queryRunner.installPlugin(new TestingHivePlugin(baseDir.toPath())); queryRunner.createCatalog("hive", "hive", ImmutableMap.of("hive.security", "sql-standard")); + metastore = TestingHiveUtils.getConnectorService(queryRunner, HiveMetastoreFactory.class) + .createMetastore(Optional.empty()); + return queryRunner; } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetastoreAccessOperations.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetastoreAccessOperations.java index 005a0cfecccec..a19861c2f78b6 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetastoreAccessOperations.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetastoreAccessOperations.java @@ -76,7 +76,11 @@ protected DistributedQueryRunner createQueryRunner() File baseDir = queryRunner.getCoordinator().getBaseDataDir().resolve("iceberg_data").toFile(); metastore = new CountingAccessHiveMetastore(createTestingFileHiveMetastore(baseDir)); - queryRunner.installPlugin(new TestingIcebergPlugin(Optional.of(new TestingIcebergFileMetastoreCatalogModule(metastore)), Optional.empty(), EMPTY_MODULE)); + queryRunner.installPlugin(new TestingIcebergPlugin( + baseDir.toPath(), + Optional.of(new TestingIcebergFileMetastoreCatalogModule(metastore)), + Optional.empty(), + EMPTY_MODULE)); queryRunner.createCatalog("iceberg", "iceberg"); queryRunner.execute("CREATE SCHEMA test_schema"); diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMigrateProcedure.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMigrateProcedure.java index 126b116dcb497..f814f1829bdf2 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMigrateProcedure.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMigrateProcedure.java @@ -45,10 +45,10 @@ protected QueryRunner createQueryRunner() { dataDirectory = Files.createTempDirectory("_test_hidden"); DistributedQueryRunner queryRunner = IcebergQueryRunner.builder().setMetastoreDirectory(dataDirectory.toFile()).build(); - queryRunner.installPlugin(new TestingHivePlugin()); + queryRunner.installPlugin(new TestingHivePlugin(dataDirectory)); queryRunner.createCatalog("hive", "hive", ImmutableMap.builder() - .put("hive.metastore", "file") - .put("hive.metastore.catalog.dir", dataDirectory.toString()) +// .put("hive.metastore", "file") +// .put("hive.metastore.catalog.dir", dataDirectory.toString()) .put("hive.security", "allow-all") .buildOrThrow()); return queryRunner; diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergOrcMetricsCollection.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergOrcMetricsCollection.java index 0371c613249a4..fd003829577a5 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergOrcMetricsCollection.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergOrcMetricsCollection.java @@ -19,11 +19,11 @@ import io.trino.plugin.base.CatalogName; import io.trino.plugin.hive.TrinoViewHiveMetastore; 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.plugin.iceberg.catalog.IcebergTableOperationsProvider; import io.trino.plugin.iceberg.catalog.TrinoCatalog; import io.trino.plugin.iceberg.catalog.file.FileMetastoreTableOperationsProvider; -import io.trino.plugin.iceberg.catalog.file.TestingIcebergFileMetastoreCatalogModule; import io.trino.plugin.iceberg.catalog.hms.TrinoHiveCatalog; import io.trino.plugin.tpch.TpchPlugin; import io.trino.spi.connector.SchemaTableName; @@ -43,13 +43,11 @@ import java.util.Map; import java.util.Optional; -import static com.google.inject.util.Modules.EMPTY_MODULE; import static io.trino.SystemSessionProperties.MAX_DRIVERS_PER_TASK; import static io.trino.SystemSessionProperties.TASK_CONCURRENCY; import static io.trino.SystemSessionProperties.TASK_MAX_WRITER_COUNT; import static io.trino.SystemSessionProperties.TASK_MIN_WRITER_COUNT; import static io.trino.plugin.hive.metastore.cache.CachingHiveMetastore.createPerTransactionCache; -import static io.trino.plugin.hive.metastore.file.TestingFileHiveMetastore.createTestingFileHiveMetastore; import static io.trino.plugin.iceberg.DataFileRecord.toDataFileRecord; import static io.trino.plugin.iceberg.IcebergQueryRunner.ICEBERG_CATALOG; import static io.trino.plugin.iceberg.IcebergTestUtils.getFileSystemFactory; @@ -80,13 +78,17 @@ protected QueryRunner createQueryRunner() .build(); File baseDir = queryRunner.getCoordinator().getBaseDataDir().resolve("iceberg_data").toFile(); - HiveMetastore metastore = createTestingFileHiveMetastore(baseDir); - queryRunner.installPlugin(new TestingIcebergPlugin(Optional.of(new TestingIcebergFileMetastoreCatalogModule(metastore)), Optional.empty(), EMPTY_MODULE)); + queryRunner.installPlugin(new TestingIcebergPlugin(baseDir.toPath())); queryRunner.createCatalog(ICEBERG_CATALOG, "iceberg", ImmutableMap.of("iceberg.file-format", "ORC")); TrinoFileSystemFactory fileSystemFactory = getFileSystemFactory(queryRunner); tableOperationsProvider = new FileMetastoreTableOperationsProvider(fileSystemFactory); + + HiveMetastore metastore = ((IcebergConnector) queryRunner.getCoordinator().getConnector(ICEBERG_CATALOG)).getInjector() + .getInstance(HiveMetastoreFactory.class) + .createMetastore(Optional.empty()); + CachingHiveMetastore cachingHiveMetastore = createPerTransactionCache(metastore, 1000); trinoCatalog = new TrinoHiveCatalog( new CatalogName("catalog"), diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergParquetConnectorTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergParquetConnectorTest.java index f2830b0408c12..6f6b2d83f663e 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergParquetConnectorTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergParquetConnectorTest.java @@ -13,6 +13,7 @@ */ package io.trino.plugin.iceberg; +import io.trino.filesystem.Location; import io.trino.testing.MaterializedResult; import io.trino.testing.sql.TestTable; import org.junit.jupiter.api.Test; @@ -93,6 +94,6 @@ public void testDropAmbiguousRowFieldCaseSensitivity() @Override protected boolean isFileSorted(String path, String sortColumnName) { - return checkParquetFileSorting(path, sortColumnName); + return checkParquetFileSorting(fileSystem.newInputFile(Location.of(path)), sortColumnName); } } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergParquetWithBloomFilters.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergParquetWithBloomFilters.java index 54b3c936b1d14..fa8d634704c49 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergParquetWithBloomFilters.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergParquetWithBloomFilters.java @@ -39,10 +39,8 @@ protected QueryRunner createQueryRunner() dataDirectory = queryRunner.getCoordinator().getBaseDataDir().resolve("iceberg_data"); // create hive catalog - queryRunner.installPlugin(new TestingHivePlugin()); + queryRunner.installPlugin(new TestingHivePlugin(dataDirectory)); queryRunner.createCatalog("hive", "hive", ImmutableMap.builder() - .put("hive.metastore", "file") - .put("hive.metastore.catalog.dir", dataDirectory.toString()) .put("hive.security", "allow-all") .buildOrThrow()); diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergProjectionPushdownPlans.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergProjectionPushdownPlans.java index 6ab65da9f99fd..bd2953f7646bd 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergProjectionPushdownPlans.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergProjectionPushdownPlans.java @@ -22,7 +22,7 @@ import io.trino.metadata.TableHandle; import io.trino.plugin.hive.metastore.Database; import io.trino.plugin.hive.metastore.HiveMetastore; -import io.trino.plugin.iceberg.catalog.file.TestingIcebergFileMetastoreCatalogModule; +import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.spi.connector.ColumnHandle; import io.trino.spi.predicate.Domain; import io.trino.spi.predicate.TupleDomain; @@ -43,8 +43,6 @@ import static com.google.common.base.Predicates.equalTo; import static com.google.common.io.MoreFiles.deleteRecursively; import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; -import static com.google.inject.util.Modules.EMPTY_MODULE; -import static io.trino.plugin.hive.metastore.file.TestingFileHiveMetastore.createTestingFileHiveMetastore; import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.sql.planner.assertions.PlanMatchPattern.any; import static io.trino.sql.planner.assertions.PlanMatchPattern.anyTree; @@ -80,7 +78,6 @@ protected LocalQueryRunner createLocalQueryRunner() catch (IOException e) { throw new UncheckedIOException(e); } - HiveMetastore metastore = createTestingFileHiveMetastore(metastoreDir); LocalQueryRunner queryRunner = LocalQueryRunner.create(session); InternalFunctionBundle.InternalFunctionBundleBuilder functions = InternalFunctionBundle.builder(); @@ -89,9 +86,13 @@ protected LocalQueryRunner createLocalQueryRunner() queryRunner.createCatalog( CATALOG, - new TestingIcebergConnectorFactory(Optional.of(new TestingIcebergFileMetastoreCatalogModule(metastore)), Optional.empty(), EMPTY_MODULE), + new TestingIcebergConnectorFactory(metastoreDir.toPath()), ImmutableMap.of()); + HiveMetastore metastore = ((IcebergConnector) queryRunner.getConnector(CATALOG)).getInjector() + .getInstance(HiveMetastoreFactory.class) + .createMetastore(Optional.empty()); + Database database = Database.builder() .setDatabaseName(SCHEMA) .setOwnerName(Optional.of("public")) diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergRegisterTableProcedure.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergRegisterTableProcedure.java index 079f9af759d5d..a8550d28f94d8 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergRegisterTableProcedure.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergRegisterTableProcedure.java @@ -20,7 +20,10 @@ import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.plugin.hive.metastore.HiveMetastore; +import io.trino.plugin.iceberg.catalog.file.TestingIcebergFileMetastoreCatalogModule; +import io.trino.plugin.tpch.TpchPlugin; import io.trino.testing.AbstractTestQueryFramework; +import io.trino.testing.DistributedQueryRunner; import io.trino.testing.MaterializedResult; import io.trino.testing.QueryRunner; import org.apache.hadoop.conf.Configuration; @@ -43,18 +46,23 @@ import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; +import java.util.Optional; import java.util.regex.Matcher; import java.util.regex.Pattern; import static com.google.common.base.Verify.verify; import static com.google.common.io.MoreFiles.deleteRecursively; import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; +import static com.google.inject.util.Modules.EMPTY_MODULE; +import static io.trino.plugin.hive.HiveTestUtils.HDFS_FILE_SYSTEM_FACTORY; import static io.trino.plugin.hive.metastore.file.TestingFileHiveMetastore.createTestingFileHiveMetastore; +import static io.trino.plugin.iceberg.IcebergQueryRunner.ICEBERG_CATALOG; import static io.trino.plugin.iceberg.IcebergTestUtils.getFileSystemFactory; import static io.trino.plugin.iceberg.IcebergUtil.METADATA_FOLDER_NAME; import static io.trino.plugin.iceberg.procedure.RegisterTableProcedure.getLatestMetadataLocation; import static io.trino.testing.TestingConnectorSession.SESSION; import static io.trino.testing.TestingNames.randomNameSuffix; +import static io.trino.testing.TestingSession.testSessionBuilder; import static java.lang.String.format; import static java.util.Locale.ENGLISH; import static org.apache.iceberg.Files.localInput; @@ -73,11 +81,26 @@ protected QueryRunner createQueryRunner() { metastoreDir = Files.createTempDirectory("test_iceberg_register_table").toFile(); metastoreDir.deleteOnExit(); - metastore = createTestingFileHiveMetastore(metastoreDir); - return IcebergQueryRunner.builder() - .setMetastoreDirectory(metastoreDir) - .setIcebergProperties(ImmutableMap.of("iceberg.register-table-procedure.enabled", "true")) - .build(); + metastore = createTestingFileHiveMetastore(HDFS_FILE_SYSTEM_FACTORY, Location.of(metastoreDir.getAbsolutePath())); + + // TODO: convert to IcebergQueryRunner when there is a replacement for HadoopTables that works with TrinoFileSystem + DistributedQueryRunner queryRunner = DistributedQueryRunner.builder(testSessionBuilder() + .setCatalog(ICEBERG_CATALOG) + .setSchema("tpch") + .build()).build(); + + queryRunner.installPlugin(new TpchPlugin()); + queryRunner.createCatalog("tpch", "tpch"); + + Path dataDir = queryRunner.getCoordinator().getBaseDataDir().resolve("iceberg_data"); + queryRunner.installPlugin(new TestingIcebergPlugin( + dataDir, + Optional.of(new TestingIcebergFileMetastoreCatalogModule(metastore)), + Optional.empty(), + EMPTY_MODULE)); + queryRunner.createCatalog(ICEBERG_CATALOG, "iceberg", ImmutableMap.of("iceberg.register-table-procedure.enabled", "true")); + queryRunner.execute("CREATE SCHEMA iceberg.tpch"); + return queryRunner; } @BeforeAll diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergSplitSource.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergSplitSource.java index 0f54199c85e79..98504649f63ce 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergSplitSource.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergSplitSource.java @@ -21,6 +21,7 @@ import io.trino.plugin.base.CatalogName; import io.trino.plugin.hive.TrinoViewHiveMetastore; 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.plugin.hive.orc.OrcReaderConfig; import io.trino.plugin.hive.orc.OrcWriterConfig; @@ -69,7 +70,7 @@ import static com.google.common.io.MoreFiles.deleteRecursively; import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; import static io.trino.plugin.hive.metastore.cache.CachingHiveMetastore.createPerTransactionCache; -import static io.trino.plugin.hive.metastore.file.TestingFileHiveMetastore.createTestingFileHiveMetastore; +import static io.trino.plugin.iceberg.IcebergQueryRunner.ICEBERG_CATALOG; import static io.trino.plugin.iceberg.IcebergTestUtils.getFileSystemFactory; import static io.trino.spi.connector.Constraint.alwaysTrue; import static io.trino.spi.type.BigintType.BIGINT; @@ -102,13 +103,16 @@ protected QueryRunner createQueryRunner() { File tempDir = Files.createTempDirectory("test_iceberg_split_source").toFile(); this.metastoreDir = new File(tempDir, "iceberg_data"); - HiveMetastore metastore = createTestingFileHiveMetastore(metastoreDir); DistributedQueryRunner queryRunner = IcebergQueryRunner.builder() .setInitialTables(NATION) .setMetastoreDirectory(metastoreDir) .build(); + HiveMetastore metastore = ((IcebergConnector) queryRunner.getCoordinator().getConnector(ICEBERG_CATALOG)).getInjector() + .getInstance(HiveMetastoreFactory.class) + .createMetastore(Optional.empty()); + this.fileSystemFactory = getFileSystemFactory(queryRunner); CachingHiveMetastore cachingHiveMetastore = createPerTransactionCache(metastore, 1000); this.catalog = new TrinoHiveCatalog( diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergTableWithCustomLocation.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergTableWithCustomLocation.java index eb2cf82fe9e90..93786b701d8a8 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergTableWithCustomLocation.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergTableWithCustomLocation.java @@ -15,26 +15,22 @@ import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; +import io.trino.plugin.hive.metastore.HiveMetastore; +import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.metastore.Table; -import io.trino.plugin.hive.metastore.file.FileHiveMetastore; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.DistributedQueryRunner; import io.trino.testing.MaterializedResult; -import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; -import java.io.File; import java.io.IOException; -import java.nio.file.Files; import java.util.Map; import java.util.Optional; -import static com.google.common.io.MoreFiles.deleteRecursively; -import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; import static io.trino.plugin.hive.TableType.EXTERNAL_TABLE; -import static io.trino.plugin.hive.metastore.file.TestingFileHiveMetastore.createTestingFileHiveMetastore; import static io.trino.plugin.iceberg.DataFileRecord.toDataFileRecord; +import static io.trino.plugin.iceberg.IcebergQueryRunner.ICEBERG_CATALOG; import static io.trino.plugin.iceberg.IcebergTestUtils.getFileSystemFactory; import static io.trino.testing.TestingConnectorSession.SESSION; import static java.lang.String.format; @@ -43,21 +39,22 @@ public class TestIcebergTableWithCustomLocation extends AbstractTestQueryFramework { - private FileHiveMetastore metastore; - private File metastoreDir; + private HiveMetastore metastore; private TrinoFileSystem fileSystem; @Override protected DistributedQueryRunner createQueryRunner() throws Exception { - metastoreDir = Files.createTempDirectory("test_iceberg").toFile(); - metastore = createTestingFileHiveMetastore(metastoreDir); - - return IcebergQueryRunner.builder() + DistributedQueryRunner queryRunner = IcebergQueryRunner.builder() .setIcebergProperties(Map.of("iceberg.unique-table-location", "true")) - .setMetastoreDirectory(metastoreDir) .build(); + + metastore = ((IcebergConnector) queryRunner.getCoordinator().getConnector(ICEBERG_CATALOG)).getInjector() + .getInstance(HiveMetastoreFactory.class) + .createMetastore(Optional.empty()); + + return queryRunner; } @BeforeAll @@ -66,13 +63,6 @@ public void initFileSystem() fileSystem = getFileSystemFactory(getDistributedQueryRunner()).create(SESSION); } - @AfterAll - public void tearDown() - throws IOException - { - deleteRecursively(metastoreDir.toPath(), ALLOW_INSECURE); - } - @Test public void testTableHasUuidSuffixInLocation() { diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergTableWithExternalLocation.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergTableWithExternalLocation.java index a8d8a88cf42ba..1c297f3edac57 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergTableWithExternalLocation.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergTableWithExternalLocation.java @@ -15,25 +15,23 @@ import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; +import io.trino.plugin.hive.metastore.HiveMetastore; +import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.metastore.Table; -import io.trino.plugin.hive.metastore.file.FileHiveMetastore; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.DistributedQueryRunner; import io.trino.testing.MaterializedResult; -import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.TestInstance; import java.io.File; import java.io.IOException; -import java.nio.file.Files; +import java.util.Optional; -import static com.google.common.io.MoreFiles.deleteRecursively; -import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; import static io.trino.plugin.hive.TableType.EXTERNAL_TABLE; -import static io.trino.plugin.hive.metastore.file.TestingFileHiveMetastore.createTestingFileHiveMetastore; import static io.trino.plugin.iceberg.DataFileRecord.toDataFileRecord; +import static io.trino.plugin.iceberg.IcebergQueryRunner.ICEBERG_CATALOG; import static io.trino.plugin.iceberg.IcebergTestUtils.getFileSystemFactory; import static io.trino.testing.TestingConnectorSession.SESSION; import static io.trino.testing.TestingNames.randomNameSuffix; @@ -45,20 +43,21 @@ public class TestIcebergTableWithExternalLocation extends AbstractTestQueryFramework { - private FileHiveMetastore metastore; - private File metastoreDir; + private HiveMetastore metastore; private TrinoFileSystem fileSystem; @Override protected DistributedQueryRunner createQueryRunner() throws Exception { - metastoreDir = Files.createTempDirectory("test_iceberg").toFile(); - metastore = createTestingFileHiveMetastore(metastoreDir); - - return IcebergQueryRunner.builder() - .setMetastoreDirectory(metastoreDir) + DistributedQueryRunner queryRunner = IcebergQueryRunner.builder() .build(); + + metastore = ((IcebergConnector) queryRunner.getCoordinator().getConnector(ICEBERG_CATALOG)).getInjector() + .getInstance(HiveMetastoreFactory.class) + .createMetastore(Optional.empty()); + + return queryRunner; } @BeforeAll @@ -67,13 +66,6 @@ public void initFileSystem() fileSystem = getFileSystemFactory(getDistributedQueryRunner()).create(SESSION); } - @AfterAll - public void tearDown() - throws IOException - { - deleteRecursively(metastoreDir.toPath(), ALLOW_INSECURE); - } - @Test public void testCreateAndDrop() throws IOException diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergV2.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergV2.java index 89afa7897ee25..a7e9b8f42650a 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergV2.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergV2.java @@ -22,6 +22,7 @@ import io.trino.plugin.blackhole.BlackHolePlugin; import io.trino.plugin.hive.TrinoViewHiveMetastore; 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.plugin.iceberg.catalog.IcebergTableOperationsProvider; import io.trino.plugin.iceberg.catalog.TrinoCatalog; @@ -40,7 +41,6 @@ import io.trino.testing.DistributedQueryRunner; import io.trino.testing.QueryRunner; import io.trino.testing.sql.TestTable; -import org.apache.hadoop.fs.Path; import org.apache.iceberg.BaseTable; import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; @@ -62,16 +62,12 @@ import org.apache.iceberg.io.FileIO; import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.types.Types; -import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.TestInstance; import java.io.Closeable; -import java.io.File; -import java.io.IOException; import java.nio.ByteBuffer; -import java.nio.file.Files; import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -86,10 +82,8 @@ import static com.google.common.base.Verify.verify; import static com.google.common.collect.ImmutableList.toImmutableList; import static com.google.common.collect.Iterables.getOnlyElement; -import static com.google.common.io.MoreFiles.deleteRecursively; -import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; import static io.trino.plugin.hive.metastore.cache.CachingHiveMetastore.createPerTransactionCache; -import static io.trino.plugin.hive.metastore.file.TestingFileHiveMetastore.createTestingFileHiveMetastore; +import static io.trino.plugin.iceberg.IcebergQueryRunner.ICEBERG_CATALOG; import static io.trino.plugin.iceberg.IcebergTestUtils.getFileSystemFactory; import static io.trino.plugin.iceberg.IcebergUtil.loadIcebergTable; import static io.trino.spi.type.IntegerType.INTEGER; @@ -112,23 +106,20 @@ public class TestIcebergV2 extends AbstractTestQueryFramework { private HiveMetastore metastore; - private java.nio.file.Path tempDir; - private File metastoreDir; private TrinoFileSystemFactory fileSystemFactory; @Override protected QueryRunner createQueryRunner() throws Exception { - tempDir = Files.createTempDirectory("test_iceberg_v2"); - metastoreDir = tempDir.resolve("iceberg_data").toFile(); - metastore = createTestingFileHiveMetastore(metastoreDir); - DistributedQueryRunner queryRunner = IcebergQueryRunner.builder() .setInitialTables(NATION) - .setMetastoreDirectory(metastoreDir) .build(); + metastore = ((IcebergConnector) queryRunner.getCoordinator().getConnector(ICEBERG_CATALOG)).getInjector() + .getInstance(HiveMetastoreFactory.class) + .createMetastore(Optional.empty()); + try { queryRunner.installPlugin(new BlackHolePlugin()); queryRunner.createCatalog("blackhole", "blackhole"); @@ -147,13 +138,6 @@ public void initFileSystemFactory() fileSystemFactory = getFileSystemFactory(getDistributedQueryRunner()); } - @AfterAll - public void tearDown() - throws IOException - { - deleteRecursively(tempDir, ALLOW_INSECURE); - } - @Test public void testSettingFormatVersion() { @@ -195,12 +179,9 @@ public void testV2TableWithPositionDelete() String dataFilePath = (String) computeActual("SELECT file_path FROM \"" + tableName + "$files\" LIMIT 1").getOnlyValue(); - Path metadataDir = new Path(metastoreDir.toURI()); - String deleteFileName = "delete_file_" + UUID.randomUUID(); FileIO fileIo = new ForwardingFileIo(fileSystemFactory.create(SESSION)); - Path path = new Path(metadataDir, deleteFileName); - PositionDeleteWriter writer = Parquet.writeDeletes(fileIo.newOutputFile(path.toString())) + PositionDeleteWriter writer = Parquet.writeDeletes(fileIo.newOutputFile("local:///delete_file_" + UUID.randomUUID())) .createWriterFunc(GenericParquetWriter::buildWriter) .forTable(icebergTable) .overwrite() @@ -761,7 +742,7 @@ public void testFilesTable() throws Exception { String tableName = "test_files_table_" + randomNameSuffix(); - String tableLocation = metastoreDir.getPath() + "/" + tableName; + String tableLocation = "local:///" + tableName; assertUpdate("CREATE TABLE " + tableName + " WITH (location = '" + tableLocation + "', format_version = 2) AS SELECT * FROM tpch.tiny.nation", 25); BaseTable table = loadTable(tableName); Metrics metrics = new Metrics( @@ -986,11 +967,9 @@ private void writeEqualityDeleteToNationTableWithDeleteColumns( List equalityDeleteFieldIds) throws Exception { - Path metadataDir = new Path(metastoreDir.toURI()); - String deleteFileName = "delete_file_" + UUID.randomUUID(); FileIO fileIo = new ForwardingFileIo(fileSystemFactory.create(SESSION)); - Parquet.DeleteWriteBuilder writerBuilder = Parquet.writeDeletes(fileIo.newOutputFile(new Path(metadataDir, deleteFileName).toString())) + Parquet.DeleteWriteBuilder writerBuilder = Parquet.writeDeletes(fileIo.newOutputFile("local:///delete_file_" + UUID.randomUUID())) .forTable(icebergTable) .rowSchema(deleteRowSchema) .createWriterFunc(GenericParquetWriter::buildWriter) diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestMetadataQueryOptimization.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestMetadataQueryOptimization.java index 6f1365a367904..cd10f096b7759 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestMetadataQueryOptimization.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestMetadataQueryOptimization.java @@ -19,7 +19,7 @@ import io.trino.metadata.InternalFunctionBundle; import io.trino.plugin.hive.metastore.Database; import io.trino.plugin.hive.metastore.HiveMetastore; -import io.trino.plugin.iceberg.catalog.file.TestingIcebergFileMetastoreCatalogModule; +import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.spi.security.PrincipalType; import io.trino.sql.planner.assertions.BasePushdownPlanTest; import io.trino.sql.tree.LongLiteral; @@ -35,9 +35,7 @@ import static com.google.common.io.MoreFiles.deleteRecursively; import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; -import static com.google.inject.util.Modules.EMPTY_MODULE; import static io.trino.SystemSessionProperties.TASK_MAX_WRITER_COUNT; -import static io.trino.plugin.hive.metastore.file.TestingFileHiveMetastore.createTestingFileHiveMetastore; import static io.trino.sql.planner.assertions.PlanMatchPattern.anyTree; import static io.trino.sql.planner.assertions.PlanMatchPattern.values; import static io.trino.testing.TestingSession.testSessionBuilder; @@ -66,7 +64,6 @@ protected LocalQueryRunner createLocalQueryRunner() catch (IOException e) { throw new UncheckedIOException(e); } - HiveMetastore metastore = createTestingFileHiveMetastore(baseDir); LocalQueryRunner queryRunner = LocalQueryRunner.create(session); InternalFunctionBundle.InternalFunctionBundleBuilder functions = InternalFunctionBundle.builder(); @@ -75,9 +72,13 @@ protected LocalQueryRunner createLocalQueryRunner() queryRunner.createCatalog( ICEBERG_CATALOG, - new TestingIcebergConnectorFactory(Optional.of(new TestingIcebergFileMetastoreCatalogModule(metastore)), Optional.empty(), EMPTY_MODULE), + new TestingIcebergConnectorFactory(baseDir.toPath()), ImmutableMap.of()); + HiveMetastore metastore = ((IcebergConnector) queryRunner.getConnector(ICEBERG_CATALOG)).getInjector() + .getInstance(HiveMetastoreFactory.class) + .createMetastore(Optional.empty()); + Database database = Database.builder() .setDatabaseName(SCHEMA_NAME) .setOwnerName(Optional.of("public")) diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestSharedHiveMetastore.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestSharedHiveMetastore.java index 130edc81cd825..73c16750d31da 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestSharedHiveMetastore.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestSharedHiveMetastore.java @@ -27,7 +27,6 @@ import java.nio.file.Path; -import static io.trino.plugin.hive.metastore.file.TestingFileHiveMetastore.createTestingFileHiveMetastore; import static io.trino.plugin.iceberg.IcebergQueryRunner.ICEBERG_CATALOG; import static io.trino.plugin.tpch.TpchMetadata.TINY_SCHEMA_NAME; import static io.trino.testing.QueryAssertions.copyTpchTables; @@ -80,7 +79,7 @@ protected QueryRunner createQueryRunner() "hive.metastore.catalog.dir", dataDirectory.toString(), "iceberg.hive-catalog-name", "hive")); - queryRunner.installPlugin(new TestingHivePlugin(createTestingFileHiveMetastore(dataDirectory.toFile()))); + queryRunner.installPlugin(new TestingHivePlugin(dataDirectory)); queryRunner.createCatalog(HIVE_CATALOG, "hive", ImmutableMap.of("hive.allow-drop-table", "true")); queryRunner.createCatalog( "hive_with_redirections", @@ -105,12 +104,12 @@ public void cleanup() @Override protected String getExpectedHiveCreateSchema(String catalogName) { - String expectedHiveCreateSchema = "CREATE SCHEMA %s.%s\n" + - "WITH (\n" + - " location = 'file:%s/%s'\n" + - ")"; - - return format(expectedHiveCreateSchema, catalogName, schema, dataDirectory, schema); + return """ + CREATE SCHEMA %s.%s + WITH ( + location = 'local:///%s' + )""" + .formatted(catalogName, schema, schema); } @Override diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestingIcebergConnectorFactory.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestingIcebergConnectorFactory.java index fbc412b548c2e..3e61a6e831a92 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestingIcebergConnectorFactory.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestingIcebergConnectorFactory.java @@ -13,15 +13,22 @@ */ package io.trino.plugin.iceberg; +import com.google.common.collect.ImmutableMap; import com.google.inject.Module; import io.trino.filesystem.TrinoFileSystemFactory; +import io.trino.filesystem.local.LocalFileSystemFactory; +import io.trino.plugin.hive.metastore.file.FileHiveMetastoreConfig; import io.trino.spi.connector.Connector; import io.trino.spi.connector.ConnectorContext; import io.trino.spi.connector.ConnectorFactory; +import java.nio.file.Path; import java.util.Map; import java.util.Optional; +import static com.google.inject.multibindings.MapBinder.newMapBinder; +import static com.google.inject.util.Modules.EMPTY_MODULE; +import static io.airlift.configuration.ConfigBinder.configBinder; import static io.trino.plugin.iceberg.InternalIcebergConnectorFactory.createConnector; import static java.util.Objects.requireNonNull; @@ -32,11 +39,27 @@ public class TestingIcebergConnectorFactory private final Optional fileSystemFactory; private final Module module; - public TestingIcebergConnectorFactory(Optional icebergCatalogModule, Optional fileSystemFactory, Module module) + public TestingIcebergConnectorFactory(Path localFileSystemRootPath) { + this(localFileSystemRootPath, Optional.empty(), Optional.empty(), EMPTY_MODULE); + } + + @Deprecated + public TestingIcebergConnectorFactory( + Path localFileSystemRootPath, + Optional icebergCatalogModule, + Optional fileSystemFactory, + Module module) + { + localFileSystemRootPath.toFile().mkdirs(); this.icebergCatalogModule = requireNonNull(icebergCatalogModule, "icebergCatalogModule is null"); this.fileSystemFactory = requireNonNull(fileSystemFactory, "fileSystemFactory is null"); - this.module = requireNonNull(module, "module is null"); + this.module = binder -> { + binder.install(module); + newMapBinder(binder, String.class, TrinoFileSystemFactory.class) + .addBinding("local").toInstance(new LocalFileSystemFactory(localFileSystemRootPath)); + configBinder(binder).bindConfigDefaults(FileHiveMetastoreConfig.class, config -> config.setCatalogDirectory("local:///")); + }; } @Override @@ -48,6 +71,12 @@ public String getName() @Override public Connector create(String catalogName, Map config, ConnectorContext context) { + if (!config.containsKey("iceberg.catalog.type")) { + config = ImmutableMap.builder() + .putAll(config) + .put("iceberg.catalog.type", "TESTING_FILE_METASTORE") + .buildOrThrow(); + } return createConnector(catalogName, config, context, module, icebergCatalogModule, fileSystemFactory); } } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestingIcebergPlugin.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestingIcebergPlugin.java index 7ed7cb4c18c81..3ea6e645d32d3 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestingIcebergPlugin.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestingIcebergPlugin.java @@ -18,21 +18,31 @@ import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.spi.connector.ConnectorFactory; +import java.nio.file.Path; import java.util.List; import java.util.Optional; import static com.google.common.base.Verify.verify; +import static com.google.inject.util.Modules.EMPTY_MODULE; import static java.util.Objects.requireNonNull; public class TestingIcebergPlugin extends IcebergPlugin { + private final Path localFileSystemRootPath; private final Optional icebergCatalogModule; private final Optional fileSystemFactory; private final Module module; - public TestingIcebergPlugin(Optional icebergCatalogModule, Optional fileSystemFactory, Module module) + public TestingIcebergPlugin(Path localFileSystemRootPath) { + this(localFileSystemRootPath, Optional.empty(), Optional.empty(), EMPTY_MODULE); + } + + @Deprecated + public TestingIcebergPlugin(Path localFileSystemRootPath, Optional icebergCatalogModule, Optional fileSystemFactory, Module module) + { + this.localFileSystemRootPath = requireNonNull(localFileSystemRootPath, "localFileSystemRootPath is null"); this.icebergCatalogModule = requireNonNull(icebergCatalogModule, "icebergCatalogModule is null"); this.fileSystemFactory = requireNonNull(fileSystemFactory, "fileSystemFactory is null"); this.module = requireNonNull(module, "module is null"); @@ -44,6 +54,6 @@ public Iterable getConnectorFactories() List connectorFactories = ImmutableList.copyOf(super.getConnectorFactories()); verify(connectorFactories.size() == 1, "Unexpected connector factories: %s", connectorFactories); - return ImmutableList.of(new TestingIcebergConnectorFactory(icebergCatalogModule, fileSystemFactory, module)); + return ImmutableList.of(new TestingIcebergConnectorFactory(localFileSystemRootPath, icebergCatalogModule, fileSystemFactory, module)); } } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/BaseTrinoCatalogTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/BaseTrinoCatalogTest.java index 36b15e3942180..cd896dcd9895f 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/BaseTrinoCatalogTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/BaseTrinoCatalogTest.java @@ -39,12 +39,12 @@ import org.junit.jupiter.api.Test; import java.io.IOException; -import java.io.UncheckedIOException; import java.nio.file.Files; import java.nio.file.Path; import java.util.HashMap; import java.util.Map; import java.util.Optional; +import java.util.UUID; import static io.airlift.json.JsonCodec.jsonCodec; import static io.trino.plugin.hive.HiveErrorCode.HIVE_DATABASE_LOCATION_ERROR; @@ -73,7 +73,7 @@ public void testCreateNamespaceWithLocation() TrinoCatalog catalog = createTrinoCatalog(false); String namespace = "test_create_namespace_with_location_" + randomNameSuffix(); Map namespaceProperties = new HashMap<>(defaultNamespaceProperties(namespace)); - String namespaceLocation = (String) namespaceProperties.computeIfAbsent(LOCATION_PROPERTY, ignored -> "/a/path/"); + String namespaceLocation = (String) namespaceProperties.computeIfAbsent(LOCATION_PROPERTY, ignored -> "local:///a/path/"); namespaceProperties = ImmutableMap.copyOf(namespaceProperties); catalog.createNamespace(SESSION, namespace, namespaceProperties, new TrinoPrincipal(PrincipalType.USER, SESSION.getUser())); assertThat(catalog.listNamespaces(SESSION)).contains(namespace); @@ -300,16 +300,9 @@ public void testUseUniqueTableLocations() String table = "tableName"; SchemaTableName schemaTableName = new SchemaTableName(namespace, table); Map namespaceProperties = new HashMap<>(defaultNamespaceProperties(namespace)); - String namespaceLocation = (String) namespaceProperties.computeIfAbsent(LOCATION_PROPERTY, ignored -> { - try { - Path tmpDirectory = Files.createTempDirectory("iceberg_catalog_test_rename_table_"); - tmpDirectory.toFile().deleteOnExit(); - return tmpDirectory.toString(); - } - catch (IOException e) { - throw new UncheckedIOException(e); - } - }); + String namespaceLocation = (String) namespaceProperties.computeIfAbsent( + LOCATION_PROPERTY, + ignored -> "local:///iceberg_catalog_test_rename_table_" + UUID.randomUUID()); catalog.createNamespace(SESSION, namespace, namespaceProperties, new TrinoPrincipal(PrincipalType.USER, SESSION.getUser())); try { diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestTrinoHiveCatalogWithFileMetastore.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestTrinoHiveCatalogWithFileMetastore.java index 2798d5ce6d0fb..474fd270a7bca 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestTrinoHiveCatalogWithFileMetastore.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestTrinoHiveCatalogWithFileMetastore.java @@ -13,7 +13,9 @@ */ package io.trino.plugin.iceberg.catalog.file; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystemFactory; +import io.trino.filesystem.local.LocalFileSystemFactory; import io.trino.plugin.base.CatalogName; import io.trino.plugin.hive.TrinoViewHiveMetastore; import io.trino.plugin.hive.metastore.HiveMetastore; @@ -35,7 +37,6 @@ import static com.google.common.io.MoreFiles.deleteRecursively; import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; -import static io.trino.plugin.hive.HiveTestUtils.HDFS_FILE_SYSTEM_FACTORY; import static io.trino.plugin.hive.metastore.cache.CachingHiveMetastore.createPerTransactionCache; import static io.trino.plugin.hive.metastore.file.TestingFileHiveMetastore.createTestingFileHiveMetastore; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; @@ -46,8 +47,9 @@ public class TestTrinoHiveCatalogWithFileMetastore extends BaseTrinoCatalogTest { - private HiveMetastore metastore; private Path tempDir; + private TrinoFileSystemFactory fileSystemFactory; + private HiveMetastore metastore; @BeforeAll public void setUp() @@ -55,7 +57,9 @@ public void setUp() { tempDir = Files.createTempDirectory("test_trino_hive_catalog"); File metastoreDir = tempDir.resolve("iceberg_data").toFile(); - metastore = createTestingFileHiveMetastore(metastoreDir); + metastoreDir.mkdirs(); + fileSystemFactory = new LocalFileSystemFactory(metastoreDir.toPath()); + metastore = createTestingFileHiveMetastore(fileSystemFactory, Location.of("local:///")); } @AfterAll @@ -68,7 +72,6 @@ public void tearDown() @Override protected TrinoCatalog createTrinoCatalog(boolean useUniqueTableLocations) { - TrinoFileSystemFactory fileSystemFactory = HDFS_FILE_SYSTEM_FACTORY; CachingHiveMetastore cachingHiveMetastore = createPerTransactionCache(metastore, 1000); return new TrinoHiveCatalog( new CatalogName("catalog"), diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogAccessOperations.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogAccessOperations.java index 317441595a764..ce44f319d69f0 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogAccessOperations.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogAccessOperations.java @@ -107,6 +107,7 @@ protected QueryRunner createQueryRunner() trackingFileSystemFactory = new TrackingFileSystemFactory(new HdfsFileSystemFactory(HDFS_ENVIRONMENT, HDFS_FILE_SYSTEM_STATS)); queryRunner.installPlugin(new TestingIcebergPlugin( + tmp.toPath(), Optional.empty(), Optional.of(trackingFileSystemFactory), EMPTY_MODULE)); diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueTableOperationsInsertFailure.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueTableOperationsInsertFailure.java index cc358d58da676..10a3447da4354 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueTableOperationsInsertFailure.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueTableOperationsInsertFailure.java @@ -89,14 +89,14 @@ protected LocalQueryRunner createQueryRunner() new IcebergPlugin().getFunctions().forEach(functions::functions); queryRunner.addFunctions(functions.build()); + Path dataDirectory = Files.createTempDirectory("iceberg_data"); + dataDirectory.toFile().deleteOnExit(); + queryRunner.createCatalog( ICEBERG_CATALOG, - new TestingIcebergConnectorFactory(Optional.of(new TestingIcebergGlueCatalogModule(awsGlueAsyncAdapterProvider)), Optional.empty(), EMPTY_MODULE), + new TestingIcebergConnectorFactory(dataDirectory, Optional.of(new TestingIcebergGlueCatalogModule(awsGlueAsyncAdapterProvider)), Optional.empty(), EMPTY_MODULE), ImmutableMap.of()); - Path dataDirectory = Files.createTempDirectory("iceberg_data"); - dataDirectory.toFile().deleteOnExit(); - glueHiveMetastore = createTestingGlueHiveMetastore(dataDirectory); Database database = Database.builder() diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestSharedGlueMetastore.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestSharedGlueMetastore.java index 8b18b345f2e34..a82525f3b53e6 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestSharedGlueMetastore.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestSharedGlueMetastore.java @@ -94,7 +94,7 @@ protected QueryRunner createQueryRunner() "iceberg.hive-catalog-name", "hive")); this.glueMetastore = createTestingGlueHiveMetastore(dataDirectory); - queryRunner.installPlugin(new TestingHivePlugin(glueMetastore)); + queryRunner.installPlugin(new TestingHivePlugin(queryRunner.getCoordinator().getBaseDataDir().resolve("hive_data"), glueMetastore)); queryRunner.createCatalog(HIVE_CATALOG, "hive"); queryRunner.createCatalog( "hive_with_redirections", diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/hms/TestIcebergHiveMetastoreTableOperationsReleaseLockFailure.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/hms/TestIcebergHiveMetastoreTableOperationsReleaseLockFailure.java index 0ee57e0c3f1ff..34891e418eb9c 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/hms/TestIcebergHiveMetastoreTableOperationsReleaseLockFailure.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/hms/TestIcebergHiveMetastoreTableOperationsReleaseLockFailure.java @@ -72,7 +72,7 @@ protected LocalQueryRunner createQueryRunner() queryRunner.createCatalog( ICEBERG_CATALOG, - new TestingIcebergConnectorFactory(Optional.of(testModule), Optional.empty(), EMPTY_MODULE), + new TestingIcebergConnectorFactory(baseDir.toPath(), Optional.of(testModule), Optional.empty(), EMPTY_MODULE), ImmutableMap.of()); Database database = Database.builder() diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/optimizer/TestConnectorPushdownRulesWithIceberg.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/optimizer/TestConnectorPushdownRulesWithIceberg.java index 900c649508e63..d2f22f1591503 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/optimizer/TestConnectorPushdownRulesWithIceberg.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/optimizer/TestConnectorPushdownRulesWithIceberg.java @@ -23,12 +23,13 @@ import io.trino.plugin.hive.HiveTransactionHandle; import io.trino.plugin.hive.metastore.Database; import io.trino.plugin.hive.metastore.HiveMetastore; +import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.iceberg.ColumnIdentity; import io.trino.plugin.iceberg.IcebergColumnHandle; +import io.trino.plugin.iceberg.IcebergConnector; import io.trino.plugin.iceberg.IcebergPlugin; import io.trino.plugin.iceberg.IcebergTableHandle; import io.trino.plugin.iceberg.TestingIcebergConnectorFactory; -import io.trino.plugin.iceberg.catalog.file.TestingIcebergFileMetastoreCatalogModule; import io.trino.spi.connector.CatalogHandle; import io.trino.spi.predicate.Domain; import io.trino.spi.predicate.TupleDomain; @@ -61,8 +62,6 @@ import static com.google.common.io.MoreFiles.deleteRecursively; import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; -import static com.google.inject.util.Modules.EMPTY_MODULE; -import static io.trino.plugin.hive.metastore.file.TestingFileHiveMetastore.createTestingFileHiveMetastore; import static io.trino.plugin.iceberg.ColumnIdentity.TypeCategory.STRUCT; import static io.trino.plugin.iceberg.ColumnIdentity.primitiveColumnIdentity; import static io.trino.plugin.iceberg.TableType.DATA; @@ -106,16 +105,6 @@ protected Optional createLocalQueryRunner() catch (IOException e) { throw new UncheckedIOException(e); } - metastore = createTestingFileHiveMetastore(baseDir); - Database database = Database.builder() - .setDatabaseName(SCHEMA_NAME) - .setOwnerName(Optional.of("public")) - .setOwnerType(Optional.of(PrincipalType.ROLE)) - .build(); - - metastore.createDatabase(database); - - HiveMetastore metastore = createTestingFileHiveMetastore(baseDir); LocalQueryRunner queryRunner = LocalQueryRunner.create(ICEBERG_SESSION); InternalFunctionBundle.InternalFunctionBundleBuilder functions = InternalFunctionBundle.builder(); @@ -124,9 +113,21 @@ protected Optional createLocalQueryRunner() queryRunner.createCatalog( TEST_CATALOG_NAME, - new TestingIcebergConnectorFactory(Optional.of(new TestingIcebergFileMetastoreCatalogModule(metastore)), Optional.empty(), EMPTY_MODULE), + new TestingIcebergConnectorFactory(baseDir.toPath()), ImmutableMap.of()); catalogHandle = queryRunner.getCatalogHandle(TEST_CATALOG_NAME); + + metastore = ((IcebergConnector) queryRunner.getConnector(TEST_CATALOG_NAME)).getInjector() + .getInstance(HiveMetastoreFactory.class) + .createMetastore(Optional.empty()); + Database database = Database.builder() + .setDatabaseName(SCHEMA_NAME) + .setOwnerName(Optional.of("public")) + .setOwnerType(Optional.of(PrincipalType.ROLE)) + .build(); + + metastore.createDatabase(database); + return Optional.of(queryRunner); } diff --git a/testing/trino-faulttolerant-tests/src/test/java/io/trino/faulttolerant/iceberg/TestIcebergParquetFaultTolerantExecutionConnectorTest.java b/testing/trino-faulttolerant-tests/src/test/java/io/trino/faulttolerant/iceberg/TestIcebergParquetFaultTolerantExecutionConnectorTest.java index 32d6c5d0a2db8..d46ecfe22a61e 100644 --- a/testing/trino-faulttolerant-tests/src/test/java/io/trino/faulttolerant/iceberg/TestIcebergParquetFaultTolerantExecutionConnectorTest.java +++ b/testing/trino-faulttolerant-tests/src/test/java/io/trino/faulttolerant/iceberg/TestIcebergParquetFaultTolerantExecutionConnectorTest.java @@ -13,6 +13,7 @@ */ package io.trino.faulttolerant.iceberg; +import io.trino.filesystem.Location; import io.trino.plugin.exchange.filesystem.FileSystemExchangePlugin; import io.trino.plugin.exchange.filesystem.containers.MinioStorage; import io.trino.plugin.iceberg.IcebergQueryRunner; @@ -78,7 +79,7 @@ public void testStatsBasedRepartitionDataOnInsert() @Override protected boolean isFileSorted(String path, String sortColumnName) { - return checkParquetFileSorting(path, sortColumnName); + return checkParquetFileSorting(fileSystem.newInputFile(Location.of(path)), sortColumnName); } @AfterAll From 72cdf4c6f549eac422ad90e7f4f1f4235f4031b0 Mon Sep 17 00:00:00 2001 From: Dain Sundstrom Date: Tue, 28 Nov 2023 13:14:48 -0800 Subject: [PATCH 012/350] Remove InMemoryThriftMetastore --- .../metastore/thrift/ThriftHiveMetastore.java | 2 +- .../metastore/thrift/ThriftMetastore.java | 3 +- .../hive/TestHiveInMemoryMetastore.java | 104 --- .../thrift/InMemoryThriftMetastore.java | 743 ------------------ ...toreTableOperationsReleaseLockFailure.java | 140 ---- 5 files changed, 3 insertions(+), 989 deletions(-) delete mode 100644 plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveInMemoryMetastore.java delete mode 100644 plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/InMemoryThriftMetastore.java delete mode 100644 plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/hms/TestIcebergHiveMetastoreTableOperationsReleaseLockFailure.java diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHiveMetastore.java index 0ae82fd221828..7ae93dc4e1ab3 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHiveMetastore.java @@ -144,7 +144,7 @@ import static java.util.Objects.requireNonNull; @ThreadSafe -public class ThriftHiveMetastore +public final class ThriftHiveMetastore implements ThriftMetastore { private static final Logger log = Logger.get(ThriftHiveMetastore.class); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastore.java index 03043b64effae..4d8456d1ef62f 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftMetastore.java @@ -46,7 +46,8 @@ import static io.trino.plugin.hive.HiveErrorCode.HIVE_INVALID_METADATA; import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; -public interface ThriftMetastore +public sealed interface ThriftMetastore + permits ThriftHiveMetastore { void createDatabase(Database database); diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveInMemoryMetastore.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveInMemoryMetastore.java deleted file mode 100644 index 16617349ceb59..0000000000000 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveInMemoryMetastore.java +++ /dev/null @@ -1,104 +0,0 @@ -/* - * 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.hive; - -import io.trino.plugin.hive.metastore.HiveMetastore; -import io.trino.plugin.hive.metastore.Table; -import io.trino.plugin.hive.metastore.thrift.BridgingHiveMetastore; -import io.trino.plugin.hive.metastore.thrift.InMemoryThriftMetastore; -import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreConfig; -import org.junit.jupiter.api.Test; - -import java.io.File; -import java.net.URI; - -import static java.nio.file.Files.createDirectories; -import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.junit.jupiter.api.Assumptions.abort; - -public class TestHiveInMemoryMetastore - extends AbstractTestHiveLocal -{ - @Override - protected HiveMetastore createMetastore(File tempDir) - { - File baseDir = new File(tempDir, "metastore"); - ThriftMetastoreConfig metastoreConfig = new ThriftMetastoreConfig(); - InMemoryThriftMetastore hiveMetastore = new InMemoryThriftMetastore(baseDir, metastoreConfig); - return new BridgingHiveMetastore(hiveMetastore); - } - - @Override - protected void createTestTable(Table table) - throws Exception - { - createDirectories(new File(URI.create(table.getStorage().getLocation())).toPath()); - super.createTestTable(table); - } - - @Test - @Override - public void testMetadataDelete() - { - // InMemoryHiveMetastore ignores "removeData" flag in dropPartition - } - - @Test - @Override - public void testTransactionDeleteInsert() - { - // InMemoryHiveMetastore does not check whether partition exist in createPartition and dropPartition - } - - @Test - @Override - public void testHideDeltaLakeTables() - { - abort("not supported"); - } - - @Test - @Override - public void testDisallowQueryingOfIcebergTables() - { - abort("not supported"); - } - - @Test - @Override - public void testDataColumnProperties() - { - // Column properties are currently not supported in ThriftHiveMetastore - assertThatThrownBy(super::testDataColumnProperties) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Persisting column properties is not supported: Column{name=id, type=bigint}"); - } - - @Test - @Override - public void testPartitionColumnProperties() - { - // Column properties are currently not supported in ThriftHiveMetastore - assertThatThrownBy(super::testPartitionColumnProperties) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Persisting column properties is not supported: Column{name=part_key, type=varchar(256)}"); - } - - @Test - @Override - public void testPartitionSchemaMismatch() - { - abort("not supported"); - } -} diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/InMemoryThriftMetastore.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/InMemoryThriftMetastore.java deleted file mode 100644 index e4a2b5ee09f10..0000000000000 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/InMemoryThriftMetastore.java +++ /dev/null @@ -1,743 +0,0 @@ -/* - * 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.hive.metastore.thrift; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; -import com.google.errorprone.annotations.concurrent.GuardedBy; -import io.trino.hive.thrift.metastore.Database; -import io.trino.hive.thrift.metastore.FieldSchema; -import io.trino.hive.thrift.metastore.Partition; -import io.trino.hive.thrift.metastore.PrincipalPrivilegeSet; -import io.trino.hive.thrift.metastore.PrincipalType; -import io.trino.hive.thrift.metastore.Table; -import io.trino.plugin.hive.HiveColumnStatisticType; -import io.trino.plugin.hive.PartitionStatistics; -import io.trino.plugin.hive.SchemaAlreadyExistsException; -import io.trino.plugin.hive.TableAlreadyExistsException; -import io.trino.plugin.hive.TableType; -import io.trino.plugin.hive.acid.AcidTransaction; -import io.trino.plugin.hive.metastore.HivePrincipal; -import io.trino.plugin.hive.metastore.HivePrivilegeInfo; -import io.trino.plugin.hive.metastore.HivePrivilegeInfo.HivePrivilege; -import io.trino.plugin.hive.metastore.PartitionWithStatistics; -import io.trino.spi.TrinoException; -import io.trino.spi.connector.SchemaNotFoundException; -import io.trino.spi.connector.SchemaTableName; -import io.trino.spi.connector.TableNotFoundException; -import io.trino.spi.predicate.TupleDomain; -import io.trino.spi.security.RoleGrant; -import io.trino.spi.type.Type; -import org.apache.hadoop.fs.Path; - -import java.io.File; -import java.io.IOException; -import java.io.UncheckedIOException; -import java.net.URI; -import java.util.Collection; -import java.util.EnumSet; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Optional; -import java.util.Set; -import java.util.function.Function; - -import static com.google.common.base.MoreObjects.toStringHelper; -import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.collect.ImmutableList.toImmutableList; -import static com.google.common.collect.ImmutableSet.toImmutableSet; -import static com.google.common.io.MoreFiles.deleteRecursively; -import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; -import static io.trino.plugin.hive.HiveBasicStatistics.createEmptyStatistics; -import static io.trino.plugin.hive.TableType.EXTERNAL_TABLE; -import static io.trino.plugin.hive.TableType.MANAGED_TABLE; -import static io.trino.plugin.hive.TableType.VIRTUAL_VIEW; -import static io.trino.plugin.hive.metastore.MetastoreUtil.partitionKeyFilterToStringList; -import static io.trino.plugin.hive.metastore.thrift.ThriftMetastoreUtil.toMetastoreApiPartition; -import static io.trino.plugin.hive.util.HiveUtil.toPartitionValues; -import static io.trino.spi.StandardErrorCode.SCHEMA_NOT_EMPTY; -import static java.util.Locale.US; -import static java.util.Objects.requireNonNull; -import static org.apache.hadoop.hive.common.FileUtils.makePartName; - -public class InMemoryThriftMetastore - implements ThriftMetastore -{ - @GuardedBy("this") - private final Map databases = new HashMap<>(); - @GuardedBy("this") - private final Map relations = new HashMap<>(); - @GuardedBy("this") - private final Map views = new HashMap<>(); - @GuardedBy("this") - private final Map partitions = new HashMap<>(); - @GuardedBy("this") - private final Map columnStatistics = new HashMap<>(); - @GuardedBy("this") - private final Map partitionColumnStatistics = new HashMap<>(); - @GuardedBy("this") - private final Map> tablePrivileges = new HashMap<>(); - - private final File baseDirectory; - private final boolean assumeCanonicalPartitionKeys; - - public InMemoryThriftMetastore(File baseDirectory, ThriftMetastoreConfig metastoreConfig) - { - this.baseDirectory = requireNonNull(baseDirectory, "baseDirectory is null"); - this.assumeCanonicalPartitionKeys = requireNonNull(metastoreConfig).isAssumeCanonicalPartitionKeys(); - checkArgument(!baseDirectory.exists(), "Base directory already exists"); - checkArgument(baseDirectory.mkdirs(), "Could not create base directory"); - } - - @Override - public synchronized void createDatabase(Database database) - { - requireNonNull(database, "database is null"); - - File directory; - if (database.getLocationUri() != null) { - directory = new File(URI.create(database.getLocationUri())); - } - else { - // use Hive default naming convention - directory = new File(baseDirectory, database.getName() + ".db"); - database = database.deepCopy(); - database.setLocationUri(directory.toURI().toString()); - } - - checkArgument(!directory.exists(), "Database directory already exists"); - checkArgument(isParentDir(directory, baseDirectory), "Database directory must be inside of the metastore base directory"); - checkArgument(directory.mkdirs(), "Could not create database directory"); - - if (databases.putIfAbsent(database.getName(), database) != null) { - throw new SchemaAlreadyExistsException(database.getName()); - } - } - - // TODO: respect deleteData - @Override - public synchronized void dropDatabase(String databaseName, boolean deleteData) - { - if (!databases.containsKey(databaseName)) { - throw new SchemaNotFoundException(databaseName); - } - if (!getAllTables(databaseName).isEmpty()) { - throw new TrinoException(SCHEMA_NOT_EMPTY, "Schema not empty: " + databaseName); - } - databases.remove(databaseName); - } - - @Override - public synchronized void alterDatabase(String databaseName, Database newDatabase) - { - String newDatabaseName = newDatabase.getName(); - - if (databaseName.equals(newDatabaseName)) { - if (databases.replace(databaseName, newDatabase) == null) { - throw new SchemaNotFoundException(databaseName); - } - return; - } - - Database database = databases.get(databaseName); - if (database == null) { - throw new SchemaNotFoundException(databaseName); - } - if (databases.putIfAbsent(newDatabaseName, database) != null) { - throw new SchemaAlreadyExistsException(newDatabaseName); - } - databases.remove(databaseName); - - rewriteKeys(relations, name -> new SchemaTableName(newDatabaseName, name.getTableName())); - rewriteKeys(views, name -> new SchemaTableName(newDatabaseName, name.getTableName())); - rewriteKeys(partitions, name -> name.withSchemaName(newDatabaseName)); - rewriteKeys(tablePrivileges, name -> name.withDatabase(newDatabaseName)); - } - - @Override - public synchronized List getAllDatabases() - { - return ImmutableList.copyOf(databases.keySet()); - } - - @Override - public synchronized void createTable(Table table) - { - TableType tableType = TableType.valueOf(table.getTableType()); - checkArgument(EnumSet.of(MANAGED_TABLE, EXTERNAL_TABLE, VIRTUAL_VIEW).contains(tableType), "Invalid table type: %s", tableType); - - if (tableType == VIRTUAL_VIEW) { - checkArgument(table.getSd().getLocation() == null, "Storage location for view must be null"); - } - else { - File directory = new File(new Path(table.getSd().getLocation()).toUri()); - checkArgument(directory.exists(), "Table directory [%s] does not exist", directory); - if (tableType == MANAGED_TABLE) { - checkArgument(isParentDir(directory, baseDirectory), "Table directory must be inside of the metastore base directory"); - } - } - - SchemaTableName schemaTableName = new SchemaTableName(table.getDbName(), table.getTableName()); - Table tableCopy = table.deepCopy(); - - if (relations.putIfAbsent(schemaTableName, tableCopy) != null) { - throw new TableAlreadyExistsException(schemaTableName); - } - - if (tableType == VIRTUAL_VIEW) { - views.put(schemaTableName, tableCopy); - } - - PrincipalPrivilegeSet privileges = table.getPrivileges(); - if (privileges != null && (!privileges.getUserPrivileges().isEmpty() || !privileges.getGroupPrivileges().isEmpty() || !privileges.getRolePrivileges().isEmpty())) { - throw new UnsupportedOperationException(); - } - } - - @Override - public synchronized void dropTable(String databaseName, String tableName, boolean deleteData) - { - List locations = listAllDataPaths(this, databaseName, tableName); - - SchemaTableName schemaTableName = new SchemaTableName(databaseName, tableName); - Table table = relations.remove(schemaTableName); - if (table == null) { - throw new TableNotFoundException(schemaTableName); - } - views.remove(schemaTableName); - partitions.keySet().removeIf(partitionName -> partitionName.matches(databaseName, tableName)); - - // remove data - if (deleteData && table.getTableType().equals(MANAGED_TABLE.name())) { - for (String location : locations) { - if (location != null) { - File directory = new File(new Path(location).toUri()); - checkArgument(isParentDir(directory, baseDirectory), "Table directory must be inside of the metastore base directory"); - deleteDirectory(directory); - } - } - } - } - - private static List listAllDataPaths(ThriftMetastore metastore, String schemaName, String tableName) - { - ImmutableList.Builder locations = ImmutableList.builder(); - Table table = metastore.getTable(schemaName, tableName).get(); - if (table.getSd().getLocation() != null) { - // For unpartitioned table, there should be nothing directly under this directory. - // But including this location in the set makes the directory content assert more - // extensive, which is desirable. - locations.add(table.getSd().getLocation()); - } - List partitionColumnNames = table.getPartitionKeys().stream() - .map(FieldSchema::getName) - .collect(toImmutableList()); - Optional> partitionNames = metastore.getPartitionNamesByFilter(schemaName, tableName, partitionColumnNames, TupleDomain.all()); - if (partitionNames.isPresent()) { - metastore.getPartitionsByNames(schemaName, tableName, partitionNames.get()).stream() - .map(partition -> partition.getSd().getLocation()) - .filter(location -> !location.startsWith(table.getSd().getLocation())) - .forEach(locations::add); - } - - return locations.build(); - } - - @Override - public synchronized void alterTable(String databaseName, String tableName, Table newTable) - { - SchemaTableName oldName = new SchemaTableName(databaseName, tableName); - SchemaTableName newName = new SchemaTableName(newTable.getDbName(), newTable.getTableName()); - - // if the name did not change, this is a simple schema change - if (oldName.equals(newName)) { - if (relations.replace(oldName, newTable) == null) { - throw new TableNotFoundException(oldName); - } - return; - } - - // remove old table definition and add the new one - Table table = relations.get(oldName); - if (table == null) { - throw new TableNotFoundException(oldName); - } - - if (relations.putIfAbsent(newName, newTable) != null) { - throw new TableAlreadyExistsException(newName); - } - relations.remove(oldName); - } - - @Override - public void alterTransactionalTable(Table table, long transactionId, long writeId) - { - alterTable(table.getDbName(), table.getTableName(), table); - } - - @Override - public synchronized List getAllTables(String databaseName) - { - ImmutableList.Builder tables = ImmutableList.builder(); - for (SchemaTableName schemaTableName : this.relations.keySet()) { - if (schemaTableName.getSchemaName().equals(databaseName)) { - tables.add(schemaTableName.getTableName()); - } - } - return tables.build(); - } - - @Override - public synchronized List getTablesWithParameter(String databaseName, String parameterKey, String parameterValue) - { - requireNonNull(parameterKey, "parameterKey is null"); - requireNonNull(parameterValue, "parameterValue is null"); - - return relations.entrySet().stream() - .filter(entry -> entry.getKey().getSchemaName().equals(databaseName) - && parameterValue.equals(entry.getValue().getParameters().get(parameterKey))) - .map(entry -> entry.getKey().getTableName()) - .collect(toImmutableList()); - } - - @Override - public synchronized List getAllViews(String databaseName) - { - ImmutableList.Builder tables = ImmutableList.builder(); - for (SchemaTableName schemaTableName : this.views.keySet()) { - if (schemaTableName.getSchemaName().equals(databaseName)) { - tables.add(schemaTableName.getTableName()); - } - } - return tables.build(); - } - - @Override - public synchronized Optional> getAllTables() - { - return Optional.of(ImmutableList.copyOf(relations.keySet())); - } - - @Override - public synchronized Optional> getAllViews() - { - return Optional.of(ImmutableList.copyOf(views.keySet())); - } - - @Override - public synchronized Optional getDatabase(String databaseName) - { - return Optional.ofNullable(databases.get(databaseName)); - } - - @Override - public synchronized void addPartitions(String databaseName, String tableName, List partitionsWithStatistics) - { - for (PartitionWithStatistics partitionWithStatistics : partitionsWithStatistics) { - Partition partition = toMetastoreApiPartition(partitionWithStatistics.getPartition()); - if (partition.getParameters() == null) { - partition.setParameters(ImmutableMap.of()); - } - PartitionName partitionKey = PartitionName.partition(databaseName, tableName, partitionWithStatistics.getPartitionName()); - partitions.put(partitionKey, partition); - partitionColumnStatistics.put(partitionKey, partitionWithStatistics.getStatistics()); - } - } - - @Override - public synchronized void dropPartition(String databaseName, String tableName, List parts, boolean deleteData) - { - partitions.entrySet().removeIf(entry -> - entry.getKey().matches(databaseName, tableName) && entry.getValue().getValues().equals(parts)); - } - - @Override - public synchronized void alterPartition(String databaseName, String tableName, PartitionWithStatistics partitionWithStatistics) - { - Partition partition = toMetastoreApiPartition(partitionWithStatistics.getPartition()); - if (partition.getParameters() == null) { - partition.setParameters(ImmutableMap.of()); - } - PartitionName partitionKey = PartitionName.partition(databaseName, tableName, partitionWithStatistics.getPartitionName()); - partitions.put(partitionKey, partition); - partitionColumnStatistics.put(partitionKey, partitionWithStatistics.getStatistics()); - } - - @Override - public synchronized Optional getPartition(String databaseName, String tableName, List partitionValues) - { - PartitionName name = PartitionName.partition(databaseName, tableName, partitionValues); - Partition partition = partitions.get(name); - if (partition == null) { - return Optional.empty(); - } - return Optional.of(partition.deepCopy()); - } - - @Override - public synchronized Optional> getPartitionNamesByFilter(String databaseName, String tableName, List columnNames, TupleDomain partitionKeysFilter) - { - Optional> parts = partitionKeyFilterToStringList(columnNames, partitionKeysFilter, assumeCanonicalPartitionKeys); - - if (parts.isEmpty()) { - return Optional.of(ImmutableList.of()); - } - return Optional.of(partitions.entrySet().stream() - .filter(entry -> partitionMatches(entry.getValue(), databaseName, tableName, parts.get())) - .map(entry -> entry.getKey().getPartitionName()) - .collect(toImmutableList())); - } - - private static boolean partitionMatches(Partition partition, String databaseName, String tableName, List parts) - { - if (!partition.getDbName().equals(databaseName) || - !partition.getTableName().equals(tableName)) { - return false; - } - List values = partition.getValues(); - if (values.size() != parts.size()) { - return false; - } - for (int i = 0; i < values.size(); i++) { - String part = parts.get(i); - if (!part.isEmpty() && !values.get(i).equals(part)) { - return false; - } - } - return true; - } - - @Override - public synchronized List getPartitionsByNames(String databaseName, String tableName, List partitionNames) - { - ImmutableList.Builder builder = ImmutableList.builder(); - for (String name : partitionNames) { - PartitionName partitionName = PartitionName.partition(databaseName, tableName, name); - Partition partition = partitions.get(partitionName); - if (partition == null) { - return ImmutableList.of(); - } - builder.add(partition.deepCopy()); - } - return builder.build(); - } - - @Override - public synchronized Optional
getTable(String databaseName, String tableName) - { - SchemaTableName schemaTableName = new SchemaTableName(databaseName, tableName); - return Optional.ofNullable(relations.get(schemaTableName)); - } - - @Override - public Set getSupportedColumnStatistics(Type type) - { - return ThriftMetastoreUtil.getSupportedColumnStatistics(type); - } - - @Override - public synchronized PartitionStatistics getTableStatistics(Table table) - { - return getTableStatistics(table.getDbName(), table.getTableName()); - } - - private synchronized PartitionStatistics getTableStatistics(String databaseName, String tableName) - { - SchemaTableName schemaTableName = new SchemaTableName(databaseName, tableName); - PartitionStatistics statistics = columnStatistics.get(schemaTableName); - if (statistics == null) { - statistics = new PartitionStatistics(createEmptyStatistics(), ImmutableMap.of()); - } - return statistics; - } - - @Override - public synchronized Map getPartitionStatistics(Table table, List partitions) - { - List partitionColumns = table.getPartitionKeys().stream() - .map(FieldSchema::getName) - .collect(toImmutableList()); - Set partitionNames = partitions.stream() - .map(partition -> makePartName(partitionColumns, partition.getValues())) - .collect(toImmutableSet()); - return getPartitionStatistics(table.getDbName(), table.getTableName(), partitionNames); - } - - private synchronized Map getPartitionStatistics(String databaseName, String tableName, Set partitionNames) - { - ImmutableMap.Builder result = ImmutableMap.builder(); - for (String partitionName : partitionNames) { - PartitionName partitionKey = PartitionName.partition(databaseName, tableName, partitionName); - PartitionStatistics statistics = partitionColumnStatistics.get(partitionKey); - if (statistics == null) { - statistics = new PartitionStatistics(createEmptyStatistics(), ImmutableMap.of()); - } - result.put(partitionName, statistics); - } - return result.buildOrThrow(); - } - - @Override - public synchronized void updateTableStatistics(String databaseName, String tableName, AcidTransaction transaction, Function update) - { - columnStatistics.put(new SchemaTableName(databaseName, tableName), update.apply(getTableStatistics(databaseName, tableName))); - } - - @Override - public synchronized void updatePartitionStatistics(Table table, String partitionName, Function update) - { - PartitionName partitionKey = PartitionName.partition(table.getDbName(), table.getTableName(), partitionName); - partitionColumnStatistics.put(partitionKey, update.apply(getPartitionStatistics(table.getDbName(), table.getTableName(), ImmutableSet.of(partitionName)).get(partitionName))); - } - - @Override - public void createRole(String role, String grantor) - { - throw new UnsupportedOperationException(); - } - - @Override - public void dropRole(String role) - { - throw new UnsupportedOperationException(); - } - - @Override - public Set listRoles() - { - throw new UnsupportedOperationException(); - } - - @Override - public void grantRoles(Set roles, Set grantees, boolean adminOption, HivePrincipal grantor) - { - throw new UnsupportedOperationException(); - } - - @Override - public void revokeRoles(Set roles, Set grantees, boolean adminOption, HivePrincipal grantor) - { - throw new UnsupportedOperationException(); - } - - @Override - public Set listRoleGrants(HivePrincipal principal) - { - throw new UnsupportedOperationException(); - } - - @Override - public Set listTablePrivileges(String databaseName, String tableName, Optional tableOwner, Optional principal) - { - return ImmutableSet.of(); - } - - @Override - public void grantTablePrivileges(String databaseName, String tableName, String tableOwner, HivePrincipal grantee, HivePrincipal grantor, Set privileges, boolean grantOption) - { - throw new UnsupportedOperationException(); - } - - @Override - public void revokeTablePrivileges(String databaseName, String tableName, String tableOwner, HivePrincipal grantee, HivePrincipal grantor, Set privileges, boolean grantOption) - { - throw new UnsupportedOperationException(); - } - - @Override - public Optional getFunction(String databaseName, String functionName) - { - throw new UnsupportedOperationException(); - } - - @Override - public Collection getFunctions(String databaseName, String functionName) - { - throw new UnsupportedOperationException(); - } - - @Override - public void createFunction(io.trino.hive.thrift.metastore.Function function) - { - throw new UnsupportedOperationException(); - } - - @Override - public void alterFunction(io.trino.hive.thrift.metastore.Function function) - { - throw new UnsupportedOperationException(); - } - - @Override - public void dropFunction(String databaseName, String functionName) - { - throw new UnsupportedOperationException(); - } - - private static boolean isParentDir(File directory, File baseDirectory) - { - for (File parent = directory.getParentFile(); parent != null; parent = parent.getParentFile()) { - if (parent.equals(baseDirectory)) { - return true; - } - } - return false; - } - - private static void deleteDirectory(File dir) - { - try { - deleteRecursively(dir.toPath(), ALLOW_INSECURE); - } - catch (IOException e) { - throw new UncheckedIOException(e); - } - } - - private static class PartitionName - { - private final String schemaName; - private final String tableName; - private final List partitionValues; - private final String partitionName; // does not participate in equals and hashValue - - private PartitionName(String schemaName, String tableName, List partitionValues, String partitionName) - { - this.schemaName = schemaName.toLowerCase(US); - this.tableName = tableName.toLowerCase(US); - this.partitionValues = requireNonNull(partitionValues, "partitionValues is null"); - this.partitionName = partitionName; - } - - public static PartitionName partition(String schemaName, String tableName, String partitionName) - { - return new PartitionName(schemaName.toLowerCase(US), tableName.toLowerCase(US), toPartitionValues(partitionName), partitionName); - } - - public static PartitionName partition(String schemaName, String tableName, List partitionValues) - { - return new PartitionName(schemaName.toLowerCase(US), tableName.toLowerCase(US), partitionValues, null); - } - - public String getPartitionName() - { - return requireNonNull(partitionName, "partitionName is null"); - } - - public boolean matches(String schemaName, String tableName) - { - return this.schemaName.equals(schemaName) && - this.tableName.equals(tableName); - } - - public PartitionName withSchemaName(String schemaName) - { - return new PartitionName(schemaName, tableName, partitionValues, partitionName); - } - - @Override - public int hashCode() - { - return Objects.hash(schemaName, tableName, partitionValues); - } - - @Override - public boolean equals(Object obj) - { - if (this == obj) { - return true; - } - if (obj == null || getClass() != obj.getClass()) { - return false; - } - PartitionName other = (PartitionName) obj; - return Objects.equals(this.schemaName, other.schemaName) - && Objects.equals(this.tableName, other.tableName) - && Objects.equals(this.partitionValues, other.partitionValues); - } - - @Override - public String toString() - { - return schemaName + "/" + tableName + "/" + partitionName; - } - } - - private static class PrincipalTableKey - { - private final String principalName; - private final PrincipalType principalType; - private final String database; - private final String table; - - public PrincipalTableKey(String principalName, PrincipalType principalType, String table, String database) - { - this.principalName = requireNonNull(principalName, "principalName is null"); - this.principalType = requireNonNull(principalType, "principalType is null"); - this.table = requireNonNull(table, "table is null"); - this.database = requireNonNull(database, "database is null"); - } - - public PrincipalTableKey withDatabase(String database) - { - return new PrincipalTableKey(principalName, principalType, table, database); - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - PrincipalTableKey that = (PrincipalTableKey) o; - return Objects.equals(principalName, that.principalName) && - principalType == that.principalType && - Objects.equals(table, that.table) && - Objects.equals(database, that.database); - } - - @Override - public int hashCode() - { - return Objects.hash(principalName, principalType, table, database); - } - - @Override - public String toString() - { - return toStringHelper(this) - .add("principalName", principalName) - .add("principalType", principalType) - .add("table", table) - .add("database", database) - .toString(); - } - } - - private static void rewriteKeys(Map map, Function keyRewriter) - { - for (K key : ImmutableSet.copyOf(map.keySet())) { - K newKey = keyRewriter.apply(key); - if (!newKey.equals(key)) { - map.put(newKey, map.remove(key)); - } - } - } -} diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/hms/TestIcebergHiveMetastoreTableOperationsReleaseLockFailure.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/hms/TestIcebergHiveMetastoreTableOperationsReleaseLockFailure.java deleted file mode 100644 index 34891e418eb9c..0000000000000 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/hms/TestIcebergHiveMetastoreTableOperationsReleaseLockFailure.java +++ /dev/null @@ -1,140 +0,0 @@ -/* - * 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.iceberg.catalog.hms; - -import com.google.common.collect.ImmutableMap; -import io.trino.Session; -import io.trino.hive.thrift.metastore.Table; -import io.trino.metadata.InternalFunctionBundle; -import io.trino.plugin.hive.metastore.AcidTransactionOwner; -import io.trino.plugin.hive.metastore.Database; -import io.trino.plugin.hive.metastore.HiveMetastore; -import io.trino.plugin.hive.metastore.thrift.BridgingHiveMetastore; -import io.trino.plugin.hive.metastore.thrift.InMemoryThriftMetastore; -import io.trino.plugin.hive.metastore.thrift.ThriftMetastore; -import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreConfig; -import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreFactory; -import io.trino.plugin.iceberg.IcebergPlugin; -import io.trino.plugin.iceberg.TestingIcebergConnectorFactory; -import io.trino.spi.security.ConnectorIdentity; -import io.trino.spi.security.PrincipalType; -import io.trino.testing.AbstractTestQueryFramework; -import io.trino.testing.LocalQueryRunner; -import org.junit.jupiter.api.Test; - -import java.io.File; -import java.nio.file.Files; -import java.util.Optional; - -import static com.google.inject.util.Modules.EMPTY_MODULE; -import static io.trino.testing.TestingSession.testSessionBuilder; -import static java.lang.String.format; - -public class TestIcebergHiveMetastoreTableOperationsReleaseLockFailure - extends AbstractTestQueryFramework -{ - private static final String ICEBERG_CATALOG = "iceberg"; - private static final String SCHEMA_NAME = "test_schema"; - private File baseDir; - - @Override - protected LocalQueryRunner createQueryRunner() - throws Exception - { - Session session = testSessionBuilder() - .setCatalog(ICEBERG_CATALOG) - .setSchema(SCHEMA_NAME) - .build(); - - baseDir = Files.createTempDirectory(null).toFile(); - baseDir.deleteOnExit(); - - LocalQueryRunner queryRunner = LocalQueryRunner.create(session); - - InternalFunctionBundle.InternalFunctionBundleBuilder functions = InternalFunctionBundle.builder(); - new IcebergPlugin().getFunctions().forEach(functions::functions); - queryRunner.addFunctions(functions.build()); - - ThriftMetastore thriftMetastore = createMetastoreWithReleaseLockFailure(); - HiveMetastore hiveMetastore = new BridgingHiveMetastore(thriftMetastore); - TestingIcebergHiveMetastoreCatalogModule testModule = new TestingIcebergHiveMetastoreCatalogModule(hiveMetastore, buildThriftMetastoreFactory(thriftMetastore)); - - queryRunner.createCatalog( - ICEBERG_CATALOG, - new TestingIcebergConnectorFactory(baseDir.toPath(), Optional.of(testModule), Optional.empty(), EMPTY_MODULE), - ImmutableMap.of()); - - Database database = Database.builder() - .setDatabaseName(SCHEMA_NAME) - .setOwnerName(Optional.of("public")) - .setOwnerType(Optional.of(PrincipalType.ROLE)) - .build(); - hiveMetastore.createDatabase(database); - - return queryRunner; - } - - @Test - public void testReleaseLockFailureDoesNotCorruptTheTable() - { - String tableName = "test_release_lock_failure"; - query(format("CREATE TABLE %s (a_varchar) AS VALUES ('Trino')", tableName)); - query(format("INSERT INTO %s VALUES 'rocks'", tableName)); - assertQuery("SELECT * FROM " + tableName, "VALUES 'Trino', 'rocks'"); - } - - private InMemoryThriftMetastore createMetastoreWithReleaseLockFailure() - { - return new InMemoryThriftMetastore(new File(baseDir + "/metastore"), new ThriftMetastoreConfig()) { - @Override - public long acquireTableExclusiveLock(AcidTransactionOwner transactionOwner, String queryId, String dbName, String tableName) - { - // returning dummy lock - return 100; - } - - @Override - public void releaseTableLock(long lockId) - { - throw new RuntimeException("Release table lock has failed!"); - } - - @Override - public synchronized void createTable(Table table) - { - // InMemoryThriftMetastore throws an exception if the table has any privileges set - table.setPrivileges(null); - super.createTable(table); - } - }; - } - - private static ThriftMetastoreFactory buildThriftMetastoreFactory(ThriftMetastore thriftMetastore) - { - return new ThriftMetastoreFactory() - { - @Override - public boolean isImpersonationEnabled() - { - return false; - } - - @Override - public ThriftMetastore createMetastore(Optional identity) - { - return thriftMetastore; - } - }; - } -} From 5ab33a19e090a9dd28e92134b7ed57886b0acbdc Mon Sep 17 00:00:00 2001 From: Dain Sundstrom Date: Thu, 30 Nov 2023 22:13:13 -0800 Subject: [PATCH 013/350] Use Thrift metastore in TestHiveMetastoreMetadataQueriesAccessOperations --- ...astoreMetadataQueriesAccessOperations.java | 425 +++--------------- 1 file changed, 53 insertions(+), 372 deletions(-) diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreMetadataQueriesAccessOperations.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreMetadataQueriesAccessOperations.java index 1e12b9b8659f4..4c73ecf68734e 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreMetadataQueriesAccessOperations.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreMetadataQueriesAccessOperations.java @@ -17,33 +17,31 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMultiset; import com.google.common.collect.Multiset; +import io.trino.plugin.hive.HiveConfig; import io.trino.plugin.hive.HiveType; import io.trino.plugin.hive.TestingHivePlugin; +import io.trino.plugin.hive.containers.HiveHadoop; import io.trino.plugin.hive.metastore.Column; import io.trino.plugin.hive.metastore.CountingAccessHiveMetastore; import io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Method; import io.trino.plugin.hive.metastore.CountingAccessHiveMetastoreUtil; +import io.trino.plugin.hive.metastore.Database; +import io.trino.plugin.hive.metastore.HiveMetastore; import io.trino.plugin.hive.metastore.Table; -import io.trino.plugin.hive.metastore.UnimplementedHiveMetastore; -import io.trino.spi.connector.RelationType; -import io.trino.spi.connector.SchemaTableName; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.DistributedQueryRunner; import io.trino.testing.QueryRunner; import org.intellij.lang.annotations.Language; +import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.parallel.Execution; -import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.function.Function; -import java.util.stream.IntStream; -import static com.google.common.collect.ImmutableList.toImmutableList; -import static com.google.common.collect.ImmutableMap.toImmutableMap; -import static io.trino.plugin.hive.HiveStorageFormat.ORC; +import static io.trino.plugin.hive.HiveStorageFormat.PARQUET; import static io.trino.plugin.hive.TableType.MANAGED_TABLE; +import static io.trino.plugin.hive.TestingThriftHiveMetastoreBuilder.testingThriftHiveMetastoreBuilder; import static io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Method.GET_ALL_DATABASES; import static io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Method.GET_ALL_RELATION_TYPES; import static io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Method.GET_ALL_TABLES; @@ -52,6 +50,7 @@ import static io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Method.GET_ALL_VIEWS_FROM_DATABASE; import static io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Method.GET_RELATION_TYPES_FROM_DATABASE; import static io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Method.GET_TABLE; +import static io.trino.plugin.hive.metastore.PrincipalPrivileges.NO_PRIVILEGES; import static io.trino.plugin.hive.metastore.StorageFormat.fromHiveStorageFormat; import static io.trino.testing.TestingSession.testSessionBuilder; import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; @@ -65,13 +64,46 @@ public class TestHiveMetastoreMetadataQueriesAccessOperations private static final int TEST_TABLES_IN_SCHEMA_COUNT = MAX_PREFIXES_COUNT + 3; private static final int TEST_ALL_TABLES_COUNT = TEST_SCHEMAS_COUNT * TEST_TABLES_IN_SCHEMA_COUNT; - private MockHiveMetastore mockMetastore; + private HiveHadoop hiveHadoop; private CountingAccessHiveMetastore metastore; @Override protected QueryRunner createQueryRunner() throws Exception { + hiveHadoop = HiveHadoop.builder().build(); + hiveHadoop.start(); + + HiveMetastore thriftMetastore = new BridgingHiveMetastore(testingThriftHiveMetastoreBuilder() + .metastoreClient(hiveHadoop.getHiveMetastoreEndpoint()) + .thriftMetastoreConfig(new ThriftMetastoreConfig() + .setBatchMetadataFetchEnabled(true) + .setDeleteFilesOnDrop(true)) + .hiveConfig(new HiveConfig().setTranslateHiveViews(true)) + .build()); + for (int databaseId = 0; databaseId < TEST_SCHEMAS_COUNT; databaseId++) { + String databaseName = "test_schema_" + databaseId; + thriftMetastore.createDatabase(Database.builder() + .setDatabaseName(databaseName) + .setOwnerName(Optional.empty()) + .setOwnerType(Optional.empty()) + .build()); + + for (int tableId = 0; tableId < TEST_TABLES_IN_SCHEMA_COUNT; tableId++) { + Table.Builder table = Table.builder() + .setDatabaseName(databaseName) + .setTableName("test_table_" + tableId) + .setTableType(MANAGED_TABLE.name()) + .setDataColumns(ImmutableList.of( + new Column("id", HiveType.HIVE_INT, Optional.empty(), Map.of()), + new Column("name", HiveType.HIVE_STRING, Optional.empty(), Map.of()))) + .setOwner(Optional.empty()); + table.getStorageBuilder() + .setStorageFormat(fromHiveStorageFormat(PARQUET)); + thriftMetastore.createTable(table.build(), NO_PRIVILEGES); + } + } + DistributedQueryRunner queryRunner = DistributedQueryRunner.builder( testSessionBuilder() .setCatalog("hive") @@ -82,23 +114,22 @@ protected QueryRunner createQueryRunner() .addCoordinatorProperty("optimizer.experimental-max-prefetched-information-schema-prefixes", Integer.toString(MAX_PREFIXES_COUNT)) .build(); - mockMetastore = new MockHiveMetastore(); - metastore = new CountingAccessHiveMetastore(mockMetastore); + metastore = new CountingAccessHiveMetastore(thriftMetastore); + queryRunner.installPlugin(new TestingHivePlugin(queryRunner.getCoordinator().getBaseDataDir().resolve("hive_data"), metastore)); queryRunner.createCatalog("hive", "hive", ImmutableMap.of()); return queryRunner; } - private void resetMetastoreSetup() + @AfterAll + void afterAll() { - mockMetastore.setAllTablesViewsImplemented(false); + hiveHadoop.stop(); } @Test public void testSelectSchemasWithoutPredicate() { - resetMetastoreSetup(); - assertMetastoreInvocations("SELECT * FROM information_schema.schemata", ImmutableMultiset.of(GET_ALL_DATABASES)); assertMetastoreInvocations("SELECT * FROM system.jdbc.schemas", ImmutableMultiset.of(GET_ALL_DATABASES)); } @@ -106,8 +137,6 @@ public void testSelectSchemasWithoutPredicate() @Test public void testSelectSchemasWithFilterByInformationSchema() { - resetMetastoreSetup(); - assertMetastoreInvocations("SELECT * FROM information_schema.schemata WHERE schema_name = 'information_schema'", ImmutableMultiset.of(GET_ALL_DATABASES)); assertMetastoreInvocations("SELECT * FROM system.jdbc.schemas WHERE table_schem = 'information_schema'", ImmutableMultiset.of(GET_ALL_DATABASES)); } @@ -115,8 +144,6 @@ public void testSelectSchemasWithFilterByInformationSchema() @Test public void testSelectSchemasWithLikeOverSchemaName() { - resetMetastoreSetup(); - assertMetastoreInvocations("SELECT * FROM information_schema.schemata WHERE schema_name LIKE 'test%'", ImmutableMultiset.of(GET_ALL_DATABASES)); assertMetastoreInvocations("SELECT * FROM system.jdbc.schemas WHERE table_schem LIKE 'test%'", ImmutableMultiset.of(GET_ALL_DATABASES)); } @@ -124,29 +151,16 @@ public void testSelectSchemasWithLikeOverSchemaName() @Test public void testSelectTablesWithoutPredicate() { - resetMetastoreSetup(); - - mockMetastore.setAllTablesViewsImplemented(true); Multiset tables = ImmutableMultiset.builder() .add(GET_ALL_RELATION_TYPES) .build(); assertMetastoreInvocations("SELECT * FROM information_schema.tables", tables); assertMetastoreInvocations("SELECT * FROM system.jdbc.tables", tables); - - mockMetastore.setAllTablesViewsImplemented(false); - tables = ImmutableMultiset.builder() - .add(GET_ALL_DATABASES) - .addCopies(GET_RELATION_TYPES_FROM_DATABASE, TEST_SCHEMAS_COUNT) - .build(); - assertMetastoreInvocations("SELECT * FROM information_schema.tables", tables); - assertMetastoreInvocations("SELECT * FROM system.jdbc.tables", tables); } @Test public void testSelectTablesWithFilterByInformationSchema() { - resetMetastoreSetup(); - assertMetastoreInvocations("SELECT * FROM information_schema.tables WHERE table_schema = 'information_schema'", ImmutableMultiset.of()); assertMetastoreInvocations("SELECT * FROM system.jdbc.tables WHERE table_schem = 'information_schema'", ImmutableMultiset.of()); } @@ -154,8 +168,6 @@ public void testSelectTablesWithFilterByInformationSchema() @Test public void testSelectTablesWithFilterBySchema() { - resetMetastoreSetup(); - assertMetastoreInvocations( "SELECT * FROM information_schema.tables WHERE table_schema = 'test_schema_0'", ImmutableMultiset.builder() @@ -171,9 +183,6 @@ public void testSelectTablesWithFilterBySchema() @Test public void testSelectTablesWithLikeOverSchema() { - resetMetastoreSetup(); - - mockMetastore.setAllTablesViewsImplemented(true); assertMetastoreInvocations( "SELECT * FROM information_schema.tables WHERE table_schema LIKE 'test%'", ImmutableMultiset.builder() @@ -185,22 +194,11 @@ public void testSelectTablesWithLikeOverSchema() ImmutableMultiset.builder() .add(GET_ALL_RELATION_TYPES) .build()); - - mockMetastore.setAllTablesViewsImplemented(false); - Multiset tables = ImmutableMultiset.builder() - .add(GET_ALL_DATABASES) - .addCopies(GET_RELATION_TYPES_FROM_DATABASE, TEST_SCHEMAS_COUNT) - .build(); - assertMetastoreInvocations("SELECT * FROM information_schema.tables WHERE table_schema LIKE 'test%'", tables); - assertMetastoreInvocations("SELECT * FROM system.jdbc.tables WHERE table_schem LIKE 'test%'", tables); } @Test public void testSelectTablesWithFilterByTableName() { - resetMetastoreSetup(); - - mockMetastore.setAllTablesViewsImplemented(true); assertMetastoreInvocations( "SELECT * FROM information_schema.tables WHERE table_name = 'test_table_0'", ImmutableMultiset.builder() @@ -214,24 +212,11 @@ public void testSelectTablesWithFilterByTableName() assertMetastoreInvocations("SELECT * FROM system.jdbc.tables WHERE table_name = 'test_table_0'", tables); assertMetastoreInvocations("SELECT * FROM system.jdbc.tables WHERE table_name LIKE 'test\\_table\\_0' ESCAPE '\\'", tables); assertMetastoreInvocations("SELECT * FROM system.jdbc.tables WHERE table_name LIKE 'test_table_0' ESCAPE '\\'", tables); - - mockMetastore.setAllTablesViewsImplemented(false); - tables = ImmutableMultiset.builder() - .add(GET_ALL_DATABASES) - .addCopies(GET_RELATION_TYPES_FROM_DATABASE, TEST_SCHEMAS_COUNT) - .build(); - assertMetastoreInvocations("SELECT * FROM information_schema.tables WHERE table_name = 'test_table_0'", tables); - assertMetastoreInvocations("SELECT * FROM system.jdbc.tables WHERE table_name = 'test_table_0'", tables); - assertMetastoreInvocations("SELECT * FROM system.jdbc.tables WHERE table_name LIKE 'test\\_table\\_0' ESCAPE '\\'", tables); - assertMetastoreInvocations("SELECT * FROM system.jdbc.tables WHERE table_name LIKE 'test_table_0' ESCAPE '\\'", tables); } @Test public void testSelectTablesWithLikeOverTableName() { - resetMetastoreSetup(); - - mockMetastore.setAllTablesViewsImplemented(true); assertMetastoreInvocations( "SELECT * FROM information_schema.tables WHERE table_name LIKE 'test%'", ImmutableMultiset.builder() @@ -243,49 +228,22 @@ public void testSelectTablesWithLikeOverTableName() ImmutableMultiset.builder() .add(GET_ALL_RELATION_TYPES) .build()); - - mockMetastore.setAllTablesViewsImplemented(false); - Multiset tables = ImmutableMultiset.builder() - .add(GET_ALL_DATABASES) - .addCopies(GET_RELATION_TYPES_FROM_DATABASE, TEST_SCHEMAS_COUNT) - .build(); - assertMetastoreInvocations("SELECT * FROM information_schema.tables WHERE table_name LIKE 'test%'", tables); - assertMetastoreInvocations("SELECT * FROM system.jdbc.tables WHERE table_name LIKE 'test%'", tables); } @Test public void testSelectViewsWithoutPredicate() { - resetMetastoreSetup(); - - mockMetastore.setAllTablesViewsImplemented(true); assertMetastoreInvocations("SELECT * FROM information_schema.views", ImmutableMultiset.of(GET_ALL_VIEWS)); assertMetastoreInvocations( "SELECT * FROM system.jdbc.tables WHERE table_type = 'VIEW'", ImmutableMultiset.builder() .add(GET_ALL_RELATION_TYPES) .build()); - - mockMetastore.setAllTablesViewsImplemented(false); - assertMetastoreInvocations( - "SELECT * FROM information_schema.views", - ImmutableMultiset.builder() - .add(GET_ALL_DATABASES) - .addCopies(GET_ALL_VIEWS_FROM_DATABASE, TEST_SCHEMAS_COUNT) - .build()); - assertMetastoreInvocations( - "SELECT * FROM system.jdbc.tables WHERE table_type = 'VIEW'", - ImmutableMultiset.builder() - .add(GET_ALL_DATABASES) - .addCopies(GET_RELATION_TYPES_FROM_DATABASE, TEST_SCHEMAS_COUNT) - .build()); } @Test public void testSelectViewsWithFilterByInformationSchema() { - resetMetastoreSetup(); - assertMetastoreInvocations("SELECT * FROM information_schema.views WHERE table_schema = 'information_schema'", ImmutableMultiset.of()); assertMetastoreInvocations("SELECT * FROM system.jdbc.tables WHERE table_type = 'VIEW' AND table_schem = 'information_schema'", ImmutableMultiset.of()); } @@ -293,8 +251,6 @@ public void testSelectViewsWithFilterByInformationSchema() @Test public void testSelectViewsWithFilterBySchema() { - resetMetastoreSetup(); - assertMetastoreInvocations("SELECT * FROM information_schema.views WHERE table_schema = 'test_schema_0'", ImmutableMultiset.of(GET_ALL_VIEWS_FROM_DATABASE)); assertMetastoreInvocations("SELECT * FROM system.jdbc.tables WHERE table_type = 'VIEW' AND table_schem = 'test_schema_0'", ImmutableMultiset.builder() @@ -305,9 +261,6 @@ public void testSelectViewsWithFilterBySchema() @Test public void testSelectViewsWithLikeOverSchema() { - resetMetastoreSetup(); - - mockMetastore.setAllTablesViewsImplemented(true); assertMetastoreInvocations( "SELECT * FROM information_schema.views WHERE table_schema LIKE 'test%'", ImmutableMultiset.builder() @@ -319,28 +272,11 @@ public void testSelectViewsWithLikeOverSchema() ImmutableMultiset.builder() .add(GET_ALL_RELATION_TYPES) .build()); - - mockMetastore.setAllTablesViewsImplemented(false); - assertMetastoreInvocations( - "SELECT * FROM information_schema.views WHERE table_schema LIKE 'test%'", - ImmutableMultiset.builder() - .add(GET_ALL_DATABASES) - .addCopies(GET_ALL_VIEWS_FROM_DATABASE, TEST_SCHEMAS_COUNT) - .build()); - assertMetastoreInvocations( - "SELECT * FROM system.jdbc.tables WHERE table_type = 'VIEW' AND table_schem LIKE 'test%'", - ImmutableMultiset.builder() - .add(GET_ALL_DATABASES) - .addCopies(GET_RELATION_TYPES_FROM_DATABASE, TEST_SCHEMAS_COUNT) - .build()); } @Test public void testSelectViewsWithFilterByTableName() { - resetMetastoreSetup(); - - mockMetastore.setAllTablesViewsImplemented(true); assertMetastoreInvocations( "SELECT * FROM information_schema.views WHERE table_name = 'test_table_0'", ImmutableMultiset.builder() @@ -352,28 +288,11 @@ public void testSelectViewsWithFilterByTableName() ImmutableMultiset.builder() .add(GET_ALL_RELATION_TYPES) .build()); - - mockMetastore.setAllTablesViewsImplemented(false); - assertMetastoreInvocations( - "SELECT * FROM information_schema.views WHERE table_name = 'test_table_0'", - ImmutableMultiset.builder() - .add(GET_ALL_DATABASES) - .addCopies(GET_ALL_VIEWS_FROM_DATABASE, TEST_SCHEMAS_COUNT) - .build()); - assertMetastoreInvocations( - "SELECT * FROM system.jdbc.tables WHERE table_type = 'VIEW' AND table_name = 'test_table_0'", - ImmutableMultiset.builder() - .add(GET_ALL_DATABASES) - .addCopies(GET_RELATION_TYPES_FROM_DATABASE, TEST_SCHEMAS_COUNT) - .build()); } @Test public void testSelectViewsWithLikeOverTableName() { - resetMetastoreSetup(); - - mockMetastore.setAllTablesViewsImplemented(true); assertMetastoreInvocations( "SELECT * FROM information_schema.views WHERE table_name LIKE 'test%'", ImmutableMultiset.builder() @@ -385,28 +304,11 @@ public void testSelectViewsWithLikeOverTableName() ImmutableMultiset.builder() .add(GET_ALL_RELATION_TYPES) .build()); - - mockMetastore.setAllTablesViewsImplemented(false); - assertMetastoreInvocations( - "SELECT * FROM information_schema.views WHERE table_name LIKE 'test%'", - ImmutableMultiset.builder() - .add(GET_ALL_DATABASES) - .addCopies(GET_ALL_VIEWS_FROM_DATABASE, TEST_SCHEMAS_COUNT) - .build()); - assertMetastoreInvocations( - "SELECT * FROM system.jdbc.tables WHERE table_type = 'VIEW' AND table_name LIKE 'test%'", - ImmutableMultiset.builder() - .add(GET_ALL_DATABASES) - .addCopies(GET_RELATION_TYPES_FROM_DATABASE, TEST_SCHEMAS_COUNT) - .build()); } @Test public void testSelectColumnsWithoutPredicate() { - resetMetastoreSetup(); - - mockMetastore.setAllTablesViewsImplemented(true); ImmutableMultiset tables = ImmutableMultiset.builder() .add(GET_ALL_TABLES) .add(GET_ALL_VIEWS) @@ -414,23 +316,11 @@ public void testSelectColumnsWithoutPredicate() .build(); assertMetastoreInvocations("SELECT * FROM information_schema.columns", tables); assertMetastoreInvocations("SELECT * FROM system.jdbc.columns", tables); - - mockMetastore.setAllTablesViewsImplemented(false); - tables = ImmutableMultiset.builder() - .add(GET_ALL_DATABASES) - .addCopies(GET_ALL_TABLES_FROM_DATABASE, TEST_SCHEMAS_COUNT) - .addCopies(GET_ALL_VIEWS_FROM_DATABASE, TEST_SCHEMAS_COUNT) - .addCopies(GET_TABLE, TEST_ALL_TABLES_COUNT) - .build(); - assertMetastoreInvocations("SELECT * FROM information_schema.columns", tables); - assertMetastoreInvocations("SELECT * FROM system.jdbc.columns", tables); } @Test public void testSelectColumnsFilterByInformationSchema() { - resetMetastoreSetup(); - assertMetastoreInvocations("SELECT * FROM information_schema.columns WHERE table_schema = 'information_schema'", ImmutableMultiset.of()); assertMetastoreInvocations("SELECT * FROM system.jdbc.columns WHERE table_schem = 'information_schema'", ImmutableMultiset.of()); } @@ -438,8 +328,6 @@ public void testSelectColumnsFilterByInformationSchema() @Test public void testSelectColumnsFilterBySchema() { - resetMetastoreSetup(); - assertMetastoreInvocations("SELECT * FROM information_schema.columns WHERE table_schema = 'test_schema_0'", ImmutableMultiset.builder() .add(GET_ALL_TABLES_FROM_DATABASE) @@ -470,9 +358,6 @@ public void testSelectColumnsFilterBySchema() @Test public void testSelectColumnsWithLikeOverSchema() { - resetMetastoreSetup(); - - mockMetastore.setAllTablesViewsImplemented(true); assertMetastoreInvocations( "SELECT * FROM information_schema.columns WHERE table_schema LIKE 'test%'", ImmutableMultiset.builder() @@ -488,31 +373,11 @@ public void testSelectColumnsWithLikeOverSchema() .addCopies(GET_ALL_TABLES_FROM_DATABASE, TEST_SCHEMAS_COUNT) .addCopies(GET_TABLE, TEST_ALL_TABLES_COUNT) .build()); - - mockMetastore.setAllTablesViewsImplemented(false); - assertMetastoreInvocations( - "SELECT * FROM information_schema.columns WHERE table_schema LIKE 'test%'", - ImmutableMultiset.builder() - .add(GET_ALL_DATABASES) - .addCopies(GET_ALL_TABLES_FROM_DATABASE, TEST_SCHEMAS_COUNT) - .addCopies(GET_ALL_VIEWS_FROM_DATABASE, TEST_SCHEMAS_COUNT) - .addCopies(GET_TABLE, TEST_ALL_TABLES_COUNT) - .build()); - assertMetastoreInvocations( - "SELECT * FROM system.jdbc.columns WHERE table_schem LIKE 'test%'", - ImmutableMultiset.builder() - .add(GET_ALL_DATABASES) - .addCopies(GET_ALL_TABLES_FROM_DATABASE, TEST_SCHEMAS_COUNT) - .addCopies(GET_TABLE, TEST_ALL_TABLES_COUNT) - .build()); } @Test public void testSelectColumnsFilterByTableName() { - resetMetastoreSetup(); - - mockMetastore.setAllTablesViewsImplemented(true); assertMetastoreInvocations( "SELECT * FROM information_schema.columns WHERE table_name = 'test_table_0'", ImmutableMultiset.builder() @@ -527,49 +392,21 @@ public void testSelectColumnsFilterByTableName() "SELECT * FROM system.jdbc.columns WHERE table_name = 'test_table_0'", ImmutableMultiset.builder() .add(GET_ALL_DATABASES) - .addCopies(GET_TABLE, TEST_SCHEMAS_COUNT) - .build()); - assertMetastoreInvocations( - "SELECT * FROM system.jdbc.columns WHERE table_name LIKE 'test\\_table\\_0' ESCAPE '\\'", - ImmutableMultiset.builder() - .add(GET_ALL_DATABASES) - .addCopies(GET_TABLE, TEST_SCHEMAS_COUNT) - .build()); - assertMetastoreInvocations( - "SELECT * FROM system.jdbc.columns WHERE table_name LIKE 'test_table_0' ESCAPE '\\'", - ImmutableMultiset.builder() - .add(GET_ALL_DATABASES) - .addCopies(GET_ALL_TABLES_FROM_DATABASE, TEST_SCHEMAS_COUNT) - .addCopies(GET_TABLE, TEST_SCHEMAS_COUNT) - .build()); - - mockMetastore.setAllTablesViewsImplemented(false); - assertMetastoreInvocations( - "SELECT * FROM information_schema.columns WHERE table_name = 'test_table_0'", - ImmutableMultiset.builder() - .add(GET_ALL_DATABASES) - .addCopies(GET_ALL_TABLES_FROM_DATABASE, TEST_SCHEMAS_COUNT) - .addCopies(GET_ALL_VIEWS_FROM_DATABASE, TEST_SCHEMAS_COUNT) - // TODO When there are many schemas, there are no "prefixes" and we end up calling ConnectorMetadata without any filter whatsoever. - // If such queries are common enough, we could iterate over schemas and for each schema try getting a table by given name. - .addCopies(GET_TABLE, TEST_ALL_TABLES_COUNT) - .build()); - assertMetastoreInvocations( - "SELECT * FROM system.jdbc.columns WHERE table_name = 'test_table_0'", - ImmutableMultiset.builder() - .add(GET_ALL_DATABASES) - .addCopies(GET_TABLE, TEST_SCHEMAS_COUNT) + .add(GET_ALL_TABLES_FROM_DATABASE) + .addCopies(GET_TABLE, TEST_SCHEMAS_COUNT + 1) .build()); assertMetastoreInvocations( "SELECT * FROM system.jdbc.columns WHERE table_name LIKE 'test\\_table\\_0' ESCAPE '\\'", ImmutableMultiset.builder() .add(GET_ALL_DATABASES) - .addCopies(GET_TABLE, TEST_SCHEMAS_COUNT) + .add(GET_ALL_TABLES_FROM_DATABASE) + .addCopies(GET_TABLE, TEST_SCHEMAS_COUNT + 1) .build()); assertMetastoreInvocations( "SELECT * FROM system.jdbc.columns WHERE table_name LIKE 'test_table_0' ESCAPE '\\'", ImmutableMultiset.builder() .add(GET_ALL_DATABASES) + .add(GET_ALL_TABLES_FROM_DATABASE) .addCopies(GET_ALL_TABLES_FROM_DATABASE, TEST_SCHEMAS_COUNT) .addCopies(GET_TABLE, TEST_SCHEMAS_COUNT) .build()); @@ -578,9 +415,6 @@ public void testSelectColumnsFilterByTableName() @Test public void testSelectColumnsWithLikeOverTableName() { - resetMetastoreSetup(); - - mockMetastore.setAllTablesViewsImplemented(true); assertMetastoreInvocations("SELECT * FROM information_schema.columns WHERE table_name LIKE 'test%'", ImmutableMultiset.builder() .add(GET_ALL_DATABASES) @@ -591,24 +425,7 @@ public void testSelectColumnsWithLikeOverTableName() assertMetastoreInvocations("SELECT * FROM system.jdbc.columns WHERE table_name LIKE 'test%'", ImmutableMultiset.builder() .add(GET_ALL_DATABASES) - .addCopies(GET_ALL_TABLES_FROM_DATABASE, TEST_SCHEMAS_COUNT) - .addCopies(GET_TABLE, TEST_ALL_TABLES_COUNT) - .build()); - - mockMetastore.setAllTablesViewsImplemented(false); - assertMetastoreInvocations( - "SELECT * FROM information_schema.columns WHERE table_name LIKE 'test%'", - ImmutableMultiset.builder() - .add(GET_ALL_DATABASES) - .addCopies(GET_ALL_TABLES_FROM_DATABASE, TEST_SCHEMAS_COUNT) - .addCopies(GET_ALL_VIEWS_FROM_DATABASE, TEST_SCHEMAS_COUNT) - .addCopies(GET_TABLE, TEST_ALL_TABLES_COUNT) - .build()); - assertMetastoreInvocations( - "SELECT * FROM system.jdbc.columns WHERE table_name LIKE 'test%'", - ImmutableMultiset.builder() - .add(GET_ALL_DATABASES) - .addCopies(GET_ALL_TABLES_FROM_DATABASE, TEST_SCHEMAS_COUNT) + .addCopies(GET_ALL_TABLES_FROM_DATABASE, TEST_SCHEMAS_COUNT + 1) .addCopies(GET_TABLE, TEST_ALL_TABLES_COUNT) .build()); } @@ -616,9 +433,6 @@ public void testSelectColumnsWithLikeOverTableName() @Test public void testSelectColumnsFilterByColumn() { - resetMetastoreSetup(); - - mockMetastore.setAllTablesViewsImplemented(true); assertMetastoreInvocations( "SELECT * FROM information_schema.columns WHERE column_name = 'name'", ImmutableMultiset.builder() @@ -633,32 +447,11 @@ public void testSelectColumnsFilterByColumn() .add(GET_ALL_VIEWS) .addCopies(GET_TABLE, TEST_ALL_TABLES_COUNT) .build()); - - mockMetastore.setAllTablesViewsImplemented(false); - assertMetastoreInvocations( - "SELECT * FROM information_schema.columns WHERE column_name = 'name'", - ImmutableMultiset.builder() - .add(GET_ALL_DATABASES) - .addCopies(GET_ALL_TABLES_FROM_DATABASE, TEST_SCHEMAS_COUNT) - .addCopies(GET_ALL_VIEWS_FROM_DATABASE, TEST_SCHEMAS_COUNT) - .addCopies(GET_TABLE, TEST_ALL_TABLES_COUNT) - .build()); - assertMetastoreInvocations( - "SELECT * FROM system.jdbc.columns WHERE column_name = 'name'", - ImmutableMultiset.builder() - .add(GET_ALL_DATABASES) - .addCopies(GET_ALL_TABLES_FROM_DATABASE, TEST_SCHEMAS_COUNT) - .addCopies(GET_ALL_VIEWS_FROM_DATABASE, TEST_SCHEMAS_COUNT) - .addCopies(GET_TABLE, TEST_ALL_TABLES_COUNT) - .build()); } @Test public void testSelectColumnsWithLikeOverColumn() { - resetMetastoreSetup(); - - mockMetastore.setAllTablesViewsImplemented(true); assertMetastoreInvocations( "SELECT * FROM information_schema.columns WHERE column_name LIKE 'n%'", ImmutableMultiset.builder() @@ -674,31 +467,11 @@ public void testSelectColumnsWithLikeOverColumn() .add(GET_ALL_VIEWS) .addCopies(GET_TABLE, TEST_ALL_TABLES_COUNT) .build()); - - mockMetastore.setAllTablesViewsImplemented(false); - assertMetastoreInvocations( - "SELECT * FROM information_schema.columns WHERE column_name LIKE 'n%'", - ImmutableMultiset.builder() - .add(GET_ALL_DATABASES) - .addCopies(GET_ALL_TABLES_FROM_DATABASE, TEST_SCHEMAS_COUNT) - .addCopies(GET_ALL_VIEWS_FROM_DATABASE, TEST_SCHEMAS_COUNT) - .addCopies(GET_TABLE, TEST_ALL_TABLES_COUNT) - .build()); - assertMetastoreInvocations( - "SELECT * FROM system.jdbc.columns WHERE column_name LIKE 'n%'", - ImmutableMultiset.builder() - .add(GET_ALL_DATABASES) - .addCopies(GET_ALL_TABLES_FROM_DATABASE, TEST_SCHEMAS_COUNT) - .addCopies(GET_ALL_VIEWS_FROM_DATABASE, TEST_SCHEMAS_COUNT) - .addCopies(GET_TABLE, TEST_ALL_TABLES_COUNT) - .build()); } @Test public void testSelectColumnsFilterByTableAndSchema() { - resetMetastoreSetup(); - assertMetastoreInvocations("SELECT * FROM information_schema.columns WHERE table_schema = 'test_schema_0' AND table_name = 'test_table_0'", ImmutableMultiset.of(GET_TABLE)); assertMetastoreInvocations("SELECT * FROM system.jdbc.columns WHERE table_schem = 'test_schema_0' AND table_name = 'test_table_0'", ImmutableMultiset.of(GET_TABLE)); assertMetastoreInvocations("SELECT * FROM system.jdbc.columns WHERE table_schem LIKE 'test\\_schema\\_0' ESCAPE '\\' AND table_name LIKE 'test\\_table\\_0' ESCAPE '\\'", ImmutableMultiset.of(GET_TABLE)); @@ -714,96 +487,4 @@ private void assertMetastoreInvocations(@Language("SQL") String query, Multiset< { CountingAccessHiveMetastoreUtil.assertMetastoreInvocations(metastore, getQueryRunner(), getQueryRunner().getDefaultSession(), query, expectedInvocations); } - - private static class MockHiveMetastore - extends UnimplementedHiveMetastore - { - private static final List SCHEMAS = IntStream.range(0, TEST_SCHEMAS_COUNT) - .mapToObj("test_schema_%d"::formatted) - .collect(toImmutableList()); - private static final List TABLES_PER_SCHEMA = IntStream.range(0, TEST_TABLES_IN_SCHEMA_COUNT) - .mapToObj("test_table_%d"::formatted) - .collect(toImmutableList()); - private static final ImmutableList ALL_TABLES = SCHEMAS.stream() - .flatMap(schema -> TABLES_PER_SCHEMA.stream() - .map(table -> new SchemaTableName(schema, table))) - .collect(toImmutableList()); - - private boolean allTablesViewsImplemented; - - @Override - public List getAllDatabases() - { - return SCHEMAS; - } - - @Override - public List getAllTables(String databaseName) - { - return TABLES_PER_SCHEMA; - } - - @Override - public Optional> getAllTables() - { - if (allTablesViewsImplemented) { - return Optional.of(ALL_TABLES); - } - return Optional.empty(); - } - - @Override - public Map getRelationTypes(String databaseName) - { - return TABLES_PER_SCHEMA.stream() - .collect(toImmutableMap(Function.identity(), ignore -> RelationType.TABLE)); - } - - @Override - public Optional> getRelationTypes() - { - if (allTablesViewsImplemented) { - return Optional.of(ALL_TABLES.stream() - .collect(toImmutableMap(Function.identity(), ignore -> RelationType.TABLE))); - } - return Optional.empty(); - } - - @Override - public List getAllViews(String databaseName) - { - return ImmutableList.of(); - } - - @Override - public Optional> getAllViews() - { - if (allTablesViewsImplemented) { - return Optional.of(ImmutableList.of()); - } - return Optional.empty(); - } - - @Override - public Optional
getTable(String databaseName, String tableName) - { - return Optional.of(Table.builder() - .setDatabaseName(databaseName) - .setTableName(tableName) - .setDataColumns(ImmutableList.of( - new Column("id", HiveType.HIVE_INT, Optional.empty(), Map.of()), - new Column("name", HiveType.HIVE_STRING, Optional.empty(), Map.of()))) - .setOwner(Optional.empty()) - .setTableType(MANAGED_TABLE.name()) - .withStorage(storage -> - storage.setStorageFormat(fromHiveStorageFormat(ORC)) - .setLocation(Optional.empty())) - .build()); - } - - public void setAllTablesViewsImplemented(boolean allTablesViewsImplemented) - { - this.allTablesViewsImplemented = allTablesViewsImplemented; - } - } } From 1321a1cbff4e7cac2821ad9fa88d806c95ebb966 Mon Sep 17 00:00:00 2001 From: Dain Sundstrom Date: Thu, 30 Nov 2023 22:54:26 -0800 Subject: [PATCH 014/350] Use FileMetastore in TestDeltaLakeSplitManager --- .../trino/plugin/deltalake/TestDeltaLakeSplitManager.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSplitManager.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSplitManager.java index 20ba4c7e23436..1971f297ee58c 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSplitManager.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSplitManager.java @@ -19,7 +19,9 @@ import io.airlift.json.JsonCodec; import io.airlift.json.JsonCodecFactory; import io.airlift.units.DataSize; +import io.trino.filesystem.Location; import io.trino.filesystem.hdfs.HdfsFileSystemFactory; +import io.trino.filesystem.memory.MemoryFileSystemFactory; import io.trino.plugin.deltalake.statistics.CachingExtendedStatisticsAccess; import io.trino.plugin.deltalake.statistics.ExtendedStatistics; import io.trino.plugin.deltalake.statistics.MetaDirStatisticsAccess; @@ -38,7 +40,6 @@ import io.trino.plugin.hive.HiveTransactionHandle; import io.trino.plugin.hive.NodeVersion; import io.trino.plugin.hive.metastore.HiveMetastoreFactory; -import io.trino.plugin.hive.metastore.UnimplementedHiveMetastore; import io.trino.plugin.hive.parquet.ParquetReaderConfig; import io.trino.plugin.hive.parquet.ParquetWriterConfig; import io.trino.spi.SplitWeight; @@ -63,6 +64,7 @@ import static io.trino.plugin.hive.HiveTestUtils.HDFS_ENVIRONMENT; import static io.trino.plugin.hive.HiveTestUtils.HDFS_FILE_SYSTEM_FACTORY; import static io.trino.plugin.hive.HiveTestUtils.HDFS_FILE_SYSTEM_STATS; +import static io.trino.plugin.hive.metastore.file.TestingFileHiveMetastore.createTestingFileHiveMetastore; import static org.assertj.core.api.Assertions.assertThat; public class TestDeltaLakeSplitManager @@ -208,7 +210,7 @@ public List getActiveFiles( JsonCodec.jsonCodec(LastCheckpoint.class)); DeltaLakeMetadataFactory metadataFactory = new DeltaLakeMetadataFactory( - HiveMetastoreFactory.ofInstance(new UnimplementedHiveMetastore()), + HiveMetastoreFactory.ofInstance(createTestingFileHiveMetastore(new MemoryFileSystemFactory(), Location.of("memory:///"))), hdfsFileSystemFactory, transactionLogAccess, typeManager, From 1ec1e372f4399d3788559d396ce1e92357bd3120 Mon Sep 17 00:00:00 2001 From: Dain Sundstrom Date: Thu, 30 Nov 2023 22:54:53 -0800 Subject: [PATCH 015/350] Remove unused UnimplementedHiveMetastore --- .../metastore/UnimplementedHiveMetastore.java | 343 ------------------ 1 file changed, 343 deletions(-) delete mode 100644 plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/UnimplementedHiveMetastore.java diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/UnimplementedHiveMetastore.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/UnimplementedHiveMetastore.java deleted file mode 100644 index ca2fd5dba8c89..0000000000000 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/UnimplementedHiveMetastore.java +++ /dev/null @@ -1,343 +0,0 @@ -/* - * 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.hive.metastore; - -import io.trino.plugin.hive.HiveColumnStatisticType; -import io.trino.plugin.hive.HiveType; -import io.trino.plugin.hive.PartitionStatistics; -import io.trino.plugin.hive.acid.AcidTransaction; -import io.trino.plugin.hive.metastore.HivePrivilegeInfo.HivePrivilege; -import io.trino.spi.connector.RelationType; -import io.trino.spi.connector.SchemaTableName; -import io.trino.spi.function.LanguageFunction; -import io.trino.spi.predicate.TupleDomain; -import io.trino.spi.security.RoleGrant; -import io.trino.spi.type.Type; - -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.function.Function; - -public class UnimplementedHiveMetastore - implements HiveMetastore -{ - @Override - public Optional getDatabase(String databaseName) - { - throw new UnsupportedOperationException(); - } - - @Override - public List getAllDatabases() - { - throw new UnsupportedOperationException(); - } - - @Override - public Optional
getTable(String databaseName, String tableName) - { - throw new UnsupportedOperationException(); - } - - @Override - public Set getSupportedColumnStatistics(Type type) - { - throw new UnsupportedOperationException(); - } - - @Override - public PartitionStatistics getTableStatistics(Table table) - { - throw new UnsupportedOperationException(); - } - - @Override - public Map getPartitionStatistics(Table table, List partitions) - { - throw new UnsupportedOperationException(); - } - - @Override - public void updateTableStatistics(String databaseName, - String tableName, - AcidTransaction transaction, - Function update) - { - throw new UnsupportedOperationException(); - } - - @Override - public void updatePartitionStatistics(Table table, Map> updates) - { - throw new UnsupportedOperationException(); - } - - @Override - public List getAllTables(String databaseName) - { - throw new UnsupportedOperationException(); - } - - @Override - public Optional> getAllTables() - { - throw new UnsupportedOperationException(); - } - - @Override - public Map getRelationTypes(String databaseName) - { - throw new UnsupportedOperationException(); - } - - @Override - public Optional> getRelationTypes() - { - throw new UnsupportedOperationException(); - } - - @Override - public List getTablesWithParameter(String databaseName, String parameterKey, String parameterValue) - { - throw new UnsupportedOperationException(); - } - - @Override - public List getAllViews(String databaseName) - { - throw new UnsupportedOperationException(); - } - - @Override - public Optional> getAllViews() - { - throw new UnsupportedOperationException(); - } - - @Override - public void createDatabase(Database database) - { - throw new UnsupportedOperationException(); - } - - @Override - public void dropDatabase(String databaseName, boolean deleteData) - { - throw new UnsupportedOperationException(); - } - - @Override - public void renameDatabase(String databaseName, String newDatabaseName) - { - throw new UnsupportedOperationException(); - } - - @Override - public void setDatabaseOwner(String databaseName, HivePrincipal principal) - { - throw new UnsupportedOperationException(); - } - - @Override - public void setTableOwner(String databaseName, String tableName, HivePrincipal principal) - { - throw new UnsupportedOperationException(); - } - - @Override - public void createTable(Table table, PrincipalPrivileges principalPrivileges) - { - throw new UnsupportedOperationException(); - } - - @Override - public void dropTable(String databaseName, String tableName, boolean deleteData) - { - throw new UnsupportedOperationException(); - } - - @Override - public void replaceTable(String databaseName, String tableName, Table newTable, PrincipalPrivileges principalPrivileges) - { - throw new UnsupportedOperationException(); - } - - @Override - public void renameTable(String databaseName, String tableName, String newDatabaseName, String newTableName) - { - throw new UnsupportedOperationException(); - } - - @Override - public void commentTable(String databaseName, String tableName, Optional comment) - { - throw new UnsupportedOperationException(); - } - - @Override - public void commentColumn(String databaseName, String tableName, String columnName, Optional comment) - { - throw new UnsupportedOperationException(); - } - - @Override - public void addColumn(String databaseName, String tableName, String columnName, HiveType columnType, String columnComment) - { - throw new UnsupportedOperationException(); - } - - @Override - public void renameColumn(String databaseName, String tableName, String oldColumnName, String newColumnName) - { - throw new UnsupportedOperationException(); - } - - @Override - public void dropColumn(String databaseName, String tableName, String columnName) - { - throw new UnsupportedOperationException(); - } - - @Override - public Optional getPartition(Table table, List partitionValues) - { - throw new UnsupportedOperationException(); - } - - @Override - public Optional> getPartitionNamesByFilter(String databaseName, - String tableName, - List columnNames, - TupleDomain partitionKeysFilter) - { - throw new UnsupportedOperationException(); - } - - @Override - public Map> getPartitionsByNames(Table table, List partitionNames) - { - throw new UnsupportedOperationException(); - } - - @Override - public void addPartitions(String databaseName, String tableName, List partitions) - { - throw new UnsupportedOperationException(); - } - - @Override - public void dropPartition(String databaseName, String tableName, List parts, boolean deleteData) - { - throw new UnsupportedOperationException(); - } - - @Override - public void alterPartition(String databaseName, String tableName, PartitionWithStatistics partition) - { - throw new UnsupportedOperationException(); - } - - @Override - public Set listTablePrivileges(String databaseName, String tableName, Optional tableOwner, Optional prestoPrincipal) - { - throw new UnsupportedOperationException(); - } - - @Override - public void grantTablePrivileges(String databaseName, String tableName, String tableOwner, HivePrincipal grantee, HivePrincipal grantor, Set privileges, boolean grantOption) - { - throw new UnsupportedOperationException(); - } - - @Override - public void revokeTablePrivileges(String databaseName, String tableName, String tableOwner, HivePrincipal grantee, HivePrincipal grantor, Set privileges, boolean grantOption) - { - throw new UnsupportedOperationException(); - } - - @Override - public void createRole(String role, String grantor) - { - throw new UnsupportedOperationException(); - } - - @Override - public void dropRole(String role) - { - throw new UnsupportedOperationException(); - } - - @Override - public Set listRoles() - { - throw new UnsupportedOperationException(); - } - - @Override - public void grantRoles(Set roles, Set grantees, boolean adminOption, HivePrincipal grantor) - { - throw new UnsupportedOperationException(); - } - - @Override - public void revokeRoles(Set roles, Set grantees, boolean adminOption, HivePrincipal grantor) - { - throw new UnsupportedOperationException(); - } - - @Override - public Set listRoleGrants(HivePrincipal principal) - { - throw new UnsupportedOperationException(); - } - - @Override - public boolean functionExists(String databaseName, String functionName, String signatureToken) - { - throw new UnsupportedOperationException(); - } - - @Override - public Collection getFunctions(String databaseName) - { - throw new UnsupportedOperationException(); - } - - @Override - public Collection getFunctions(String databaseName, String functionName) - { - throw new UnsupportedOperationException(); - } - - @Override - public void createFunction(String databaseName, String functionName, LanguageFunction function) - { - throw new UnsupportedOperationException(); - } - - @Override - public void replaceFunction(String databaseName, String functionName, LanguageFunction function) - { - throw new UnsupportedOperationException(); - } - - @Override - public void dropFunction(String databaseName, String functionName, String signatureToken) - { - throw new UnsupportedOperationException(); - } -} From bb4d8e7acd3569898bb85bc100c161df6ab04c3b Mon Sep 17 00:00:00 2001 From: Assaf Bern Date: Mon, 4 Dec 2023 07:53:41 +0200 Subject: [PATCH 016/350] Fix potential infinite loop at IterativeOptimizer with Iceberg --- .../src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java index f13626380ea60..c10307912c06f 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java @@ -2551,8 +2551,7 @@ else if (isMetadataColumnId(columnHandle.getId())) { if (newEnforcedConstraint.equals(table.getEnforcedPredicate()) && newUnenforcedConstraint.equals(table.getUnenforcedPredicate()) - && newConstraintColumns.equals(table.getConstraintColumns()) - && constraint.getPredicateColumns().isEmpty()) { + && newConstraintColumns.equals(table.getConstraintColumns())) { return Optional.empty(); } From d0dcbc49bb7a4b438deb4e98e3a604232149dbcb Mon Sep 17 00:00:00 2001 From: Kamil Endruszkiewicz Date: Wed, 6 Dec 2023 09:26:57 +0100 Subject: [PATCH 017/350] Delta lake change predicate test order for split enumeration --- .../java/io/trino/plugin/deltalake/DeltaLakeSplitSource.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSplitSource.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSplitSource.java index c4bb1da90ecd7..964361376ca95 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSplitSource.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSplitSource.java @@ -123,8 +123,8 @@ public CompletableFuture getNextBatch(int maxSize) .collect(toImmutableMap(Map.Entry::getKey, Map.Entry::getValue)); List filteredSplits = splits.stream() .map(DeltaLakeSplit.class::cast) - .filter(split -> split.getStatisticsPredicate().overlaps(dynamicFilterPredicate) && - partitionMatchesPredicate(split.getPartitionKeys(), partitionColumnDomains)) + .filter(split -> partitionMatchesPredicate(split.getPartitionKeys(), partitionColumnDomains) && + split.getStatisticsPredicate().overlaps(dynamicFilterPredicate)) .collect(toImmutableList()); if (recordScannedFiles) { filteredSplits.forEach(split -> scannedFilePaths.add(((DeltaLakeSplit) split).getPath())); From ac44709b1bac8d3c9b887ff839b6eeaeccac25b2 Mon Sep 17 00:00:00 2001 From: Gaurav Sehgal Date: Sun, 12 Nov 2023 20:20:55 -0500 Subject: [PATCH 018/350] Use concurrent hash map for tracking query memory This will increase the performance of getQueryMemoryReservation(queryId) method since that is called from every different local exchange thread. --- .../src/main/java/io/trino/memory/MemoryPool.java | 7 ++++--- .../src/main/java/io/trino/memory/QueryContext.java | 2 +- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/core/trino-main/src/main/java/io/trino/memory/MemoryPool.java b/core/trino-main/src/main/java/io/trino/memory/MemoryPool.java index b330931af945d..191179a8ae168 100644 --- a/core/trino-main/src/main/java/io/trino/memory/MemoryPool.java +++ b/core/trino-main/src/main/java/io/trino/memory/MemoryPool.java @@ -31,6 +31,7 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CopyOnWriteArrayList; import static com.google.common.base.MoreObjects.toStringHelper; @@ -53,9 +54,9 @@ public class MemoryPool @GuardedBy("this") private NonCancellableMemoryFuture future; - @GuardedBy("this") // TODO: It would be better if we just tracked QueryContexts, but their lifecycle is managed by a weak reference, so we can't do that - private final Map queryMemoryReservations = new HashMap<>(); + // It is guarded for updates by this, but can be read without holding a lock + private final Map queryMemoryReservations = new ConcurrentHashMap<>(); // This map keeps track of all the tagged allocations, e.g., query-1 -> ['TableScanOperator': 10MB, 'LazyOutputBuffer': 5MB, ...] @GuardedBy("this") @@ -347,7 +348,7 @@ public synchronized long getReservedRevocableBytes() return reservedRevocableBytes; } - synchronized long getQueryMemoryReservation(QueryId queryId) + long getQueryMemoryReservation(QueryId queryId) { return queryMemoryReservations.getOrDefault(queryId, 0L); } diff --git a/core/trino-main/src/main/java/io/trino/memory/QueryContext.java b/core/trino-main/src/main/java/io/trino/memory/QueryContext.java index c30e3ad282f71..0c7711e8d139c 100644 --- a/core/trino-main/src/main/java/io/trino/memory/QueryContext.java +++ b/core/trino-main/src/main/java/io/trino/memory/QueryContext.java @@ -225,7 +225,7 @@ public synchronized MemoryPool getMemoryPool() return memoryPool; } - public synchronized long getUserMemoryReservation() + public long getUserMemoryReservation() { return memoryPool.getQueryMemoryReservation(queryId); } From 60f3e632b2e590abbbbf83589fc7d3cfd910f4c8 Mon Sep 17 00:00:00 2001 From: Karol Sobczak Date: Wed, 6 Dec 2023 12:23:46 +0100 Subject: [PATCH 019/350] Do not synch on getting memory pool since it's final --- core/trino-main/src/main/java/io/trino/memory/QueryContext.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/trino-main/src/main/java/io/trino/memory/QueryContext.java b/core/trino-main/src/main/java/io/trino/memory/QueryContext.java index 0c7711e8d139c..571771987181b 100644 --- a/core/trino-main/src/main/java/io/trino/memory/QueryContext.java +++ b/core/trino-main/src/main/java/io/trino/memory/QueryContext.java @@ -220,7 +220,7 @@ public synchronized void freeSpill(long bytes) spillSpaceTracker.free(bytes); } - public synchronized MemoryPool getMemoryPool() + public MemoryPool getMemoryPool() { return memoryPool; } From c95b3629e8e688ae9bdb03d98110e9eaaabfe608 Mon Sep 17 00:00:00 2001 From: Gaurav Sehgal Date: Tue, 21 Nov 2023 15:36:32 -0500 Subject: [PATCH 020/350] Update scaling state only if memory is below limit Only update the scaling state if the memory used is below the limit. Otherwise, if we keep updating the scaling state and the memory used is fluctuating around the limit, then we could do massive scaling in a single rebalancing cycle which could cause OOM error. --- .../ScaleWriterPartitioningExchanger.java | 39 +++++--- .../operator/exchange/TestLocalExchange.java | 99 ++++++++++++++++++- 2 files changed, 122 insertions(+), 16 deletions(-) diff --git a/core/trino-main/src/main/java/io/trino/operator/exchange/ScaleWriterPartitioningExchanger.java b/core/trino-main/src/main/java/io/trino/operator/exchange/ScaleWriterPartitioningExchanger.java index c4f7e56fcc469..8bc67b4d0b5c9 100644 --- a/core/trino-main/src/main/java/io/trino/operator/exchange/ScaleWriterPartitioningExchanger.java +++ b/core/trino-main/src/main/java/io/trino/operator/exchange/ScaleWriterPartitioningExchanger.java @@ -31,6 +31,7 @@ public class ScaleWriterPartitioningExchanger implements LocalExchanger { + private static final double SCALE_WRITER_MEMORY_PERCENTAGE = 0.7; private final List> buffers; private final LocalExchangeMemoryManager memoryManager; private final long maxBufferedBytes; @@ -83,11 +84,18 @@ public ScaleWriterPartitioningExchanger( @Override public void accept(Page page) { + // Reset the value of partition row count, writer ids and data processed for this page + long dataProcessed = 0; + for (int partitionId = 0; partitionId < partitionRowCounts.length; partitionId++) { + partitionRowCounts[partitionId] = 0; + partitionWriterIds[partitionId] = -1; + } + // Scale up writers when current buffer memory utilization is more than 50% of the maximum. - // Do not scale up if total memory used is greater than 50% of max memory per node. + // Do not scale up if total memory used is greater than 70% of max memory per node. // We have to be conservative here otherwise scaling of writers will happen first // before we hit this limit, and then we won't be able to do anything to stop OOM error. - if (memoryManager.getBufferedBytes() > maxBufferedBytes * 0.5 && totalMemoryUsed.get() < maxMemoryPerNode * 0.5) { + if (memoryManager.getBufferedBytes() > maxBufferedBytes * 0.5 && totalMemoryUsed.get() < maxMemoryPerNode * SCALE_WRITER_MEMORY_PERCENTAGE) { partitionRebalancer.rebalance(); } @@ -111,13 +119,6 @@ public void accept(Page page) writerAssignments[writerId].add(position); } - for (int partitionId = 0; partitionId < partitionRowCounts.length; partitionId++) { - partitionRebalancer.addPartitionRowCount(partitionId, partitionRowCounts[partitionId]); - // Reset the value of partition row count - partitionRowCounts[partitionId] = 0; - partitionWriterIds[partitionId] = -1; - } - // build a page for each writer for (int bucket = 0; bucket < writerAssignments.length; bucket++) { IntArrayList positionsList = writerAssignments[bucket]; @@ -135,12 +136,22 @@ public void accept(Page page) // whole input page will go to this partition, compact the input page avoid over-retaining memory and to // match the behavior of sub-partitioned pages that copy positions out page.compact(); - sendPageToPartition(buffers.get(bucket), page); - return; + dataProcessed += sendPageToPartition(buffers.get(bucket), page); + break; } Page pageSplit = page.copyPositions(positions, 0, bucketSize); - sendPageToPartition(buffers.get(bucket), pageSplit); + dataProcessed += sendPageToPartition(buffers.get(bucket), pageSplit); + } + + // Only update the scaling state if the memory used is below the SCALE_WRITER_MEMORY_PERCENTAGE limit. Otherwise, if we keep updating + // the scaling state and the memory used is fluctuating around the limit, then we could do massive scaling + // in a single rebalancing cycle which could cause OOM error. + if (totalMemoryUsed.get() < maxMemoryPerNode * SCALE_WRITER_MEMORY_PERCENTAGE) { + for (int partitionId = 0; partitionId < partitionRowCounts.length; partitionId++) { + partitionRebalancer.addPartitionRowCount(partitionId, partitionRowCounts[partitionId]); + } + partitionRebalancer.addDataProcessed(dataProcessed); } } @@ -155,11 +166,11 @@ private int getNextWriterId(int partitionId) return partitionRebalancer.getTaskId(partitionId, partitionWriterIndexes[partitionId]++); } - private void sendPageToPartition(Consumer buffer, Page pageSplit) + private long sendPageToPartition(Consumer buffer, Page pageSplit) { long retainedSizeInBytes = pageSplit.getRetainedSizeInBytes(); - partitionRebalancer.addDataProcessed(retainedSizeInBytes); memoryManager.updateMemoryUsage(retainedSizeInBytes); buffer.accept(pageSplit); + return retainedSizeInBytes; } } diff --git a/core/trino-main/src/test/java/io/trino/operator/exchange/TestLocalExchange.java b/core/trino-main/src/test/java/io/trino/operator/exchange/TestLocalExchange.java index 92deb657a8fe4..e1485f8a778e0 100644 --- a/core/trino-main/src/test/java/io/trino/operator/exchange/TestLocalExchange.java +++ b/core/trino-main/src/test/java/io/trino/operator/exchange/TestLocalExchange.java @@ -702,13 +702,13 @@ public void testNoScalingWhenTotalMemoryUsedIsGreaterThanLimit(PartitioningHandl sink.addPage(createSingleValuePage(0, 1000)); sink.addPage(createSingleValuePage(0, 1000)); - // Scaling since total memory used is less than 10 MBs + // Scaling since total memory used is less than 14 MBs (20 MBs * 70%) assertSource(sourceA, 2); assertSource(sourceB, 2); assertSource(sourceC, 0); assertSource(sourceD, 4); - totalMemoryUsed.set(DataSize.of(13, MEGABYTE).toBytes()); + totalMemoryUsed.set(DataSize.of(15, MEGABYTE).toBytes()); sink.addPage(createSingleValuePage(0, 1000)); sink.addPage(createSingleValuePage(0, 1000)); @@ -723,6 +723,101 @@ public void testNoScalingWhenTotalMemoryUsedIsGreaterThanLimit(PartitioningHandl }); } + @Test(dataProvider = "scalingPartitionHandles") + public void testDoNotUpdateScalingStateWhenMemoryIsAboveLimit(PartitioningHandle partitioningHandle) + { + AtomicLong totalMemoryUsed = new AtomicLong(); + LocalExchange localExchange = new LocalExchange( + nodePartitioningManager, + testSessionBuilder() + .setSystemProperty(SKEWED_PARTITION_MIN_DATA_PROCESSED_REBALANCE_THRESHOLD, "20kB") + .setSystemProperty(QUERY_MAX_MEMORY_PER_NODE, "20MB") + .build(), + 4, + partitioningHandle, + ImmutableList.of(0), + TYPES, + Optional.empty(), + DataSize.ofBytes(retainedSizeOfPages(2)), + TYPE_OPERATORS, + DataSize.of(10, KILOBYTE), + totalMemoryUsed::get); + + run(localExchange, exchange -> { + assertThat(exchange.getBufferCount()).isEqualTo(4); + assertExchangeTotalBufferedBytes(exchange, 0); + + LocalExchangeSinkFactory sinkFactory = exchange.createSinkFactory(); + sinkFactory.noMoreSinkFactories(); + LocalExchangeSink sink = sinkFactory.createSink(); + assertSinkCanWrite(sink); + sinkFactory.close(); + + LocalExchangeSource sourceA = exchange.getNextSource(); + assertSource(sourceA, 0); + + LocalExchangeSource sourceB = exchange.getNextSource(); + assertSource(sourceB, 0); + + LocalExchangeSource sourceC = exchange.getNextSource(); + assertSource(sourceC, 0); + + LocalExchangeSource sourceD = exchange.getNextSource(); + assertSource(sourceD, 0); + + sink.addPage(createSingleValuePage(0, 1000)); + sink.addPage(createSingleValuePage(0, 1000)); + sink.addPage(createSingleValuePage(1, 2)); + sink.addPage(createSingleValuePage(1, 2)); + + // Two partitions are assigned to two different writers + assertSource(sourceA, 2); + assertSource(sourceB, 0); + assertSource(sourceC, 0); + assertSource(sourceD, 2); + + totalMemoryUsed.set(DataSize.of(5, MEGABYTE).toBytes()); + + sink.addPage(createSingleValuePage(0, 1000)); + sink.addPage(createSingleValuePage(0, 1000)); + sink.addPage(createSingleValuePage(0, 1000)); + sink.addPage(createSingleValuePage(0, 1000)); + + // Scaling since total memory used is less than 14 MBs (20 MBs * 70%) + assertSource(sourceA, 2); + assertSource(sourceB, 2); + assertSource(sourceC, 0); + assertSource(sourceD, 4); + + totalMemoryUsed.set(DataSize.of(15, MEGABYTE).toBytes()); + + sink.addPage(createSingleValuePage(0, 1000)); + sink.addPage(createSingleValuePage(0, 1000)); + sink.addPage(createSingleValuePage(0, 1000)); + sink.addPage(createSingleValuePage(0, 1000)); + + // No scaling since total memory used is greater than 14 MBs (20 MBs * 70%) + assertSource(sourceA, 2); + assertSource(sourceB, 4); + assertSource(sourceC, 0); + assertSource(sourceD, 6); + + // Memory reduced due to closing of some writers + totalMemoryUsed.set(DataSize.of(13, MEGABYTE).toBytes()); + + sink.addPage(createSingleValuePage(0, 10)); + sink.addPage(createSingleValuePage(0, 10)); + sink.addPage(createSingleValuePage(0, 10)); + sink.addPage(createSingleValuePage(0, 10)); + // No scaling since not enough data has been processed, and we are not considering data written + // when memory utilization is above the limit. + assertSource(sourceA, 3); + assertSource(sourceB, 6); + assertSource(sourceC, 0); + assertSource(sourceD, 7); + }); + } + @Test(dataProvider = "scalingPartitionHandles") public void testNoScalingWhenMaxScaledPartitionsPerTaskIsSmall(PartitioningHandle partitioningHandle) { From 77cb0d6c89ae9f485e4488933b706ca3720973e0 Mon Sep 17 00:00:00 2001 From: Gaurav Sehgal Date: Sun, 12 Nov 2023 21:18:45 -0500 Subject: [PATCH 021/350] Remove max scaling limit from local and remote exchange --- .../operator/exchange/LocalExchange.java | 9 +- .../output/SkewedPartitionRebalancer.java | 26 +---- .../sql/planner/LocalExecutionPlanner.java | 7 +- .../operator/exchange/TestLocalExchange.java | 109 ------------------ .../output/TestSkewedPartitionRebalancer.java | 83 +------------ 5 files changed, 10 insertions(+), 224 deletions(-) diff --git a/core/trino-main/src/main/java/io/trino/operator/exchange/LocalExchange.java b/core/trino-main/src/main/java/io/trino/operator/exchange/LocalExchange.java index ea5486a637509..cda37c84c4785 100644 --- a/core/trino-main/src/main/java/io/trino/operator/exchange/LocalExchange.java +++ b/core/trino-main/src/main/java/io/trino/operator/exchange/LocalExchange.java @@ -53,7 +53,6 @@ import static io.trino.SystemSessionProperties.getSkewedPartitionMinDataProcessedRebalanceThreshold; import static io.trino.operator.InterpretedHashGenerator.createChannelsHashGenerator; import static io.trino.operator.exchange.LocalExchangeSink.finishedLocalExchangeSink; -import static io.trino.operator.output.SkewedPartitionRebalancer.getScaleWritersMaxSkewedPartitions; import static io.trino.sql.planner.PartitioningHandle.isScaledWriterHashDistribution; import static io.trino.sql.planner.SystemPartitioningHandle.FIXED_ARBITRARY_DISTRIBUTION; import static io.trino.sql.planner.SystemPartitioningHandle.FIXED_BROADCAST_DISTRIBUTION; @@ -61,7 +60,6 @@ import static io.trino.sql.planner.SystemPartitioningHandle.FIXED_PASSTHROUGH_DISTRIBUTION; import static io.trino.sql.planner.SystemPartitioningHandle.SCALED_WRITER_ROUND_ROBIN_DISTRIBUTION; import static io.trino.sql.planner.SystemPartitioningHandle.SINGLE_DISTRIBUTION; -import static java.lang.Math.max; import static java.util.Objects.requireNonNull; import static java.util.function.Function.identity; @@ -147,12 +145,7 @@ else if (isScaledWriterHashDistribution(partitioning)) { bufferCount, 1, writerScalingMinDataProcessed.toBytes(), - getSkewedPartitionMinDataProcessedRebalanceThreshold(session).toBytes(), - // Keep the maxPartitionsToRebalance to atleast writer count such that single partition writes do - // not suffer from skewness and can scale uniformly across all writers. Additionally, note that - // maxWriterCount is calculated considering memory into account. So, it is safe to set the - // maxPartitionsToRebalance to maximum number of writers. - max(getScaleWritersMaxSkewedPartitions(session), bufferCount)); + getSkewedPartitionMinDataProcessedRebalanceThreshold(session).toBytes()); LocalExchangeMemoryManager memoryManager = new LocalExchangeMemoryManager(maxBufferedBytes.toBytes()); sources = IntStream.range(0, bufferCount) .mapToObj(i -> new LocalExchangeSource(memoryManager, source -> checkAllSourcesFinished())) diff --git a/core/trino-main/src/main/java/io/trino/operator/output/SkewedPartitionRebalancer.java b/core/trino-main/src/main/java/io/trino/operator/output/SkewedPartitionRebalancer.java index 458b4ccd17d37..799106f08d242 100644 --- a/core/trino-main/src/main/java/io/trino/operator/output/SkewedPartitionRebalancer.java +++ b/core/trino-main/src/main/java/io/trino/operator/output/SkewedPartitionRebalancer.java @@ -32,7 +32,6 @@ import java.util.List; import java.util.Objects; import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLongArray; import java.util.stream.IntStream; @@ -86,12 +85,10 @@ public class SkewedPartitionRebalancer private final int taskBucketCount; private final long minPartitionDataProcessedRebalanceThreshold; private final long minDataProcessedRebalanceThreshold; - private final int maxPartitionsToRebalance; private final AtomicLongArray partitionRowCount; private final AtomicLong dataProcessed; private final AtomicLong dataProcessedAtLastRebalance; - private final AtomicInteger numOfRebalancedPartitions; @GuardedBy("this") private final long[] partitionDataSize; @@ -158,12 +155,6 @@ public static int getMaxWritersBasedOnMemory(Session session) return (int) ceil((double) getQueryMaxMemoryPerNode(session).toBytes() / getMaxMemoryPerPartitionWriter(session).toBytes()); } - public static int getScaleWritersMaxSkewedPartitions(Session session) - { - // Set the value of maxSkewedPartitions to scale to 60% of maximum number of writers possible per node. - return (int) (getMaxWritersBasedOnMemory(session) * 0.60); - } - public static int getTaskCount(PartitioningScheme partitioningScheme) { // Todo: Handle skewness if there are more nodes/tasks than the buckets coming from connector @@ -179,20 +170,17 @@ public SkewedPartitionRebalancer( int taskCount, int taskBucketCount, long minPartitionDataProcessedRebalanceThreshold, - long maxDataProcessedRebalanceThreshold, - int maxPartitionsToRebalance) + long maxDataProcessedRebalanceThreshold) { this.partitionCount = partitionCount; this.taskCount = taskCount; this.taskBucketCount = taskBucketCount; this.minPartitionDataProcessedRebalanceThreshold = minPartitionDataProcessedRebalanceThreshold; this.minDataProcessedRebalanceThreshold = max(minPartitionDataProcessedRebalanceThreshold, maxDataProcessedRebalanceThreshold); - this.maxPartitionsToRebalance = maxPartitionsToRebalance; this.partitionRowCount = new AtomicLongArray(partitionCount); this.dataProcessed = new AtomicLong(); this.dataProcessedAtLastRebalance = new AtomicLong(); - this.numOfRebalancedPartitions = new AtomicInteger(); this.partitionDataSize = new long[partitionCount]; this.partitionDataSizeAtLastRebalance = new long[partitionCount]; @@ -254,9 +242,7 @@ public void rebalance() private boolean shouldRebalance(long dataProcessed) { // Rebalance only when total bytes processed since last rebalance is greater than rebalance threshold. - // Check if the number of rebalanced partitions is less than maxPartitionsToRebalance. - return (dataProcessed - dataProcessedAtLastRebalance.get()) >= minDataProcessedRebalanceThreshold - && numOfRebalancedPartitions.get() < maxPartitionsToRebalance; + return (dataProcessed - dataProcessedAtLastRebalance.get()) >= minDataProcessedRebalanceThreshold; } private synchronized void rebalancePartitions(long dataProcessed) @@ -412,12 +398,6 @@ private boolean rebalancePartition( return false; } - // If the number of rebalanced partitions is less than maxPartitionsToRebalance then assign - // the partition to the task. - if (numOfRebalancedPartitions.get() >= maxPartitionsToRebalance) { - return false; - } - assignments.add(toTaskBucket); int newTaskCount = assignments.size(); @@ -438,8 +418,6 @@ private boolean rebalancePartition( minTasks.addOrUpdate(taskBucket, Long.MAX_VALUE - estimatedTaskBucketDataSizeSinceLastRebalance[taskBucket.id]); } - // Increment the number of rebalanced partitions. - numOfRebalancedPartitions.incrementAndGet(); log.debug("Rebalanced partition %s to task %s with taskCount %s", partitionId, toTaskBucket.taskId, assignments.size()); return true; } diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/LocalExecutionPlanner.java b/core/trino-main/src/main/java/io/trino/sql/planner/LocalExecutionPlanner.java index c333b05615e6a..6de51b87e18f4 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/LocalExecutionPlanner.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/LocalExecutionPlanner.java @@ -333,7 +333,6 @@ import static io.trino.operator.output.SkewedPartitionRebalancer.checkCanScalePartitionsRemotely; import static io.trino.operator.output.SkewedPartitionRebalancer.createPartitionFunction; import static io.trino.operator.output.SkewedPartitionRebalancer.getMaxWritersBasedOnMemory; -import static io.trino.operator.output.SkewedPartitionRebalancer.getScaleWritersMaxSkewedPartitions; import static io.trino.operator.output.SkewedPartitionRebalancer.getTaskCount; import static io.trino.operator.window.pattern.PhysicalValuePointer.CLASSIFIER; import static io.trino.operator.window.pattern.PhysicalValuePointer.MATCH_NUMBER; @@ -382,7 +381,6 @@ import static io.trino.util.SpatialJoinUtils.extractSupportedSpatialComparisons; import static io.trino.util.SpatialJoinUtils.extractSupportedSpatialFunctions; import static java.lang.Math.ceil; -import static java.lang.Math.max; import static java.lang.Math.min; import static java.lang.Math.toIntExact; import static java.lang.String.format; @@ -593,10 +591,7 @@ public LocalExecutionPlan plan( taskCount, taskBucketCount, getWriterScalingMinDataProcessed(taskContext.getSession()).toBytes(), - getSkewedPartitionMinDataProcessedRebalanceThreshold(taskContext.getSession()).toBytes(), - // Keep the maxPartitionsToRebalance to atleast task count such that single partition writes do - // not suffer from skewness and can scale uniformly across all tasks. - max(getScaleWritersMaxSkewedPartitions(taskContext.getSession()), taskCount))); + getSkewedPartitionMinDataProcessedRebalanceThreshold(taskContext.getSession()).toBytes())); } else { partitionFunction = nodePartitioningManager.getPartitionFunction(taskContext.getSession(), partitioningScheme, partitionChannelTypes); diff --git a/core/trino-main/src/test/java/io/trino/operator/exchange/TestLocalExchange.java b/core/trino-main/src/test/java/io/trino/operator/exchange/TestLocalExchange.java index e1485f8a778e0..0587d0d7edc43 100644 --- a/core/trino-main/src/test/java/io/trino/operator/exchange/TestLocalExchange.java +++ b/core/trino-main/src/test/java/io/trino/operator/exchange/TestLocalExchange.java @@ -818,115 +818,6 @@ public void testDoNotUpdateScalingStateWhenMemoryIsAboveLimit(PartitioningHandle }); } - @Test(dataProvider = "scalingPartitionHandles") - public void testNoScalingWhenMaxScaledPartitionsPerTaskIsSmall(PartitioningHandle partitioningHandle) - { - LocalExchange localExchange = new LocalExchange( - nodePartitioningManager, - testSessionBuilder() - .setSystemProperty(SKEWED_PARTITION_MIN_DATA_PROCESSED_REBALANCE_THRESHOLD, "20kB") - .setSystemProperty(QUERY_MAX_MEMORY_PER_NODE, "256MB") - .build(), - 4, - partitioningHandle, - ImmutableList.of(0), - TYPES, - Optional.empty(), - DataSize.ofBytes(retainedSizeOfPages(2)), - TYPE_OPERATORS, - DataSize.of(10, KILOBYTE), - TOTAL_MEMORY_USED); - - run(localExchange, exchange -> { - assertThat(exchange.getBufferCount()).isEqualTo(4); - assertExchangeTotalBufferedBytes(exchange, 0); - - LocalExchangeSinkFactory sinkFactory = exchange.createSinkFactory(); - sinkFactory.noMoreSinkFactories(); - LocalExchangeSink sink = sinkFactory.createSink(); - assertSinkCanWrite(sink); - sinkFactory.close(); - - LocalExchangeSource sourceA = exchange.getNextSource(); - assertSource(sourceA, 0); - - LocalExchangeSource sourceB = exchange.getNextSource(); - assertSource(sourceB, 0); - - LocalExchangeSource sourceC = exchange.getNextSource(); - assertSource(sourceC, 0); - - LocalExchangeSource sourceD = exchange.getNextSource(); - assertSource(sourceD, 0); - - sink.addPage(createSingleValuePage(0, 1000)); - sink.addPage(createSingleValuePage(0, 1000)); - sink.addPage(createSingleValuePage(1, 2)); - sink.addPage(createSingleValuePage(1, 2)); - - // Two partitions are assigned to two different writers - assertSource(sourceA, 2); - assertSource(sourceB, 0); - assertSource(sourceC, 0); - assertSource(sourceD, 2); - - sink.addPage(createSingleValuePage(0, 1000)); - sink.addPage(createSingleValuePage(0, 1000)); - sink.addPage(createSingleValuePage(0, 1000)); - sink.addPage(createSingleValuePage(0, 1000)); - - // partition 0 is assigned to writer B after scaling. - assertSource(sourceA, 2); - assertSource(sourceB, 2); - assertSource(sourceC, 0); - assertSource(sourceD, 4); - - sink.addPage(createSingleValuePage(0, 1000)); - sink.addPage(createSingleValuePage(0, 1000)); - sink.addPage(createSingleValuePage(0, 1000)); - sink.addPage(createSingleValuePage(0, 1000)); - - // partition 0 is assigned to writer A after scaling. - assertSource(sourceA, 3); - assertSource(sourceB, 4); - assertSource(sourceC, 0); - assertSource(sourceD, 5); - - sink.addPage(createSingleValuePage(0, 1000)); - sink.addPage(createSingleValuePage(0, 1000)); - sink.addPage(createSingleValuePage(0, 1000)); - sink.addPage(createSingleValuePage(0, 1000)); - - // partition 0 is assigned to writer C after scaling. - assertSource(sourceA, 4); - assertSource(sourceB, 5); - assertSource(sourceC, 1); - assertSource(sourceD, 6); - - sink.addPage(createSingleValuePage(1, 10000)); - sink.addPage(createSingleValuePage(1, 10000)); - sink.addPage(createSingleValuePage(1, 10000)); - sink.addPage(createSingleValuePage(1, 10000)); - - // partition 1 is assigned to writer B after scaling. - assertSource(sourceA, 6); - assertSource(sourceB, 7); - assertSource(sourceC, 1); - assertSource(sourceD, 6); - - sink.addPage(createSingleValuePage(1, 10000)); - sink.addPage(createSingleValuePage(1, 10000)); - sink.addPage(createSingleValuePage(1, 10000)); - sink.addPage(createSingleValuePage(1, 10000)); - - // no scaling will happen since we have scaled to maximum limit which is the number of writer count. - assertSource(sourceA, 8); - assertSource(sourceB, 9); - assertSource(sourceC, 1); - assertSource(sourceD, 6); - }); - } - @Test public void testNoScalingWhenNoWriterSkewness() { diff --git a/core/trino-main/src/test/java/io/trino/operator/output/TestSkewedPartitionRebalancer.java b/core/trino-main/src/test/java/io/trino/operator/output/TestSkewedPartitionRebalancer.java index 5a61bf221287e..548f25b6da200 100644 --- a/core/trino-main/src/test/java/io/trino/operator/output/TestSkewedPartitionRebalancer.java +++ b/core/trino-main/src/test/java/io/trino/operator/output/TestSkewedPartitionRebalancer.java @@ -32,7 +32,6 @@ class TestSkewedPartitionRebalancer { private static final long MIN_PARTITION_DATA_PROCESSED_REBALANCE_THRESHOLD = DataSize.of(1, MEGABYTE).toBytes(); private static final long MIN_DATA_PROCESSED_REBALANCE_THRESHOLD = DataSize.of(50, MEGABYTE).toBytes(); - private static final int MAX_REBALANCED_PARTITIONS = 30; @Test void testRebalanceWithSkewness() @@ -43,8 +42,7 @@ void testRebalanceWithSkewness() 3, 3, MIN_PARTITION_DATA_PROCESSED_REBALANCE_THRESHOLD, - MIN_DATA_PROCESSED_REBALANCE_THRESHOLD, - MAX_REBALANCED_PARTITIONS); + MIN_DATA_PROCESSED_REBALANCE_THRESHOLD); SkewedPartitionFunction function = new SkewedPartitionFunction(new TestPartitionFunction(partitionCount), rebalancer); rebalancer.addPartitionRowCount(0, 1000); @@ -104,8 +102,7 @@ void testRebalanceWithoutSkewness() 3, 2, MIN_PARTITION_DATA_PROCESSED_REBALANCE_THRESHOLD, - MIN_DATA_PROCESSED_REBALANCE_THRESHOLD, - MAX_REBALANCED_PARTITIONS); + MIN_DATA_PROCESSED_REBALANCE_THRESHOLD); SkewedPartitionFunction function = new SkewedPartitionFunction(new TestPartitionFunction(partitionCount), rebalancer); rebalancer.addPartitionRowCount(0, 1000); @@ -136,8 +133,7 @@ void testNoRebalanceWhenDataWrittenIsLessThanTheRebalanceLimit() 3, 3, MIN_PARTITION_DATA_PROCESSED_REBALANCE_THRESHOLD, - MIN_DATA_PROCESSED_REBALANCE_THRESHOLD, - MAX_REBALANCED_PARTITIONS); + MIN_DATA_PROCESSED_REBALANCE_THRESHOLD); SkewedPartitionFunction function = new SkewedPartitionFunction(new TestPartitionFunction(partitionCount), rebalancer); rebalancer.addPartitionRowCount(0, 1000); @@ -166,8 +162,7 @@ void testNoRebalanceWhenDataWrittenByThePartitionIsLessThanWriterScalingMinDataP 3, 3, minPartitionDataProcessedRebalanceThreshold, - MIN_DATA_PROCESSED_REBALANCE_THRESHOLD, - MAX_REBALANCED_PARTITIONS); + MIN_DATA_PROCESSED_REBALANCE_THRESHOLD); SkewedPartitionFunction function = new SkewedPartitionFunction(new TestPartitionFunction(partitionCount), rebalancer); rebalancer.addPartitionRowCount(0, 1000); @@ -195,8 +190,7 @@ void testRebalancePartitionToSingleTaskInARebalancingLoop() 3, 3, MIN_PARTITION_DATA_PROCESSED_REBALANCE_THRESHOLD, - MIN_DATA_PROCESSED_REBALANCE_THRESHOLD, - MAX_REBALANCED_PARTITIONS); + MIN_DATA_PROCESSED_REBALANCE_THRESHOLD); SkewedPartitionFunction function = new SkewedPartitionFunction(new TestPartitionFunction(partitionCount), rebalancer); rebalancer.addPartitionRowCount(0, 1000); @@ -240,8 +234,7 @@ public void testConsiderSkewedPartitionOnlyWithinACycle() 3, 1, MIN_PARTITION_DATA_PROCESSED_REBALANCE_THRESHOLD, - MIN_DATA_PROCESSED_REBALANCE_THRESHOLD, - MAX_REBALANCED_PARTITIONS); + MIN_DATA_PROCESSED_REBALANCE_THRESHOLD); SkewedPartitionFunction function = new SkewedPartitionFunction( new TestPartitionFunction(partitionCount), rebalancer); @@ -280,70 +273,6 @@ public void testConsiderSkewedPartitionOnlyWithinACycle() .containsExactly(ImmutableList.of(0, 2), ImmutableList.of(1), ImmutableList.of(2, 0)); } - @Test - public void testRebalancePartitionWithMaxRebalancedPartitionsPerTask() - { - int partitionCount = 3; - SkewedPartitionRebalancer rebalancer = new SkewedPartitionRebalancer( - partitionCount, - 3, - 3, - MIN_PARTITION_DATA_PROCESSED_REBALANCE_THRESHOLD, - MIN_DATA_PROCESSED_REBALANCE_THRESHOLD, - 2); - SkewedPartitionFunction function = new SkewedPartitionFunction( - new TestPartitionFunction(partitionCount), - rebalancer); - - rebalancer.addPartitionRowCount(0, 1000); - rebalancer.addPartitionRowCount(1, 1000); - rebalancer.addPartitionRowCount(2, 1000); - rebalancer.addDataProcessed(DataSize.of(40, MEGABYTE).toBytes()); - - // rebalancing will only happen to single task even though two tasks are available - rebalancer.rebalance(); - - assertThat(getPartitionPositions(function, 17)) - .containsExactly( - new IntArrayList(ImmutableList.of(0, 3, 6, 9, 12, 15)), - new IntArrayList(ImmutableList.of(1, 4, 7, 10, 13, 16)), - new IntArrayList(ImmutableList.of(2, 5, 8, 11, 14))); - assertThat(rebalancer.getPartitionAssignments()) - .containsExactly(ImmutableList.of(0), ImmutableList.of(1), ImmutableList.of(2)); - - rebalancer.addPartitionRowCount(0, 1000); - rebalancer.addPartitionRowCount(1, 1000); - rebalancer.addPartitionRowCount(2, 1000); - rebalancer.addDataProcessed(DataSize.of(20, MEGABYTE).toBytes()); - // Rebalancing will happen since we crossed the data processed limit. - // Part0 -> Task1 (Bucket1), Part1 -> Task0 (Bucket1) - rebalancer.rebalance(); - - assertThat(getPartitionPositions(function, 17)) - .containsExactly( - new IntArrayList(ImmutableList.of(0, 4, 6, 10, 12, 16)), - new IntArrayList(ImmutableList.of(1, 3, 7, 9, 13, 15)), - new IntArrayList(ImmutableList.of(2, 5, 8, 11, 14))); - assertThat(rebalancer.getPartitionAssignments()) - .containsExactly(ImmutableList.of(0, 1), ImmutableList.of(1, 0), ImmutableList.of(2)); - - rebalancer.addPartitionRowCount(0, 1000); - rebalancer.addPartitionRowCount(1, 1000); - rebalancer.addPartitionRowCount(2, 1000); - rebalancer.addDataProcessed(DataSize.of(200, MEGABYTE).toBytes()); - - // No rebalancing will happen since we crossed the max rebalanced partitions limit. - rebalancer.rebalance(); - - assertThat(getPartitionPositions(function, 17)) - .containsExactly( - new IntArrayList(ImmutableList.of(0, 4, 6, 10, 12, 16)), - new IntArrayList(ImmutableList.of(1, 3, 7, 9, 13, 15)), - new IntArrayList(ImmutableList.of(2, 5, 8, 11, 14))); - assertThat(rebalancer.getPartitionAssignments()) - .containsExactly(ImmutableList.of(0, 1), ImmutableList.of(1, 0), ImmutableList.of(2)); - } - private static List> getPartitionPositions(PartitionFunction function, int maxPosition) { List> partitionPositions = new ArrayList<>(); From b5d978de8414071fc4ec75a34fa48ba6f894f48e Mon Sep 17 00:00:00 2001 From: Gaurav Sehgal Date: Tue, 21 Nov 2023 16:54:59 -0500 Subject: [PATCH 022/350] Limit partitionDataSizeSinceLastRebalance to 0 Since we estimate the partitionDataSize based on partitionRowCount and total data processed. It is possible that the estimated partitionDataSize is slightly less than it was estimated at the last rebalance cycle. That's because for a given partition, row count hasn't increased, however overall data processed has increased. Therefore, we need to make sure that the estimated partitionDataSize since last rebalance is always greater than 0. Otherwise, it will affect the ordering of minTaskBuckets priority queue. --- .../output/SkewedPartitionRebalancer.java | 10 +- .../operator/exchange/TestLocalExchange.java | 109 ++++++++++++++++++ 2 files changed, 118 insertions(+), 1 deletion(-) diff --git a/core/trino-main/src/main/java/io/trino/operator/output/SkewedPartitionRebalancer.java b/core/trino-main/src/main/java/io/trino/operator/output/SkewedPartitionRebalancer.java index 799106f08d242..97a839b74b27c 100644 --- a/core/trino-main/src/main/java/io/trino/operator/output/SkewedPartitionRebalancer.java +++ b/core/trino-main/src/main/java/io/trino/operator/output/SkewedPartitionRebalancer.java @@ -303,7 +303,15 @@ private void calculatePartitionDataSize(long dataProcessed) } for (int partition = 0; partition < partitionCount; partition++) { - partitionDataSize[partition] = (partitionRowCount.get(partition) * dataProcessed) / totalPartitionRowCount; + // Since we estimate the partitionDataSize based on partitionRowCount and total data processed. It is possible + // that the estimated partitionDataSize is slightly less than it was estimated at the last rebalance cycle. + // That's because for a given partition, row count hasn't increased, however overall data processed + // has increased. Therefore, we need to make sure that the estimated partitionDataSize should be + // at least partitionDataSizeAtLastRebalance. Otherwise, it will affect the ordering of minTaskBuckets + // priority queue. + partitionDataSize[partition] = max( + (partitionRowCount.get(partition) * dataProcessed) / totalPartitionRowCount, + partitionDataSize[partition]); } } diff --git a/core/trino-main/src/test/java/io/trino/operator/exchange/TestLocalExchange.java b/core/trino-main/src/test/java/io/trino/operator/exchange/TestLocalExchange.java index 0587d0d7edc43..6f5b3763addc3 100644 --- a/core/trino-main/src/test/java/io/trino/operator/exchange/TestLocalExchange.java +++ b/core/trino-main/src/test/java/io/trino/operator/exchange/TestLocalExchange.java @@ -332,6 +332,115 @@ public void testNoWriterScalingWhenOnlyBufferSizeLimitIsExceeded() }); } + @Test(dataProvider = "scalingPartitionHandles") + public void testScalingWithTwoDifferentPartitions(PartitioningHandle partitioningHandle) + { + LocalExchange localExchange = new LocalExchange( + nodePartitioningManager, + testSessionBuilder() + .setSystemProperty(SKEWED_PARTITION_MIN_DATA_PROCESSED_REBALANCE_THRESHOLD, "20kB") + .setSystemProperty(QUERY_MAX_MEMORY_PER_NODE, "256MB") + .build(), + 4, + partitioningHandle, + ImmutableList.of(0), + TYPES, + Optional.empty(), + DataSize.ofBytes(retainedSizeOfPages(2)), + TYPE_OPERATORS, + DataSize.of(10, KILOBYTE), + TOTAL_MEMORY_USED); + + run(localExchange, exchange -> { + assertThat(exchange.getBufferCount()).isEqualTo(4); + assertExchangeTotalBufferedBytes(exchange, 0); + + LocalExchangeSinkFactory sinkFactory = exchange.createSinkFactory(); + sinkFactory.noMoreSinkFactories(); + LocalExchangeSink sink = sinkFactory.createSink(); + assertSinkCanWrite(sink); + sinkFactory.close(); + + LocalExchangeSource sourceA = exchange.getNextSource(); + assertSource(sourceA, 0); + + LocalExchangeSource sourceB = exchange.getNextSource(); + assertSource(sourceB, 0); + + LocalExchangeSource sourceC = exchange.getNextSource(); + assertSource(sourceC, 0); + + LocalExchangeSource sourceD = exchange.getNextSource(); + assertSource(sourceD, 0); + + sink.addPage(createSingleValuePage(0, 1000)); + sink.addPage(createSingleValuePage(0, 1000)); + sink.addPage(createSingleValuePage(1, 2)); + sink.addPage(createSingleValuePage(1, 2)); + + // Two partitions are assigned to two different writers + assertSource(sourceA, 2); + assertSource(sourceB, 0); + assertSource(sourceC, 0); + assertSource(sourceD, 2); + + sink.addPage(createSingleValuePage(0, 1000)); + sink.addPage(createSingleValuePage(0, 1000)); + sink.addPage(createSingleValuePage(0, 1000)); + sink.addPage(createSingleValuePage(0, 1000)); + + // partition 0 is assigned to writer B after scaling. + assertSource(sourceA, 2); + assertSource(sourceB, 2); + assertSource(sourceC, 0); + assertSource(sourceD, 4); + + sink.addPage(createSingleValuePage(0, 1000)); + sink.addPage(createSingleValuePage(0, 1000)); + sink.addPage(createSingleValuePage(0, 1000)); + sink.addPage(createSingleValuePage(0, 1000)); + + // partition 0 is assigned to writer A after scaling. + assertSource(sourceA, 3); + assertSource(sourceB, 4); + assertSource(sourceC, 0); + assertSource(sourceD, 5); + + sink.addPage(createSingleValuePage(0, 1000)); + sink.addPage(createSingleValuePage(0, 1000)); + sink.addPage(createSingleValuePage(0, 1000)); + sink.addPage(createSingleValuePage(0, 1000)); + + // partition 0 is assigned to writer C after scaling. + assertSource(sourceA, 4); + assertSource(sourceB, 5); + assertSource(sourceC, 1); + assertSource(sourceD, 6); + + sink.addPage(createSingleValuePage(1, 10000)); + sink.addPage(createSingleValuePage(1, 10000)); + sink.addPage(createSingleValuePage(1, 10000)); + sink.addPage(createSingleValuePage(1, 10000)); + + // partition 1 is assigned to writer B after scaling. + assertSource(sourceA, 5); + assertSource(sourceB, 8); + assertSource(sourceC, 1); + assertSource(sourceD, 6); + + sink.addPage(createSingleValuePage(1, 10000)); + sink.addPage(createSingleValuePage(1, 10000)); + sink.addPage(createSingleValuePage(1, 10000)); + sink.addPage(createSingleValuePage(1, 10000)); + + // partition 1 is assigned to writer C and D after scaling. + assertSource(sourceA, 6); + assertSource(sourceB, 9); + assertSource(sourceC, 2); + assertSource(sourceD, 7); + }); + } + @Test public void testScaledWriterRoundRobinExchangerWhenTotalMemoryUsedIsGreaterThanLimit() { From 9141bf18d9b5989a00a9b7587eebf88a00b38037 Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Tue, 5 Dec 2023 20:56:46 +0100 Subject: [PATCH 023/350] Update zstd-jni to 1.5.5-11 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 84976e4108e47..be48ffe45422f 100644 --- a/pom.xml +++ b/pom.xml @@ -495,7 +495,7 @@ com.github.luben zstd-jni - 1.5.5-10 + 1.5.5-11 From 06d5daee7625ad7fdcbc54f372e21840aee4282b Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Tue, 5 Dec 2023 20:57:11 +0100 Subject: [PATCH 024/350] Update oshi-core to 6.4.8 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index be48ffe45422f..0539039d37d53 100644 --- a/pom.xml +++ b/pom.xml @@ -501,7 +501,7 @@ com.github.oshi oshi-core - 6.4.7 + 6.4.8 From 221436ff70241073a9ed1adad91497d3e0e9b568 Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Tue, 5 Dec 2023 20:58:44 +0100 Subject: [PATCH 025/350] Update commons-io to 2.15.1 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 0539039d37d53..e3ce25832368c 100644 --- a/pom.xml +++ b/pom.xml @@ -636,7 +636,7 @@ commons-io commons-io - 2.15.0 + 2.15.1 From cdca155bebd86c4785939b3ba47063b2568cafc2 Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Tue, 5 Dec 2023 20:59:24 +0100 Subject: [PATCH 026/350] Update wire to 4.8.1 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index e3ce25832368c..e7588583e9bc0 100644 --- a/pom.xml +++ b/pom.xml @@ -199,7 +199,7 @@ 2.1.2 2.0.62.Final 201 - 4.8.0 + 4.8.1 From 09f83665eab9eaf0ca83e2fead4fc99dc2a2bd49 Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Tue, 5 Dec 2023 21:00:45 +0100 Subject: [PATCH 027/350] Update AWS SDK v2 to 2.21.38 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index e7588583e9bc0..2d083834b7104 100644 --- a/pom.xml +++ b/pom.xml @@ -296,7 +296,7 @@ software.amazon.awssdk bom - 2.21.37 + 2.21.38 pom import From d7d4038df006bacab7ded64b9fd1f4a4eb86d914 Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Tue, 5 Dec 2023 21:01:48 +0100 Subject: [PATCH 028/350] Update AWS SDK v1 to 1.12.605 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 2d083834b7104..f340be9b53e2c 100644 --- a/pom.xml +++ b/pom.xml @@ -177,7 +177,7 @@ 4.13.1 14.0.1 1.11.3 - 1.12.604 + 1.12.605 4.17.0 7.5.1 87 From 9dd7f05dd869e117afe80d7622f922d9efe89e74 Mon Sep 17 00:00:00 2001 From: Marius Grama Date: Wed, 6 Dec 2023 12:38:00 +0100 Subject: [PATCH 029/350] Test checkpoint filtering on table with multiple partition fields --- .../TestCheckpointEntryIterator.java | 138 ++++++++++++++++++ 1 file changed, 138 insertions(+) diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/transactionlog/checkpoint/TestCheckpointEntryIterator.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/transactionlog/checkpoint/TestCheckpointEntryIterator.java index 2a8d84a65b732..dcf83f52d74a4 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/transactionlog/checkpoint/TestCheckpointEntryIterator.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/transactionlog/checkpoint/TestCheckpointEntryIterator.java @@ -51,6 +51,7 @@ import java.time.LocalDate; import java.time.LocalDateTime; import java.time.ZonedDateTime; +import java.time.format.DateTimeFormatter; import java.util.List; import java.util.Map; import java.util.Objects; @@ -740,6 +741,143 @@ public void testSkipAddEntriesThroughPartitionPruning() assertThat(addEntryIteratorWithRangePartitionFilter.getCompletedPositions().orElseThrow()).isEqualTo(3L); } + @Test + public void testSkipAddEntriesThroughComposedPartitionPruning() + throws IOException + { + MetadataEntry metadataEntry = new MetadataEntry( + "metadataId", + "metadataName", + "metadataDescription", + new MetadataEntry.Format( + "metadataFormatProvider", + ImmutableMap.of()), + "{\"type\":\"struct\",\"fields\":" + + "[{\"name\":\"ts\",\"type\":\"timestamp\",\"nullable\":true,\"metadata\":{}}," + + "{\"name\":\"part_day\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}," + + "{\"name\":\"part_hour\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]}", + ImmutableList.of( + "part_day", + "part_hour"), + ImmutableMap.of(), + 1000); + ProtocolEntry protocolEntry = new ProtocolEntry(1, 1, Optional.empty(), Optional.empty()); + LocalDateTime date = LocalDateTime.of(2023, 12, 1, 0, 0); + DateTimeFormatter dayFormatter = DateTimeFormatter.ofPattern("yyyyMMdd"); + DateTimeFormatter hourFormatter = DateTimeFormatter.ofPattern("HH"); + int numAddEntries = 100; + Set addFileEntries = IntStream.rangeClosed(1, numAddEntries).mapToObj(index -> + new AddFileEntry( + "addFilePath", + ImmutableMap.of( + "part_day", date.plusHours(index).format(dayFormatter), + "part_hour", date.plusHours(index).format(hourFormatter)), + 1000, + 1001, + true, + Optional.of("{" + + "\"numRecords\":20," + + "\"minValues\":{" + + "\"ts\":\"1990-10-31T01:00:00.000Z\"" + + "}," + + "\"maxValues\":{" + + "\"ts\":\"1990-10-31T02:00:00.000Z\"" + + "}," + + "\"nullCount\":{" + + "\"ts\":1" + + "}}"), + Optional.empty(), + ImmutableMap.of(), + Optional.empty())) + .collect(toImmutableSet()); + + CheckpointEntries entries = new CheckpointEntries( + metadataEntry, + protocolEntry, + ImmutableSet.of(), + addFileEntries, + ImmutableSet.of()); + + CheckpointWriter writer = new CheckpointWriter( + TESTING_TYPE_MANAGER, + checkpointSchemaManager, + "test", + ParquetWriterOptions.builder() // approximately 2 rows per row group + .setMaxBlockSize(DataSize.ofBytes(128L)) + .setMaxPageSize(DataSize.ofBytes(128L)) + .build()); + + File targetFile = File.createTempFile("testSkipAddEntries-", ".checkpoint.parquet"); + targetFile.deleteOnExit(); + + String targetPath = "file://" + targetFile.getAbsolutePath(); + targetFile.delete(); // file must not exist when writer is called + writer.write(entries, createOutputFile(targetPath)); + + CheckpointEntryIterator metadataAndProtocolEntryIterator = createCheckpointEntryIterator( + URI.create(targetPath), + ImmutableSet.of(METADATA, PROTOCOL), + Optional.empty(), + Optional.empty(), + TupleDomain.all(), + Optional.empty()); + + DeltaLakeColumnHandle partitionDayField = new DeltaLakeColumnHandle( + "part_day", + VARCHAR, + OptionalInt.empty(), + "part_day", + VARCHAR, + REGULAR, + Optional.empty()); + DeltaLakeColumnHandle partitionHourField = new DeltaLakeColumnHandle( + "part_hour", + VARCHAR, + OptionalInt.empty(), + "part_hour", + VARCHAR, + REGULAR, + Optional.empty()); + + CheckpointEntryIterator addEntryIterator = createCheckpointEntryIterator( + URI.create(targetPath), + ImmutableSet.of(ADD), + Optional.of(metadataEntry), + Optional.of(protocolEntry), + TupleDomain.all(), + Optional.of(alwaysTrue())); + + CheckpointEntryIterator addEntryIteratorEqualityDayPartitionFilter = createCheckpointEntryIterator( + URI.create(targetPath), + ImmutableSet.of(ADD), + Optional.of(metadataEntry), + Optional.of(protocolEntry), + TupleDomain.withColumnDomains(ImmutableMap.of(partitionDayField, singleValue(VARCHAR, utf8Slice("20231202")))), + Optional.of(alwaysTrue())); + + CheckpointEntryIterator addEntryIteratorWithDayAndHourEqualityPartitionFilter = createCheckpointEntryIterator( + URI.create(targetPath), + ImmutableSet.of(ADD), + Optional.of(metadataEntry), + Optional.of(protocolEntry), + TupleDomain.withColumnDomains(ImmutableMap.of( + partitionDayField, singleValue(VARCHAR, utf8Slice("20231202")), + partitionHourField, singleValue(VARCHAR, utf8Slice("10")))), + Optional.of(alwaysTrue())); + + assertThat(Iterators.size(metadataAndProtocolEntryIterator)).isEqualTo(2); + assertThat(metadataAndProtocolEntryIterator.getCompletedPositions().orElseThrow()).isEqualTo(3L); + + assertThat(Iterators.size(addEntryIterator)).isEqualTo(numAddEntries); + assertThat(addEntryIterator.getCompletedPositions().orElseThrow()).isEqualTo(101L); + + assertThat(Iterators.size(addEntryIteratorEqualityDayPartitionFilter)).isEqualTo(24); + assertThat(addEntryIteratorEqualityDayPartitionFilter.getCompletedPositions().orElseThrow()).isEqualTo(24L); + + assertThat(Iterators.size(addEntryIteratorWithDayAndHourEqualityPartitionFilter)).isEqualTo(1); + assertThat(addEntryIteratorWithDayAndHourEqualityPartitionFilter.getCompletedPositions().orElseThrow()).isEqualTo(1L); + } + @Test public void testSkipRemoveEntries() throws IOException From ae7849e5b81b0c0eff7e980222502bf392a9a7f3 Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Wed, 6 Dec 2023 10:21:55 +0100 Subject: [PATCH 030/350] Run trino-tests JUnit tests Fixes bunch of failing tests due to the tests being not executed for some time --- testing/trino-tests/pom.xml | 16 +++++++++++----- .../trino/execution/TestExecutionJmxMetrics.java | 5 +---- .../execution/TestRefreshMaterializedView.java | 3 +++ .../io/trino/tests/TestQueryPlanDeterminism.java | 10 ++++++++++ .../io/trino/tests/TestQuerySpillLimits.java | 4 ++-- .../src/test/java/io/trino/tests/TestServer.java | 6 ++++++ 6 files changed, 33 insertions(+), 11 deletions(-) diff --git a/testing/trino-tests/pom.xml b/testing/trino-tests/pom.xml index 3d58172bd50e0..baab8ee092859 100644 --- a/testing/trino-tests/pom.xml +++ b/testing/trino-tests/pom.xml @@ -301,6 +301,12 @@ test + + org.junit.jupiter + junit-jupiter-engine + test + + org.openjdk.jmh jmh-core @@ -327,11 +333,11 @@ maven-surefire-plugin - - - - - + + org.apache.maven.surefire + surefire-junit-platform + ${dep.plugin.surefire.version} + org.apache.maven.surefire surefire-testng diff --git a/testing/trino-tests/src/test/java/io/trino/execution/TestExecutionJmxMetrics.java b/testing/trino-tests/src/test/java/io/trino/execution/TestExecutionJmxMetrics.java index 7a9cf391c1f7c..d4943f4340c70 100644 --- a/testing/trino-tests/src/test/java/io/trino/execution/TestExecutionJmxMetrics.java +++ b/testing/trino-tests/src/test/java/io/trino/execution/TestExecutionJmxMetrics.java @@ -17,7 +17,6 @@ import io.trino.Session; import io.trino.execution.resourcegroups.InternalResourceGroupManager; import io.trino.plugin.resourcegroups.ResourceGroupManagerPlugin; -import io.trino.server.PrefixObjectNameGeneratorModule; import io.trino.spi.QueryId; import io.trino.testing.DistributedQueryRunner; import io.trino.tests.tpch.TpchQueryRunnerBuilder; @@ -45,9 +44,7 @@ public class TestExecutionJmxMetrics public void testQueryStats() throws Exception { - try (DistributedQueryRunner queryRunner = TpchQueryRunnerBuilder.builder() - .setAdditionalModule(new PrefixObjectNameGeneratorModule("io.trino")) - .build()) { + try (DistributedQueryRunner queryRunner = TpchQueryRunnerBuilder.builder().build()) { queryRunner.installPlugin(new ResourceGroupManagerPlugin()); InternalResourceGroupManager resourceGroupManager = queryRunner.getCoordinator().getResourceGroupManager() .orElseThrow(() -> new IllegalStateException("Resource manager not configured")); diff --git a/testing/trino-tests/src/test/java/io/trino/execution/TestRefreshMaterializedView.java b/testing/trino-tests/src/test/java/io/trino/execution/TestRefreshMaterializedView.java index 3d26bf9bf5af5..423f58b13101e 100644 --- a/testing/trino-tests/src/test/java/io/trino/execution/TestRefreshMaterializedView.java +++ b/testing/trino-tests/src/test/java/io/trino/execution/TestRefreshMaterializedView.java @@ -37,6 +37,7 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.TestInstance; import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.parallel.Execution; import java.time.Duration; import java.util.Optional; @@ -54,8 +55,10 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; @TestInstance(PER_CLASS) +@Execution(SAME_THREAD) public class TestRefreshMaterializedView extends AbstractTestQueryFramework { diff --git a/testing/trino-tests/src/test/java/io/trino/tests/TestQueryPlanDeterminism.java b/testing/trino-tests/src/test/java/io/trino/tests/TestQueryPlanDeterminism.java index 9a6bd296c921b..488696ac7ac65 100644 --- a/testing/trino-tests/src/test/java/io/trino/tests/TestQueryPlanDeterminism.java +++ b/testing/trino-tests/src/test/java/io/trino/tests/TestQueryPlanDeterminism.java @@ -34,6 +34,7 @@ import static io.trino.plugin.tpch.TpchMetadata.TINY_SCHEMA_NAME; import static io.trino.testing.TestingSession.testSessionBuilder; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; @TestInstance(PER_CLASS) @@ -274,4 +275,13 @@ public void testLargeIn() // testLargeIn is expensive Assumptions.abort("Skipping testLargeIn"); } + + @Test + @Override + public void testUnionAllAboveBroadcastJoin() + { + // TODO: https://github.com/trinodb/trino/issues/20043 + assertThatThrownBy(super::testUnionAllAboveBroadcastJoin) + .hasMessageContaining("bytes is negative"); + } } diff --git a/testing/trino-tests/src/test/java/io/trino/tests/TestQuerySpillLimits.java b/testing/trino-tests/src/test/java/io/trino/tests/TestQuerySpillLimits.java index 9fb634752fa44..f13db591fc164 100644 --- a/testing/trino-tests/src/test/java/io/trino/tests/TestQuerySpillLimits.java +++ b/testing/trino-tests/src/test/java/io/trino/tests/TestQuerySpillLimits.java @@ -73,7 +73,7 @@ public void testMaxSpillPerNodeLimit() } }) .isInstanceOf(RuntimeException.class) - .hasMessage(".*Query exceeded local spill limit of 10B"); + .hasMessage("Query exceeded local spill limit of 10B"); } @Test @@ -86,7 +86,7 @@ public void testQueryMaxSpillPerNodeLimit() } }) .isInstanceOf(RuntimeException.class) - .hasMessageMatching(".*Query exceeded per-query local spill limit of 10B"); + .hasMessageMatching("Query exceeded per-query local spill limit of 10B"); } private LocalQueryRunner createLocalQueryRunner(NodeSpillConfig nodeSpillConfig) diff --git a/testing/trino-tests/src/test/java/io/trino/tests/TestServer.java b/testing/trino-tests/src/test/java/io/trino/tests/TestServer.java index e0dbbb86d8be1..526bd17c261e3 100644 --- a/testing/trino-tests/src/test/java/io/trino/tests/TestServer.java +++ b/testing/trino-tests/src/test/java/io/trino/tests/TestServer.java @@ -37,6 +37,8 @@ import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.parallel.Execution; import java.net.URI; import java.util.Collections; @@ -82,7 +84,11 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.Fail.fail; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; +@TestInstance(PER_CLASS) +@Execution(CONCURRENT) public class TestServer { private static final JsonCodec QUERY_RESULTS_CODEC = jsonCodec(QueryResults.class); From c2589481701171a26f99e94e4626e03fedff314f Mon Sep 17 00:00:00 2001 From: Dejan Mircevski Date: Tue, 28 Nov 2023 15:52:49 -0500 Subject: [PATCH 031/350] Mention SELinux in README Docker section ... to get rid of errors like this: Configuration error: config directory doesn't contain a conf.py file (/docs/src/main/sphinx) --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index a95e04aac1fd0..118582736e6aa 100644 --- a/README.md +++ b/README.md @@ -36,6 +36,7 @@ information about reporting vulnerabilities. * Mac OS X or Linux * Java 17.0.4+, 64-bit * Docker + * Turn SELinux off on the build host, allowing containers to mount parts of the Trino source tree ## Building Trino From 53bafd26ce51ee366cdde8f73fe659c783665ff0 Mon Sep 17 00:00:00 2001 From: Dejan Mircevski Date: Tue, 28 Nov 2023 16:38:37 -0500 Subject: [PATCH 032/350] Reword SELinux mention in README.md Thanks @mosabua for the sugestion. Co-authored-by: Manfred Moser --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 118582736e6aa..7fc649cf0aa8b 100644 --- a/README.md +++ b/README.md @@ -36,7 +36,7 @@ information about reporting vulnerabilities. * Mac OS X or Linux * Java 17.0.4+, 64-bit * Docker - * Turn SELinux off on the build host, allowing containers to mount parts of the Trino source tree + * Turn SELinux or other systems disabling write access to the local checkout off, to allow containers to mount parts of the Trino source tree ## Building Trino From f1506480892bedd023782ec3ab19e2c8c3be5f7b Mon Sep 17 00:00:00 2001 From: Dejan Mircevski Date: Tue, 28 Nov 2023 17:37:06 -0500 Subject: [PATCH 033/350] Wrap new README line at 80 chars --- README.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index 7fc649cf0aa8b..b59e4ace4b18f 100644 --- a/README.md +++ b/README.md @@ -36,7 +36,8 @@ information about reporting vulnerabilities. * Mac OS X or Linux * Java 17.0.4+, 64-bit * Docker - * Turn SELinux or other systems disabling write access to the local checkout off, to allow containers to mount parts of the Trino source tree + * Turn SELinux or other systems disabling write access to the local checkout + off, to allow containers to mount parts of the Trino source tree ## Building Trino From 22b3d034e64dd7c8c324f1664bcee71fc1ed2dbe Mon Sep 17 00:00:00 2001 From: Dejan Mircevski Date: Tue, 28 Nov 2023 16:16:17 -0500 Subject: [PATCH 034/350] Turn off SELinux in doc build container If SELinux is protecting the host directory where trino/docs resides, the build container cannot mount it, and we get this error: Configuration error: config directory doesn't contain a conf.py file (/docs/src/main/sphinx) This patch makes Docker explicitly disable security policies that protect the host, eliminating the error. --- docs/build | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/build b/docs/build index 7cbb83a6980ba..5a72bbdbcf576 100755 --- a/docs/build +++ b/docs/build @@ -8,5 +8,5 @@ test -t 1 && OPTS='-it' || OPTS='' SPHINX_IMAGE=${SPHINX_IMAGE:-ghcr.io/trinodb/build/sphinx:7} -docker run --rm $OPTS -e TRINO_VERSION -u $(id -u):$(id -g) -v "$PWD":/docs $SPHINX_IMAGE \ +docker run --security-opt label:disable --rm $OPTS -e TRINO_VERSION -u $(id -u):$(id -g) -v "$PWD":/docs $SPHINX_IMAGE \ sphinx-build -q -j auto -b html -W -d target/doctrees src/main/sphinx target/html From a0f81782d6dff04de13feb5d9fd466b2e34d4e27 Mon Sep 17 00:00:00 2001 From: Marius Grama Date: Tue, 5 Dec 2023 12:44:08 +0100 Subject: [PATCH 035/350] Add numeric integer to decimal coercion for hive tables --- docs/src/main/sphinx/connector/hive.md | 8 +- .../plugin/hive/coercions/CoercionUtils.java | 19 +++- .../hive/coercions/DecimalCoercers.java | 55 +++++++++++ .../plugin/hive/orc/OrcTypeTranslator.java | 16 ++++ .../plugin/hive/util/HiveCoercionPolicy.java | 8 +- .../hive/coercions/TestDecimalCoercers.java | 87 ++++++++++++++++- .../product/hive/BaseTestHiveCoercion.java | 96 +++++++++++++++++++ .../TestHiveCoercionOnPartitionedTable.java | 8 ++ .../TestHiveCoercionOnUnpartitionedTable.java | 8 ++ 9 files changed, 291 insertions(+), 14 deletions(-) diff --git a/docs/src/main/sphinx/connector/hive.md b/docs/src/main/sphinx/connector/hive.md index 9697b8a927919..2f051615a43e3 100644 --- a/docs/src/main/sphinx/connector/hive.md +++ b/docs/src/main/sphinx/connector/hive.md @@ -644,13 +644,13 @@ type conversions. * - `CHAR` - narrowing conversions for `CHAR` * - `TINYINT` - - `VARCHAR`, `SMALLINT`, `INTEGER`, `BIGINT`, `DOUBLE` + - `VARCHAR`, `SMALLINT`, `INTEGER`, `BIGINT`, `DOUBLE`, `DECIMAL` * - `SMALLINT` - - `VARCHAR`, `INTEGER`, `BIGINT`, `DOUBLE` + - `VARCHAR`, `INTEGER`, `BIGINT`, `DOUBLE`, `DECIMAL` * - `INTEGER` - - `VARCHAR`, `BIGINT`, `DOUBLE` + - `VARCHAR`, `BIGINT`, `DOUBLE`, `DECIMAL` * - `BIGINT` - - `VARCHAR`, `DOUBLE` + - `VARCHAR`, `DOUBLE`, `DECIMAL` * - `REAL` - `DOUBLE`, `DECIMAL` * - `DOUBLE` diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/coercions/CoercionUtils.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/coercions/CoercionUtils.java index a0aa9435c00ee..aa530a32e890c 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/coercions/CoercionUtils.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/coercions/CoercionUtils.java @@ -69,6 +69,7 @@ import static io.trino.plugin.hive.coercions.DecimalCoercers.createDecimalToRealCoercer; import static io.trino.plugin.hive.coercions.DecimalCoercers.createDecimalToVarcharCoercer; import static io.trino.plugin.hive.coercions.DecimalCoercers.createDoubleToDecimalCoercer; +import static io.trino.plugin.hive.coercions.DecimalCoercers.createIntegerNumberToDecimalCoercer; import static io.trino.plugin.hive.coercions.DecimalCoercers.createRealToDecimalCoercer; import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; import static io.trino.spi.block.ColumnarArray.toColumnarArray; @@ -139,6 +140,9 @@ public static Type createTypeFromCoercer(TypeManager typeManager, HiveType fromH if (toHiveType.equals(HIVE_DOUBLE)) { return Optional.of(new IntegerNumberToDoubleCoercer<>(fromType)); } + if (toType instanceof DecimalType toDecimalType) { + return Optional.of(createIntegerNumberToDecimalCoercer(fromType, toDecimalType)); + } } if (fromHiveType.equals(HIVE_SHORT)) { if (toHiveType.equals(HIVE_INT) || toHiveType.equals(HIVE_LONG)) { @@ -147,6 +151,9 @@ public static Type createTypeFromCoercer(TypeManager typeManager, HiveType fromH if (toHiveType.equals(HIVE_DOUBLE)) { return Optional.of(new IntegerNumberToDoubleCoercer<>(fromType)); } + if (toType instanceof DecimalType toDecimalType) { + return Optional.of(createIntegerNumberToDecimalCoercer(fromType, toDecimalType)); + } } if (fromHiveType.equals(HIVE_INT)) { if (toHiveType.equals(HIVE_LONG)) { @@ -155,9 +162,17 @@ public static Type createTypeFromCoercer(TypeManager typeManager, HiveType fromH if (toHiveType.equals(HIVE_DOUBLE)) { return Optional.of(new IntegerNumberToDoubleCoercer<>(fromType)); } + if (toType instanceof DecimalType toDecimalType) { + return Optional.of(createIntegerNumberToDecimalCoercer(fromType, toDecimalType)); + } } - if (fromHiveType.equals(HIVE_LONG) && toHiveType.equals(HIVE_DOUBLE)) { - return Optional.of(new IntegerNumberToDoubleCoercer<>(fromType)); + if (fromHiveType.equals(HIVE_LONG)) { + if (toHiveType.equals(HIVE_DOUBLE)) { + return Optional.of(new IntegerNumberToDoubleCoercer<>(fromType)); + } + if (toType instanceof DecimalType toDecimalType) { + return Optional.of(createIntegerNumberToDecimalCoercer(fromType, toDecimalType)); + } } if (fromHiveType.equals(HIVE_FLOAT) && toHiveType.equals(HIVE_DOUBLE)) { return Optional.of(new FloatToDoubleCoercer()); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/coercions/DecimalCoercers.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/coercions/DecimalCoercers.java index 5dc835cb463b6..2c91d7173681f 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/coercions/DecimalCoercers.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/coercions/DecimalCoercers.java @@ -25,6 +25,8 @@ import io.trino.spi.type.Type; import io.trino.spi.type.VarcharType; +import java.math.BigDecimal; + import static io.trino.spi.StandardErrorCode.INVALID_ARGUMENTS; import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; import static io.trino.spi.type.BigintType.BIGINT; @@ -41,6 +43,9 @@ import static io.trino.spi.type.DecimalConversions.shortToLongCast; import static io.trino.spi.type.DecimalConversions.shortToShortCast; import static io.trino.spi.type.Decimals.longTenToNth; +import static io.trino.spi.type.Decimals.overflows; +import static io.trino.spi.type.Decimals.writeBigDecimal; +import static io.trino.spi.type.Decimals.writeShortDecimal; import static io.trino.spi.type.DoubleType.DOUBLE; import static io.trino.spi.type.IntegerType.INTEGER; import static io.trino.spi.type.RealType.REAL; @@ -465,4 +470,54 @@ protected void applyCoercedValue(BlockBuilder blockBuilder, Block block, int pos realToLongDecimal(fromType.getFloat(block, position), toType.getPrecision(), toType.getScale())); } } + + public static TypeCoercer createIntegerNumberToDecimalCoercer(F fromType, DecimalType toType) + { + if (toType.isShort()) { + return new IntegerNumberToShortDecimalCoercer<>(fromType, toType); + } + return new IntegerNumberToLongDecimalCoercer<>(fromType, toType); + } + + private static class IntegerNumberToShortDecimalCoercer + extends TypeCoercer + { + public IntegerNumberToShortDecimalCoercer(F fromType, DecimalType toType) + { + super(fromType, toType); + } + + @Override + protected void applyCoercedValue(BlockBuilder blockBuilder, Block block, int position) + { + BigDecimal bigDecimal = BigDecimal.valueOf(fromType.getLong(block, position)).setScale(toType.getScale()); + if (overflows(bigDecimal, toType.getPrecision())) { + blockBuilder.appendNull(); + } + else { + writeShortDecimal(blockBuilder, bigDecimal.unscaledValue().longValueExact()); + } + } + } + + private static class IntegerNumberToLongDecimalCoercer + extends TypeCoercer + { + public IntegerNumberToLongDecimalCoercer(F fromType, DecimalType toType) + { + super(fromType, toType); + } + + @Override + protected void applyCoercedValue(BlockBuilder blockBuilder, Block block, int position) + { + BigDecimal bigDecimal = BigDecimal.valueOf(fromType.getLong(block, position)).setScale(toType.getScale()); + if (overflows(bigDecimal, toType.getPrecision())) { + blockBuilder.appendNull(); + } + else { + writeBigDecimal(toType, blockBuilder, bigDecimal); + } + } + } } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/orc/OrcTypeTranslator.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/orc/OrcTypeTranslator.java index 00f050fa97b46..d0f698012e0a0 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/orc/OrcTypeTranslator.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/orc/OrcTypeTranslator.java @@ -25,6 +25,7 @@ import io.trino.plugin.hive.coercions.TypeCoercer; import io.trino.plugin.hive.coercions.VarcharToDoubleCoercer; import io.trino.spi.type.DateType; +import io.trino.spi.type.DecimalType; import io.trino.spi.type.DoubleType; import io.trino.spi.type.TimestampType; import io.trino.spi.type.Type; @@ -41,6 +42,7 @@ import static io.trino.orc.metadata.OrcType.OrcTypeKind.STRING; import static io.trino.orc.metadata.OrcType.OrcTypeKind.TIMESTAMP; import static io.trino.orc.metadata.OrcType.OrcTypeKind.VARCHAR; +import static io.trino.plugin.hive.coercions.DecimalCoercers.createIntegerNumberToDecimalCoercer; import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.spi.type.IntegerType.INTEGER; import static io.trino.spi.type.SmallintType.SMALLINT; @@ -98,6 +100,20 @@ private OrcTypeTranslator() {} return Optional.of(new IntegerNumberToDoubleCoercer<>(BIGINT)); } } + if (toTrinoType instanceof DecimalType decimalType) { + if (fromOrcType == BYTE) { + return Optional.of(createIntegerNumberToDecimalCoercer(TINYINT, decimalType)); + } + if (fromOrcType == SHORT) { + return Optional.of(createIntegerNumberToDecimalCoercer(SMALLINT, decimalType)); + } + if (fromOrcType == INT) { + return Optional.of(createIntegerNumberToDecimalCoercer(INTEGER, decimalType)); + } + if (fromOrcType == LONG) { + return Optional.of(createIntegerNumberToDecimalCoercer(BIGINT, decimalType)); + } + } return Optional.empty(); } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/HiveCoercionPolicy.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/HiveCoercionPolicy.java index d424fcda56eeb..28f22757c2df5 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/HiveCoercionPolicy.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/HiveCoercionPolicy.java @@ -87,16 +87,16 @@ private boolean canCoerce(HiveType fromHiveType, HiveType toHiveType, HiveTimest return fromHiveType.equals(HIVE_TIMESTAMP); } if (fromHiveType.equals(HIVE_BYTE)) { - return toHiveType.equals(HIVE_SHORT) || toHiveType.equals(HIVE_INT) || toHiveType.equals(HIVE_LONG) || toHiveType.equals(HIVE_DOUBLE); + return toHiveType.equals(HIVE_SHORT) || toHiveType.equals(HIVE_INT) || toHiveType.equals(HIVE_LONG) || toHiveType.equals(HIVE_DOUBLE) || toType instanceof DecimalType; } if (fromHiveType.equals(HIVE_SHORT)) { - return toHiveType.equals(HIVE_INT) || toHiveType.equals(HIVE_LONG) || toHiveType.equals(HIVE_DOUBLE); + return toHiveType.equals(HIVE_INT) || toHiveType.equals(HIVE_LONG) || toHiveType.equals(HIVE_DOUBLE) || toType instanceof DecimalType; } if (fromHiveType.equals(HIVE_INT)) { - return toHiveType.equals(HIVE_LONG) || toHiveType.equals(HIVE_DOUBLE); + return toHiveType.equals(HIVE_LONG) || toHiveType.equals(HIVE_DOUBLE) || toType instanceof DecimalType; } if (fromHiveType.equals(HIVE_LONG)) { - return toHiveType.equals(HIVE_DOUBLE); + return toHiveType.equals(HIVE_DOUBLE) || toType instanceof DecimalType; } if (fromHiveType.equals(HIVE_FLOAT)) { return toHiveType.equals(HIVE_DOUBLE) || toType instanceof DecimalType; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/coercions/TestDecimalCoercers.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/coercions/TestDecimalCoercers.java index dadfea248de9d..dc43dcd85c58f 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/coercions/TestDecimalCoercers.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/coercions/TestDecimalCoercers.java @@ -16,15 +16,17 @@ import io.trino.spi.block.Block; import io.trino.spi.type.DecimalParseResult; import io.trino.spi.type.Decimals; +import io.trino.spi.type.Int128; import io.trino.spi.type.Type; import org.junit.jupiter.api.Test; -import static io.trino.plugin.hive.HiveTimestampPrecision.NANOSECONDS; +import static io.trino.plugin.hive.HiveTimestampPrecision.DEFAULT_PRECISION; import static io.trino.plugin.hive.HiveType.toHiveType; import static io.trino.plugin.hive.coercions.CoercionUtils.createCoercer; import static io.trino.spi.predicate.Utils.blockToNativeValue; import static io.trino.spi.predicate.Utils.nativeValueToBlock; import static io.trino.spi.type.BigintType.BIGINT; +import static io.trino.spi.type.DecimalType.createDecimalType; import static io.trino.spi.type.IntegerType.INTEGER; import static io.trino.spi.type.SmallintType.SMALLINT; import static io.trino.spi.type.TinyintType.TINYINT; @@ -73,12 +75,89 @@ private void testDecimalToIntCoercion(String decimalString, Type coercedType, Ob else { assertThat(parseResult.getType().isShort()).isTrue(); } - assertDecimalToIntCoercion(parseResult.getType(), parseResult.getObject(), coercedType, expectedValue); + assertCoercion(parseResult.getType(), parseResult.getObject(), coercedType, expectedValue); } - private void assertDecimalToIntCoercion(Type fromType, Object valueToBeCoerced, Type toType, Object expectedValue) + @Test + public void testTinyintToDecimalCoercion() + { + // Short decimal coercion + assertCoercion(TINYINT, 12L, createDecimalType(10), 12L); + assertCoercion(TINYINT, 12L, createDecimalType(10, 2), 1_200L); + assertCoercion(TINYINT, 12L, createDecimalType(10, 5), 1_200_000L); + // Long decimal coercion + assertCoercion(TINYINT, 0L, createDecimalType(), Int128.ZERO); + assertCoercion(TINYINT, 0L, createDecimalType(), Int128.ZERO); + assertCoercion(TINYINT, 12L, createDecimalType(), Int128.valueOf(12)); + assertCoercion(TINYINT, -12L, createDecimalType(), Int128.valueOf(-12)); + assertCoercion(TINYINT, (long) Byte.MAX_VALUE, createDecimalType(), Int128.valueOf(Byte.MAX_VALUE)); + assertCoercion(TINYINT, (long) Byte.MIN_VALUE, createDecimalType(), Int128.valueOf(Byte.MIN_VALUE)); + assertCoercion(TINYINT, 12L, createDecimalType(20, 10), Int128.valueOf("120000000000")); + // Coercion overflow + assertCoercion(TINYINT, 42L, createDecimalType(6, 5), null); + } + + @Test + public void testSmallintToDecimalCoercion() + { + // Short decimal coercion + assertCoercion(SMALLINT, 12L, createDecimalType(10), 12L); + assertCoercion(SMALLINT, 12L, createDecimalType(10, 2), 1_200L); + assertCoercion(SMALLINT, 12L, createDecimalType(10, 5), 1_200_000L); + // Long decimal coercion + assertCoercion(SMALLINT, 12L, createDecimalType(20, 10), Int128.valueOf("120000000000")); + assertCoercion(SMALLINT, 0L, createDecimalType(), Int128.ZERO); + assertCoercion(SMALLINT, 128L, createDecimalType(), Int128.valueOf(128)); + assertCoercion(SMALLINT, -128L, createDecimalType(), Int128.valueOf(-128)); + assertCoercion(SMALLINT, (long) Short.MAX_VALUE, createDecimalType(), Int128.valueOf(Short.MAX_VALUE)); + assertCoercion(SMALLINT, (long) Short.MIN_VALUE, createDecimalType(), Int128.valueOf(Short.MIN_VALUE)); + // Coercion overflow + assertCoercion(SMALLINT, 128L, createDecimalType(7, 5), null); + assertCoercion(SMALLINT, 128L, createDecimalType(20, 18), null); + } + + @Test + public void testIntToDecimalCoercion() + { + // Short decimal coercion + assertCoercion(INTEGER, 123_456L, createDecimalType(10), 123_456L); + assertCoercion(INTEGER, 123_456L, createDecimalType(10, 3), 123_456_000L); + // Long decimal coercion + assertCoercion(INTEGER, 0L, createDecimalType(), Int128.ZERO); + assertCoercion(INTEGER, 128L, createDecimalType(), Int128.valueOf(128)); + assertCoercion(INTEGER, -128L, createDecimalType(), Int128.valueOf(-128)); + assertCoercion(INTEGER, (long) Integer.MAX_VALUE, createDecimalType(), Int128.valueOf(Integer.MAX_VALUE)); + assertCoercion(INTEGER, (long) Integer.MIN_VALUE, createDecimalType(), Int128.valueOf(Integer.MIN_VALUE)); + assertCoercion(INTEGER, 123_456L, createDecimalType(20, 10), Int128.valueOf("1234560000000000")); + // Coercion overflow + assertCoercion(INTEGER, 123_456_789L, createDecimalType(10, 5), null); + assertCoercion(INTEGER, 123_456_789L, createDecimalType(20, 13), null); + } + + @Test + public void testBigintToDecimalCoercion() + { + // Short decimal coercion + assertCoercion(BIGINT, 0L, createDecimalType(10), 0L); + assertCoercion(BIGINT, 123_456_789L, createDecimalType(12), 123_456_789L); + assertCoercion(BIGINT, 123_456_789L, createDecimalType(12, 3), 123_456_789_000L); + // Long decimal coercion + assertCoercion(BIGINT, 0L, createDecimalType(), Int128.ZERO); + assertCoercion(BIGINT, 128L, createDecimalType(), Int128.valueOf(128)); + assertCoercion(BIGINT, -128L, createDecimalType(), Int128.valueOf(-128)); + assertCoercion(BIGINT, Long.MAX_VALUE, createDecimalType(), Int128.valueOf(Long.MAX_VALUE)); + assertCoercion(BIGINT, Long.MIN_VALUE, createDecimalType(), Int128.valueOf(Long.MIN_VALUE)); + assertCoercion(BIGINT, 123_456_789L, createDecimalType(20, 5), Int128.valueOf("12345678900000")); + assertCoercion(BIGINT, 123_456_789L, createDecimalType(20, 10), Int128.valueOf("1234567890000000000")); + assertCoercion(BIGINT, Long.MAX_VALUE, createDecimalType(38, 2), Int128.valueOf("922337203685477580700")); + // Coercion overflow + assertCoercion(BIGINT, 123_456_789L, createDecimalType(10, 5), null); + assertCoercion(BIGINT, Long.MAX_VALUE, createDecimalType(25, 8), null); + } + + private static void assertCoercion(Type fromType, Object valueToBeCoerced, Type toType, Object expectedValue) { - Block coercedValue = createCoercer(TESTING_TYPE_MANAGER, toHiveType(fromType), toHiveType(toType), new CoercionUtils.CoercionContext(NANOSECONDS, false)).orElseThrow() + Block coercedValue = createCoercer(TESTING_TYPE_MANAGER, toHiveType(fromType), toHiveType(toType), new CoercionUtils.CoercionContext(DEFAULT_PRECISION, true)).orElseThrow() .apply(nativeValueToBlock(fromType, valueToBeCoerced)); assertThat(blockToNativeValue(toType, coercedValue)) .isEqualTo(expectedValue); diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/BaseTestHiveCoercion.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/BaseTestHiveCoercion.java index f02df83c37c1b..fea12c95e4bd4 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/BaseTestHiveCoercion.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/BaseTestHiveCoercion.java @@ -105,13 +105,21 @@ protected void doTestHiveCoercion(HiveTableDefinition tableDefinition) "tinyint_to_int", "tinyint_to_bigint", "tinyint_to_double", + "tinyint_to_shortdecimal", + "tinyint_to_longdecimal", "smallint_to_int", "smallint_to_bigint", "smallint_to_double", + "smallint_to_shortdecimal", + "smallint_to_longdecimal", "int_to_bigint", "int_to_double", + "int_to_shortdecimal", + "int_to_longdecimal", "bigint_to_double", "bigint_to_varchar", + "bigint_to_shortdecimal", + "bigint_to_longdecimal", "float_to_double", "double_to_float", "double_to_string", @@ -195,13 +203,21 @@ protected void insertTableRows(String tableName, String floatToDoubleType) " TINYINT '2', " + " TINYINT '-3', " + " TINYINT '4', " + + " TINYINT '5', " + + " TINYINT '6', " + " SMALLINT '100', " + " SMALLINT '-101', " + " SMALLINT '1024', " + + " SMALLINT '2048', " + + " SMALLINT '4096', " + " INTEGER '2323', " + " INTEGER '16384', " + + " INTEGER '16385', " + + " INTEGER '16386', " + " 1234567890, " + " 12345, " + + " 9223372, " + + " 9223372036, " + " REAL '0.5', " + " DOUBLE '0.5', " + " DOUBLE '12345.12345', " + @@ -257,13 +273,21 @@ protected void insertTableRows(String tableName, String floatToDoubleType) " TINYINT '-2', " + " NULL, " + " TINYINT '-4', " + + " TINYINT '-5', " + + " TINYINT '-6', " + " SMALLINT '-100', " + " SMALLINT '101', " + " SMALLINT '-1024', " + + " SMALLINT '-2048', " + + " SMALLINT '-4096', " + " INTEGER '-2323', " + " INTEGER '-16384', " + + " INTEGER '-16385', " + + " INTEGER '-16386', " + " -1234567890, " + " -12345, " + + " -9223372, " + + " -9223372036, " + " REAL '-1.5', " + " DOUBLE '-1.5', " + " DOUBLE 'NaN', " + @@ -405,6 +429,12 @@ else if (getHiveVersionMajor() == 3 && isFormat.test("orc")) { .put("tinyint_to_double", Arrays.asList( -4D, 4D)) + .put("tinyint_to_shortdecimal", Arrays.asList( + new BigDecimal(-5), + new BigDecimal(5))) + .put("tinyint_to_longdecimal", Arrays.asList( + new BigDecimal(-6), + new BigDecimal(6))) .put("smallint_to_int", ImmutableList.of( 100, -100)) @@ -414,18 +444,36 @@ else if (getHiveVersionMajor() == 3 && isFormat.test("orc")) { .put("smallint_to_double", ImmutableList.of( -1024D, 1024D)) + .put("smallint_to_shortdecimal", Arrays.asList( + new BigDecimal(-2048), + new BigDecimal(-2048))) + .put("smallint_to_longdecimal", Arrays.asList( + new BigDecimal(-4096), + new BigDecimal(4096))) .put("int_to_bigint", ImmutableList.of( 2323L, -2323L)) .put("int_to_double", ImmutableList.of( -16384D, 16384D)) + .put("int_to_shortdecimal", Arrays.asList( + new BigDecimal(-16385), + new BigDecimal(16385))) + .put("int_to_longdecimal", Arrays.asList( + new BigDecimal(-16386), + new BigDecimal(16386))) .put("bigint_to_double", ImmutableList.of( -1234567890D, 1234567890D)) .put("bigint_to_varchar", ImmutableList.of( "12345", "-12345")) + .put("bigint_to_shortdecimal", Arrays.asList( + new BigDecimal(-9223372L), + new BigDecimal(9223372L))) + .put("bigint_to_longdecimal", Arrays.asList( + new BigDecimal(-9223372036L), + new BigDecimal(9223372036L))) .put("float_to_double", ImmutableList.of( 0.5, -1.5)) @@ -801,11 +849,19 @@ protected Map expectedExceptionsWithHiveContext() .put(columnContext("1.1", "parquet", "map_to_map"), "org.apache.hadoop.io.LongWritable cannot be cast to org.apache.hadoop.hive.serde2.io.ByteWritable") .put(columnContext("1.1", "parquet", "tinyint_to_bigint"), "org.apache.hadoop.io.LongWritable cannot be cast to org.apache.hadoop.hive.serde2.io.ByteWritable") .put(columnContext("1.1", "parquet", "tinyint_to_double"), "org.apache.hadoop.io.DoubleWritable cannot be cast to org.apache.hadoop.hive.serde2.io.ByteWritable") + .put(columnContext("1.1", "parquet", "tinyint_to_shortdecimal"), "org.apache.hadoop.hive.serde2.io.HiveDecimalWritable cannot be cast to org.apache.hadoop.io.ByteWritable") + .put(columnContext("1.1", "parquet", "tinyint_to_longdecimal"), "org.apache.hadoop.hive.serde2.io.HiveDecimalWritable cannot be cast to org.apache.hadoop.io.ByteWritable") .put(columnContext("1.1", "parquet", "smallint_to_bigint"), "org.apache.hadoop.io.LongWritable cannot be cast to org.apache.hadoop.hive.serde2.io.ShortWritable") .put(columnContext("1.1", "parquet", "smallint_to_double"), "org.apache.hadoop.io.DoubleWritable cannot be cast to org.apache.hadoop.hive.serde2.io.ShortWritable") + .put(columnContext("1.1", "parquet", "smallint_to_shortdecimal"), "org.apache.hadoop.hive.serde2.io.HiveDecimalWritable cannot be cast to org.apache.hadoop.io.ShortWritable") + .put(columnContext("1.1", "parquet", "smallint_to_longdecimal"), "org.apache.hadoop.hive.serde2.io.HiveDecimalWritable cannot be cast to org.apache.hadoop.io.ShortWritable") .put(columnContext("1.1", "parquet", "int_to_bigint"), "org.apache.hadoop.io.LongWritable cannot be cast to org.apache.hadoop.io.IntWritable") .put(columnContext("1.1", "parquet", "int_to_double"), "org.apache.hadoop.io.DoubleWritable cannot be cast to org.apache.hadoop.io.IntWritable") + .put(columnContext("1.1", "parquet", "int_to_shortdecimal"), "org.apache.hadoop.hive.serde2.io.HiveDecimalWritable cannot be cast to org.apache.hadoop.io.IntWritable") + .put(columnContext("1.1", "parquet", "int_to_longdecimal"), "org.apache.hadoop.hive.serde2.io.HiveDecimalWritable cannot be cast to org.apache.hadoop.io.IntWritable") .put(columnContext("1.1", "parquet", "bigint_to_double"), "org.apache.hadoop.io.DoubleWritable cannot be cast to org.apache.hadoop.io.LongWritable") + .put(columnContext("1.1", "parquet", "bigint_to_shortdecimal"), "org.apache.hadoop.hive.serde2.io.HiveDecimalWritable cannot be cast to org.apache.hadoop.io.LongWritable") + .put(columnContext("1.1", "parquet", "bigint_to_longdecimal"), "org.apache.hadoop.hive.serde2.io.HiveDecimalWritable cannot be cast to org.apache.hadoop.io.LongWritable") // Rcbinary .put(columnContext("1.1", "rcbinary", "row_to_row"), "java.util.ArrayList cannot be cast to org.apache.hadoop.hive.serde2.lazybinary.LazyBinaryStruct") .put(columnContext("1.1", "rcbinary", "list_to_list"), "java.util.ArrayList cannot be cast to org.apache.hadoop.hive.serde2.lazybinary.LazyBinaryArray") @@ -834,11 +890,19 @@ protected Map expectedExceptionsWithHiveContext() .put(columnContext("2.1", "parquet", "map_to_map"), "org.apache.hadoop.io.LongWritable cannot be cast to org.apache.hadoop.hive.serde2.io.ByteWritable") .put(columnContext("2.1", "parquet", "tinyint_to_bigint"), "org.apache.hadoop.io.LongWritable cannot be cast to org.apache.hadoop.hive.serde2.io.ByteWritable") .put(columnContext("2.1", "parquet", "tinyint_to_double"), "org.apache.hadoop.io.DoubleWritable cannot be cast to org.apache.hadoop.hive.serde2.io.ByteWritable") + .put(columnContext("2.1", "parquet", "tinyint_to_shortdecimal"), "org.apache.hadoop.hive.serde2.io.HiveDecimalWritable cannot be cast to org.apache.hadoop.io.ByteWritable") + .put(columnContext("2.1", "parquet", "tinyint_to_longdecimal"), "org.apache.hadoop.hive.serde2.io.HiveDecimalWritable cannot be cast to org.apache.hadoop.io.ByteWritable") .put(columnContext("2.1", "parquet", "smallint_to_bigint"), "org.apache.hadoop.io.LongWritable cannot be cast to org.apache.hadoop.hive.serde2.io.ShortWritable") .put(columnContext("2.1", "parquet", "smallint_to_double"), "org.apache.hadoop.io.DoubleWritable cannot be cast to org.apache.hadoop.hive.serde2.io.ShortWritable") + .put(columnContext("2.1", "parquet", "smallint_to_shortdecimal"), "org.apache.hadoop.hive.serde2.io.HiveDecimalWritable cannot be cast to org.apache.hadoop.io.ShortWritable") + .put(columnContext("2.1", "parquet", "smallint_to_longdecimal"), "org.apache.hadoop.hive.serde2.io.HiveDecimalWritable cannot be cast to org.apache.hadoop.io.ShortWritable") .put(columnContext("2.1", "parquet", "int_to_bigint"), "org.apache.hadoop.io.LongWritable cannot be cast to org.apache.hadoop.io.IntWritable") .put(columnContext("2.1", "parquet", "int_to_double"), "org.apache.hadoop.io.DoubleWritable cannot be cast to org.apache.hadoop.io.IntWritable") + .put(columnContext("2.1", "parquet", "int_to_shortdecimal"), "org.apache.hadoop.hive.serde2.io.HiveDecimalWritable cannot be cast to org.apache.hadoop.io.IntWritable") + .put(columnContext("2.1", "parquet", "int_to_longdecimal"), "org.apache.hadoop.hive.serde2.io.HiveDecimalWritable cannot be cast to org.apache.hadoop.io.IntWritable") .put(columnContext("2.1", "parquet", "bigint_to_double"), "org.apache.hadoop.io.DoubleWritable cannot be cast to org.apache.hadoop.io.LongWritable") + .put(columnContext("2.1", "parquet", "bigint_to_shortdecimal"), "org.apache.hadoop.hive.serde2.io.HiveDecimalWritable cannot be cast to org.apache.hadoop.io.LongWritable") + .put(columnContext("2.1", "parquet", "bigint_to_longdecimal"), "org.apache.hadoop.hive.serde2.io.HiveDecimalWritable cannot be cast to org.apache.hadoop.io.LongWritable") // Rcbinary .put(columnContext("2.1", "rcbinary", "row_to_row"), "java.util.ArrayList cannot be cast to org.apache.hadoop.hive.serde2.lazybinary.LazyBinaryStruct") .put(columnContext("2.1", "rcbinary", "list_to_list"), "java.util.ArrayList cannot be cast to org.apache.hadoop.hive.serde2.lazybinary.LazyBinaryArray") @@ -851,11 +915,19 @@ protected Map expectedExceptionsWithHiveContext() .put(columnContext("3.1", "parquet", "map_to_map"), "org.apache.hadoop.io.LongWritable cannot be cast to org.apache.hadoop.hive.serde2.io.ByteWritable") .put(columnContext("3.1", "parquet", "tinyint_to_bigint"), "org.apache.hadoop.io.LongWritable cannot be cast to org.apache.hadoop.hive.serde2.io.ByteWritable") .put(columnContext("3.1", "parquet", "tinyint_to_double"), "org.apache.hadoop.io.DoubleWritable cannot be cast to org.apache.hadoop.hive.serde2.io.ByteWritable") + .put(columnContext("3.1", "parquet", "tinyint_to_shortdecimal"), "org.apache.hadoop.hive.serde2.io.HiveDecimalWritable cannot be cast to org.apache.hadoop.io.ByteWritable") + .put(columnContext("3.1", "parquet", "tinyint_to_longdecimal"), "org.apache.hadoop.hive.serde2.io.HiveDecimalWritable cannot be cast to org.apache.hadoop.io.ByteWritable") .put(columnContext("3.1", "parquet", "smallint_to_bigint"), "org.apache.hadoop.io.LongWritable cannot be cast to org.apache.hadoop.hive.serde2.io.ShortWritable") .put(columnContext("3.1", "parquet", "smallint_to_double"), "org.apache.hadoop.io.DoubleWritable cannot be cast to org.apache.hadoop.hive.serde2.io.ShortWritable") + .put(columnContext("3.1", "parquet", "smallint_to_shortdecimal"), "org.apache.hadoop.hive.serde2.io.HiveDecimalWritable cannot be cast to org.apache.hadoop.io.ShortWritable") + .put(columnContext("3.1", "parquet", "smallint_to_longdecimal"), "org.apache.hadoop.hive.serde2.io.HiveDecimalWritable cannot be cast to org.apache.hadoop.io.ShortWritable") .put(columnContext("3.1", "parquet", "int_to_bigint"), "org.apache.hadoop.io.LongWritable cannot be cast to org.apache.hadoop.io.IntWritable") .put(columnContext("3.1", "parquet", "int_to_double"), "org.apache.hadoop.io.DoubleWritable cannot be cast to org.apache.hadoop.io.IntWritable") + .put(columnContext("3.1", "parquet", "int_to_shortdecimal"), "org.apache.hadoop.hive.serde2.io.HiveDecimalWritable cannot be cast to org.apache.hadoop.io.IntWritable") + .put(columnContext("3.1", "parquet", "int_to_longdecimal"), "org.apache.hadoop.hive.serde2.io.HiveDecimalWritable cannot be cast to org.apache.hadoop.io.IntWritable") .put(columnContext("3.1", "parquet", "bigint_to_double"), "org.apache.hadoop.io.DoubleWritable cannot be cast to org.apache.hadoop.io.LongWritable") + .put(columnContext("3.1", "parquet", "bigint_to_shortdecimal"), "org.apache.hadoop.hive.serde2.io.HiveDecimalWritable cannot be cast to org.apache.hadoop.io.LongWritable") + .put(columnContext("3.1", "parquet", "bigint_to_longdecimal"), "org.apache.hadoop.hive.serde2.io.HiveDecimalWritable cannot be cast to org.apache.hadoop.io.LongWritable") // Rcbinary .put(columnContext("3.1", "rcbinary", "row_to_row"), "java.util.ArrayList cannot be cast to org.apache.hadoop.hive.serde2.lazybinary.LazyBinaryStruct") .put(columnContext("3.1", "rcbinary", "list_to_list"), "java.util.ArrayList cannot be cast to org.apache.hadoop.hive.serde2.lazybinary.LazyBinaryArray") @@ -935,13 +1007,21 @@ private void assertProperAlteredTableSchema(String tableName) row("tinyint_to_int", "integer"), row("tinyint_to_bigint", "bigint"), row("tinyint_to_double", "double"), + row("tinyint_to_shortdecimal", "decimal(10,2)"), + row("tinyint_to_longdecimal", "decimal(20,2)"), row("smallint_to_int", "integer"), row("smallint_to_bigint", "bigint"), row("smallint_to_double", "double"), + row("smallint_to_shortdecimal", "decimal(10,2)"), + row("smallint_to_longdecimal", "decimal(20,2)"), row("int_to_bigint", "bigint"), row("int_to_double", "double"), + row("int_to_shortdecimal", "decimal(10,2)"), + row("int_to_longdecimal", "decimal(20,2)"), row("bigint_to_double", "double"), row("bigint_to_varchar", "varchar"), + row("bigint_to_shortdecimal", "decimal(10,2)"), + row("bigint_to_longdecimal", "decimal(20,2)"), row("float_to_double", "double"), row("double_to_float", floatType), row("double_to_string", "varchar"), @@ -1013,13 +1093,21 @@ private void assertColumnTypes( .put("tinyint_to_int", INTEGER) .put("tinyint_to_bigint", BIGINT) .put("tinyint_to_double", DOUBLE) + .put("tinyint_to_shortdecimal", DECIMAL) + .put("tinyint_to_longdecimal", DECIMAL) .put("smallint_to_int", INTEGER) .put("smallint_to_bigint", BIGINT) .put("smallint_to_double", DOUBLE) + .put("smallint_to_shortdecimal", DECIMAL) + .put("smallint_to_longdecimal", DECIMAL) .put("int_to_bigint", BIGINT) .put("int_to_double", DOUBLE) + .put("int_to_shortdecimal", DECIMAL) + .put("int_to_longdecimal", DECIMAL) .put("bigint_to_double", DOUBLE) .put("bigint_to_varchar", VARCHAR) + .put("bigint_to_shortdecimal", DECIMAL) + .put("bigint_to_longdecimal", DECIMAL) .put("float_to_double", DOUBLE) .put("double_to_float", floatType) .put("double_to_string", VARCHAR) @@ -1091,13 +1179,21 @@ private static void alterTableColumnTypes(String tableName) onHive().executeQuery(format("ALTER TABLE %s CHANGE COLUMN tinyint_to_int tinyint_to_int int", tableName)); onHive().executeQuery(format("ALTER TABLE %s CHANGE COLUMN tinyint_to_bigint tinyint_to_bigint bigint", tableName)); onHive().executeQuery(format("ALTER TABLE %s CHANGE COLUMN tinyint_to_double tinyint_to_double double", tableName)); + onHive().executeQuery(format("ALTER TABLE %s CHANGE COLUMN tinyint_to_shortdecimal tinyint_to_shortdecimal decimal(10,2)", tableName)); + onHive().executeQuery(format("ALTER TABLE %s CHANGE COLUMN tinyint_to_longdecimal tinyint_to_longdecimal decimal(20,2)", tableName)); onHive().executeQuery(format("ALTER TABLE %s CHANGE COLUMN smallint_to_int smallint_to_int int", tableName)); onHive().executeQuery(format("ALTER TABLE %s CHANGE COLUMN smallint_to_bigint smallint_to_bigint bigint", tableName)); onHive().executeQuery(format("ALTER TABLE %s CHANGE COLUMN smallint_to_double smallint_to_double double", tableName)); + onHive().executeQuery(format("ALTER TABLE %s CHANGE COLUMN smallint_to_shortdecimal smallint_to_shortdecimal decimal(10,2)", tableName)); + onHive().executeQuery(format("ALTER TABLE %s CHANGE COLUMN smallint_to_longdecimal smallint_to_longdecimal decimal(20,2)", tableName)); onHive().executeQuery(format("ALTER TABLE %s CHANGE COLUMN int_to_bigint int_to_bigint bigint", tableName)); onHive().executeQuery(format("ALTER TABLE %s CHANGE COLUMN int_to_double int_to_double double", tableName)); + onHive().executeQuery(format("ALTER TABLE %s CHANGE COLUMN int_to_shortdecimal int_to_shortdecimal decimal(10,2)", tableName)); + onHive().executeQuery(format("ALTER TABLE %s CHANGE COLUMN int_to_longdecimal int_to_longdecimal decimal(20,2)", tableName)); onHive().executeQuery(format("ALTER TABLE %s CHANGE COLUMN bigint_to_double bigint_to_double double", tableName)); onHive().executeQuery(format("ALTER TABLE %s CHANGE COLUMN bigint_to_varchar bigint_to_varchar string", tableName)); + onHive().executeQuery(format("ALTER TABLE %s CHANGE COLUMN bigint_to_shortdecimal bigint_to_shortdecimal decimal(10,2)", tableName)); + onHive().executeQuery(format("ALTER TABLE %s CHANGE COLUMN bigint_to_longdecimal bigint_to_longdecimal decimal(20,2)", tableName)); onHive().executeQuery(format("ALTER TABLE %s CHANGE COLUMN float_to_double float_to_double double", tableName)); onHive().executeQuery(format("ALTER TABLE %s CHANGE COLUMN double_to_float double_to_float %s", tableName, floatType)); onHive().executeQuery(format("ALTER TABLE %s CHANGE COLUMN double_to_string double_to_string string", tableName)); diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveCoercionOnPartitionedTable.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveCoercionOnPartitionedTable.java index 8960585ad40ae..b19900000b183 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveCoercionOnPartitionedTable.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveCoercionOnPartitionedTable.java @@ -108,13 +108,21 @@ private static HiveTableDefinition.HiveTableDefinitionBuilder tableDefinitionBui " tinyint_to_int TINYINT," + " tinyint_to_bigint TINYINT," + " tinyint_to_double TINYINT," + + " tinyint_to_shortdecimal TINYINT," + + " tinyint_to_longdecimal TINYINT," + " smallint_to_int SMALLINT," + " smallint_to_bigint SMALLINT," + " smallint_to_double SMALLINT," + + " smallint_to_shortdecimal SMALLINT," + + " smallint_to_longdecimal SMALLINT," + " int_to_bigint INT," + " int_to_double INT," + + " int_to_shortdecimal INT," + + " int_to_longdecimal INT," + " bigint_to_double BIGINT," + " bigint_to_varchar BIGINT," + + " bigint_to_shortdecimal BIGINT," + + " bigint_to_longdecimal BIGINT," + " float_to_double " + floatType + "," + " double_to_float DOUBLE," + " double_to_string DOUBLE," + diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveCoercionOnUnpartitionedTable.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveCoercionOnUnpartitionedTable.java index f11ceb4a1520f..128545f391fa3 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveCoercionOnUnpartitionedTable.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveCoercionOnUnpartitionedTable.java @@ -57,13 +57,21 @@ private static HiveTableDefinition.HiveTableDefinitionBuilder tableDefinitionBui tinyint_to_int TINYINT, tinyint_to_bigint TINYINT, tinyint_to_double TINYINT, + tinyint_to_shortdecimal TINYINT, + tinyint_to_longdecimal TINYINT, smallint_to_int SMALLINT, smallint_to_bigint SMALLINT, smallint_to_double SMALLINT, + smallint_to_shortdecimal SMALLINT, + smallint_to_longdecimal SMALLINT, int_to_bigint INT, int_to_double INT, + int_to_shortdecimal INT, + int_to_longdecimal INT, bigint_to_double BIGINT, bigint_to_varchar BIGINT, + bigint_to_shortdecimal BIGINT, + bigint_to_longdecimal BIGINT, float_to_double FLOAT, double_to_float DOUBLE, double_to_string DOUBLE, From dc880e9567a8163453a7844765d8da3ccfc029d3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jan=20Wa=C5=9B?= Date: Thu, 7 Dec 2023 13:52:36 +0100 Subject: [PATCH 036/350] Produce a more actionable error on wrong JDBC URL --- .../main/java/io/trino/plugin/jdbc/TracingDataSource.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/TracingDataSource.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/TracingDataSource.java index c9a1a91dca7fb..453d6956c0a6d 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/TracingDataSource.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/TracingDataSource.java @@ -25,6 +25,7 @@ import java.util.Properties; import java.util.logging.Logger; +import static com.google.common.base.Preconditions.checkState; import static java.util.Objects.requireNonNull; public class TracingDataSource @@ -70,7 +71,9 @@ public JdbcDataSource(Driver driver, String connectionUrl, Properties properties public Connection getConnection() throws SQLException { - return driver.connect(connectionUrl, properties); + Connection connection = driver.connect(connectionUrl, properties); + checkState(connection != null, "Driver returned null connection, make sure the connection URL '%s' is valid for the driver %s", connectionUrl, driver); + return connection; } @Override From e84f984467e01931bb9733cdaaa35a521937cae0 Mon Sep 17 00:00:00 2001 From: Brad <1299656830@qq.com> Date: Tue, 28 Nov 2023 18:41:45 +0800 Subject: [PATCH 037/350] Fix orc input stream ArrayIndexOutOfBoundsException --- .../io/trino/orc/stream/OrcInputStream.java | 4 +++- .../io/trino/orc/stream/TestLongStreamV2.java | 17 ++++++++++++++++- 2 files changed, 19 insertions(+), 2 deletions(-) diff --git a/lib/trino-orc/src/main/java/io/trino/orc/stream/OrcInputStream.java b/lib/trino-orc/src/main/java/io/trino/orc/stream/OrcInputStream.java index 3f37981fc5e48..cd0b6bf0b36f6 100644 --- a/lib/trino-orc/src/main/java/io/trino/orc/stream/OrcInputStream.java +++ b/lib/trino-orc/src/main/java/io/trino/orc/stream/OrcInputStream.java @@ -227,7 +227,9 @@ public void seekToCheckpoint(long checkpoint) int decompressedOffset = decodeDecompressedOffset(checkpoint); // if checkpoint is within the current buffer, seek locally int currentDecompressedBufferOffset = decodeDecompressedOffset(lastCheckpoint); - if (current != null && compressedOffset == decodeCompressedBlockOffset(lastCheckpoint) && decompressedOffset < currentDecompressedBufferOffset + current.length()) { + if (current != null && compressedOffset == decodeCompressedBlockOffset(lastCheckpoint) + && decompressedOffset >= currentDecompressedBufferOffset + && decompressedOffset < currentDecompressedBufferOffset + current.length()) { current.setPosition(decompressedOffset - currentDecompressedBufferOffset); return; } diff --git a/lib/trino-orc/src/test/java/io/trino/orc/stream/TestLongStreamV2.java b/lib/trino-orc/src/test/java/io/trino/orc/stream/TestLongStreamV2.java index b8a4106b07289..8333dc3ab29fe 100644 --- a/lib/trino-orc/src/test/java/io/trino/orc/stream/TestLongStreamV2.java +++ b/lib/trino-orc/src/test/java/io/trino/orc/stream/TestLongStreamV2.java @@ -33,7 +33,7 @@ public class TestLongStreamV2 extends AbstractTestValueStream { @Test - public void test() + public void testLargeValue() throws IOException { List> groups = new ArrayList<>(); @@ -47,6 +47,21 @@ public void test() testWriteValue(groups); } + @Test + public void testSmallValue() + throws IOException + { + List> groups = new ArrayList<>(); + for (int groupIndex = 0; groupIndex < 22; groupIndex++) { + List group = new ArrayList<>(); + for (int i = 0; i < 1_000_000; i++) { + group.add((long) (groupIndex * 0 + i)); + } + groups.add(group); + } + testWriteValue(groups); + } + @Override protected LongOutputStreamV2 createValueOutputStream() { From 0ce49c4352642259e2164bd052743dd900aa84f7 Mon Sep 17 00:00:00 2001 From: Raunaq Morarka Date: Thu, 7 Dec 2023 16:56:14 +0530 Subject: [PATCH 038/350] Improve readability of HiveColumnStatistics#toString --- .../hive/metastore/HiveColumnStatistics.java | 23 ++++++++++--------- 1 file changed, 12 insertions(+), 11 deletions(-) diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/HiveColumnStatistics.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/HiveColumnStatistics.java index 1469540e3fa52..1400effa9fd92 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/HiveColumnStatistics.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/HiveColumnStatistics.java @@ -26,6 +26,7 @@ import java.util.OptionalDouble; import java.util.OptionalLong; +import static com.google.common.base.MoreObjects.ToStringHelper; import static com.google.common.base.MoreObjects.toStringHelper; import static com.google.common.base.Preconditions.checkArgument; import static java.util.Objects.requireNonNull; @@ -174,17 +175,17 @@ public int hashCode() @Override public String toString() { - return toStringHelper(this) - .add("integerStatistics", integerStatistics) - .add("doubleStatistics", doubleStatistics) - .add("decimalStatistics", decimalStatistics) - .add("dateStatistics", dateStatistics) - .add("booleanStatistics", booleanStatistics) - .add("maxValueSizeInBytes", maxValueSizeInBytes) - .add("totalSizeInBytes", totalSizeInBytes) - .add("nullsCount", nullsCount) - .add("distinctValuesCount", distinctValuesCount) - .toString(); + ToStringHelper toStringHelper = toStringHelper(this); + integerStatistics.ifPresent(stats -> toStringHelper.add("integerStatistics", stats)); + doubleStatistics.ifPresent(stats -> toStringHelper.add("doubleStatistics", stats)); + decimalStatistics.ifPresent(stats -> toStringHelper.add("decimalStatistics", stats)); + dateStatistics.ifPresent(stats -> toStringHelper.add("dateStatistics", stats)); + booleanStatistics.ifPresent(stats -> toStringHelper.add("booleanStatistics", stats)); + maxValueSizeInBytes.ifPresent(stats -> toStringHelper.add("maxValueSizeInBytes", stats)); + totalSizeInBytes.ifPresent(stats -> toStringHelper.add("totalSizeInBytes", stats)); + nullsCount.ifPresent(stats -> toStringHelper.add("nullsCount", stats)); + distinctValuesCount.ifPresent(stats -> toStringHelper.add("distinctValuesCount", stats)); + return toStringHelper.toString(); } public static HiveColumnStatistics createIntegerColumnStatistics(OptionalLong min, OptionalLong max, OptionalLong nullsCount, OptionalLong distinctValuesCount) From e950bfc6d505c30d80812c9b872986ce13b75509 Mon Sep 17 00:00:00 2001 From: Raunaq Morarka Date: Fri, 8 Dec 2023 08:00:32 +0530 Subject: [PATCH 039/350] Remove unused code in TestCachingHiveMetastore --- .../cache/TestCachingHiveMetastore.java | 16 ---------------- 1 file changed, 16 deletions(-) diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastore.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastore.java index ddffedb8a9aa9..8b47e45de48a3 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastore.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastore.java @@ -19,7 +19,6 @@ import com.google.common.collect.Iterables; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.ThreadFactoryBuilder; -import io.airlift.log.Logger; import io.airlift.units.Duration; import io.trino.hive.thrift.metastore.ColumnStatisticsData; import io.trino.hive.thrift.metastore.ColumnStatisticsObj; @@ -69,7 +68,6 @@ import java.util.concurrent.TimeUnit; import java.util.function.Consumer; -import static com.google.common.base.Preconditions.checkState; import static com.google.common.util.concurrent.MoreExecutors.listeningDecorator; import static io.airlift.concurrent.Threads.daemonThreadsNamed; import static io.airlift.slice.Slices.utf8Slice; @@ -114,8 +112,6 @@ @Execution(SAME_THREAD) public class TestCachingHiveMetastore { - private static final Logger log = Logger.get(TestCachingHiveMetastore.class); - private static final PartitionStatistics TEST_STATS = PartitionStatistics.builder() .setBasicStatistics(new HiveBasicStatistics(OptionalLong.empty(), OptionalLong.of(2398040535435L), OptionalLong.empty(), OptionalLong.empty())) .setColumnStatistics(ImmutableMap.of(TEST_COLUMN, createIntegerColumnStatistics(OptionalLong.empty(), OptionalLong.empty(), OptionalLong.empty(), OptionalLong.empty()))) @@ -1017,18 +1013,6 @@ private static HiveColumnStatistics intColumnStats(int nullsCount) return createIntegerColumnStatistics(OptionalLong.empty(), OptionalLong.empty(), OptionalLong.of(nullsCount), OptionalLong.empty()); } - private static void await(CountDownLatch latch, long timeout, TimeUnit unit) - { - try { - boolean awaited = latch.await(timeout, unit); - checkState(awaited, "wait timed out"); - } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException(); - } - } - private PartitionCachingAssertions assertThatCachingWithDisabledPartitionCache() { return new PartitionCachingAssertions(executor); From 3adf701a120091f3e2fa6c57ffe3600df434d549 Mon Sep 17 00:00:00 2001 From: Raunaq Morarka Date: Tue, 5 Dec 2023 20:03:03 +0530 Subject: [PATCH 040/350] Fix negative caching for Hive column statistics When column statistics are absent, CachingHiveMetastore should cache this and avoid loading column statistics from the metastore for future requests --- .../plugin/hive/PartitionStatistics.java | 10 ++ .../metastore/cache/CachingHiveMetastore.java | 31 ++++-- .../cache/TestCachingHiveMetastore.java | 96 +++++++++++++++++++ .../thrift/MockThriftMetastoreClient.java | 58 ++++++----- 4 files changed, 155 insertions(+), 40 deletions(-) diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/PartitionStatistics.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/PartitionStatistics.java index f77000f0ea10a..151289496aaaa 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/PartitionStatistics.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/PartitionStatistics.java @@ -24,6 +24,7 @@ import java.util.Objects; import static com.google.common.base.MoreObjects.toStringHelper; +import static com.google.common.collect.ImmutableMap.toImmutableMap; import static java.util.Objects.requireNonNull; @Immutable @@ -99,6 +100,15 @@ public PartitionStatistics withBasicStatistics(HiveBasicStatistics basicStatisti return new PartitionStatistics(basicStatistics, columnStatistics); } + public PartitionStatistics withEmptyColumnStatisticsRemoved() + { + return new PartitionStatistics( + basicStatistics, + columnStatistics.entrySet().stream() + .filter(entry -> !entry.getValue().equals(HiveColumnStatistics.empty())) + .collect(toImmutableMap(Map.Entry::getKey, Map.Entry::getValue))); + } + public static class Builder { private HiveBasicStatistics basicStatistics = HiveBasicStatistics.createEmptyStatistics(); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/CachingHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/CachingHiveMetastore.java index 595e4f684e850..4c7bc9c86f839 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/CachingHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/CachingHiveMetastore.java @@ -59,6 +59,7 @@ import org.weakref.jmx.Managed; import org.weakref.jmx.Nested; +import java.util.AbstractMap; import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; @@ -473,7 +474,9 @@ public PartitionStatistics getTableStatistics(Table table) Table tableWithOnlyMissingColumns = table.withSelectedDataColumnsOnly(missingColumns); return delegate.getTableStatistics(tableWithOnlyMissingColumns); }, - CachingHiveMetastore::mergePartitionColumnStatistics); + (currentStats, newStats) -> mergePartitionColumnStatistics(currentStats, newStats, dataColumns)) + // HiveColumnStatistics.empty() are removed to make output consistent with non-cached metastore which simplifies testing + .withEmptyColumnStatisticsRemoved(); } /** @@ -494,23 +497,31 @@ public Map getPartitionStatistics(Table table, List partitionsByName.keySet(), missingPartitions -> loadPartitionsColumnStatistics(table, partitionsByName, missingPartitions), currentStats -> currentStats.getColumnStatistics().keySet().containsAll(dataColumns), - CachingHiveMetastore::mergePartitionColumnStatistics); + (currentStats, newStats) -> mergePartitionColumnStatistics(currentStats, newStats, dataColumns)); return statistics.entrySet().stream() - .collect(toImmutableMap(entry -> entry.getKey().getPartitionName().orElseThrow(), Entry::getValue)); + .collect(toImmutableMap( + entry -> entry.getKey().getPartitionName().orElseThrow(), + // HiveColumnStatistics.empty() are removed to make output consistent with non-cached metastore which simplifies testing + entry -> entry.getValue().withEmptyColumnStatisticsRemoved())); } - private static PartitionStatistics mergePartitionColumnStatistics(PartitionStatistics currentStats, PartitionStatistics newStats) + private PartitionStatistics mergePartitionColumnStatistics(PartitionStatistics currentStats, PartitionStatistics newStats, Set dataColumns) { requireNonNull(newStats, "newStats is null"); - if (currentStats == null) { - return newStats; + ImmutableMap.Builder columnStatisticsBuilder = ImmutableMap.builder(); + // Populate empty statistics for all requested columns to cache absence of column statistics for future requests. + if (cacheMissing) { + columnStatisticsBuilder.putAll(Iterables.transform( + dataColumns, + column -> new AbstractMap.SimpleEntry<>(column, HiveColumnStatistics.empty()))); } + if (currentStats != null) { + columnStatisticsBuilder.putAll(currentStats.getColumnStatistics()); + } + columnStatisticsBuilder.putAll(newStats.getColumnStatistics()); return new PartitionStatistics( newStats.getBasicStatistics(), - ImmutableMap.builder() - .putAll(currentStats.getColumnStatistics()) - .putAll(newStats.getColumnStatistics()) - .buildKeepingLast()); + columnStatisticsBuilder.buildKeepingLast()); } private Map loadPartitionsColumnStatistics(Table table, Map partitionsByName, Collection partitionNamesToLoad) diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastore.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastore.java index 8b47e45de48a3..a9fe3a4f988a3 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastore.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastore.java @@ -568,6 +568,52 @@ public void testGetTableStatistics() assertThat(mockClient.getAccessCount()).isEqualTo(6); } + @Test + public void testGetTableStatisticsWithEmptyColumnStats() + { + assertThat(mockClient.getAccessCount()).isEqualTo(0); + + Table table = metastore.getTable(TEST_DATABASE, TEST_TABLE).orElseThrow(); + assertThat(mockClient.getAccessCount()).isEqualTo(1); + + // Force TEST_TABLE to not have column statistics available + mockClient.mockColumnStats(TEST_DATABASE, TEST_TABLE, ImmutableMap.of()); + PartitionStatistics expectedStats = PartitionStatistics.builder() + .setBasicStatistics(new HiveBasicStatistics(OptionalLong.empty(), OptionalLong.of(2398040535435L), OptionalLong.empty(), OptionalLong.empty())) + .setColumnStatistics(ImmutableMap.of()) + .build(); + assertThat(metastore.getTableStatistics(table)).isEqualTo(expectedStats); + assertThat(mockClient.getAccessCount()).isEqualTo(2); + + // Absence of column statistics should get cached and metastore client access count should stay the same + assertThat(metastore.getTableStatistics(table)).isEqualTo(expectedStats); + assertThat(mockClient.getAccessCount()).isEqualTo(2); + } + + @Test + public void testTableStatisticsWithEmptyColumnStatsWithNoCacheMissing() + { + CachingHiveMetastore metastore = createCachingHiveMetastore(new BridgingHiveMetastore(thriftHiveMetastore), CACHE_TTL, false, true, executor); + + assertThat(mockClient.getAccessCount()).isEqualTo(0); + + Table table = metastore.getTable(TEST_DATABASE, TEST_TABLE).orElseThrow(); + assertThat(mockClient.getAccessCount()).isEqualTo(1); + + // Force TEST_TABLE to not have column statistics available + mockClient.mockColumnStats(TEST_DATABASE, TEST_TABLE, ImmutableMap.of()); + PartitionStatistics expectedStats = PartitionStatistics.builder() + .setBasicStatistics(new HiveBasicStatistics(OptionalLong.empty(), OptionalLong.of(2398040535435L), OptionalLong.empty(), OptionalLong.empty())) + .setColumnStatistics(ImmutableMap.of()) + .build(); + assertThat(metastore.getTableStatistics(table)).isEqualTo(expectedStats); + assertThat(mockClient.getAccessCount()).isEqualTo(2); + + // Absence of column statistics does not get cached and metastore client access count increases + assertThat(metastore.getTableStatistics(table)).isEqualTo(expectedStats); + assertThat(mockClient.getAccessCount()).isEqualTo(3); + } + @Test public void testGetTableStatisticsWithoutMetadataCache() { @@ -736,6 +782,56 @@ public void testGetPartitionStatistics() .containsEntry("col3", intColumnStats(33)); } + @Test + public void testGetPartitionStatisticsWithEmptyColumnStats() + { + assertThat(mockClient.getAccessCount()).isEqualTo(0); + + Table table = metastore.getTable(TEST_DATABASE, TEST_TABLE).orElseThrow(); + assertThat(mockClient.getAccessCount()).isEqualTo(1); + + Partition partition = metastore.getPartition(table, TEST_PARTITION_VALUES2).orElseThrow(); + assertThat(mockClient.getAccessCount()).isEqualTo(2); + + // TEST_PARTITION2 does not have column statistics available + PartitionStatistics expectedStats = PartitionStatistics.builder() + .setBasicStatistics(new HiveBasicStatistics(OptionalLong.empty(), OptionalLong.of(2398040535435L), OptionalLong.empty(), OptionalLong.empty())) + .setColumnStatistics(ImmutableMap.of()) + .build(); + assertThat(metastore.getPartitionStatistics(table, ImmutableList.of(partition))).isEqualTo(ImmutableMap.of(TEST_PARTITION2, expectedStats)); + assertThat(mockClient.getAccessCount()).isEqualTo(3); + + // Absence of column statistics should get cached and metastore client access count should stay the same + assertThat(metastore.getPartitionStatistics(table, ImmutableList.of(partition))).isEqualTo(ImmutableMap.of(TEST_PARTITION2, expectedStats)); + assertThat(mockClient.getAccessCount()).isEqualTo(3); + } + + @Test + public void testGetPartitionStatisticsWithEmptyColumnStatsWithNoCacheMissing() + { + CachingHiveMetastore metastore = createCachingHiveMetastore(new BridgingHiveMetastore(thriftHiveMetastore), CACHE_TTL, false, true, executor); + + assertThat(mockClient.getAccessCount()).isEqualTo(0); + + Table table = metastore.getTable(TEST_DATABASE, TEST_TABLE).orElseThrow(); + assertThat(mockClient.getAccessCount()).isEqualTo(1); + + Partition partition = metastore.getPartition(table, TEST_PARTITION_VALUES2).orElseThrow(); + assertThat(mockClient.getAccessCount()).isEqualTo(2); + + // TEST_PARTITION2 does not have column statistics available + PartitionStatistics expectedStats = PartitionStatistics.builder() + .setBasicStatistics(new HiveBasicStatistics(OptionalLong.empty(), OptionalLong.of(2398040535435L), OptionalLong.empty(), OptionalLong.empty())) + .setColumnStatistics(ImmutableMap.of()) + .build(); + assertThat(metastore.getPartitionStatistics(table, ImmutableList.of(partition))).isEqualTo(ImmutableMap.of(TEST_PARTITION2, expectedStats)); + assertThat(mockClient.getAccessCount()).isEqualTo(3); + + // Absence of column statistics does not get cached and metastore client access count increases + assertThat(metastore.getPartitionStatistics(table, ImmutableList.of(partition))).isEqualTo(ImmutableMap.of(TEST_PARTITION2, expectedStats)); + assertThat(mockClient.getAccessCount()).isEqualTo(4); + } + @Test public void testGetPartitionStatisticsWithoutMetadataCache() { diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/MockThriftMetastoreClient.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/MockThriftMetastoreClient.java index 7cce4d2495a88..6856a02e978e2 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/MockThriftMetastoreClient.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/MockThriftMetastoreClient.java @@ -96,35 +96,27 @@ public MockThriftMetastoreClient() public void mockColumnStats(String database, String table, Map columnStatistics) { - this.columnStatistics.compute(new SchemaTableName(database, table), (ignored, oldColumnStats) -> { - if (oldColumnStats == null) { - oldColumnStats = new HashMap<>(); - } - oldColumnStats.putAll(Maps.transformEntries(columnStatistics, (columnName, stats) -> { - ColumnStatisticsObj statsObj = new ColumnStatisticsObj(); - statsObj.setColName(columnName); - statsObj.setStatsData(stats); - return statsObj; - })); - return oldColumnStats; - }); + this.columnStatistics.put( + new SchemaTableName(database, table), + Maps.transformEntries(columnStatistics, (columnName, stats) -> { + ColumnStatisticsObj statsObj = new ColumnStatisticsObj(); + statsObj.setColName(columnName); + statsObj.setStatsData(stats); + return statsObj; + })); } public void mockPartitionColumnStats(String database, String table, String partitionName, Map columnStatistics) { Map> tablePartitionColumnStatistics = databaseTablePartitionColumnStatistics.computeIfAbsent(new SchemaTableName(database, table), key -> new HashMap<>()); - tablePartitionColumnStatistics.compute(partitionName, (ignored, oldColumnStats) -> { - if (oldColumnStats == null) { - oldColumnStats = new HashMap<>(); - } - oldColumnStats.putAll(Maps.transformEntries(columnStatistics, (columnName, stats) -> { - ColumnStatisticsObj statsObj = new ColumnStatisticsObj(); - statsObj.setColName(columnName); - statsObj.setStatsData(stats); - return statsObj; - })); - return oldColumnStats; - }); + tablePartitionColumnStatistics.put( + partitionName, + Maps.transformEntries(columnStatistics, (columnName, stats) -> { + ColumnStatisticsObj statsObj = new ColumnStatisticsObj(); + statsObj.setColName(columnName); + statsObj.setStatsData(stats); + return statsObj; + })); } private static ColumnStatisticsData createLongColumnStats() @@ -259,11 +251,14 @@ public List getTableColumnStatistics(String databaseName, S Map columnStatistics = this.columnStatistics.get(new SchemaTableName(databaseName, tableName)); - if (columnStatistics == null || !columnStatistics.keySet().containsAll(columnNames)) { - throw new NoSuchObjectException(); + if (columnStatistics == null) { + return ImmutableList.of(); } - return columnNames.stream().map(columnStatistics::get).collect(toImmutableList()); + return columnNames.stream() + .filter(columnStatistics::containsKey) + .map(columnStatistics::get) + .collect(toImmutableList()); } @Override @@ -294,10 +289,13 @@ public Map> getPartitionColumnStatistics(Strin for (String partition : partitionNames) { Map columnStatistics = tablePartitionColumnStatistics.get(partition); - if (columnStatistics == null || !columnStatistics.keySet().containsAll(columnNames)) { - throw new NoSuchObjectException(); + if (columnStatistics == null) { + continue; } - result.put(partition, ImmutableList.copyOf(columnStatistics.values())); + result.put(partition, columnNames.stream() + .filter(columnStatistics::containsKey) + .map(columnStatistics::get) + .collect(toImmutableList())); } return result.buildOrThrow(); From 81a9d2e95545363f61c99c6c53cb01044eaf2adb Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 1 Dec 2023 17:44:10 +0100 Subject: [PATCH 041/350] Fix typo --- .../java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java index e2ea22c7d46a2..e89e35ef453f5 100644 --- a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java @@ -1273,13 +1273,13 @@ public void testJoinPushdown() assertJoinConditionallyPushedDown( withoutDynamicFiltering, format("SELECT r.name, n.name FROM nation n %s region r ON n.regionkey %s r.regionkey", joinOperator, operator), - expectJoinPushdown(operator) && expectJoinPushdowOnInequalityOperator(joinOperator)); + expectJoinPushdown(operator) && expectJoinPushdownOnInequalityOperator(joinOperator)); // varchar inequality predicate assertJoinConditionallyPushedDown( withoutDynamicFiltering, format("SELECT n.name, nl.name FROM nation n %s %s nl ON n.name %s nl.name", joinOperator, nationLowercaseTable.getName(), operator), - expectVarcharJoinPushdown(operator) && expectJoinPushdowOnInequalityOperator(joinOperator)); + expectVarcharJoinPushdown(operator) && expectJoinPushdownOnInequalityOperator(joinOperator)); } // inequality along with an equality, which constitutes an equi-condition and allows filter to remain as part of the Join @@ -1415,7 +1415,7 @@ protected boolean expectJoinPushdown(String operator) throw new AssertionError(); // unreachable } - protected boolean expectJoinPushdowOnInequalityOperator(JoinOperator joinOperator) + protected boolean expectJoinPushdownOnInequalityOperator(JoinOperator joinOperator) { // Currently no pushdown as inequality predicate is removed from Join to maintain Cross Join and Filter as separate nodes return joinOperator != JOIN; From d9918c5de2a01ffcc0ca4f036233049fb2d35c3f Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 1 Dec 2023 15:26:47 +0100 Subject: [PATCH 042/350] Fix code indentation --- .../test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java index e89e35ef453f5..27ea0e6544e30 100644 --- a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java @@ -2055,8 +2055,8 @@ public void testJoinPushdownWithLongIdentifiers() try (TestTable left = new TestTable(getQueryRunner()::execute, "test_long_id_l", format("(%s BIGINT)", validColumnName)); TestTable right = new TestTable(getQueryRunner()::execute, "test_long_id_r", format("(%s BIGINT)", validColumnName))) { assertThat(query(joinPushdownEnabled(getSession()), """ - SELECT l.%1$s, r.%1$s - FROM %2$s l JOIN %3$s r ON l.%1$s = r.%1$s""".formatted(validColumnName, left.getName(), right.getName()))) + SELECT l.%1$s, r.%1$s + FROM %2$s l JOIN %3$s r ON l.%1$s = r.%1$s""".formatted(validColumnName, left.getName(), right.getName()))) .isFullyPushedDown(); } } From 184f024f8c287cf2789d1281b6741ebb53312c4f Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 1 Dec 2023 17:55:02 +0100 Subject: [PATCH 043/350] Improve exceptions when test fails For a test code like this ``` for (String operator : nonEqualities) { assertJoinConditionallyPushedDown( ... ``` the raised exception would contain no information which case was the failing one. --- .../plugin/jdbc/BaseJdbcConnectorTest.java | 30 +++++++++++++------ 1 file changed, 21 insertions(+), 9 deletions(-) diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java index 27ea0e6544e30..1c63e6b76231d 100644 --- a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java @@ -1373,11 +1373,17 @@ protected QueryAssert assertJoinConditionallyPushedDown( @Language("SQL") String query, boolean condition) { - QueryAssert queryAssert = assertThat(query(session, query)); - if (condition) { - return queryAssert.isFullyPushedDown(); + try { + QueryAssert queryAssert = assertThat(query(session, query)); + if (condition) { + return queryAssert.isFullyPushedDown(); + } + return queryAssert.joinIsNotFullyPushedDown(); + } + catch (Throwable e) { + e.addSuppressed(new Exception("Query: " + query)); + throw e; } - return queryAssert.joinIsNotFullyPushedDown(); } protected void assertConditionallyOrderedPushedDown( @@ -1386,12 +1392,18 @@ protected void assertConditionallyOrderedPushedDown( boolean condition, PlanMatchPattern otherwiseExpected) { - QueryAssert queryAssert = assertThat(query(session, query)).ordered(); - if (condition) { - queryAssert.isFullyPushedDown(); + try { + QueryAssert queryAssert = assertThat(query(session, query)).ordered(); + if (condition) { + queryAssert.isFullyPushedDown(); + } + else { + queryAssert.isNotFullyPushedDown(otherwiseExpected); + } } - else { - queryAssert.isNotFullyPushedDown(otherwiseExpected); + catch (Throwable e) { + e.addSuppressed(new Exception("Query: " + query)); + throw e; } } From a79ed39ed53a06bfa9434c2e7b225d84a3e6b6a0 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 1 Dec 2023 17:52:58 +0100 Subject: [PATCH 044/350] Use switch expression --- .../plugin/jdbc/BaseJdbcConnectorTest.java | 34 +++++-------------- 1 file changed, 9 insertions(+), 25 deletions(-) diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java index 1c63e6b76231d..09684a16e22d0 100644 --- a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java @@ -1413,18 +1413,10 @@ protected boolean expectJoinPushdown(String operator) // TODO (https://github.com/trinodb/trino/issues/6967) support join pushdown for IS NOT DISTINCT FROM return false; } - switch (toJoinConditionOperator(operator)) { - case EQUAL: - case NOT_EQUAL: - case LESS_THAN: - case LESS_THAN_OR_EQUAL: - case GREATER_THAN: - case GREATER_THAN_OR_EQUAL: - return true; - case IS_DISTINCT_FROM: - return hasBehavior(SUPPORTS_JOIN_PUSHDOWN_WITH_DISTINCT_FROM); - } - throw new AssertionError(); // unreachable + return switch (toJoinConditionOperator(operator)) { + case EQUAL, NOT_EQUAL, LESS_THAN, LESS_THAN_OR_EQUAL, GREATER_THAN, GREATER_THAN_OR_EQUAL -> true; + case IS_DISTINCT_FROM -> hasBehavior(SUPPORTS_JOIN_PUSHDOWN_WITH_DISTINCT_FROM); + }; } protected boolean expectJoinPushdownOnInequalityOperator(JoinOperator joinOperator) @@ -1439,19 +1431,11 @@ private boolean expectVarcharJoinPushdown(String operator) // TODO (https://github.com/trinodb/trino/issues/6967) support join pushdown for IS NOT DISTINCT FROM return false; } - switch (toJoinConditionOperator(operator)) { - case EQUAL: - case NOT_EQUAL: - return hasBehavior(SUPPORTS_JOIN_PUSHDOWN_WITH_VARCHAR_EQUALITY); - case LESS_THAN: - case LESS_THAN_OR_EQUAL: - case GREATER_THAN: - case GREATER_THAN_OR_EQUAL: - return hasBehavior(SUPPORTS_JOIN_PUSHDOWN_WITH_VARCHAR_INEQUALITY); - case IS_DISTINCT_FROM: - return hasBehavior(SUPPORTS_JOIN_PUSHDOWN_WITH_DISTINCT_FROM) && hasBehavior(SUPPORTS_JOIN_PUSHDOWN_WITH_VARCHAR_EQUALITY); - } - throw new AssertionError(); // unreachable + return switch (toJoinConditionOperator(operator)) { + case EQUAL, NOT_EQUAL -> hasBehavior(SUPPORTS_JOIN_PUSHDOWN_WITH_VARCHAR_EQUALITY); + case LESS_THAN, LESS_THAN_OR_EQUAL, GREATER_THAN, GREATER_THAN_OR_EQUAL -> hasBehavior(SUPPORTS_JOIN_PUSHDOWN_WITH_VARCHAR_INEQUALITY); + case IS_DISTINCT_FROM -> hasBehavior(SUPPORTS_JOIN_PUSHDOWN_WITH_DISTINCT_FROM) && hasBehavior(SUPPORTS_JOIN_PUSHDOWN_WITH_VARCHAR_EQUALITY); + }; } private JoinCondition.Operator toJoinConditionOperator(String operator) From 043ff4fe2ce3dd3a8bcfc0a676b1d84f0c2508f9 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 1 Dec 2023 18:07:39 +0100 Subject: [PATCH 045/350] Fix erroneous returns after test parameters removed Apparently a syntactic transformation was applied: from "with parameters invoke test method" to "test method with a for loop over parameters". This didn't account for return statement that could potentially lead to cutting off portion of test coverage. --- .../trino/plugin/jdbc/BaseJdbcConnectorTest.java | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java index 09684a16e22d0..a2b7d170f33be 100644 --- a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java @@ -1212,18 +1212,18 @@ public void verifySupportsJoinPushdownWithFullJoinDeclaration() @Test public void testJoinPushdown() { - for (JoinOperator joinOperator : JoinOperator.values()) { - Session session = joinPushdownEnabled(getSession()); + Session session = joinPushdownEnabled(getSession()); - if (!hasBehavior(SUPPORTS_JOIN_PUSHDOWN)) { - assertThat(query(session, "SELECT r.name, n.name FROM nation n JOIN region r ON n.regionkey = r.regionkey")) - .joinIsNotFullyPushedDown(); - return; - } + if (!hasBehavior(SUPPORTS_JOIN_PUSHDOWN)) { + assertThat(query(session, "SELECT r.name, n.name FROM nation n JOIN region r ON n.regionkey = r.regionkey")) + .joinIsNotFullyPushedDown(); + return; + } + for (JoinOperator joinOperator : JoinOperator.values()) { if (joinOperator == FULL_JOIN && !hasBehavior(SUPPORTS_JOIN_PUSHDOWN_WITH_FULL_JOIN)) { // Covered by verifySupportsJoinPushdownWithFullJoinDeclaration - return; + continue; } // Disable DF here for the sake of negative test cases' expected plan. With DF enabled, some operators return in DF's FilterNode and some do not. From c27e3b17b41e1d860eae26285d3f6558aaad6547 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 1 Dec 2023 18:10:04 +0100 Subject: [PATCH 046/350] Simplify test methods around full join pushdown No need for separate method, if all testing is already covered by `testJoinPushdown`. --- .../plugin/jdbc/BaseJdbcConnectorTest.java | 18 ++---------------- 1 file changed, 2 insertions(+), 16 deletions(-) diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java index a2b7d170f33be..b38c6b95b6260 100644 --- a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java @@ -1194,21 +1194,6 @@ public void verifySupportsJoinPushdownDeclaration() .joinIsNotFullyPushedDown(); } - /** - * Verify !SUPPORTS_JOIN_PUSHDOWN_WITH_FULL_JOIN declaration is true. - */ - @Test - public void verifySupportsJoinPushdownWithFullJoinDeclaration() - { - if (hasBehavior(SUPPORTS_JOIN_PUSHDOWN_WITH_FULL_JOIN)) { - // Covered by testJoinPushdown - return; - } - - assertThat(query(joinPushdownEnabled(getSession()), "SELECT r.name, n.name FROM nation n FULL JOIN region r ON n.regionkey = r.regionkey")) - .joinIsNotFullyPushedDown(); - } - @Test public void testJoinPushdown() { @@ -1222,7 +1207,8 @@ public void testJoinPushdown() for (JoinOperator joinOperator : JoinOperator.values()) { if (joinOperator == FULL_JOIN && !hasBehavior(SUPPORTS_JOIN_PUSHDOWN_WITH_FULL_JOIN)) { - // Covered by verifySupportsJoinPushdownWithFullJoinDeclaration + assertThat(query(session, "SELECT r.name, n.name FROM nation n FULL JOIN region r ON n.regionkey = r.regionkey")) + .joinIsNotFullyPushedDown(); continue; } From 3b1cd7f549c63cd242670850edbf114ab1d389a7 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 1 Dec 2023 18:11:08 +0100 Subject: [PATCH 047/350] Create test table once in testJoinPushdown There is no need to recreate the test table for overy of the `for (JoinOperator` loop runs. --- .../plugin/jdbc/BaseJdbcConnectorTest.java | 48 +++++++++---------- 1 file changed, 24 insertions(+), 24 deletions(-) diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java index b38c6b95b6260..58f8b07d228fe 100644 --- a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java @@ -1205,31 +1205,31 @@ public void testJoinPushdown() return; } - for (JoinOperator joinOperator : JoinOperator.values()) { - if (joinOperator == FULL_JOIN && !hasBehavior(SUPPORTS_JOIN_PUSHDOWN_WITH_FULL_JOIN)) { - assertThat(query(session, "SELECT r.name, n.name FROM nation n FULL JOIN region r ON n.regionkey = r.regionkey")) - .joinIsNotFullyPushedDown(); - continue; - } + try (TestTable nationLowercaseTable = new TestTable( + // If a connector supports Join pushdown, but does not allow CTAS, we need to make the table creation here overridable. + getQueryRunner()::execute, + "nation_lowercase", + "AS SELECT nationkey, lower(name) name, regionkey FROM nation")) { + for (JoinOperator joinOperator : JoinOperator.values()) { + if (joinOperator == FULL_JOIN && !hasBehavior(SUPPORTS_JOIN_PUSHDOWN_WITH_FULL_JOIN)) { + assertThat(query(session, "SELECT r.name, n.name FROM nation n FULL JOIN region r ON n.regionkey = r.regionkey")) + .joinIsNotFullyPushedDown(); + continue; + } + + // Disable DF here for the sake of negative test cases' expected plan. With DF enabled, some operators return in DF's FilterNode and some do not. + Session withoutDynamicFiltering = Session.builder(session) + .setSystemProperty("enable_dynamic_filtering", "false") + .build(); + + String notDistinctOperator = "IS NOT DISTINCT FROM"; + List nonEqualities = Stream.concat( + Stream.of(JoinCondition.Operator.values()) + .filter(operator -> operator != JoinCondition.Operator.EQUAL) + .map(JoinCondition.Operator::getValue), + Stream.of(notDistinctOperator)) + .collect(toImmutableList()); - // Disable DF here for the sake of negative test cases' expected plan. With DF enabled, some operators return in DF's FilterNode and some do not. - Session withoutDynamicFiltering = Session.builder(session) - .setSystemProperty("enable_dynamic_filtering", "false") - .build(); - - String notDistinctOperator = "IS NOT DISTINCT FROM"; - List nonEqualities = Stream.concat( - Stream.of(JoinCondition.Operator.values()) - .filter(operator -> operator != JoinCondition.Operator.EQUAL) - .map(JoinCondition.Operator::getValue), - Stream.of(notDistinctOperator)) - .collect(toImmutableList()); - - try (TestTable nationLowercaseTable = new TestTable( - // If a connector supports Join pushdown, but does not allow CTAS, we need to make the table creation here overridable. - getQueryRunner()::execute, - "nation_lowercase", - "AS SELECT nationkey, lower(name) name, regionkey FROM nation")) { // basic case assertThat(query(session, format("SELECT r.name, n.name FROM nation n %s region r ON n.regionkey = r.regionkey", joinOperator))).isFullyPushedDown(); From 5fe7e12cbfbf217f4c93efaf02870540e31ad3b2 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 8 Dec 2023 13:48:31 +0100 Subject: [PATCH 048/350] Remove redundant test Verification that "!SUPPORTS_JOIN_PUSHDOWN => join pushdown does not occur" is covered by `testJoinPushdown`, so `verifySupportsJoinPushdownDeclaration` was redundant. --- .../trino/plugin/jdbc/BaseJdbcConnectorTest.java | 15 --------------- 1 file changed, 15 deletions(-) diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java index 58f8b07d228fe..97c158051b2b8 100644 --- a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java @@ -1179,21 +1179,6 @@ public void testJoinPushdownDisabled() .joinIsNotFullyPushedDown(); } - /** - * Verify !SUPPORTS_JOIN_PUSHDOWN declaration is true. - */ - @Test - public void verifySupportsJoinPushdownDeclaration() - { - if (hasBehavior(SUPPORTS_JOIN_PUSHDOWN)) { - // Covered by testJoinPushdown - return; - } - - assertThat(query(joinPushdownEnabled(getSession()), "SELECT r.name, n.name FROM nation n JOIN region r ON n.regionkey = r.regionkey")) - .joinIsNotFullyPushedDown(); - } - @Test public void testJoinPushdown() { From c2c798993853d154619d4774f59416c07768ff8f Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 8 Dec 2023 13:35:38 +0100 Subject: [PATCH 049/350] Monitor number of active query runners Monitor number of concurrently active `DistributedQueryRunner` instances. --- .../main/java/io/trino/testing/DistributedQueryRunner.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/testing/trino-testing/src/main/java/io/trino/testing/DistributedQueryRunner.java b/testing/trino-testing/src/main/java/io/trino/testing/DistributedQueryRunner.java index e5a8b1b3084bd..36f485b11153c 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/DistributedQueryRunner.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/DistributedQueryRunner.java @@ -69,6 +69,7 @@ import java.util.Map; import java.util.Optional; import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -95,6 +96,7 @@ public class DistributedQueryRunner { private static final Logger log = Logger.get(DistributedQueryRunner.class); private static final String ENVIRONMENT = "testing"; + private static final AtomicInteger unclosedInstances = new AtomicInteger(); private TestingDiscoveryServer discoveryServer; private TestingTrinoServer coordinator; @@ -197,7 +199,7 @@ private DistributedQueryRunner( this.trinoClient = closer.register(testingTrinoClientFactory.create(coordinator, defaultSession)); ensureNodesGloballyVisible(); - log.info("Created DistributedQueryRunner in %s", nanosSince(start)); + log.info("Created DistributedQueryRunner in %s (unclosed instances = %s)", nanosSince(start), unclosedInstances.incrementAndGet()); } private TestingTrinoServer createServer( @@ -593,6 +595,7 @@ public final void close() servers.clear(); functionBundles.clear(); plugins.clear(); + unclosedInstances.decrementAndGet(); trinoClient = null; closed = true; } From e9fbe6d5a04bc991b2b74bb549a7a25917a2c26c Mon Sep 17 00:00:00 2001 From: Assaf Bern Date: Wed, 6 Dec 2023 15:00:58 +0200 Subject: [PATCH 050/350] Rely only on table handle for stopping condition In order to avoid a potential infinite loop at IterativeOptimizer with Iceberg --- .../io/trino/plugin/iceberg/IcebergMetadata.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java index c10307912c06f..34c85ee6a89ae 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java @@ -2543,11 +2543,11 @@ else if (isMetadataColumnId(columnHandle.getId())) { remainingConstraint = TupleDomain.withColumnDomains(newUnenforced).intersect(TupleDomain.withColumnDomains(unsupported)); } - Set newConstraintColumns = constraint.getPredicateColumns() - .map(columnHandles -> columnHandles.stream() - .map(columnHandle -> (IcebergColumnHandle) columnHandle) - .collect(toImmutableSet())) - .orElse(ImmutableSet.of()); + Set newConstraintColumns = Streams.concat( + table.getConstraintColumns().stream(), + constraint.getPredicateColumns().orElseGet(ImmutableSet::of).stream() + .map(columnHandle -> (IcebergColumnHandle) columnHandle)) + .collect(toImmutableSet()); if (newEnforcedConstraint.equals(table.getEnforcedPredicate()) && newUnenforcedConstraint.equals(table.getUnenforcedPredicate()) @@ -2574,7 +2574,7 @@ else if (isMetadataColumnId(columnHandle.getId())) { table.getStorageProperties(), table.isRecordScannedFiles(), table.getMaxScannedFileSize(), - Sets.union(table.getConstraintColumns(), newConstraintColumns), + newConstraintColumns, table.getForAnalyze()), remainingConstraint.transformKeys(ColumnHandle.class::cast), extractionResult.remainingExpression(), From 81b7638df6f8b7695b756947c1544794bbca7acd Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Fri, 8 Dec 2023 11:35:48 +0100 Subject: [PATCH 051/350] Upgrade GCS connector to 3.0.0 On previous version: ``` public static final ImmutableList DEFAULT_SCOPES = ImmutableList.of("https://www.googleapis.com/auth/cloud-platform"); ``` --- .../hdfs/gcs/GcsAccessTokenProvider.java | 4 ++- .../io/trino/hdfs/gcs/GcsStorageFactory.java | 25 ++++++++++--------- .../GoogleGcsConfigurationInitializer.java | 14 ++++++----- pom.xml | 2 +- 4 files changed, 25 insertions(+), 20 deletions(-) diff --git a/lib/trino-hdfs/src/main/java/io/trino/hdfs/gcs/GcsAccessTokenProvider.java b/lib/trino-hdfs/src/main/java/io/trino/hdfs/gcs/GcsAccessTokenProvider.java index 215e6c0d4b286..21b4877ff704a 100644 --- a/lib/trino-hdfs/src/main/java/io/trino/hdfs/gcs/GcsAccessTokenProvider.java +++ b/lib/trino-hdfs/src/main/java/io/trino/hdfs/gcs/GcsAccessTokenProvider.java @@ -16,6 +16,8 @@ import com.google.cloud.hadoop.util.AccessTokenProvider; import org.apache.hadoop.conf.Configuration; +import java.time.Instant; + import static com.google.common.base.Strings.nullToEmpty; import static java.util.concurrent.TimeUnit.HOURS; @@ -29,7 +31,7 @@ public class GcsAccessTokenProvider @Override public AccessToken getAccessToken() { - return new AccessToken(nullToEmpty(config.get(GCS_ACCESS_TOKEN_CONF)), EXPIRATION_TIME_MILLISECONDS); + return new AccessToken(nullToEmpty(config.get(GCS_ACCESS_TOKEN_CONF)), Instant.now().plusMillis(EXPIRATION_TIME_MILLISECONDS)); } @Override diff --git a/lib/trino-hdfs/src/main/java/io/trino/hdfs/gcs/GcsStorageFactory.java b/lib/trino-hdfs/src/main/java/io/trino/hdfs/gcs/GcsStorageFactory.java index 2579685950fc6..55384769f2592 100644 --- a/lib/trino-hdfs/src/main/java/io/trino/hdfs/gcs/GcsStorageFactory.java +++ b/lib/trino-hdfs/src/main/java/io/trino/hdfs/gcs/GcsStorageFactory.java @@ -13,14 +13,14 @@ */ package io.trino.hdfs.gcs; -import com.google.cloud.hadoop.repackaged.gcs.com.google.api.client.googleapis.auth.oauth2.GoogleCredential; import com.google.cloud.hadoop.repackaged.gcs.com.google.api.client.http.HttpTransport; import com.google.cloud.hadoop.repackaged.gcs.com.google.api.client.json.jackson2.JacksonFactory; import com.google.cloud.hadoop.repackaged.gcs.com.google.api.services.storage.Storage; +import com.google.cloud.hadoop.repackaged.gcs.com.google.auth.oauth2.GoogleCredentials; import com.google.cloud.hadoop.repackaged.gcs.com.google.cloud.hadoop.gcsio.GoogleCloudStorageOptions; -import com.google.cloud.hadoop.repackaged.gcs.com.google.cloud.hadoop.util.CredentialFactory; import com.google.cloud.hadoop.repackaged.gcs.com.google.cloud.hadoop.util.HttpTransportFactory; import com.google.cloud.hadoop.repackaged.gcs.com.google.cloud.hadoop.util.RetryHttpInitializer; +import com.google.cloud.hadoop.repackaged.gcs.com.google.cloud.hadoop.util.RetryHttpInitializerOptions; import com.google.inject.Inject; import io.trino.hdfs.HdfsContext; import io.trino.hdfs.HdfsEnvironment; @@ -31,10 +31,10 @@ import java.io.FileInputStream; import java.io.IOException; import java.io.InputStream; -import java.time.Duration; import java.util.Optional; import static com.google.cloud.hadoop.fs.gcs.TrinoGoogleHadoopFileSystemConfiguration.getGcsOptionsBuilder; +import static com.google.cloud.hadoop.repackaged.gcs.com.google.cloud.hadoop.util.HadoopCredentialsConfiguration.CLOUD_PLATFORM_SCOPE; import static com.google.common.base.Strings.nullToEmpty; import static io.trino.hdfs.gcs.GcsConfigurationProvider.GCS_OAUTH_KEY; import static io.trino.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR; @@ -46,7 +46,7 @@ public class GcsStorageFactory private static final String APPLICATION_NAME = "Trino"; private final boolean useGcsAccessToken; - private final Optional jsonGoogleCredential; + private final Optional jsonGoogleCredential; @Inject public GcsStorageFactory(HiveGcsConfig hiveGcsConfig) @@ -58,12 +58,12 @@ public GcsStorageFactory(HiveGcsConfig hiveGcsConfig) String jsonKeyFilePath = hiveGcsConfig.getJsonKeyFilePath(); if (jsonKey != null) { try (InputStream inputStream = new ByteArrayInputStream(jsonKey.getBytes(UTF_8))) { - jsonGoogleCredential = Optional.of(GoogleCredential.fromStream(inputStream).createScoped(CredentialFactory.DEFAULT_SCOPES)); + jsonGoogleCredential = Optional.of(GoogleCredentials.fromStream(inputStream).createScoped(CLOUD_PLATFORM_SCOPE)); } } else if (jsonKeyFilePath != null) { try (FileInputStream inputStream = new FileInputStream(jsonKeyFilePath)) { - jsonGoogleCredential = Optional.of(GoogleCredential.fromStream(inputStream).createScoped(CredentialFactory.DEFAULT_SCOPES)); + jsonGoogleCredential = Optional.of(GoogleCredentials.fromStream(inputStream).createScoped(CLOUD_PLATFORM_SCOPE)); } } else { @@ -76,22 +76,23 @@ public Storage create(HdfsEnvironment environment, HdfsContext context, Path pat try { GoogleCloudStorageOptions gcsOptions = getGcsOptionsBuilder(environment.getConfiguration(context, path)).build(); HttpTransport httpTransport = HttpTransportFactory.createHttpTransport( - gcsOptions.getTransportType(), gcsOptions.getProxyAddress(), gcsOptions.getProxyUsername(), - gcsOptions.getProxyPassword(), - Duration.ofMillis(gcsOptions.getHttpRequestReadTimeout())); - GoogleCredential credential; + gcsOptions.getProxyPassword()); + GoogleCredentials credential; if (useGcsAccessToken) { String accessToken = nullToEmpty(context.getIdentity().getExtraCredentials().get(GCS_OAUTH_KEY)); try (ByteArrayInputStream inputStream = new ByteArrayInputStream(accessToken.getBytes(UTF_8))) { - credential = GoogleCredential.fromStream(inputStream).createScoped(CredentialFactory.DEFAULT_SCOPES); + credential = GoogleCredentials.fromStream(inputStream).createScoped(CLOUD_PLATFORM_SCOPE); } } else { credential = jsonGoogleCredential.orElseThrow(() -> new IllegalStateException("GCS credentials not configured")); } - return new Storage.Builder(httpTransport, JacksonFactory.getDefaultInstance(), new RetryHttpInitializer(credential, APPLICATION_NAME)) + return new Storage.Builder(httpTransport, JacksonFactory.getDefaultInstance(), new RetryHttpInitializer(credential, RetryHttpInitializerOptions.builder() + .setReadTimeout(gcsOptions.getHttpRequestReadTimeout()) + .setMaxRequestRetries(gcsOptions.getMaxHttpRequestRetries()) + .build())) .setApplicationName(APPLICATION_NAME) .build(); } diff --git a/lib/trino-hdfs/src/main/java/io/trino/hdfs/gcs/GoogleGcsConfigurationInitializer.java b/lib/trino-hdfs/src/main/java/io/trino/hdfs/gcs/GoogleGcsConfigurationInitializer.java index 2cb3ba0eb57e7..4e979bf2e1a3e 100644 --- a/lib/trino-hdfs/src/main/java/io/trino/hdfs/gcs/GoogleGcsConfigurationInitializer.java +++ b/lib/trino-hdfs/src/main/java/io/trino/hdfs/gcs/GoogleGcsConfigurationInitializer.java @@ -29,9 +29,11 @@ import java.util.Optional; import static com.google.cloud.hadoop.fs.gcs.GoogleHadoopFileSystemConfiguration.GCS_CONFIG_PREFIX; -import static com.google.cloud.hadoop.fs.gcs.HadoopCredentialConfiguration.ACCESS_TOKEN_PROVIDER_IMPL_SUFFIX; -import static com.google.cloud.hadoop.fs.gcs.HadoopCredentialConfiguration.ENABLE_SERVICE_ACCOUNTS_SUFFIX; -import static com.google.cloud.hadoop.fs.gcs.HadoopCredentialConfiguration.SERVICE_ACCOUNT_JSON_KEYFILE_SUFFIX; +import static com.google.cloud.hadoop.fs.gcs.HadoopCredentialsConfiguration.SERVICE_ACCOUNT_JSON_KEYFILE_SUFFIX; +import static com.google.cloud.hadoop.repackaged.gcs.com.google.cloud.hadoop.util.HadoopCredentialsConfiguration.ACCESS_TOKEN_PROVIDER_SUFFIX; +import static com.google.cloud.hadoop.repackaged.gcs.com.google.cloud.hadoop.util.HadoopCredentialsConfiguration.AUTHENTICATION_TYPE_SUFFIX; +import static com.google.cloud.hadoop.repackaged.gcs.com.google.cloud.hadoop.util.HadoopCredentialsConfiguration.AuthenticationType.ACCESS_TOKEN_PROVIDER; +import static com.google.cloud.hadoop.repackaged.gcs.com.google.cloud.hadoop.util.HadoopCredentialsConfiguration.AuthenticationType.SERVICE_ACCOUNT_JSON_KEYFILE; import static java.nio.file.attribute.PosixFilePermission.OWNER_READ; import static java.nio.file.attribute.PosixFilePermission.OWNER_WRITE; @@ -72,12 +74,12 @@ public void initializeConfiguration(Configuration config) if (useGcsAccessToken) { // use oauth token to authenticate with Google Cloud Storage - config.setBoolean(GCS_CONFIG_PREFIX + ENABLE_SERVICE_ACCOUNTS_SUFFIX.getKey(), false); - config.setClass(GCS_CONFIG_PREFIX + ACCESS_TOKEN_PROVIDER_IMPL_SUFFIX.getKey(), GcsAccessTokenProvider.class, AccessTokenProvider.class); + config.setEnum(GCS_CONFIG_PREFIX + AUTHENTICATION_TYPE_SUFFIX.getKey(), ACCESS_TOKEN_PROVIDER); + config.setClass(GCS_CONFIG_PREFIX + ACCESS_TOKEN_PROVIDER_SUFFIX.getKey(), GcsAccessTokenProvider.class, AccessTokenProvider.class); } else if (jsonKeyFilePath != null) { // use service account key file - config.setBoolean(GCS_CONFIG_PREFIX + ENABLE_SERVICE_ACCOUNTS_SUFFIX.getKey(), true); + config.setEnum(GCS_CONFIG_PREFIX + AUTHENTICATION_TYPE_SUFFIX.getKey(), SERVICE_ACCOUNT_JSON_KEYFILE); config.set(GCS_CONFIG_PREFIX + SERVICE_ACCOUNT_JSON_KEYFILE_SUFFIX.getKey(), jsonKeyFilePath); } } diff --git a/pom.xml b/pom.xml index f340be9b53e2c..7faff11c08d13 100644 --- a/pom.xml +++ b/pom.xml @@ -513,7 +513,7 @@ com.google.cloud.bigdataoss gcs-connector - hadoop3-2.2.18 + 3.0.0 shaded From dd71cfeac7881d353845ae66bf3d82be3d7b2748 Mon Sep 17 00:00:00 2001 From: Martin Traverso Date: Thu, 7 Dec 2023 17:17:22 -0800 Subject: [PATCH 052/350] Remove reference to internal TestNG API --- .../src/test/java/io/trino/sql/ExpressionTestUtils.java | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/core/trino-main/src/test/java/io/trino/sql/ExpressionTestUtils.java b/core/trino-main/src/test/java/io/trino/sql/ExpressionTestUtils.java index f289a9e287dc0..008dd7f0e21ec 100644 --- a/core/trino-main/src/test/java/io/trino/sql/ExpressionTestUtils.java +++ b/core/trino-main/src/test/java/io/trino/sql/ExpressionTestUtils.java @@ -44,9 +44,6 @@ import static io.trino.sql.analyzer.TypeSignatureTranslator.toSqlType; import static io.trino.sql.planner.TypeAnalyzer.createTestingTypeAnalyzer; import static io.trino.transaction.TransactionBuilder.transaction; -import static org.testng.internal.EclipseInterface.ASSERT_LEFT; -import static org.testng.internal.EclipseInterface.ASSERT_MIDDLE; -import static org.testng.internal.EclipseInterface.ASSERT_RIGHT; public final class ExpressionTestUtils { @@ -73,7 +70,7 @@ private static void failNotEqual(Object actual, Object expected, String message) if (message != null) { formatted = message + " "; } - throw new AssertionError(formatted + ASSERT_LEFT + expected + ASSERT_MIDDLE + actual + ASSERT_RIGHT); + throw new AssertionError(formatted + " expected [" + expected + "] but found [" + actual + "]"); } public static Expression createExpression(Session session, String expression, TransactionManager transactionManager, PlannerContext plannerContext, TypeProvider symbolTypes) From 7dc77ca0763bb7c2a85336e578747e333ec12d19 Mon Sep 17 00:00:00 2001 From: Martin Traverso Date: Thu, 7 Dec 2023 13:12:37 -0800 Subject: [PATCH 053/350] Remove unnecessary data providers --- .../TestGroupedTopNRankAccumulator.java | 61 ++- .../operator/TestGroupedTopNRankBuilder.java | 27 +- .../TestGroupedTopNRowNumberBuilder.java | 34 +- .../operator/TestHashAggregationOperator.java | 139 +++-- .../operator/TestHashSemiJoinOperator.java | 67 ++- .../operator/TestMarkDistinctOperator.java | 48 +- .../trino/operator/TestOrderByOperator.java | 64 ++- .../operator/TestTopNPeerGroupLookup.java | 31 +- .../io/trino/operator/TestWindowOperator.java | 186 +++++-- .../AbstractTestApproximateCountDistinct.java | 55 +- .../TestAggregationMaskCompiler.java | 60 +- .../TestApproximateCountDistinctBoolean.java | 33 +- .../TestApproximateSetGenericBoolean.java | 32 +- .../TestDecimalAverageAggregation.java | 56 +- ...malWithOverflowAndLongStateSerializer.java | 48 +- ...ongDecimalWithOverflowStateSerializer.java | 32 +- .../operator/exchange/TestLocalExchange.java | 68 ++- .../io/trino/operator/join/JoinTestUtils.java | 6 +- .../operator/join/TestHashJoinOperator.java | 488 ++++++++++++----- .../join/unspilled/TestHashJoinOperator.java | 511 +++++++++++++++--- .../operator/output/TestPagePartitioner.java | 351 ++++++++---- .../output/TestPositionsAppender.java | 389 ++++++------- .../TestDictionaryAwarePageProjection.java | 72 +-- .../server/security/TestResourceSecurity.java | 37 +- .../oauth2/TestJweTokenSerializer.java | 34 +- .../security/oauth2/TestOidcDiscovery.java | 37 +- .../TestCounterBasedAnonymizer.java | 71 ++- .../TestValidateScaledWritersUsage.java | 76 ++- ...TestLongLong2LongOpenCustomBigHashMap.java | 37 +- ...aultTolerantExecutionDynamicFiltering.java | 6 +- ...stractTestCoordinatorDynamicFiltering.java | 131 +++-- 31 files changed, 2174 insertions(+), 1113 deletions(-) diff --git a/core/trino-main/src/test/java/io/trino/operator/TestGroupedTopNRankAccumulator.java b/core/trino-main/src/test/java/io/trino/operator/TestGroupedTopNRankAccumulator.java index 7cf8f1f114f51..d95953791bf9a 100644 --- a/core/trino-main/src/test/java/io/trino/operator/TestGroupedTopNRankAccumulator.java +++ b/core/trino-main/src/test/java/io/trino/operator/TestGroupedTopNRankAccumulator.java @@ -15,14 +15,12 @@ import io.trino.array.LongBigArray; import it.unimi.dsi.fastutil.longs.LongArrayList; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import java.util.ArrayList; import java.util.Arrays; import java.util.List; -import static com.google.common.collect.Lists.cartesianProduct; import static java.lang.Math.min; import static org.assertj.core.api.Assertions.assertThat; @@ -43,27 +41,20 @@ public long hashCode(long rowId) } }; - @DataProvider - public static Object[][] parameters() - { - List topNs = Arrays.asList(1, 2, 3); - List valueCounts = Arrays.asList(0, 1, 2, 4, 8); - List groupCounts = Arrays.asList(1, 2, 3); - List drainWithRankings = Arrays.asList(true, false); - return to2DArray(cartesianProduct(topNs, valueCounts, groupCounts, drainWithRankings)); - } - - private static Object[][] to2DArray(List> nestedList) + @Test + public void testSinglePeerGroupInsert() { - Object[][] array = new Object[nestedList.size()][]; - for (int i = 0; i < nestedList.size(); i++) { - array[i] = nestedList.get(i).toArray(); + for (int topN : Arrays.asList(1, 2, 3)) { + for (int valueCount : Arrays.asList(0, 1, 2, 4, 8)) { + for (int groupCount : Arrays.asList(1, 2, 3)) { + testSinglePeerGroupInsert(topN, valueCount, groupCount, true); + testSinglePeerGroupInsert(topN, valueCount, groupCount, false); + } + } } - return array; } - @Test(dataProvider = "parameters") - public void testSinglePeerGroupInsert(int topN, long valueCount, long groupCount, boolean drainWithRanking) + private void testSinglePeerGroupInsert(int topN, long valueCount, long groupCount, boolean drainWithRanking) { List evicted = new LongArrayList(); GroupedTopNRankAccumulator accumulator = new GroupedTopNRankAccumulator(STRATEGY, topN, evicted::add); @@ -103,8 +94,20 @@ public void testSinglePeerGroupInsert(int topN, long valueCount, long groupCount } } - @Test(dataProvider = "parameters") - public void testIncreasingAllUniqueValues(int topN, long valueCount, long groupCount, boolean drainWithRanking) + @Test + public void testIncreasingAllUniqueValues() + { + for (int topN : Arrays.asList(1, 2, 3)) { + for (int valueCount : Arrays.asList(0, 1, 2, 4, 8)) { + for (int groupCount : Arrays.asList(1, 2, 3)) { + testIncreasingAllUniqueValues(topN, valueCount, groupCount, true); + testIncreasingAllUniqueValues(topN, valueCount, groupCount, false); + } + } + } + } + + private void testIncreasingAllUniqueValues(int topN, long valueCount, long groupCount, boolean drainWithRanking) { List evicted = new LongArrayList(); GroupedTopNRankAccumulator accumulator = new GroupedTopNRankAccumulator(STRATEGY, topN, evicted::add); @@ -144,8 +147,20 @@ public void testIncreasingAllUniqueValues(int topN, long valueCount, long groupC } } - @Test(dataProvider = "parameters") - public void testDecreasingAllUniqueValues(int topN, long valueCount, long groupCount, boolean drainWithRanking) + @Test + public void testDecreasingAllUniqueValues() + { + for (int topN : Arrays.asList(1, 2, 3)) { + for (int valueCount : Arrays.asList(0, 1, 2, 4, 8)) { + for (int groupCount : Arrays.asList(1, 2, 3)) { + testDecreasingAllUniqueValues(topN, valueCount, groupCount, true); + testDecreasingAllUniqueValues(topN, valueCount, groupCount, false); + } + } + } + } + + private void testDecreasingAllUniqueValues(int topN, long valueCount, long groupCount, boolean drainWithRanking) { List evicted = new LongArrayList(); GroupedTopNRankAccumulator accumulator = new GroupedTopNRankAccumulator(STRATEGY, topN, evicted::add); diff --git a/core/trino-main/src/test/java/io/trino/operator/TestGroupedTopNRankBuilder.java b/core/trino-main/src/test/java/io/trino/operator/TestGroupedTopNRankBuilder.java index 5b5a67a8b7d56..5fa6210a3dd96 100644 --- a/core/trino-main/src/test/java/io/trino/operator/TestGroupedTopNRankBuilder.java +++ b/core/trino-main/src/test/java/io/trino/operator/TestGroupedTopNRankBuilder.java @@ -19,7 +19,6 @@ import io.trino.spi.type.TypeOperators; import io.trino.sql.gen.JoinCompiler; import io.trino.type.BlockTypeOperators; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import java.util.List; @@ -39,12 +38,6 @@ public class TestGroupedTopNRankBuilder { - @DataProvider - public static Object[][] produceRanking() - { - return new Object[][] {{true}, {false}}; - } - @Test public void testEmptyInput() { @@ -74,8 +67,14 @@ public long hashCode(Page page, int position) assertThat(groupedTopNBuilder.buildResult().hasNext()).isFalse(); } - @Test(dataProvider = "produceRanking") - public void testSingleGroupTopN(boolean produceRanking) + @Test + public void testSingleGroupTopN() + { + testSingleGroupTopN(true); + testSingleGroupTopN(false); + } + + private void testSingleGroupTopN(boolean produceRanking) { TypeOperators typeOperators = new TypeOperators(); BlockTypeOperators blockTypeOperators = new BlockTypeOperators(typeOperators); @@ -133,8 +132,14 @@ public void testSingleGroupTopN(boolean produceRanking) assertPageEquals(outputTypes, getOnlyElement(output), expected); } - @Test(dataProvider = "produceRanking") - public void testMultiGroupTopN(boolean produceRanking) + @Test + public void testMultiGroupTopN() + { + testMultiGroupTopN(true); + testMultiGroupTopN(false); + } + + private void testMultiGroupTopN(boolean produceRanking) { TypeOperators typeOperators = new TypeOperators(); BlockTypeOperators blockTypeOperators = new BlockTypeOperators(typeOperators); diff --git a/core/trino-main/src/test/java/io/trino/operator/TestGroupedTopNRowNumberBuilder.java b/core/trino-main/src/test/java/io/trino/operator/TestGroupedTopNRowNumberBuilder.java index 06d1acce72194..7ad787c6b95bd 100644 --- a/core/trino-main/src/test/java/io/trino/operator/TestGroupedTopNRowNumberBuilder.java +++ b/core/trino-main/src/test/java/io/trino/operator/TestGroupedTopNRowNumberBuilder.java @@ -18,7 +18,6 @@ import io.trino.spi.type.Type; import io.trino.spi.type.TypeOperators; import io.trino.sql.gen.JoinCompiler; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import java.util.List; @@ -36,19 +35,6 @@ public class TestGroupedTopNRowNumberBuilder { private static final TypeOperators TYPE_OPERATORS_CACHE = new TypeOperators(); - @DataProvider - public static Object[][] produceRowNumbers() - { - return new Object[][] {{true}, {false}}; - } - - @DataProvider - public static Object[][] pageRowCounts() - { - // make either page or row count > 1024 to expand the big arrays - return new Object[][] {{10000, 20}, {20, 10000}}; - } - @Test public void testEmptyInput() { @@ -64,8 +50,14 @@ public void testEmptyInput() assertThat(groupedTopNBuilder.buildResult().hasNext()).isFalse(); } - @Test(dataProvider = "produceRowNumbers") - public void testMultiGroupTopN(boolean produceRowNumbers) + @Test + public void testMultiGroupTopN() + { + testMultiGroupTopN(true); + testMultiGroupTopN(false); + } + + private void testMultiGroupTopN(boolean produceRowNumbers) { List types = ImmutableList.of(BIGINT, DOUBLE); List input = rowPagesBuilder(types) @@ -131,8 +123,14 @@ public void testMultiGroupTopN(boolean produceRowNumbers) } } - @Test(dataProvider = "produceRowNumbers") - public void testSingleGroupTopN(boolean produceRowNumbers) + @Test + public void testSingleGroupTopN() + { + testSingleGroupTopN(true); + testSingleGroupTopN(false); + } + + private void testSingleGroupTopN(boolean produceRowNumbers) { List types = ImmutableList.of(BIGINT, DOUBLE); List input = rowPagesBuilder(types) diff --git a/core/trino-main/src/test/java/io/trino/operator/TestHashAggregationOperator.java b/core/trino-main/src/test/java/io/trino/operator/TestHashAggregationOperator.java index babf7dbdf044c..cbf8cf4a800ad 100644 --- a/core/trino-main/src/test/java/io/trino/operator/TestHashAggregationOperator.java +++ b/core/trino-main/src/test/java/io/trino/operator/TestHashAggregationOperator.java @@ -42,7 +42,6 @@ import io.trino.testing.TestingTaskContext; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import java.io.IOException; @@ -129,36 +128,24 @@ public void tearDown() scheduledExecutor.shutdownNow(); } - @DataProvider(name = "hashEnabled") - public static Object[][] hashEnabled() - { - return new Object[][] {{true}, {false}}; - } - - @DataProvider(name = "hashEnabledAndMemoryLimitForMergeValues") - public static Object[][] hashEnabledAndMemoryLimitForMergeValuesProvider() + @Test + public void testHashAggregation() { - return new Object[][] { - {true, true, true, 8, Integer.MAX_VALUE}, - {true, true, false, 8, Integer.MAX_VALUE}, - {false, false, false, 0, 0}, - {false, true, true, 0, 0}, - {false, true, false, 0, 0}, - {false, true, true, 8, 0}, - {false, true, false, 8, 0}, - {false, true, true, 8, Integer.MAX_VALUE}, - {false, true, false, 8, Integer.MAX_VALUE}}; + testHashAggregation(true, true, true, 8, Integer.MAX_VALUE); + testHashAggregation(true, true, false, 8, Integer.MAX_VALUE); + testHashAggregation(false, false, false, 0, 0); + testHashAggregation(false, true, true, 0, 0); + testHashAggregation(false, true, false, 0, 0); + testHashAggregation(false, true, true, 8, 0); + testHashAggregation(false, true, false, 8, 0); + testHashAggregation(false, true, true, 8, Integer.MAX_VALUE); + testHashAggregation(false, true, false, 8, Integer.MAX_VALUE); } - @DataProvider - public Object[][] dataType() + private void testHashAggregation(boolean hashEnabled, boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimitForMerge, long memoryLimitForMergeWithMemory) { - return new Object[][] {{VARCHAR}, {BIGINT}}; - } + DummySpillerFactory spillerFactory = new DummySpillerFactory(); - @Test(dataProvider = "hashEnabledAndMemoryLimitForMergeValues") - public void testHashAggregation(boolean hashEnabled, boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimitForMerge, long memoryLimitForMergeWithMemory) - { // make operator produce multiple pages during finish phase int numberOfRows = 40_000; TestingAggregationFunction countVarcharColumn = FUNCTION_RESOLUTION.getAggregateFunction("count", fromTypes(VARCHAR)); @@ -215,8 +202,21 @@ public void testHashAggregation(boolean hashEnabled, boolean spillEnabled, boole .isTrue(); } - @Test(dataProvider = "hashEnabledAndMemoryLimitForMergeValues") - public void testHashAggregationWithGlobals(boolean hashEnabled, boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimitForMerge, long memoryLimitForMergeWithMemory) + @Test + public void testHashAggregationWithGlobals() + { + testHashAggregationWithGlobals(true, true, true, 8, Integer.MAX_VALUE); + testHashAggregationWithGlobals(true, true, false, 8, Integer.MAX_VALUE); + testHashAggregationWithGlobals(false, false, false, 0, 0); + testHashAggregationWithGlobals(false, true, true, 0, 0); + testHashAggregationWithGlobals(false, true, false, 0, 0); + testHashAggregationWithGlobals(false, true, true, 8, 0); + testHashAggregationWithGlobals(false, true, false, 8, 0); + testHashAggregationWithGlobals(false, true, true, 8, Integer.MAX_VALUE); + testHashAggregationWithGlobals(false, true, false, 8, Integer.MAX_VALUE); + } + + private void testHashAggregationWithGlobals(boolean hashEnabled, boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimitForMerge, long memoryLimitForMergeWithMemory) { TestingAggregationFunction countVarcharColumn = FUNCTION_RESOLUTION.getAggregateFunction("count", fromTypes(VARCHAR)); TestingAggregationFunction countBooleanColumn = FUNCTION_RESOLUTION.getAggregateFunction("count", fromTypes(BOOLEAN)); @@ -263,8 +263,21 @@ public void testHashAggregationWithGlobals(boolean hashEnabled, boolean spillEna assertOperatorEqualsIgnoreOrder(operatorFactory, driverContext, input, expected, hashEnabled, Optional.of(groupByChannels.size()), revokeMemoryWhenAddingPages); } - @Test(dataProvider = "hashEnabledAndMemoryLimitForMergeValues") - public void testHashAggregationMemoryReservation(boolean hashEnabled, boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimitForMerge, long memoryLimitForMergeWithMemory) + @Test + public void testHashAggregationMemoryReservation() + { + testHashAggregationMemoryReservation(true, true, true, 8, Integer.MAX_VALUE); + testHashAggregationMemoryReservation(true, true, false, 8, Integer.MAX_VALUE); + testHashAggregationMemoryReservation(false, false, false, 0, 0); + testHashAggregationMemoryReservation(false, true, true, 0, 0); + testHashAggregationMemoryReservation(false, true, false, 0, 0); + testHashAggregationMemoryReservation(false, true, true, 8, 0); + testHashAggregationMemoryReservation(false, true, false, 8, 0); + testHashAggregationMemoryReservation(false, true, true, 8, Integer.MAX_VALUE); + testHashAggregationMemoryReservation(false, true, false, 8, Integer.MAX_VALUE); + } + + private void testHashAggregationMemoryReservation(boolean hashEnabled, boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimitForMerge, long memoryLimitForMergeWithMemory) { TestingAggregationFunction arrayAggColumn = FUNCTION_RESOLUTION.getAggregateFunction("array_agg", fromTypes(BIGINT)); @@ -308,8 +321,14 @@ public void testHashAggregationMemoryReservation(boolean hashEnabled, boolean sp assertThat(getOnlyElement(operator.getOperatorContext().getNestedOperatorStats()).getRevocableMemoryReservation().toBytes()).isEqualTo(0); } - @Test(dataProvider = "hashEnabled", expectedExceptions = ExceededMemoryLimitException.class, expectedExceptionsMessageRegExp = "Query exceeded per-node memory limit of 10B.*") - public void testMemoryLimit(boolean hashEnabled) + @Test(expectedExceptions = ExceededMemoryLimitException.class, expectedExceptionsMessageRegExp = "Query exceeded per-node memory limit of 10B.*") + public void testMemoryLimit() + { + testMemoryLimit(true); + testMemoryLimit(false); + } + + private void testMemoryLimit(boolean hashEnabled) { TestingAggregationFunction maxVarcharColumn = FUNCTION_RESOLUTION.getAggregateFunction("max", fromTypes(VARCHAR)); @@ -347,8 +366,21 @@ public void testMemoryLimit(boolean hashEnabled) toPages(operatorFactory, driverContext, input); } - @Test(dataProvider = "hashEnabledAndMemoryLimitForMergeValues") - public void testHashBuilderResize(boolean hashEnabled, boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimitForMerge, long memoryLimitForMergeWithMemory) + @Test + public void testHashBuilderResize() + { + testHashBuilderResize(true, true, true, 8, Integer.MAX_VALUE); + testHashBuilderResize(true, true, false, 8, Integer.MAX_VALUE); + testHashBuilderResize(false, false, false, 0, 0); + testHashBuilderResize(false, true, true, 0, 0); + testHashBuilderResize(false, true, false, 0, 0); + testHashBuilderResize(false, true, true, 8, 0); + testHashBuilderResize(false, true, false, 8, 0); + testHashBuilderResize(false, true, true, 8, Integer.MAX_VALUE); + testHashBuilderResize(false, true, false, 8, Integer.MAX_VALUE); + } + + private void testHashBuilderResize(boolean hashEnabled, boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimitForMerge, long memoryLimitForMergeWithMemory) { BlockBuilder builder = VARCHAR.createBlockBuilder(null, 1, MAX_BLOCK_SIZE_IN_BYTES); VARCHAR.writeSlice(builder, Slices.allocate(200_000)); // this must be larger than MAX_BLOCK_SIZE_IN_BYTES, 64K @@ -388,7 +420,13 @@ public void testHashBuilderResize(boolean hashEnabled, boolean spillEnabled, boo toPages(operatorFactory, driverContext, input, revokeMemoryWhenAddingPages); } - @Test(dataProvider = "dataType") + @Test + public void testMemoryReservationYield() + { + testMemoryReservationYield(VARCHAR); + testMemoryReservationYield(BIGINT); + } + public void testMemoryReservationYield(Type type) { List input = createPagesWithDistinctHashKeys(type, 6_000, 600); @@ -426,8 +464,14 @@ public void testMemoryReservationYield(Type type) assertThat(count).isEqualTo(6_000 * 600); } - @Test(dataProvider = "hashEnabled", expectedExceptions = ExceededMemoryLimitException.class, expectedExceptionsMessageRegExp = "Query exceeded per-node memory limit of 3MB.*") - public void testHashBuilderResizeLimit(boolean hashEnabled) + @Test(expectedExceptions = ExceededMemoryLimitException.class, expectedExceptionsMessageRegExp = "Query exceeded per-node memory limit of 3MB.*") + public void testHashBuilderResizeLimit() + { + testHashBuilderResizeLimit(true); + testHashBuilderResizeLimit(false); + } + + private void testHashBuilderResizeLimit(boolean hashEnabled) { BlockBuilder builder = VARCHAR.createBlockBuilder(null, 1, MAX_BLOCK_SIZE_IN_BYTES); VARCHAR.writeSlice(builder, Slices.allocate(5_000_000)); // this must be larger than MAX_BLOCK_SIZE_IN_BYTES, 64K @@ -464,8 +508,14 @@ public void testHashBuilderResizeLimit(boolean hashEnabled) toPages(operatorFactory, driverContext, input); } - @Test(dataProvider = "hashEnabled") - public void testMultiSliceAggregationOutput(boolean hashEnabled) + @Test + public void testMultiSliceAggregationOutput() + { + testMultiSliceAggregationOutput(true); + testMultiSliceAggregationOutput(false); + } + + private void testMultiSliceAggregationOutput(boolean hashEnabled) { // estimate the number of entries required to create 1.5 pages of results // See InMemoryHashAggregationBuilder.buildTypes() @@ -499,8 +549,15 @@ public void testMultiSliceAggregationOutput(boolean hashEnabled) assertThat(toPages(operatorFactory, createDriverContext(), input).size()).isEqualTo(2); } - @Test(dataProvider = "hashEnabled") - public void testMultiplePartialFlushes(boolean hashEnabled) + @Test + public void testMultiplePartialFlushes() + throws Exception + { + testMultiplePartialFlushes(true); + testMultiplePartialFlushes(false); + } + + private void testMultiplePartialFlushes(boolean hashEnabled) throws Exception { List hashChannels = Ints.asList(0); diff --git a/core/trino-main/src/test/java/io/trino/operator/TestHashSemiJoinOperator.java b/core/trino-main/src/test/java/io/trino/operator/TestHashSemiJoinOperator.java index c43a6c412aa92..cdca808be1a7e 100644 --- a/core/trino-main/src/test/java/io/trino/operator/TestHashSemiJoinOperator.java +++ b/core/trino-main/src/test/java/io/trino/operator/TestHashSemiJoinOperator.java @@ -27,7 +27,6 @@ import io.trino.testing.MaterializedResult; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import java.util.List; @@ -71,20 +70,14 @@ public void tearDown() scheduledExecutor.shutdownNow(); } - @DataProvider(name = "hashEnabledValues") - public static Object[][] hashEnabledValuesProvider() + @Test + public void testSemiJoin() { - return new Object[][] {{true}, {false}}; + testSemiJoin(true); + testSemiJoin(false); } - @DataProvider - public Object[][] dataType() - { - return new Object[][] {{VARCHAR}, {BIGINT}}; - } - - @Test(dataProvider = "hashEnabledValues") - public void testSemiJoin(boolean hashEnabled) + private void testSemiJoin(boolean hashEnabled) { DriverContext driverContext = taskContext.addPipelineContext(0, true, true, false).addDriverContext(); @@ -148,8 +141,14 @@ public void testSemiJoin(boolean hashEnabled) OperatorAssertion.assertOperatorEquals(joinOperatorFactory, driverContext, probeInput, expected, hashEnabled, ImmutableList.of(probeTypes.size())); } - @Test(dataProvider = "hashEnabledValues") - public void testSemiJoinOnVarcharType(boolean hashEnabled) + @Test + public void testSemiJoinOnVarcharType() + { + testSemiJoinOnVarcharType(true); + testSemiJoinOnVarcharType(false); + } + + private void testSemiJoinOnVarcharType(boolean hashEnabled) { DriverContext driverContext = taskContext.addPipelineContext(0, true, true, false).addDriverContext(); @@ -213,8 +212,14 @@ public void testSemiJoinOnVarcharType(boolean hashEnabled) OperatorAssertion.assertOperatorEquals(joinOperatorFactory, driverContext, probeInput, expected, hashEnabled, ImmutableList.of(probeTypes.size())); } - @Test(dataProvider = "hashEnabledValues") - public void testBuildSideNulls(boolean hashEnabled) + @Test + public void testBuildSideNulls() + { + testBuildSideNulls(true); + testBuildSideNulls(false); + } + + private void testBuildSideNulls(boolean hashEnabled) { DriverContext driverContext = taskContext.addPipelineContext(0, true, true, false).addDriverContext(); @@ -272,8 +277,14 @@ public void testBuildSideNulls(boolean hashEnabled) OperatorAssertion.assertOperatorEquals(joinOperatorFactory, driverContext, probeInput, expected, hashEnabled, ImmutableList.of(probeTypes.size())); } - @Test(dataProvider = "hashEnabledValues") - public void testProbeSideNulls(boolean hashEnabled) + @Test + public void testProbeSideNulls() + { + testProbeSideNulls(true); + testProbeSideNulls(false); + } + + private void testProbeSideNulls(boolean hashEnabled) { DriverContext driverContext = taskContext.addPipelineContext(0, true, true, false).addDriverContext(); @@ -331,8 +342,14 @@ public void testProbeSideNulls(boolean hashEnabled) OperatorAssertion.assertOperatorEquals(joinOperatorFactory, driverContext, probeInput, expected, hashEnabled, ImmutableList.of(probeTypes.size())); } - @Test(dataProvider = "hashEnabledValues") - public void testProbeAndBuildNulls(boolean hashEnabled) + @Test + public void testProbeAndBuildNulls() + { + testProbeAndBuildNulls(true); + testProbeAndBuildNulls(false); + } + + private void testProbeAndBuildNulls(boolean hashEnabled) { DriverContext driverContext = taskContext.addPipelineContext(0, true, true, false).addDriverContext(); @@ -391,8 +408,14 @@ public void testProbeAndBuildNulls(boolean hashEnabled) OperatorAssertion.assertOperatorEquals(joinOperatorFactory, driverContext, probeInput, expected, hashEnabled, ImmutableList.of(probeTypes.size())); } - @Test(dataProvider = "hashEnabledValues", expectedExceptions = ExceededMemoryLimitException.class, expectedExceptionsMessageRegExp = "Query exceeded per-node memory limit of.*") - public void testMemoryLimit(boolean hashEnabled) + @Test(expectedExceptions = ExceededMemoryLimitException.class, expectedExceptionsMessageRegExp = "Query exceeded per-node memory limit of.*") + public void testMemoryLimit() + { + testMemoryLimit(true); + testMemoryLimit(false); + } + + private void testMemoryLimit(boolean hashEnabled) { DriverContext driverContext = createTaskContext(executor, scheduledExecutor, TEST_SESSION, DataSize.ofBytes(100)) .addPipelineContext(0, true, true, false) diff --git a/core/trino-main/src/test/java/io/trino/operator/TestMarkDistinctOperator.java b/core/trino-main/src/test/java/io/trino/operator/TestMarkDistinctOperator.java index 1657f51fb5d72..52f2b8e2efefd 100644 --- a/core/trino-main/src/test/java/io/trino/operator/TestMarkDistinctOperator.java +++ b/core/trino-main/src/test/java/io/trino/operator/TestMarkDistinctOperator.java @@ -27,7 +27,6 @@ import io.trino.testing.MaterializedResult; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import java.util.List; @@ -57,7 +56,6 @@ public class TestMarkDistinctOperator { private ExecutorService executor; private ScheduledExecutorService scheduledExecutor; - private DriverContext driverContext; private final TypeOperators typeOperators = new TypeOperators(); private final JoinCompiler joinCompiler = new JoinCompiler(typeOperators); @@ -66,9 +64,6 @@ public void setUp() { executor = newCachedThreadPool(daemonThreadsNamed(getClass().getSimpleName() + "-%s")); scheduledExecutor = newScheduledThreadPool(2, daemonThreadsNamed(getClass().getSimpleName() + "-scheduledExecutor-%s")); - driverContext = createTaskContext(executor, scheduledExecutor, TEST_SESSION) - .addPipelineContext(0, true, true, false) - .addDriverContext(); } @AfterMethod(alwaysRun = true) @@ -78,20 +73,14 @@ public void tearDown() scheduledExecutor.shutdownNow(); } - @DataProvider - public Object[][] dataType() - { - return new Object[][] {{VARCHAR}, {BIGINT}}; - } - - @DataProvider(name = "hashEnabledValues") - public static Object[][] hashEnabledValuesProvider() + @Test + public void testMarkDistinct() { - return new Object[][] {{true}, {false}}; + testMarkDistinct(true, newDriverContext()); + testMarkDistinct(false, newDriverContext()); } - @Test(dataProvider = "hashEnabledValues") - public void testMarkDistinct(boolean hashEnabled) + private void testMarkDistinct(boolean hashEnabled, DriverContext driverContext) { RowPagesBuilder rowPagesBuilder = rowPagesBuilder(hashEnabled, Ints.asList(0), BIGINT); List input = rowPagesBuilder @@ -116,8 +105,14 @@ public void testMarkDistinct(boolean hashEnabled) OperatorAssertion.assertOperatorEqualsIgnoreOrder(operatorFactory, driverContext, input, expected.build(), hashEnabled, Optional.of(1)); } - @Test(dataProvider = "hashEnabledValues") - public void testRleDistinctMask(boolean hashEnabled) + @Test + public void testRleDistinctMask() + { + testRleDistinctMask(true, newDriverContext()); + testRleDistinctMask(false, newDriverContext()); + } + + private void testRleDistinctMask(boolean hashEnabled, DriverContext driverContext) { RowPagesBuilder rowPagesBuilder = rowPagesBuilder(hashEnabled, Ints.asList(0), BIGINT); List inputs = rowPagesBuilder @@ -180,8 +175,14 @@ public void testRleDistinctMask(boolean hashEnabled) } } - @Test(dataProvider = "dataType") - public void testMemoryReservationYield(Type type) + @Test + public void testMemoryReservationYield() + { + testMemoryReservationYield(BIGINT); + testMemoryReservationYield(VARCHAR); + } + + private void testMemoryReservationYield(Type type) { List input = createPagesWithDistinctHashKeys(type, 6_000, 600); @@ -202,4 +203,11 @@ public void testMemoryReservationYield(Type type) } assertThat(count).isEqualTo(6_000 * 600); } + + private DriverContext newDriverContext() + { + return createTaskContext(executor, scheduledExecutor, TEST_SESSION) + .addPipelineContext(0, true, true, false) + .addDriverContext(); + } } diff --git a/core/trino-main/src/test/java/io/trino/operator/TestOrderByOperator.java b/core/trino-main/src/test/java/io/trino/operator/TestOrderByOperator.java index ed23a1d2860dd..934bc644238d8 100644 --- a/core/trino-main/src/test/java/io/trino/operator/TestOrderByOperator.java +++ b/core/trino-main/src/test/java/io/trino/operator/TestOrderByOperator.java @@ -25,7 +25,6 @@ import io.trino.testing.TestingTaskContext; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import java.util.List; @@ -62,17 +61,6 @@ public class TestOrderByOperator private DummySpillerFactory spillerFactory; private final TypeOperators typeOperators = new TypeOperators(); - @DataProvider - public static Object[][] spillEnabled() - { - return new Object[][] { - {false, false, 0}, - {true, false, 8}, - {true, true, 8}, - {true, false, 0}, - {true, true, 0}}; - } - @BeforeMethod public void setUp() { @@ -89,8 +77,17 @@ public void tearDown() spillerFactory = null; } - @Test(dataProvider = "spillEnabled") - public void testMultipleOutputPages(boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimit) + @Test + public void testMultipleOutputPages() + { + testMultipleOutputPages(false, false, 0); + testMultipleOutputPages(true, false, 8); + testMultipleOutputPages(true, true, 8); + testMultipleOutputPages(true, false, 0); + testMultipleOutputPages(true, true, 0); + } + + private void testMultipleOutputPages(boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimit) { // make operator produce multiple pages during finish phase int numberOfRows = 80_000; @@ -129,8 +126,17 @@ public void testMultipleOutputPages(boolean spillEnabled, boolean revokeMemoryWh .isTrue(); } - @Test(dataProvider = "spillEnabled") - public void testSingleFieldKey(boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimit) + @Test + public void testSingleFieldKey() + { + testSingleFieldKey(false, false, 0); + testSingleFieldKey(true, false, 8); + testSingleFieldKey(true, true, 8); + testSingleFieldKey(true, false, 0); + testSingleFieldKey(true, true, 0); + } + + private void testSingleFieldKey(boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimit) { List input = rowPagesBuilder(BIGINT, DOUBLE) .row(1L, 0.1) @@ -164,8 +170,17 @@ public void testSingleFieldKey(boolean spillEnabled, boolean revokeMemoryWhenAdd assertOperatorEquals(operatorFactory, driverContext, input, expected, revokeMemoryWhenAddingPages); } - @Test(dataProvider = "spillEnabled") - public void testMultiFieldKey(boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimit) + @Test + public void testMultiFieldKey() + { + testMultiFieldKey(false, false, 0); + testMultiFieldKey(true, false, 8); + testMultiFieldKey(true, true, 8); + testMultiFieldKey(true, false, 0); + testMultiFieldKey(true, true, 0); + } + + private void testMultiFieldKey(boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimit) { List input = rowPagesBuilder(VARCHAR, BIGINT) .row("a", 1L) @@ -199,8 +214,17 @@ public void testMultiFieldKey(boolean spillEnabled, boolean revokeMemoryWhenAddi assertOperatorEquals(operatorFactory, driverContext, input, expected, revokeMemoryWhenAddingPages); } - @Test(dataProvider = "spillEnabled") - public void testReverseOrder(boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimit) + @Test + public void testReverseOrder() + { + testReverseOrder(false, false, 0); + testReverseOrder(true, false, 8); + testReverseOrder(true, true, 8); + testReverseOrder(true, false, 0); + testReverseOrder(true, true, 0); + } + + private void testReverseOrder(boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimit) { List input = rowPagesBuilder(BIGINT, DOUBLE) .row(1L, 0.1) diff --git a/core/trino-main/src/test/java/io/trino/operator/TestTopNPeerGroupLookup.java b/core/trino-main/src/test/java/io/trino/operator/TestTopNPeerGroupLookup.java index d88a5cb00c7ab..9c261c935cd51 100644 --- a/core/trino-main/src/test/java/io/trino/operator/TestTopNPeerGroupLookup.java +++ b/core/trino-main/src/test/java/io/trino/operator/TestTopNPeerGroupLookup.java @@ -13,13 +13,10 @@ */ package io.trino.operator; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import java.util.Arrays; -import java.util.List; -import static com.google.common.collect.Lists.cartesianProduct; import static org.assertj.core.api.Assertions.assertThat; public class TestTopNPeerGroupLookup @@ -41,28 +38,20 @@ public long hashCode(long rowId) private static final long UNMAPPED_GROUP_ID = Long.MIN_VALUE; private static final long DEFAULT_RETURN_VALUE = -1L; - @DataProvider - public static Object[][] parameters() + @Test + public void testCombinations() { - List expectedSizes = Arrays.asList(0, 1, 2, 3, 1_000); - List fillFactors = Arrays.asList(0.1f, 0.9f, 1f); - List totalGroupIds = Arrays.asList(1L, 10L); - List totalRowIds = Arrays.asList(1L, 1_000L); - - return to2DArray(cartesianProduct(expectedSizes, fillFactors, totalGroupIds, totalRowIds)); - } - - private static Object[][] to2DArray(List> nestedList) - { - Object[][] array = new Object[nestedList.size()][]; - for (int i = 0; i < nestedList.size(); i++) { - array[i] = nestedList.get(i).toArray(); + for (int expectedSize : Arrays.asList(0, 1, 2, 3, 1_000)) { + for (float fillFactor : Arrays.asList(0.1f, 0.9f, 1f)) { + testCombinations(expectedSize, fillFactor, 1L, 1L); + testCombinations(expectedSize, fillFactor, 10L, 1L); + testCombinations(expectedSize, fillFactor, 1L, 1_000L); + testCombinations(expectedSize, fillFactor, 10L, 1_000L); + } } - return array; } - @Test(dataProvider = "parameters") - public void testCombinations(int expectedSize, float fillFactor, long totalGroupIds, long totalRowIds) + private void testCombinations(int expectedSize, float fillFactor, long totalGroupIds, long totalRowIds) { TopNPeerGroupLookup lookup = new TopNPeerGroupLookup(expectedSize, fillFactor, HASH_STRATEGY, UNMAPPED_GROUP_ID, DEFAULT_RETURN_VALUE); diff --git a/core/trino-main/src/test/java/io/trino/operator/TestWindowOperator.java b/core/trino-main/src/test/java/io/trino/operator/TestWindowOperator.java index f9c3e5f6f4aaa..d1d39d691e376 100644 --- a/core/trino-main/src/test/java/io/trino/operator/TestWindowOperator.java +++ b/core/trino-main/src/test/java/io/trino/operator/TestWindowOperator.java @@ -40,7 +40,6 @@ import io.trino.testing.TestingTaskContext; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import java.util.List; @@ -120,19 +119,17 @@ public void tearDown() spillerFactory = null; } - @DataProvider - public static Object[][] spillEnabled() + @Test + public void testMultipleOutputPages() { - return new Object[][] { - {false, false, 0}, - {true, false, 8}, - {true, true, 8}, - {true, false, 0}, - {true, true, 0}}; + testMultipleOutputPages(false, false, 0); + testMultipleOutputPages(true, false, 8); + testMultipleOutputPages(true, true, 8); + testMultipleOutputPages(true, false, 0); + testMultipleOutputPages(true, true, 0); } - @Test(dataProvider = "spillEnabled") - public void testMultipleOutputPages(boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimit) + private void testMultipleOutputPages(boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimit) { // make operator produce multiple pages during finish phase int numberOfRows = 80_000; @@ -167,8 +164,17 @@ public void testMultipleOutputPages(boolean spillEnabled, boolean revokeMemoryWh .isTrue(); } - @Test(dataProvider = "spillEnabled") - public void testRowNumber(boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimit) + @Test + public void testRowNumber() + { + testRowNumber(false, false, 0); + testRowNumber(true, false, 8); + testRowNumber(true, true, 8); + testRowNumber(true, false, 0); + testRowNumber(true, true, 0); + } + + private void testRowNumber(boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimit) { List input = rowPagesBuilder(BIGINT, DOUBLE) .row(2L, 0.3) @@ -200,8 +206,17 @@ public void testRowNumber(boolean spillEnabled, boolean revokeMemoryWhenAddingPa assertOperatorEquals(operatorFactory, driverContext, input, expected, revokeMemoryWhenAddingPages); } - @Test(dataProvider = "spillEnabled") - public void testRowNumberPartition(boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimit) + @Test + public void testRowNumberPartition() + { + testRowNumberPartition(false, false, 0); + testRowNumberPartition(true, false, 8); + testRowNumberPartition(true, true, 8); + testRowNumberPartition(true, false, 0); + testRowNumberPartition(true, true, 0); + } + + private void testRowNumberPartition(boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimit) { List input = rowPagesBuilder(VARCHAR, BIGINT, DOUBLE, BOOLEAN) .row("b", -1L, -0.1, true) @@ -311,7 +326,16 @@ public void testRowNumberArbitraryWithSpill() assertOperatorEquals(operatorFactory, driverContext, input, expected); } - @Test(dataProvider = "spillEnabled") + @Test + public void testDistinctPartitionAndPeers() + { + testDistinctPartitionAndPeers(false, false, 0); + testDistinctPartitionAndPeers(true, false, 8); + testDistinctPartitionAndPeers(true, true, 8); + testDistinctPartitionAndPeers(true, false, 0); + testDistinctPartitionAndPeers(true, true, 0); + } + public void testDistinctPartitionAndPeers(boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimit) { List input = rowPagesBuilder(DOUBLE, DOUBLE) @@ -399,8 +423,17 @@ public void testMemoryLimit() toPages(operatorFactory, driverContext, input); } - @Test(dataProvider = "spillEnabled") - public void testFirstValuePartition(boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimit) + @Test + public void testFirstValuePartition() + { + testFirstValuePartition(false, false, 0); + testFirstValuePartition(true, false, 8); + testFirstValuePartition(true, true, 8); + testFirstValuePartition(true, false, 0); + testFirstValuePartition(true, true, 0); + } + + private void testFirstValuePartition(boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimit) { List input = rowPagesBuilder(VARCHAR, VARCHAR, BIGINT, BOOLEAN, VARCHAR) .row("b", "A1", 1L, true, "") @@ -469,8 +502,17 @@ public void testClose() operator.close(); } - @Test(dataProvider = "spillEnabled") - public void testLastValuePartition(boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimit) + @Test + public void testLastValuePartition() + { + testLastValuePartition(false, false, 0); + testLastValuePartition(true, false, 8); + testLastValuePartition(true, true, 8); + testLastValuePartition(true, false, 0); + testLastValuePartition(true, true, 0); + } + + private void testLastValuePartition(boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimit) { List input = rowPagesBuilder(VARCHAR, VARCHAR, BIGINT, BOOLEAN, VARCHAR) .row("b", "A1", 1L, true, "") @@ -503,8 +545,17 @@ public void testLastValuePartition(boolean spillEnabled, boolean revokeMemoryWhe assertOperatorEquals(operatorFactory, driverContext, input, expected, revokeMemoryWhenAddingPages); } - @Test(dataProvider = "spillEnabled") - public void testNthValuePartition(boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimit) + @Test + public void testNthValuePartition() + { + testNthValuePartition(false, false, 0); + testNthValuePartition(true, false, 8); + testNthValuePartition(true, true, 8); + testNthValuePartition(true, false, 0); + testNthValuePartition(true, true, 0); + } + + private void testNthValuePartition(boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimit) { List input = rowPagesBuilder(VARCHAR, VARCHAR, BIGINT, BIGINT, BOOLEAN, VARCHAR) .row("b", "A1", 1L, 2L, true, "") @@ -538,8 +589,17 @@ public void testNthValuePartition(boolean spillEnabled, boolean revokeMemoryWhen assertOperatorEquals(operatorFactory, driverContext, input, expected, revokeMemoryWhenAddingPages); } - @Test(dataProvider = "spillEnabled") - public void testLagPartition(boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimit) + @Test + public void testLagPartition() + { + testLagPartition(false, false, 0); + testLagPartition(true, false, 8); + testLagPartition(true, true, 8); + testLagPartition(true, false, 0); + testLagPartition(true, true, 0); + } + + private void testLagPartition(boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimit) { List input = rowPagesBuilder(VARCHAR, VARCHAR, BIGINT, BIGINT, VARCHAR, BOOLEAN, VARCHAR) .row("b", "A1", 1L, 1L, "D", true, "") @@ -573,8 +633,17 @@ public void testLagPartition(boolean spillEnabled, boolean revokeMemoryWhenAddin assertOperatorEquals(operatorFactory, driverContext, input, expected, revokeMemoryWhenAddingPages); } - @Test(dataProvider = "spillEnabled") - public void testLeadPartition(boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimit) + @Test + public void testLeadPartition() + { + testLeadPartition(false, false, 0); + testLeadPartition(true, false, 8); + testLeadPartition(true, true, 8); + testLeadPartition(true, false, 0); + testLeadPartition(true, true, 0); + } + + private void testLeadPartition(boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimit) { List input = rowPagesBuilder(VARCHAR, VARCHAR, BIGINT, BIGINT, VARCHAR, BOOLEAN, VARCHAR) .row("b", "A1", 1L, 1L, "D", true, "") @@ -608,8 +677,17 @@ public void testLeadPartition(boolean spillEnabled, boolean revokeMemoryWhenAddi assertOperatorEquals(operatorFactory, driverContext, input, expected, revokeMemoryWhenAddingPages); } - @Test(dataProvider = "spillEnabled") - public void testPartiallyPreGroupedPartitionWithEmptyInput(boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimit) + @Test + public void testPartiallyPreGroupedPartitionWithEmptyInput() + { + testPartiallyPreGroupedPartitionWithEmptyInput(false, false, 0); + testPartiallyPreGroupedPartitionWithEmptyInput(true, false, 8); + testPartiallyPreGroupedPartitionWithEmptyInput(true, true, 8); + testPartiallyPreGroupedPartitionWithEmptyInput(true, false, 0); + testPartiallyPreGroupedPartitionWithEmptyInput(true, true, 0); + } + + private void testPartiallyPreGroupedPartitionWithEmptyInput(boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimit) { List input = rowPagesBuilder(BIGINT, VARCHAR, BIGINT, VARCHAR) .pageBreak() @@ -634,8 +712,17 @@ public void testPartiallyPreGroupedPartitionWithEmptyInput(boolean spillEnabled, assertOperatorEquals(operatorFactory, driverContext, input, expected, revokeMemoryWhenAddingPages); } - @Test(dataProvider = "spillEnabled") - public void testPartiallyPreGroupedPartition(boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimit) + @Test + public void testPartiallyPreGroupedPartition() + { + testPartiallyPreGroupedPartition(false, false, 0); + testPartiallyPreGroupedPartition(true, false, 8); + testPartiallyPreGroupedPartition(true, true, 8); + testPartiallyPreGroupedPartition(true, false, 0); + testPartiallyPreGroupedPartition(true, true, 0); + } + + private void testPartiallyPreGroupedPartition(boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimit) { List input = rowPagesBuilder(BIGINT, VARCHAR, BIGINT, VARCHAR) .pageBreak() @@ -674,8 +761,17 @@ public void testPartiallyPreGroupedPartition(boolean spillEnabled, boolean revok assertOperatorEqualsIgnoreOrder(operatorFactory, driverContext, input, expected, revokeMemoryWhenAddingPages); } - @Test(dataProvider = "spillEnabled") - public void testFullyPreGroupedPartition(boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimit) + @Test + public void testFullyPreGroupedPartition() + { + testFullyPreGroupedPartition(false, false, 0); + testFullyPreGroupedPartition(true, false, 8); + testFullyPreGroupedPartition(true, true, 8); + testFullyPreGroupedPartition(true, false, 0); + testFullyPreGroupedPartition(true, true, 0); + } + + private void testFullyPreGroupedPartition(boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimit) { List input = rowPagesBuilder(BIGINT, VARCHAR, BIGINT, VARCHAR) .pageBreak() @@ -716,8 +812,17 @@ public void testFullyPreGroupedPartition(boolean spillEnabled, boolean revokeMem assertOperatorEqualsIgnoreOrder(operatorFactory, driverContext, input, expected, revokeMemoryWhenAddingPages); } - @Test(dataProvider = "spillEnabled") - public void testFullyPreGroupedAndPartiallySortedPartition(boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimit) + @Test + public void testFullyPreGroupedAndPartiallySortedPartition() + { + testFullyPreGroupedAndPartiallySortedPartition(false, false, 0); + testFullyPreGroupedAndPartiallySortedPartition(true, false, 8); + testFullyPreGroupedAndPartiallySortedPartition(true, true, 8); + testFullyPreGroupedAndPartiallySortedPartition(true, false, 0); + testFullyPreGroupedAndPartiallySortedPartition(true, true, 0); + } + + private void testFullyPreGroupedAndPartiallySortedPartition(boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimit) { List input = rowPagesBuilder(BIGINT, VARCHAR, BIGINT, VARCHAR) .pageBreak() @@ -760,8 +865,17 @@ public void testFullyPreGroupedAndPartiallySortedPartition(boolean spillEnabled, assertOperatorEqualsIgnoreOrder(operatorFactory, driverContext, input, expected, revokeMemoryWhenAddingPages); } - @Test(dataProvider = "spillEnabled") - public void testFullyPreGroupedAndFullySortedPartition(boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimit) + @Test + public void testFullyPreGroupedAndFullySortedPartition() + { + testFullyPreGroupedAndFullySortedPartition(false, false, 0); + testFullyPreGroupedAndFullySortedPartition(true, false, 8); + testFullyPreGroupedAndFullySortedPartition(true, true, 8); + testFullyPreGroupedAndFullySortedPartition(true, false, 0); + testFullyPreGroupedAndFullySortedPartition(true, true, 0); + } + + private void testFullyPreGroupedAndFullySortedPartition(boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimit) { List input = rowPagesBuilder(BIGINT, VARCHAR, BIGINT, VARCHAR) .pageBreak() diff --git a/core/trino-main/src/test/java/io/trino/operator/aggregation/AbstractTestApproximateCountDistinct.java b/core/trino-main/src/test/java/io/trino/operator/aggregation/AbstractTestApproximateCountDistinct.java index a697a552f7e5d..d686b2b423fd4 100644 --- a/core/trino-main/src/test/java/io/trino/operator/aggregation/AbstractTestApproximateCountDistinct.java +++ b/core/trino-main/src/test/java/io/trino/operator/aggregation/AbstractTestApproximateCountDistinct.java @@ -21,7 +21,6 @@ import io.trino.spi.block.BlockBuilder; import io.trino.spi.type.Type; import org.apache.commons.math3.stat.descriptive.DescriptiveStatistics; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import java.util.ArrayList; @@ -52,35 +51,35 @@ protected int getUniqueValuesCount() return 20000; } - @DataProvider(name = "provideStandardErrors") - public Object[][] provideStandardErrors() + @Test + public void testNoPositions() { - return new Object[][] { - {0.0230}, // 2k buckets - {0.0115}, // 8k buckets - }; + assertCount(ImmutableList.of(), 0.0230, 0); + assertCount(ImmutableList.of(), 0.0115, 0); } - @Test(dataProvider = "provideStandardErrors") - public void testNoPositions(double maxStandardError) + @Test + public void testSinglePosition() { - assertCount(ImmutableList.of(), maxStandardError, 0); + assertCount(ImmutableList.of(randomValue()), 0.0230, 1); + assertCount(ImmutableList.of(randomValue()), 0.0115, 1); } - @Test(dataProvider = "provideStandardErrors") - public void testSinglePosition(double maxStandardError) + @Test + public void testAllPositionsNull() { - assertCount(ImmutableList.of(randomValue()), maxStandardError, 1); + assertCount(Collections.nCopies(100, null), 0.0230, 0); + assertCount(Collections.nCopies(100, null), 0.0115, 0); } - @Test(dataProvider = "provideStandardErrors") - public void testAllPositionsNull(double maxStandardError) + @Test + public void testMixedNullsAndNonNulls() { - assertCount(Collections.nCopies(100, null), maxStandardError, 0); + testMixedNullsAndNonNulls(0.0230); + testMixedNullsAndNonNulls(0.0115); } - @Test(dataProvider = "provideStandardErrors") - public void testMixedNullsAndNonNulls(double maxStandardError) + private void testMixedNullsAndNonNulls(double maxStandardError) { int uniques = getUniqueValuesCount(); List baseline = createRandomSample(uniques, (int) (uniques * 1.5)); @@ -96,8 +95,14 @@ public void testMixedNullsAndNonNulls(double maxStandardError) assertCount(mixed, maxStandardError, estimateGroupByCount(baseline, maxStandardError)); } - @Test(dataProvider = "provideStandardErrors") - public void testMultiplePositions(double maxStandardError) + @Test + public void testMultiplePositions() + { + testMultiplePositions(0.0230); + testMultiplePositions(0.0115); + } + + private void testMultiplePositions(double maxStandardError) { DescriptiveStatistics stats = new DescriptiveStatistics(); @@ -116,8 +121,14 @@ public void testMultiplePositions(double maxStandardError) assertLessThan(stats.getStandardDeviation(), 1.0e-2 + maxStandardError); } - @Test(dataProvider = "provideStandardErrors") - public void testMultiplePositionsPartial(double maxStandardError) + @Test + public void testMultiplePositionsPartial() + { + testMultiplePositionsPartial(0.0230); + testMultiplePositionsPartial(0.0115); + } + + private void testMultiplePositionsPartial(double maxStandardError) { for (int i = 0; i < 100; ++i) { int uniques = ThreadLocalRandom.current().nextInt(getUniqueValuesCount()) + 1; diff --git a/core/trino-main/src/test/java/io/trino/operator/aggregation/TestAggregationMaskCompiler.java b/core/trino-main/src/test/java/io/trino/operator/aggregation/TestAggregationMaskCompiler.java index 322028b2075f2..ad982f171207c 100644 --- a/core/trino-main/src/test/java/io/trino/operator/aggregation/TestAggregationMaskCompiler.java +++ b/core/trino-main/src/test/java/io/trino/operator/aggregation/TestAggregationMaskCompiler.java @@ -19,7 +19,6 @@ import io.trino.spi.block.IntArrayBlock; import io.trino.spi.block.RunLengthEncodedBlock; import io.trino.spi.block.ShortArrayBlock; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import java.util.Arrays; @@ -32,23 +31,24 @@ public class TestAggregationMaskCompiler { - @DataProvider - public Object[][] maskBuilderSuppliers() + private static final Supplier INTERPRETED_MASK_BUILDER_SUPPLIER = () -> new InterpretedAggregationMaskBuilder(1); + private static final Supplier COMPILED_MASK_BUILDER_SUPPLIER = () -> { + try { + return generateAggregationMaskBuilder(1).newInstance(); + } + catch (ReflectiveOperationException e) { + throw new RuntimeException(e); + } + }; + + @Test + public void testSupplier() { - Supplier interpretedMaskBuilderSupplier = () -> new InterpretedAggregationMaskBuilder(1); - Supplier compiledMaskBuilderSupplier = () -> { - try { - return generateAggregationMaskBuilder(1).newInstance(); - } - catch (ReflectiveOperationException e) { - throw new RuntimeException(e); - } - }; - return new Object[][] {{compiledMaskBuilderSupplier}, {interpretedMaskBuilderSupplier}}; + testSupplier(INTERPRETED_MASK_BUILDER_SUPPLIER); + testSupplier(COMPILED_MASK_BUILDER_SUPPLIER); } - @Test(dataProvider = "maskBuilderSuppliers") - public void testSupplier(Supplier maskBuilderSupplier) + private void testSupplier(Supplier maskBuilderSupplier) { // each builder produced from a supplier could be completely independent assertThat(maskBuilderSupplier.get()).isNotSameAs(maskBuilderSupplier.get()); @@ -74,8 +74,14 @@ public void testSupplier(Supplier maskBuilderSupplier) .isSameAs(maskBuilder.buildAggregationMask(pageWithNulls, Optional.empty()).getSelectedPositions()); } - @Test(dataProvider = "maskBuilderSuppliers") - public void testUnsetNulls(Supplier maskBuilderSupplier) + @Test + public void testUnsetNulls() + { + testUnsetNulls(INTERPRETED_MASK_BUILDER_SUPPLIER); + testUnsetNulls(COMPILED_MASK_BUILDER_SUPPLIER); + } + + private void testUnsetNulls(Supplier maskBuilderSupplier) { AggregationMaskBuilder maskBuilder = maskBuilderSupplier.get(); AggregationMask aggregationMask = maskBuilder.buildAggregationMask(buildSingleColumnPage(0), Optional.empty()); @@ -107,8 +113,14 @@ public void testUnsetNulls(Supplier maskBuilderSupplier) } } - @Test(dataProvider = "maskBuilderSuppliers") - public void testApplyMask(Supplier maskBuilderSupplier) + @Test + public void testApplyMask() + { + testApplyMask(INTERPRETED_MASK_BUILDER_SUPPLIER); + testApplyMask(COMPILED_MASK_BUILDER_SUPPLIER); + } + + private void testApplyMask(Supplier maskBuilderSupplier) { AggregationMaskBuilder maskBuilder = maskBuilderSupplier.get(); @@ -135,8 +147,14 @@ public void testApplyMask(Supplier maskBuilderSupplier) } } - @Test(dataProvider = "maskBuilderSuppliers") - public void testApplyMaskNulls(Supplier maskBuilderSupplier) + @Test + public void testApplyMaskNulls() + { + testApplyMaskNulls(INTERPRETED_MASK_BUILDER_SUPPLIER); + testApplyMaskNulls(COMPILED_MASK_BUILDER_SUPPLIER); + } + + private void testApplyMaskNulls(Supplier maskBuilderSupplier) { AggregationMaskBuilder maskBuilder = maskBuilderSupplier.get(); diff --git a/core/trino-main/src/test/java/io/trino/operator/aggregation/TestApproximateCountDistinctBoolean.java b/core/trino-main/src/test/java/io/trino/operator/aggregation/TestApproximateCountDistinctBoolean.java index a766c9c019e56..85d57c48ced7c 100644 --- a/core/trino-main/src/test/java/io/trino/operator/aggregation/TestApproximateCountDistinctBoolean.java +++ b/core/trino-main/src/test/java/io/trino/operator/aggregation/TestApproximateCountDistinctBoolean.java @@ -14,13 +14,10 @@ package io.trino.operator.aggregation; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableSet; import com.google.common.primitives.Booleans; import io.trino.spi.type.Type; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; -import java.util.List; import java.util.concurrent.ThreadLocalRandom; import static io.trino.spi.type.BooleanType.BOOLEAN; @@ -40,24 +37,15 @@ protected Object randomValue() return ThreadLocalRandom.current().nextBoolean(); } - @DataProvider(name = "inputSequences") - public Object[][] inputSequences() - { - return new Object[][] { - {true}, - {false}, - {true, false}, - {true, true, true}, - {false, false, false}, - {true, false, true, false}, - }; - } - - @Test(dataProvider = "inputSequences") - public void testNonEmptyInputs(boolean... inputSequence) + @Test + public void testNonEmptyInputs() { - List values = Booleans.asList(inputSequence); - assertCount(values, 0, distinctCount(values)); + assertCount(Booleans.asList(true), 0, 1); + assertCount(Booleans.asList(false), 0, 1); + assertCount(Booleans.asList(true, false), 0, 2); + assertCount(Booleans.asList(true, true, true), 0, 1); + assertCount(Booleans.asList(false, false, false), 0, 1); + assertCount(Booleans.asList(true, false, true, false), 0, 2); } @Test @@ -66,11 +54,6 @@ public void testNoInput() assertCount(ImmutableList.of(), 0, 0); } - private long distinctCount(List inputSequence) - { - return ImmutableSet.copyOf(inputSequence).size(); - } - @Override protected int getUniqueValuesCount() { diff --git a/core/trino-main/src/test/java/io/trino/operator/aggregation/TestApproximateSetGenericBoolean.java b/core/trino-main/src/test/java/io/trino/operator/aggregation/TestApproximateSetGenericBoolean.java index 624ba42722b5f..ee12864d5ce15 100644 --- a/core/trino-main/src/test/java/io/trino/operator/aggregation/TestApproximateSetGenericBoolean.java +++ b/core/trino-main/src/test/java/io/trino/operator/aggregation/TestApproximateSetGenericBoolean.java @@ -14,10 +14,8 @@ package io.trino.operator.aggregation; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableSet; import com.google.common.primitives.Booleans; import io.trino.spi.type.Type; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import java.util.List; @@ -40,29 +38,15 @@ protected Object randomValue() return ThreadLocalRandom.current().nextBoolean(); } - @DataProvider(name = "inputSequences") - public Object[][] inputSequences() + @Test + public void testNonEmptyInputs() { - return new Object[][] { - {true}, - {false}, - {true, false}, - {true, true, true}, - {false, false, false}, - {true, false, true, false}, - }; - } - - @Test(dataProvider = "inputSequences") - public void testNonEmptyInputs(boolean... inputSequence) - { - List values = Booleans.asList(inputSequence); - assertCount(values, distinctCount(values)); - } - - private long distinctCount(List inputSequence) - { - return ImmutableSet.copyOf(inputSequence).size(); + assertCount(Booleans.asList(true), 1); + assertCount(Booleans.asList(false), 1); + assertCount(Booleans.asList(true, false), 2); + assertCount(Booleans.asList(true, true, true), 1); + assertCount(Booleans.asList(false, false, false), 1); + assertCount(Booleans.asList(true, false, true, false), 2); } @Override diff --git a/core/trino-main/src/test/java/io/trino/operator/aggregation/TestDecimalAverageAggregation.java b/core/trino-main/src/test/java/io/trino/operator/aggregation/TestDecimalAverageAggregation.java index a81785db935c0..0a5c9ee2040a5 100644 --- a/core/trino-main/src/test/java/io/trino/operator/aggregation/TestDecimalAverageAggregation.java +++ b/core/trino-main/src/test/java/io/trino/operator/aggregation/TestDecimalAverageAggregation.java @@ -22,7 +22,6 @@ import io.trino.spi.type.Decimals; import io.trino.spi.type.Int128; import org.testng.annotations.BeforeMethod; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import java.math.BigDecimal; @@ -139,6 +138,8 @@ public void testCombineOverflow() @Test public void testCombineUnderflow() { + LongDecimalWithOverflowAndLongState state = new LongDecimalWithOverflowAndLongStateFactory().createSingleState(); + addToState(state, TWO.pow(125).negate()); addToState(state, TWO.pow(126).negate()); @@ -163,11 +164,36 @@ public void testCombineUnderflow() assertAverageEquals(expectedAverage); } - @Test(dataProvider = "testNoOverflowDataProvider") - public void testNoOverflow(List numbers) + @Test + public void testNoOverflow() { - testNoOverflow(createDecimalType(38, 0), numbers); - testNoOverflow(createDecimalType(38, 2), numbers); + testNoOverflow(createDecimalType(38, 0), ImmutableList.of(TEN.pow(37), ZERO)); + testNoOverflow(createDecimalType(38, 0), ImmutableList.of(TEN.pow(37).negate(), ZERO)); + testNoOverflow(createDecimalType(38, 0), ImmutableList.of(TWO, ONE)); + testNoOverflow(createDecimalType(38, 0), ImmutableList.of(ZERO, ONE)); + testNoOverflow(createDecimalType(38, 0), ImmutableList.of(TWO.negate(), ONE.negate())); + testNoOverflow(createDecimalType(38, 0), ImmutableList.of(ONE.negate(), ZERO)); + testNoOverflow(createDecimalType(38, 0), ImmutableList.of(ONE.negate(), ZERO, ZERO)); + testNoOverflow(createDecimalType(38, 0), ImmutableList.of(TWO.negate(), ZERO, ZERO)); + testNoOverflow(createDecimalType(38, 0), ImmutableList.of(TWO.negate(), ZERO)); + testNoOverflow(createDecimalType(38, 0), ImmutableList.of(TWO_HUNDRED, ONE_HUNDRED)); + testNoOverflow(createDecimalType(38, 0), ImmutableList.of(ZERO, ONE_HUNDRED)); + testNoOverflow(createDecimalType(38, 0), ImmutableList.of(TWO_HUNDRED.negate(), ONE_HUNDRED.negate())); + testNoOverflow(createDecimalType(38, 0), ImmutableList.of(ONE_HUNDRED.negate(), ZERO)); + + testNoOverflow(createDecimalType(38, 2), ImmutableList.of(TEN.pow(37), ZERO)); + testNoOverflow(createDecimalType(38, 2), ImmutableList.of(TEN.pow(37).negate(), ZERO)); + testNoOverflow(createDecimalType(38, 2), ImmutableList.of(TWO, ONE)); + testNoOverflow(createDecimalType(38, 2), ImmutableList.of(ZERO, ONE)); + testNoOverflow(createDecimalType(38, 2), ImmutableList.of(TWO.negate(), ONE.negate())); + testNoOverflow(createDecimalType(38, 2), ImmutableList.of(ONE.negate(), ZERO)); + testNoOverflow(createDecimalType(38, 2), ImmutableList.of(ONE.negate(), ZERO, ZERO)); + testNoOverflow(createDecimalType(38, 2), ImmutableList.of(TWO.negate(), ZERO, ZERO)); + testNoOverflow(createDecimalType(38, 2), ImmutableList.of(TWO.negate(), ZERO)); + testNoOverflow(createDecimalType(38, 2), ImmutableList.of(TWO_HUNDRED, ONE_HUNDRED)); + testNoOverflow(createDecimalType(38, 2), ImmutableList.of(ZERO, ONE_HUNDRED)); + testNoOverflow(createDecimalType(38, 2), ImmutableList.of(TWO_HUNDRED.negate(), ONE_HUNDRED.negate())); + testNoOverflow(createDecimalType(38, 2), ImmutableList.of(ONE_HUNDRED.negate(), ZERO)); } private void testNoOverflow(DecimalType type, List numbers) @@ -185,26 +211,6 @@ private void testNoOverflow(DecimalType type, List numbers) assertThat(decodeBigDecimal(type, average(state, type))).isEqualTo(expectedAverage); } - @DataProvider - public static Object[][] testNoOverflowDataProvider() - { - return new Object[][] { - {ImmutableList.of(TEN.pow(37), ZERO)}, - {ImmutableList.of(TEN.pow(37).negate(), ZERO)}, - {ImmutableList.of(TWO, ONE)}, - {ImmutableList.of(ZERO, ONE)}, - {ImmutableList.of(TWO.negate(), ONE.negate())}, - {ImmutableList.of(ONE.negate(), ZERO)}, - {ImmutableList.of(ONE.negate(), ZERO, ZERO)}, - {ImmutableList.of(TWO.negate(), ZERO, ZERO)}, - {ImmutableList.of(TWO.negate(), ZERO)}, - {ImmutableList.of(TWO_HUNDRED, ONE_HUNDRED)}, - {ImmutableList.of(ZERO, ONE_HUNDRED)}, - {ImmutableList.of(TWO_HUNDRED.negate(), ONE_HUNDRED.negate())}, - {ImmutableList.of(ONE_HUNDRED.negate(), ZERO)} - }; - } - private static BigDecimal decodeBigDecimal(DecimalType type, Int128 average) { BigInteger unscaledVal = average.toBigInteger(); diff --git a/core/trino-main/src/test/java/io/trino/operator/aggregation/state/TestLongDecimalWithOverflowAndLongStateSerializer.java b/core/trino-main/src/test/java/io/trino/operator/aggregation/state/TestLongDecimalWithOverflowAndLongStateSerializer.java index 257d63587ba75..8f56532e775f1 100644 --- a/core/trino-main/src/test/java/io/trino/operator/aggregation/state/TestLongDecimalWithOverflowAndLongStateSerializer.java +++ b/core/trino-main/src/test/java/io/trino/operator/aggregation/state/TestLongDecimalWithOverflowAndLongStateSerializer.java @@ -16,7 +16,6 @@ import io.trino.spi.block.Block; import io.trino.spi.block.BlockBuilder; import io.trino.spi.block.VariableWidthBlockBuilder; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import static org.assertj.core.api.Assertions.assertThat; @@ -25,8 +24,28 @@ public class TestLongDecimalWithOverflowAndLongStateSerializer { private static final LongDecimalWithOverflowAndLongStateFactory STATE_FACTORY = new LongDecimalWithOverflowAndLongStateFactory(); - @Test(dataProvider = "input") - public void testSerde(long low, long high, long overflow, long count, int expectedLength) + @Test + public void testSerde() + { + testSerde(3, 0, 0, 1, 1); + testSerde(3, 5, 0, 1, 2); + testSerde(3, 5, 7, 1, 4); + testSerde(3, 0, 0, 2, 3); + testSerde(3, 5, 0, 2, 4); + testSerde(3, 5, 7, 2, 4); + testSerde(3, 0, 7, 1, 3); + testSerde(3, 0, 7, 2, 3); + testSerde(0, 0, 0, 1, 1); + testSerde(0, 5, 0, 1, 2); + testSerde(0, 5, 7, 1, 4); + testSerde(0, 0, 0, 2, 3); + testSerde(0, 5, 0, 2, 4); + testSerde(0, 5, 7, 2, 4); + testSerde(0, 0, 7, 1, 3); + testSerde(0, 0, 7, 2, 3); + } + + private void testSerde(long low, long high, long overflow, long count, int expectedLength) { LongDecimalWithOverflowAndLongState state = STATE_FACTORY.createSingleState(); state.getDecimalArray()[0] = high; @@ -66,27 +85,4 @@ private LongDecimalWithOverflowAndLongState roundTrip(LongDecimalWithOverflowAnd serializer.deserialize(serialized, 0, outState); return outState; } - - @DataProvider - public Object[][] input() - { - return new Object[][] { - {3, 0, 0, 1, 1}, - {3, 5, 0, 1, 2}, - {3, 5, 7, 1, 4}, - {3, 0, 0, 2, 3}, - {3, 5, 0, 2, 4}, - {3, 5, 7, 2, 4}, - {3, 0, 7, 1, 3}, - {3, 0, 7, 2, 3}, - {0, 0, 0, 1, 1}, - {0, 5, 0, 1, 2}, - {0, 5, 7, 1, 4}, - {0, 0, 0, 2, 3}, - {0, 5, 0, 2, 4}, - {0, 5, 7, 2, 4}, - {0, 0, 7, 1, 3}, - {0, 0, 7, 2, 3} - }; - } } diff --git a/core/trino-main/src/test/java/io/trino/operator/aggregation/state/TestLongDecimalWithOverflowStateSerializer.java b/core/trino-main/src/test/java/io/trino/operator/aggregation/state/TestLongDecimalWithOverflowStateSerializer.java index 57638bd7bfd79..6833785bc4ee5 100644 --- a/core/trino-main/src/test/java/io/trino/operator/aggregation/state/TestLongDecimalWithOverflowStateSerializer.java +++ b/core/trino-main/src/test/java/io/trino/operator/aggregation/state/TestLongDecimalWithOverflowStateSerializer.java @@ -16,7 +16,6 @@ import io.trino.spi.block.Block; import io.trino.spi.block.BlockBuilder; import io.trino.spi.block.VariableWidthBlockBuilder; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import static org.assertj.core.api.Assertions.assertThat; @@ -25,8 +24,20 @@ public class TestLongDecimalWithOverflowStateSerializer { private static final LongDecimalWithOverflowStateFactory STATE_FACTORY = new LongDecimalWithOverflowStateFactory(); - @Test(dataProvider = "input") - public void testSerde(long low, long high, long overflow, int expectedLength) + @Test + public void testSerde() + { + testSerde(3, 0, 0, 1); + testSerde(3, 5, 0, 2); + testSerde(3, 5, 7, 3); + testSerde(3, 0, 7, 3); + testSerde(0, 0, 0, 1); + testSerde(0, 5, 0, 2); + testSerde(0, 5, 7, 3); + testSerde(0, 0, 7, 3); + } + + private void testSerde(long low, long high, long overflow, int expectedLength) { LongDecimalWithOverflowState state = STATE_FACTORY.createSingleState(); state.getDecimalArray()[0] = high; @@ -66,19 +77,4 @@ private LongDecimalWithOverflowState roundTrip(LongDecimalWithOverflowState stat serializer.deserialize(serialized, 0, outState); return outState; } - - @DataProvider - public Object[][] input() - { - return new Object[][] { - {3, 0, 0, 1}, - {3, 5, 0, 2}, - {3, 5, 7, 3}, - {3, 0, 7, 3}, - {0, 0, 0, 1}, - {0, 5, 0, 2}, - {0, 5, 7, 3}, - {0, 0, 7, 3} - }; - } } diff --git a/core/trino-main/src/test/java/io/trino/operator/exchange/TestLocalExchange.java b/core/trino-main/src/test/java/io/trino/operator/exchange/TestLocalExchange.java index 6f5b3763addc3..2e28dd74e8a85 100644 --- a/core/trino-main/src/test/java/io/trino/operator/exchange/TestLocalExchange.java +++ b/core/trino-main/src/test/java/io/trino/operator/exchange/TestLocalExchange.java @@ -42,7 +42,6 @@ import io.trino.testing.TestingTransactionHandle; import io.trino.util.FinalizerService; import org.testng.annotations.BeforeMethod; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import java.util.List; @@ -88,6 +87,7 @@ public class TestLocalExchange private final ConcurrentMap partitionManagers = new ConcurrentHashMap<>(); private NodePartitioningManager nodePartitioningManager; + private final PartitioningHandle customScalingPartitioningHandle = getCustomScalingPartitioningHandle(); @BeforeMethod public void setUp() @@ -332,8 +332,14 @@ public void testNoWriterScalingWhenOnlyBufferSizeLimitIsExceeded() }); } - @Test(dataProvider = "scalingPartitionHandles") - public void testScalingWithTwoDifferentPartitions(PartitioningHandle partitioningHandle) + @Test + public void testScalingWithTwoDifferentPartitions() + { + testScalingWithTwoDifferentPartitions(customScalingPartitioningHandle); + testScalingWithTwoDifferentPartitions(SCALED_WRITER_HASH_DISTRIBUTION); + } + + private void testScalingWithTwoDifferentPartitions(PartitioningHandle partitioningHandle) { LocalExchange localExchange = new LocalExchange( nodePartitioningManager, @@ -533,8 +539,14 @@ public void testNoWriterScalingWhenOnlyWriterScalingMinDataProcessedLimitIsExcee }); } - @Test(dataProvider = "scalingPartitionHandles") - public void testScalingForSkewedWriters(PartitioningHandle partitioningHandle) + @Test + public void testScalingForSkewedWriters() + { + testScalingForSkewedWriters(customScalingPartitioningHandle); + testScalingForSkewedWriters(SCALED_WRITER_HASH_DISTRIBUTION); + } + + private void testScalingForSkewedWriters(PartitioningHandle partitioningHandle) { LocalExchange localExchange = new LocalExchange( nodePartitioningManager, @@ -623,8 +635,14 @@ public void testScalingForSkewedWriters(PartitioningHandle partitioningHandle) }); } - @Test(dataProvider = "scalingPartitionHandles") - public void testNoScalingWhenDataWrittenIsLessThanMinFileSize(PartitioningHandle partitioningHandle) + @Test + public void testNoScalingWhenDataWrittenIsLessThanMinFileSize() + { + testNoScalingWhenDataWrittenIsLessThanMinFileSize(customScalingPartitioningHandle); + testNoScalingWhenDataWrittenIsLessThanMinFileSize(SCALED_WRITER_HASH_DISTRIBUTION); + } + + private void testNoScalingWhenDataWrittenIsLessThanMinFileSize(PartitioningHandle partitioningHandle) { LocalExchange localExchange = new LocalExchange( nodePartitioningManager, @@ -687,8 +705,14 @@ public void testNoScalingWhenDataWrittenIsLessThanMinFileSize(PartitioningHandle }); } - @Test(dataProvider = "scalingPartitionHandles") - public void testNoScalingWhenBufferUtilizationIsLessThanLimit(PartitioningHandle partitioningHandle) + @Test + public void testNoScalingWhenBufferUtilizationIsLessThanLimit() + { + testNoScalingWhenBufferUtilizationIsLessThanLimit(customScalingPartitioningHandle); + testNoScalingWhenBufferUtilizationIsLessThanLimit(SCALED_WRITER_HASH_DISTRIBUTION); + } + + private void testNoScalingWhenBufferUtilizationIsLessThanLimit(PartitioningHandle partitioningHandle) { LocalExchange localExchange = new LocalExchange( nodePartitioningManager, @@ -751,8 +775,14 @@ public void testNoScalingWhenBufferUtilizationIsLessThanLimit(PartitioningHandle }); } - @Test(dataProvider = "scalingPartitionHandles") - public void testNoScalingWhenTotalMemoryUsedIsGreaterThanLimit(PartitioningHandle partitioningHandle) + @Test + public void testNoScalingWhenTotalMemoryUsedIsGreaterThanLimit() + { + testNoScalingWhenTotalMemoryUsedIsGreaterThanLimit(customScalingPartitioningHandle); + testNoScalingWhenTotalMemoryUsedIsGreaterThanLimit(SCALED_WRITER_HASH_DISTRIBUTION); + } + + private void testNoScalingWhenTotalMemoryUsedIsGreaterThanLimit(PartitioningHandle partitioningHandle) { AtomicLong totalMemoryUsed = new AtomicLong(); LocalExchange localExchange = new LocalExchange( @@ -832,8 +862,14 @@ public void testNoScalingWhenTotalMemoryUsedIsGreaterThanLimit(PartitioningHandl }); } - @Test(dataProvider = "scalingPartitionHandles") - public void testDoNotUpdateScalingStateWhenMemoryIsAboveLimit(PartitioningHandle partitioningHandle) + @Test + public void testDoNotUpdateScalingStateWhenMemoryIsAboveLimit() + { + testDoNotUpdateScalingStateWhenMemoryIsAboveLimit(customScalingPartitioningHandle); + testDoNotUpdateScalingStateWhenMemoryIsAboveLimit(SCALED_WRITER_HASH_DISTRIBUTION); + } + + private void testDoNotUpdateScalingStateWhenMemoryIsAboveLimit(PartitioningHandle partitioningHandle) { AtomicLong totalMemoryUsed = new AtomicLong(); LocalExchange localExchange = new LocalExchange( @@ -1316,12 +1352,6 @@ public void writeUnblockWhenAllReadersFinishAndPagesConsumed() }); } - @DataProvider - public Object[][] scalingPartitionHandles() - { - return new Object[][] {{SCALED_WRITER_HASH_DISTRIBUTION}, {getCustomScalingPartitioningHandle()}}; - } - private PartitioningHandle getCustomScalingPartitioningHandle() { ConnectorPartitioningHandle connectorPartitioningHandle = new ConnectorPartitioningHandle() {}; diff --git a/core/trino-main/src/test/java/io/trino/operator/join/JoinTestUtils.java b/core/trino-main/src/test/java/io/trino/operator/join/JoinTestUtils.java index 3f119282e95f8..514c2dcc5c6c9 100644 --- a/core/trino-main/src/test/java/io/trino/operator/join/JoinTestUtils.java +++ b/core/trino-main/src/test/java/io/trino/operator/join/JoinTestUtils.java @@ -318,14 +318,16 @@ public static class DummySpillerFactory private volatile boolean failSpill; private volatile boolean failUnspill; - public void failSpill() + public DummySpillerFactory failSpill() { failSpill = true; + return this; } - public void failUnspill() + public DummySpillerFactory failUnspill() { failUnspill = true; + return this; } @Override diff --git a/core/trino-main/src/test/java/io/trino/operator/join/TestHashJoinOperator.java b/core/trino-main/src/test/java/io/trino/operator/join/TestHashJoinOperator.java index 091b7f3b49735..7d4f0f8ea181a 100644 --- a/core/trino-main/src/test/java/io/trino/operator/join/TestHashJoinOperator.java +++ b/core/trino-main/src/test/java/io/trino/operator/join/TestHashJoinOperator.java @@ -64,11 +64,9 @@ import io.trino.util.FinalizerService; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; import java.util.Optional; import java.util.OptionalInt; @@ -106,7 +104,6 @@ import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.spi.type.IntegerType.INTEGER; import static io.trino.spi.type.VarcharType.VARCHAR; -import static java.lang.String.format; import static java.util.Arrays.asList; import static java.util.Collections.nCopies; import static java.util.Collections.singletonList; @@ -114,7 +111,6 @@ import static java.util.concurrent.Executors.newScheduledThreadPool; import static java.util.concurrent.TimeUnit.NANOSECONDS; import static java.util.concurrent.TimeUnit.SECONDS; -import static java.util.stream.Collectors.toList; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -166,22 +162,20 @@ public void tearDown() scheduledExecutor.shutdownNow(); } - @DataProvider(name = "hashJoinTestValues") - public static Object[][] hashJoinTestValuesProvider() + @Test + public void testInnerJoin() { - return new Object[][] { - {true, true, true}, - {true, true, false}, - {true, false, true}, - {true, false, false}, - {false, true, true}, - {false, true, false}, - {false, false, true}, - {false, false, false}}; + testInnerJoin(true, true, true); + testInnerJoin(true, true, false); + testInnerJoin(true, false, true); + testInnerJoin(true, false, false); + testInnerJoin(false, true, true); + testInnerJoin(false, true, false); + testInnerJoin(false, false, true); + testInnerJoin(false, false, false); } - @Test(dataProvider = "hashJoinTestValues") - public void testInnerJoin(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + private void testInnerJoin(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) { TaskContext taskContext = createTaskContext(); @@ -382,90 +376,76 @@ private enum WhenSpill DURING_BUILD, AFTER_BUILD, DURING_USAGE, NEVER } - private enum WhenSpillFails - { - SPILL_BUILD, SPILL_JOIN, UNSPILL_BUILD, UNSPILL_JOIN - } - - @DataProvider - public Object[][] joinWithSpillValues() - { - return joinWithSpillParameters(true).stream() - .map(List::toArray) - .toArray(Object[][]::new); - } - - @DataProvider - public Object[][] joinWithFailingSpillValues() + @Test + public void testInnerJoinWithSpill() + throws Exception { - List> spillFailValues = Arrays.stream(WhenSpillFails.values()) - .map(ImmutableList::of) - .collect(toList()); - return product(joinWithSpillParameters(false), spillFailValues).stream() - .map(List::toArray) - .toArray(Object[][]::new); + for (boolean probeHashEnabled : ImmutableList.of(false, true)) { + // spill all + innerJoinWithSpill(probeHashEnabled, nCopies(PARTITION_COUNT, WhenSpill.NEVER), SINGLE_STREAM_SPILLER_FACTORY, PARTITIONING_SPILLER_FACTORY); + innerJoinWithSpill(probeHashEnabled, nCopies(PARTITION_COUNT, WhenSpill.DURING_BUILD), SINGLE_STREAM_SPILLER_FACTORY, PARTITIONING_SPILLER_FACTORY); + innerJoinWithSpill(probeHashEnabled, nCopies(PARTITION_COUNT, WhenSpill.AFTER_BUILD), SINGLE_STREAM_SPILLER_FACTORY, PARTITIONING_SPILLER_FACTORY); + innerJoinWithSpill(probeHashEnabled, nCopies(PARTITION_COUNT, WhenSpill.DURING_USAGE), SINGLE_STREAM_SPILLER_FACTORY, PARTITIONING_SPILLER_FACTORY); + + // spill one + innerJoinWithSpill(probeHashEnabled, concat(singletonList(WhenSpill.DURING_BUILD), nCopies(PARTITION_COUNT - 1, WhenSpill.NEVER)), SINGLE_STREAM_SPILLER_FACTORY, PARTITIONING_SPILLER_FACTORY); + innerJoinWithSpill(probeHashEnabled, concat(singletonList(WhenSpill.AFTER_BUILD), nCopies(PARTITION_COUNT - 1, WhenSpill.NEVER)), SINGLE_STREAM_SPILLER_FACTORY, PARTITIONING_SPILLER_FACTORY); + innerJoinWithSpill(probeHashEnabled, concat(singletonList(WhenSpill.DURING_USAGE), nCopies(PARTITION_COUNT - 1, WhenSpill.NEVER)), SINGLE_STREAM_SPILLER_FACTORY, PARTITIONING_SPILLER_FACTORY); + + innerJoinWithSpill(probeHashEnabled, concat(asList(WhenSpill.DURING_BUILD, WhenSpill.AFTER_BUILD), nCopies(PARTITION_COUNT - 2, WhenSpill.NEVER)), SINGLE_STREAM_SPILLER_FACTORY, PARTITIONING_SPILLER_FACTORY); + innerJoinWithSpill(probeHashEnabled, concat(asList(WhenSpill.DURING_BUILD, WhenSpill.DURING_USAGE), nCopies(PARTITION_COUNT - 2, WhenSpill.NEVER)), SINGLE_STREAM_SPILLER_FACTORY, PARTITIONING_SPILLER_FACTORY); + } } - private static List> joinWithSpillParameters(boolean allowNoSpill) + @Test + public void testInnerJoinWithFailingSpill() { - List> result = new ArrayList<>(); for (boolean probeHashEnabled : ImmutableList.of(false, true)) { - for (WhenSpill whenSpill : WhenSpill.values()) { - // spill all - if (allowNoSpill || whenSpill != WhenSpill.NEVER) { - result.add(ImmutableList.of(probeHashEnabled, nCopies(PARTITION_COUNT, whenSpill))); - } - - if (whenSpill != WhenSpill.NEVER) { - // spill one - result.add(ImmutableList.of(probeHashEnabled, concat(singletonList(whenSpill), nCopies(PARTITION_COUNT - 1, WhenSpill.NEVER)))); - } - } - - result.add(ImmutableList.of(probeHashEnabled, concat(asList(WhenSpill.DURING_BUILD, WhenSpill.AFTER_BUILD), nCopies(PARTITION_COUNT - 2, WhenSpill.NEVER)))); - result.add(ImmutableList.of(probeHashEnabled, concat(asList(WhenSpill.DURING_BUILD, WhenSpill.DURING_USAGE), nCopies(PARTITION_COUNT - 2, WhenSpill.NEVER)))); + // spill all + testInnerJoinWithFailingSpill(probeHashEnabled, nCopies(PARTITION_COUNT, WhenSpill.DURING_USAGE)); + testInnerJoinWithFailingSpill(probeHashEnabled, nCopies(PARTITION_COUNT, WhenSpill.DURING_BUILD)); + testInnerJoinWithFailingSpill(probeHashEnabled, nCopies(PARTITION_COUNT, WhenSpill.AFTER_BUILD)); + + // spill one + testInnerJoinWithFailingSpill(probeHashEnabled, concat(singletonList(WhenSpill.DURING_USAGE), nCopies(PARTITION_COUNT - 1, WhenSpill.NEVER))); + testInnerJoinWithFailingSpill(probeHashEnabled, concat(singletonList(WhenSpill.DURING_BUILD), nCopies(PARTITION_COUNT - 1, WhenSpill.NEVER))); + testInnerJoinWithFailingSpill(probeHashEnabled, concat(singletonList(WhenSpill.AFTER_BUILD), nCopies(PARTITION_COUNT - 1, WhenSpill.NEVER))); + + testInnerJoinWithFailingSpill(probeHashEnabled, concat(asList(WhenSpill.DURING_BUILD, WhenSpill.AFTER_BUILD), nCopies(PARTITION_COUNT - 2, WhenSpill.NEVER))); + testInnerJoinWithFailingSpill(probeHashEnabled, concat(asList(WhenSpill.DURING_BUILD, WhenSpill.DURING_USAGE), nCopies(PARTITION_COUNT - 2, WhenSpill.NEVER))); } - return result; } - @Test(dataProvider = "joinWithSpillValues") - public void testInnerJoinWithSpill(boolean probeHashEnabled, List whenSpill) - throws Exception + private void testInnerJoinWithFailingSpill(boolean probeHashEnabled, List whenSpill) { - innerJoinWithSpill(probeHashEnabled, whenSpill, SINGLE_STREAM_SPILLER_FACTORY, PARTITIONING_SPILLER_FACTORY); - } + assertThatThrownBy(() -> innerJoinWithSpill( + probeHashEnabled, + whenSpill, + new DummySpillerFactory().failSpill(), + new GenericPartitioningSpillerFactory(new DummySpillerFactory()))) + .isInstanceOf(RuntimeException.class) + .hasMessage("Spill failed"); - @Test(dataProvider = "joinWithFailingSpillValues") - public void testInnerJoinWithFailingSpill(boolean probeHashEnabled, List whenSpill, WhenSpillFails whenSpillFails) - { - DummySpillerFactory buildSpillerFactory = new DummySpillerFactory(); - DummySpillerFactory joinSpillerFactory = new DummySpillerFactory(); - PartitioningSpillerFactory partitioningSpillerFactory = new GenericPartitioningSpillerFactory(joinSpillerFactory); - - String expectedMessage; - switch (whenSpillFails) { - case SPILL_BUILD: - buildSpillerFactory.failSpill(); - expectedMessage = "Spill failed"; - break; - case SPILL_JOIN: - joinSpillerFactory.failSpill(); - expectedMessage = "Spill failed"; - break; - case UNSPILL_BUILD: - buildSpillerFactory.failUnspill(); - expectedMessage = "Unspill failed"; - break; - case UNSPILL_JOIN: - joinSpillerFactory.failUnspill(); - expectedMessage = "Unspill failed"; - break; - default: - throw new IllegalArgumentException(format("Unsupported option: %s", whenSpillFails)); - } - assertThatThrownBy(() -> innerJoinWithSpill(probeHashEnabled, whenSpill, buildSpillerFactory, partitioningSpillerFactory)) + assertThatThrownBy(() -> innerJoinWithSpill(probeHashEnabled, + whenSpill, + new DummySpillerFactory(), + new GenericPartitioningSpillerFactory(new DummySpillerFactory().failSpill()))) + .isInstanceOf(RuntimeException.class) + .hasMessage("Spill failed"); + + assertThatThrownBy(() -> innerJoinWithSpill(probeHashEnabled, + whenSpill, + new DummySpillerFactory().failUnspill(), + new GenericPartitioningSpillerFactory(new DummySpillerFactory()))) .isInstanceOf(RuntimeException.class) - .hasMessage(expectedMessage); + .hasMessage("Unspill failed"); + + assertThatThrownBy(() -> innerJoinWithSpill(probeHashEnabled, + whenSpill, + new DummySpillerFactory(), + new GenericPartitioningSpillerFactory(new DummySpillerFactory().failUnspill()))) + .isInstanceOf(RuntimeException.class) + .hasMessage("Unspill failed"); } private void innerJoinWithSpill(boolean probeHashEnabled, List whenSpill, SingleStreamSpillerFactory buildSpillerFactory, PartitioningSpillerFactory joinSpillerFactory) @@ -679,8 +659,20 @@ public void testBuildGracefulSpill() assertThat(hashBuilderOperator.isFinished()).isTrue(); } - @Test(dataProvider = "hashJoinTestValues") - public void testInnerJoinWithNullProbe(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + @Test + public void testInnerJoinWithNullProbe() + { + testInnerJoinWithNullProbe(true, true, true); + testInnerJoinWithNullProbe(true, true, false); + testInnerJoinWithNullProbe(true, false, true); + testInnerJoinWithNullProbe(true, false, false); + testInnerJoinWithNullProbe(false, true, true); + testInnerJoinWithNullProbe(false, true, false); + testInnerJoinWithNullProbe(false, false, true); + testInnerJoinWithNullProbe(false, false, false); + } + + private void testInnerJoinWithNullProbe(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) { TaskContext taskContext = createTaskContext(); @@ -719,8 +711,20 @@ public void testInnerJoinWithNullProbe(boolean parallelBuild, boolean probeHashE assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); } - @Test(dataProvider = "hashJoinTestValues") - public void testInnerJoinWithOutputSingleMatch(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + @Test + public void testInnerJoinWithOutputSingleMatch() + { + testInnerJoinWithOutputSingleMatch(true, true, true); + testInnerJoinWithOutputSingleMatch(true, true, false); + testInnerJoinWithOutputSingleMatch(true, false, true); + testInnerJoinWithOutputSingleMatch(true, false, false); + testInnerJoinWithOutputSingleMatch(false, true, true); + testInnerJoinWithOutputSingleMatch(false, true, false); + testInnerJoinWithOutputSingleMatch(false, false, true); + testInnerJoinWithOutputSingleMatch(false, false, false); + } + + private void testInnerJoinWithOutputSingleMatch(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) { TaskContext taskContext = createTaskContext(); // build factory @@ -755,8 +759,20 @@ public void testInnerJoinWithOutputSingleMatch(boolean parallelBuild, boolean pr assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); } - @Test(dataProvider = "hashJoinTestValues") - public void testInnerJoinWithNullBuild(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + @Test + public void testInnerJoinWithNullBuild() + { + testInnerJoinWithNullBuild(true, true, true); + testInnerJoinWithNullBuild(true, true, false); + testInnerJoinWithNullBuild(true, false, true); + testInnerJoinWithNullBuild(true, false, false); + testInnerJoinWithNullBuild(false, true, true); + testInnerJoinWithNullBuild(false, true, false); + testInnerJoinWithNullBuild(false, false, true); + testInnerJoinWithNullBuild(false, false, false); + } + + private void testInnerJoinWithNullBuild(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) { TaskContext taskContext = createTaskContext(); @@ -795,8 +811,20 @@ public void testInnerJoinWithNullBuild(boolean parallelBuild, boolean probeHashE assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); } - @Test(dataProvider = "hashJoinTestValues") - public void testInnerJoinWithNullOnBothSides(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + @Test + public void testInnerJoinWithNullOnBothSides() + { + testInnerJoinWithNullOnBothSides(true, true, true); + testInnerJoinWithNullOnBothSides(true, true, false); + testInnerJoinWithNullOnBothSides(true, false, true); + testInnerJoinWithNullOnBothSides(true, false, false); + testInnerJoinWithNullOnBothSides(false, true, true); + testInnerJoinWithNullOnBothSides(false, true, false); + testInnerJoinWithNullOnBothSides(false, false, true); + testInnerJoinWithNullOnBothSides(false, false, false); + } + + private void testInnerJoinWithNullOnBothSides(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) { TaskContext taskContext = createTaskContext(); @@ -836,8 +864,20 @@ public void testInnerJoinWithNullOnBothSides(boolean parallelBuild, boolean prob assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); } - @Test(dataProvider = "hashJoinTestValues") - public void testProbeOuterJoin(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + @Test + public void testProbeOuterJoin() + { + testProbeOuterJoin(true, true, true); + testProbeOuterJoin(true, true, false); + testProbeOuterJoin(true, false, true); + testProbeOuterJoin(true, false, false); + testProbeOuterJoin(false, true, true); + testProbeOuterJoin(false, true, false); + testProbeOuterJoin(false, false, true); + testProbeOuterJoin(false, false, false); + } + + private void testProbeOuterJoin(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) { TaskContext taskContext = createTaskContext(); @@ -882,8 +922,20 @@ public void testProbeOuterJoin(boolean parallelBuild, boolean probeHashEnabled, assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); } - @Test(dataProvider = "hashJoinTestValues") - public void testProbeOuterJoinWithFilterFunction(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + @Test + public void testProbeOuterJoinWithFilterFunction() + { + testProbeOuterJoinWithFilterFunction(true, true, true); + testProbeOuterJoinWithFilterFunction(true, true, false); + testProbeOuterJoinWithFilterFunction(true, false, true); + testProbeOuterJoinWithFilterFunction(true, false, false); + testProbeOuterJoinWithFilterFunction(false, true, true); + testProbeOuterJoinWithFilterFunction(false, true, false); + testProbeOuterJoinWithFilterFunction(false, false, true); + testProbeOuterJoinWithFilterFunction(false, false, false); + } + + private void testProbeOuterJoinWithFilterFunction(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) { TaskContext taskContext = createTaskContext(); @@ -931,8 +983,20 @@ public void testProbeOuterJoinWithFilterFunction(boolean parallelBuild, boolean assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); } - @Test(dataProvider = "hashJoinTestValues") - public void testOuterJoinWithNullProbe(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + @Test + public void testOuterJoinWithNullProbe() + { + testOuterJoinWithNullProbe(true, true, true); + testOuterJoinWithNullProbe(true, true, false); + testOuterJoinWithNullProbe(true, false, true); + testOuterJoinWithNullProbe(true, false, false); + testOuterJoinWithNullProbe(false, true, true); + testOuterJoinWithNullProbe(false, true, false); + testOuterJoinWithNullProbe(false, false, true); + testOuterJoinWithNullProbe(false, false, false); + } + + private void testOuterJoinWithNullProbe(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) { TaskContext taskContext = createTaskContext(); @@ -973,8 +1037,20 @@ public void testOuterJoinWithNullProbe(boolean parallelBuild, boolean probeHashE assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); } - @Test(dataProvider = "hashJoinTestValues") - public void testOuterJoinWithNullProbeAndFilterFunction(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + @Test + public void testOuterJoinWithNullProbeAndFilterFunction() + { + testOuterJoinWithNullProbeAndFilterFunction(true, true, true); + testOuterJoinWithNullProbeAndFilterFunction(true, true, false); + testOuterJoinWithNullProbeAndFilterFunction(true, false, true); + testOuterJoinWithNullProbeAndFilterFunction(true, false, false); + testOuterJoinWithNullProbeAndFilterFunction(false, true, true); + testOuterJoinWithNullProbeAndFilterFunction(false, true, false); + testOuterJoinWithNullProbeAndFilterFunction(false, false, true); + testOuterJoinWithNullProbeAndFilterFunction(false, false, false); + } + + private void testOuterJoinWithNullProbeAndFilterFunction(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) { TaskContext taskContext = createTaskContext(); @@ -1018,8 +1094,20 @@ public void testOuterJoinWithNullProbeAndFilterFunction(boolean parallelBuild, b assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); } - @Test(dataProvider = "hashJoinTestValues") - public void testOuterJoinWithNullBuild(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + @Test + public void testOuterJoinWithNullBuild() + { + testOuterJoinWithNullBuild(true, true, true); + testOuterJoinWithNullBuild(true, true, false); + testOuterJoinWithNullBuild(true, false, true); + testOuterJoinWithNullBuild(true, false, false); + testOuterJoinWithNullBuild(false, true, true); + testOuterJoinWithNullBuild(false, true, false); + testOuterJoinWithNullBuild(false, false, true); + testOuterJoinWithNullBuild(false, false, false); + } + + private void testOuterJoinWithNullBuild(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) { TaskContext taskContext = createTaskContext(); @@ -1059,8 +1147,20 @@ public void testOuterJoinWithNullBuild(boolean parallelBuild, boolean probeHashE assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); } - @Test(dataProvider = "hashJoinTestValues") - public void testOuterJoinWithNullBuildAndFilterFunction(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + @Test + public void testOuterJoinWithNullBuildAndFilterFunction() + { + testOuterJoinWithNullBuildAndFilterFunction(true, true, true); + testOuterJoinWithNullBuildAndFilterFunction(true, true, false); + testOuterJoinWithNullBuildAndFilterFunction(true, false, true); + testOuterJoinWithNullBuildAndFilterFunction(true, false, false); + testOuterJoinWithNullBuildAndFilterFunction(false, true, true); + testOuterJoinWithNullBuildAndFilterFunction(false, true, false); + testOuterJoinWithNullBuildAndFilterFunction(false, false, true); + testOuterJoinWithNullBuildAndFilterFunction(false, false, false); + } + + private void testOuterJoinWithNullBuildAndFilterFunction(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) { TaskContext taskContext = createTaskContext(); @@ -1104,8 +1204,20 @@ public void testOuterJoinWithNullBuildAndFilterFunction(boolean parallelBuild, b assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); } - @Test(dataProvider = "hashJoinTestValues") - public void testOuterJoinWithNullOnBothSides(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + @Test + public void testOuterJoinWithNullOnBothSides() + { + testOuterJoinWithNullOnBothSides(true, true, true); + testOuterJoinWithNullOnBothSides(true, true, false); + testOuterJoinWithNullOnBothSides(true, false, true); + testOuterJoinWithNullOnBothSides(true, false, false); + testOuterJoinWithNullOnBothSides(false, true, true); + testOuterJoinWithNullOnBothSides(false, true, false); + testOuterJoinWithNullOnBothSides(false, false, true); + testOuterJoinWithNullOnBothSides(false, false, false); + } + + private void testOuterJoinWithNullOnBothSides(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) { TaskContext taskContext = createTaskContext(); @@ -1146,8 +1258,20 @@ public void testOuterJoinWithNullOnBothSides(boolean parallelBuild, boolean prob assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); } - @Test(dataProvider = "hashJoinTestValues") - public void testOuterJoinWithNullOnBothSidesAndFilterFunction(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + @Test + public void testOuterJoinWithNullOnBothSidesAndFilterFunction() + { + testOuterJoinWithNullOnBothSidesAndFilterFunction(true, true, true); + testOuterJoinWithNullOnBothSidesAndFilterFunction(true, true, false); + testOuterJoinWithNullOnBothSidesAndFilterFunction(true, false, true); + testOuterJoinWithNullOnBothSidesAndFilterFunction(true, false, false); + testOuterJoinWithNullOnBothSidesAndFilterFunction(false, true, true); + testOuterJoinWithNullOnBothSidesAndFilterFunction(false, true, false); + testOuterJoinWithNullOnBothSidesAndFilterFunction(false, false, true); + testOuterJoinWithNullOnBothSidesAndFilterFunction(false, false, false); + } + + private void testOuterJoinWithNullOnBothSidesAndFilterFunction(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) { TaskContext taskContext = createTaskContext(); @@ -1192,8 +1316,16 @@ public void testOuterJoinWithNullOnBothSidesAndFilterFunction(boolean parallelBu assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); } - @Test(dataProvider = "testMemoryLimitProvider") - public void testMemoryLimit(boolean parallelBuild, boolean buildHashEnabled) + @Test + public void testMemoryLimit() + { + testMemoryLimit(true, true); + testMemoryLimit(true, false); + testMemoryLimit(false, true); + testMemoryLimit(false, false); + } + + private void testMemoryLimit(boolean parallelBuild, boolean buildHashEnabled) { TaskContext taskContext = TestingTaskContext.createTaskContext(executor, scheduledExecutor, TEST_SESSION, DataSize.ofBytes(100)); @@ -1207,8 +1339,20 @@ public void testMemoryLimit(boolean parallelBuild, boolean buildHashEnabled) .hasMessageMatching("Query exceeded per-node memory limit of.*"); } - @Test(dataProvider = "hashJoinTestValues") - public void testInnerJoinWithEmptyLookupSource(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + @Test + public void testInnerJoinWithEmptyLookupSource() + { + testInnerJoinWithEmptyLookupSource(true, true, true); + testInnerJoinWithEmptyLookupSource(true, true, false); + testInnerJoinWithEmptyLookupSource(true, false, true); + testInnerJoinWithEmptyLookupSource(true, false, false); + testInnerJoinWithEmptyLookupSource(false, true, true); + testInnerJoinWithEmptyLookupSource(false, true, false); + testInnerJoinWithEmptyLookupSource(false, false, true); + testInnerJoinWithEmptyLookupSource(false, false, false); + } + + private void testInnerJoinWithEmptyLookupSource(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) { TaskContext taskContext = createTaskContext(); @@ -1246,8 +1390,20 @@ public void testInnerJoinWithEmptyLookupSource(boolean parallelBuild, boolean pr assertThat(outputPage).isNull(); } - @Test(dataProvider = "hashJoinTestValues") - public void testLookupOuterJoinWithEmptyLookupSource(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + @Test + public void testLookupOuterJoinWithEmptyLookupSource() + { + testLookupOuterJoinWithEmptyLookupSource(true, true, true); + testLookupOuterJoinWithEmptyLookupSource(true, true, false); + testLookupOuterJoinWithEmptyLookupSource(true, false, true); + testLookupOuterJoinWithEmptyLookupSource(true, false, false); + testLookupOuterJoinWithEmptyLookupSource(false, true, true); + testLookupOuterJoinWithEmptyLookupSource(false, true, false); + testLookupOuterJoinWithEmptyLookupSource(false, false, true); + testLookupOuterJoinWithEmptyLookupSource(false, false, false); + } + + private void testLookupOuterJoinWithEmptyLookupSource(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) { TaskContext taskContext = createTaskContext(); @@ -1285,8 +1441,20 @@ public void testLookupOuterJoinWithEmptyLookupSource(boolean parallelBuild, bool assertThat(outputPage).isNull(); } - @Test(dataProvider = "hashJoinTestValues") - public void testProbeOuterJoinWithEmptyLookupSource(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + @Test + public void testProbeOuterJoinWithEmptyLookupSource() + { + testProbeOuterJoinWithEmptyLookupSource(true, true, true); + testProbeOuterJoinWithEmptyLookupSource(true, true, false); + testProbeOuterJoinWithEmptyLookupSource(true, false, true); + testProbeOuterJoinWithEmptyLookupSource(true, false, false); + testProbeOuterJoinWithEmptyLookupSource(false, true, true); + testProbeOuterJoinWithEmptyLookupSource(false, true, false); + testProbeOuterJoinWithEmptyLookupSource(false, false, true); + testProbeOuterJoinWithEmptyLookupSource(false, false, false); + } + + private void testProbeOuterJoinWithEmptyLookupSource(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) { TaskContext taskContext = createTaskContext(); @@ -1333,8 +1501,20 @@ public void testProbeOuterJoinWithEmptyLookupSource(boolean parallelBuild, boole assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); } - @Test(dataProvider = "hashJoinTestValues") - public void testFullOuterJoinWithEmptyLookupSource(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + @Test + public void testFullOuterJoinWithEmptyLookupSource() + { + testFullOuterJoinWithEmptyLookupSource(true, true, true); + testFullOuterJoinWithEmptyLookupSource(true, true, false); + testFullOuterJoinWithEmptyLookupSource(true, false, true); + testFullOuterJoinWithEmptyLookupSource(true, false, false); + testFullOuterJoinWithEmptyLookupSource(false, true, true); + testFullOuterJoinWithEmptyLookupSource(false, true, false); + testFullOuterJoinWithEmptyLookupSource(false, false, true); + testFullOuterJoinWithEmptyLookupSource(false, false, false); + } + + private void testFullOuterJoinWithEmptyLookupSource(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) { TaskContext taskContext = createTaskContext(); @@ -1381,8 +1561,20 @@ public void testFullOuterJoinWithEmptyLookupSource(boolean parallelBuild, boolea assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); } - @Test(dataProvider = "hashJoinTestValues") - public void testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + @Test + public void testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe() + { + testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(true, true, true); + testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(true, true, false); + testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(true, false, true); + testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(true, false, false); + testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(false, true, true); + testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(false, true, false); + testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(false, false, true); + testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(false, false, false); + } + + private void testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) { TaskContext taskContext = createTaskContext(); @@ -1423,8 +1615,21 @@ public void testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(boolean parallelB assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); } - @Test(dataProvider = "hashJoinTestValues") - public void testInnerJoinWithBlockingLookupSourceAndEmptyProbe(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + @Test + public void testInnerJoinWithBlockingLookupSourceAndEmptyProbe() + throws Exception + { + testInnerJoinWithBlockingLookupSourceAndEmptyProbe(true, true, true); + testInnerJoinWithBlockingLookupSourceAndEmptyProbe(true, true, false); + testInnerJoinWithBlockingLookupSourceAndEmptyProbe(true, false, true); + testInnerJoinWithBlockingLookupSourceAndEmptyProbe(true, false, false); + testInnerJoinWithBlockingLookupSourceAndEmptyProbe(false, true, true); + testInnerJoinWithBlockingLookupSourceAndEmptyProbe(false, true, false); + testInnerJoinWithBlockingLookupSourceAndEmptyProbe(false, false, true); + testInnerJoinWithBlockingLookupSourceAndEmptyProbe(false, false, false); + } + + private void testInnerJoinWithBlockingLookupSourceAndEmptyProbe(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) throws Exception { // join that waits for build side to be collected @@ -1459,8 +1664,21 @@ public void testInnerJoinWithBlockingLookupSourceAndEmptyProbe(boolean parallelB } } - @Test(dataProvider = "hashJoinTestValues") - public void testInnerJoinWithBlockingLookupSource(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + @Test + public void testInnerJoinWithBlockingLookupSource() + throws Exception + { + testInnerJoinWithBlockingLookupSource(true, true, true); + testInnerJoinWithBlockingLookupSource(true, true, false); + testInnerJoinWithBlockingLookupSource(true, false, true); + testInnerJoinWithBlockingLookupSource(true, false, false); + testInnerJoinWithBlockingLookupSource(false, true, true); + testInnerJoinWithBlockingLookupSource(false, true, false); + testInnerJoinWithBlockingLookupSource(false, false, true); + testInnerJoinWithBlockingLookupSource(false, false, false); + } + + private void testInnerJoinWithBlockingLookupSource(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) throws Exception { RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), ImmutableList.of(VARCHAR)); @@ -1612,16 +1830,6 @@ private OperatorFactory createJoinOperatorFactoryWithBlockingLookupSource(TaskCo return joinOperatorFactory; } - @DataProvider - public static Object[][] testMemoryLimitProvider() - { - return new Object[][] { - {true, true}, - {true, false}, - {false, true}, - {false, false}}; - } - private TaskContext createTaskContext() { return TestingTaskContext.createTaskContext(executor, scheduledExecutor, TEST_SESSION); diff --git a/core/trino-main/src/test/java/io/trino/operator/join/unspilled/TestHashJoinOperator.java b/core/trino-main/src/test/java/io/trino/operator/join/unspilled/TestHashJoinOperator.java index da5dc12327293..9fe885684469a 100644 --- a/core/trino-main/src/test/java/io/trino/operator/join/unspilled/TestHashJoinOperator.java +++ b/core/trino-main/src/test/java/io/trino/operator/join/unspilled/TestHashJoinOperator.java @@ -51,13 +51,11 @@ import io.trino.spi.type.TypeOperators; import io.trino.sql.planner.NodePartitioningManager; import io.trino.sql.planner.plan.PlanNodeId; -import io.trino.testing.DataProviders; import io.trino.testing.MaterializedResult; import io.trino.testing.TestingTaskContext; import io.trino.util.FinalizerService; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import java.util.List; @@ -92,8 +90,6 @@ import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.spi.type.IntegerType.INTEGER; import static io.trino.spi.type.VarcharType.VARCHAR; -import static io.trino.testing.DataProviders.cartesianProduct; -import static io.trino.testing.DataProviders.trueFalse; import static java.util.concurrent.Executors.newScheduledThreadPool; import static java.util.concurrent.TimeUnit.SECONDS; import static org.assertj.core.api.Assertions.assertThat; @@ -144,8 +140,20 @@ public void tearDown() scheduledExecutor.shutdownNow(); } - @Test(dataProvider = "hashJoinTestValues") - public void testInnerJoin(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + @Test + public void testInnerJoin() + { + testInnerJoin(false, false, false); + testInnerJoin(false, false, true); + testInnerJoin(false, true, false); + testInnerJoin(false, true, true); + testInnerJoin(true, false, false); + testInnerJoin(true, false, true); + testInnerJoin(true, true, false); + testInnerJoin(true, true, true); + } + + private void testInnerJoin(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) { TaskContext taskContext = createTaskContext(); @@ -183,8 +191,20 @@ public void testInnerJoin(boolean parallelBuild, boolean probeHashEnabled, boole assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); } - @Test(dataProvider = "hashJoinRleProbeTestValues") - public void testInnerJoinWithRunLengthEncodedProbe(boolean withFilter, boolean probeHashEnabled, boolean singleBigintLookupSource) + @Test + public void testInnerJoinWithRunLengthEncodedProbe() + { + testInnerJoinWithRunLengthEncodedProbe(false, false, false); + testInnerJoinWithRunLengthEncodedProbe(false, false, true); + testInnerJoinWithRunLengthEncodedProbe(false, true, false); + testInnerJoinWithRunLengthEncodedProbe(false, true, true); + testInnerJoinWithRunLengthEncodedProbe(true, false, false); + testInnerJoinWithRunLengthEncodedProbe(true, false, true); + testInnerJoinWithRunLengthEncodedProbe(true, true, false); + testInnerJoinWithRunLengthEncodedProbe(true, true, true); + } + + private void testInnerJoinWithRunLengthEncodedProbe(boolean withFilter, boolean probeHashEnabled, boolean singleBigintLookupSource) { TaskContext taskContext = createTaskContext(); @@ -252,14 +272,14 @@ private JoinOperatorInfo getJoinOperatorInfo(DriverContext driverContext) return (JoinOperatorInfo) getOnlyElement(driverContext.getOperatorStats()).getInfo(); } - @DataProvider(name = "hashJoinRleProbeTestValues") - public static Object[][] hashJoinRleProbeTestValuesProvider() + @Test + public void testUnwrapsLazyBlocks() { - return cartesianProduct(trueFalse(), trueFalse(), trueFalse()); + testUnwrapsLazyBlocks(false); + testUnwrapsLazyBlocks(true); } - @Test(dataProvider = "singleBigintLookupSourceProvider") - public void testUnwrapsLazyBlocks(boolean singleBigintLookupSource) + private void testUnwrapsLazyBlocks(boolean singleBigintLookupSource) { TaskContext taskContext = createTaskContext(); DriverContext driverContext = taskContext.addPipelineContext(0, true, true, false).addDriverContext(); @@ -304,8 +324,14 @@ public void testUnwrapsLazyBlocks(boolean singleBigintLookupSource) assertThat(output.getBlock(1) instanceof LazyBlock).isFalse(); } - @Test(dataProvider = "singleBigintLookupSourceProvider") - public void testYield(boolean singleBigintLookupSource) + @Test + public void testYield() + { + testYield(false); + testYield(true); + } + + private void testYield(boolean singleBigintLookupSource) { // create a filter function that yields for every probe match // verify we will yield #match times totally @@ -375,8 +401,28 @@ public void testYield(boolean singleBigintLookupSource) assertThat(output.getPositionCount()).isEqualTo(entries); } - @Test(dataProvider = "hashJoinTestValuesAndsingleBigintLookupSourceProvider") - public void testInnerJoinWithNullProbe(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled, boolean singleBigintLookupSource) + @Test + public void testInnerJoinWithNullProbe() + { + testInnerJoinWithNullProbe(false, false, false, false); + testInnerJoinWithNullProbe(false, false, false, true); + testInnerJoinWithNullProbe(false, false, true, false); + testInnerJoinWithNullProbe(false, false, true, true); + testInnerJoinWithNullProbe(false, true, false, false); + testInnerJoinWithNullProbe(false, true, false, true); + testInnerJoinWithNullProbe(false, true, true, false); + testInnerJoinWithNullProbe(false, true, true, true); + testInnerJoinWithNullProbe(true, false, false, false); + testInnerJoinWithNullProbe(true, false, false, true); + testInnerJoinWithNullProbe(true, false, true, false); + testInnerJoinWithNullProbe(true, false, true, true); + testInnerJoinWithNullProbe(true, true, false, false); + testInnerJoinWithNullProbe(true, true, false, true); + testInnerJoinWithNullProbe(true, true, true, false); + testInnerJoinWithNullProbe(true, true, true, true); + } + + private void testInnerJoinWithNullProbe(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled, boolean singleBigintLookupSource) { TaskContext taskContext = createTaskContext(); @@ -415,8 +461,28 @@ public void testInnerJoinWithNullProbe(boolean parallelBuild, boolean probeHashE assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); } - @Test(dataProvider = "hashJoinTestValuesAndsingleBigintLookupSourceProvider") - public void testInnerJoinWithOutputSingleMatch(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled, boolean singleBigintLookupSource) + @Test + public void testInnerJoinWithOutputSingleMatch() + { + testInnerJoinWithOutputSingleMatch(false, false, false, false); + testInnerJoinWithOutputSingleMatch(false, false, false, true); + testInnerJoinWithOutputSingleMatch(false, false, true, false); + testInnerJoinWithOutputSingleMatch(false, false, true, true); + testInnerJoinWithOutputSingleMatch(false, true, false, false); + testInnerJoinWithOutputSingleMatch(false, true, false, true); + testInnerJoinWithOutputSingleMatch(false, true, true, false); + testInnerJoinWithOutputSingleMatch(false, true, true, true); + testInnerJoinWithOutputSingleMatch(true, false, false, false); + testInnerJoinWithOutputSingleMatch(true, false, false, true); + testInnerJoinWithOutputSingleMatch(true, false, true, false); + testInnerJoinWithOutputSingleMatch(true, false, true, true); + testInnerJoinWithOutputSingleMatch(true, true, false, false); + testInnerJoinWithOutputSingleMatch(true, true, false, true); + testInnerJoinWithOutputSingleMatch(true, true, true, false); + testInnerJoinWithOutputSingleMatch(true, true, true, true); + } + + private void testInnerJoinWithOutputSingleMatch(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled, boolean singleBigintLookupSource) { TaskContext taskContext = createTaskContext(); // build factory @@ -451,8 +517,20 @@ public void testInnerJoinWithOutputSingleMatch(boolean parallelBuild, boolean pr assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); } - @Test(dataProvider = "hashJoinTestValuesAndsingleBigintLookupSourceProvider") - public void testInnerJoinWithNullBuild(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled, boolean singleBigintLookupSource) + @Test + public void testInnerJoinWithNullBuild() + { + testInnerJoinWithNullBuild(false, false, false); + testInnerJoinWithNullBuild(false, false, true); + testInnerJoinWithNullBuild(false, true, false); + testInnerJoinWithNullBuild(false, true, true); + testInnerJoinWithNullBuild(true, false, false); + testInnerJoinWithNullBuild(true, false, true); + testInnerJoinWithNullBuild(true, true, false); + testInnerJoinWithNullBuild(true, true, true); + } + + private void testInnerJoinWithNullBuild(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) { TaskContext taskContext = createTaskContext(); @@ -491,8 +569,20 @@ public void testInnerJoinWithNullBuild(boolean parallelBuild, boolean probeHashE assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); } - @Test(dataProvider = "hashJoinTestValuesAndsingleBigintLookupSourceProvider") - public void testInnerJoinWithNullOnBothSides(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled, boolean singleBigintLookupSource) + @Test + public void testInnerJoinWithNullOnBothSides() + { + testInnerJoinWithNullOnBothSides(false, false, false); + testInnerJoinWithNullOnBothSides(false, false, true); + testInnerJoinWithNullOnBothSides(false, true, false); + testInnerJoinWithNullOnBothSides(false, true, true); + testInnerJoinWithNullOnBothSides(true, false, false); + testInnerJoinWithNullOnBothSides(true, false, true); + testInnerJoinWithNullOnBothSides(true, true, false); + testInnerJoinWithNullOnBothSides(true, true, true); + } + + private void testInnerJoinWithNullOnBothSides(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) { TaskContext taskContext = createTaskContext(); @@ -532,8 +622,20 @@ public void testInnerJoinWithNullOnBothSides(boolean parallelBuild, boolean prob assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); } - @Test(dataProvider = "hashJoinTestValues") - public void testProbeOuterJoin(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + @Test + public void testProbeOuterJoin() + { + testProbeOuterJoin(false, false, false); + testProbeOuterJoin(false, false, true); + testProbeOuterJoin(false, true, false); + testProbeOuterJoin(false, true, true); + testProbeOuterJoin(true, false, false); + testProbeOuterJoin(true, false, true); + testProbeOuterJoin(true, true, false); + testProbeOuterJoin(true, true, true); + } + + private void testProbeOuterJoin(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) { TaskContext taskContext = createTaskContext(); @@ -578,8 +680,20 @@ public void testProbeOuterJoin(boolean parallelBuild, boolean probeHashEnabled, assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); } - @Test(dataProvider = "hashJoinTestValues") - public void testProbeOuterJoinWithFilterFunction(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + @Test + public void testProbeOuterJoinWithFilterFunction() + { + testProbeOuterJoinWithFilterFunction(false, false, false); + testProbeOuterJoinWithFilterFunction(false, false, true); + testProbeOuterJoinWithFilterFunction(false, true, false); + testProbeOuterJoinWithFilterFunction(false, true, true); + testProbeOuterJoinWithFilterFunction(true, false, false); + testProbeOuterJoinWithFilterFunction(true, false, true); + testProbeOuterJoinWithFilterFunction(true, true, false); + testProbeOuterJoinWithFilterFunction(true, true, true); + } + + private void testProbeOuterJoinWithFilterFunction(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) { TaskContext taskContext = createTaskContext(); @@ -627,8 +741,28 @@ public void testProbeOuterJoinWithFilterFunction(boolean parallelBuild, boolean assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); } - @Test(dataProvider = "hashJoinTestValuesAndsingleBigintLookupSourceProvider") - public void testOuterJoinWithNullProbe(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled, boolean singleBigintLookupSource) + @Test + public void testOuterJoinWithNullProbe() + { + testOuterJoinWithNullProbe(false, false, false, false); + testOuterJoinWithNullProbe(false, false, false, true); + testOuterJoinWithNullProbe(false, false, true, false); + testOuterJoinWithNullProbe(false, false, true, true); + testOuterJoinWithNullProbe(false, true, false, false); + testOuterJoinWithNullProbe(false, true, false, true); + testOuterJoinWithNullProbe(false, true, true, false); + testOuterJoinWithNullProbe(false, true, true, true); + testOuterJoinWithNullProbe(true, false, false, false); + testOuterJoinWithNullProbe(true, false, false, true); + testOuterJoinWithNullProbe(true, false, true, false); + testOuterJoinWithNullProbe(true, false, true, true); + testOuterJoinWithNullProbe(true, true, false, false); + testOuterJoinWithNullProbe(true, true, false, true); + testOuterJoinWithNullProbe(true, true, true, false); + testOuterJoinWithNullProbe(true, true, true, true); + } + + private void testOuterJoinWithNullProbe(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled, boolean singleBigintLookupSource) { TaskContext taskContext = createTaskContext(); @@ -669,8 +803,28 @@ public void testOuterJoinWithNullProbe(boolean parallelBuild, boolean probeHashE assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); } - @Test(dataProvider = "hashJoinTestValuesAndsingleBigintLookupSourceProvider") - public void testOuterJoinWithNullProbeAndFilterFunction(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled, boolean singleBigintLookupSource) + @Test + public void testOuterJoinWithNullProbeAndFilterFunction() + { + testOuterJoinWithNullProbeAndFilterFunction(false, false, false, false); + testOuterJoinWithNullProbeAndFilterFunction(false, false, false, true); + testOuterJoinWithNullProbeAndFilterFunction(false, false, true, false); + testOuterJoinWithNullProbeAndFilterFunction(false, false, true, true); + testOuterJoinWithNullProbeAndFilterFunction(false, true, false, false); + testOuterJoinWithNullProbeAndFilterFunction(false, true, false, true); + testOuterJoinWithNullProbeAndFilterFunction(false, true, true, false); + testOuterJoinWithNullProbeAndFilterFunction(false, true, true, true); + testOuterJoinWithNullProbeAndFilterFunction(true, false, false, false); + testOuterJoinWithNullProbeAndFilterFunction(true, false, false, true); + testOuterJoinWithNullProbeAndFilterFunction(true, false, true, false); + testOuterJoinWithNullProbeAndFilterFunction(true, false, true, true); + testOuterJoinWithNullProbeAndFilterFunction(true, true, false, false); + testOuterJoinWithNullProbeAndFilterFunction(true, true, false, true); + testOuterJoinWithNullProbeAndFilterFunction(true, true, true, false); + testOuterJoinWithNullProbeAndFilterFunction(true, true, true, true); + } + + private void testOuterJoinWithNullProbeAndFilterFunction(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled, boolean singleBigintLookupSource) { TaskContext taskContext = createTaskContext(); @@ -714,8 +868,28 @@ public void testOuterJoinWithNullProbeAndFilterFunction(boolean parallelBuild, b assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); } - @Test(dataProvider = "hashJoinTestValuesAndsingleBigintLookupSourceProvider") - public void testOuterJoinWithNullBuild(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled, boolean singleBigintLookupSource) + @Test + public void testOuterJoinWithNullBuild() + { + testOuterJoinWithNullBuild(false, false, false, false); + testOuterJoinWithNullBuild(false, false, false, true); + testOuterJoinWithNullBuild(false, false, true, false); + testOuterJoinWithNullBuild(false, false, true, true); + testOuterJoinWithNullBuild(false, true, false, false); + testOuterJoinWithNullBuild(false, true, false, true); + testOuterJoinWithNullBuild(false, true, true, false); + testOuterJoinWithNullBuild(false, true, true, true); + testOuterJoinWithNullBuild(true, false, false, false); + testOuterJoinWithNullBuild(true, false, false, true); + testOuterJoinWithNullBuild(true, false, true, false); + testOuterJoinWithNullBuild(true, false, true, true); + testOuterJoinWithNullBuild(true, true, false, false); + testOuterJoinWithNullBuild(true, true, false, true); + testOuterJoinWithNullBuild(true, true, true, false); + testOuterJoinWithNullBuild(true, true, true, true); + } + + private void testOuterJoinWithNullBuild(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled, boolean singleBigintLookupSource) { TaskContext taskContext = createTaskContext(); @@ -755,8 +929,28 @@ public void testOuterJoinWithNullBuild(boolean parallelBuild, boolean probeHashE assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); } - @Test(dataProvider = "hashJoinTestValuesAndsingleBigintLookupSourceProvider") - public void testOuterJoinWithNullBuildAndFilterFunction(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled, boolean singleBigintLookupSource) + @Test + public void testOuterJoinWithNullBuildAndFilterFunction() + { + testOuterJoinWithNullBuildAndFilterFunction(false, false, false, false); + testOuterJoinWithNullBuildAndFilterFunction(false, false, false, true); + testOuterJoinWithNullBuildAndFilterFunction(false, false, true, false); + testOuterJoinWithNullBuildAndFilterFunction(false, false, true, true); + testOuterJoinWithNullBuildAndFilterFunction(false, true, false, false); + testOuterJoinWithNullBuildAndFilterFunction(false, true, false, true); + testOuterJoinWithNullBuildAndFilterFunction(false, true, true, false); + testOuterJoinWithNullBuildAndFilterFunction(false, true, true, true); + testOuterJoinWithNullBuildAndFilterFunction(true, false, false, false); + testOuterJoinWithNullBuildAndFilterFunction(true, false, false, true); + testOuterJoinWithNullBuildAndFilterFunction(true, false, true, false); + testOuterJoinWithNullBuildAndFilterFunction(true, false, true, true); + testOuterJoinWithNullBuildAndFilterFunction(true, true, false, false); + testOuterJoinWithNullBuildAndFilterFunction(true, true, false, true); + testOuterJoinWithNullBuildAndFilterFunction(true, true, true, false); + testOuterJoinWithNullBuildAndFilterFunction(true, true, true, true); + } + + private void testOuterJoinWithNullBuildAndFilterFunction(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled, boolean singleBigintLookupSource) { TaskContext taskContext = createTaskContext(); @@ -800,8 +994,28 @@ public void testOuterJoinWithNullBuildAndFilterFunction(boolean parallelBuild, b assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); } - @Test(dataProvider = "hashJoinTestValuesAndsingleBigintLookupSourceProvider") - public void testOuterJoinWithNullOnBothSides(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled, boolean singleBigintLookupSource) + @Test + public void testOuterJoinWithNullOnBothSides() + { + testOuterJoinWithNullOnBothSides(false, false, false, false); + testOuterJoinWithNullOnBothSides(false, false, false, true); + testOuterJoinWithNullOnBothSides(false, false, true, false); + testOuterJoinWithNullOnBothSides(false, false, true, true); + testOuterJoinWithNullOnBothSides(false, true, false, false); + testOuterJoinWithNullOnBothSides(false, true, false, true); + testOuterJoinWithNullOnBothSides(false, true, true, false); + testOuterJoinWithNullOnBothSides(false, true, true, true); + testOuterJoinWithNullOnBothSides(true, false, false, false); + testOuterJoinWithNullOnBothSides(true, false, false, true); + testOuterJoinWithNullOnBothSides(true, false, true, false); + testOuterJoinWithNullOnBothSides(true, false, true, true); + testOuterJoinWithNullOnBothSides(true, true, false, false); + testOuterJoinWithNullOnBothSides(true, true, false, true); + testOuterJoinWithNullOnBothSides(true, true, true, false); + testOuterJoinWithNullOnBothSides(true, true, true, true); + } + + private void testOuterJoinWithNullOnBothSides(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled, boolean singleBigintLookupSource) { TaskContext taskContext = createTaskContext(); @@ -842,8 +1056,28 @@ public void testOuterJoinWithNullOnBothSides(boolean parallelBuild, boolean prob assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); } - @Test(dataProvider = "hashJoinTestValuesAndsingleBigintLookupSourceProvider") - public void testOuterJoinWithNullOnBothSidesAndFilterFunction(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled, boolean singleBigintLookupSource) + @Test + public void testOuterJoinWithNullOnBothSidesAndFilterFunction() + { + testOuterJoinWithNullOnBothSidesAndFilterFunction(false, false, false, false); + testOuterJoinWithNullOnBothSidesAndFilterFunction(false, false, false, true); + testOuterJoinWithNullOnBothSidesAndFilterFunction(false, false, true, false); + testOuterJoinWithNullOnBothSidesAndFilterFunction(false, false, true, true); + testOuterJoinWithNullOnBothSidesAndFilterFunction(false, true, false, false); + testOuterJoinWithNullOnBothSidesAndFilterFunction(false, true, false, true); + testOuterJoinWithNullOnBothSidesAndFilterFunction(false, true, true, false); + testOuterJoinWithNullOnBothSidesAndFilterFunction(false, true, true, true); + testOuterJoinWithNullOnBothSidesAndFilterFunction(true, false, false, false); + testOuterJoinWithNullOnBothSidesAndFilterFunction(true, false, false, true); + testOuterJoinWithNullOnBothSidesAndFilterFunction(true, false, true, false); + testOuterJoinWithNullOnBothSidesAndFilterFunction(true, false, true, true); + testOuterJoinWithNullOnBothSidesAndFilterFunction(true, true, false, false); + testOuterJoinWithNullOnBothSidesAndFilterFunction(true, true, false, true); + testOuterJoinWithNullOnBothSidesAndFilterFunction(true, true, true, false); + testOuterJoinWithNullOnBothSidesAndFilterFunction(true, true, true, true); + } + + private void testOuterJoinWithNullOnBothSidesAndFilterFunction(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled, boolean singleBigintLookupSource) { TaskContext taskContext = createTaskContext(); @@ -888,8 +1122,16 @@ public void testOuterJoinWithNullOnBothSidesAndFilterFunction(boolean parallelBu assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); } - @Test(dataProvider = "testMemoryLimitProvider") - public void testMemoryLimit(boolean parallelBuild, boolean buildHashEnabled) + @Test + public void testMemoryLimit() + { + testMemoryLimit(false, false); + testMemoryLimit(false, true); + testMemoryLimit(true, false); + testMemoryLimit(true, true); + } + + private void testMemoryLimit(boolean parallelBuild, boolean buildHashEnabled) { TaskContext taskContext = TestingTaskContext.createTaskContext(executor, scheduledExecutor, TEST_SESSION, DataSize.ofBytes(100)); @@ -903,8 +1145,28 @@ public void testMemoryLimit(boolean parallelBuild, boolean buildHashEnabled) .hasMessageMatching("Query exceeded per-node memory limit of.*"); } - @Test(dataProvider = "hashJoinTestValuesAndsingleBigintLookupSourceProvider") - public void testInnerJoinWithEmptyLookupSource(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled, boolean singleBigintLookupSource) + @Test + public void testInnerJoinWithEmptyLookupSource() + { + testInnerJoinWithEmptyLookupSource(false, false, false, false); + testInnerJoinWithEmptyLookupSource(false, false, false, true); + testInnerJoinWithEmptyLookupSource(false, false, true, false); + testInnerJoinWithEmptyLookupSource(false, false, true, true); + testInnerJoinWithEmptyLookupSource(false, true, false, false); + testInnerJoinWithEmptyLookupSource(false, true, false, true); + testInnerJoinWithEmptyLookupSource(false, true, true, false); + testInnerJoinWithEmptyLookupSource(false, true, true, true); + testInnerJoinWithEmptyLookupSource(true, false, false, false); + testInnerJoinWithEmptyLookupSource(true, false, false, true); + testInnerJoinWithEmptyLookupSource(true, false, true, false); + testInnerJoinWithEmptyLookupSource(true, false, true, true); + testInnerJoinWithEmptyLookupSource(true, true, false, false); + testInnerJoinWithEmptyLookupSource(true, true, false, true); + testInnerJoinWithEmptyLookupSource(true, true, true, false); + testInnerJoinWithEmptyLookupSource(true, true, true, true); + } + + private void testInnerJoinWithEmptyLookupSource(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled, boolean singleBigintLookupSource) { TaskContext taskContext = createTaskContext(); @@ -940,8 +1202,28 @@ public void testInnerJoinWithEmptyLookupSource(boolean parallelBuild, boolean pr assertThat(outputPage).isNull(); } - @Test(dataProvider = "hashJoinTestValuesAndsingleBigintLookupSourceProvider") - public void testLookupOuterJoinWithEmptyLookupSource(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled, boolean singleBigintLookupSource) + @Test + public void testLookupOuterJoinWithEmptyLookupSource() + { + testLookupOuterJoinWithEmptyLookupSource(false, false, false, false); + testLookupOuterJoinWithEmptyLookupSource(false, false, false, true); + testLookupOuterJoinWithEmptyLookupSource(false, false, true, false); + testLookupOuterJoinWithEmptyLookupSource(false, false, true, true); + testLookupOuterJoinWithEmptyLookupSource(false, true, false, false); + testLookupOuterJoinWithEmptyLookupSource(false, true, false, true); + testLookupOuterJoinWithEmptyLookupSource(false, true, true, false); + testLookupOuterJoinWithEmptyLookupSource(false, true, true, true); + testLookupOuterJoinWithEmptyLookupSource(true, false, false, false); + testLookupOuterJoinWithEmptyLookupSource(true, false, false, true); + testLookupOuterJoinWithEmptyLookupSource(true, false, true, false); + testLookupOuterJoinWithEmptyLookupSource(true, false, true, true); + testLookupOuterJoinWithEmptyLookupSource(true, true, false, false); + testLookupOuterJoinWithEmptyLookupSource(true, true, false, true); + testLookupOuterJoinWithEmptyLookupSource(true, true, true, false); + testLookupOuterJoinWithEmptyLookupSource(true, true, true, true); + } + + private void testLookupOuterJoinWithEmptyLookupSource(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled, boolean singleBigintLookupSource) { TaskContext taskContext = createTaskContext(); @@ -977,8 +1259,28 @@ public void testLookupOuterJoinWithEmptyLookupSource(boolean parallelBuild, bool assertThat(outputPage).isNull(); } - @Test(dataProvider = "hashJoinTestValuesAndsingleBigintLookupSourceProvider") - public void testProbeOuterJoinWithEmptyLookupSource(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled, boolean singleBigintLookupSource) + @Test + public void testProbeOuterJoinWithEmptyLookupSource() + { + testProbeOuterJoinWithEmptyLookupSource(false, false, false, false); + testProbeOuterJoinWithEmptyLookupSource(false, false, false, true); + testProbeOuterJoinWithEmptyLookupSource(false, false, true, false); + testProbeOuterJoinWithEmptyLookupSource(false, false, true, true); + testProbeOuterJoinWithEmptyLookupSource(false, true, false, false); + testProbeOuterJoinWithEmptyLookupSource(false, true, false, true); + testProbeOuterJoinWithEmptyLookupSource(false, true, true, false); + testProbeOuterJoinWithEmptyLookupSource(false, true, true, true); + testProbeOuterJoinWithEmptyLookupSource(true, false, false, false); + testProbeOuterJoinWithEmptyLookupSource(true, false, false, true); + testProbeOuterJoinWithEmptyLookupSource(true, false, true, false); + testProbeOuterJoinWithEmptyLookupSource(true, false, true, true); + testProbeOuterJoinWithEmptyLookupSource(true, true, false, false); + testProbeOuterJoinWithEmptyLookupSource(true, true, false, true); + testProbeOuterJoinWithEmptyLookupSource(true, true, true, false); + testProbeOuterJoinWithEmptyLookupSource(true, true, true, true); + } + + private void testProbeOuterJoinWithEmptyLookupSource(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled, boolean singleBigintLookupSource) { TaskContext taskContext = createTaskContext(); @@ -1023,8 +1325,28 @@ public void testProbeOuterJoinWithEmptyLookupSource(boolean parallelBuild, boole assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); } - @Test(dataProvider = "hashJoinTestValuesAndsingleBigintLookupSourceProvider") - public void testFullOuterJoinWithEmptyLookupSource(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled, boolean singleBigintLookupSource) + @Test + public void testFullOuterJoinWithEmptyLookupSource() + { + testFullOuterJoinWithEmptyLookupSource(false, false, false, false); + testFullOuterJoinWithEmptyLookupSource(false, false, false, true); + testFullOuterJoinWithEmptyLookupSource(false, false, true, false); + testFullOuterJoinWithEmptyLookupSource(false, false, true, true); + testFullOuterJoinWithEmptyLookupSource(false, true, false, false); + testFullOuterJoinWithEmptyLookupSource(false, true, false, true); + testFullOuterJoinWithEmptyLookupSource(false, true, true, false); + testFullOuterJoinWithEmptyLookupSource(false, true, true, true); + testFullOuterJoinWithEmptyLookupSource(true, false, false, false); + testFullOuterJoinWithEmptyLookupSource(true, false, false, true); + testFullOuterJoinWithEmptyLookupSource(true, false, true, false); + testFullOuterJoinWithEmptyLookupSource(true, false, true, true); + testFullOuterJoinWithEmptyLookupSource(true, true, false, false); + testFullOuterJoinWithEmptyLookupSource(true, true, false, true); + testFullOuterJoinWithEmptyLookupSource(true, true, true, false); + testFullOuterJoinWithEmptyLookupSource(true, true, true, true); + } + + private void testFullOuterJoinWithEmptyLookupSource(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled, boolean singleBigintLookupSource) { TaskContext taskContext = createTaskContext(); @@ -1069,8 +1391,28 @@ public void testFullOuterJoinWithEmptyLookupSource(boolean parallelBuild, boolea assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); } - @Test(dataProvider = "hashJoinTestValuesAndsingleBigintLookupSourceProvider") - public void testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled, boolean singleBigintLookupSource) + @Test + public void testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe() + { + testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(false, false, false, false); + testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(false, false, false, true); + testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(false, false, true, false); + testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(false, false, true, true); + testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(false, true, false, false); + testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(false, true, false, true); + testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(false, true, true, false); + testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(false, true, true, true); + testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(true, false, false, false); + testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(true, false, false, true); + testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(true, false, true, false); + testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(true, false, true, true); + testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(true, true, false, false); + testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(true, true, false, true); + testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(true, true, true, false); + testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(true, true, true, true); + } + + private void testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled, boolean singleBigintLookupSource) { TaskContext taskContext = createTaskContext(); @@ -1109,8 +1451,21 @@ public void testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(boolean parallelB assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); } - @Test(dataProvider = "hashJoinTestValues") - public void testInnerJoinWithBlockingLookupSourceAndEmptyProbe(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + @Test + public void testInnerJoinWithBlockingLookupSourceAndEmptyProbe() + throws Exception + { + testInnerJoinWithBlockingLookupSourceAndEmptyProbe(false, false, false); + testInnerJoinWithBlockingLookupSourceAndEmptyProbe(false, false, true); + testInnerJoinWithBlockingLookupSourceAndEmptyProbe(false, true, false); + testInnerJoinWithBlockingLookupSourceAndEmptyProbe(false, true, true); + testInnerJoinWithBlockingLookupSourceAndEmptyProbe(true, false, false); + testInnerJoinWithBlockingLookupSourceAndEmptyProbe(true, false, true); + testInnerJoinWithBlockingLookupSourceAndEmptyProbe(true, true, false); + testInnerJoinWithBlockingLookupSourceAndEmptyProbe(true, true, true); + } + + private void testInnerJoinWithBlockingLookupSourceAndEmptyProbe(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) throws Exception { // join that waits for build side to be collected @@ -1145,8 +1500,21 @@ public void testInnerJoinWithBlockingLookupSourceAndEmptyProbe(boolean parallelB } } - @Test(dataProvider = "hashJoinTestValues") - public void testInnerJoinWithBlockingLookupSource(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + @Test + public void testInnerJoinWithBlockingLookupSource() + throws Exception + { + testInnerJoinWithBlockingLookupSource(false, false, false); + testInnerJoinWithBlockingLookupSource(false, false, true); + testInnerJoinWithBlockingLookupSource(false, true, false); + testInnerJoinWithBlockingLookupSource(false, true, true); + testInnerJoinWithBlockingLookupSource(true, false, false); + testInnerJoinWithBlockingLookupSource(true, false, true); + testInnerJoinWithBlockingLookupSource(true, true, false); + testInnerJoinWithBlockingLookupSource(true, true, true); + } + + private void testInnerJoinWithBlockingLookupSource(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) throws Exception { RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), ImmutableList.of(VARCHAR)); @@ -1296,39 +1664,6 @@ private OperatorFactory createJoinOperatorFactoryWithBlockingLookupSource(TaskCo return joinOperatorFactory; } - @DataProvider(name = "hashJoinTestValues") - public static Object[][] hashJoinTestValuesProvider() - { - return DataProviders.cartesianProduct( - new Object[][] {{true}, {false}}, - new Object[][] {{true}, {false}}, - new Object[][] {{true}, {false}}); - } - - @DataProvider - public static Object[][] testMemoryLimitProvider() - { - return DataProviders.cartesianProduct( - new Object[][] {{true}, {false}}, - new Object[][] {{true}, {false}}); - } - - @DataProvider(name = "singleBigintLookupSourceProvider") - public static Object[][] singleBigintLookupSourceProvider() - { - return new Object[][] {{true}, {false}}; - } - - @DataProvider(name = "hashJoinTestValuesAndsingleBigintLookupSourceProvider") - public static Object[][] hashJoinTestValuesAndsingleBigintLookupSourceProvider() - { - return DataProviders.cartesianProduct( - new Object[][] {{true}, {false}}, - new Object[][] {{true}, {false}}, - new Object[][] {{true}, {false}}, - new Object[][] {{true}, {false}}); - } - private TaskContext createTaskContext() { return TestingTaskContext.createTaskContext(executor, scheduledExecutor, TEST_SESSION); diff --git a/core/trino-main/src/test/java/io/trino/operator/output/TestPagePartitioner.java b/core/trino-main/src/test/java/io/trino/operator/output/TestPagePartitioner.java index 444e029656335..fce46ff49eff1 100644 --- a/core/trino-main/src/test/java/io/trino/operator/output/TestPagePartitioner.java +++ b/core/trino-main/src/test/java/io/trino/operator/output/TestPagePartitioner.java @@ -42,7 +42,6 @@ import io.trino.spi.block.RunLengthEncodedBlock; import io.trino.spi.block.TestingBlockEncodingSerde; import io.trino.spi.predicate.NullableValue; -import io.trino.spi.type.AbstractType; import io.trino.spi.type.ArrayType; import io.trino.spi.type.Decimals; import io.trino.spi.type.TimestampType; @@ -52,8 +51,6 @@ import io.trino.type.BlockTypeOperators; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import java.util.ArrayList; @@ -113,7 +110,6 @@ public class TestPagePartitioner private ExecutorService executor; private ScheduledExecutorService scheduledExecutor; - private TestOutputBuffer outputBuffer; @BeforeClass public void setUpClass() @@ -131,16 +127,11 @@ public void tearDownClass() scheduledExecutor = null; } - @BeforeMethod - public void setUp() - { - outputBuffer = new TestOutputBuffer(); - } - @Test public void testOutputForEmptyPage() { - PagePartitioner pagePartitioner = pagePartitioner(BIGINT).build(); + TestOutputBuffer outputBuffer = new TestOutputBuffer(); + PagePartitioner pagePartitioner = pagePartitioner(outputBuffer, BIGINT).build(); Page page = new Page(createLongsBlock(ImmutableList.of())); pagePartitioner.partitionPage(page, operatorContext()); @@ -156,10 +147,18 @@ private OperatorContext operatorContext() .addOperatorContext(0, new PlanNodeId("plan-node-0"), PartitionedOutputOperator.class.getSimpleName()); } - @Test(dataProvider = "partitioningMode") - public void testOutputEqualsInput(PartitioningMode partitioningMode) + @Test + public void testOutputEqualsInput() + { + testOutputEqualsInput(PartitioningMode.ROW_WISE); + testOutputEqualsInput(PartitioningMode.COLUMNAR); + } + + private void testOutputEqualsInput(PartitioningMode partitioningMode) { - PagePartitioner pagePartitioner = pagePartitioner(BIGINT).build(); + TestOutputBuffer outputBuffer = new TestOutputBuffer(); + + PagePartitioner pagePartitioner = pagePartitioner(outputBuffer, BIGINT).build(); Page page = new Page(createLongSequenceBlock(0, POSITIONS_PER_PAGE)); List expected = readLongs(Stream.of(page), 0); @@ -169,10 +168,18 @@ public void testOutputEqualsInput(PartitioningMode partitioningMode) assertThat(partitioned).containsExactlyInAnyOrderElementsOf(expected); // order is different due to 2 partitions joined } - @Test(dataProvider = "partitioningMode") - public void testOutputForPageWithNoBlockPartitionFunction(PartitioningMode partitioningMode) + @Test + public void testOutputForPageWithNoBlockPartitionFunction() + { + testOutputForPageWithNoBlockPartitionFunction(PartitioningMode.ROW_WISE); + testOutputForPageWithNoBlockPartitionFunction(PartitioningMode.COLUMNAR); + } + + private void testOutputForPageWithNoBlockPartitionFunction(PartitioningMode partitioningMode) { - PagePartitioner pagePartitioner = pagePartitioner(BIGINT) + TestOutputBuffer outputBuffer = new TestOutputBuffer(); + + PagePartitioner pagePartitioner = pagePartitioner(outputBuffer, BIGINT) .withPartitionFunction(new BucketPartitionFunction( ROUND_ROBIN.createBucketFunction(null, false, PARTITION_COUNT, null), IntStream.range(0, PARTITION_COUNT).toArray())) @@ -188,10 +195,18 @@ public void testOutputForPageWithNoBlockPartitionFunction(PartitioningMode parti assertThat(partition1).containsExactly(1L, 3L, 5L, 7L); } - @Test(dataProvider = "partitioningMode") - public void testOutputForMultipleSimplePages(PartitioningMode partitioningMode) + @Test + public void testOutputForMultipleSimplePages() + { + testOutputForMultipleSimplePages(PartitioningMode.ROW_WISE); + testOutputForMultipleSimplePages(PartitioningMode.COLUMNAR); + } + + private void testOutputForMultipleSimplePages(PartitioningMode partitioningMode) { - PagePartitioner pagePartitioner = pagePartitioner(BIGINT).build(); + TestOutputBuffer outputBuffer = new TestOutputBuffer(); + + PagePartitioner pagePartitioner = pagePartitioner(outputBuffer, BIGINT).build(); Page page1 = new Page(createLongSequenceBlock(0, POSITIONS_PER_PAGE)); Page page2 = new Page(createLongSequenceBlock(1, POSITIONS_PER_PAGE)); Page page3 = new Page(createLongSequenceBlock(2, POSITIONS_PER_PAGE)); @@ -203,10 +218,17 @@ public void testOutputForMultipleSimplePages(PartitioningMode partitioningMode) assertThat(partitioned).containsExactlyInAnyOrderElementsOf(expected); // order is different due to 2 partitions joined } - @Test(dataProvider = "partitioningMode") - public void testOutputForSimplePageWithReplication(PartitioningMode partitioningMode) + @Test + public void testOutputForSimplePageWithReplication() + { + testOutputForSimplePageWithReplication(PartitioningMode.ROW_WISE); + testOutputForSimplePageWithReplication(PartitioningMode.COLUMNAR); + } + + private void testOutputForSimplePageWithReplication(PartitioningMode partitioningMode) { - PagePartitioner pagePartitioner = pagePartitioner(BIGINT).replicate().build(); + TestOutputBuffer outputBuffer = new TestOutputBuffer(); + PagePartitioner pagePartitioner = pagePartitioner(outputBuffer, BIGINT).replicate().build(); Page page = new Page(createLongsBlock(0L, 1L, 2L, 3L, null)); processPages(pagePartitioner, partitioningMode, page); @@ -217,10 +239,17 @@ public void testOutputForSimplePageWithReplication(PartitioningMode partitioning assertThat(partition1).containsExactly(0L, 1L, 3L); // position 0 copied to all partitions } - @Test(dataProvider = "partitioningMode") - public void testOutputForSimplePageWithNullChannel(PartitioningMode partitioningMode) + @Test + public void testOutputForSimplePageWithNullChannel() + { + testOutputForSimplePageWithNullChannel(PartitioningMode.ROW_WISE); + testOutputForSimplePageWithNullChannel(PartitioningMode.COLUMNAR); + } + + private void testOutputForSimplePageWithNullChannel(PartitioningMode partitioningMode) { - PagePartitioner pagePartitioner = pagePartitioner(BIGINT).withNullChannel(0).build(); + TestOutputBuffer outputBuffer = new TestOutputBuffer(); + PagePartitioner pagePartitioner = pagePartitioner(outputBuffer, BIGINT).withNullChannel(0).build(); Page page = new Page(createLongsBlock(0L, 1L, 2L, 3L, null)); processPages(pagePartitioner, partitioningMode, page); @@ -231,10 +260,17 @@ public void testOutputForSimplePageWithNullChannel(PartitioningMode partitioning assertThat(partition1).containsExactlyInAnyOrder(1L, 3L, null); // null copied to all partitions } - @Test(dataProvider = "partitioningMode") - public void testOutputForSimplePageWithPartitionConstant(PartitioningMode partitioningMode) + @Test + public void testOutputForSimplePageWithPartitionConstant() + { + testOutputForSimplePageWithPartitionConstant(PartitioningMode.ROW_WISE); + testOutputForSimplePageWithPartitionConstant(PartitioningMode.COLUMNAR); + } + + private void testOutputForSimplePageWithPartitionConstant(PartitioningMode partitioningMode) { - PagePartitioner pagePartitioner = pagePartitioner(BIGINT) + TestOutputBuffer outputBuffer = new TestOutputBuffer(); + PagePartitioner pagePartitioner = pagePartitioner(outputBuffer, BIGINT) .withPartitionConstants(ImmutableList.of(Optional.of(new NullableValue(BIGINT, 1L)))) .withPartitionChannels(-1) .build(); @@ -249,10 +285,17 @@ public void testOutputForSimplePageWithPartitionConstant(PartitioningMode partit assertThat(partition1).containsExactlyElementsOf(allValues); } - @Test(dataProvider = "partitioningMode") - public void testOutputForSimplePageWithPartitionConstantAndHashBlock(PartitioningMode partitioningMode) + @Test + public void testOutputForSimplePageWithPartitionConstantAndHashBlock() { - PagePartitioner pagePartitioner = pagePartitioner(BIGINT) + testOutputForSimplePageWithPartitionConstantAndHashBlock(PartitioningMode.ROW_WISE); + testOutputForSimplePageWithPartitionConstantAndHashBlock(PartitioningMode.COLUMNAR); + } + + private void testOutputForSimplePageWithPartitionConstantAndHashBlock(PartitioningMode partitioningMode) + { + TestOutputBuffer outputBuffer = new TestOutputBuffer(); + PagePartitioner pagePartitioner = pagePartitioner(outputBuffer, BIGINT) .withPartitionConstants(ImmutableList.of(Optional.empty(), Optional.of(new NullableValue(BIGINT, 1L)))) .withPartitionChannels(0, -1) // use first block and constant block at index 1 as input to partitionFunction .withHashChannels(0, 1) // use both channels to calculate partition (a+b) mod 2 @@ -267,10 +310,17 @@ public void testOutputForSimplePageWithPartitionConstantAndHashBlock(Partitionin assertThat(partition1).containsExactly(0L, 2L); } - @Test(dataProvider = "partitioningMode") - public void testPartitionPositionsWithRleNotNull(PartitioningMode partitioningMode) + @Test + public void testPartitionPositionsWithRleNotNull() + { + testPartitionPositionsWithRleNotNull(PartitioningMode.ROW_WISE); + testPartitionPositionsWithRleNotNull(PartitioningMode.COLUMNAR); + } + + private void testPartitionPositionsWithRleNotNull(PartitioningMode partitioningMode) { - PagePartitioner pagePartitioner = pagePartitioner(BIGINT, BIGINT).build(); + TestOutputBuffer outputBuffer = new TestOutputBuffer(); + PagePartitioner pagePartitioner = pagePartitioner(outputBuffer, BIGINT, BIGINT).build(); Page page = new Page(createRepeatedValuesBlock(0, POSITIONS_PER_PAGE), createLongSequenceBlock(0, POSITIONS_PER_PAGE)); processPages(pagePartitioner, partitioningMode, page); @@ -282,10 +332,17 @@ public void testPartitionPositionsWithRleNotNull(PartitioningMode partitioningMo assertThat(outputBuffer.getEnqueuedDeserialized(1)).isEmpty(); } - @Test(dataProvider = "partitioningMode") - public void testPartitionPositionsWithRleNotNullWithReplication(PartitioningMode partitioningMode) + @Test + public void testPartitionPositionsWithRleNotNullWithReplication() + { + testPartitionPositionsWithRleNotNullWithReplication(PartitioningMode.ROW_WISE); + testPartitionPositionsWithRleNotNullWithReplication(PartitioningMode.COLUMNAR); + } + + private void testPartitionPositionsWithRleNotNullWithReplication(PartitioningMode partitioningMode) { - PagePartitioner pagePartitioner = pagePartitioner(BIGINT, BIGINT).replicate().build(); + TestOutputBuffer outputBuffer = new TestOutputBuffer(); + PagePartitioner pagePartitioner = pagePartitioner(outputBuffer, BIGINT, BIGINT).replicate().build(); Page page = new Page(createRepeatedValuesBlock(0, POSITIONS_PER_PAGE), createLongSequenceBlock(0, POSITIONS_PER_PAGE)); processPages(pagePartitioner, partitioningMode, page); @@ -296,10 +353,17 @@ public void testPartitionPositionsWithRleNotNullWithReplication(PartitioningMode assertThat(partition1).containsExactly(0L); // position 0 copied to all partitions } - @Test(dataProvider = "partitioningMode") - public void testPartitionPositionsWithRleNullWithNullChannel(PartitioningMode partitioningMode) + @Test + public void testPartitionPositionsWithRleNullWithNullChannel() + { + testPartitionPositionsWithRleNullWithNullChannel(PartitioningMode.ROW_WISE); + testPartitionPositionsWithRleNullWithNullChannel(PartitioningMode.COLUMNAR); + } + + private void testPartitionPositionsWithRleNullWithNullChannel(PartitioningMode partitioningMode) { - PagePartitioner pagePartitioner = pagePartitioner(BIGINT, BIGINT).withNullChannel(0).build(); + TestOutputBuffer outputBuffer = new TestOutputBuffer(); + PagePartitioner pagePartitioner = pagePartitioner(outputBuffer, BIGINT, BIGINT).withNullChannel(0).build(); Page page = new Page(RunLengthEncodedBlock.create(createLongsBlock((Long) null), POSITIONS_PER_PAGE), createLongSequenceBlock(0, POSITIONS_PER_PAGE)); processPages(pagePartitioner, partitioningMode, page); @@ -310,10 +374,17 @@ public void testPartitionPositionsWithRleNullWithNullChannel(PartitioningMode pa assertThat(partition1).containsExactlyElementsOf(readLongs(Stream.of(page), 1)); } - @Test(dataProvider = "partitioningMode") - public void testOutputForDictionaryBlock(PartitioningMode partitioningMode) + @Test + public void testOutputForDictionaryBlock() + { + testOutputForDictionaryBlock(PartitioningMode.ROW_WISE); + testOutputForDictionaryBlock(PartitioningMode.COLUMNAR); + } + + private void testOutputForDictionaryBlock(PartitioningMode partitioningMode) { - PagePartitioner pagePartitioner = pagePartitioner(BIGINT).build(); + TestOutputBuffer outputBuffer = new TestOutputBuffer(); + PagePartitioner pagePartitioner = pagePartitioner(outputBuffer, BIGINT).build(); Page page = new Page(createLongDictionaryBlock(0, 10)); // must have at least 10 position to have non-trivial dict processPages(pagePartitioner, partitioningMode, page); @@ -324,10 +395,17 @@ public void testOutputForDictionaryBlock(PartitioningMode partitioningMode) assertThat(partition1).containsExactlyElementsOf(nCopies(5, 1L)); } - @Test(dataProvider = "partitioningMode") - public void testOutputForOneValueDictionaryBlock(PartitioningMode partitioningMode) + @Test + public void testOutputForOneValueDictionaryBlock() { - PagePartitioner pagePartitioner = pagePartitioner(BIGINT).build(); + testOutputForOneValueDictionaryBlock(PartitioningMode.ROW_WISE); + testOutputForOneValueDictionaryBlock(PartitioningMode.COLUMNAR); + } + + private void testOutputForOneValueDictionaryBlock(PartitioningMode partitioningMode) + { + TestOutputBuffer outputBuffer = new TestOutputBuffer(); + PagePartitioner pagePartitioner = pagePartitioner(outputBuffer, BIGINT).build(); Page page = new Page(DictionaryBlock.create(4, createLongsBlock(0), new int[] {0, 0, 0, 0})); processPages(pagePartitioner, partitioningMode, page); @@ -338,10 +416,17 @@ public void testOutputForOneValueDictionaryBlock(PartitioningMode partitioningMo assertThat(partition1).isEmpty(); } - @Test(dataProvider = "partitioningMode") - public void testOutputForViewDictionaryBlock(PartitioningMode partitioningMode) + @Test + public void testOutputForViewDictionaryBlock() + { + testOutputForViewDictionaryBlock(PartitioningMode.ROW_WISE); + testOutputForViewDictionaryBlock(PartitioningMode.COLUMNAR); + } + + private void testOutputForViewDictionaryBlock(PartitioningMode partitioningMode) { - PagePartitioner pagePartitioner = pagePartitioner(BIGINT).build(); + TestOutputBuffer outputBuffer = new TestOutputBuffer(); + PagePartitioner pagePartitioner = pagePartitioner(outputBuffer, BIGINT).build(); Page page = new Page(DictionaryBlock.create(4, createLongSequenceBlock(4, 8), new int[] {1, 0, 3, 2})); processPages(pagePartitioner, partitioningMode, page); @@ -352,10 +437,48 @@ public void testOutputForViewDictionaryBlock(PartitioningMode partitioningMode) assertThat(partition1).containsExactlyInAnyOrder(5L, 7L); } - @Test(dataProvider = "typesWithPartitioningMode") - public void testOutputForSimplePageWithType(Type type, PartitioningMode partitioningMode) - { - PagePartitioner pagePartitioner = pagePartitioner(BIGINT, type).build(); + @Test + public void testOutputForSimplePageWithType() + { + testOutputForSimplePageWithType(BIGINT, PartitioningMode.ROW_WISE); + testOutputForSimplePageWithType(BOOLEAN, PartitioningMode.ROW_WISE); + testOutputForSimplePageWithType(INTEGER, PartitioningMode.ROW_WISE); + testOutputForSimplePageWithType(createCharType(10), PartitioningMode.ROW_WISE); + testOutputForSimplePageWithType(createUnboundedVarcharType(), PartitioningMode.ROW_WISE); + testOutputForSimplePageWithType(DOUBLE, PartitioningMode.ROW_WISE); + testOutputForSimplePageWithType(SMALLINT, PartitioningMode.ROW_WISE); + testOutputForSimplePageWithType(TINYINT, PartitioningMode.ROW_WISE); + testOutputForSimplePageWithType(UUID, PartitioningMode.ROW_WISE); + testOutputForSimplePageWithType(VARBINARY, PartitioningMode.ROW_WISE); + testOutputForSimplePageWithType(createDecimalType(1), PartitioningMode.ROW_WISE); + testOutputForSimplePageWithType(createDecimalType(Decimals.MAX_SHORT_PRECISION + 1), PartitioningMode.ROW_WISE); + testOutputForSimplePageWithType(new ArrayType(BIGINT), PartitioningMode.ROW_WISE); + testOutputForSimplePageWithType(TimestampType.createTimestampType(9), PartitioningMode.ROW_WISE); + testOutputForSimplePageWithType(TimestampType.createTimestampType(3), PartitioningMode.ROW_WISE); + testOutputForSimplePageWithType(IPADDRESS, PartitioningMode.ROW_WISE); + + testOutputForSimplePageWithType(BIGINT, PartitioningMode.COLUMNAR); + testOutputForSimplePageWithType(BOOLEAN, PartitioningMode.COLUMNAR); + testOutputForSimplePageWithType(INTEGER, PartitioningMode.COLUMNAR); + testOutputForSimplePageWithType(createCharType(10), PartitioningMode.COLUMNAR); + testOutputForSimplePageWithType(createUnboundedVarcharType(), PartitioningMode.COLUMNAR); + testOutputForSimplePageWithType(DOUBLE, PartitioningMode.COLUMNAR); + testOutputForSimplePageWithType(SMALLINT, PartitioningMode.COLUMNAR); + testOutputForSimplePageWithType(TINYINT, PartitioningMode.COLUMNAR); + testOutputForSimplePageWithType(UUID, PartitioningMode.COLUMNAR); + testOutputForSimplePageWithType(VARBINARY, PartitioningMode.COLUMNAR); + testOutputForSimplePageWithType(createDecimalType(1), PartitioningMode.COLUMNAR); + testOutputForSimplePageWithType(createDecimalType(Decimals.MAX_SHORT_PRECISION + 1), PartitioningMode.COLUMNAR); + testOutputForSimplePageWithType(new ArrayType(BIGINT), PartitioningMode.COLUMNAR); + testOutputForSimplePageWithType(TimestampType.createTimestampType(9), PartitioningMode.COLUMNAR); + testOutputForSimplePageWithType(TimestampType.createTimestampType(3), PartitioningMode.COLUMNAR); + testOutputForSimplePageWithType(IPADDRESS, PartitioningMode.COLUMNAR); + } + + private void testOutputForSimplePageWithType(Type type, PartitioningMode partitioningMode) + { + TestOutputBuffer outputBuffer = new TestOutputBuffer(); + PagePartitioner pagePartitioner = pagePartitioner(outputBuffer, BIGINT, type).build(); Page page = new Page( createLongSequenceBlock(0, POSITIONS_PER_PAGE), // partition block createBlockForType(type, POSITIONS_PER_PAGE)); @@ -367,18 +490,56 @@ public void testOutputForSimplePageWithType(Type type, PartitioningMode partitio assertThat(partitioned).containsExactlyInAnyOrderElementsOf(expected); // order is different due to 2 partitions joined } - @Test(dataProvider = "types") - public void testOutputWithMixedRowWiseAndColumnarPartitioning(Type type) + @Test + public void testOutputWithMixedRowWiseAndColumnarPartitioning() + { + testOutputEqualsInput(BIGINT, PartitioningMode.COLUMNAR, PartitioningMode.ROW_WISE); + testOutputEqualsInput(BOOLEAN, PartitioningMode.COLUMNAR, PartitioningMode.ROW_WISE); + testOutputEqualsInput(INTEGER, PartitioningMode.COLUMNAR, PartitioningMode.ROW_WISE); + testOutputEqualsInput(createCharType(10), PartitioningMode.COLUMNAR, PartitioningMode.ROW_WISE); + testOutputEqualsInput(createUnboundedVarcharType(), PartitioningMode.COLUMNAR, PartitioningMode.ROW_WISE); + testOutputEqualsInput(DOUBLE, PartitioningMode.COLUMNAR, PartitioningMode.ROW_WISE); + testOutputEqualsInput(SMALLINT, PartitioningMode.COLUMNAR, PartitioningMode.ROW_WISE); + testOutputEqualsInput(TINYINT, PartitioningMode.COLUMNAR, PartitioningMode.ROW_WISE); + testOutputEqualsInput(UUID, PartitioningMode.COLUMNAR, PartitioningMode.ROW_WISE); + testOutputEqualsInput(VARBINARY, PartitioningMode.COLUMNAR, PartitioningMode.ROW_WISE); + testOutputEqualsInput(createDecimalType(1), PartitioningMode.COLUMNAR, PartitioningMode.ROW_WISE); + testOutputEqualsInput(createDecimalType(Decimals.MAX_SHORT_PRECISION + 1), PartitioningMode.COLUMNAR, PartitioningMode.ROW_WISE); + testOutputEqualsInput(new ArrayType(BIGINT), PartitioningMode.COLUMNAR, PartitioningMode.ROW_WISE); + testOutputEqualsInput(TimestampType.createTimestampType(9), PartitioningMode.COLUMNAR, PartitioningMode.ROW_WISE); + testOutputEqualsInput(TimestampType.createTimestampType(3), PartitioningMode.COLUMNAR, PartitioningMode.ROW_WISE); + testOutputEqualsInput(IPADDRESS, PartitioningMode.COLUMNAR, PartitioningMode.ROW_WISE); + + testOutputEqualsInput(BIGINT, PartitioningMode.ROW_WISE, PartitioningMode.COLUMNAR); + testOutputEqualsInput(BOOLEAN, PartitioningMode.ROW_WISE, PartitioningMode.COLUMNAR); + testOutputEqualsInput(INTEGER, PartitioningMode.ROW_WISE, PartitioningMode.COLUMNAR); + testOutputEqualsInput(createCharType(10), PartitioningMode.ROW_WISE, PartitioningMode.COLUMNAR); + testOutputEqualsInput(createUnboundedVarcharType(), PartitioningMode.ROW_WISE, PartitioningMode.COLUMNAR); + testOutputEqualsInput(DOUBLE, PartitioningMode.ROW_WISE, PartitioningMode.COLUMNAR); + testOutputEqualsInput(SMALLINT, PartitioningMode.ROW_WISE, PartitioningMode.COLUMNAR); + testOutputEqualsInput(TINYINT, PartitioningMode.ROW_WISE, PartitioningMode.COLUMNAR); + testOutputEqualsInput(UUID, PartitioningMode.ROW_WISE, PartitioningMode.COLUMNAR); + testOutputEqualsInput(VARBINARY, PartitioningMode.ROW_WISE, PartitioningMode.COLUMNAR); + testOutputEqualsInput(createDecimalType(1), PartitioningMode.ROW_WISE, PartitioningMode.COLUMNAR); + testOutputEqualsInput(createDecimalType(Decimals.MAX_SHORT_PRECISION + 1), PartitioningMode.ROW_WISE, PartitioningMode.COLUMNAR); + testOutputEqualsInput(new ArrayType(BIGINT), PartitioningMode.ROW_WISE, PartitioningMode.COLUMNAR); + testOutputEqualsInput(TimestampType.createTimestampType(9), PartitioningMode.ROW_WISE, PartitioningMode.COLUMNAR); + testOutputEqualsInput(TimestampType.createTimestampType(3), PartitioningMode.ROW_WISE, PartitioningMode.COLUMNAR); + testOutputEqualsInput(IPADDRESS, PartitioningMode.ROW_WISE, PartitioningMode.COLUMNAR); + } + + @Test + public void testMemoryReleased() { - testOutputEqualsInput(type, PartitioningMode.COLUMNAR, PartitioningMode.ROW_WISE); - testOutputEqualsInput(type, PartitioningMode.ROW_WISE, PartitioningMode.COLUMNAR); + testMemoryReleased(PartitioningMode.ROW_WISE); + testMemoryReleased(PartitioningMode.COLUMNAR); } - @Test(dataProvider = "partitioningMode") - public void testMemoryReleased(PartitioningMode partitioningMode) + private void testMemoryReleased(PartitioningMode partitioningMode) { AggregatedMemoryContext memoryContext = newSimpleAggregatedMemoryContext(); - PagePartitioner pagePartitioner = pagePartitioner(BIGINT).withMemoryContext(memoryContext).build(); + TestOutputBuffer outputBuffer = new TestOutputBuffer(); + PagePartitioner pagePartitioner = pagePartitioner(outputBuffer, BIGINT).withMemoryContext(memoryContext).build(); Page page = new Page(createLongsBlock(0L, 1L, 2L, 3L, null)); processPages(pagePartitioner, partitioningMode, page); @@ -386,13 +547,20 @@ public void testMemoryReleased(PartitioningMode partitioningMode) assertThat(memoryContext.getBytes()).isEqualTo(0); } - @Test(dataProvider = "partitioningMode") - public void testMemoryReleasedOnFailure(PartitioningMode partitioningMode) + @Test + public void testMemoryReleasedOnFailure() + { + testMemoryReleasedOnFailure(PartitioningMode.ROW_WISE); + testMemoryReleasedOnFailure(PartitioningMode.COLUMNAR); + } + + private void testMemoryReleasedOnFailure(PartitioningMode partitioningMode) { AggregatedMemoryContext memoryContext = newSimpleAggregatedMemoryContext(); RuntimeException exception = new RuntimeException(); + TestOutputBuffer outputBuffer = new TestOutputBuffer(); outputBuffer.throwOnEnqueue(exception); - PagePartitioner pagePartitioner = pagePartitioner(BIGINT).withMemoryContext(memoryContext).build(); + PagePartitioner pagePartitioner = pagePartitioner(outputBuffer, BIGINT).withMemoryContext(memoryContext).build(); Page page = new Page(createLongsBlock(0L, 1L, 2L, 3L, null)); partitioningMode.partitionPage(pagePartitioner, page); @@ -403,7 +571,8 @@ public void testMemoryReleasedOnFailure(PartitioningMode partitioningMode) private void testOutputEqualsInput(Type type, PartitioningMode mode1, PartitioningMode mode2) { - PagePartitionerBuilder pagePartitionerBuilder = pagePartitioner(BIGINT, type, type); + TestOutputBuffer outputBuffer = new TestOutputBuffer(); + PagePartitionerBuilder pagePartitionerBuilder = pagePartitioner(outputBuffer, BIGINT, type, type); PagePartitioner pagePartitioner = pagePartitionerBuilder.build(); Page input = new Page( createLongSequenceBlock(0, POSITIONS_PER_PAGE), // partition block @@ -422,48 +591,6 @@ private void testOutputEqualsInput(Type type, PartitioningMode mode1, Partitioni outputBuffer.clear(); } - @DataProvider(name = "partitioningMode") - public static Object[][] partitioningMode() - { - return new Object[][] {{PartitioningMode.ROW_WISE}, {PartitioningMode.COLUMNAR}}; - } - - @DataProvider(name = "types") - public static Object[][] types() - { - return getTypes().stream().map(type -> new Object[] {type}).toArray(Object[][]::new); - } - - @DataProvider(name = "typesWithPartitioningMode") - public static Object[][] typesWithPartitioningMode() - { - return getTypes().stream() - .flatMap(type -> Stream.of(PartitioningMode.values()) - .map(partitioningMode -> new Object[] {type, partitioningMode})) - .toArray(Object[][]::new); - } - - private static ImmutableList getTypes() - { - return ImmutableList.of( - BIGINT, - BOOLEAN, - INTEGER, - createCharType(10), - createUnboundedVarcharType(), - DOUBLE, - SMALLINT, - TINYINT, - UUID, - VARBINARY, - createDecimalType(1), - createDecimalType(Decimals.MAX_SHORT_PRECISION + 1), - new ArrayType(BIGINT), - TimestampType.createTimestampType(9), - TimestampType.createTimestampType(3), - IPADDRESS); - } - private static Block createBlockForType(Type type, int positionsPerPage) { return createRandomBlockForType(type, positionsPerPage, 0.2F); @@ -500,17 +627,17 @@ private static List readChannel(Stream pages, int channel, Type ty return unmodifiableList(result); } - private PagePartitionerBuilder pagePartitioner(Type... types) + private PagePartitionerBuilder pagePartitioner(TestOutputBuffer outputBuffer, Type... types) { - return pagePartitioner(ImmutableList.copyOf(types)); + return pagePartitioner(ImmutableList.copyOf(types), outputBuffer); } - private PagePartitionerBuilder pagePartitioner(List types) + private PagePartitionerBuilder pagePartitioner(List types, TestOutputBuffer outputBuffer) { - return pagePartitioner().withTypes(types); + return pagePartitioner(outputBuffer).withTypes(types); } - private PagePartitionerBuilder pagePartitioner() + private PagePartitionerBuilder pagePartitioner(TestOutputBuffer outputBuffer) { return new PagePartitionerBuilder(executor, scheduledExecutor, outputBuffer); } diff --git a/core/trino-main/src/test/java/io/trino/operator/output/TestPositionsAppender.java b/core/trino-main/src/test/java/io/trino/operator/output/TestPositionsAppender.java index a41bd26a53341..b320ccac48291 100644 --- a/core/trino-main/src/test/java/io/trino/operator/output/TestPositionsAppender.java +++ b/core/trino-main/src/test/java/io/trino/operator/output/TestPositionsAppender.java @@ -40,12 +40,9 @@ import io.trino.spi.type.VarbinaryType; import io.trino.spi.type.VarcharType; import io.trino.type.BlockTypeOperators; -import io.trino.type.UnknownType; import it.unimi.dsi.fastutil.ints.IntArrayList; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; -import java.util.Arrays; import java.util.List; import java.util.Optional; import java.util.function.Function; @@ -83,72 +80,97 @@ public class TestPositionsAppender { private static final PositionsAppenderFactory POSITIONS_APPENDER_FACTORY = new PositionsAppenderFactory(new BlockTypeOperators()); - @Test(dataProvider = "types") - public void testMixedBlockTypes(TestType type) - { - List input = ImmutableList.of( - input(emptyBlock(type)), - input(nullBlock(type, 3), 0, 2), - input(notNullBlock(type, 3), 1, 2), - input(partiallyNullBlock(type, 4), 0, 1, 2, 3), - input(partiallyNullBlock(type, 4)), // empty position list - input(rleBlock(type, 4), 0, 2), - input(rleBlock(type, 2), 0, 1), // rle all positions - input(nullRleBlock(type, 4), 1, 2), - input(dictionaryBlock(type, 4, 2, 0), 0, 3), // dict not null - input(dictionaryBlock(type, 8, 4, 0.5F), 1, 3, 5), // dict mixed - input(dictionaryBlock(type, 8, 4, 1), 1, 3, 5), // dict null - input(rleBlock(dictionaryBlock(type, 1, 2, 0), 3), 2), // rle -> dict - input(rleBlock(dictionaryBlock(notNullBlock(type, 2), new int[] {1}), 3), 2), // rle -> dict with position 0 mapped to > 0 - input(rleBlock(dictionaryBlock(rleBlock(type, 4), 1), 3), 1), // rle -> dict -> rle - input(dictionaryBlock(dictionaryBlock(type, 5, 4, 0.5F), 3), 2), // dict -> dict - input(dictionaryBlock(dictionaryBlock(dictionaryBlock(type, 5, 4, 0.5F), 3), 3), 2), // dict -> dict -> dict - input(dictionaryBlock(rleBlock(type, 4), 3), 0, 2), // dict -> rle - input(notNullBlock(type, 4).getRegion(2, 2), 0, 1), // not null block with offset - input(partiallyNullBlock(type, 4).getRegion(2, 2), 0, 1), // nullable block with offset - input(rleBlock(notNullBlock(type, 4).getRegion(2, 1), 3), 1)); // rle block with offset - - testAppend(type, input); + @Test + public void testMixedBlockTypes() + { + for (TestType type : TestType.values()) { + List input = ImmutableList.of( + input(emptyBlock(type)), + input(nullBlock(type, 3), 0, 2), + input(notNullBlock(type, 3), 1, 2), + input(partiallyNullBlock(type, 4), 0, 1, 2, 3), + input(partiallyNullBlock(type, 4)), // empty position list + input(rleBlock(type, 4), 0, 2), + input(rleBlock(type, 2), 0, 1), // rle all positions + input(nullRleBlock(type, 4), 1, 2), + input(dictionaryBlock(type, 4, 2, 0), 0, 3), // dict not null + input(dictionaryBlock(type, 8, 4, 0.5F), 1, 3, 5), // dict mixed + input(dictionaryBlock(type, 8, 4, 1), 1, 3, 5), // dict null + input(rleBlock(dictionaryBlock(type, 1, 2, 0), 3), 2), // rle -> dict + input(rleBlock(dictionaryBlock(notNullBlock(type, 2), new int[] {1}), 3), 2), // rle -> dict with position 0 mapped to > 0 + input(rleBlock(dictionaryBlock(rleBlock(type, 4), 1), 3), 1), // rle -> dict -> rle + input(dictionaryBlock(dictionaryBlock(type, 5, 4, 0.5F), 3), 2), // dict -> dict + input(dictionaryBlock(dictionaryBlock(dictionaryBlock(type, 5, 4, 0.5F), 3), 3), 2), // dict -> dict -> dict + input(dictionaryBlock(rleBlock(type, 4), 3), 0, 2), // dict -> rle + input(notNullBlock(type, 4).getRegion(2, 2), 0, 1), // not null block with offset + input(partiallyNullBlock(type, 4).getRegion(2, 2), 0, 1), // nullable block with offset + input(rleBlock(notNullBlock(type, 4).getRegion(2, 1), 3), 1)); // rle block with offset + + testAppend(type, input); + } } - @Test(dataProvider = "types") - public void testNullRle(TestType type) + @Test + public void testNullRle() { - testNullRle(type.getType(), nullBlock(type, 2)); - testNullRle(type.getType(), nullRleBlock(type, 2)); - testNullRle(type.getType(), createRandomBlockForType(type, 4, 0.5f)); + for (TestType type : TestType.values()) { + testNullRle(type.getType(), nullBlock(type, 2)); + testNullRle(type.getType(), nullRleBlock(type, 2)); + testNullRle(type.getType(), createRandomBlockForType(type, 4, 0.5f)); + } } - @Test(dataProvider = "types") - public void testRleSwitchToFlat(TestType type) - { - List inputs = ImmutableList.of( - input(rleBlock(type, 3), 0, 1), - input(notNullBlock(type, 2), 0, 1)); - testAppend(type, inputs); + @Test + public void testRleSwitchToFlat() + { + for (TestType type : TestType.values()) { + List inputs = ImmutableList.of( + input(rleBlock(type, 3), 0, 1), + input(notNullBlock(type, 2), 0, 1)); + testAppend(type, inputs); + + List dictionaryInputs = ImmutableList.of( + input(rleBlock(type, 3), 0, 1), + input(dictionaryBlock(type, 2, 4, 0), 0, 1)); + testAppend(type, dictionaryInputs); + } + } - List dictionaryInputs = ImmutableList.of( - input(rleBlock(type, 3), 0, 1), - input(dictionaryBlock(type, 2, 4, 0), 0, 1)); - testAppend(type, dictionaryInputs); + @Test + public void testFlatAppendRle() + { + for (TestType type : TestType.values()) { + List inputs = ImmutableList.of( + input(notNullBlock(type, 2), 0, 1), + input(rleBlock(type, 3), 0, 1)); + testAppend(type, inputs); + + List dictionaryInputs = ImmutableList.of( + input(dictionaryBlock(type, 2, 4, 0), 0, 1), + input(rleBlock(type, 3), 0, 1)); + testAppend(type, dictionaryInputs); + } } - @Test(dataProvider = "types") - public void testFlatAppendRle(TestType type) + @Test + public void testMultipleRleBlocksWithDifferentValues() { - List inputs = ImmutableList.of( - input(notNullBlock(type, 2), 0, 1), - input(rleBlock(type, 3), 0, 1)); - testAppend(type, inputs); - - List dictionaryInputs = ImmutableList.of( - input(dictionaryBlock(type, 2, 4, 0), 0, 1), - input(rleBlock(type, 3), 0, 1)); - testAppend(type, dictionaryInputs); + testMultipleRleBlocksWithDifferentValues(TestType.BIGINT, createLongsBlock(0), createLongsBlock(1)); + testMultipleRleBlocksWithDifferentValues(TestType.BOOLEAN, createBooleansBlock(true), createBooleansBlock(false)); + testMultipleRleBlocksWithDifferentValues(TestType.INTEGER, createIntsBlock(0), createIntsBlock(1)); + testMultipleRleBlocksWithDifferentValues(TestType.CHAR_10, createStringsBlock("0"), createStringsBlock("1")); + testMultipleRleBlocksWithDifferentValues(TestType.VARCHAR, createStringsBlock("0"), createStringsBlock("1")); + testMultipleRleBlocksWithDifferentValues(TestType.DOUBLE, createDoublesBlock(0.0), createDoublesBlock(1.0)); + testMultipleRleBlocksWithDifferentValues(TestType.SMALLINT, createSmallintsBlock(0), createSmallintsBlock(1)); + testMultipleRleBlocksWithDifferentValues(TestType.TINYINT, createTinyintsBlock(0), createTinyintsBlock(1)); + testMultipleRleBlocksWithDifferentValues(TestType.VARBINARY, createSlicesBlock(Slices.allocate(Long.BYTES)), createSlicesBlock(Slices.allocate(Long.BYTES).getOutput().appendLong(1).slice())); + testMultipleRleBlocksWithDifferentValues(TestType.LONG_DECIMAL, createLongDecimalsBlock("0"), createLongDecimalsBlock("1")); + testMultipleRleBlocksWithDifferentValues(TestType.ARRAY_BIGINT, createArrayBigintBlock(ImmutableList.of(ImmutableList.of(0L))), createArrayBigintBlock(ImmutableList.of(ImmutableList.of(1L)))); + testMultipleRleBlocksWithDifferentValues(TestType.LONG_TIMESTAMP, createLongTimestampBlock(createTimestampType(9), new LongTimestamp(0, 0)), createLongTimestampBlock(createTimestampType(9), new LongTimestamp(1, 0))); + testMultipleRleBlocksWithDifferentValues(TestType.VARCHAR_WITH_TEST_BLOCK, adapt(createStringsBlock("0")), adapt(createStringsBlock("1"))); } - @Test(dataProvider = "differentValues") - public void testMultipleRleBlocksWithDifferentValues(TestType type, Block value1, Block value2) + private void testMultipleRleBlocksWithDifferentValues(TestType type, Block value1, Block value2) { List input = ImmutableList.of( input(rleBlock(value1, 3), 0, 1), @@ -156,44 +178,30 @@ public void testMultipleRleBlocksWithDifferentValues(TestType type, Block value1 testAppend(type, input); } - @DataProvider(name = "differentValues") - public static Object[][] differentValues() + @Test + public void testMultipleRleWithTheSameValueProduceRle() { - return new Object[][] - { - {TestType.BIGINT, createLongsBlock(0), createLongsBlock(1)}, - {TestType.BOOLEAN, createBooleansBlock(true), createBooleansBlock(false)}, - {TestType.INTEGER, createIntsBlock(0), createIntsBlock(1)}, - {TestType.CHAR_10, createStringsBlock("0"), createStringsBlock("1")}, - {TestType.VARCHAR, createStringsBlock("0"), createStringsBlock("1")}, - {TestType.DOUBLE, createDoublesBlock(0.0), createDoublesBlock(1.0)}, - {TestType.SMALLINT, createSmallintsBlock(0), createSmallintsBlock(1)}, - {TestType.TINYINT, createTinyintsBlock(0), createTinyintsBlock(1)}, - {TestType.VARBINARY, createSlicesBlock(Slices.allocate(Long.BYTES)), createSlicesBlock(Slices.allocate(Long.BYTES).getOutput().appendLong(1).slice())}, - {TestType.LONG_DECIMAL, createLongDecimalsBlock("0"), createLongDecimalsBlock("1")}, - {TestType.ARRAY_BIGINT, createArrayBigintBlock(ImmutableList.of(ImmutableList.of(0L))), createArrayBigintBlock(ImmutableList.of(ImmutableList.of(1L)))}, - {TestType.LONG_TIMESTAMP, createLongTimestampBlock(createTimestampType(9), new LongTimestamp(0, 0)), - createLongTimestampBlock(createTimestampType(9), new LongTimestamp(1, 0))}, - {TestType.VARCHAR_WITH_TEST_BLOCK, adapt(createStringsBlock("0")), adapt(createStringsBlock("1"))} - }; - } + for (TestType type : TestType.values()) { + UnnestingPositionsAppender positionsAppender = POSITIONS_APPENDER_FACTORY.create(type.getType(), 10, DEFAULT_MAX_PAGE_SIZE_IN_BYTES); - @Test(dataProvider = "types") - public void testMultipleRleWithTheSameValueProduceRle(TestType type) - { - UnnestingPositionsAppender positionsAppender = POSITIONS_APPENDER_FACTORY.create(type.getType(), 10, DEFAULT_MAX_PAGE_SIZE_IN_BYTES); + Block value = notNullBlock(type, 1); + positionsAppender.append(allPositions(3), rleBlock(value, 3)); + positionsAppender.append(allPositions(2), rleBlock(value, 2)); - Block value = notNullBlock(type, 1); - positionsAppender.append(allPositions(3), rleBlock(value, 3)); - positionsAppender.append(allPositions(2), rleBlock(value, 2)); + Block actual = positionsAppender.build(); + assertThat(actual.getPositionCount()).isEqualTo(5); + assertInstanceOf(actual, RunLengthEncodedBlock.class); + } + } - Block actual = positionsAppender.build(); - assertThat(actual.getPositionCount()).isEqualTo(5); - assertInstanceOf(actual, RunLengthEncodedBlock.class); + @Test + public void testRleAppendForComplexTypeWithNullElement() + { + testRleAppendForComplexTypeWithNullElement(TestType.ROW_BIGINT_VARCHAR, RowBlock.fromFieldBlocks(1, new Block[] {nullBlock(BIGINT, 1), nullBlock(VARCHAR, 1)})); + testRleAppendForComplexTypeWithNullElement(TestType.ARRAY_BIGINT, ArrayBlock.fromElementBlock(1, Optional.empty(), new int[] {0, 1}, nullBlock(BIGINT, 1))); } - @Test(dataProvider = "complexTypesWithNullElementBlock") - public void testRleAppendForComplexTypeWithNullElement(TestType type, Block value) + private void testRleAppendForComplexTypeWithNullElement(TestType type, Block value) { checkArgument(value.getPositionCount() == 1); UnnestingPositionsAppender positionsAppender = POSITIONS_APPENDER_FACTORY.create(type.getType(), 10, DEFAULT_MAX_PAGE_SIZE_IN_BYTES); @@ -207,31 +215,35 @@ public void testRleAppendForComplexTypeWithNullElement(TestType type, Block valu assertBlockEquals(type.getType(), actual, RunLengthEncodedBlock.create(value, 5)); } - @Test(dataProvider = "types") - public void testRleAppendedWithSinglePositionDoesNotProduceRle(TestType type) + @Test + public void testRleAppendedWithSinglePositionDoesNotProduceRle() { - UnnestingPositionsAppender positionsAppender = POSITIONS_APPENDER_FACTORY.create(type.getType(), 10, DEFAULT_MAX_PAGE_SIZE_IN_BYTES); + for (TestType type : TestType.values()) { + UnnestingPositionsAppender positionsAppender = POSITIONS_APPENDER_FACTORY.create(type.getType(), 10, DEFAULT_MAX_PAGE_SIZE_IN_BYTES); - Block value = notNullBlock(type, 1); - positionsAppender.append(allPositions(3), rleBlock(value, 3)); - positionsAppender.append(allPositions(2), rleBlock(value, 2)); - positionsAppender.append(0, rleBlock(value, 2)); + Block value = notNullBlock(type, 1); + positionsAppender.append(allPositions(3), rleBlock(value, 3)); + positionsAppender.append(allPositions(2), rleBlock(value, 2)); + positionsAppender.append(0, rleBlock(value, 2)); - Block actual = positionsAppender.build(); - assertThat(actual.getPositionCount()).isEqualTo(6); - assertThat(actual instanceof RunLengthEncodedBlock) - .describedAs(actual.getClass().getSimpleName()) - .isFalse(); + Block actual = positionsAppender.build(); + assertThat(actual.getPositionCount()).isEqualTo(6); + assertThat(actual instanceof RunLengthEncodedBlock) + .describedAs(actual.getClass().getSimpleName()) + .isFalse(); + } } - @Test(dataProvider = "types") - public static void testMultipleTheSameDictionariesProduceDictionary(TestType type) + @Test + public void testMultipleTheSameDictionariesProduceDictionary() { - UnnestingPositionsAppender positionsAppender = POSITIONS_APPENDER_FACTORY.create(type.getType(), 10, DEFAULT_MAX_PAGE_SIZE_IN_BYTES); + for (TestType type : TestType.values()) { + UnnestingPositionsAppender positionsAppender = POSITIONS_APPENDER_FACTORY.create(type.getType(), 10, DEFAULT_MAX_PAGE_SIZE_IN_BYTES); - testMultipleTheSameDictionariesProduceDictionary(type, positionsAppender); - // test if appender can accept different dictionary after a build - testMultipleTheSameDictionariesProduceDictionary(type, positionsAppender); + testMultipleTheSameDictionariesProduceDictionary(type, positionsAppender); + // test if appender can accept different dictionary after a build + testMultipleTheSameDictionariesProduceDictionary(type, positionsAppender); + } } private static void testMultipleTheSameDictionariesProduceDictionary(TestType type, UnnestingPositionsAppender positionsAppender) @@ -246,85 +258,95 @@ private static void testMultipleTheSameDictionariesProduceDictionary(TestType ty assertThat(((DictionaryBlock) actual).getDictionary()).isEqualTo(dictionary); } - @Test(dataProvider = "types") - public void testDictionarySwitchToFlat(TestType type) + @Test + public void testDictionarySwitchToFlat() { - List inputs = ImmutableList.of( - input(dictionaryBlock(type, 3, 4, 0), 0, 1), - input(notNullBlock(type, 2), 0, 1)); - testAppend(type, inputs); + for (TestType type : TestType.values()) { + List inputs = ImmutableList.of( + input(dictionaryBlock(type, 3, 4, 0), 0, 1), + input(notNullBlock(type, 2), 0, 1)); + testAppend(type, inputs); + } } - @Test(dataProvider = "types") - public void testFlatAppendDictionary(TestType type) + @Test + public void testFlatAppendDictionary() { - List inputs = ImmutableList.of( - input(notNullBlock(type, 2), 0, 1), - input(dictionaryBlock(type, 3, 4, 0), 0, 1)); - testAppend(type, inputs); + for (TestType type : TestType.values()) { + List inputs = ImmutableList.of( + input(notNullBlock(type, 2), 0, 1), + input(dictionaryBlock(type, 3, 4, 0), 0, 1)); + testAppend(type, inputs); + } } - @Test(dataProvider = "types") - public void testDictionaryAppendDifferentDictionary(TestType type) + @Test + public void testDictionaryAppendDifferentDictionary() { - List dictionaryInputs = ImmutableList.of( - input(dictionaryBlock(type, 3, 4, 0), 0, 1), - input(dictionaryBlock(type, 2, 4, 0), 0, 1)); - testAppend(type, dictionaryInputs); + for (TestType type : TestType.values()) { + List dictionaryInputs = ImmutableList.of( + input(dictionaryBlock(type, 3, 4, 0), 0, 1), + input(dictionaryBlock(type, 2, 4, 0), 0, 1)); + testAppend(type, dictionaryInputs); + } } - @Test(dataProvider = "types") - public void testDictionarySingleThenFlat(TestType type) + @Test + public void testDictionarySingleThenFlat() { - BlockView firstInput = input(dictionaryBlock(type, 1, 4, 0), 0); - BlockView secondInput = input(dictionaryBlock(type, 2, 4, 0), 0, 1); - UnnestingPositionsAppender positionsAppender = POSITIONS_APPENDER_FACTORY.create(type.getType(), 10, DEFAULT_MAX_PAGE_SIZE_IN_BYTES); - long initialRetainedSize = positionsAppender.getRetainedSizeInBytes(); + for (TestType type : TestType.values()) { + BlockView firstInput = input(dictionaryBlock(type, 1, 4, 0), 0); + BlockView secondInput = input(dictionaryBlock(type, 2, 4, 0), 0, 1); + UnnestingPositionsAppender positionsAppender = POSITIONS_APPENDER_FACTORY.create(type.getType(), 10, DEFAULT_MAX_PAGE_SIZE_IN_BYTES); + long initialRetainedSize = positionsAppender.getRetainedSizeInBytes(); - firstInput.positions().forEach((int position) -> positionsAppender.append(position, firstInput.block())); - positionsAppender.append(secondInput.positions(), secondInput.block()); + firstInput.positions().forEach((int position) -> positionsAppender.append(position, firstInput.block())); + positionsAppender.append(secondInput.positions(), secondInput.block()); - assertBuildResult(type, ImmutableList.of(firstInput, secondInput), positionsAppender, initialRetainedSize); + assertBuildResult(type, ImmutableList.of(firstInput, secondInput), positionsAppender, initialRetainedSize); + } } - @Test(dataProvider = "types") - public void testConsecutiveBuilds(TestType type) - { - UnnestingPositionsAppender positionsAppender = POSITIONS_APPENDER_FACTORY.create(type.getType(), 10, DEFAULT_MAX_PAGE_SIZE_IN_BYTES); - - // empty block - positionsAppender.append(positions(), emptyBlock(type)); - assertThat(positionsAppender.build().getPositionCount()).isEqualTo(0); - - Block block = createRandomBlockForType(type, 2, 0.5f); - // append only null position - int nullPosition = block.isNull(0) ? 0 : 1; - positionsAppender.append(positions(nullPosition), block); - Block actualNullBlock = positionsAppender.build(); - assertThat(actualNullBlock.getPositionCount()).isEqualTo(1); - assertThat(actualNullBlock.isNull(0)).isTrue(); - - // append null and not null position - positionsAppender.append(allPositions(2), block); - assertBlockEquals(type.getType(), positionsAppender.build(), block); - - // append not null rle - Block rleBlock = rleBlock(type, 10); - positionsAppender.append(allPositions(10), rleBlock); - assertBlockEquals(type.getType(), positionsAppender.build(), rleBlock); - - // append null rle - Block nullRleBlock = nullRleBlock(type, 10); - positionsAppender.append(allPositions(10), nullRleBlock); - assertBlockEquals(type.getType(), positionsAppender.build(), nullRleBlock); - - // append dictionary - Block dictionaryBlock = dictionaryBlock(type, 10, 5, 0); - positionsAppender.append(allPositions(10), dictionaryBlock); - assertBlockEquals(type.getType(), positionsAppender.build(), dictionaryBlock); - - // just build to confirm appender was reset - assertThat(positionsAppender.build().getPositionCount()).isEqualTo(0); + @Test + public void testConsecutiveBuilds() + { + for (TestType type : TestType.values()) { + UnnestingPositionsAppender positionsAppender = POSITIONS_APPENDER_FACTORY.create(type.getType(), 10, DEFAULT_MAX_PAGE_SIZE_IN_BYTES); + + // empty block + positionsAppender.append(positions(), emptyBlock(type)); + assertThat(positionsAppender.build().getPositionCount()).isEqualTo(0); + + Block block = createRandomBlockForType(type, 2, 0.5f); + // append only null position + int nullPosition = block.isNull(0) ? 0 : 1; + positionsAppender.append(positions(nullPosition), block); + Block actualNullBlock = positionsAppender.build(); + assertThat(actualNullBlock.getPositionCount()).isEqualTo(1); + assertThat(actualNullBlock.isNull(0)).isTrue(); + + // append null and not null position + positionsAppender.append(allPositions(2), block); + assertBlockEquals(type.getType(), positionsAppender.build(), block); + + // append not null rle + Block rleBlock = rleBlock(type, 10); + positionsAppender.append(allPositions(10), rleBlock); + assertBlockEquals(type.getType(), positionsAppender.build(), rleBlock); + + // append null rle + Block nullRleBlock = nullRleBlock(type, 10); + positionsAppender.append(allPositions(10), nullRleBlock); + assertBlockEquals(type.getType(), positionsAppender.build(), nullRleBlock); + + // append dictionary + Block dictionaryBlock = dictionaryBlock(type, 10, 5, 0); + positionsAppender.append(allPositions(10), dictionaryBlock); + assertBlockEquals(type.getType(), positionsAppender.build(), dictionaryBlock); + + // just build to confirm appender was reset + assertThat(positionsAppender.build().getPositionCount()).isEqualTo(0); + } } // testcase for jit bug described https://github.com/trinodb/trino/issues/12821. @@ -362,23 +384,6 @@ public void testRowWithNestedFields() assertBlockEquals(type, actual, rowBLock); } - @DataProvider(name = "complexTypesWithNullElementBlock") - public static Object[][] complexTypesWithNullElementBlock() - { - return new Object[][] { - {TestType.ROW_BIGINT_VARCHAR, RowBlock.fromFieldBlocks(1, new Block[] {nullBlock(BIGINT, 1), nullBlock(VARCHAR, 1)})}, - {TestType.ARRAY_BIGINT, ArrayBlock.fromElementBlock(1, Optional.empty(), new int[] {0, 1}, nullBlock(BIGINT, 1))}}; - } - - @DataProvider(name = "types") - public static Object[][] types() - { - return Arrays.stream(TestType.values()) - .filter(testType -> testType != TestType.UNKNOWN) - .map(type -> new Object[] {type}) - .toArray(Object[][]::new); - } - private static ValueBlock singleValueBlock(String value) { BlockBuilder blockBuilder = VARCHAR.createBlockBuilder(null, 1); @@ -577,8 +582,8 @@ private enum TestType LONG_TIMESTAMP(createTimestampType(9)), ROW_BIGINT_VARCHAR(anonymousRow(BigintType.BIGINT, VarcharType.VARCHAR)), ARRAY_BIGINT(new ArrayType(BigintType.BIGINT)), - VARCHAR_WITH_TEST_BLOCK(VarcharType.VARCHAR, adaptation()), - UNKNOWN(UnknownType.UNKNOWN); + VARCHAR_WITH_TEST_BLOCK(VarcharType.VARCHAR, adaptation()); +// UNKNOWN(UnknownType.UNKNOWN); private final Type type; private final Function blockAdaptation; diff --git a/core/trino-main/src/test/java/io/trino/operator/project/TestDictionaryAwarePageProjection.java b/core/trino-main/src/test/java/io/trino/operator/project/TestDictionaryAwarePageProjection.java index bdcc3daca7a7a..a42bcaaa6e617 100644 --- a/core/trino-main/src/test/java/io/trino/operator/project/TestDictionaryAwarePageProjection.java +++ b/core/trino-main/src/test/java/io/trino/operator/project/TestDictionaryAwarePageProjection.java @@ -23,10 +23,10 @@ import io.trino.spi.block.LazyBlock; import io.trino.spi.block.LongArrayBlock; import io.trino.spi.block.RunLengthEncodedBlock; +import io.trino.spi.block.ValueBlock; import io.trino.spi.connector.ConnectorSession; import io.trino.spi.type.Type; import org.testng.annotations.AfterClass; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import java.util.Arrays; @@ -49,15 +49,6 @@ public class TestDictionaryAwarePageProjection { private static final ScheduledExecutorService executor = newSingleThreadScheduledExecutor(daemonThreadsNamed("TestDictionaryAwarePageProjection-%s")); - @DataProvider(name = "forceYield") - public static Object[][] forceYieldAndProduceLazyBlock() - { - return new Object[][] { - {true, false}, - {false, true}, - {false, false}}; - } - @AfterClass(alwaysRun = true) public void tearDown() { @@ -73,54 +64,66 @@ public void testDelegateMethods() assertThat(projection.getType()).isEqualTo(BIGINT); } - @Test(dataProvider = "forceYield") - public void testSimpleBlock(boolean forceYield, boolean produceLazyBlock) + @Test + public void testSimpleBlock() { - Block block = createLongSequenceBlock(0, 100); - testProject(block, block.getClass(), forceYield, produceLazyBlock); + ValueBlock block = createLongSequenceBlock(0, 100); + testProject(block, block.getClass(), true, false); + testProject(block, block.getClass(), false, true); + testProject(block, block.getClass(), false, false); } - @Test(dataProvider = "forceYield") - public void testRleBlock(boolean forceYield, boolean produceLazyBlock) + @Test + public void testRleBlock() { Block value = createLongSequenceBlock(42, 43); RunLengthEncodedBlock block = (RunLengthEncodedBlock) RunLengthEncodedBlock.create(value, 100); - testProject(block, RunLengthEncodedBlock.class, forceYield, produceLazyBlock); + testProject(block, RunLengthEncodedBlock.class, true, false); + testProject(block, RunLengthEncodedBlock.class, false, true); + testProject(block, RunLengthEncodedBlock.class, false, false); } - @Test(dataProvider = "forceYield") - public void testRleBlockWithFailure(boolean forceYield, boolean produceLazyBlock) + @Test + public void testRleBlockWithFailure() { Block value = createLongSequenceBlock(-43, -42); RunLengthEncodedBlock block = (RunLengthEncodedBlock) RunLengthEncodedBlock.create(value, 100); - testProjectFails(block, RunLengthEncodedBlock.class, forceYield, produceLazyBlock); + testProjectFails(block, RunLengthEncodedBlock.class, true, false); + testProjectFails(block, RunLengthEncodedBlock.class, false, true); + testProjectFails(block, RunLengthEncodedBlock.class, false, false); } - @Test(dataProvider = "forceYield") - public void testDictionaryBlock(boolean forceYield, boolean produceLazyBlock) + @Test + public void testDictionaryBlock() { Block block = createDictionaryBlock(10, 100); - testProject(block, DictionaryBlock.class, forceYield, produceLazyBlock); + testProject(block, DictionaryBlock.class, true, false); + testProject(block, DictionaryBlock.class, false, true); + testProject(block, DictionaryBlock.class, false, false); } - @Test(dataProvider = "forceYield") - public void testDictionaryBlockWithFailure(boolean forceYield, boolean produceLazyBlock) + @Test + public void testDictionaryBlockWithFailure() { Block block = createDictionaryBlockWithFailure(10, 100); - testProjectFails(block, DictionaryBlock.class, forceYield, produceLazyBlock); + testProjectFails(block, DictionaryBlock.class, true, false); + testProjectFails(block, DictionaryBlock.class, false, true); + testProjectFails(block, DictionaryBlock.class, false, false); } - @Test(dataProvider = "forceYield") - public void testDictionaryBlockProcessingWithUnusedFailure(boolean forceYield, boolean produceLazyBlock) + @Test + public void testDictionaryBlockProcessingWithUnusedFailure() { Block block = createDictionaryBlockWithUnusedEntries(10, 100); // failures in the dictionary processing will cause a fallback to normal columnar processing - testProject(block, LongArrayBlock.class, forceYield, produceLazyBlock); + testProject(block, LongArrayBlock.class, true, false); + testProject(block, LongArrayBlock.class, false, true); + testProject(block, LongArrayBlock.class, false, false); } @Test @@ -136,8 +139,15 @@ public void testDictionaryProcessingIgnoreYield() testProjectFastReturnIgnoreYield(block, projection, false); } - @Test(dataProvider = "forceYield") - public void testDictionaryProcessingEnableDisable(boolean forceYield, boolean produceLazyBlock) + @Test + public void testDictionaryProcessingEnableDisable() + { + testDictionaryProcessingEnableDisable(true, false); + testDictionaryProcessingEnableDisable(false, true); + testDictionaryProcessingEnableDisable(false, false); + } + + private void testDictionaryProcessingEnableDisable(boolean forceYield, boolean produceLazyBlock) { DictionaryAwarePageProjection projection = createProjection(produceLazyBlock); diff --git a/core/trino-main/src/test/java/io/trino/server/security/TestResourceSecurity.java b/core/trino-main/src/test/java/io/trino/server/security/TestResourceSecurity.java index c1d7139e5e02d..cdb476f884d2d 100644 --- a/core/trino-main/src/test/java/io/trino/server/security/TestResourceSecurity.java +++ b/core/trino-main/src/test/java/io/trino/server/security/TestResourceSecurity.java @@ -59,7 +59,6 @@ import okhttp3.Request; import okhttp3.Response; import org.testng.annotations.BeforeClass; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import javax.crypto.SecretKey; @@ -784,8 +783,16 @@ public HttpCookie getNonceCookie() } } - @Test(dataProvider = "groups") - public void testOAuth2Groups(Optional> groups) + @Test + public void testOAuth2Groups() + throws Exception + { + testOAuth2Groups(Optional.empty()); + testOAuth2Groups(Optional.of(ImmutableSet.of())); + testOAuth2Groups(Optional.of(ImmutableSet.of("admin", "public"))); + } + + private void testOAuth2Groups(Optional> groups) throws Exception { try (TokenServer tokenServer = new TokenServer(Optional.empty()); @@ -855,18 +862,15 @@ public List loadForRequest(HttpUrl url) } } - @DataProvider(name = "groups") - public static Object[][] groups() + @Test + public void testJwtAndOAuth2AuthenticatorsSeparation() + throws Exception { - return new Object[][] { - {Optional.empty()}, - {Optional.of(ImmutableSet.of())}, - {Optional.of(ImmutableSet.of("admin", "public"))} - }; + testJwtAndOAuth2AuthenticatorsSeparation("jwt,oauth2"); + testJwtAndOAuth2AuthenticatorsSeparation("oauth2,jwt"); } - @Test(dataProvider = "authenticators") - public void testJwtAndOAuth2AuthenticatorsSeparation(String authenticators) + private void testJwtAndOAuth2AuthenticatorsSeparation(String authenticators) throws Exception { TestingHttpServer jwkServer = createTestingJwkServer(); @@ -914,15 +918,6 @@ public void testJwtAndOAuth2AuthenticatorsSeparation(String authenticators) } } - @DataProvider(name = "authenticators") - public static Object[][] authenticators() - { - return new Object[][] { - {"jwt,oauth2"}, - {"oauth2,jwt"} - }; - } - @Test public void testJwtWithRefreshTokensForOAuth2Enabled() throws Exception diff --git a/core/trino-main/src/test/java/io/trino/server/security/oauth2/TestJweTokenSerializer.java b/core/trino-main/src/test/java/io/trino/server/security/oauth2/TestJweTokenSerializer.java index adfeb76d2c9a5..01e3fb0be12c3 100644 --- a/core/trino-main/src/test/java/io/trino/server/security/oauth2/TestJweTokenSerializer.java +++ b/core/trino-main/src/test/java/io/trino/server/security/oauth2/TestJweTokenSerializer.java @@ -18,7 +18,6 @@ import io.jsonwebtoken.ExpiredJwtException; import io.jsonwebtoken.Jwts; import io.trino.server.security.oauth2.TokenPairSerializer.TokenPair; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import java.net.URI; @@ -60,24 +59,29 @@ public void testSerialization() assertThat(deserializedTokenPair.refreshToken()).isEqualTo(Optional.of("refresh_token")); } - @Test(dataProvider = "wrongSecretsProvider") - public void testDeserializationWithWrongSecret(String encryptionSecret, String decryptionSecret) + @Test + public void testDeserializationWithWrongSecret() { - assertThatThrownBy(() -> assertRoundTrip(Optional.ofNullable(encryptionSecret), Optional.ofNullable(decryptionSecret))) + assertThatThrownBy(() -> assertRoundTrip(Optional.of(randomEncodedSecret()), Optional.of(randomEncodedSecret()))) .isInstanceOf(RuntimeException.class) .hasMessageContaining("decryption failed: Tag mismatch"); - } - @DataProvider - public Object[][] wrongSecretsProvider() - { - return new Object[][]{ - {randomEncodedSecret(), randomEncodedSecret()}, - {randomEncodedSecret(16), randomEncodedSecret(24)}, - {null, null}, // This will generate two different secret keys - {null, randomEncodedSecret()}, - {randomEncodedSecret(), null} - }; + assertThatThrownBy(() -> assertRoundTrip(Optional.of(randomEncodedSecret(16)), Optional.of(randomEncodedSecret(24)))) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining("decryption failed: Tag mismatch"); + + // This will generate two different secret keys + assertThatThrownBy(() -> assertRoundTrip(Optional.empty(), Optional.empty())) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining("decryption failed: Tag mismatch"); + + assertThatThrownBy(() -> assertRoundTrip(Optional.empty(), Optional.of(randomEncodedSecret()))) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining("decryption failed: Tag mismatch"); + + assertThatThrownBy(() -> assertRoundTrip(Optional.of(randomEncodedSecret()), Optional.empty())) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining("decryption failed: Tag mismatch"); } @Test diff --git a/core/trino-main/src/test/java/io/trino/server/security/oauth2/TestOidcDiscovery.java b/core/trino-main/src/test/java/io/trino/server/security/oauth2/TestOidcDiscovery.java index bac63b99be72d..ee7d76dce6b26 100644 --- a/core/trino-main/src/test/java/io/trino/server/security/oauth2/TestOidcDiscovery.java +++ b/core/trino-main/src/test/java/io/trino/server/security/oauth2/TestOidcDiscovery.java @@ -29,7 +29,6 @@ import jakarta.servlet.http.HttpServlet; import jakarta.servlet.http.HttpServletRequest; import jakarta.servlet.http.HttpServletResponse; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import java.io.IOException; @@ -49,8 +48,15 @@ public class TestOidcDiscovery { - @Test(dataProvider = "staticConfiguration") - public void testStaticConfiguration(Optional accessTokenPath, Optional userinfoPath) + @Test + public void testStaticConfiguration() + throws Exception + { + testStaticConfiguration(Optional.empty(), Optional.empty()); + testStaticConfiguration(Optional.of("/access-token-issuer"), Optional.of("/userinfo")); + } + + private void testStaticConfiguration(Optional accessTokenPath, Optional userinfoPath) throws Exception { try (MetadataServer metadataServer = new MetadataServer(ImmutableMap.of("/jwks.json", "jwk/jwk-public.json"))) { @@ -72,17 +78,16 @@ public void testStaticConfiguration(Optional accessTokenPath, Optional accessTokenIssuer, Optional userinfoUrl) + private void testOidcDiscovery(String configuration, Optional accessTokenIssuer, Optional userinfoUrl) throws Exception { try (MetadataServer metadataServer = new MetadataServer( @@ -100,16 +105,6 @@ public void testOidcDiscovery(String configuration, Optional accessToken } } - @DataProvider(name = "oidcDiscovery") - public static Object[][] oidcDiscovery() - { - return new Object[][] { - {"openid-configuration.json", Optional.empty(), Optional.of("/connect/userinfo")}, - {"openid-configuration-without-userinfo.json", Optional.empty(), Optional.empty()}, - {"openid-configuration-with-access-token-issuer.json", Optional.of("http://access-token-issuer.com/adfs/services/trust"), Optional.of("/connect/userinfo")}, - }; - } - @Test public void testIssuerCheck() { diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/planprinter/TestCounterBasedAnonymizer.java b/core/trino-main/src/test/java/io/trino/sql/planner/planprinter/TestCounterBasedAnonymizer.java index d343eceee6236..bb7d8f8726949 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/planprinter/TestCounterBasedAnonymizer.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/planprinter/TestCounterBasedAnonymizer.java @@ -22,7 +22,6 @@ import io.trino.sql.tree.DoubleLiteral; import io.trino.sql.tree.GenericLiteral; import io.trino.sql.tree.IntervalLiteral; -import io.trino.sql.tree.Literal; import io.trino.sql.tree.LogicalExpression; import io.trino.sql.tree.LongLiteral; import io.trino.sql.tree.NullLiteral; @@ -30,7 +29,6 @@ import io.trino.sql.tree.SymbolReference; import io.trino.sql.tree.TimeLiteral; import io.trino.sql.tree.TimestampLiteral; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import java.util.Optional; @@ -64,34 +62,51 @@ public void testSymbolReferenceAnonymization() .isEqualTo("((\"symbol_1\" > 'long_literal_1') AND (\"symbol_2\" < 'long_literal_2') AND (\"symbol_3\" = 'long_literal_3'))"); } - @Test(dataProvider = "literals") - public void testLiteralAnonymization(Literal actual, String expected) + @Test + public void testLiteralAnonymization() { CounterBasedAnonymizer anonymizer = new CounterBasedAnonymizer(); - assertThat(anonymizer.anonymize(actual)).isEqualTo(expected); - } - @DataProvider - public static Object[][] literals() - { - return new Object[][] { - {new BinaryLiteral("DEF321"), "'binary_literal_1'"}, - {new StringLiteral("abc"), "'string_literal_1'"}, - {new GenericLiteral("bigint", "1"), "'bigint_literal_1'"}, - {new CharLiteral("a"), "'char_literal_1'"}, - {new DecimalLiteral("123"), "'decimal_literal_1'"}, - {new DoubleLiteral(String.valueOf(6554)), "'double_literal_1'"}, - {new DoubleLiteral(String.valueOf(Double.MAX_VALUE)), "'double_literal_1'"}, - {new LongLiteral(String.valueOf(6554)), "'long_literal_1'"}, - {new LongLiteral(String.valueOf(Long.MAX_VALUE)), "'long_literal_1'"}, - {new BooleanLiteral("true"), "true"}, - {new TimeLiteral("03:04:05"), "'time_literal_1'"}, - {new TimestampLiteral("2012-10-31 01:00 UTC"), "'timestamp_literal_1'"}, - {new NullLiteral(), "null"}, - { - new IntervalLiteral("33", IntervalLiteral.Sign.POSITIVE, IntervalLiteral.IntervalField.DAY, Optional.empty()), - "'interval_literal_1'" - } - }; + assertThat(anonymizer.anonymize(new BinaryLiteral("DEF321"))) + .isEqualTo("'binary_literal_1'"); + + assertThat(anonymizer.anonymize(new StringLiteral("abc"))) + .isEqualTo("'string_literal_1'"); + + assertThat(anonymizer.anonymize(new GenericLiteral("bigint", "1"))) + .isEqualTo("'bigint_literal_1'"); + + assertThat(anonymizer.anonymize(new CharLiteral("a"))) + .isEqualTo("'char_literal_1'"); + + assertThat(anonymizer.anonymize(new DecimalLiteral("123"))) + .isEqualTo("'decimal_literal_1'"); + + assertThat(anonymizer.anonymize(new DoubleLiteral(String.valueOf(6554)))) + .isEqualTo("'double_literal_1'"); + + assertThat(anonymizer.anonymize(new DoubleLiteral(String.valueOf(Double.MAX_VALUE)))) + .isEqualTo("'double_literal_1'"); + + assertThat(anonymizer.anonymize(new LongLiteral(String.valueOf(6554)))) + .isEqualTo("'long_literal_1'"); + + assertThat(anonymizer.anonymize(new LongLiteral(String.valueOf(Long.MAX_VALUE)))) + .isEqualTo("'long_literal_1'"); + + assertThat(anonymizer.anonymize(new BooleanLiteral("true"))) + .isEqualTo("true"); + + assertThat(anonymizer.anonymize(new TimeLiteral("03:04:05"))) + .isEqualTo("'time_literal_1'"); + + assertThat(anonymizer.anonymize(new TimestampLiteral("2012-10-31 01:00 UTC"))) + .isEqualTo("'timestamp_literal_1'"); + + assertThat(anonymizer.anonymize(new NullLiteral())) + .isEqualTo("null"); + + assertThat(anonymizer.anonymize(new IntervalLiteral("33", IntervalLiteral.Sign.POSITIVE, IntervalLiteral.IntervalField.DAY, Optional.empty()))) + .isEqualTo("'interval_literal_1'"); } } diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/sanity/TestValidateScaledWritersUsage.java b/core/trino-main/src/test/java/io/trino/sql/planner/sanity/TestValidateScaledWritersUsage.java index b980fe60c51eb..e5498743864cc 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/sanity/TestValidateScaledWritersUsage.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/sanity/TestValidateScaledWritersUsage.java @@ -41,7 +41,6 @@ import io.trino.testing.TestingTransactionHandle; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import java.util.Optional; @@ -59,6 +58,12 @@ public class TestValidateScaledWritersUsage extends BasePlanTest { + private static final PartitioningHandle CUSTOM_HANDLE = new PartitioningHandle( + Optional.of(TEST_CATALOG_HANDLE), + Optional.of(new ConnectorTransactionHandle() { }), + new ConnectorPartitioningHandle() { }, + true); + private LocalQueryRunner queryRunner; private PlannerContext plannerContext; private PlanBuilder planBuilder; @@ -104,8 +109,15 @@ private MockConnectorFactory createConnectorFactory(String name) .build(); } - @Test(dataProvider = "scaledWriterPartitioningHandles") - public void testScaledWritersUsedAndTargetSupportsIt(PartitioningHandle scaledWriterPartitionHandle) + @Test + public void testScaledWritersUsedAndTargetSupportsIt() + { + testScaledWritersUsedAndTargetSupportsIt(SCALED_WRITER_ROUND_ROBIN_DISTRIBUTION); + testScaledWritersUsedAndTargetSupportsIt(SCALED_WRITER_HASH_DISTRIBUTION); + testScaledWritersUsedAndTargetSupportsIt(CUSTOM_HANDLE); + } + + private void testScaledWritersUsedAndTargetSupportsIt(PartitioningHandle scaledWriterPartitionHandle) { PlanNode tableWriterSource = planBuilder.exchange(ex -> ex @@ -125,8 +137,15 @@ public void testScaledWritersUsedAndTargetSupportsIt(PartitioningHandle scaledWr validatePlan(root); } - @Test(dataProvider = "scaledWriterPartitioningHandles") - public void testScaledWritersUsedAndTargetDoesNotSupportScalingPerTask(PartitioningHandle scaledWriterPartitionHandle) + @Test + public void testScaledWritersUsedAndTargetDoesNotSupportScalingPerTask() + { + testScaledWritersUsedAndTargetDoesNotSupportScalingPerTask(SCALED_WRITER_ROUND_ROBIN_DISTRIBUTION); + testScaledWritersUsedAndTargetDoesNotSupportScalingPerTask(SCALED_WRITER_HASH_DISTRIBUTION); + testScaledWritersUsedAndTargetDoesNotSupportScalingPerTask(CUSTOM_HANDLE); + } + + private void testScaledWritersUsedAndTargetDoesNotSupportScalingPerTask(PartitioningHandle scaledWriterPartitionHandle) { PlanNode tableWriterSource = planBuilder.exchange(ex -> ex @@ -149,8 +168,15 @@ public void testScaledWritersUsedAndTargetDoesNotSupportScalingPerTask(Partition .hasMessage("The scaled writer per task partitioning scheme is set but writer target catalog:INSTANCE doesn't support it"); } - @Test(dataProvider = "scaledWriterPartitioningHandles") - public void testScaledWritersUsedAndTargetDoesNotSupportScalingAcrossTasks(PartitioningHandle scaledWriterPartitionHandle) + @Test + public void testScaledWritersUsedAndTargetDoesNotSupportScalingAcrossTasks() + { + testScaledWritersUsedAndTargetDoesNotSupportScalingAcrossTasks(SCALED_WRITER_ROUND_ROBIN_DISTRIBUTION); + testScaledWritersUsedAndTargetDoesNotSupportScalingAcrossTasks(SCALED_WRITER_HASH_DISTRIBUTION); + testScaledWritersUsedAndTargetDoesNotSupportScalingAcrossTasks(CUSTOM_HANDLE); + } + + private void testScaledWritersUsedAndTargetDoesNotSupportScalingAcrossTasks(PartitioningHandle scaledWriterPartitionHandle) { PlanNode tableWriterSource = planBuilder.exchange(ex -> ex @@ -173,8 +199,15 @@ public void testScaledWritersUsedAndTargetDoesNotSupportScalingAcrossTasks(Parti .hasMessage("The scaled writer across tasks partitioning scheme is set but writer target catalog:INSTANCE doesn't support it"); } - @Test(dataProvider = "scaledWriterPartitioningHandles") - public void testScaledWriterUsedAndTargetDoesNotSupportMultipleWritersPerPartition(PartitioningHandle scaledWriterPartitionHandle) + @Test + public void testScaledWriterUsedAndTargetDoesNotSupportMultipleWritersPerPartition() + { + testScaledWriterUsedAndTargetDoesNotSupportMultipleWritersPerPartition(SCALED_WRITER_ROUND_ROBIN_DISTRIBUTION); + testScaledWriterUsedAndTargetDoesNotSupportMultipleWritersPerPartition(SCALED_WRITER_HASH_DISTRIBUTION); + testScaledWriterUsedAndTargetDoesNotSupportMultipleWritersPerPartition(CUSTOM_HANDLE); + } + + private void testScaledWriterUsedAndTargetDoesNotSupportMultipleWritersPerPartition(PartitioningHandle scaledWriterPartitionHandle) { PlanNode tableWriterSource = planBuilder.exchange(ex -> ex @@ -202,8 +235,15 @@ public void testScaledWriterUsedAndTargetDoesNotSupportMultipleWritersPerPartiti } } - @Test(dataProvider = "scaledWriterPartitioningHandles") - public void testScaledWriterWithMultipleSourceExchangesAndTargetDoesNotSupportMultipleWritersPerPartition(PartitioningHandle scaledWriterPartitionHandle) + @Test + public void testScaledWriterWithMultipleSourceExchangesAndTargetDoesNotSupportMultipleWritersPerPartition() + { + testScaledWriterWithMultipleSourceExchangesAndTargetDoesNotSupportMultipleWritersPerPartition(SCALED_WRITER_ROUND_ROBIN_DISTRIBUTION); + testScaledWriterWithMultipleSourceExchangesAndTargetDoesNotSupportMultipleWritersPerPartition(SCALED_WRITER_HASH_DISTRIBUTION); + testScaledWriterWithMultipleSourceExchangesAndTargetDoesNotSupportMultipleWritersPerPartition(CUSTOM_HANDLE); + } + + private void testScaledWriterWithMultipleSourceExchangesAndTargetDoesNotSupportMultipleWritersPerPartition(PartitioningHandle scaledWriterPartitionHandle) { PlanNode tableWriterSource = planBuilder.exchange(ex -> ex @@ -237,20 +277,6 @@ public void testScaledWriterWithMultipleSourceExchangesAndTargetDoesNotSupportMu } } - @DataProvider - public Object[][] scaledWriterPartitioningHandles() - { - return new Object[][] { - {SCALED_WRITER_ROUND_ROBIN_DISTRIBUTION}, - {SCALED_WRITER_HASH_DISTRIBUTION}, - {new PartitioningHandle( - Optional.of(TEST_CATALOG_HANDLE), - Optional.of(new ConnectorTransactionHandle() {}), - new ConnectorPartitioningHandle() {}, - true)} - }; - } - private void validatePlan(PlanNode root) { queryRunner.inTransaction(session -> { diff --git a/core/trino-main/src/test/java/io/trino/util/TestLongLong2LongOpenCustomBigHashMap.java b/core/trino-main/src/test/java/io/trino/util/TestLongLong2LongOpenCustomBigHashMap.java index e51ccfa23b876..d6687dae37c59 100644 --- a/core/trino-main/src/test/java/io/trino/util/TestLongLong2LongOpenCustomBigHashMap.java +++ b/core/trino-main/src/test/java/io/trino/util/TestLongLong2LongOpenCustomBigHashMap.java @@ -13,7 +13,6 @@ */ package io.trino.util; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import java.util.Arrays; @@ -38,14 +37,16 @@ public boolean equals(long a1, long a2, long b1, long b2) } }; - @DataProvider - public static Object[][] nullKeyValues() + @Test + public void testBasicOps() { - return new Object[][] {{0L, 0L}, {1L, 1L}, {-1L, -1L}, {0L, -1L}}; + testBasicOps(0L, 0L); + testBasicOps(1L, 1L); + testBasicOps(-1L, -1L); + testBasicOps(0L, -1L); } - @Test(dataProvider = "nullKeyValues") - public void testBasicOps(long nullKey1, long nullKey2) + private void testBasicOps(long nullKey1, long nullKey2) { int expected = 100_000; LongLong2LongOpenCustomBigHashMap map = new LongLong2LongOpenCustomBigHashMap(expected, DEFAULT_STRATEGY, nullKey1, nullKey2); @@ -101,8 +102,16 @@ public void testBasicOps(long nullKey1, long nullKey2) } } - @Test(dataProvider = "nullKeyValues") - public void testHashCollision(long nullKey1, long nullKey2) + @Test + public void testHashCollision() + { + testHashCollision(0L, 0L); + testHashCollision(1L, 1L); + testHashCollision(-1L, -1L); + testHashCollision(0L, -1L); + } + + private void testHashCollision(long nullKey1, long nullKey2) { LongLong2LongOpenCustomBigHashMap.HashStrategy collisionHashStrategy = new LongLong2LongOpenCustomBigHashMap.HashStrategy() { @@ -168,8 +177,16 @@ public boolean equals(long a1, long a2, long b1, long b2) } } - @Test(dataProvider = "nullKeyValues") - public void testRehash(long nullKey1, long nullKey2) + @Test + public void testRehash() + { + testRehash(0L, 0L); + testRehash(1L, 1L); + testRehash(-1L, -1L); + testRehash(0L, -1L); + } + + private void testRehash(long nullKey1, long nullKey2) { int initialCapacity = 1; LongLong2LongOpenCustomBigHashMap map = new LongLong2LongOpenCustomBigHashMap(initialCapacity, DEFAULT_STRATEGY, nullKey1, nullKey2); diff --git a/testing/trino-faulttolerant-tests/src/test/java/io/trino/faulttolerant/TestFaultTolerantExecutionDynamicFiltering.java b/testing/trino-faulttolerant-tests/src/test/java/io/trino/faulttolerant/TestFaultTolerantExecutionDynamicFiltering.java index 9247c1ce3c60f..be1ec2efae1a0 100644 --- a/testing/trino-faulttolerant-tests/src/test/java/io/trino/faulttolerant/TestFaultTolerantExecutionDynamicFiltering.java +++ b/testing/trino-faulttolerant-tests/src/test/java/io/trino/faulttolerant/TestFaultTolerantExecutionDynamicFiltering.java @@ -71,8 +71,7 @@ protected RetryPolicy getRetryPolicy() // results in each instance of DynamicFilterSourceOperator receiving fewer input rows. Therefore, testing max-distinct-values-per-driver // requires larger build side and the assertions on the collected domain are adjusted for multiple ranges instead of single range. @Override - @Test(timeOut = 30_000, dataProvider = "testJoinDistributionType") - public void testSemiJoinWithNonSelectiveBuildSide(JoinDistributionType joinDistributionType, boolean coordinatorDynamicFiltersDistribution) + protected void testSemiJoinWithNonSelectiveBuildSide(JoinDistributionType joinDistributionType, boolean coordinatorDynamicFiltersDistribution) { assertQueryDynamicFilters( noJoinReordering(joinDistributionType, coordinatorDynamicFiltersDistribution), @@ -90,8 +89,7 @@ public void testSemiJoinWithNonSelectiveBuildSide(JoinDistributionType joinDistr } @Override - @Test(timeOut = 30_000, dataProvider = "testJoinDistributionType") - public void testJoinWithNonSelectiveBuildSide(JoinDistributionType joinDistributionType, boolean coordinatorDynamicFiltersDistribution) + protected void testJoinWithNonSelectiveBuildSide(JoinDistributionType joinDistributionType, boolean coordinatorDynamicFiltersDistribution) { assertQueryDynamicFilters( noJoinReordering(joinDistributionType, coordinatorDynamicFiltersDistribution), diff --git a/testing/trino-tests/src/test/java/io/trino/execution/AbstractTestCoordinatorDynamicFiltering.java b/testing/trino-tests/src/test/java/io/trino/execution/AbstractTestCoordinatorDynamicFiltering.java index 3ceec0e49c327..106766f64214b 100644 --- a/testing/trino-tests/src/test/java/io/trino/execution/AbstractTestCoordinatorDynamicFiltering.java +++ b/testing/trino-tests/src/test/java/io/trino/execution/AbstractTestCoordinatorDynamicFiltering.java @@ -49,7 +49,6 @@ import io.trino.testing.TestingTransactionHandle; import org.intellij.lang.annotations.Language; import org.testng.annotations.BeforeClass; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import java.util.List; @@ -119,8 +118,15 @@ public void setup() protected abstract RetryPolicy getRetryPolicy(); - @Test(timeOut = 30_000, dataProvider = "testJoinDistributionType") - public void testJoinWithEmptyBuildSide(JoinDistributionType joinDistributionType, boolean coordinatorDynamicFiltersDistribution) + @Test(timeOut = 30_000) + public void testJoinWithEmptyBuildSide() + { + testJoinWithEmptyBuildSide(BROADCAST, true); + testJoinWithEmptyBuildSide(PARTITIONED, true); + testJoinWithEmptyBuildSide(PARTITIONED, false); + } + + private void testJoinWithEmptyBuildSide(JoinDistributionType joinDistributionType, boolean coordinatorDynamicFiltersDistribution) { assertQueryDynamicFilters( noJoinReordering(joinDistributionType, coordinatorDynamicFiltersDistribution), @@ -129,8 +135,15 @@ public void testJoinWithEmptyBuildSide(JoinDistributionType joinDistributionType TupleDomain.none()); } - @Test(timeOut = 30_000, dataProvider = "testJoinDistributionType") - public void testJoinWithLargeBuildSide(JoinDistributionType joinDistributionType, boolean coordinatorDynamicFiltersDistribution) + @Test(timeOut = 30_000) + public void testJoinWithLargeBuildSide() + { + testJoinWithLargeBuildSide(BROADCAST, true); + testJoinWithLargeBuildSide(PARTITIONED, true); + testJoinWithLargeBuildSide(PARTITIONED, false); + } + + private void testJoinWithLargeBuildSide(JoinDistributionType joinDistributionType, boolean coordinatorDynamicFiltersDistribution) { assertQueryDynamicFilters( noJoinReordering(joinDistributionType, coordinatorDynamicFiltersDistribution), @@ -139,8 +152,15 @@ public void testJoinWithLargeBuildSide(JoinDistributionType joinDistributionType TupleDomain.all()); } - @Test(timeOut = 30_000, dataProvider = "testJoinDistributionType") - public void testMultiColumnJoinWithDifferentCardinalitiesInBuildSide(JoinDistributionType joinDistributionType, boolean coordinatorDynamicFiltersDistribution) + @Test(timeOut = 30_000) + public void testMultiColumnJoinWithDifferentCardinalitiesInBuildSide() + { + testMultiColumnJoinWithDifferentCardinalitiesInBuildSide(BROADCAST, true); + testMultiColumnJoinWithDifferentCardinalitiesInBuildSide(PARTITIONED, true); + testMultiColumnJoinWithDifferentCardinalitiesInBuildSide(PARTITIONED, false); + } + + private void testMultiColumnJoinWithDifferentCardinalitiesInBuildSide(JoinDistributionType joinDistributionType, boolean coordinatorDynamicFiltersDistribution) { // orderkey has high cardinality, suppkey has low cardinality due to filter assertQueryDynamicFilters( @@ -154,8 +174,15 @@ public void testMultiColumnJoinWithDifferentCardinalitiesInBuildSide(JoinDistrib multipleValues(BIGINT, LongStream.rangeClosed(1L, 10L).boxed().collect(toImmutableList()))))); } - @Test(timeOut = 30_000, dataProvider = "testJoinDistributionType") - public void testJoinWithSelectiveBuildSide(JoinDistributionType joinDistributionType, boolean coordinatorDynamicFiltersDistribution) + @Test(timeOut = 30_000) + public void testJoinWithSelectiveBuildSide() + { + testJoinWithSelectiveBuildSide(BROADCAST, true); + testJoinWithSelectiveBuildSide(PARTITIONED, true); + testJoinWithSelectiveBuildSide(PARTITIONED, false); + } + + private void testJoinWithSelectiveBuildSide(JoinDistributionType joinDistributionType, boolean coordinatorDynamicFiltersDistribution) { assertQueryDynamicFilters( noJoinReordering(joinDistributionType, coordinatorDynamicFiltersDistribution), @@ -246,8 +273,15 @@ public void testJoinWithImplicitCoercion() multipleValues(createVarcharType(40), values)))); } - @Test(timeOut = 30_000, dataProvider = "testJoinDistributionType") - public void testJoinWithNonSelectiveBuildSide(JoinDistributionType joinDistributionType, boolean coordinatorDynamicFiltersDistribution) + @Test(timeOut = 30_000) + public void testJoinWithNonSelectiveBuildSide() + { + testJoinWithNonSelectiveBuildSide(BROADCAST, true); + testJoinWithNonSelectiveBuildSide(PARTITIONED, true); + testJoinWithNonSelectiveBuildSide(PARTITIONED, false); + } + + protected void testJoinWithNonSelectiveBuildSide(JoinDistributionType joinDistributionType, boolean coordinatorDynamicFiltersDistribution) { assertQueryDynamicFilters( noJoinReordering(joinDistributionType, coordinatorDynamicFiltersDistribution), @@ -258,8 +292,15 @@ public void testJoinWithNonSelectiveBuildSide(JoinDistributionType joinDistribut Domain.create(ValueSet.ofRanges(range(BIGINT, 1L, true, 100L, true)), false)))); } - @Test(timeOut = 30_000, dataProvider = "testJoinDistributionType") - public void testJoinWithMultipleDynamicFiltersOnProbe(JoinDistributionType joinDistributionType, boolean coordinatorDynamicFiltersDistribution) + @Test(timeOut = 30_000) + public void testJoinWithMultipleDynamicFiltersOnProbe() + { + testJoinWithMultipleDynamicFiltersOnProbe(BROADCAST, true); + testJoinWithMultipleDynamicFiltersOnProbe(PARTITIONED, true); + testJoinWithMultipleDynamicFiltersOnProbe(PARTITIONED, false); + } + + private void testJoinWithMultipleDynamicFiltersOnProbe(JoinDistributionType joinDistributionType, boolean coordinatorDynamicFiltersDistribution) { // supplier names Supplier#000000001 and Supplier#000000002 match suppkey 1 and 2 assertQueryDynamicFilters( @@ -305,8 +346,15 @@ public void testRightJoinWithSelectiveBuildSide() singleValue(BIGINT, 1L)))); } - @Test(timeOut = 30_000, dataProvider = "testJoinDistributionType") - public void testSemiJoinWithEmptyBuildSide(JoinDistributionType joinDistributionType, boolean coordinatorDynamicFiltersDistribution) + @Test(timeOut = 30_000) + public void testSemiJoinWithEmptyBuildSide() + { + testSemiJoinWithEmptyBuildSide(BROADCAST, true); + testSemiJoinWithEmptyBuildSide(PARTITIONED, true); + testSemiJoinWithEmptyBuildSide(PARTITIONED, false); + } + + private void testSemiJoinWithEmptyBuildSide(JoinDistributionType joinDistributionType, boolean coordinatorDynamicFiltersDistribution) { assertQueryDynamicFilters( noJoinReordering(joinDistributionType, coordinatorDynamicFiltersDistribution), @@ -315,8 +363,15 @@ public void testSemiJoinWithEmptyBuildSide(JoinDistributionType joinDistribution TupleDomain.none()); } - @Test(timeOut = 30_000, dataProvider = "testJoinDistributionType") - public void testSemiJoinWithLargeBuildSide(JoinDistributionType joinDistributionType, boolean coordinatorDynamicFiltersDistribution) + @Test(timeOut = 30_000) + public void testSemiJoinWithLargeBuildSide() + { + testSemiJoinWithLargeBuildSide(BROADCAST, true); + testSemiJoinWithLargeBuildSide(PARTITIONED, true); + testSemiJoinWithLargeBuildSide(PARTITIONED, false); + } + + private void testSemiJoinWithLargeBuildSide(JoinDistributionType joinDistributionType, boolean coordinatorDynamicFiltersDistribution) { assertQueryDynamicFilters( noJoinReordering(joinDistributionType, coordinatorDynamicFiltersDistribution), @@ -325,8 +380,15 @@ public void testSemiJoinWithLargeBuildSide(JoinDistributionType joinDistribution TupleDomain.all()); } - @Test(timeOut = 30_000, dataProvider = "testJoinDistributionType") - public void testSemiJoinWithSelectiveBuildSide(JoinDistributionType joinDistributionType, boolean coordinatorDynamicFiltersDistribution) + @Test(timeOut = 30_000) + public void testSemiJoinWithSelectiveBuildSide() + { + testSemiJoinWithSelectiveBuildSide(BROADCAST, true); + testSemiJoinWithSelectiveBuildSide(PARTITIONED, true); + testSemiJoinWithSelectiveBuildSide(PARTITIONED, false); + } + + private void testSemiJoinWithSelectiveBuildSide(JoinDistributionType joinDistributionType, boolean coordinatorDynamicFiltersDistribution) { assertQueryDynamicFilters( noJoinReordering(joinDistributionType, coordinatorDynamicFiltersDistribution), @@ -337,8 +399,15 @@ public void testSemiJoinWithSelectiveBuildSide(JoinDistributionType joinDistribu singleValue(BIGINT, 1L)))); } - @Test(timeOut = 30_000, dataProvider = "testJoinDistributionType") - public void testSemiJoinWithNonSelectiveBuildSide(JoinDistributionType joinDistributionType, boolean coordinatorDynamicFiltersDistribution) + @Test(timeOut = 30_000) + public void testSemiJoinWithNonSelectiveBuildSide() + { + testSemiJoinWithNonSelectiveBuildSide(BROADCAST, true); + testSemiJoinWithNonSelectiveBuildSide(PARTITIONED, true); + testSemiJoinWithNonSelectiveBuildSide(PARTITIONED, false); + } + + protected void testSemiJoinWithNonSelectiveBuildSide(JoinDistributionType joinDistributionType, boolean coordinatorDynamicFiltersDistribution) { assertQueryDynamicFilters( noJoinReordering(joinDistributionType, coordinatorDynamicFiltersDistribution), @@ -349,8 +418,15 @@ public void testSemiJoinWithNonSelectiveBuildSide(JoinDistributionType joinDistr Domain.create(ValueSet.ofRanges(range(BIGINT, 1L, true, 100L, true)), false)))); } - @Test(timeOut = 30_000, dataProvider = "testJoinDistributionType") - public void testSemiJoinWithMultipleDynamicFiltersOnProbe(JoinDistributionType joinDistributionType, boolean coordinatorDynamicFiltersDistribution) + @Test(timeOut = 30_000) + public void testSemiJoinWithMultipleDynamicFiltersOnProbe() + { + testSemiJoinWithMultipleDynamicFiltersOnProbe(BROADCAST, true); + testSemiJoinWithMultipleDynamicFiltersOnProbe(PARTITIONED, true); + testSemiJoinWithMultipleDynamicFiltersOnProbe(PARTITIONED, false); + } + + private void testSemiJoinWithMultipleDynamicFiltersOnProbe(JoinDistributionType joinDistributionType, boolean coordinatorDynamicFiltersDistribution) { // supplier names Supplier#000000001 and Supplier#000000002 match suppkey 1 and 2 assertQueryDynamicFilters( @@ -378,15 +454,6 @@ protected Session getDefaultSession() .build(); } - @DataProvider - public Object[][] testJoinDistributionType() - { - return new Object[][] { - {BROADCAST, true}, - {PARTITIONED, true}, - {PARTITIONED, false}}; - } - protected Session noJoinReordering(JoinDistributionType distributionType, boolean coordinatorDynamicFiltersDistribution) { return Session.builder(noJoinReordering(distributionType)) From ed0bd266881dcd33d58d164a8fa5cd0a363d36c8 Mon Sep 17 00:00:00 2001 From: Martin Traverso Date: Thu, 7 Dec 2023 10:56:35 -0800 Subject: [PATCH 054/350] Migrate tests to JUnit --- core/trino-main/pom.xml | 67 +++++----- .../TestGroupedTopNRankAccumulator.java | 2 +- .../operator/TestGroupedTopNRankBuilder.java | 2 +- .../TestGroupedTopNRowNumberBuilder.java | 2 +- .../operator/TestHashAggregationOperator.java | 58 ++++---- .../operator/TestHashSemiJoinOperator.java | 29 ++-- .../operator/TestMarkDistinctOperator.java | 25 ++-- .../trino/operator/TestOrderByOperator.java | 38 +++--- .../operator/TestTopNPeerGroupLookup.java | 2 +- .../io/trino/operator/TestWindowOperator.java | 102 ++++++++------ .../AbstractTestApproximateCountDistinct.java | 2 +- .../TestAggregationMaskCompiler.java | 2 +- .../TestApproximateCountDistinctBoolean.java | 2 +- .../TestApproximateSetGenericBoolean.java | 2 +- .../TestDecimalAverageAggregation.java | 36 +++-- ...malWithOverflowAndLongStateSerializer.java | 2 +- ...ongDecimalWithOverflowStateSerializer.java | 2 +- .../operator/exchange/TestLocalExchange.java | 13 +- .../operator/join/TestHashJoinOperator.java | 64 ++++----- .../join/unspilled/TestHashJoinOperator.java | 60 +++------ .../operator/output/TestPagePartitioner.java | 27 ++-- .../output/TestPositionsAppender.java | 7 +- .../TestDictionaryAwarePageProjection.java | 12 +- .../operator/scalar/TestScalarValidation.java | 124 ++++++++++++------ .../server/security/TestResourceSecurity.java | 12 +- .../oauth2/TestJweTokenSerializer.java | 2 +- .../security/oauth2/TestOidcDiscovery.java | 2 +- .../TestEffectivePredicateExtractor.java | 13 +- .../TestCounterBasedAnonymizer.java | 24 ++-- .../TestValidateScaledWritersUsage.java | 16 ++- ...TestLongLong2LongOpenCustomBigHashMap.java | 2 +- testing/trino-faulttolerant-tests/pom.xml | 33 ----- ...aultTolerantExecutionDynamicFiltering.java | 10 +- ...stractTestCoordinatorDynamicFiltering.java | 64 ++++++--- .../TestCoordinatorDynamicFiltering.java | 5 +- 35 files changed, 458 insertions(+), 407 deletions(-) diff --git a/core/trino-main/pom.xml b/core/trino-main/pom.xml index 1363a9fa9a7c6..73b9cd58314f0 100644 --- a/core/trino-main/pom.xml +++ b/core/trino-main/pom.xml @@ -13,15 +13,6 @@ ${project.parent.basedir} - - - instances @@ -390,12 +381,6 @@ provided - - org.testng - testng - provided - - com.squareup.okhttp3 okhttp @@ -533,25 +518,35 @@ - - - - org.apache.maven.plugins - maven-surefire-plugin - - - - org.apache.maven.surefire - surefire-junit-platform - ${dep.plugin.surefire.version} - - - org.apache.maven.surefire - surefire-testng - ${dep.plugin.surefire.version} - - - - - + + + benchmarks + + + + org.codehaus.mojo + exec-maven-plugin + + ${java.home}/bin/java + + -DoutputDirectory=benchmark_outputs + -classpath + + io.trino.benchmark.BenchmarkSuite + + test + + + + benchmarks + + exec + + + + + + + + diff --git a/core/trino-main/src/test/java/io/trino/operator/TestGroupedTopNRankAccumulator.java b/core/trino-main/src/test/java/io/trino/operator/TestGroupedTopNRankAccumulator.java index d95953791bf9a..ca21f4cca5e5a 100644 --- a/core/trino-main/src/test/java/io/trino/operator/TestGroupedTopNRankAccumulator.java +++ b/core/trino-main/src/test/java/io/trino/operator/TestGroupedTopNRankAccumulator.java @@ -15,7 +15,7 @@ import io.trino.array.LongBigArray; import it.unimi.dsi.fastutil.longs.LongArrayList; -import org.testng.annotations.Test; +import org.junit.jupiter.api.Test; import java.util.ArrayList; import java.util.Arrays; diff --git a/core/trino-main/src/test/java/io/trino/operator/TestGroupedTopNRankBuilder.java b/core/trino-main/src/test/java/io/trino/operator/TestGroupedTopNRankBuilder.java index 5fa6210a3dd96..4f6c6a0a43a0c 100644 --- a/core/trino-main/src/test/java/io/trino/operator/TestGroupedTopNRankBuilder.java +++ b/core/trino-main/src/test/java/io/trino/operator/TestGroupedTopNRankBuilder.java @@ -19,7 +19,7 @@ import io.trino.spi.type.TypeOperators; import io.trino.sql.gen.JoinCompiler; import io.trino.type.BlockTypeOperators; -import org.testng.annotations.Test; +import org.junit.jupiter.api.Test; import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; diff --git a/core/trino-main/src/test/java/io/trino/operator/TestGroupedTopNRowNumberBuilder.java b/core/trino-main/src/test/java/io/trino/operator/TestGroupedTopNRowNumberBuilder.java index 7ad787c6b95bd..a150a4e47dcf4 100644 --- a/core/trino-main/src/test/java/io/trino/operator/TestGroupedTopNRowNumberBuilder.java +++ b/core/trino-main/src/test/java/io/trino/operator/TestGroupedTopNRowNumberBuilder.java @@ -18,7 +18,7 @@ import io.trino.spi.type.Type; import io.trino.spi.type.TypeOperators; import io.trino.sql.gen.JoinCompiler; -import org.testng.annotations.Test; +import org.junit.jupiter.api.Test; import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; diff --git a/core/trino-main/src/test/java/io/trino/operator/TestHashAggregationOperator.java b/core/trino-main/src/test/java/io/trino/operator/TestHashAggregationOperator.java index cbf8cf4a800ad..cdeb3e7487787 100644 --- a/core/trino-main/src/test/java/io/trino/operator/TestHashAggregationOperator.java +++ b/core/trino-main/src/test/java/io/trino/operator/TestHashAggregationOperator.java @@ -40,9 +40,10 @@ import io.trino.sql.planner.plan.PlanNodeId; import io.trino.testing.MaterializedResult; import io.trino.testing.TestingTaskContext; -import org.testng.annotations.AfterMethod; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.parallel.Execution; import java.io.IOException; import java.util.ArrayList; @@ -93,8 +94,11 @@ import static java.util.concurrent.Executors.newScheduledThreadPool; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; -@Test(singleThreaded = true) +@TestInstance(PER_CLASS) +@Execution(CONCURRENT) public class TestHashAggregationOperator { private static final TestingFunctionResolution FUNCTION_RESOLUTION = new TestingFunctionResolution(); @@ -106,24 +110,14 @@ public class TestHashAggregationOperator private static final int MAX_BLOCK_SIZE_IN_BYTES = 64 * 1024; - private ExecutorService executor; - private ScheduledExecutorService scheduledExecutor; + private final ExecutorService executor = newCachedThreadPool(daemonThreadsNamed(getClass().getSimpleName() + "-%s")); + private final ScheduledExecutorService scheduledExecutor = newScheduledThreadPool(2, daemonThreadsNamed(getClass().getSimpleName() + "-scheduledExecutor-%s")); private final TypeOperators typeOperators = new TypeOperators(); private final JoinCompiler joinCompiler = new JoinCompiler(typeOperators); - private DummySpillerFactory spillerFactory; - @BeforeMethod - public void setUp() - { - executor = newCachedThreadPool(daemonThreadsNamed(getClass().getSimpleName() + "-%s")); - scheduledExecutor = newScheduledThreadPool(2, daemonThreadsNamed(getClass().getSimpleName() + "-scheduledExecutor-%s")); - spillerFactory = new DummySpillerFactory(); - } - - @AfterMethod(alwaysRun = true) + @AfterAll public void tearDown() { - spillerFactory = null; executor.shutdownNow(); scheduledExecutor.shutdownNow(); } @@ -218,6 +212,8 @@ public void testHashAggregationWithGlobals() private void testHashAggregationWithGlobals(boolean hashEnabled, boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimitForMerge, long memoryLimitForMergeWithMemory) { + DummySpillerFactory spillerFactory = new DummySpillerFactory(); + TestingAggregationFunction countVarcharColumn = FUNCTION_RESOLUTION.getAggregateFunction("count", fromTypes(VARCHAR)); TestingAggregationFunction countBooleanColumn = FUNCTION_RESOLUTION.getAggregateFunction("count", fromTypes(BOOLEAN)); TestingAggregationFunction maxVarcharColumn = FUNCTION_RESOLUTION.getAggregateFunction("max", fromTypes(VARCHAR)); @@ -279,6 +275,8 @@ public void testHashAggregationMemoryReservation() private void testHashAggregationMemoryReservation(boolean hashEnabled, boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimitForMerge, long memoryLimitForMergeWithMemory) { + DummySpillerFactory spillerFactory = new DummySpillerFactory(); + TestingAggregationFunction arrayAggColumn = FUNCTION_RESOLUTION.getAggregateFunction("array_agg", fromTypes(BIGINT)); List hashChannels = Ints.asList(1); @@ -321,11 +319,16 @@ private void testHashAggregationMemoryReservation(boolean hashEnabled, boolean s assertThat(getOnlyElement(operator.getOperatorContext().getNestedOperatorStats()).getRevocableMemoryReservation().toBytes()).isEqualTo(0); } - @Test(expectedExceptions = ExceededMemoryLimitException.class, expectedExceptionsMessageRegExp = "Query exceeded per-node memory limit of 10B.*") + @Test public void testMemoryLimit() { - testMemoryLimit(true); - testMemoryLimit(false); + assertThatThrownBy(() -> testMemoryLimit(true)) + .isInstanceOf(ExceededMemoryLimitException.class) + .hasMessageMatching("Query exceeded per-node memory limit of 10B.*"); + + assertThatThrownBy(() -> testMemoryLimit(false)) + .isInstanceOf(ExceededMemoryLimitException.class) + .hasMessageMatching("Query exceeded per-node memory limit of 10B.*"); } private void testMemoryLimit(boolean hashEnabled) @@ -382,6 +385,8 @@ public void testHashBuilderResize() private void testHashBuilderResize(boolean hashEnabled, boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimitForMerge, long memoryLimitForMergeWithMemory) { + DummySpillerFactory spillerFactory = new DummySpillerFactory(); + BlockBuilder builder = VARCHAR.createBlockBuilder(null, 1, MAX_BLOCK_SIZE_IN_BYTES); VARCHAR.writeSlice(builder, Slices.allocate(200_000)); // this must be larger than MAX_BLOCK_SIZE_IN_BYTES, 64K builder.build(); @@ -464,11 +469,16 @@ public void testMemoryReservationYield(Type type) assertThat(count).isEqualTo(6_000 * 600); } - @Test(expectedExceptions = ExceededMemoryLimitException.class, expectedExceptionsMessageRegExp = "Query exceeded per-node memory limit of 3MB.*") + @Test public void testHashBuilderResizeLimit() { - testHashBuilderResizeLimit(true); - testHashBuilderResizeLimit(false); + assertThatThrownBy(() -> testHashBuilderResizeLimit(true)) + .isInstanceOf(ExceededMemoryLimitException.class) + .hasMessageMatching("Query exceeded per-node memory limit of 3MB.*"); + + assertThatThrownBy(() -> testHashBuilderResizeLimit(false)) + .isInstanceOf(ExceededMemoryLimitException.class) + .hasMessageMatching("Query exceeded per-node memory limit of 3MB.*"); } private void testHashBuilderResizeLimit(boolean hashEnabled) @@ -641,6 +651,8 @@ private void testMultiplePartialFlushes(boolean hashEnabled) @Test public void testMergeWithMemorySpill() { + DummySpillerFactory spillerFactory = new DummySpillerFactory(); + RowPagesBuilder rowPagesBuilder = rowPagesBuilder(BIGINT); int smallPagesSpillThresholdSize = 150000; diff --git a/core/trino-main/src/test/java/io/trino/operator/TestHashSemiJoinOperator.java b/core/trino-main/src/test/java/io/trino/operator/TestHashSemiJoinOperator.java index cdca808be1a7e..3171e6c59a7b5 100644 --- a/core/trino-main/src/test/java/io/trino/operator/TestHashSemiJoinOperator.java +++ b/core/trino-main/src/test/java/io/trino/operator/TestHashSemiJoinOperator.java @@ -25,9 +25,11 @@ import io.trino.sql.gen.JoinCompiler; import io.trino.sql.planner.plan.PlanNodeId; import io.trino.testing.MaterializedResult; -import org.testng.annotations.AfterMethod; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.parallel.Execution; import java.util.List; import java.util.Optional; @@ -45,8 +47,12 @@ import static io.trino.testing.TestingTaskContext.createTaskContext; import static java.util.concurrent.Executors.newCachedThreadPool; import static java.util.concurrent.Executors.newScheduledThreadPool; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_METHOD; +import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; -@Test(singleThreaded = true) +@TestInstance(PER_METHOD) +@Execution(SAME_THREAD) public class TestHashSemiJoinOperator { private ExecutorService executor; @@ -54,7 +60,7 @@ public class TestHashSemiJoinOperator private TaskContext taskContext; private TypeOperators typeOperators; - @BeforeMethod + @BeforeEach public void setUp() { executor = newCachedThreadPool(daemonThreadsNamed(getClass().getSimpleName() + "-%s")); @@ -63,7 +69,7 @@ public void setUp() typeOperators = new TypeOperators(); } - @AfterMethod(alwaysRun = true) + @AfterEach public void tearDown() { executor.shutdownNow(); @@ -408,11 +414,16 @@ private void testProbeAndBuildNulls(boolean hashEnabled) OperatorAssertion.assertOperatorEquals(joinOperatorFactory, driverContext, probeInput, expected, hashEnabled, ImmutableList.of(probeTypes.size())); } - @Test(expectedExceptions = ExceededMemoryLimitException.class, expectedExceptionsMessageRegExp = "Query exceeded per-node memory limit of.*") + @Test public void testMemoryLimit() { - testMemoryLimit(true); - testMemoryLimit(false); + assertThatThrownBy(() -> testMemoryLimit(true)) + .isInstanceOf(ExceededMemoryLimitException.class) + .hasMessageMatching("Query exceeded per-node memory limit of.*"); + + assertThatThrownBy(() -> testMemoryLimit(false)) + .isInstanceOf(ExceededMemoryLimitException.class) + .hasMessageMatching("Query exceeded per-node memory limit of.*"); } private void testMemoryLimit(boolean hashEnabled) diff --git a/core/trino-main/src/test/java/io/trino/operator/TestMarkDistinctOperator.java b/core/trino-main/src/test/java/io/trino/operator/TestMarkDistinctOperator.java index 52f2b8e2efefd..f79b8b979bcee 100644 --- a/core/trino-main/src/test/java/io/trino/operator/TestMarkDistinctOperator.java +++ b/core/trino-main/src/test/java/io/trino/operator/TestMarkDistinctOperator.java @@ -25,9 +25,10 @@ import io.trino.sql.gen.JoinCompiler; import io.trino.sql.planner.plan.PlanNodeId; import io.trino.testing.MaterializedResult; -import org.testng.annotations.AfterMethod; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.parallel.Execution; import java.util.List; import java.util.Optional; @@ -50,23 +51,19 @@ import static java.util.concurrent.Executors.newCachedThreadPool; import static java.util.concurrent.Executors.newScheduledThreadPool; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; -@Test(singleThreaded = true) +@TestInstance(PER_CLASS) +@Execution(CONCURRENT) public class TestMarkDistinctOperator { - private ExecutorService executor; - private ScheduledExecutorService scheduledExecutor; + private final ExecutorService executor = newCachedThreadPool(daemonThreadsNamed(getClass().getSimpleName() + "-%s")); + private final ScheduledExecutorService scheduledExecutor = newScheduledThreadPool(2, daemonThreadsNamed(getClass().getSimpleName() + "-scheduledExecutor-%s")); private final TypeOperators typeOperators = new TypeOperators(); private final JoinCompiler joinCompiler = new JoinCompiler(typeOperators); - @BeforeMethod - public void setUp() - { - executor = newCachedThreadPool(daemonThreadsNamed(getClass().getSimpleName() + "-%s")); - scheduledExecutor = newScheduledThreadPool(2, daemonThreadsNamed(getClass().getSimpleName() + "-scheduledExecutor-%s")); - } - - @AfterMethod(alwaysRun = true) + @AfterAll public void tearDown() { executor.shutdownNow(); diff --git a/core/trino-main/src/test/java/io/trino/operator/TestOrderByOperator.java b/core/trino-main/src/test/java/io/trino/operator/TestOrderByOperator.java index 934bc644238d8..2fbe0ac967933 100644 --- a/core/trino-main/src/test/java/io/trino/operator/TestOrderByOperator.java +++ b/core/trino-main/src/test/java/io/trino/operator/TestOrderByOperator.java @@ -23,9 +23,10 @@ import io.trino.sql.planner.plan.PlanNodeId; import io.trino.testing.MaterializedResult; import io.trino.testing.TestingTaskContext; -import org.testng.annotations.AfterMethod; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.parallel.Execution; import java.util.List; import java.util.Optional; @@ -52,29 +53,22 @@ import static java.util.concurrent.Executors.newScheduledThreadPool; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; -@Test(singleThreaded = true) +@TestInstance(PER_CLASS) +@Execution(CONCURRENT) public class TestOrderByOperator { - private ExecutorService executor; - private ScheduledExecutorService scheduledExecutor; - private DummySpillerFactory spillerFactory; + private final ExecutorService executor = newCachedThreadPool(daemonThreadsNamed(getClass().getSimpleName() + "-%s")); + private final ScheduledExecutorService scheduledExecutor = newScheduledThreadPool(2, daemonThreadsNamed(getClass().getSimpleName() + "-scheduledExecutor-%s")); private final TypeOperators typeOperators = new TypeOperators(); - @BeforeMethod - public void setUp() - { - executor = newCachedThreadPool(daemonThreadsNamed(getClass().getSimpleName() + "-%s")); - scheduledExecutor = newScheduledThreadPool(2, daemonThreadsNamed(getClass().getSimpleName() + "-scheduledExecutor-%s")); - spillerFactory = new DummySpillerFactory(); - } - - @AfterMethod(alwaysRun = true) + @AfterAll public void tearDown() { executor.shutdownNow(); scheduledExecutor.shutdownNow(); - spillerFactory = null; } @Test @@ -89,6 +83,8 @@ public void testMultipleOutputPages() private void testMultipleOutputPages(boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimit) { + DummySpillerFactory spillerFactory = new DummySpillerFactory(); + // make operator produce multiple pages during finish phase int numberOfRows = 80_000; List input = rowPagesBuilder(BIGINT, DOUBLE) @@ -156,7 +152,7 @@ private void testSingleFieldKey(boolean spillEnabled, boolean revokeMemoryWhenAd ImmutableList.of(ASC_NULLS_LAST), new PagesIndex.TestingFactory(false), spillEnabled, - Optional.of(spillerFactory), + Optional.of(new DummySpillerFactory()), new OrderingCompiler(typeOperators)); DriverContext driverContext = createDriverContext(memoryLimit); @@ -200,7 +196,7 @@ private void testMultiFieldKey(boolean spillEnabled, boolean revokeMemoryWhenAdd ImmutableList.of(ASC_NULLS_LAST, DESC_NULLS_LAST), new PagesIndex.TestingFactory(false), spillEnabled, - Optional.of(spillerFactory), + Optional.of(new DummySpillerFactory()), new OrderingCompiler(typeOperators)); DriverContext driverContext = createDriverContext(memoryLimit); @@ -244,7 +240,7 @@ private void testReverseOrder(boolean spillEnabled, boolean revokeMemoryWhenAddi ImmutableList.of(DESC_NULLS_LAST), new PagesIndex.TestingFactory(false), spillEnabled, - Optional.of(spillerFactory), + Optional.of(new DummySpillerFactory()), new OrderingCompiler(typeOperators)); DriverContext driverContext = createDriverContext(memoryLimit); @@ -283,7 +279,7 @@ public void testMemoryLimit() ImmutableList.of(ASC_NULLS_LAST), new PagesIndex.TestingFactory(false), false, - Optional.of(spillerFactory), + Optional.of(new DummySpillerFactory()), new OrderingCompiler(typeOperators)); assertThatThrownBy(() -> toPages(operatorFactory, driverContext, input)) diff --git a/core/trino-main/src/test/java/io/trino/operator/TestTopNPeerGroupLookup.java b/core/trino-main/src/test/java/io/trino/operator/TestTopNPeerGroupLookup.java index 9c261c935cd51..a2c6c192026c3 100644 --- a/core/trino-main/src/test/java/io/trino/operator/TestTopNPeerGroupLookup.java +++ b/core/trino-main/src/test/java/io/trino/operator/TestTopNPeerGroupLookup.java @@ -13,7 +13,7 @@ */ package io.trino.operator; -import org.testng.annotations.Test; +import org.junit.jupiter.api.Test; import java.util.Arrays; diff --git a/core/trino-main/src/test/java/io/trino/operator/TestWindowOperator.java b/core/trino-main/src/test/java/io/trino/operator/TestWindowOperator.java index d1d39d691e376..1c55c5d36569e 100644 --- a/core/trino-main/src/test/java/io/trino/operator/TestWindowOperator.java +++ b/core/trino-main/src/test/java/io/trino/operator/TestWindowOperator.java @@ -38,9 +38,10 @@ import io.trino.sql.planner.plan.PlanNodeId; import io.trino.testing.MaterializedResult; import io.trino.testing.TestingTaskContext; -import org.testng.annotations.AfterMethod; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.parallel.Execution; import java.util.List; import java.util.Optional; @@ -71,8 +72,12 @@ import static java.util.concurrent.Executors.newCachedThreadPool; import static java.util.concurrent.Executors.newScheduledThreadPool; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; -@Test(singleThreaded = true) +@TestInstance(PER_CLASS) +@Execution(CONCURRENT) public class TestWindowOperator { private static final TypeOperators TYPE_OPERATORS_CACHE = new TypeOperators(); @@ -99,24 +104,14 @@ public class TestWindowOperator private static final List LEAD = ImmutableList.of( window(new ReflectionWindowFunctionSupplier(3, LeadFunction.class), VARCHAR, UNBOUNDED_FRAME, false, ImmutableList.of(), 1, 3, 4)); - private ExecutorService executor; - private ScheduledExecutorService scheduledExecutor; - private DummySpillerFactory spillerFactory; + private final ExecutorService executor = newCachedThreadPool(daemonThreadsNamed(getClass().getSimpleName() + "-%s")); + private final ScheduledExecutorService scheduledExecutor = newScheduledThreadPool(2, daemonThreadsNamed(getClass().getSimpleName() + "-scheduledExecutor-%s")); - @BeforeMethod - public void setUp() - { - executor = newCachedThreadPool(daemonThreadsNamed(getClass().getSimpleName() + "-%s")); - scheduledExecutor = newScheduledThreadPool(2, daemonThreadsNamed(getClass().getSimpleName() + "-scheduledExecutor-%s")); - spillerFactory = new DummySpillerFactory(); - } - - @AfterMethod(alwaysRun = true) + @AfterAll public void tearDown() { executor.shutdownNow(); scheduledExecutor.shutdownNow(); - spillerFactory = null; } @Test @@ -131,6 +126,8 @@ public void testMultipleOutputPages() private void testMultipleOutputPages(boolean spillEnabled, boolean revokeMemoryWhenAddingPages, long memoryLimit) { + DummySpillerFactory spillerFactory = new DummySpillerFactory(); + // make operator produce multiple pages during finish phase int numberOfRows = 80_000; List input = rowPagesBuilder(BIGINT, DOUBLE) @@ -144,6 +141,7 @@ private void testMultipleOutputPages(boolean spillEnabled, boolean revokeMemoryW Ints.asList(), Ints.asList(0), ImmutableList.copyOf(new SortOrder[] {SortOrder.DESC_NULLS_FIRST}), + spillerFactory, spillEnabled); DriverContext driverContext = createDriverContext(memoryLimit); @@ -192,6 +190,7 @@ private void testRowNumber(boolean spillEnabled, boolean revokeMemoryWhenAddingP Ints.asList(), Ints.asList(0), ImmutableList.copyOf(new SortOrder[] {SortOrder.ASC_NULLS_LAST}), + new DummySpillerFactory(), spillEnabled); DriverContext driverContext = createDriverContext(memoryLimit); @@ -234,6 +233,7 @@ private void testRowNumberPartition(boolean spillEnabled, boolean revokeMemoryWh Ints.asList(0), Ints.asList(1), ImmutableList.copyOf(new SortOrder[] {SortOrder.ASC_NULLS_LAST}), + new DummySpillerFactory(), spillEnabled); DriverContext driverContext = createDriverContext(memoryLimit); @@ -270,6 +270,7 @@ public void testRowNumberArbitrary() Ints.asList(), Ints.asList(), ImmutableList.copyOf(new SortOrder[] {}), + new DummySpillerFactory(), false); DriverContext driverContext = createDriverContext(); @@ -309,6 +310,7 @@ public void testRowNumberArbitraryWithSpill() Ints.asList(), Ints.asList(), ImmutableList.copyOf(new SortOrder[] {}), + new DummySpillerFactory(), true); DriverContext driverContext = createDriverContext(); @@ -368,6 +370,7 @@ public void testDistinctPartitionAndPeers(boolean spillEnabled, boolean revokeMe Ints.asList(0), Ints.asList(1), ImmutableList.copyOf(new SortOrder[] {SortOrder.ASC_NULLS_LAST}), + new DummySpillerFactory(), spillEnabled); DriverContext driverContext = createDriverContext(memoryLimit); @@ -396,31 +399,36 @@ public void testDistinctPartitionAndPeers(boolean spillEnabled, boolean revokeMe assertOperatorEquals(operatorFactory, driverContext, input, expected, revokeMemoryWhenAddingPages); } - @Test(expectedExceptions = ExceededMemoryLimitException.class, expectedExceptionsMessageRegExp = "Query exceeded per-node memory limit of 10B.*") + @Test public void testMemoryLimit() { - List input = rowPagesBuilder(BIGINT, DOUBLE) - .row(1L, 0.1) - .row(2L, 0.2) - .pageBreak() - .row(-1L, -0.1) - .row(4L, 0.4) - .build(); - - DriverContext driverContext = createTaskContext(executor, scheduledExecutor, TEST_SESSION, DataSize.ofBytes(10)) - .addPipelineContext(0, true, true, false) - .addDriverContext(); - - WindowOperatorFactory operatorFactory = createFactoryUnbounded( - ImmutableList.of(BIGINT, DOUBLE), - Ints.asList(1), - ROW_NUMBER, - Ints.asList(), - Ints.asList(0), - ImmutableList.copyOf(new SortOrder[] {SortOrder.ASC_NULLS_LAST}), - false); - - toPages(operatorFactory, driverContext, input); + assertThatThrownBy(() -> { + List input = rowPagesBuilder(BIGINT, DOUBLE) + .row(1L, 0.1) + .row(2L, 0.2) + .pageBreak() + .row(-1L, -0.1) + .row(4L, 0.4) + .build(); + + DriverContext driverContext = createTaskContext(executor, scheduledExecutor, TEST_SESSION, DataSize.ofBytes(10)) + .addPipelineContext(0, true, true, false) + .addDriverContext(); + + WindowOperatorFactory operatorFactory = createFactoryUnbounded( + ImmutableList.of(BIGINT, DOUBLE), + Ints.asList(1), + ROW_NUMBER, + Ints.asList(), + Ints.asList(0), + ImmutableList.copyOf(new SortOrder[] {SortOrder.ASC_NULLS_LAST}), + new DummySpillerFactory(), + false); + + toPages(operatorFactory, driverContext, input); + }) + .isInstanceOf(ExceededMemoryLimitException.class) + .hasMessageMatching("Query exceeded per-node memory limit of 10B.*"); } @Test @@ -452,6 +460,7 @@ private void testFirstValuePartition(boolean spillEnabled, boolean revokeMemoryW Ints.asList(0), Ints.asList(2), ImmutableList.copyOf(new SortOrder[] {SortOrder.ASC_NULLS_LAST}), + new DummySpillerFactory(), spillEnabled); DriverContext driverContext = createDriverContext(memoryLimit); @@ -487,6 +496,7 @@ public void testClose() Ints.asList(0), Ints.asList(1), ImmutableList.copyOf(new SortOrder[] {SortOrder.ASC_NULLS_LAST}), + new DummySpillerFactory(), false); DriverContext driverContext = createDriverContext(1000); @@ -532,6 +542,7 @@ private void testLastValuePartition(boolean spillEnabled, boolean revokeMemoryWh Ints.asList(0), Ints.asList(2), ImmutableList.copyOf(new SortOrder[] {SortOrder.ASC_NULLS_LAST}), + new DummySpillerFactory(), spillEnabled); MaterializedResult expected = resultBuilder(driverContext.getSession(), VARCHAR, VARCHAR, BIGINT, BOOLEAN, VARCHAR) @@ -574,6 +585,7 @@ private void testNthValuePartition(boolean spillEnabled, boolean revokeMemoryWhe Ints.asList(0), Ints.asList(2), ImmutableList.copyOf(new SortOrder[] {SortOrder.ASC_NULLS_LAST}), + new DummySpillerFactory(), spillEnabled); DriverContext driverContext = createDriverContext(memoryLimit); @@ -618,6 +630,7 @@ private void testLagPartition(boolean spillEnabled, boolean revokeMemoryWhenAddi Ints.asList(0), Ints.asList(2), ImmutableList.copyOf(new SortOrder[] {SortOrder.ASC_NULLS_LAST}), + new DummySpillerFactory(), spillEnabled); DriverContext driverContext = createDriverContext(memoryLimit); @@ -662,6 +675,7 @@ private void testLeadPartition(boolean spillEnabled, boolean revokeMemoryWhenAdd Ints.asList(0), Ints.asList(2), ImmutableList.copyOf(new SortOrder[] {SortOrder.ASC_NULLS_LAST}), + new DummySpillerFactory(), spillEnabled); DriverContext driverContext = createDriverContext(memoryLimit); @@ -703,6 +717,7 @@ private void testPartiallyPreGroupedPartitionWithEmptyInput(boolean spillEnabled Ints.asList(3), ImmutableList.of(SortOrder.ASC_NULLS_LAST), 0, + new DummySpillerFactory(), spillEnabled); DriverContext driverContext = createDriverContext(memoryLimit); @@ -746,6 +761,7 @@ private void testPartiallyPreGroupedPartition(boolean spillEnabled, boolean revo Ints.asList(3), ImmutableList.of(SortOrder.ASC_NULLS_LAST), 0, + new DummySpillerFactory(), spillEnabled); DriverContext driverContext = createDriverContext(memoryLimit); @@ -796,6 +812,7 @@ private void testFullyPreGroupedPartition(boolean spillEnabled, boolean revokeMe Ints.asList(3), ImmutableList.of(SortOrder.ASC_NULLS_LAST), 0, + new DummySpillerFactory(), spillEnabled); DriverContext driverContext = createDriverContext(memoryLimit); @@ -848,6 +865,7 @@ private void testFullyPreGroupedAndPartiallySortedPartition(boolean spillEnabled Ints.asList(3, 2), ImmutableList.of(SortOrder.ASC_NULLS_LAST, SortOrder.ASC_NULLS_LAST), 1, + new DummySpillerFactory(), spillEnabled); DriverContext driverContext = createDriverContext(memoryLimit); @@ -901,6 +919,7 @@ private void testFullyPreGroupedAndFullySortedPartition(boolean spillEnabled, bo Ints.asList(3), ImmutableList.of(SortOrder.ASC_NULLS_LAST), 1, + new DummySpillerFactory(), spillEnabled); DriverContext driverContext = createDriverContext(memoryLimit); @@ -958,6 +977,7 @@ private WindowOperatorFactory createFactoryUnbounded( List partitionChannels, List sortChannels, List sortOrder, + SpillerFactory spillerFactory, boolean spillEnabled) { return createFactoryUnbounded( @@ -969,6 +989,7 @@ private WindowOperatorFactory createFactoryUnbounded( sortChannels, sortOrder, 0, + spillerFactory, spillEnabled); } @@ -981,6 +1002,7 @@ private WindowOperatorFactory createFactoryUnbounded( List sortChannels, List sortOrder, int preSortedChannelPrefix, + DummySpillerFactory spillerFactory, boolean spillEnabled) { return new WindowOperatorFactory( diff --git a/core/trino-main/src/test/java/io/trino/operator/aggregation/AbstractTestApproximateCountDistinct.java b/core/trino-main/src/test/java/io/trino/operator/aggregation/AbstractTestApproximateCountDistinct.java index d686b2b423fd4..6ee64e55a1c96 100644 --- a/core/trino-main/src/test/java/io/trino/operator/aggregation/AbstractTestApproximateCountDistinct.java +++ b/core/trino-main/src/test/java/io/trino/operator/aggregation/AbstractTestApproximateCountDistinct.java @@ -21,7 +21,7 @@ import io.trino.spi.block.BlockBuilder; import io.trino.spi.type.Type; import org.apache.commons.math3.stat.descriptive.DescriptiveStatistics; -import org.testng.annotations.Test; +import org.junit.jupiter.api.Test; import java.util.ArrayList; import java.util.Collections; diff --git a/core/trino-main/src/test/java/io/trino/operator/aggregation/TestAggregationMaskCompiler.java b/core/trino-main/src/test/java/io/trino/operator/aggregation/TestAggregationMaskCompiler.java index ad982f171207c..f9623ca027487 100644 --- a/core/trino-main/src/test/java/io/trino/operator/aggregation/TestAggregationMaskCompiler.java +++ b/core/trino-main/src/test/java/io/trino/operator/aggregation/TestAggregationMaskCompiler.java @@ -19,7 +19,7 @@ import io.trino.spi.block.IntArrayBlock; import io.trino.spi.block.RunLengthEncodedBlock; import io.trino.spi.block.ShortArrayBlock; -import org.testng.annotations.Test; +import org.junit.jupiter.api.Test; import java.util.Arrays; import java.util.Optional; diff --git a/core/trino-main/src/test/java/io/trino/operator/aggregation/TestApproximateCountDistinctBoolean.java b/core/trino-main/src/test/java/io/trino/operator/aggregation/TestApproximateCountDistinctBoolean.java index 85d57c48ced7c..ede51acce9688 100644 --- a/core/trino-main/src/test/java/io/trino/operator/aggregation/TestApproximateCountDistinctBoolean.java +++ b/core/trino-main/src/test/java/io/trino/operator/aggregation/TestApproximateCountDistinctBoolean.java @@ -16,7 +16,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.primitives.Booleans; import io.trino.spi.type.Type; -import org.testng.annotations.Test; +import org.junit.jupiter.api.Test; import java.util.concurrent.ThreadLocalRandom; diff --git a/core/trino-main/src/test/java/io/trino/operator/aggregation/TestApproximateSetGenericBoolean.java b/core/trino-main/src/test/java/io/trino/operator/aggregation/TestApproximateSetGenericBoolean.java index ee12864d5ce15..ff49acd651cbd 100644 --- a/core/trino-main/src/test/java/io/trino/operator/aggregation/TestApproximateSetGenericBoolean.java +++ b/core/trino-main/src/test/java/io/trino/operator/aggregation/TestApproximateSetGenericBoolean.java @@ -16,7 +16,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.primitives.Booleans; import io.trino.spi.type.Type; -import org.testng.annotations.Test; +import org.junit.jupiter.api.Test; import java.util.List; import java.util.concurrent.ThreadLocalRandom; diff --git a/core/trino-main/src/test/java/io/trino/operator/aggregation/TestDecimalAverageAggregation.java b/core/trino-main/src/test/java/io/trino/operator/aggregation/TestDecimalAverageAggregation.java index 0a5c9ee2040a5..1c6cb3d5a3d2c 100644 --- a/core/trino-main/src/test/java/io/trino/operator/aggregation/TestDecimalAverageAggregation.java +++ b/core/trino-main/src/test/java/io/trino/operator/aggregation/TestDecimalAverageAggregation.java @@ -21,8 +21,7 @@ import io.trino.spi.type.DecimalType; import io.trino.spi.type.Decimals; import io.trino.spi.type.Int128; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.Test; +import org.junit.jupiter.api.Test; import java.math.BigDecimal; import java.math.BigInteger; @@ -37,7 +36,6 @@ import static java.math.RoundingMode.HALF_UP; import static org.assertj.core.api.Assertions.assertThat; -@Test(singleThreaded = true) public class TestDecimalAverageAggregation { private static final BigInteger TWO = new BigInteger("2"); @@ -45,17 +43,11 @@ public class TestDecimalAverageAggregation private static final BigInteger TWO_HUNDRED = new BigInteger("200"); private static final DecimalType TYPE = createDecimalType(38, 0); - private LongDecimalWithOverflowAndLongState state; - - @BeforeMethod - public void setUp() - { - state = new LongDecimalWithOverflowAndLongStateFactory().createSingleState(); - } - @Test public void testOverflow() { + LongDecimalWithOverflowAndLongState state = new LongDecimalWithOverflowAndLongStateFactory().createSingleState(); + addToState(state, TWO.pow(126)); assertThat(state.getLong()).isEqualTo(1); @@ -68,12 +60,14 @@ public void testOverflow() assertThat(state.getOverflow()).isEqualTo(1); assertThat(getDecimal(state)).isEqualTo(Int128.valueOf(1L << 63, 0)); - assertAverageEquals(TWO.pow(126)); + assertAverageEquals(state, TWO.pow(126)); } @Test public void testUnderflow() { + LongDecimalWithOverflowAndLongState state = new LongDecimalWithOverflowAndLongStateFactory().createSingleState(); + addToState(state, Decimals.MIN_UNSCALED_DECIMAL.toBigInteger()); assertThat(state.getLong()).isEqualTo(1); @@ -86,12 +80,14 @@ public void testUnderflow() assertThat(state.getOverflow()).isEqualTo(-1); assertThat(getDecimal(state)).isEqualTo(Int128.valueOf(0x698966AF4AF2770BL, 0xECEBBB8000000002L)); - assertAverageEquals(Decimals.MIN_UNSCALED_DECIMAL.toBigInteger()); + assertAverageEquals(state, Decimals.MIN_UNSCALED_DECIMAL.toBigInteger()); } @Test public void testUnderflowAfterOverflow() { + LongDecimalWithOverflowAndLongState state = new LongDecimalWithOverflowAndLongStateFactory().createSingleState(); + addToState(state, TWO.pow(126)); addToState(state, TWO.pow(126)); addToState(state, TWO.pow(125)); @@ -106,12 +102,14 @@ public void testUnderflowAfterOverflow() assertThat(state.getOverflow()).isEqualTo(0); assertThat(getDecimal(state)).isEqualTo(Int128.valueOf(TWO.pow(125).negate())); - assertAverageEquals(TWO.pow(125).negate().divide(BigInteger.valueOf(6))); + assertAverageEquals(state, TWO.pow(125).negate().divide(BigInteger.valueOf(6))); } @Test public void testCombineOverflow() { + LongDecimalWithOverflowAndLongState state = new LongDecimalWithOverflowAndLongStateFactory().createSingleState(); + addToState(state, TWO.pow(126)); addToState(state, TWO.pow(126)); @@ -132,7 +130,7 @@ public void testCombineOverflow() .add(TWO.pow(126)) .divide(BigInteger.valueOf(4)); - assertAverageEquals(expectedAverage); + assertAverageEquals(state, expectedAverage); } @Test @@ -161,7 +159,7 @@ public void testCombineUnderflow() .negate() .divide(BigInteger.valueOf(4)); - assertAverageEquals(expectedAverage); + assertAverageEquals(state, expectedAverage); } @Test @@ -217,12 +215,12 @@ private static BigDecimal decodeBigDecimal(DecimalType type, Int128 average) return new BigDecimal(unscaledVal, type.getScale(), new MathContext(type.getPrecision())); } - private void assertAverageEquals(BigInteger expectedAverage) + private void assertAverageEquals(LongDecimalWithOverflowAndLongState state, BigInteger expectedAverage) { - assertAverageEquals(expectedAverage, TYPE); + assertAverageEquals(state, expectedAverage, TYPE); } - private void assertAverageEquals(BigInteger expectedAverage, DecimalType type) + private void assertAverageEquals(LongDecimalWithOverflowAndLongState state, BigInteger expectedAverage, DecimalType type) { assertThat(average(state, type).toBigInteger()).isEqualTo(expectedAverage); } diff --git a/core/trino-main/src/test/java/io/trino/operator/aggregation/state/TestLongDecimalWithOverflowAndLongStateSerializer.java b/core/trino-main/src/test/java/io/trino/operator/aggregation/state/TestLongDecimalWithOverflowAndLongStateSerializer.java index 8f56532e775f1..fbdafaa434bd9 100644 --- a/core/trino-main/src/test/java/io/trino/operator/aggregation/state/TestLongDecimalWithOverflowAndLongStateSerializer.java +++ b/core/trino-main/src/test/java/io/trino/operator/aggregation/state/TestLongDecimalWithOverflowAndLongStateSerializer.java @@ -16,7 +16,7 @@ import io.trino.spi.block.Block; import io.trino.spi.block.BlockBuilder; import io.trino.spi.block.VariableWidthBlockBuilder; -import org.testng.annotations.Test; +import org.junit.jupiter.api.Test; import static org.assertj.core.api.Assertions.assertThat; diff --git a/core/trino-main/src/test/java/io/trino/operator/aggregation/state/TestLongDecimalWithOverflowStateSerializer.java b/core/trino-main/src/test/java/io/trino/operator/aggregation/state/TestLongDecimalWithOverflowStateSerializer.java index 6833785bc4ee5..4c33f3016bfe5 100644 --- a/core/trino-main/src/test/java/io/trino/operator/aggregation/state/TestLongDecimalWithOverflowStateSerializer.java +++ b/core/trino-main/src/test/java/io/trino/operator/aggregation/state/TestLongDecimalWithOverflowStateSerializer.java @@ -16,7 +16,7 @@ import io.trino.spi.block.Block; import io.trino.spi.block.BlockBuilder; import io.trino.spi.block.VariableWidthBlockBuilder; -import org.testng.annotations.Test; +import org.junit.jupiter.api.Test; import static org.assertj.core.api.Assertions.assertThat; diff --git a/core/trino-main/src/test/java/io/trino/operator/exchange/TestLocalExchange.java b/core/trino-main/src/test/java/io/trino/operator/exchange/TestLocalExchange.java index 2e28dd74e8a85..9f8cdedef0c0c 100644 --- a/core/trino-main/src/test/java/io/trino/operator/exchange/TestLocalExchange.java +++ b/core/trino-main/src/test/java/io/trino/operator/exchange/TestLocalExchange.java @@ -41,8 +41,10 @@ import io.trino.sql.planner.PartitioningHandle; import io.trino.testing.TestingTransactionHandle; import io.trino.util.FinalizerService; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.parallel.Execution; import java.util.List; import java.util.Optional; @@ -72,8 +74,11 @@ import static io.trino.testing.TestingSession.testSessionBuilder; import static java.util.stream.IntStream.range; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_METHOD; +import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; -@Test(singleThreaded = true) +@TestInstance(PER_METHOD) +@Execution(SAME_THREAD) public class TestLocalExchange { private static final List TYPES = ImmutableList.of(BIGINT); @@ -89,7 +94,7 @@ public class TestLocalExchange private NodePartitioningManager nodePartitioningManager; private final PartitioningHandle customScalingPartitioningHandle = getCustomScalingPartitioningHandle(); - @BeforeMethod + @BeforeEach public void setUp() { NodeScheduler nodeScheduler = new NodeScheduler(new UniformNodeSelectorFactory( diff --git a/core/trino-main/src/test/java/io/trino/operator/join/TestHashJoinOperator.java b/core/trino-main/src/test/java/io/trino/operator/join/TestHashJoinOperator.java index 7d4f0f8ea181a..e4ac6f8ec309e 100644 --- a/core/trino-main/src/test/java/io/trino/operator/join/TestHashJoinOperator.java +++ b/core/trino-main/src/test/java/io/trino/operator/join/TestHashJoinOperator.java @@ -62,9 +62,11 @@ import io.trino.testing.MaterializedResult; import io.trino.testing.TestingTaskContext; import io.trino.util.FinalizerService; -import org.testng.annotations.AfterMethod; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.parallel.Execution; import java.util.ArrayList; import java.util.List; @@ -72,8 +74,6 @@ import java.util.OptionalInt; import java.util.concurrent.ExecutorService; import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.SynchronousQueue; -import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -108,13 +108,17 @@ import static java.util.Collections.nCopies; import static java.util.Collections.singletonList; import static java.util.Objects.requireNonNull; +import static java.util.concurrent.Executors.newCachedThreadPool; import static java.util.concurrent.Executors.newScheduledThreadPool; import static java.util.concurrent.TimeUnit.NANOSECONDS; import static java.util.concurrent.TimeUnit.SECONDS; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; -@Test(singleThreaded = true) +@TestInstance(PER_CLASS) +@Execution(CONCURRENT) public class TestHashJoinOperator { private static final int PARTITION_COUNT = 4; @@ -122,40 +126,17 @@ public class TestHashJoinOperator private static final PartitioningSpillerFactory PARTITIONING_SPILLER_FACTORY = new GenericPartitioningSpillerFactory(SINGLE_STREAM_SPILLER_FACTORY); private static final TypeOperators TYPE_OPERATORS = new TypeOperators(); - private ExecutorService executor; - private ScheduledExecutorService scheduledExecutor; - private NodePartitioningManager nodePartitioningManager; - - @BeforeMethod - public void setUp() - { - // Before/AfterMethod is chosen here because the executor needs to be shutdown - // after every single test case to terminate outstanding threads, if any. - - // The line below is the same as newCachedThreadPool(daemonThreadsNamed(...)) except RejectionExecutionHandler. - // RejectionExecutionHandler is set to DiscardPolicy (instead of the default AbortPolicy) here. - // Otherwise, a large number of RejectedExecutionException will flood logging, resulting in Travis failure. - executor = new ThreadPoolExecutor( - 0, - Integer.MAX_VALUE, - 60L, - SECONDS, - new SynchronousQueue<>(), - daemonThreadsNamed("test-executor-%s"), - new ThreadPoolExecutor.DiscardPolicy()); - scheduledExecutor = newScheduledThreadPool(2, daemonThreadsNamed(getClass().getSimpleName() + "-scheduledExecutor-%s")); - - NodeScheduler nodeScheduler = new NodeScheduler(new UniformNodeSelectorFactory( - new InMemoryNodeManager(), - new NodeSchedulerConfig().setIncludeCoordinator(true), - new NodeTaskMap(new FinalizerService()))); - nodePartitioningManager = new NodePartitioningManager( - nodeScheduler, - TYPE_OPERATORS, - CatalogServiceProvider.fail()); - } - - @AfterMethod(alwaysRun = true) + private final ExecutorService executor = newCachedThreadPool(daemonThreadsNamed("test-executor-%s")); + private final ScheduledExecutorService scheduledExecutor = newScheduledThreadPool(2, daemonThreadsNamed(getClass().getSimpleName() + "-scheduledExecutor-%s")); + private final NodePartitioningManager nodePartitioningManager = new NodePartitioningManager( + new NodeScheduler(new UniformNodeSelectorFactory( + new InMemoryNodeManager(), + new NodeSchedulerConfig().setIncludeCoordinator(true), + new NodeTaskMap(new FinalizerService()))), + TYPE_OPERATORS, + CatalogServiceProvider.fail()); + + @AfterAll public void tearDown() { executor.shutdownNow(); @@ -624,7 +605,8 @@ private static MaterializedResult getProperColumns(Operator joinOperator, List(), - daemonThreadsNamed("test-executor-%s"), - new ThreadPoolExecutor.DiscardPolicy()); - scheduledExecutor = newScheduledThreadPool(2, daemonThreadsNamed(getClass().getSimpleName() + "-scheduledExecutor-%s")); - - NodeScheduler nodeScheduler = new NodeScheduler(new UniformNodeSelectorFactory( - new InMemoryNodeManager(), - new NodeSchedulerConfig().setIncludeCoordinator(true), - new NodeTaskMap(new FinalizerService()))); - nodePartitioningManager = new NodePartitioningManager( - nodeScheduler, - TYPE_OPERATORS, - CatalogServiceProvider.fail()); - } - - @AfterMethod(alwaysRun = true) + private final ExecutorService executor = newCachedThreadPool(daemonThreadsNamed("test-executor-%s")); + private final ScheduledExecutorService scheduledExecutor = newScheduledThreadPool(2, daemonThreadsNamed(getClass().getSimpleName() + "-scheduledExecutor-%s")); + private final NodePartitioningManager nodePartitioningManager = new NodePartitioningManager( + new NodeScheduler(new UniformNodeSelectorFactory( + new InMemoryNodeManager(), + new NodeSchedulerConfig().setIncludeCoordinator(true), + new NodeTaskMap(new FinalizerService()))), + TYPE_OPERATORS, + CatalogServiceProvider.fail()); + + @AfterAll public void tearDown() { executor.shutdownNow(); diff --git a/core/trino-main/src/test/java/io/trino/operator/output/TestPagePartitioner.java b/core/trino-main/src/test/java/io/trino/operator/output/TestPagePartitioner.java index fce46ff49eff1..c16c6c200c7bb 100644 --- a/core/trino-main/src/test/java/io/trino/operator/output/TestPagePartitioner.java +++ b/core/trino-main/src/test/java/io/trino/operator/output/TestPagePartitioner.java @@ -49,9 +49,10 @@ import io.trino.sql.planner.plan.PlanNodeId; import io.trino.testing.TestingTaskContext; import io.trino.type.BlockTypeOperators; -import org.testng.annotations.AfterClass; -import org.testng.annotations.BeforeClass; -import org.testng.annotations.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.parallel.Execution; import java.util.ArrayList; import java.util.Collection; @@ -95,8 +96,11 @@ import static java.util.concurrent.Executors.newScheduledThreadPool; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; -@Test(singleThreaded = true) +@TestInstance(PER_CLASS) +@Execution(SAME_THREAD) public class TestPagePartitioner { private static final DataSize MAX_MEMORY = DataSize.of(50, MEGABYTE); @@ -108,23 +112,14 @@ public class TestPagePartitioner private static final PagesSerdeFactory PAGES_SERDE_FACTORY = new PagesSerdeFactory(new TestingBlockEncodingSerde(), false); private static final PageDeserializer PAGE_DESERIALIZER = PAGES_SERDE_FACTORY.createDeserializer(Optional.empty()); - private ExecutorService executor; - private ScheduledExecutorService scheduledExecutor; + private final ExecutorService executor = newCachedThreadPool(daemonThreadsNamed(getClass().getSimpleName() + "-executor-%s")); + private final ScheduledExecutorService scheduledExecutor = newScheduledThreadPool(1, daemonThreadsNamed(getClass().getSimpleName() + "-scheduledExecutor-%s")); - @BeforeClass - public void setUpClass() - { - executor = newCachedThreadPool(daemonThreadsNamed(getClass().getSimpleName() + "-executor-%s")); - scheduledExecutor = newScheduledThreadPool(1, daemonThreadsNamed(getClass().getSimpleName() + "-scheduledExecutor-%s")); - } - - @AfterClass(alwaysRun = true) + @AfterAll public void tearDownClass() { executor.shutdownNow(); - executor = null; scheduledExecutor.shutdownNow(); - scheduledExecutor = null; } @Test diff --git a/core/trino-main/src/test/java/io/trino/operator/output/TestPositionsAppender.java b/core/trino-main/src/test/java/io/trino/operator/output/TestPositionsAppender.java index b320ccac48291..01ece57858d9a 100644 --- a/core/trino-main/src/test/java/io/trino/operator/output/TestPositionsAppender.java +++ b/core/trino-main/src/test/java/io/trino/operator/output/TestPositionsAppender.java @@ -41,7 +41,7 @@ import io.trino.spi.type.VarcharType; import io.trino.type.BlockTypeOperators; import it.unimi.dsi.fastutil.ints.IntArrayList; -import org.testng.annotations.Test; +import org.junit.jupiter.api.Test; import java.util.List; import java.util.Optional; @@ -349,10 +349,7 @@ public void testConsecutiveBuilds() } } - // testcase for jit bug described https://github.com/trinodb/trino/issues/12821. - // this test needs to be run first (hence the lowest priority) as the test order - // influences jit compilation, making this problem to not occur if other tests are run first. - @Test(priority = Integer.MIN_VALUE) + @Test public void testSliceRle() { UnnestingPositionsAppender positionsAppender = POSITIONS_APPENDER_FACTORY.create(VARCHAR, 10, DEFAULT_MAX_PAGE_SIZE_IN_BYTES); diff --git a/core/trino-main/src/test/java/io/trino/operator/project/TestDictionaryAwarePageProjection.java b/core/trino-main/src/test/java/io/trino/operator/project/TestDictionaryAwarePageProjection.java index a42bcaaa6e617..069feb0b1169e 100644 --- a/core/trino-main/src/test/java/io/trino/operator/project/TestDictionaryAwarePageProjection.java +++ b/core/trino-main/src/test/java/io/trino/operator/project/TestDictionaryAwarePageProjection.java @@ -26,8 +26,10 @@ import io.trino.spi.block.ValueBlock; import io.trino.spi.connector.ConnectorSession; import io.trino.spi.type.Type; -import org.testng.annotations.AfterClass; -import org.testng.annotations.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.parallel.Execution; import java.util.Arrays; import java.util.concurrent.ScheduledExecutorService; @@ -44,12 +46,16 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.Fail.fail; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; +@TestInstance(PER_CLASS) +@Execution(CONCURRENT) public class TestDictionaryAwarePageProjection { private static final ScheduledExecutorService executor = newSingleThreadScheduledExecutor(daemonThreadsNamed("TestDictionaryAwarePageProjection-%s")); - @AfterClass(alwaysRun = true) + @AfterAll public void tearDown() { executor.shutdownNow(); diff --git a/core/trino-main/src/test/java/io/trino/operator/scalar/TestScalarValidation.java b/core/trino-main/src/test/java/io/trino/operator/scalar/TestScalarValidation.java index 3e4927d963877..e73030b32fa14 100644 --- a/core/trino-main/src/test/java/io/trino/operator/scalar/TestScalarValidation.java +++ b/core/trino-main/src/test/java/io/trino/operator/scalar/TestScalarValidation.java @@ -24,15 +24,19 @@ import io.trino.spi.type.StandardTypes; import io.trino.spi.type.Type; import jakarta.annotation.Nullable; -import org.testng.annotations.Test; +import org.junit.jupiter.api.Test; + +import static org.assertj.core.api.Assertions.assertThatThrownBy; @SuppressWarnings("UtilityClassWithoutPrivateConstructor") public class TestScalarValidation { - @Test(expectedExceptions = IllegalArgumentException.class, expectedExceptionsMessageRegExp = "Parametric class method .* is annotated with @ScalarFunction") + @Test public void testBogusParametricMethodAnnotation() { - extractParametricScalar(BogusParametricMethodAnnotation.class); + assertThatThrownBy(() -> extractParametricScalar(BogusParametricMethodAnnotation.class)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageMatching("Parametric class method .* is annotated with @ScalarFunction"); } @ScalarFunction @@ -42,20 +46,24 @@ public static final class BogusParametricMethodAnnotation public static void bad() {} } - @Test(expectedExceptions = TrinoException.class, expectedExceptionsMessageRegExp = "Parametric class .* does not have any annotated methods") + @Test public void testNoParametricMethods() { - extractParametricScalar(NoParametricMethods.class); + assertThatThrownBy(() -> extractParametricScalar(NoParametricMethods.class)) + .isInstanceOf(TrinoException.class) + .hasMessageMatching("Parametric class .* does not have any annotated methods"); } @SuppressWarnings("EmptyClass") @ScalarFunction public static final class NoParametricMethods {} - @Test(expectedExceptions = IllegalArgumentException.class, expectedExceptionsMessageRegExp = "Method .* is missing @SqlType annotation") + @Test public void testMethodMissingReturnAnnotation() { - extractScalars(MethodMissingReturnAnnotation.class); + assertThatThrownBy(() -> extractScalars(MethodMissingReturnAnnotation.class)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageMatching("Method .* is missing @SqlType annotation"); } public static final class MethodMissingReturnAnnotation @@ -64,10 +72,12 @@ public static final class MethodMissingReturnAnnotation public static void bad() {} } - @Test(expectedExceptions = TrinoException.class, expectedExceptionsMessageRegExp = "Method .* annotated with @SqlType is missing @ScalarFunction or @ScalarOperator") + @Test public void testMethodMissingScalarAnnotation() { - extractScalars(MethodMissingScalarAnnotation.class); + assertThatThrownBy(() -> extractScalars(MethodMissingScalarAnnotation.class)) + .isInstanceOf(TrinoException.class) + .hasMessageMatching("Method .* annotated with @SqlType is missing @ScalarFunction or @ScalarOperator"); } public static final class MethodMissingScalarAnnotation @@ -77,10 +87,12 @@ public static final class MethodMissingScalarAnnotation public static void bad() {} } - @Test(expectedExceptions = IllegalArgumentException.class, expectedExceptionsMessageRegExp = "Method .* has wrapper return type Long but is missing @SqlNullable") + @Test public void testPrimitiveWrapperReturnWithoutNullable() { - extractScalars(PrimitiveWrapperReturnWithoutNullable.class); + assertThatThrownBy(() -> extractScalars(PrimitiveWrapperReturnWithoutNullable.class)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageMatching("Method .* has wrapper return type Long but is missing @SqlNullable"); } public static final class PrimitiveWrapperReturnWithoutNullable @@ -93,10 +105,12 @@ public static Long bad() } } - @Test(expectedExceptions = IllegalArgumentException.class, expectedExceptionsMessageRegExp = "Method .* annotated with @SqlNullable has primitive return type long") + @Test public void testPrimitiveReturnWithNullable() { - extractScalars(PrimitiveReturnWithNullable.class); + assertThatThrownBy(() -> extractScalars(PrimitiveReturnWithNullable.class)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageMatching("Method .* annotated with @SqlNullable has primitive return type long"); } public static final class PrimitiveReturnWithNullable @@ -110,10 +124,12 @@ public static long bad() } } - @Test(expectedExceptions = TrinoException.class, expectedExceptionsMessageRegExp = "A parameter with USE_NULL_FLAG or RETURN_NULL_ON_NULL convention must not use wrapper type. Found in method .*") + @Test public void testPrimitiveWrapperParameterWithoutNullable() { - extractScalars(PrimitiveWrapperParameterWithoutNullable.class); + assertThatThrownBy(() -> extractScalars(PrimitiveWrapperParameterWithoutNullable.class)) + .isInstanceOf(TrinoException.class) + .hasMessageMatching("A parameter with USE_NULL_FLAG or RETURN_NULL_ON_NULL convention must not use wrapper type. Found in method .*"); } public static final class PrimitiveWrapperParameterWithoutNullable @@ -126,10 +142,12 @@ public static long bad(@SqlType(StandardTypes.BOOLEAN) Boolean boxed) } } - @Test(expectedExceptions = TrinoException.class, expectedExceptionsMessageRegExp = "Method .* has parameter with primitive type double annotated with @SqlNullable") + @Test public void testPrimitiveParameterWithNullable() { - extractScalars(PrimitiveParameterWithNullable.class); + assertThatThrownBy(() -> extractScalars(PrimitiveParameterWithNullable.class)) + .isInstanceOf(TrinoException.class) + .hasMessageMatching("Method .* has parameter with primitive type double annotated with @SqlNullable"); } public static final class PrimitiveParameterWithNullable @@ -142,10 +160,12 @@ public static long bad(@SqlNullable @SqlType(StandardTypes.DOUBLE) double primit } } - @Test(expectedExceptions = IllegalArgumentException.class, expectedExceptionsMessageRegExp = "Method .* is missing @SqlType annotation for parameter") + @Test public void testParameterWithoutType() { - extractScalars(ParameterWithoutType.class); + assertThatThrownBy(() -> extractScalars(ParameterWithoutType.class)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageMatching("Method .* is missing @SqlType annotation for parameter"); } public static final class ParameterWithoutType @@ -158,10 +178,12 @@ public static long bad(long missing) } } - @Test(expectedExceptions = IllegalArgumentException.class, expectedExceptionsMessageRegExp = "Method .* annotated with @ScalarFunction must be public") + @Test public void testNonPublicAnnnotatedMethod() { - extractScalars(NonPublicAnnnotatedMethod.class); + assertThatThrownBy(() -> extractScalars(NonPublicAnnnotatedMethod.class)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageMatching("Method .* annotated with @ScalarFunction must be public"); } public static final class NonPublicAnnnotatedMethod @@ -174,10 +196,12 @@ private static long bad() } } - @Test(expectedExceptions = IllegalArgumentException.class, expectedExceptionsMessageRegExp = "Method .* is annotated with @Nullable but not @SqlNullable") + @Test public void testMethodWithLegacyNullable() { - extractScalars(MethodWithLegacyNullable.class); + assertThatThrownBy(() -> extractScalars(MethodWithLegacyNullable.class)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageMatching("Method .* is annotated with @Nullable but not @SqlNullable"); } public static final class MethodWithLegacyNullable @@ -191,10 +215,12 @@ public static Long bad() } } - @Test(expectedExceptions = IllegalArgumentException.class, expectedExceptionsMessageRegExp = "Method .* has @IsNull parameter that does not follow a @SqlType parameter") + @Test public void testParameterWithConnectorAndIsNull() { - extractScalars(ParameterWithConnectorAndIsNull.class); + assertThatThrownBy(() -> extractScalars(ParameterWithConnectorAndIsNull.class)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageMatching("Method .* has @IsNull parameter that does not follow a @SqlType parameter"); } public static final class ParameterWithConnectorAndIsNull @@ -207,10 +233,12 @@ public static long bad(ConnectorSession session, @IsNull boolean isNull) } } - @Test(expectedExceptions = IllegalArgumentException.class, expectedExceptionsMessageRegExp = "Method .* has @IsNull parameter that does not follow a @SqlType parameter") + @Test public void testParameterWithOnlyIsNull() { - extractScalars(ParameterWithOnlyIsNull.class); + assertThatThrownBy(() -> extractScalars(ParameterWithOnlyIsNull.class)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageMatching("Method .* has @IsNull parameter that does not follow a @SqlType parameter"); } public static final class ParameterWithOnlyIsNull @@ -223,10 +251,12 @@ public static long bad(@IsNull boolean isNull) } } - @Test(expectedExceptions = IllegalArgumentException.class, expectedExceptionsMessageRegExp = "Method .* has non-boolean parameter with @IsNull") + @Test public void testParameterWithNonBooleanIsNull() { - extractScalars(ParameterWithNonBooleanIsNull.class); + assertThatThrownBy(() -> extractScalars(ParameterWithNonBooleanIsNull.class)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageMatching("Method .* has non-boolean parameter with @IsNull"); } public static final class ParameterWithNonBooleanIsNull @@ -239,10 +269,12 @@ public static long bad(@SqlType(StandardTypes.BIGINT) long value, @IsNull int is } } - @Test(expectedExceptions = TrinoException.class, expectedExceptionsMessageRegExp = "A parameter with USE_NULL_FLAG or RETURN_NULL_ON_NULL convention must not use wrapper type. Found in method .*") + @Test public void testParameterWithBoxedPrimitiveIsNull() { - extractScalars(ParameterWithBoxedPrimitiveIsNull.class); + assertThatThrownBy(() -> extractScalars(ParameterWithBoxedPrimitiveIsNull.class)) + .isInstanceOf(TrinoException.class) + .hasMessageMatching("A parameter with USE_NULL_FLAG or RETURN_NULL_ON_NULL convention must not use wrapper type. Found in method .*"); } public static final class ParameterWithBoxedPrimitiveIsNull @@ -255,10 +287,12 @@ public static long bad(@SqlType(StandardTypes.BIGINT) Long value, @IsNull boolea } } - @Test(expectedExceptions = IllegalArgumentException.class, expectedExceptionsMessageRegExp = "Method .* has @IsNull parameter that has other annotations") + @Test public void testParameterWithOtherAnnotationsWithIsNull() { - extractScalars(ParameterWithOtherAnnotationsWithIsNull.class); + assertThatThrownBy(() -> extractScalars(ParameterWithOtherAnnotationsWithIsNull.class)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageMatching("Method .* has @IsNull parameter that has other annotations"); } public static final class ParameterWithOtherAnnotationsWithIsNull @@ -271,10 +305,12 @@ public static long bad(@SqlType(StandardTypes.BIGINT) long value, @IsNull @SqlNu } } - @Test(expectedExceptions = IllegalArgumentException.class, expectedExceptionsMessageRegExp = "Expected type parameter to only contain A-Z and 0-9 \\(starting with A-Z\\), but got bad on method .*") + @Test public void testNonUpperCaseTypeParameters() { - extractScalars(TypeParameterWithNonUpperCaseAnnotation.class); + assertThatThrownBy(() -> extractScalars(TypeParameterWithNonUpperCaseAnnotation.class)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageMatching("Expected type parameter to only contain A-Z and 0-9 \\(starting with A-Z\\), but got bad on method .*"); } public static final class TypeParameterWithNonUpperCaseAnnotation @@ -288,10 +324,12 @@ public static long bad(@TypeParameter("array(bad)") Type type, @SqlType(Standard } } - @Test(expectedExceptions = IllegalArgumentException.class, expectedExceptionsMessageRegExp = "Expected type parameter to only contain A-Z and 0-9 \\(starting with A-Z\\), but got 1E on method .*") + @Test public void testLeadingNumericTypeParameters() { - extractScalars(TypeParameterWithLeadingNumbers.class); + assertThatThrownBy(() -> extractScalars(TypeParameterWithLeadingNumbers.class)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageMatching("Expected type parameter to only contain A-Z and 0-9 \\(starting with A-Z\\), but got 1E on method .*"); } public static final class TypeParameterWithLeadingNumbers @@ -305,10 +343,12 @@ public static long bad(@TypeParameter("array(1E)") Type type, @SqlType(StandardT } } - @Test(expectedExceptions = IllegalArgumentException.class, expectedExceptionsMessageRegExp = "Expected type parameter not to take parameters, but got 'e' on method .*") + @Test public void testNonPrimitiveTypeParameters() { - extractScalars(TypeParameterWithNonPrimitiveAnnotation.class); + assertThatThrownBy(() -> extractScalars(TypeParameterWithNonPrimitiveAnnotation.class)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageMatching("Expected type parameter not to take parameters, but got 'e' on method .*"); } public static final class TypeParameterWithNonPrimitiveAnnotation @@ -357,10 +397,12 @@ public void testValidTypeParametersForConstructors() extractParametricScalar(ConstructorWithValidTypeParameters.class); } - @Test(expectedExceptions = IllegalArgumentException.class, expectedExceptionsMessageRegExp = "Expected type parameter not to take parameters, but got 'k' on method .*") + @Test public void testInvalidTypeParametersForConstructors() { - extractParametricScalar(ConstructorWithInvalidTypeParameters.class); + assertThatThrownBy(() -> extractParametricScalar(ConstructorWithInvalidTypeParameters.class)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageMatching("Expected type parameter not to take parameters, but got 'k' on method .*"); } private static void extractParametricScalar(Class clazz) diff --git a/core/trino-main/src/test/java/io/trino/server/security/TestResourceSecurity.java b/core/trino-main/src/test/java/io/trino/server/security/TestResourceSecurity.java index cdb476f884d2d..e46f873524460 100644 --- a/core/trino-main/src/test/java/io/trino/server/security/TestResourceSecurity.java +++ b/core/trino-main/src/test/java/io/trino/server/security/TestResourceSecurity.java @@ -58,8 +58,10 @@ import okhttp3.OkHttpClient; import okhttp3.Request; import okhttp3.Response; -import org.testng.annotations.BeforeClass; -import org.testng.annotations.Test; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.parallel.Execution; import javax.crypto.SecretKey; @@ -121,7 +123,11 @@ import static java.util.Objects.requireNonNull; import static java.util.concurrent.TimeUnit.MINUTES; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; +@TestInstance(PER_CLASS) +@Execution(CONCURRENT) public class TestResourceSecurity { private static final String LOCALHOST_KEYSTORE = Resources.getResource("cert/localhost.pem").getPath(); @@ -163,7 +169,7 @@ public class TestResourceSecurity private OkHttpClient client; private Path passwordConfigDummy; - @BeforeClass + @BeforeAll public void setup() throws IOException { diff --git a/core/trino-main/src/test/java/io/trino/server/security/oauth2/TestJweTokenSerializer.java b/core/trino-main/src/test/java/io/trino/server/security/oauth2/TestJweTokenSerializer.java index 01e3fb0be12c3..918cd48e784ef 100644 --- a/core/trino-main/src/test/java/io/trino/server/security/oauth2/TestJweTokenSerializer.java +++ b/core/trino-main/src/test/java/io/trino/server/security/oauth2/TestJweTokenSerializer.java @@ -18,7 +18,7 @@ import io.jsonwebtoken.ExpiredJwtException; import io.jsonwebtoken.Jwts; import io.trino.server.security.oauth2.TokenPairSerializer.TokenPair; -import org.testng.annotations.Test; +import org.junit.jupiter.api.Test; import java.net.URI; import java.security.GeneralSecurityException; diff --git a/core/trino-main/src/test/java/io/trino/server/security/oauth2/TestOidcDiscovery.java b/core/trino-main/src/test/java/io/trino/server/security/oauth2/TestOidcDiscovery.java index ee7d76dce6b26..ee2871dcf2847 100644 --- a/core/trino-main/src/test/java/io/trino/server/security/oauth2/TestOidcDiscovery.java +++ b/core/trino-main/src/test/java/io/trino/server/security/oauth2/TestOidcDiscovery.java @@ -29,7 +29,7 @@ import jakarta.servlet.http.HttpServlet; import jakarta.servlet.http.HttpServletRequest; import jakarta.servlet.http.HttpServletResponse; -import org.testng.annotations.Test; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.net.URI; diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/TestEffectivePredicateExtractor.java b/core/trino-main/src/test/java/io/trino/sql/planner/TestEffectivePredicateExtractor.java index 32a3e4bc03213..d46dcfd2abd71 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/TestEffectivePredicateExtractor.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/TestEffectivePredicateExtractor.java @@ -76,8 +76,10 @@ import io.trino.testing.TestingSession; import io.trino.testing.TestingTransactionHandle; import io.trino.transaction.TestingTransactionManager; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.parallel.Execution; import java.util.Arrays; import java.util.Collection; @@ -115,8 +117,11 @@ import static io.trino.transaction.TransactionBuilder.transaction; import static io.trino.type.UnknownType.UNKNOWN; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_METHOD; +import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; -@Test(singleThreaded = true) +@TestInstance(PER_METHOD) +@Execution(SAME_THREAD) public class TestEffectivePredicateExtractor { private static final Symbol A = new Symbol("a"); @@ -176,7 +181,7 @@ public TableProperties getTableProperties(Session session, TableHandle handle) private TableScanNode baseTableScan; private ExpressionIdentityNormalizer expressionNormalizer; - @BeforeMethod + @BeforeEach public void setUp() { scanAssignments = ImmutableMap.builder() diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/planprinter/TestCounterBasedAnonymizer.java b/core/trino-main/src/test/java/io/trino/sql/planner/planprinter/TestCounterBasedAnonymizer.java index bb7d8f8726949..6380bffab53e8 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/planprinter/TestCounterBasedAnonymizer.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/planprinter/TestCounterBasedAnonymizer.java @@ -29,7 +29,7 @@ import io.trino.sql.tree.SymbolReference; import io.trino.sql.tree.TimeLiteral; import io.trino.sql.tree.TimestampLiteral; -import org.testng.annotations.Test; +import org.junit.jupiter.api.Test; import java.util.Optional; @@ -71,42 +71,42 @@ public void testLiteralAnonymization() .isEqualTo("'binary_literal_1'"); assertThat(anonymizer.anonymize(new StringLiteral("abc"))) - .isEqualTo("'string_literal_1'"); + .isEqualTo("'string_literal_2'"); assertThat(anonymizer.anonymize(new GenericLiteral("bigint", "1"))) - .isEqualTo("'bigint_literal_1'"); + .isEqualTo("'bigint_literal_3'"); assertThat(anonymizer.anonymize(new CharLiteral("a"))) - .isEqualTo("'char_literal_1'"); + .isEqualTo("'char_literal_4'"); assertThat(anonymizer.anonymize(new DecimalLiteral("123"))) - .isEqualTo("'decimal_literal_1'"); + .isEqualTo("'decimal_literal_5'"); assertThat(anonymizer.anonymize(new DoubleLiteral(String.valueOf(6554)))) - .isEqualTo("'double_literal_1'"); + .isEqualTo("'double_literal_6'"); assertThat(anonymizer.anonymize(new DoubleLiteral(String.valueOf(Double.MAX_VALUE)))) - .isEqualTo("'double_literal_1'"); + .isEqualTo("'double_literal_7'"); assertThat(anonymizer.anonymize(new LongLiteral(String.valueOf(6554)))) - .isEqualTo("'long_literal_1'"); + .isEqualTo("'long_literal_8'"); assertThat(anonymizer.anonymize(new LongLiteral(String.valueOf(Long.MAX_VALUE)))) - .isEqualTo("'long_literal_1'"); + .isEqualTo("'long_literal_9'"); assertThat(anonymizer.anonymize(new BooleanLiteral("true"))) .isEqualTo("true"); assertThat(anonymizer.anonymize(new TimeLiteral("03:04:05"))) - .isEqualTo("'time_literal_1'"); + .isEqualTo("'time_literal_10'"); assertThat(anonymizer.anonymize(new TimestampLiteral("2012-10-31 01:00 UTC"))) - .isEqualTo("'timestamp_literal_1'"); + .isEqualTo("'timestamp_literal_11'"); assertThat(anonymizer.anonymize(new NullLiteral())) .isEqualTo("null"); assertThat(anonymizer.anonymize(new IntervalLiteral("33", IntervalLiteral.Sign.POSITIVE, IntervalLiteral.IntervalField.DAY, Optional.empty()))) - .isEqualTo("'interval_literal_1'"); + .isEqualTo("'interval_literal_12'"); } } diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/sanity/TestValidateScaledWritersUsage.java b/core/trino-main/src/test/java/io/trino/sql/planner/sanity/TestValidateScaledWritersUsage.java index e5498743864cc..0012f7cd62181 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/sanity/TestValidateScaledWritersUsage.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/sanity/TestValidateScaledWritersUsage.java @@ -39,9 +39,11 @@ import io.trino.sql.planner.plan.TableScanNode; import io.trino.testing.LocalQueryRunner; import io.trino.testing.TestingTransactionHandle; -import org.testng.annotations.AfterClass; -import org.testng.annotations.BeforeClass; -import org.testng.annotations.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.parallel.Execution; import java.util.Optional; @@ -54,7 +56,11 @@ import static io.trino.testing.TestingHandles.TEST_CATALOG_HANDLE; import static io.trino.testing.TestingHandles.createTestCatalogHandle; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; +@TestInstance(PER_CLASS) +@Execution(CONCURRENT) public class TestValidateScaledWritersUsage extends BasePlanTest { @@ -72,7 +78,7 @@ public class TestValidateScaledWritersUsage private CatalogHandle catalog; private SchemaTableName schemaTableName; - @BeforeClass + @BeforeAll public void setup() { schemaTableName = new SchemaTableName("any", "any"); @@ -90,7 +96,7 @@ public void setup() tableScanNode = planBuilder.tableScan(nationTableHandle, ImmutableList.of(symbol), ImmutableMap.of(symbol, nationkeyColumnHandle)); } - @AfterClass(alwaysRun = true) + @AfterAll public void tearDown() { queryRunner.close(); diff --git a/core/trino-main/src/test/java/io/trino/util/TestLongLong2LongOpenCustomBigHashMap.java b/core/trino-main/src/test/java/io/trino/util/TestLongLong2LongOpenCustomBigHashMap.java index d6687dae37c59..b4a1b0164dbae 100644 --- a/core/trino-main/src/test/java/io/trino/util/TestLongLong2LongOpenCustomBigHashMap.java +++ b/core/trino-main/src/test/java/io/trino/util/TestLongLong2LongOpenCustomBigHashMap.java @@ -13,7 +13,7 @@ */ package io.trino.util; -import org.testng.annotations.Test; +import org.junit.jupiter.api.Test; import java.util.Arrays; import java.util.List; diff --git a/testing/trino-faulttolerant-tests/pom.xml b/testing/trino-faulttolerant-tests/pom.xml index 266fb41468e8e..b0135bbd5c0bd 100644 --- a/testing/trino-faulttolerant-tests/pom.xml +++ b/testing/trino-faulttolerant-tests/pom.xml @@ -13,16 +13,6 @@ ${project.parent.basedir} - - - instances @@ -434,33 +424,10 @@ testcontainers test - - - org.testng - testng - test - - - org.apache.maven.plugins - maven-surefire-plugin - - - - org.apache.maven.surefire - surefire-junit-platform - ${dep.plugin.surefire.version} - - - org.apache.maven.surefire - surefire-testng - ${dep.plugin.surefire.version} - - - org.basepom.maven duplicate-finder-maven-plugin diff --git a/testing/trino-faulttolerant-tests/src/test/java/io/trino/faulttolerant/TestFaultTolerantExecutionDynamicFiltering.java b/testing/trino-faulttolerant-tests/src/test/java/io/trino/faulttolerant/TestFaultTolerantExecutionDynamicFiltering.java index be1ec2efae1a0..facbec507a3c2 100644 --- a/testing/trino-faulttolerant-tests/src/test/java/io/trino/faulttolerant/TestFaultTolerantExecutionDynamicFiltering.java +++ b/testing/trino-faulttolerant-tests/src/test/java/io/trino/faulttolerant/TestFaultTolerantExecutionDynamicFiltering.java @@ -25,7 +25,9 @@ import io.trino.testing.FaultTolerantExecutionConnectorTestHelper; import io.trino.testing.QueryRunner; import io.trino.testing.TestingMetadata; -import org.testng.annotations.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.parallel.Execution; import java.util.Set; @@ -34,8 +36,9 @@ import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.sql.planner.OptimizerConfig.JoinDistributionType; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; -@Test(singleThreaded = true) +@Execution(SAME_THREAD) public class TestFaultTolerantExecutionDynamicFiltering extends AbstractTestCoordinatorDynamicFiltering { @@ -107,7 +110,8 @@ protected void testJoinWithNonSelectiveBuildSide(JoinDistributionType joinDistri } @Override - @Test(timeOut = 30_000) + @Test + @Timeout(30) public void testRightJoinWithNonSelectiveBuildSide() { assertQueryDynamicFilters( diff --git a/testing/trino-tests/src/test/java/io/trino/execution/AbstractTestCoordinatorDynamicFiltering.java b/testing/trino-tests/src/test/java/io/trino/execution/AbstractTestCoordinatorDynamicFiltering.java index 106766f64214b..0c441452f3e5d 100644 --- a/testing/trino-tests/src/test/java/io/trino/execution/AbstractTestCoordinatorDynamicFiltering.java +++ b/testing/trino-tests/src/test/java/io/trino/execution/AbstractTestCoordinatorDynamicFiltering.java @@ -48,8 +48,11 @@ import io.trino.testing.TestingPageSinkProvider; import io.trino.testing.TestingTransactionHandle; import org.intellij.lang.annotations.Language; -import org.testng.annotations.BeforeClass; -import org.testng.annotations.Test; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.parallel.Execution; import java.util.List; import java.util.Map; @@ -83,7 +86,11 @@ import static java.util.Objects.requireNonNull; import static java.util.concurrent.CompletableFuture.completedFuture; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; +@TestInstance(PER_CLASS) +@Execution(CONCURRENT) public abstract class AbstractTestCoordinatorDynamicFiltering extends AbstractTestQueryFramework { @@ -96,7 +103,7 @@ public abstract class AbstractTestCoordinatorDynamicFiltering private volatile Consumer> expectedCoordinatorDynamicFilterAssertion; private volatile Consumer> expectedTableScanDynamicFilterAssertion; - @BeforeClass + @BeforeAll public void setup() { // create lineitem table in test connector @@ -118,7 +125,8 @@ public void setup() protected abstract RetryPolicy getRetryPolicy(); - @Test(timeOut = 30_000) + @Test + @Timeout(30) public void testJoinWithEmptyBuildSide() { testJoinWithEmptyBuildSide(BROADCAST, true); @@ -135,7 +143,8 @@ private void testJoinWithEmptyBuildSide(JoinDistributionType joinDistributionTyp TupleDomain.none()); } - @Test(timeOut = 30_000) + @Test + @Timeout(30) public void testJoinWithLargeBuildSide() { testJoinWithLargeBuildSide(BROADCAST, true); @@ -152,7 +161,8 @@ private void testJoinWithLargeBuildSide(JoinDistributionType joinDistributionTyp TupleDomain.all()); } - @Test(timeOut = 30_000) + @Test + @Timeout(30) public void testMultiColumnJoinWithDifferentCardinalitiesInBuildSide() { testMultiColumnJoinWithDifferentCardinalitiesInBuildSide(BROADCAST, true); @@ -174,7 +184,8 @@ private void testMultiColumnJoinWithDifferentCardinalitiesInBuildSide(JoinDistri multipleValues(BIGINT, LongStream.rangeClosed(1L, 10L).boxed().collect(toImmutableList()))))); } - @Test(timeOut = 30_000) + @Test + @Timeout(30) public void testJoinWithSelectiveBuildSide() { testJoinWithSelectiveBuildSide(BROADCAST, true); @@ -193,7 +204,8 @@ private void testJoinWithSelectiveBuildSide(JoinDistributionType joinDistributio singleValue(BIGINT, 1L)))); } - @Test(timeOut = 30_000) + @Test + @Timeout(30) public void testInequalityJoinWithSelectiveBuildSide() { assertQueryDynamicFilters( @@ -222,7 +234,8 @@ public void testInequalityJoinWithSelectiveBuildSide() Domain.create(ValueSet.ofRanges(Range.greaterThan(BIGINT, 1L)), false)))); } - @Test(timeOut = 30_000) + @Test + @Timeout(30) public void testIsNotDistinctFromJoinWithSelectiveBuildSide() { assertQueryDynamicFilters( @@ -245,7 +258,8 @@ public void testIsNotDistinctFromJoinWithSelectiveBuildSide() Domain.onlyNull(BIGINT)))); } - @Test(timeOut = 30_000) + @Test + @Timeout(30) public void testJoinWithImplicitCoercion() { // setup fact table with integer suppkey @@ -273,7 +287,8 @@ public void testJoinWithImplicitCoercion() multipleValues(createVarcharType(40), values)))); } - @Test(timeOut = 30_000) + @Test + @Timeout(30) public void testJoinWithNonSelectiveBuildSide() { testJoinWithNonSelectiveBuildSide(BROADCAST, true); @@ -292,7 +307,8 @@ protected void testJoinWithNonSelectiveBuildSide(JoinDistributionType joinDistri Domain.create(ValueSet.ofRanges(range(BIGINT, 1L, true, 100L, true)), false)))); } - @Test(timeOut = 30_000) + @Test + @Timeout(30) public void testJoinWithMultipleDynamicFiltersOnProbe() { testJoinWithMultipleDynamicFiltersOnProbe(BROADCAST, true); @@ -315,7 +331,8 @@ private void testJoinWithMultipleDynamicFiltersOnProbe(JoinDistributionType join singleValue(BIGINT, 2L)))); } - @Test(timeOut = 30_000) + @Test + @Timeout(30) public void testRightJoinWithEmptyBuildSide() { assertQueryDynamicFilters( @@ -324,7 +341,8 @@ public void testRightJoinWithEmptyBuildSide() TupleDomain.none()); } - @Test(timeOut = 30_000) + @Test + @Timeout(30) public void testRightJoinWithNonSelectiveBuildSide() { assertQueryDynamicFilters( @@ -335,7 +353,8 @@ public void testRightJoinWithNonSelectiveBuildSide() Domain.create(ValueSet.ofRanges(range(BIGINT, 1L, true, 100L, true)), false)))); } - @Test(timeOut = 30_000) + @Test + @Timeout(30) public void testRightJoinWithSelectiveBuildSide() { assertQueryDynamicFilters( @@ -346,7 +365,8 @@ public void testRightJoinWithSelectiveBuildSide() singleValue(BIGINT, 1L)))); } - @Test(timeOut = 30_000) + @Test + @Timeout(30) public void testSemiJoinWithEmptyBuildSide() { testSemiJoinWithEmptyBuildSide(BROADCAST, true); @@ -363,7 +383,8 @@ private void testSemiJoinWithEmptyBuildSide(JoinDistributionType joinDistributio TupleDomain.none()); } - @Test(timeOut = 30_000) + @Test + @Timeout(30) public void testSemiJoinWithLargeBuildSide() { testSemiJoinWithLargeBuildSide(BROADCAST, true); @@ -380,7 +401,8 @@ private void testSemiJoinWithLargeBuildSide(JoinDistributionType joinDistributio TupleDomain.all()); } - @Test(timeOut = 30_000) + @Test + @Timeout(30) public void testSemiJoinWithSelectiveBuildSide() { testSemiJoinWithSelectiveBuildSide(BROADCAST, true); @@ -399,7 +421,8 @@ private void testSemiJoinWithSelectiveBuildSide(JoinDistributionType joinDistrib singleValue(BIGINT, 1L)))); } - @Test(timeOut = 30_000) + @Test + @Timeout(30) public void testSemiJoinWithNonSelectiveBuildSide() { testSemiJoinWithNonSelectiveBuildSide(BROADCAST, true); @@ -418,7 +441,8 @@ protected void testSemiJoinWithNonSelectiveBuildSide(JoinDistributionType joinDi Domain.create(ValueSet.ofRanges(range(BIGINT, 1L, true, 100L, true)), false)))); } - @Test(timeOut = 30_000) + @Test + @Timeout(30) public void testSemiJoinWithMultipleDynamicFiltersOnProbe() { testSemiJoinWithMultipleDynamicFiltersOnProbe(BROADCAST, true); diff --git a/testing/trino-tests/src/test/java/io/trino/execution/TestCoordinatorDynamicFiltering.java b/testing/trino-tests/src/test/java/io/trino/execution/TestCoordinatorDynamicFiltering.java index 1a52a4fe8fc9a..d85e6430513dd 100644 --- a/testing/trino-tests/src/test/java/io/trino/execution/TestCoordinatorDynamicFiltering.java +++ b/testing/trino-tests/src/test/java/io/trino/execution/TestCoordinatorDynamicFiltering.java @@ -17,11 +17,12 @@ import io.trino.operator.RetryPolicy; import io.trino.testing.DistributedQueryRunner; import io.trino.testing.QueryRunner; -import org.testng.annotations.Test; +import org.junit.jupiter.api.parallel.Execution; import static io.trino.operator.RetryPolicy.NONE; +import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; -@Test(singleThreaded = true) +@Execution(SAME_THREAD) public class TestCoordinatorDynamicFiltering extends AbstractTestCoordinatorDynamicFiltering { From bb17dec4365f9ad65ae76dd8056d1be525004954 Mon Sep 17 00:00:00 2001 From: Martin Traverso Date: Thu, 7 Dec 2023 13:09:44 -0800 Subject: [PATCH 055/350] Remove unnecessary method --- .../aggregation/TestDecimalAverageAggregation.java | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/core/trino-main/src/test/java/io/trino/operator/aggregation/TestDecimalAverageAggregation.java b/core/trino-main/src/test/java/io/trino/operator/aggregation/TestDecimalAverageAggregation.java index 1c6cb3d5a3d2c..8897405f6150e 100644 --- a/core/trino-main/src/test/java/io/trino/operator/aggregation/TestDecimalAverageAggregation.java +++ b/core/trino-main/src/test/java/io/trino/operator/aggregation/TestDecimalAverageAggregation.java @@ -217,12 +217,7 @@ private static BigDecimal decodeBigDecimal(DecimalType type, Int128 average) private void assertAverageEquals(LongDecimalWithOverflowAndLongState state, BigInteger expectedAverage) { - assertAverageEquals(state, expectedAverage, TYPE); - } - - private void assertAverageEquals(LongDecimalWithOverflowAndLongState state, BigInteger expectedAverage, DecimalType type) - { - assertThat(average(state, type).toBigInteger()).isEqualTo(expectedAverage); + assertThat(average(state, TYPE).toBigInteger()).isEqualTo(expectedAverage); } private static void addToState(LongDecimalWithOverflowAndLongState state, BigInteger value) From 8e1ac4d17c2b5229c0af2a7660eb8d588d54bd94 Mon Sep 17 00:00:00 2001 From: Martin Traverso Date: Fri, 8 Dec 2023 10:52:20 -0800 Subject: [PATCH 056/350] Remove unnecessary data provider --- .../hive/BaseS3AndGlueMetastoreTest.java | 53 ++- .../hive/TestHiveS3AndGlueMetastoreTest.java | 33 +- ...stCachingHiveMetastoreWithQueryRunner.java | 41 +- .../parquet/AbstractTestParquetReader.java | 156 +++---- .../hive/parquet/TestFullParquetReader.java | 7 +- .../parquet/TestParquetDecimalScaling.java | 425 ++++++++++++------ .../parquet/TestParquetPageSourceFactory.java | 20 +- .../hive/parquet/TestParquetReader.java | 7 +- .../predicate/TestParquetPredicateUtils.java | 81 ++-- .../TestIcebergS3AndGlueMetastoreTest.java | 12 +- 10 files changed, 499 insertions(+), 336 deletions(-) diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseS3AndGlueMetastoreTest.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseS3AndGlueMetastoreTest.java index fcf9f741a9a79..380f787410c10 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseS3AndGlueMetastoreTest.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseS3AndGlueMetastoreTest.java @@ -25,7 +25,6 @@ import org.intellij.lang.annotations.Language; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import java.util.List; @@ -33,14 +32,10 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; import java.util.stream.Collectors; -import java.util.stream.Stream; import static com.google.common.base.Verify.verify; import static com.google.common.collect.Sets.union; import static io.trino.plugin.hive.S3Assert.s3Path; -import static io.trino.testing.DataProviders.cartesianProduct; -import static io.trino.testing.DataProviders.toDataProvider; -import static io.trino.testing.DataProviders.trueFalse; import static io.trino.testing.TestingNames.randomNameSuffix; import static java.lang.String.format; import static java.util.Objects.requireNonNull; @@ -84,14 +79,16 @@ public void tearDown() } } - @DataProvider - public Object[][] locationPatternsDataProvider() + @Test + public void testBasicOperationsWithProvidedTableLocation() { - return cartesianProduct(trueFalse(), Stream.of(LocationPattern.values()).collect(toDataProvider())); + for (LocationPattern locationPattern : LocationPattern.values()) { + testBasicOperationsWithProvidedTableLocation(false, locationPattern); + testBasicOperationsWithProvidedTableLocation(true, locationPattern); + } } - @Test(dataProvider = "locationPatternsDataProvider") - public void testBasicOperationsWithProvidedTableLocation(boolean partitioned, LocationPattern locationPattern) + protected void testBasicOperationsWithProvidedTableLocation(boolean partitioned, LocationPattern locationPattern) { String tableName = "test_basic_operations_" + randomNameSuffix(); String location = locationPattern.locationForTable(bucketName, schemaName, tableName); @@ -121,8 +118,16 @@ public void testBasicOperationsWithProvidedTableLocation(boolean partitioned, Lo validateFilesAfterDrop(actualTableLocation); } - @Test(dataProvider = "locationPatternsDataProvider") - public void testBasicOperationsWithProvidedSchemaLocation(boolean partitioned, LocationPattern locationPattern) + @Test + public void testBasicOperationsWithProvidedSchemaLocation() + { + for (LocationPattern locationPattern : LocationPattern.values()) { + testBasicOperationsWithProvidedSchemaLocation(false, locationPattern); + testBasicOperationsWithProvidedSchemaLocation(true, locationPattern); + } + } + + protected void testBasicOperationsWithProvidedSchemaLocation(boolean partitioned, LocationPattern locationPattern) { String schemaName = "test_basic_operations_schema_" + randomNameSuffix(); String schemaLocation = locationPattern.locationForSchema(bucketName, schemaName); @@ -160,8 +165,16 @@ public void testBasicOperationsWithProvidedSchemaLocation(boolean partitioned, L assertThat(getTableFiles(actualTableLocation)).isEmpty(); } - @Test(dataProvider = "locationPatternsDataProvider") - public void testMergeWithProvidedTableLocation(boolean partitioned, LocationPattern locationPattern) + @Test + public void testMergeWithProvidedTableLocation() + { + for (LocationPattern locationPattern : LocationPattern.values()) { + testMergeWithProvidedTableLocation(false, locationPattern); + testMergeWithProvidedTableLocation(true, locationPattern); + } + } + + protected void testMergeWithProvidedTableLocation(boolean partitioned, LocationPattern locationPattern) { String tableName = "test_merge_" + randomNameSuffix(); String location = locationPattern.locationForTable(bucketName, schemaName, tableName); @@ -194,8 +207,16 @@ public void testMergeWithProvidedTableLocation(boolean partitioned, LocationPatt validateFilesAfterDrop(actualTableLocation); } - @Test(dataProvider = "locationPatternsDataProvider") - public void testOptimizeWithProvidedTableLocation(boolean partitioned, LocationPattern locationPattern) + @Test + public void testOptimizeWithProvidedTableLocation() + { + for (LocationPattern locationPattern : LocationPattern.values()) { + testOptimizeWithProvidedTableLocation(false, locationPattern); + testOptimizeWithProvidedTableLocation(true, locationPattern); + } + } + + protected void testOptimizeWithProvidedTableLocation(boolean partitioned, LocationPattern locationPattern) { String tableName = "test_optimize_" + randomNameSuffix(); String location = locationPattern.locationForTable(bucketName, schemaName, tableName); diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveS3AndGlueMetastoreTest.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveS3AndGlueMetastoreTest.java index d45371b16b0b4..f7a7dd9e9e468 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveS3AndGlueMetastoreTest.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveS3AndGlueMetastoreTest.java @@ -121,8 +121,7 @@ protected void validateFilesAfterOptimize(String location, Set initialFi } @Override // Row-level modifications are not supported for Hive tables - @Test(dataProvider = "locationPatternsDataProvider") - public void testBasicOperationsWithProvidedTableLocation(boolean partitioned, LocationPattern locationPattern) + protected void testBasicOperationsWithProvidedTableLocation(boolean partitioned, LocationPattern locationPattern) { String tableName = "test_basic_operations_" + randomNameSuffix(); String location = locationPattern.locationForTable(bucketName, schemaName, tableName); @@ -150,8 +149,16 @@ public void testBasicOperationsWithProvidedTableLocation(boolean partitioned, Lo } } - @Test(dataProvider = "locationPatternsDataProvider") - public void testBasicOperationsWithProvidedTableLocationNonCTAS(boolean partitioned, LocationPattern locationPattern) + @Test + public void testBasicOperationsWithProvidedTableLocationNonCTAS() + { + for (LocationPattern locationPattern : LocationPattern.values()) { + testBasicOperationsWithProvidedTableLocationNonCTAS(false, locationPattern); + testBasicOperationsWithProvidedTableLocationNonCTAS(true, locationPattern); + } + } + + private void testBasicOperationsWithProvidedTableLocationNonCTAS(boolean partitioned, LocationPattern locationPattern) { // this test needed, because execution path for CTAS and simple create is different String tableName = "test_basic_operations_" + randomNameSuffix(); @@ -177,8 +184,7 @@ public void testBasicOperationsWithProvidedTableLocationNonCTAS(boolean partitio } @Override // Row-level modifications are not supported for Hive tables - @Test(dataProvider = "locationPatternsDataProvider") - public void testBasicOperationsWithProvidedSchemaLocation(boolean partitioned, LocationPattern locationPattern) + protected void testBasicOperationsWithProvidedSchemaLocation(boolean partitioned, LocationPattern locationPattern) { String schemaName = "test_basic_operations_schema_" + randomNameSuffix(); String schemaLocation = locationPattern.locationForSchema(bucketName, schemaName); @@ -210,14 +216,13 @@ public void testBasicOperationsWithProvidedSchemaLocation(boolean partitioned, L } @Override - @Test(dataProvider = "locationPatternsDataProvider") public void testMergeWithProvidedTableLocation(boolean partitioned, LocationPattern locationPattern) { // Row-level modifications are not supported for Hive tables } @Override - public void testOptimizeWithProvidedTableLocation(boolean partitioned, LocationPattern locationPattern) + protected void testOptimizeWithProvidedTableLocation(boolean partitioned, LocationPattern locationPattern) { if (locationPattern == DOUBLE_SLASH || locationPattern == TRIPLE_SLASH || locationPattern == TWO_TRAILING_SLASHES) { assertThatThrownBy(() -> super.testOptimizeWithProvidedTableLocation(partitioned, locationPattern)) @@ -228,8 +233,16 @@ public void testOptimizeWithProvidedTableLocation(boolean partitioned, LocationP super.testOptimizeWithProvidedTableLocation(partitioned, locationPattern); } - @Test(dataProvider = "locationPatternsDataProvider") - public void testAnalyzeWithProvidedTableLocation(boolean partitioned, LocationPattern locationPattern) + @Test + public void testAnalyzeWithProvidedTableLocation() + { + for (LocationPattern locationPattern : LocationPattern.values()) { + testAnalyzeWithProvidedTableLocation(false, locationPattern); + testAnalyzeWithProvidedTableLocation(true, locationPattern); + } + } + + private void testAnalyzeWithProvidedTableLocation(boolean partitioned, LocationPattern locationPattern) { String tableName = "test_analyze_" + randomNameSuffix(); String location = locationPattern.locationForTable(bucketName, schemaName, tableName); diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastoreWithQueryRunner.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastoreWithQueryRunner.java index f035968afb16c..81e20492e8ae1 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastoreWithQueryRunner.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastoreWithQueryRunner.java @@ -28,14 +28,12 @@ import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.DistributedQueryRunner; import io.trino.testing.QueryRunner; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import java.util.List; import java.util.Optional; import static com.google.common.base.Verify.verify; -import static com.google.common.collect.Lists.cartesianProduct; import static io.trino.spi.security.SelectedRole.Type.ROLE; import static io.trino.testing.TestingSession.testSessionBuilder; import static java.util.Collections.nCopies; @@ -101,8 +99,27 @@ public void testCacheRefreshOnGrantAndRevoke() .hasMessageContaining("Access Denied"); } - @Test(dataProvider = "testCacheRefreshOnRoleGrantAndRevokeParams") - public void testCacheRefreshOnRoleGrantAndRevoke(List grantRoleStatements, String revokeRoleStatement) + @Test + public void testCacheRefreshOnRoleGrantAndRevoke() + { + String grantSelectStatement = "GRANT SELECT ON test TO ROLE test_role"; + String grantRoleStatement = "GRANT test_role TO " + ALICE_NAME + " IN " + CATALOG; + List> grantRoleStatements = ImmutableList.of( + ImmutableList.of(grantSelectStatement, grantRoleStatement), + ImmutableList.of(grantRoleStatement, grantSelectStatement)); + List revokeRoleStatements = ImmutableList.of( + "DROP ROLE test_role IN " + CATALOG, + "REVOKE SELECT ON test FROM ROLE test_role", + "REVOKE test_role FROM " + ALICE_NAME + " IN " + CATALOG); + + for (String roleRevoke : revokeRoleStatements) { + for (List roleGrant : grantRoleStatements) { + testCacheRefreshOnRoleGrantAndRevoke(roleGrant, roleRevoke); + } + } + } + + private void testCacheRefreshOnRoleGrantAndRevoke(List grantRoleStatements, String revokeRoleStatement) { assertThatThrownBy(() -> getQueryRunner().execute(ALICE, "SELECT * FROM test")) .hasMessageContaining("Access Denied"); @@ -188,20 +205,4 @@ public void testPartitionAppend() String expected = Joiner.on(",").join(nCopies(nodeCount + 1, row)); assertQuery("SELECT * FROM test_part_append", "VALUES " + expected); } - - @DataProvider - public Object[][] testCacheRefreshOnRoleGrantAndRevokeParams() - { - String grantSelectStatement = "GRANT SELECT ON test TO ROLE test_role"; - String grantRoleStatement = "GRANT test_role TO " + ALICE_NAME + " IN " + CATALOG; - List> grantRoleStatements = ImmutableList.of( - ImmutableList.of(grantSelectStatement, grantRoleStatement), - ImmutableList.of(grantRoleStatement, grantSelectStatement)); - List revokeRoleStatements = ImmutableList.of( - "DROP ROLE test_role IN " + CATALOG, - "REVOKE SELECT ON test FROM ROLE test_role", - "REVOKE test_role FROM " + ALICE_NAME + " IN " + CATALOG); - return cartesianProduct(grantRoleStatements, revokeRoleStatements).stream() - .map(a -> a.toArray(Object[]::new)).toArray(Object[][]::new); - } } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/AbstractTestParquetReader.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/AbstractTestParquetReader.java index 21d6cb0479187..2606050da85b2 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/AbstractTestParquetReader.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/AbstractTestParquetReader.java @@ -41,9 +41,7 @@ import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo; import org.apache.parquet.schema.MessageType; import org.joda.time.DateTimeZone; -import org.testng.annotations.BeforeClass; -import org.testng.annotations.DataProvider; -import org.testng.annotations.Test; +import org.junit.jupiter.api.Test; import java.math.BigInteger; import java.util.ArrayList; @@ -96,7 +94,6 @@ import static io.trino.spi.type.VarcharType.VARCHAR; import static io.trino.spi.type.VarcharType.createUnboundedVarcharType; import static io.trino.spi.type.VarcharType.createVarcharType; -import static io.trino.testing.DataProviders.toDataProvider; import static io.trino.testing.DateTimeTestingUtils.sqlTimestampOf; import static io.trino.testing.StructuralTestUtil.mapType; import static java.lang.Math.floorDiv; @@ -131,24 +128,15 @@ public abstract class AbstractTestParquetReader private static final int MAX_PRECISION_INT32 = toIntExact(maxPrecision(4)); private static final int MAX_PRECISION_INT64 = toIntExact(maxPrecision(8)); - @SuppressWarnings("FieldCanBeLocal") - private Logger parquetLogger; - private final ParquetTester tester; protected AbstractTestParquetReader(ParquetTester tester) { this.tester = tester; - } - - @BeforeClass - public void setUp() - { assertThat(DateTimeZone.getDefault()).isEqualTo(DateTimeZone.forID("America/Bahia_Banderas")); // Parquet has excessive logging at INFO level - parquetLogger = Logger.getLogger("org.apache.parquet.hadoop"); - parquetLogger.setLevel(Level.WARNING); + Logger.getLogger("org.apache.parquet.hadoop").setLevel(Level.WARNING); } @Test @@ -885,52 +873,47 @@ public void testParquetShortDecimalWriteToTrinoDecimalWithNonMatchingScale() tester.testRoundTrip(javaLongObjectInspector, ImmutableList.of(10L), ImmutableList.of(SqlDecimal.of(100L, 10, 2)), createDecimalType(10, 2), Optional.of(parquetSchema)); } - @Test(dataProvider = "testDecimalInputProvider") - public void testDecimals(DecimalInput decimalInput) - throws Exception - { - for (int precision = 1; precision <= decimalInput.getMaxSupportedPrecision(); precision++) { - int scale = ThreadLocalRandom.current().nextInt(precision); - MessageType parquetSchema = parseMessageType(format( - "message hive_decimal { optional %s test (DECIMAL(%d, %d)); }", - decimalInput.getPrimitiveTypeName(precision), - precision, - scale)); - ImmutableList.Builder expectedValues = ImmutableList.builder(); - ImmutableList.Builder expectedValuesMaxPrecision = ImmutableList.builder(); - ImmutableList.Builder writeValuesBuilder = ImmutableList.builder(); - - BigInteger start = BigInteger.valueOf(10).pow(precision).subtract(ONE).negate(); - BigInteger end = BigInteger.valueOf(10).pow(precision); - BigInteger step = BigInteger.valueOf(1).max(end.subtract(start).divide(BigInteger.valueOf(1_500))); - for (BigInteger value = start; value.compareTo(end) < 0; value = value.add(step)) { - writeValuesBuilder.add(decimalInput.convertToWriteValue(value, scale)); - expectedValues.add(new SqlDecimal(value, precision, scale)); - expectedValuesMaxPrecision.add(new SqlDecimal(value, MAX_PRECISION, scale)); + @Test + public void testDecimals() + throws Exception + { + for (DecimalInput decimalInput : DecimalInput.values()) { + for (int precision = 1; precision <= decimalInput.getMaxSupportedPrecision(); precision++) { + int scale = ThreadLocalRandom.current().nextInt(precision); + MessageType parquetSchema = parseMessageType(format( + "message hive_decimal { optional %s test (DECIMAL(%d, %d)); }", + decimalInput.getPrimitiveTypeName(precision), + precision, + scale)); + ImmutableList.Builder expectedValues = ImmutableList.builder(); + ImmutableList.Builder expectedValuesMaxPrecision = ImmutableList.builder(); + ImmutableList.Builder writeValuesBuilder = ImmutableList.builder(); + + BigInteger start = BigInteger.valueOf(10).pow(precision).subtract(ONE).negate(); + BigInteger end = BigInteger.valueOf(10).pow(precision); + BigInteger step = BigInteger.valueOf(1).max(end.subtract(start).divide(BigInteger.valueOf(1_500))); + for (BigInteger value = start; value.compareTo(end) < 0; value = value.add(step)) { + writeValuesBuilder.add(decimalInput.convertToWriteValue(value, scale)); + expectedValues.add(new SqlDecimal(value, precision, scale)); + expectedValuesMaxPrecision.add(new SqlDecimal(value, MAX_PRECISION, scale)); + } + List writeValues = writeValuesBuilder.build(); + tester.testRoundTrip( + decimalInput.getParquetObjectInspector(precision, scale), + writeValues, + expectedValues.build(), + createDecimalType(precision, scale), + Optional.of(parquetSchema)); + tester.testRoundTrip( + decimalInput.getParquetObjectInspector(precision, scale), + writeValues, + expectedValuesMaxPrecision.build(), + createDecimalType(MAX_PRECISION, scale), + Optional.of(parquetSchema)); } - List writeValues = writeValuesBuilder.build(); - tester.testRoundTrip( - decimalInput.getParquetObjectInspector(precision, scale), - writeValues, - expectedValues.build(), - createDecimalType(precision, scale), - Optional.of(parquetSchema)); - tester.testRoundTrip( - decimalInput.getParquetObjectInspector(precision, scale), - writeValues, - expectedValuesMaxPrecision.build(), - createDecimalType(MAX_PRECISION, scale), - Optional.of(parquetSchema)); } } - @DataProvider - public Object[][] testDecimalInputProvider() - { - return Arrays.stream(DecimalInput.values()) - .collect(toDataProvider()); - } - private enum DecimalInput { INT32 { @@ -1235,41 +1218,36 @@ public void testReadParquetInt32AsTrinoShortDecimal() .isInstanceOf(TrinoException.class); } - @Test(dataProvider = "timestampPrecision") - public void testTimestamp(HiveTimestampPrecision precision) + @Test + public void testTimestamp() throws Exception { - List epochMillisValues = ContiguousSet.create(Range.closedOpen((long) -1_000, (long) 1_000), DiscreteDomain.longs()).stream() - .map(millis -> System.currentTimeMillis() + millis) - .collect(toImmutableList()); - List writeValues = epochMillisValues.stream() - .map(AbstractTestParquetReader::longToTimestamp) - .collect(toImmutableList()); - List readValues = epochMillisValues.stream() - .map(epochMillis -> SqlTimestamp.newInstance(precision.getPrecision(), epochMillis * 1_000, 0)) - .collect(toImmutableList()); - // INT96 backed timestamps are written by the default ParquetSchemaOptions - tester.testRoundTrip( - javaTimestampObjectInspector, - writeValues, - readValues, - createTimestampType(precision.getPrecision()), - Optional.empty()); - tester.testRoundTrip( - javaTimestampObjectInspector, - writeValues, - readValues, - getOnlyElement(TEST_COLUMN), - createTimestampType(precision.getPrecision()), - Optional.empty(), - ParquetSchemaOptions.withInt64BackedTimestamps()); - } - - @DataProvider - public Object[][] timestampPrecision() - { - return Stream.of(HiveTimestampPrecision.values()) - .collect(toDataProvider()); + for (HiveTimestampPrecision precision : HiveTimestampPrecision.values()) { + List epochMillisValues = ContiguousSet.create(Range.closedOpen((long) -1_000, (long) 1_000), DiscreteDomain.longs()).stream() + .map(millis -> System.currentTimeMillis() + millis) + .collect(toImmutableList()); + List writeValues = epochMillisValues.stream() + .map(AbstractTestParquetReader::longToTimestamp) + .collect(toImmutableList()); + List readValues = epochMillisValues.stream() + .map(epochMillis -> SqlTimestamp.newInstance(precision.getPrecision(), epochMillis * 1_000, 0)) + .collect(toImmutableList()); + // INT96 backed timestamps are written by the default ParquetSchemaOptions + tester.testRoundTrip( + javaTimestampObjectInspector, + writeValues, + readValues, + createTimestampType(precision.getPrecision()), + Optional.empty()); + tester.testRoundTrip( + javaTimestampObjectInspector, + writeValues, + readValues, + getOnlyElement(TEST_COLUMN), + createTimestampType(precision.getPrecision()), + Optional.empty(), + ParquetSchemaOptions.withInt64BackedTimestamps()); + } } @Test diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestFullParquetReader.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestFullParquetReader.java index 13ba2b6d57dd3..29ba7bae611d2 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestFullParquetReader.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestFullParquetReader.java @@ -13,12 +13,15 @@ */ package io.trino.plugin.hive.parquet; -import org.testng.annotations.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.parallel.Execution; + +import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; // Failing on multiple threads because of org.apache.hadoop.hive.ql.io.parquet.write.ParquetRecordWriterWrapper // uses a single record writer across all threads. // For example org.apache.parquet.column.values.factory.DefaultValuesWriterFactory#DEFAULT_V1_WRITER_FACTORY is shared mutable state. -@Test(singleThreaded = true) +@Execution(SAME_THREAD) public class TestFullParquetReader extends AbstractTestParquetReader { diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestParquetDecimalScaling.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestParquetDecimalScaling.java index f9316bac26461..1c02d1cda860f 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestParquetDecimalScaling.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestParquetDecimalScaling.java @@ -36,7 +36,6 @@ import org.apache.parquet.schema.MessageType; import org.intellij.lang.annotations.Language; import org.joda.time.DateTimeZone; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import java.io.IOException; @@ -51,7 +50,6 @@ import java.util.Properties; import java.util.UUID; import java.util.concurrent.ThreadLocalRandom; -import java.util.stream.Stream; import static com.google.common.collect.ImmutableList.toImmutableList; import static com.google.common.collect.Iterables.transform; @@ -60,8 +58,6 @@ import static io.trino.plugin.hive.util.SerdeConstants.LIST_COLUMNS; import static io.trino.plugin.hive.util.SerdeConstants.LIST_COLUMN_TYPES; import static io.trino.spi.type.Decimals.overflows; -import static io.trino.testing.DataProviders.cartesianProduct; -import static io.trino.testing.DataProviders.toDataProvider; import static io.trino.tpch.TpchTable.NATION; import static java.lang.Integer.MAX_VALUE; import static java.lang.String.format; @@ -100,8 +96,77 @@ protected QueryRunner createQueryRunner() * Tests if Parquet decimal with given precision and scale can be read into Trino decimal with different precision and scale * if Parquet decimal value could be rescaled into Trino decimal without losing most and least significant digits. */ - @Test(dataProvider = "testReadingMatchingPrecisionDataProvider") - public void testReadingMatchingPrecision( + @Test + public void testReadingMatchingPrecision() + { + for (WriterVersion writerVersion : WriterVersion.values()) { + testReadingMatchingPrecision( + 10, + 2, + false, + ImmutableList.of("10.01", "10", "1.23", maximumValue(10, 2), minimumValue(10, 2)), + ImmutableList.of("10.01", "10.00", "1.23", maximumValue(10, 2), minimumValue(10, 2)), + writerVersion); + + testReadingMatchingPrecision( + 10, + 2, + true, + ImmutableList.of("10.01", "10", "1.23", maximumValue(10, 2), minimumValue(10, 2)), + ImmutableList.of("10.01", "10.00", "1.23", maximumValue(10, 2), minimumValue(10, 2)), + writerVersion); + + testReadingMatchingPrecision( + 4, + 2, + false, + ImmutableList.of("10.01", "10", "1.23", maximumValue(4, 2), minimumValue(4, 2)), + ImmutableList.of("10.01", "10.00", "1.23", maximumValue(4, 2), minimumValue(4, 2)), + writerVersion); + + testReadingMatchingPrecision( + 4, + 2, + true, + ImmutableList.of("10.01", "10", "1.23", maximumValue(4, 2), minimumValue(4, 2)), + ImmutableList.of("10.01", "10.00", "1.23", maximumValue(4, 2), minimumValue(4, 2)), + writerVersion); + + testReadingMatchingPrecision( + 14, + 2, + false, + ImmutableList.of("10.01", "10", "1.23", maximumValue(14, 2), minimumValue(14, 2)), + ImmutableList.of("10.01", "10.00", "1.23", maximumValue(14, 2), minimumValue(14, 2)), + writerVersion); + + testReadingMatchingPrecision( + 6, + 3, + false, + ImmutableList.of("10.01", "10", "1.23", maximumValue(6, 3), minimumValue(6, 3)), + ImmutableList.of("10.010", "10.000", "1.230", maximumValue(6, 3), minimumValue(6, 3)), + writerVersion); + + testReadingMatchingPrecision( + 6, + 3, + true, + ImmutableList.of("10.01", "10", "1.23", maximumValue(6, 3), minimumValue(6, 3)), + ImmutableList.of("10.010", "10.000", "1.230", maximumValue(6, 3), minimumValue(6, 3)), + writerVersion); + + testReadingMatchingPrecision( + 38, + 4, + false, + ImmutableList.of("10.01", "10", "1.23", maximumValue(38, 4), minimumValue(38, 4)), + ImmutableList.of("10.0100", "10.0000", "1.2300", maximumValue(38, 4), minimumValue(38, 4)), + writerVersion); + } + } + + private void testReadingMatchingPrecision( int precision, int scale, boolean forceFixedLengthArray, @@ -123,42 +188,145 @@ public void testReadingMatchingPrecision( dropTable(tableName); } - @DataProvider - public Object[][] testReadingMatchingPrecisionDataProvider() - { - return withWriterVersion(new Object[][] { - {10, 2, false, - ImmutableList.of("10.01", "10", "1.23", maximumValue(10, 2), minimumValue(10, 2)), - ImmutableList.of("10.01", "10.00", "1.23", maximumValue(10, 2), minimumValue(10, 2))}, - {10, 2, true, - ImmutableList.of("10.01", "10", "1.23", maximumValue(10, 2), minimumValue(10, 2)), - ImmutableList.of("10.01", "10.00", "1.23", maximumValue(10, 2), minimumValue(10, 2))}, - {4, 2, false, - ImmutableList.of("10.01", "10", "1.23", maximumValue(4, 2), minimumValue(4, 2)), - ImmutableList.of("10.01", "10.00", "1.23", maximumValue(4, 2), minimumValue(4, 2))}, - {4, 2, true, - ImmutableList.of("10.01", "10", "1.23", maximumValue(4, 2), minimumValue(4, 2)), - ImmutableList.of("10.01", "10.00", "1.23", maximumValue(4, 2), minimumValue(4, 2))}, - {14, 2, false, - ImmutableList.of("10.01", "10", "1.23", maximumValue(14, 2), minimumValue(14, 2)), - ImmutableList.of("10.01", "10.00", "1.23", maximumValue(14, 2), minimumValue(14, 2))}, - {6, 3, false, - ImmutableList.of("10.01", "10", "1.23", maximumValue(6, 3), minimumValue(6, 3)), - ImmutableList.of("10.010", "10.000", "1.230", maximumValue(6, 3), minimumValue(6, 3))}, - {6, 3, true, - ImmutableList.of("10.01", "10", "1.23", maximumValue(6, 3), minimumValue(6, 3)), - ImmutableList.of("10.010", "10.000", "1.230", maximumValue(6, 3), minimumValue(6, 3))}, - {38, 4, false, - ImmutableList.of("10.01", "10", "1.23", maximumValue(38, 4), minimumValue(38, 4)), - ImmutableList.of("10.0100", "10.0000", "1.2300", maximumValue(38, 4), minimumValue(38, 4))} - }); - } - /** * Tests if Parquet decimal with given precision and scale can be read into Trino decimal with different precision and scale * if Parquet decimal value could be rescaled into Trino decimal without loosing most and least significant digits. */ - @Test(dataProvider = "testReadingRescaledDecimalsProvider") + @Test + public void testReadingRescaledDecimals() + { + for (WriterVersion writerVersion : WriterVersion.values()) { + testReadingRescaledDecimals( + 10, + 2, + false, + 12, + 4, + ImmutableList.of("10.01", "10", "1.23", maximumValue(10, 2), minimumValue(10, 2)), + ImmutableList.of("10.0100", "10.0000", "1.2300", maximumValue(10, 2), minimumValue(10, 2)), + writerVersion); + + testReadingRescaledDecimals( + 10, + 2, + true, + 13, + 5, + ImmutableList.of("10.01", "10", "1.23", maximumValue(10, 2), minimumValue(10, 2)), + ImmutableList.of("10.01000", "10.0000", "1.23000", maximumValue(10, 2), minimumValue(10, 2)), + writerVersion); + + testReadingRescaledDecimals( + 4, + 2, + false, + 6, + 4, + ImmutableList.of("10.01", "10", "1.23", maximumValue(4, 2), minimumValue(4, 2)), + ImmutableList.of("10.0100", "10.0000", "1.2300", maximumValue(4, 2), minimumValue(4, 2)), + writerVersion); + + testReadingRescaledDecimals( + 4, + 2, + false, + 6, + 2, + ImmutableList.of("10.01", "10", "1.23", maximumValue(4, 2), minimumValue(4, 2)), + ImmutableList.of("10.01", "10.00", "1.23", maximumValue(4, 2), minimumValue(4, 2)), + writerVersion); + + testReadingRescaledDecimals( + 10, + 2, + false, + 11, + 3, + ImmutableList.of("10.01", "10", "1.23", maximumValue(10, 2), minimumValue(10, 2)), + ImmutableList.of("10.010", "10.000", "1.230", maximumValue(10, 2), minimumValue(10, 2)), + writerVersion); + + testReadingRescaledDecimals( + 10, 2, + true, + 12, + 4, + ImmutableList.of("10.01", "10", "1.23", maximumValue(10, 2), minimumValue(10, 2)), + ImmutableList.of("10.0100", "10.0000", "1.2300", maximumValue(10, 2), minimumValue(10, 2)), + writerVersion); + + testReadingRescaledDecimals( + 4, + 2, + false, + 10, + 5, + ImmutableList.of("10.01", "10", "1.23", maximumValue(4, 2), minimumValue(4, 2)), + ImmutableList.of("10.01000", "10.00000", "1.23000", maximumValue(4, 2), minimumValue(4, 2)), + writerVersion); + + testReadingRescaledDecimals( + 4, + 2, + true, + 10, + 5, + ImmutableList.of("10.01", "10", "1.23", maximumValue(4, 2), minimumValue(4, 2)), + ImmutableList.of("10.01000", "10.00000", "1.23000", maximumValue(4, 2), minimumValue(4, 2)), + writerVersion); + + testReadingRescaledDecimals( + 14, + 2, + false, + 20, + 3, + ImmutableList.of("10.01", "10", "1.23", maximumValue(14, 2), minimumValue(14, 2)), + ImmutableList.of("10.010", "10.000", "1.230", maximumValue(14, 2), minimumValue(14, 2)), + writerVersion); + + testReadingRescaledDecimals( + 6, + 3, + false, + 9, + 6, + ImmutableList.of("10.01", "10", "1.23", maximumValue(6, 3), minimumValue(6, 3)), + ImmutableList.of("10.010000", "10.000000", "1.230000", maximumValue(6, 3), minimumValue(6, 3)), + writerVersion); + + testReadingRescaledDecimals( + 6, + 3, + true, + 9, + 6, + ImmutableList.of("10.01", "10", "1.23", maximumValue(6, 3), minimumValue(6, 3)), + ImmutableList.of("10.010000", "10.000000", "1.230000", maximumValue(6, 3), minimumValue(6, 3)), + writerVersion); + + testReadingRescaledDecimals( + 10, + 2, + false, + 38, + 4, + ImmutableList.of("10.01", "10", "1.23", maximumValue(10, 2), minimumValue(10, 2)), + ImmutableList.of("10.0100", "10.0000", "1.2300", maximumValue(10, 2), minimumValue(10, 2)), + writerVersion); + + testReadingRescaledDecimals( + 18, + 4, + false, + 38, + 14, + ImmutableList.of("10.01", "10", "1.23", maximumValue(18, 4), minimumValue(18, 4)), + ImmutableList.of("10.0100", "10.0000", "1.2300", maximumValue(18, 4), minimumValue(18, 4)), + writerVersion); + } + } + public void testReadingRescaledDecimals( int precision, int scale, @@ -183,58 +351,56 @@ public void testReadingRescaledDecimals( dropTable(tableName); } - @DataProvider - public Object[][] testReadingRescaledDecimalsProvider() - { - // parquetPrecision, parquetScale, useFixedLengthArray, schemaPrecision, schemaScale, writeValues, expectedValues - return withWriterVersion(new Object[][] { - {10, 2, false, 12, 4, - ImmutableList.of("10.01", "10", "1.23", maximumValue(10, 2), minimumValue(10, 2)), - ImmutableList.of("10.0100", "10.0000", "1.2300", maximumValue(10, 2), minimumValue(10, 2))}, - {10, 2, true, 13, 5, - ImmutableList.of("10.01", "10", "1.23", maximumValue(10, 2), minimumValue(10, 2)), - ImmutableList.of("10.01000", "10.0000", "1.23000", maximumValue(10, 2), minimumValue(10, 2))}, - {4, 2, false, 6, 4, - ImmutableList.of("10.01", "10", "1.23", maximumValue(4, 2), minimumValue(4, 2)), - ImmutableList.of("10.0100", "10.0000", "1.2300", maximumValue(4, 2), minimumValue(4, 2))}, - {4, 2, false, 6, 2, - ImmutableList.of("10.01", "10", "1.23", maximumValue(4, 2), minimumValue(4, 2)), - ImmutableList.of("10.01", "10.00", "1.23", maximumValue(4, 2), minimumValue(4, 2))}, - {10, 2, false, 11, 3, - ImmutableList.of("10.01", "10", "1.23", maximumValue(10, 2), minimumValue(10, 2)), - ImmutableList.of("10.010", "10.000", "1.230", maximumValue(10, 2), minimumValue(10, 2))}, - {10, 2, true, 12, 4, - ImmutableList.of("10.01", "10", "1.23", maximumValue(10, 2), minimumValue(10, 2)), - ImmutableList.of("10.0100", "10.0000", "1.2300", maximumValue(10, 2), minimumValue(10, 2))}, - {4, 2, false, 10, 5, - ImmutableList.of("10.01", "10", "1.23", maximumValue(4, 2), minimumValue(4, 2)), - ImmutableList.of("10.01000", "10.00000", "1.23000", maximumValue(4, 2), minimumValue(4, 2))}, - {4, 2, true, 10, 5, - ImmutableList.of("10.01", "10", "1.23", maximumValue(4, 2), minimumValue(4, 2)), - ImmutableList.of("10.01000", "10.00000", "1.23000", maximumValue(4, 2), minimumValue(4, 2))}, - {14, 2, false, 20, 3, - ImmutableList.of("10.01", "10", "1.23", maximumValue(14, 2), minimumValue(14, 2)), - ImmutableList.of("10.010", "10.000", "1.230", maximumValue(14, 2), minimumValue(14, 2))}, - {6, 3, false, 9, 6, - ImmutableList.of("10.01", "10", "1.23", maximumValue(6, 3), minimumValue(6, 3)), - ImmutableList.of("10.010000", "10.000000", "1.230000", maximumValue(6, 3), minimumValue(6, 3))}, - {6, 3, true, 9, 6, - ImmutableList.of("10.01", "10", "1.23", maximumValue(6, 3), minimumValue(6, 3)), - ImmutableList.of("10.010000", "10.000000", "1.230000", maximumValue(6, 3), minimumValue(6, 3))}, - {10, 2, false, 38, 4, - ImmutableList.of("10.01", "10", "1.23", maximumValue(10, 2), minimumValue(10, 2)), - ImmutableList.of("10.0100", "10.0000", "1.2300", maximumValue(10, 2), minimumValue(10, 2))}, - {18, 4, false, 38, 14, - ImmutableList.of("10.01", "10", "1.23", maximumValue(18, 4), minimumValue(18, 4)), - ImmutableList.of("10.0100", "10.0000", "1.2300", maximumValue(18, 4), minimumValue(18, 4))}, - }); - } - /** * Tests if Parquet decimal with given precision and scale can be read into Trino decimal with different precision and scale * if Parquet decimal value will be rounded to fit into Trino decimal. */ - @Test(dataProvider = "testReadingRoundedDecimalsProvider") + @Test + public void testReadingRoundedDecimals() + { + for (WriterVersion writerVersion : WriterVersion.values()) { + testReadingRoundedDecimals( + 10, + 2, + false, + 12, + 1, + ImmutableList.of("10.01", "10", "1.23", maximumValue(10, 2), minimumValue(10, 2)), + ImmutableList.of("10.01", "10", "1.23", maximumValue(10, 2), minimumValue(10, 2)), + writerVersion); + + testReadingRoundedDecimals( + 9, + 2, + true, + 12, + 1, + ImmutableList.of("10.01", "10", "1.23", maximumValue(9, 2), minimumValue(9, 2)), + ImmutableList.of("10.01", "10", "1.23", maximumValue(9, 2), minimumValue(9, 2)), + writerVersion); + + testReadingRoundedDecimals( + 4, + 2, + false, + 7, + 1, + ImmutableList.of("10.01", "10", "1.23", maximumValue(4, 2), minimumValue(4, 2)), + ImmutableList.of("10.01", "10", "1.23", maximumValue(4, 2), minimumValue(4, 2)), + writerVersion); + + testReadingRoundedDecimals( + 10, + 2, + false, + 12, + 1, + ImmutableList.of("10.01", "10", "1.23", maximumValue(10, 2), minimumValue(10, 2)), + ImmutableList.of("10.01", "10", "1.23", maximumValue(10, 2), minimumValue(10, 2)), + writerVersion); + } + } + public void testReadingRoundedDecimals( int precision, int scale, @@ -259,32 +425,25 @@ public void testReadingRoundedDecimals( dropTable(tableName); } - @DataProvider - public Object[][] testReadingRoundedDecimalsProvider() - { - // parquetPrecision, parquetScale, useFixedLengthArray, schemaPrecision, schemaScale, writeValues, expectedValues - return withWriterVersion(new Object[][] { - {10, 2, false, 12, 1, - ImmutableList.of("10.01", "10", "1.23", maximumValue(10, 2), minimumValue(10, 2)), - ImmutableList.of("10.01", "10", "1.23", maximumValue(10, 2), minimumValue(10, 2))}, - {9, 2, true, 12, 1, - ImmutableList.of("10.01", "10", "1.23", maximumValue(9, 2), minimumValue(9, 2)), - ImmutableList.of("10.01", "10", "1.23", maximumValue(9, 2), minimumValue(9, 2))}, - {4, 2, false, 7, 1, - ImmutableList.of("10.01", "10", "1.23", maximumValue(4, 2), minimumValue(4, 2)), - ImmutableList.of("10.01", "10", "1.23", maximumValue(4, 2), minimumValue(4, 2))}, - {10, 2, false, 12, 1, - ImmutableList.of("10.01", "10", "1.23", maximumValue(10, 2), minimumValue(10, 2)), - ImmutableList.of("10.01", "10", "1.23", maximumValue(10, 2), minimumValue(10, 2))}, - }); - } - /** * Tests if Parquet decimal with given precision and scale cannot be read into Presto decimal with different precision and scale * because when rescaling decimal we would loose most significant digits. */ - @Test(dataProvider = "testReadingNonRescalableDecimalsProvider") - public void testReadingNonRescalableDecimals( + @Test + public void testReadingNonRescalableDecimals() + { + for (WriterVersion writerVersion : WriterVersion.values()) { + testReadingNonRescalableDecimals(4, 2, false, 4, 3, ImmutableList.of("10.01"), writerVersion); + testReadingNonRescalableDecimals(10, 2, false, 10, 3, ImmutableList.of("12345678.91"), writerVersion); + testReadingNonRescalableDecimals(10, 2, false, 3, 2, ImmutableList.of("10.01"), writerVersion); + testReadingNonRescalableDecimals(10, 2, true, 14, 7, ImmutableList.of("99999999.99"), writerVersion); + testReadingNonRescalableDecimals(10, 2, false, 10, 4, ImmutableList.of("99999999.99"), writerVersion); + testReadingNonRescalableDecimals(18, 8, false, 32, 23, ImmutableList.of("1234567890.12345678"), writerVersion); + testReadingNonRescalableDecimals(20, 8, false, 32, 21, ImmutableList.of("123456789012.12345678"), writerVersion); + } + } + + private void testReadingNonRescalableDecimals( int precision, int scale, boolean forceFixedLengthArray, @@ -310,23 +469,22 @@ public void testReadingNonRescalableDecimals( dropTable(tableName); } - @DataProvider - public Object[][] testReadingNonRescalableDecimalsProvider() + @Test + public void testParquetLongFixedLenByteArrayWithTrinoShortDecimal() { - // parquetPrecision, parquetScale, useFixedLengthArray, schemaPrecision, schemaScale, writeValues - return withWriterVersion(new Object[][] { - {4, 2, false, 4, 3, ImmutableList.of("10.01")}, - {10, 2, false, 10, 3, ImmutableList.of("12345678.91")}, - {10, 2, false, 3, 2, ImmutableList.of("10.01")}, - {10, 2, true, 14, 7, ImmutableList.of("99999999.99")}, - {10, 2, false, 10, 4, ImmutableList.of("99999999.99")}, - {18, 8, false, 32, 23, ImmutableList.of("1234567890.12345678")}, - {20, 8, false, 32, 21, ImmutableList.of("123456789012.12345678")}, - }); + for (WriterVersion writerVersion : WriterVersion.values()) { + testParquetLongFixedLenByteArrayWithTrinoShortDecimal(5, 2, 19, 2, "-5", writerVersion); + testParquetLongFixedLenByteArrayWithTrinoShortDecimal(5, 2, 20, 2, "999.99", writerVersion); + testParquetLongFixedLenByteArrayWithTrinoShortDecimal(7, 2, 24, 2, "-99999.99", writerVersion); + testParquetLongFixedLenByteArrayWithTrinoShortDecimal(10, 2, 26, 2, "99999999.99", writerVersion); + testParquetLongFixedLenByteArrayWithTrinoShortDecimal(14, 4, 30, 4, "99999999.99", writerVersion); + testParquetLongFixedLenByteArrayWithTrinoShortDecimal(18, 8, 32, 8, "1234567890.12345678", writerVersion); + testParquetLongFixedLenByteArrayWithTrinoShortDecimal(18, 8, 32, 8, "123456789012.12345678", writerVersion); + testParquetLongFixedLenByteArrayWithTrinoShortDecimal(18, 8, 38, 8, "4989875563210.12345678", writerVersion); + } } - @Test(dataProvider = "testParquetLongFixedLenByteArrayWithTrinoShortDecimalProvider") - public void testParquetLongFixedLenByteArrayWithTrinoShortDecimal( + private void testParquetLongFixedLenByteArrayWithTrinoShortDecimal( int schemaPrecision, int schemaScale, int parquetPrecision, @@ -368,22 +526,6 @@ public void testParquetLongFixedLenByteArrayWithTrinoShortDecimal( dropTable(tableName); } - @DataProvider - public Object[][] testParquetLongFixedLenByteArrayWithTrinoShortDecimalProvider() - { - // schemaPrecision, schemaScale, parquetPrecision, parquetScale, writeValue - return withWriterVersion(new Object[][] { - {5, 2, 19, 2, "-5"}, - {5, 2, 20, 2, "999.99"}, - {7, 2, 24, 2, "-99999.99"}, - {10, 2, 26, 2, "99999999.99"}, - {14, 4, 30, 4, "99999999.99"}, - {18, 8, 32, 8, "1234567890.12345678"}, - {18, 8, 32, 8, "123456789012.12345678"}, - {18, 8, 38, 8, "4989875563210.12345678"}, - }); - } - protected void createTable(String tableName, int precision, int scale) { assertUpdate(format("CREATE TABLE tpch.%s (value decimal(%d, %d)) WITH (format = 'PARQUET')", tableName, precision, scale)); @@ -518,13 +660,6 @@ private static String generateTableName(String testCase, int precision, int scal return format("%s_%d_%d_%d", testCase, precision, scale, ThreadLocalRandom.current().nextInt(1, MAX_VALUE)); } - private static Object[][] withWriterVersion(Object[][] args) - { - Object[][] versions = Stream.of(WriterVersion.values()) - .collect(toDataProvider()); - return cartesianProduct(args, versions); - } - protected static class ParquetDecimalInsert { private final String columnName; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestParquetPageSourceFactory.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestParquetPageSourceFactory.java index 2a24941ea5c0e..b596fbd6555e7 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestParquetPageSourceFactory.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestParquetPageSourceFactory.java @@ -22,7 +22,6 @@ import org.apache.parquet.schema.GroupType; import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.PrimitiveType; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import java.util.Optional; @@ -37,8 +36,14 @@ public class TestParquetPageSourceFactory { - @Test(dataProvider = "useColumnNames") - public void testGetNestedMixedRepetitionColumnType(boolean useColumnNames) + @Test + public void testGetNestedMixedRepetitionColumnType() + { + testGetNestedMixedRepetitionColumnType(true); + testGetNestedMixedRepetitionColumnType(false); + } + + private void testGetNestedMixedRepetitionColumnType(boolean useColumnNames) { RowType rowType = rowType( RowType.field( @@ -66,13 +71,4 @@ public void testGetNestedMixedRepetitionColumnType(boolean useColumnNames) new PrimitiveType(REQUIRED, INT32, "required_level3")))); assertThat(ParquetPageSourceFactory.getColumnType(columnHandle, fileSchema, useColumnNames).get()).isEqualTo(fileSchema.getType("optional_level1")); } - - @DataProvider - public Object[][] useColumnNames() - { - return new Object[][] { - {true}, // use column name - {false} // use column index - }; - } } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestParquetReader.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestParquetReader.java index b8b43f2f9877e..458f75690af97 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestParquetReader.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestParquetReader.java @@ -13,12 +13,15 @@ */ package io.trino.plugin.hive.parquet; -import org.testng.annotations.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.parallel.Execution; + +import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; // Failing on multiple threads because of org.apache.hadoop.hive.ql.io.parquet.write.ParquetRecordWriterWrapper // uses a single record writer across all threads. // For example org.apache.parquet.column.values.factory.DefaultValuesWriterFactory#DEFAULT_V1_WRITER_FACTORY is shared mutable state. -@Test(singleThreaded = true) +@Execution(SAME_THREAD) public class TestParquetReader extends AbstractTestParquetReader { diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/predicate/TestParquetPredicateUtils.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/predicate/TestParquetPredicateUtils.java index 9aec2afa6bc60..9cf41b496855e 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/predicate/TestParquetPredicateUtils.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/predicate/TestParquetPredicateUtils.java @@ -29,7 +29,6 @@ import org.apache.parquet.schema.GroupType; import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.PrimitiveType; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import java.util.List; @@ -53,8 +52,8 @@ public class TestParquetPredicateUtils { - @Test(dataProvider = "useColumnNames") - public void testParquetTupleDomainPrimitiveArray(boolean useColumnNames) + @Test + public void testParquetTupleDomainPrimitiveArray() { HiveColumnHandle columnHandle = createBaseColumn("my_array", 0, HiveType.valueOf("array"), new ArrayType(INTEGER), REGULAR, Optional.empty()); TupleDomain domain = withColumnDomains(ImmutableMap.of(columnHandle, Domain.notNull(new ArrayType(INTEGER)))); @@ -64,12 +63,13 @@ public void testParquetTupleDomainPrimitiveArray(boolean useColumnNames) new GroupType(REPEATED, "bag", new PrimitiveType(OPTIONAL, INT32, "array_element")))); Map, ColumnDescriptor> descriptorsByPath = getDescriptors(fileSchema, fileSchema); - TupleDomain tupleDomain = getParquetTupleDomain(descriptorsByPath, domain, fileSchema, useColumnNames); - assertThat(tupleDomain.isAll()).isTrue(); + + assertThat(getParquetTupleDomain(descriptorsByPath, domain, fileSchema, true).isAll()).isTrue(); + assertThat(getParquetTupleDomain(descriptorsByPath, domain, fileSchema, false).isAll()).isTrue(); } - @Test(dataProvider = "useColumnNames") - public void testParquetTupleDomainStructArray(boolean useColumnNames) + @Test + public void testParquetTupleDomainStructArray() { RowType.Field rowField = new RowType.Field(Optional.of("a"), INTEGER); RowType rowType = RowType.from(ImmutableList.of(rowField)); @@ -84,12 +84,19 @@ public void testParquetTupleDomainStructArray(boolean useColumnNames) new GroupType(OPTIONAL, "array_element", new PrimitiveType(OPTIONAL, INT32, "a"))))); Map, ColumnDescriptor> descriptorsByPath = getDescriptors(fileSchema, fileSchema); - TupleDomain tupleDomain = getParquetTupleDomain(descriptorsByPath, domain, fileSchema, useColumnNames); - assertThat(tupleDomain.isAll()).isTrue(); + + assertThat(getParquetTupleDomain(descriptorsByPath, domain, fileSchema, true).isAll()).isTrue(); + assertThat(getParquetTupleDomain(descriptorsByPath, domain, fileSchema, false).isAll()).isTrue(); } - @Test(dataProvider = "useColumnNames") - public void testParquetTupleDomainPrimitive(boolean useColumnNames) + @Test + public void testParquetTupleDomainPrimitive() + { + testParquetTupleDomainPrimitive(true); + testParquetTupleDomainPrimitive(false); + } + + private void testParquetTupleDomainPrimitive(boolean useColumnNames) { HiveColumnHandle columnHandle = createBaseColumn("my_primitive", 0, HiveType.valueOf("bigint"), BIGINT, REGULAR, Optional.empty()); Domain singleValueDomain = Domain.singleValue(BIGINT, 123L); @@ -109,8 +116,8 @@ public void testParquetTupleDomainPrimitive(boolean useColumnNames) assertThat(predicateDomain).isEqualTo(singleValueDomain); } - @Test(dataProvider = "useColumnNames") - public void testParquetTupleDomainStruct(boolean useColumnNames) + @Test + public void testParquetTupleDomainStruct() { RowType rowType = rowType( RowType.field("a", INTEGER), @@ -125,12 +132,19 @@ public void testParquetTupleDomainStruct(boolean useColumnNames) new PrimitiveType(OPTIONAL, INT32, "b"), new PrimitiveType(OPTIONAL, INT32, "c"))); Map, ColumnDescriptor> descriptorsByPath = getDescriptors(fileSchema, fileSchema); - TupleDomain tupleDomain = getParquetTupleDomain(descriptorsByPath, domain, fileSchema, useColumnNames); - assertThat(tupleDomain.isAll()).isTrue(); + + assertThat(getParquetTupleDomain(descriptorsByPath, domain, fileSchema, true).isAll()).isTrue(); + assertThat(getParquetTupleDomain(descriptorsByPath, domain, fileSchema, false).isAll()).isTrue(); } - @Test(dataProvider = "useColumnNames") - public void testParquetTupleDomainStructWithPrimitiveColumnPredicate(boolean useColumNames) + @Test + public void testParquetTupleDomainStructWithPrimitiveColumnPredicate() + { + testParquetTupleDomainStructWithPrimitiveColumnPredicate(true); + testParquetTupleDomainStructWithPrimitiveColumnPredicate(false); + } + + private void testParquetTupleDomainStructWithPrimitiveColumnPredicate(boolean useColumNames) { RowType baseType = rowType( RowType.field("a", INTEGER), @@ -167,8 +181,8 @@ public void testParquetTupleDomainStructWithPrimitiveColumnPredicate(boolean use assertThat(calculatedTupleDomain.getDomains().get()).containsEntry(selectedColumnDescriptor, predicateDomain); } - @Test(dataProvider = "useColumnNames") - public void testParquetTupleDomainStructWithComplexColumnPredicate(boolean useColumNames) + @Test + public void testParquetTupleDomainStructWithComplexColumnPredicate() { RowType c1Type = rowType( RowType.field("c1", INTEGER), @@ -206,12 +220,12 @@ public void testParquetTupleDomainStructWithComplexColumnPredicate(boolean useCo new PrimitiveType(OPTIONAL, INT32, "c2")))); Map, ColumnDescriptor> descriptorsByPath = getDescriptors(fileSchema, fileSchema); // skip looking up predicates for complex types as Parquet only stores stats for primitives - TupleDomain calculatedTupleDomain = getParquetTupleDomain(descriptorsByPath, tupleDomain, fileSchema, useColumNames); - assertThat(calculatedTupleDomain.isAll()).isTrue(); + assertThat(getParquetTupleDomain(descriptorsByPath, tupleDomain, fileSchema, true).isAll()).isTrue(); + assertThat(getParquetTupleDomain(descriptorsByPath, tupleDomain, fileSchema, false).isAll()).isTrue(); } - @Test(dataProvider = "useColumnNames") - public void testParquetTupleDomainStructWithMissingPrimitiveColumn(boolean useColumnNames) + @Test + public void testParquetTupleDomainStructWithMissingPrimitiveColumn() { RowType baseType = rowType( RowType.field("a", INTEGER), @@ -241,12 +255,12 @@ public void testParquetTupleDomainStructWithMissingPrimitiveColumn(boolean useCo new PrimitiveType(OPTIONAL, INT32, "a"), new PrimitiveType(OPTIONAL, INT32, "b"))); Map, ColumnDescriptor> descriptorsByPath = getDescriptors(fileSchema, fileSchema); - TupleDomain calculatedTupleDomain = getParquetTupleDomain(descriptorsByPath, tupleDomain, fileSchema, useColumnNames); - assertThat(calculatedTupleDomain.isAll()).isTrue(); + assertThat(getParquetTupleDomain(descriptorsByPath, tupleDomain, fileSchema, true).isAll()).isTrue(); + assertThat(getParquetTupleDomain(descriptorsByPath, tupleDomain, fileSchema, false).isAll()).isTrue(); } - @Test(dataProvider = "useColumnNames") - public void testParquetTupleDomainMap(boolean useColumnNames) + @Test + public void testParquetTupleDomainMap() { MapType mapType = new MapType(INTEGER, INTEGER, new TypeOperators()); @@ -261,16 +275,7 @@ public void testParquetTupleDomainMap(boolean useColumnNames) new PrimitiveType(OPTIONAL, INT32, "value")))); Map, ColumnDescriptor> descriptorsByPath = getDescriptors(fileSchema, fileSchema); - TupleDomain tupleDomain = getParquetTupleDomain(descriptorsByPath, domain, fileSchema, useColumnNames); - assertThat(tupleDomain.isAll()).isTrue(); - } - - @DataProvider - public Object[][] useColumnNames() - { - return new Object[][] { - {true}, // use column name - {false} // use column index - }; + assertThat(getParquetTupleDomain(descriptorsByPath, domain, fileSchema, true).isAll()).isTrue(); + assertThat(getParquetTupleDomain(descriptorsByPath, domain, fileSchema, false).isAll()).isTrue(); } } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergS3AndGlueMetastoreTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergS3AndGlueMetastoreTest.java index 326b38011ba3b..10b928858a91f 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergS3AndGlueMetastoreTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergS3AndGlueMetastoreTest.java @@ -104,8 +104,16 @@ protected Set getAllDataFilesFromTableDirectory(String tableLocation) .collect(Collectors.toUnmodifiableSet()); } - @Test(dataProvider = "locationPatternsDataProvider") - public void testAnalyzeWithProvidedTableLocation(boolean partitioned, LocationPattern locationPattern) + @Test + public void testAnalyzeWithProvidedTableLocation() + { + for (LocationPattern locationPattern : LocationPattern.values()) { + testAnalyzeWithProvidedTableLocation(false, locationPattern); + testAnalyzeWithProvidedTableLocation(true, locationPattern); + } + } + + private void testAnalyzeWithProvidedTableLocation(boolean partitioned, LocationPattern locationPattern) { String tableName = "test_analyze_" + randomNameSuffix(); String location = locationPattern.locationForTable(bucketName, schemaName, tableName); From 8cfddff78bff6fab861cb847e3a74a147bd8ccfd Mon Sep 17 00:00:00 2001 From: Martin Traverso Date: Fri, 8 Dec 2023 10:55:40 -0800 Subject: [PATCH 057/350] Migrate tests to JUnit --- .../hive/BaseS3AndGlueMetastoreTest.java | 16 +- .../hive/TestHiveS3AndGlueMetastoreTest.java | 2 +- ...stCachingHiveMetastoreWithQueryRunner.java | 9 +- .../parquet/TestParquetDecimalScaling.java | 2 +- .../parquet/TestParquetPageSourceFactory.java | 2 +- .../predicate/TestParquetPredicateUtils.java | 2 +- plugin/trino-iceberg/pom.xml | 32 ---- .../TestIcebergS3AndGlueMetastoreTest.java | 2 +- testing/trino-testing/pom.xml | 27 --- .../testing/AbstractTestQueryFramework.java | 4 - testing/trino-tests/pom.xml | 38 ---- .../runtime/TestSystemRuntimeConnector.java | 164 ++++++++++-------- 12 files changed, 117 insertions(+), 183 deletions(-) diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseS3AndGlueMetastoreTest.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseS3AndGlueMetastoreTest.java index 380f787410c10..e45f3d6d9a4ba 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseS3AndGlueMetastoreTest.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseS3AndGlueMetastoreTest.java @@ -23,9 +23,11 @@ import io.trino.spi.connector.SchemaNotFoundException; import io.trino.testing.AbstractTestQueryFramework; import org.intellij.lang.annotations.Language; -import org.testng.annotations.AfterClass; -import org.testng.annotations.BeforeClass; -import org.testng.annotations.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.parallel.Execution; import java.util.List; import java.util.Set; @@ -40,7 +42,11 @@ import static java.lang.String.format; import static java.util.Objects.requireNonNull; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; +@TestInstance(PER_CLASS) +@Execution(CONCURRENT) public abstract class BaseS3AndGlueMetastoreTest extends AbstractTestQueryFramework { @@ -60,13 +66,13 @@ protected BaseS3AndGlueMetastoreTest(String partitionByKeyword, String locationK this.bucketName = requireNonNull(bucketName, "bucketName is null"); } - @BeforeClass + @BeforeAll public void setUp() { s3 = AmazonS3ClientBuilder.standard().build(); } - @AfterClass(alwaysRun = true) + @AfterAll public void tearDown() { if (metastore != null) { diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveS3AndGlueMetastoreTest.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveS3AndGlueMetastoreTest.java index f7a7dd9e9e468..0487184e504fa 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveS3AndGlueMetastoreTest.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveS3AndGlueMetastoreTest.java @@ -20,7 +20,7 @@ import io.trino.spi.security.SelectedRole; import io.trino.testing.DistributedQueryRunner; import io.trino.testing.QueryRunner; -import org.testng.annotations.Test; +import org.junit.jupiter.api.Test; import java.nio.file.Path; import java.util.HashSet; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastoreWithQueryRunner.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastoreWithQueryRunner.java index 81e20492e8ae1..a76e5c944498d 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastoreWithQueryRunner.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastoreWithQueryRunner.java @@ -28,7 +28,9 @@ import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.DistributedQueryRunner; import io.trino.testing.QueryRunner; -import org.testng.annotations.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.parallel.Execution; import java.util.List; import java.util.Optional; @@ -38,8 +40,11 @@ import static io.trino.testing.TestingSession.testSessionBuilder; import static java.util.Collections.nCopies; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; -@Test(singleThreaded = true) +@TestInstance(PER_CLASS) +@Execution(SAME_THREAD) public class TestCachingHiveMetastoreWithQueryRunner extends AbstractTestQueryFramework { diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestParquetDecimalScaling.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestParquetDecimalScaling.java index 1c02d1cda860f..89b1a2445076b 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestParquetDecimalScaling.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestParquetDecimalScaling.java @@ -36,7 +36,7 @@ import org.apache.parquet.schema.MessageType; import org.intellij.lang.annotations.Language; import org.joda.time.DateTimeZone; -import org.testng.annotations.Test; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.io.UncheckedIOException; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestParquetPageSourceFactory.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestParquetPageSourceFactory.java index b596fbd6555e7..2e67a4838f861 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestParquetPageSourceFactory.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestParquetPageSourceFactory.java @@ -22,7 +22,7 @@ import org.apache.parquet.schema.GroupType; import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.PrimitiveType; -import org.testng.annotations.Test; +import org.junit.jupiter.api.Test; import java.util.Optional; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/predicate/TestParquetPredicateUtils.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/predicate/TestParquetPredicateUtils.java index 9cf41b496855e..8a64032f6c4bc 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/predicate/TestParquetPredicateUtils.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/predicate/TestParquetPredicateUtils.java @@ -29,7 +29,7 @@ import org.apache.parquet.schema.GroupType; import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.PrimitiveType; -import org.testng.annotations.Test; +import org.junit.jupiter.api.Test; import java.util.List; import java.util.Map; diff --git a/plugin/trino-iceberg/pom.xml b/plugin/trino-iceberg/pom.xml index ee25b31756783..9e6e78c811fb7 100644 --- a/plugin/trino-iceberg/pom.xml +++ b/plugin/trino-iceberg/pom.xml @@ -15,15 +15,6 @@ ${project.parent.basedir} - - - instances 0.71.0 @@ -603,12 +594,6 @@ test - - org.testng - testng - test - - org.xerial sqlite-jdbc @@ -646,23 +631,6 @@ - - org.apache.maven.plugins - maven-surefire-plugin - - - - org.apache.maven.surefire - surefire-junit-platform - ${dep.plugin.surefire.version} - - - org.apache.maven.surefire - surefire-testng - ${dep.plugin.surefire.version} - - - diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergS3AndGlueMetastoreTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergS3AndGlueMetastoreTest.java index 10b928858a91f..bc63f29b491ce 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergS3AndGlueMetastoreTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergS3AndGlueMetastoreTest.java @@ -19,7 +19,7 @@ import io.trino.plugin.iceberg.IcebergQueryRunner; import io.trino.testing.DistributedQueryRunner; import io.trino.testing.QueryRunner; -import org.testng.annotations.Test; +import org.junit.jupiter.api.Test; import java.nio.file.Path; import java.util.Set; diff --git a/testing/trino-testing/pom.xml b/testing/trino-testing/pom.xml index 0e3bba2867006..5325154710e6c 100644 --- a/testing/trino-testing/pom.xml +++ b/testing/trino-testing/pom.xml @@ -179,11 +179,6 @@ junit-jupiter-api - - org.testng - testng - - org.jetbrains annotations @@ -220,26 +215,4 @@ test - - - - - org.apache.maven.plugins - maven-surefire-plugin - - - - org.apache.maven.surefire - surefire-junit-platform - ${dep.plugin.surefire.version} - - - org.apache.maven.surefire - surefire-testng - ${dep.plugin.surefire.version} - - - - - diff --git a/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestQueryFramework.java b/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestQueryFramework.java index 00c67b1dca148..5b6af5f28b3d5 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestQueryFramework.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestQueryFramework.java @@ -61,8 +61,6 @@ import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.TestInstance; import org.junit.jupiter.api.parallel.Execution; -import org.testng.annotations.AfterClass; -import org.testng.annotations.BeforeClass; import java.util.List; import java.util.Map; @@ -104,7 +102,6 @@ public abstract class AbstractTestQueryFramework private H2QueryRunner h2QueryRunner; private io.trino.sql.query.QueryAssertions queryAssertions; - @BeforeClass @BeforeAll public void init() throws Exception @@ -122,7 +119,6 @@ protected abstract QueryRunner createQueryRunner() throws Exception; @AfterAll - @AfterClass(alwaysRun = true) public final void close() throws Exception { diff --git a/testing/trino-tests/pom.xml b/testing/trino-tests/pom.xml index baab8ee092859..2d1f6efffd065 100644 --- a/testing/trino-tests/pom.xml +++ b/testing/trino-tests/pom.xml @@ -13,16 +13,6 @@ ${project.parent.basedir} - - - instances @@ -318,33 +308,5 @@ jmh-generator-annprocess test - - - org.testng - testng - test - - - - - - org.apache.maven.plugins - maven-surefire-plugin - - - - org.apache.maven.surefire - surefire-junit-platform - ${dep.plugin.surefire.version} - - - org.apache.maven.surefire - surefire-testng - ${dep.plugin.surefire.version} - - - - - diff --git a/testing/trino-tests/src/test/java/io/trino/connector/system/runtime/TestSystemRuntimeConnector.java b/testing/trino-tests/src/test/java/io/trino/connector/system/runtime/TestSystemRuntimeConnector.java index a536672a36177..b8464b6100e17 100644 --- a/testing/trino-tests/src/test/java/io/trino/connector/system/runtime/TestSystemRuntimeConnector.java +++ b/testing/trino-tests/src/test/java/io/trino/connector/system/runtime/TestSystemRuntimeConnector.java @@ -29,9 +29,10 @@ import io.trino.testing.MaterializedResult; import io.trino.testing.MaterializedRow; import io.trino.testing.QueryRunner; -import org.testng.annotations.AfterClass; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.parallel.Execution; import java.time.ZonedDateTime; import java.util.List; @@ -52,15 +53,16 @@ import static java.lang.String.format; import static java.util.concurrent.TimeUnit.SECONDS; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; -@Test(singleThreaded = true) +@Execution(SAME_THREAD) public class TestSystemRuntimeConnector extends AbstractTestQueryFramework { private static final Function> DEFAULT_GET_COLUMNS = table -> ImmutableList.of(new ColumnMetadata("c", VARCHAR)); private static final AtomicLong counter = new AtomicLong(); - private static Function> getColumns = DEFAULT_GET_COLUMNS; + private static Function> getColumns; private final ExecutorService executor = Executors.newSingleThreadScheduledExecutor(threadsNamed(TestSystemRuntimeConnector.class.getSimpleName())); @@ -94,13 +96,7 @@ public Iterable getConnectorFactories() return queryRunner; } - @BeforeMethod - public void cleanup() - { - getColumns = DEFAULT_GET_COLUMNS; - } - - @AfterClass(alwaysRun = true) + @AfterAll public void tearDown() { executor.shutdownNow(); @@ -117,81 +113,88 @@ public void testRuntimeNodes() "('testversion', false, 'active')"); } - // Test is run multiple times because it is vulnerable to OS clock adjustment. See https://github.com/trinodb/trino/issues/5608 - @Test(invocationCount = 10, successPercentage = 80) + @Test public void testRuntimeQueriesTimestamps() { - ZonedDateTime timeBefore = ZonedDateTime.now(); - computeActual("SELECT 1"); - MaterializedResult result = computeActual("" + - "SELECT max(created), max(started), max(last_heartbeat), max(\"end\") " + - "FROM system.runtime.queries"); - ZonedDateTime timeAfter = ZonedDateTime.now(); + // Test is run multiple times because it is vulnerable to OS clock adjustment. See https://github.com/trinodb/trino/issues/5608 + run(10, 0.8, () -> { + ZonedDateTime timeBefore = ZonedDateTime.now(); + computeActual("SELECT 1"); + MaterializedResult result = computeActual("" + + "SELECT max(created), max(started), max(last_heartbeat), max(\"end\") " + + "FROM system.runtime.queries"); + ZonedDateTime timeAfter = ZonedDateTime.now(); - MaterializedRow row = Iterables.getOnlyElement(result.toTestTypes().getMaterializedRows()); - List fields = row.getFields(); - assertThat(fields).hasSize(4); - for (int i = 0; i < fields.size(); i++) { - Object value = fields.get(i); - assertThat((ZonedDateTime) value) - .as("value for field " + i) - .isNotNull() - .isAfterOrEqualTo(timeBefore) - .isBeforeOrEqualTo(timeAfter); - } + MaterializedRow row = Iterables.getOnlyElement(result.toTestTypes().getMaterializedRows()); + List fields = row.getFields(); + assertThat(fields).hasSize(4); + for (int i = 0; i < fields.size(); i++) { + Object value = fields.get(i); + assertThat((ZonedDateTime) value) + .as("value for field " + i) + .isNotNull() + .isAfterOrEqualTo(timeBefore) + .isBeforeOrEqualTo(timeAfter); + } + }); } - // Test is run multiple times because it is vulnerable to OS clock adjustment. See https://github.com/trinodb/trino/issues/5608 - @Test(invocationCount = 10, successPercentage = 80) + @Test public void testRuntimeTasksTimestamps() { - ZonedDateTime timeBefore = ZonedDateTime.now(); - computeActual("SELECT 1"); - MaterializedResult result = computeActual("" + - "SELECT max(created), max(start), max(last_heartbeat), max(\"end\") " + - "FROM system.runtime.tasks"); - ZonedDateTime timeAfter = ZonedDateTime.now(); + // Test is run multiple times because it is vulnerable to OS clock adjustment. See https://github.com/trinodb/trino/issues/5608 + run(10, 0.8, () -> { + ZonedDateTime timeBefore = ZonedDateTime.now(); + computeActual("SELECT 1"); + MaterializedResult result = computeActual("" + + "SELECT max(created), max(start), max(last_heartbeat), max(\"end\") " + + "FROM system.runtime.tasks"); + ZonedDateTime timeAfter = ZonedDateTime.now(); - MaterializedRow row = Iterables.getOnlyElement(result.toTestTypes().getMaterializedRows()); - List fields = row.getFields(); - assertThat(fields).hasSize(4); - for (int i = 0; i < fields.size(); i++) { - Object value = fields.get(i); - assertThat((ZonedDateTime) value) - .as("value for field " + i) - .isNotNull() - .isAfterOrEqualTo(timeBefore) - .isBeforeOrEqualTo(timeAfter); - } + MaterializedRow row = Iterables.getOnlyElement(result.toTestTypes().getMaterializedRows()); + List fields = row.getFields(); + assertThat(fields).hasSize(4); + for (int i = 0; i < fields.size(); i++) { + Object value = fields.get(i); + assertThat((ZonedDateTime) value) + .as("value for field " + i) + .isNotNull() + .isAfterOrEqualTo(timeBefore) + .isBeforeOrEqualTo(timeAfter); + } + }); } // Test is run multiple times because it is vulnerable to OS clock adjustment. See https://github.com/trinodb/trino/issues/5608 - @Test(invocationCount = 10, successPercentage = 80) + @Test public void testRuntimeTransactionsTimestamps() { - ZonedDateTime timeBefore = ZonedDateTime.now(); - computeActual("START TRANSACTION"); - MaterializedResult result = computeActual("" + - "SELECT max(create_time) " + - "FROM system.runtime.transactions"); - ZonedDateTime timeAfter = ZonedDateTime.now(); + run(10, 0.8, () -> { + ZonedDateTime timeBefore = ZonedDateTime.now(); + computeActual("START TRANSACTION"); + MaterializedResult result = computeActual("" + + "SELECT max(create_time) " + + "FROM system.runtime.transactions"); + ZonedDateTime timeAfter = ZonedDateTime.now(); - MaterializedRow row = Iterables.getOnlyElement(result.toTestTypes().getMaterializedRows()); - List fields = row.getFields(); - assertThat(fields).hasSize(1); - for (int i = 0; i < fields.size(); i++) { - Object value = fields.get(i); - assertThat((ZonedDateTime) value) - .as("value for field " + i) - .isNotNull() - .isAfterOrEqualTo(timeBefore) - .isBeforeOrEqualTo(timeAfter); - } + MaterializedRow row = Iterables.getOnlyElement(result.toTestTypes().getMaterializedRows()); + List fields = row.getFields(); + assertThat(fields).hasSize(1); + for (int i = 0; i < fields.size(); i++) { + Object value = fields.get(i); + assertThat((ZonedDateTime) value) + .as("value for field " + i) + .isNotNull() + .isAfterOrEqualTo(timeBefore) + .isBeforeOrEqualTo(timeAfter); + } + }); } @Test public void testFinishedQueryIsCaptured() { + getColumns = DEFAULT_GET_COLUMNS; String testQueryId = "test_query_id_" + counter.incrementAndGet(); getQueryRunner().execute(format("EXPLAIN SELECT 1 AS %s FROM test_table", testQueryId)); @@ -200,7 +203,8 @@ public void testFinishedQueryIsCaptured() "VALUES 'FINISHED'"); } - @Test(timeOut = 60_000) + @Test + @Timeout(60) public void testQueryDuringAnalysisIsCaptured() { SettableFuture> metadataFuture = SettableFuture.create(); @@ -236,7 +240,8 @@ public void testQueryDuringAnalysisIsCaptured() assertEventually(new Duration(5, SECONDS), () -> assertThat(queryFuture.isDone()).isTrue()); } - @Test(timeOut = 60_000) + @Test + @Timeout(60) public void testQueryKillingDuringAnalysis() { SettableFuture> metadataFuture = SettableFuture.create(); @@ -285,4 +290,23 @@ public void testTasksTable() getQueryRunner().execute("SELECT 1"); getQueryRunner().execute("SELECT * FROM system.runtime.tasks"); } + + private static void run(int repetitions, double successRate, Runnable test) + { + AssertionError lastError = null; + int failures = 0; + for (int iteration = 0; iteration < repetitions; iteration++) { + try { + test.run(); + } + catch (AssertionError e) { + failures++; + lastError = e; + } + } + + if (lastError != null && 1 - (failures * 1.0) / repetitions < successRate) { + throw lastError; + } + } } From 400a60847eb9d6d05ce28a7fce484dc7ca18bf8b Mon Sep 17 00:00:00 2001 From: Marius Grama Date: Fri, 24 Nov 2023 16:10:01 +0100 Subject: [PATCH 058/350] Rename internally used interface to CheckpointFieldExtractor --- .../transactionlog/checkpoint/CheckpointEntryIterator.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointEntryIterator.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointEntryIterator.java index 9b90eaf50930c..cdb82298c7d17 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointEntryIterator.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointEntryIterator.java @@ -136,7 +136,7 @@ public String getColumnName() private final MapType stringMap; private final ArrayType stringList; private final Queue nextEntries; - private final List extractors; + private final List extractors; private final boolean checkpointRowStatisticsWritingEnabled; private final TupleDomain partitionConstraint; private MetadataEntry metadataEntry; @@ -171,7 +171,7 @@ public CheckpointEntryIterator( this.partitionConstraint = requireNonNull(partitionConstraint, "partitionConstraint is null"); requireNonNull(addStatsMinMaxColumnFilter, "addStatsMinMaxColumnFilter is null"); checkArgument(!fields.isEmpty(), "fields is empty"); - Map extractors = ImmutableMap.builder() + Map extractors = ImmutableMap.builder() .put(TRANSACTION, this::buildTxnEntry) .put(ADD, this::buildAddEntry) .put(REMOVE, this::buildRemoveEntry) @@ -716,7 +716,7 @@ long getCompletedBytes() } @FunctionalInterface - public interface CheckPointFieldExtractor + private interface CheckpointFieldExtractor { @Nullable DeltaLakeTransactionLogEntry getEntry(ConnectorSession session, Block block, int pagePosition); From 1296fc2f0d9fd8418858c10dd219e1fbe3ccacd2 Mon Sep 17 00:00:00 2001 From: Marius Grama Date: Mon, 27 Nov 2023 17:22:13 +0100 Subject: [PATCH 059/350] Compute `deletionVectorsEnabled` only once per checkpoint file --- .../transactionlog/checkpoint/CheckpointEntryIterator.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointEntryIterator.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointEntryIterator.java index cdb82298c7d17..207733546bb3d 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointEntryIterator.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointEntryIterator.java @@ -141,6 +141,7 @@ public String getColumnName() private final TupleDomain partitionConstraint; private MetadataEntry metadataEntry; private ProtocolEntry protocolEntry; + private boolean deletionVectorsEnabled; private List schema; private List columnsWithMinMaxStats; private Page page; @@ -185,6 +186,7 @@ public CheckpointEntryIterator( this.metadataEntry = metadataEntry.get(); checkArgument(protocolEntry.isPresent(), "Protocol entry must be provided when reading ADD entries from Checkpoint files"); this.protocolEntry = protocolEntry.get(); + deletionVectorsEnabled = isDeletionVectorEnabled(this.metadataEntry, this.protocolEntry); checkArgument(addStatsMinMaxColumnFilter.isPresent(), "addStatsMinMaxColumnFilter must be provided when reading ADD entries from Checkpoint files"); this.schema = extractSchema(this.metadataEntry, this.protocolEntry, typeManager); this.columnsWithMinMaxStats = columnsWithStats(schema, this.metadataEntry.getOriginalPartitionColumns()); @@ -466,7 +468,6 @@ private DeltaLakeTransactionLogEntry buildAddEntry(ConnectorSession session, Blo return null; } RowType type = (RowType) parquetFields.get("add"); - boolean deletionVectorsEnabled = isDeletionVectorEnabled(metadataEntry, protocolEntry); SqlRow addEntryRow = block.getObject(pagePosition, SqlRow.class); log.debug("Block %s has %s fields", block, addEntryRow.getFieldCount()); CheckpointFieldReader add = new CheckpointFieldReader(session, addEntryRow, type); From a23cff2c4ab15c8324e980879df095515dc8edc6 Mon Sep 17 00:00:00 2001 From: Marius Grama Date: Fri, 17 Nov 2023 12:29:45 +0100 Subject: [PATCH 060/350] Construct AddFileEntry instance only if necessary When checkpoint filtering is applied and there are partition constraints which do not match the partition values of the entry, avoid eagerly to construct `AddFileEntry` instances. --- .../transactionlog/AddFileEntry.java | 29 +++++++++++++++++-- .../checkpoint/CheckpointEntryIterator.java | 25 +++++++++------- 2 files changed, 42 insertions(+), 12 deletions(-) diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/AddFileEntry.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/AddFileEntry.java index 5b6c96d3a31c6..ed7a4d646b9e1 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/AddFileEntry.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/AddFileEntry.java @@ -61,10 +61,35 @@ public AddFileEntry( @JsonProperty("parsedStats") Optional parsedStats, @JsonProperty("tags") @Nullable Map tags, @JsonProperty("deletionVector") Optional deletionVector) + { + this( + path, + partitionValues, + canonicalizePartitionValues(partitionValues), + size, + modificationTime, + dataChange, + stats, + parsedStats, + tags, + deletionVector); + } + + public AddFileEntry( + String path, + Map partitionValues, + Map> canonicalPartitionValues, + long size, + long modificationTime, + boolean dataChange, + Optional stats, + Optional parsedStats, + @Nullable Map tags, + Optional deletionVector) { this.path = path; - this.partitionValues = partitionValues; - this.canonicalPartitionValues = canonicalizePartitionValues(partitionValues); + this.partitionValues = requireNonNull(partitionValues, "partitionValues is null"); + this.canonicalPartitionValues = requireNonNull(canonicalPartitionValues, "canonicalPartitionValues is null"); this.size = size; this.modificationTime = modificationTime; this.dataChange = dataChange; diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointEntryIterator.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointEntryIterator.java index 207733546bb3d..b6475453390c6 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointEntryIterator.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointEntryIterator.java @@ -83,6 +83,7 @@ import static io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.isDeletionVectorEnabled; import static io.trino.plugin.deltalake.transactionlog.TransactionLogAccess.columnsWithStats; import static io.trino.plugin.deltalake.transactionlog.TransactionLogParser.START_OF_MODERN_ERA_EPOCH_DAY; +import static io.trino.plugin.deltalake.transactionlog.TransactionLogUtil.canonicalizePartitionValues; import static io.trino.plugin.deltalake.transactionlog.checkpoint.CheckpointEntryIterator.EntryType.ADD; import static io.trino.plugin.deltalake.transactionlog.checkpoint.CheckpointEntryIterator.EntryType.COMMIT; import static io.trino.plugin.deltalake.transactionlog.checkpoint.CheckpointEntryIterator.EntryType.METADATA; @@ -472,8 +473,13 @@ private DeltaLakeTransactionLogEntry buildAddEntry(ConnectorSession session, Blo log.debug("Block %s has %s fields", block, addEntryRow.getFieldCount()); CheckpointFieldReader add = new CheckpointFieldReader(session, addEntryRow, type); - String path = add.getString("path"); Map partitionValues = add.getMap(stringMap, "partitionValues"); + Map> canonicalPartitionValues = canonicalizePartitionValues(partitionValues); + if (!partitionConstraint.isAll() && !partitionMatchesPredicate(canonicalPartitionValues, partitionConstraint.getDomains().orElseThrow())) { + return null; + } + + String path = add.getString("path"); long size = add.getLong("size"); long modificationTime = add.getLong("modificationTime"); boolean dataChange = add.getBoolean("dataChange"); @@ -500,6 +506,7 @@ private DeltaLakeTransactionLogEntry buildAddEntry(ConnectorSession session, Blo AddFileEntry result = new AddFileEntry( path, partitionValues, + canonicalPartitionValues, size, modificationTime, dataChange, @@ -689,15 +696,7 @@ private void fillNextEntries() for (int i = 0; i < extractors.size(); ++i) { DeltaLakeTransactionLogEntry entry = extractors.get(i).getEntry(session, page.getBlock(i).getLoadedBlock(), pagePosition); if (entry != null) { - if (entry.getAdd() != null) { - if (partitionConstraint.isAll() || - partitionMatchesPredicate(entry.getAdd().getCanonicalPartitionValues(), partitionConstraint.getDomains().orElseThrow())) { - nextEntries.add(entry); - } - } - else { - nextEntries.add(entry); - } + nextEntries.add(entry); } } pagePosition++; @@ -719,6 +718,12 @@ long getCompletedBytes() @FunctionalInterface private interface CheckpointFieldExtractor { + /** + * Returns the transaction log entry instance corresponding to the requested position in the memory block. + * The output of the operation may be `null` in case the block has no information at the requested position + * or if the during the retrieval process it is observed that the log entry does not correspond to the + * checkpoint filter criteria. + */ @Nullable DeltaLakeTransactionLogEntry getEntry(ConnectorSession session, Block block, int pagePosition); } From 8fa02da285652ead675a66a9a39bf75e67c024dd Mon Sep 17 00:00:00 2001 From: Marius Grama Date: Wed, 22 Nov 2023 19:49:55 +0100 Subject: [PATCH 061/350] Create CheckpointFieldExtractor instance only if necessary --- .../checkpoint/CheckpointEntryIterator.java | 27 +++++++++---------- 1 file changed, 13 insertions(+), 14 deletions(-) diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointEntryIterator.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointEntryIterator.java index b6475453390c6..b2981c3e4b86f 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointEntryIterator.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointEntryIterator.java @@ -85,11 +85,6 @@ import static io.trino.plugin.deltalake.transactionlog.TransactionLogParser.START_OF_MODERN_ERA_EPOCH_DAY; import static io.trino.plugin.deltalake.transactionlog.TransactionLogUtil.canonicalizePartitionValues; import static io.trino.plugin.deltalake.transactionlog.checkpoint.CheckpointEntryIterator.EntryType.ADD; -import static io.trino.plugin.deltalake.transactionlog.checkpoint.CheckpointEntryIterator.EntryType.COMMIT; -import static io.trino.plugin.deltalake.transactionlog.checkpoint.CheckpointEntryIterator.EntryType.METADATA; -import static io.trino.plugin.deltalake.transactionlog.checkpoint.CheckpointEntryIterator.EntryType.PROTOCOL; -import static io.trino.plugin.deltalake.transactionlog.checkpoint.CheckpointEntryIterator.EntryType.REMOVE; -import static io.trino.plugin.deltalake.transactionlog.checkpoint.CheckpointEntryIterator.EntryType.TRANSACTION; import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.spi.type.DateTimeEncoding.packDateTimeWithZone; import static io.trino.spi.type.TimeZoneKey.UTC_KEY; @@ -173,14 +168,6 @@ public CheckpointEntryIterator( this.partitionConstraint = requireNonNull(partitionConstraint, "partitionConstraint is null"); requireNonNull(addStatsMinMaxColumnFilter, "addStatsMinMaxColumnFilter is null"); checkArgument(!fields.isEmpty(), "fields is empty"); - Map extractors = ImmutableMap.builder() - .put(TRANSACTION, this::buildTxnEntry) - .put(ADD, this::buildAddEntry) - .put(REMOVE, this::buildRemoveEntry) - .put(METADATA, this::buildMetadataEntry) - .put(PROTOCOL, this::buildProtocolEntry) - .put(COMMIT, this::buildCommitInfoEntry) - .buildOrThrow(); // ADD requires knowing the metadata in order to figure out the Parquet schema if (fields.contains(ADD)) { checkArgument(metadataEntry.isPresent(), "Metadata entry must be provided when reading ADD entries from Checkpoint files"); @@ -227,10 +214,22 @@ public CheckpointEntryIterator( this.parquetFields = this.pageSource.getColumnFields().stream() .collect(toImmutableMap(Column::name, e -> e.field().getType())); this.extractors = fields.stream() - .map(field -> requireNonNull(extractors.get(field), "No extractor found for field " + field)) + .map(this::createCheckpointFieldExtractor) .collect(toImmutableList()); } + private CheckpointFieldExtractor createCheckpointFieldExtractor(EntryType entryType) + { + return switch (entryType) { + case TRANSACTION -> this::buildTxnEntry; + case ADD -> this::buildAddEntry; + case REMOVE -> this::buildRemoveEntry; + case METADATA -> this::buildMetadataEntry; + case PROTOCOL -> this::buildProtocolEntry; + case COMMIT -> this::buildCommitInfoEntry; + }; + } + private DeltaLakeColumnHandle buildColumnHandle( EntryType entryType, CheckpointSchemaManager schemaManager, From 49f88f1255b14d8aba86c824bdd3cceea018ef96 Mon Sep 17 00:00:00 2001 From: Marius Grama Date: Wed, 22 Nov 2023 19:53:49 +0100 Subject: [PATCH 062/350] Support building a DeltaLakeTransactionLogEntry from multiple blocks --- .../checkpoint/CheckpointEntryIterator.java | 28 +++++++++---------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointEntryIterator.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointEntryIterator.java index b2981c3e4b86f..b27b89029d68d 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointEntryIterator.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointEntryIterator.java @@ -221,12 +221,12 @@ public CheckpointEntryIterator( private CheckpointFieldExtractor createCheckpointFieldExtractor(EntryType entryType) { return switch (entryType) { - case TRANSACTION -> this::buildTxnEntry; - case ADD -> this::buildAddEntry; - case REMOVE -> this::buildRemoveEntry; - case METADATA -> this::buildMetadataEntry; - case PROTOCOL -> this::buildProtocolEntry; - case COMMIT -> this::buildCommitInfoEntry; + case TRANSACTION -> (session, pagePosition, blocks) -> buildTxnEntry(session, pagePosition, blocks[0]); + case ADD -> (session, pagePosition, blocks) -> buildAddEntry(session, pagePosition, blocks[0]); + case REMOVE -> (session, pagePosition, blocks) -> buildRemoveEntry(session, pagePosition, blocks[0]); + case METADATA -> (session, pagePosition, blocks) -> buildMetadataEntry(session, pagePosition, blocks[0]); + case PROTOCOL -> (session, pagePosition, blocks) -> buildProtocolEntry(session, pagePosition, blocks[0]); + case COMMIT -> (session, pagePosition, blocks) -> buildCommitInfoEntry(session, pagePosition, blocks[0]); }; } @@ -316,7 +316,7 @@ private static HiveColumnHandle toPartitionValuesParsedField(HiveColumnHandle ad addColumn.getComment()); } - private DeltaLakeTransactionLogEntry buildCommitInfoEntry(ConnectorSession session, Block block, int pagePosition) + private DeltaLakeTransactionLogEntry buildCommitInfoEntry(ConnectorSession session, int pagePosition, Block block) { log.debug("Building commitInfo entry from %s pagePosition %d", block, pagePosition); if (block.isNull(pagePosition)) { @@ -372,7 +372,7 @@ private DeltaLakeTransactionLogEntry buildCommitInfoEntry(ConnectorSession sessi return DeltaLakeTransactionLogEntry.commitInfoEntry(result); } - private DeltaLakeTransactionLogEntry buildProtocolEntry(ConnectorSession session, Block block, int pagePosition) + private DeltaLakeTransactionLogEntry buildProtocolEntry(ConnectorSession session, int pagePosition, Block block) { log.debug("Building protocol entry from %s pagePosition %d", block, pagePosition); if (block.isNull(pagePosition)) { @@ -399,7 +399,7 @@ private DeltaLakeTransactionLogEntry buildProtocolEntry(ConnectorSession session return DeltaLakeTransactionLogEntry.protocolEntry(result); } - private DeltaLakeTransactionLogEntry buildMetadataEntry(ConnectorSession session, Block block, int pagePosition) + private DeltaLakeTransactionLogEntry buildMetadataEntry(ConnectorSession session, int pagePosition, Block block) { log.debug("Building metadata entry from %s pagePosition %d", block, pagePosition); if (block.isNull(pagePosition)) { @@ -438,7 +438,7 @@ private DeltaLakeTransactionLogEntry buildMetadataEntry(ConnectorSession session return DeltaLakeTransactionLogEntry.metadataEntry(result); } - private DeltaLakeTransactionLogEntry buildRemoveEntry(ConnectorSession session, Block block, int pagePosition) + private DeltaLakeTransactionLogEntry buildRemoveEntry(ConnectorSession session, int pagePosition, Block block) { log.debug("Building remove entry from %s pagePosition %d", block, pagePosition); if (block.isNull(pagePosition)) { @@ -461,7 +461,7 @@ private DeltaLakeTransactionLogEntry buildRemoveEntry(ConnectorSession session, return DeltaLakeTransactionLogEntry.removeFileEntry(result); } - private DeltaLakeTransactionLogEntry buildAddEntry(ConnectorSession session, Block block, int pagePosition) + private DeltaLakeTransactionLogEntry buildAddEntry(ConnectorSession session, int pagePosition, Block block) { log.debug("Building add entry from %s pagePosition %d", block, pagePosition); if (block.isNull(pagePosition)) { @@ -619,7 +619,7 @@ private Map parseNullCount(SqlRow row, List Date: Wed, 22 Nov 2023 20:04:20 +0100 Subject: [PATCH 063/350] Extract `buildAddEntry` method logic to `AddFileEntryExtractor` class --- .../checkpoint/CheckpointEntryIterator.java | 113 ++++++++++-------- 1 file changed, 61 insertions(+), 52 deletions(-) diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointEntryIterator.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointEntryIterator.java index b27b89029d68d..0a54bbbf4a7b9 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointEntryIterator.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointEntryIterator.java @@ -72,6 +72,7 @@ import java.util.function.Predicate; 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.collect.ImmutableList.toImmutableList; import static com.google.common.collect.ImmutableMap.toImmutableMap; @@ -222,7 +223,7 @@ private CheckpointFieldExtractor createCheckpointFieldExtractor(EntryType entryT { return switch (entryType) { case TRANSACTION -> (session, pagePosition, blocks) -> buildTxnEntry(session, pagePosition, blocks[0]); - case ADD -> (session, pagePosition, blocks) -> buildAddEntry(session, pagePosition, blocks[0]); + case ADD -> new AddFileEntryExtractor(); case REMOVE -> (session, pagePosition, blocks) -> buildRemoveEntry(session, pagePosition, blocks[0]); case METADATA -> (session, pagePosition, blocks) -> buildMetadataEntry(session, pagePosition, blocks[0]); case PROTOCOL -> (session, pagePosition, blocks) -> buildProtocolEntry(session, pagePosition, blocks[0]); @@ -461,61 +462,69 @@ private DeltaLakeTransactionLogEntry buildRemoveEntry(ConnectorSession session, return DeltaLakeTransactionLogEntry.removeFileEntry(result); } - private DeltaLakeTransactionLogEntry buildAddEntry(ConnectorSession session, int pagePosition, Block block) + private class AddFileEntryExtractor + implements CheckpointFieldExtractor { - log.debug("Building add entry from %s pagePosition %d", block, pagePosition); - if (block.isNull(pagePosition)) { - return null; - } - RowType type = (RowType) parquetFields.get("add"); - SqlRow addEntryRow = block.getObject(pagePosition, SqlRow.class); - log.debug("Block %s has %s fields", block, addEntryRow.getFieldCount()); - CheckpointFieldReader add = new CheckpointFieldReader(session, addEntryRow, type); - - Map partitionValues = add.getMap(stringMap, "partitionValues"); - Map> canonicalPartitionValues = canonicalizePartitionValues(partitionValues); - if (!partitionConstraint.isAll() && !partitionMatchesPredicate(canonicalPartitionValues, partitionConstraint.getDomains().orElseThrow())) { - return null; - } - - String path = add.getString("path"); - long size = add.getLong("size"); - long modificationTime = add.getLong("modificationTime"); - boolean dataChange = add.getBoolean("dataChange"); - - Optional deletionVector = Optional.empty(); - if (deletionVectorsEnabled) { - deletionVector = Optional.ofNullable(add.getRow("deletionVector")) - .map(row -> { - RowType.Field deletionVectorField = type.getFields().stream().filter(field -> field.getName().orElseThrow().equals("deletionVector")).collect(onlyElement()); - return parseDeletionVectorFromParquet(session, row, (RowType) deletionVectorField.getType()); - }); - } + @Nullable + @Override + public DeltaLakeTransactionLogEntry getEntry(ConnectorSession session, int pagePosition, Block... blocks) + { + checkState(blocks.length == 1, "Unexpected amount of blocks: %s", blocks.length); + Block block = blocks[0]; + log.debug("Building add entry from %s pagePosition %d", block, pagePosition); + if (block.isNull(pagePosition)) { + return null; + } + RowType type = (RowType) parquetFields.get("add"); + SqlRow addEntryRow = block.getObject(pagePosition, SqlRow.class); + log.debug("Block %s has %s fields", block, addEntryRow.getFieldCount()); + CheckpointFieldReader add = new CheckpointFieldReader(session, addEntryRow, type); + + Map partitionValues = add.getMap(stringMap, "partitionValues"); + Map> canonicalPartitionValues = canonicalizePartitionValues(partitionValues); + if (!partitionConstraint.isAll() && !partitionMatchesPredicate(canonicalPartitionValues, partitionConstraint.getDomains().orElseThrow())) { + return null; + } - Optional parsedStats = Optional.ofNullable(add.getRow("stats_parsed")).map(row -> { - RowType.Field parsedStatsField = type.getFields().stream().filter(field -> field.getName().orElseThrow().equals("stats_parsed")).collect(onlyElement()); - return parseStatisticsFromParquet(session, row, (RowType) parsedStatsField.getType()); - }); - Optional stats = Optional.empty(); - if (parsedStats.isEmpty()) { - stats = Optional.ofNullable(add.getString("stats")); - } + String path = add.getString("path"); + long size = add.getLong("size"); + long modificationTime = add.getLong("modificationTime"); + boolean dataChange = add.getBoolean("dataChange"); + + Optional deletionVector = Optional.empty(); + if (deletionVectorsEnabled) { + deletionVector = Optional.ofNullable(add.getRow("deletionVector")) + .map(row -> { + RowType.Field deletionVectorField = type.getFields().stream().filter(field -> field.getName().orElseThrow().equals("deletionVector")).collect(onlyElement()); + return parseDeletionVectorFromParquet(session, row, (RowType) deletionVectorField.getType()); + }); + } - Map tags = add.getMap(stringMap, "tags"); - AddFileEntry result = new AddFileEntry( - path, - partitionValues, - canonicalPartitionValues, - size, - modificationTime, - dataChange, - stats, - parsedStats, - tags, - deletionVector); + Optional parsedStats = Optional.ofNullable(add.getRow("stats_parsed")).map(row -> { + RowType.Field parsedStatsField = type.getFields().stream().filter(field -> field.getName().orElseThrow().equals("stats_parsed")).collect(onlyElement()); + return parseStatisticsFromParquet(session, row, (RowType) parsedStatsField.getType()); + }); + Optional stats = Optional.empty(); + if (parsedStats.isEmpty()) { + stats = Optional.ofNullable(add.getString("stats")); + } - log.debug("Result: %s", result); - return DeltaLakeTransactionLogEntry.addFileEntry(result); + Map tags = add.getMap(stringMap, "tags"); + AddFileEntry result = new AddFileEntry( + path, + partitionValues, + canonicalPartitionValues, + size, + modificationTime, + dataChange, + stats, + parsedStats, + tags, + deletionVector); + + log.debug("Result: %s", result); + return DeltaLakeTransactionLogEntry.addFileEntry(result); + } } private DeletionVectorEntry parseDeletionVectorFromParquet(ConnectorSession session, SqlRow row, RowType type) From e6c61bf1dadef373d575db47c0583af6f9eb2604 Mon Sep 17 00:00:00 2001 From: Marius Grama Date: Wed, 22 Nov 2023 22:00:20 +0100 Subject: [PATCH 064/350] Avoid materializing from the checkpoint irrelevant `add` entries In case of performing checkpoint filtering in Delta Lake, avoid reading from Parquet pages loaded in memory the `add` entries which don't match the partition predicate. --- .../checkpoint/CheckpointEntryIterator.java | 157 +++++++++++++----- .../checkpoint/CheckpointSchemaManager.java | 16 +- .../checkpoint/CheckpointWriter.java | 3 +- 3 files changed, 135 insertions(+), 41 deletions(-) diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointEntryIterator.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointEntryIterator.java index 0a54bbbf4a7b9..03915c854caf0 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointEntryIterator.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointEntryIterator.java @@ -21,6 +21,7 @@ import io.airlift.log.Logger; import io.trino.filesystem.TrinoInputFile; import io.trino.parquet.Column; +import io.trino.parquet.Field; import io.trino.parquet.ParquetReaderOptions; import io.trino.plugin.deltalake.DeltaHiveTypeTranslator; import io.trino.plugin.deltalake.DeltaLakeColumnHandle; @@ -75,8 +76,8 @@ import static com.google.common.base.Preconditions.checkState; import static com.google.common.base.Verify.verify; import static com.google.common.collect.ImmutableList.toImmutableList; -import static com.google.common.collect.ImmutableMap.toImmutableMap; import static com.google.common.collect.MoreCollectors.onlyElement; +import static com.google.common.collect.MoreCollectors.toOptional; import static io.trino.plugin.deltalake.DeltaLakeColumnType.REGULAR; import static io.trino.plugin.deltalake.DeltaLakeErrorCode.DELTA_LAKE_INVALID_SCHEMA; import static io.trino.plugin.deltalake.DeltaLakeSplitManager.partitionMatchesPredicate; @@ -86,6 +87,11 @@ import static io.trino.plugin.deltalake.transactionlog.TransactionLogParser.START_OF_MODERN_ERA_EPOCH_DAY; import static io.trino.plugin.deltalake.transactionlog.TransactionLogUtil.canonicalizePartitionValues; import static io.trino.plugin.deltalake.transactionlog.checkpoint.CheckpointEntryIterator.EntryType.ADD; +import static io.trino.plugin.deltalake.transactionlog.checkpoint.CheckpointEntryIterator.EntryType.COMMIT; +import static io.trino.plugin.deltalake.transactionlog.checkpoint.CheckpointEntryIterator.EntryType.METADATA; +import static io.trino.plugin.deltalake.transactionlog.checkpoint.CheckpointEntryIterator.EntryType.PROTOCOL; +import static io.trino.plugin.deltalake.transactionlog.checkpoint.CheckpointEntryIterator.EntryType.REMOVE; +import static io.trino.plugin.deltalake.transactionlog.checkpoint.CheckpointEntryIterator.EntryType.TRANSACTION; import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.spi.type.DateTimeEncoding.packDateTimeWithZone; import static io.trino.spi.type.TimeZoneKey.UTC_KEY; @@ -129,20 +135,26 @@ public String getColumnName() private final String checkpointPath; private final ConnectorSession session; private final ParquetPageSource pageSource; - private final Map parquetFields; private final MapType stringMap; private final ArrayType stringList; private final Queue nextEntries; private final List extractors; private final boolean checkpointRowStatisticsWritingEnabled; private final TupleDomain partitionConstraint; + private final Optional txnType; + private final Optional addType; + private final Optional addPartitionValuesType; + private final Optional removeType; + private final Optional metadataType; + private final Optional protocolType; + private final Optional commitType; + private MetadataEntry metadataEntry; private ProtocolEntry protocolEntry; private boolean deletionVectorsEnabled; private List schema; private List columnsWithMinMaxStats; private Page page; - private long pageIndex; private int pagePosition; public CheckpointEntryIterator( @@ -191,6 +203,10 @@ public CheckpointEntryIterator( HiveColumnHandle column = buildColumnHandle(field, checkpointSchemaManager, this.metadataEntry, this.protocolEntry, addStatsMinMaxColumnFilter).toHiveColumnHandle(); columnsBuilder.add(column); disjunctDomainsBuilder.add(buildTupleDomainColumnHandle(field, column)); + if (field == ADD) { + Type addEntryPartitionValuesType = checkpointSchemaManager.getAddEntryPartitionValuesType(); + columnsBuilder.add(new DeltaLakeColumnHandle("add", addEntryPartitionValuesType, OptionalInt.empty(), "add", addEntryPartitionValuesType, REGULAR, Optional.empty()).toHiveColumnHandle()); + } } ReaderPageSource pageSource = ParquetPageSourceFactory.createPageSource( @@ -211,12 +227,46 @@ public CheckpointEntryIterator( this.pageSource = (ParquetPageSource) pageSource.get(); this.nextEntries = new ArrayDeque<>(); - // The size between parquetFields and extractors may not match when the requested field doesn't exist in Parquet file - this.parquetFields = this.pageSource.getColumnFields().stream() - .collect(toImmutableMap(Column::name, e -> e.field().getType())); this.extractors = fields.stream() .map(this::createCheckpointFieldExtractor) .collect(toImmutableList()); + txnType = getParquetType(fields, TRANSACTION); + addType = getAddParquetTypeContainingField(fields, "path"); + addPartitionValuesType = getAddParquetTypeContainingField(fields, "partitionValues"); + removeType = getParquetType(fields, REMOVE); + metadataType = getParquetType(fields, METADATA); + protocolType = getParquetType(fields, PROTOCOL); + commitType = getParquetType(fields, COMMIT); + } + + private Optional getAddParquetTypeContainingField(Set fields, String fieldName) + { + return fields.contains(ADD) ? + this.pageSource.getColumnFields().stream() + .filter(column -> column.name().equals(ADD.getColumnName()) && + column.field().getType() instanceof RowType rowType && + rowType.getFields().stream().map(RowType.Field::getName).filter(Optional::isPresent).flatMap(Optional::stream).anyMatch(fieldName::equals)) + // The field even if it was requested might not exist in Parquet file + .collect(toOptional()) + .map(Column::field) + .map(Field::getType) + .map(RowType.class::cast) + : Optional.empty(); + } + + private Optional getParquetType(Set fields, EntryType field) + { + return fields.contains(field) ? getParquetType(field.getColumnName()).map(RowType.class::cast) : Optional.empty(); + } + + private Optional getParquetType(String columnName) + { + return pageSource.getColumnFields().stream() + .filter(column -> column.name().equals(columnName)) + // The field even if it was requested may not exist in Parquet file + .collect(toOptional()) + .map(Column::field) + .map(Field::getType); } private CheckpointFieldExtractor createCheckpointFieldExtractor(EntryType entryType) @@ -240,7 +290,7 @@ private DeltaLakeColumnHandle buildColumnHandle( { Type type = switch (entryType) { case TRANSACTION -> schemaManager.getTxnEntryType(); - case ADD -> schemaManager.getAddEntryType(metadataEntry, protocolEntry, addStatsMinMaxColumnFilter.orElseThrow(), true, true); + case ADD -> schemaManager.getAddEntryType(metadataEntry, protocolEntry, addStatsMinMaxColumnFilter.orElseThrow(), true, true, false); case REMOVE -> schemaManager.getRemoveEntryType(); case METADATA -> schemaManager.getMetadataEntryType(); case PROTOCOL -> schemaManager.getProtocolEntryType(true, true); @@ -323,7 +373,7 @@ private DeltaLakeTransactionLogEntry buildCommitInfoEntry(ConnectorSession sessi if (block.isNull(pagePosition)) { return null; } - RowType type = (RowType) parquetFields.get("commitinfo"); + RowType type = commitType.orElseThrow(); int commitInfoFields = 12; int jobFields = 5; int notebookFields = 1; @@ -379,7 +429,7 @@ private DeltaLakeTransactionLogEntry buildProtocolEntry(ConnectorSession session if (block.isNull(pagePosition)) { return null; } - RowType type = (RowType) parquetFields.get("protocol"); + RowType type = protocolType.orElseThrow(); int minProtocolFields = 2; int maxProtocolFields = 4; SqlRow protocolEntryRow = block.getObject(pagePosition, SqlRow.class); @@ -406,7 +456,7 @@ private DeltaLakeTransactionLogEntry buildMetadataEntry(ConnectorSession session if (block.isNull(pagePosition)) { return null; } - RowType type = (RowType) parquetFields.get("metadata"); + RowType type = metadataType.orElseThrow(); int metadataFields = 8; int formatFields = 2; SqlRow metadataEntryRow = block.getObject(pagePosition, SqlRow.class); @@ -445,7 +495,7 @@ private DeltaLakeTransactionLogEntry buildRemoveEntry(ConnectorSession session, if (block.isNull(pagePosition)) { return null; } - RowType type = (RowType) parquetFields.get("remove"); + RowType type = removeType.orElseThrow(); int removeFields = 3; SqlRow removeEntryRow = block.getObject(pagePosition, SqlRow.class); log.debug("Block %s has %s fields", block, removeEntryRow.getFieldCount()); @@ -469,47 +519,53 @@ private class AddFileEntryExtractor @Override public DeltaLakeTransactionLogEntry getEntry(ConnectorSession session, int pagePosition, Block... blocks) { - checkState(blocks.length == 1, "Unexpected amount of blocks: %s", blocks.length); - Block block = blocks[0]; - log.debug("Building add entry from %s pagePosition %d", block, pagePosition); - if (block.isNull(pagePosition)) { + checkState(blocks.length == getRequiredChannels(), "Unexpected amount of blocks: %s", blocks.length); + Block addBlock = blocks[0]; + Block addPartitionValuesBlock = blocks[1]; + log.debug("Building add entry from %s pagePosition %d", addBlock, pagePosition); + if (addBlock.isNull(pagePosition)) { return null; } - RowType type = (RowType) parquetFields.get("add"); - SqlRow addEntryRow = block.getObject(pagePosition, SqlRow.class); - log.debug("Block %s has %s fields", block, addEntryRow.getFieldCount()); - CheckpointFieldReader add = new CheckpointFieldReader(session, addEntryRow, type); - Map partitionValues = add.getMap(stringMap, "partitionValues"); + checkState(!addPartitionValuesBlock.isNull(pagePosition), "Inconsistent blocks provided while building the add file entry"); + SqlRow addPartitionValuesRow = addPartitionValuesBlock.getObject(pagePosition, SqlRow.class); + CheckpointFieldReader addPartitionValuesReader = new CheckpointFieldReader(session, addPartitionValuesRow, addPartitionValuesType.orElseThrow()); + Map partitionValues = addPartitionValuesReader.getMap(stringMap, "partitionValues"); Map> canonicalPartitionValues = canonicalizePartitionValues(partitionValues); if (!partitionConstraint.isAll() && !partitionMatchesPredicate(canonicalPartitionValues, partitionConstraint.getDomains().orElseThrow())) { return null; } - String path = add.getString("path"); - long size = add.getLong("size"); - long modificationTime = add.getLong("modificationTime"); - boolean dataChange = add.getBoolean("dataChange"); + // Materialize from Parquet the information needed to build the AddEntry instance + addBlock = addBlock.getLoadedBlock(); + SqlRow addEntryRow = addBlock.getObject(pagePosition, SqlRow.class); + log.debug("Block %s has %s fields", addBlock, addEntryRow.getFieldCount()); + CheckpointFieldReader addReader = new CheckpointFieldReader(session, addEntryRow, addType.orElseThrow()); + + String path = addReader.getString("path"); + long size = addReader.getLong("size"); + long modificationTime = addReader.getLong("modificationTime"); + boolean dataChange = addReader.getBoolean("dataChange"); Optional deletionVector = Optional.empty(); if (deletionVectorsEnabled) { - deletionVector = Optional.ofNullable(add.getRow("deletionVector")) + deletionVector = Optional.ofNullable(addReader.getRow("deletionVector")) .map(row -> { - RowType.Field deletionVectorField = type.getFields().stream().filter(field -> field.getName().orElseThrow().equals("deletionVector")).collect(onlyElement()); + RowType.Field deletionVectorField = addType.orElseThrow().getFields().stream().filter(field -> field.getName().orElseThrow().equals("deletionVector")).collect(onlyElement()); return parseDeletionVectorFromParquet(session, row, (RowType) deletionVectorField.getType()); }); } - Optional parsedStats = Optional.ofNullable(add.getRow("stats_parsed")).map(row -> { - RowType.Field parsedStatsField = type.getFields().stream().filter(field -> field.getName().orElseThrow().equals("stats_parsed")).collect(onlyElement()); + Optional parsedStats = Optional.ofNullable(addReader.getRow("stats_parsed")).map(row -> { + RowType.Field parsedStatsField = addType.orElseThrow().getFields().stream().filter(field -> field.getName().orElseThrow().equals("stats_parsed")).collect(onlyElement()); return parseStatisticsFromParquet(session, row, (RowType) parsedStatsField.getType()); }); Optional stats = Optional.empty(); if (parsedStats.isEmpty()) { - stats = Optional.ofNullable(add.getString("stats")); + stats = Optional.ofNullable(addReader.getString("stats")); } - Map tags = add.getMap(stringMap, "tags"); + Map tags = addReader.getMap(stringMap, "tags"); AddFileEntry result = new AddFileEntry( path, partitionValues, @@ -525,6 +581,12 @@ public DeltaLakeTransactionLogEntry getEntry(ConnectorSession session, int pageP log.debug("Result: %s", result); return DeltaLakeTransactionLogEntry.addFileEntry(result); } + + @Override + public int getRequiredChannels() + { + return 2; + } } private DeletionVectorEntry parseDeletionVectorFromParquet(ConnectorSession session, SqlRow row, RowType type) @@ -634,7 +696,7 @@ private DeltaLakeTransactionLogEntry buildTxnEntry(ConnectorSession session, int if (block.isNull(pagePosition)) { return null; } - RowType type = (RowType) parquetFields.get("txn"); + RowType type = txnType.orElseThrow(); int txnFields = 3; SqlRow txnEntryRow = block.getObject(pagePosition, SqlRow.class); log.debug("Block %s has %s fields", block, txnEntryRow.getFieldCount()); @@ -676,17 +738,20 @@ private boolean tryAdvancePage() pageSource.close(); return false; } + boolean isFirstPage = page == null; page = pageSource.getNextPage(); if (page == null) { return false; } - if (page.getChannelCount() != extractors.size()) { - throw new TrinoException(DELTA_LAKE_INVALID_SCHEMA, - format("Expected page %d (%s) in %s to contain %d channels, but found %d", - pageIndex, page, checkpointPath, extractors.size(), page.getChannelCount())); + if (isFirstPage) { + int requiredExtractorChannels = extractors.stream().mapToInt(CheckpointFieldExtractor::getRequiredChannels).sum(); + if (page.getChannelCount() != requiredExtractorChannels) { + throw new TrinoException(DELTA_LAKE_INVALID_SCHEMA, + format("Expected page in %s to contain %d channels, but found %d", + checkpointPath, requiredExtractorChannels, page.getChannelCount())); + } } pagePosition = 0; - pageIndex++; return true; } @@ -701,11 +766,22 @@ private void fillNextEntries() } // process page - for (int i = 0; i < extractors.size(); ++i) { - DeltaLakeTransactionLogEntry entry = extractors.get(i).getEntry(session, pagePosition, page.getBlock(i).getLoadedBlock()); + int blockIndex = 0; + for (CheckpointFieldExtractor extractor : extractors) { + DeltaLakeTransactionLogEntry entry; + if (extractor instanceof AddFileEntryExtractor) { + // Avoid unnecessary loading of the block in case there is a partition predicate mismatch for this add entry + Block addBlock = page.getBlock(blockIndex); + Block addPartitionValuesBlock = page.getBlock(blockIndex + 1); + entry = extractor.getEntry(session, pagePosition, addBlock, addPartitionValuesBlock.getLoadedBlock()); + } + else { + entry = extractor.getEntry(session, pagePosition, page.getBlock(blockIndex).getLoadedBlock()); + } if (entry != null) { nextEntries.add(entry); } + blockIndex += extractor.getRequiredChannels(); } pagePosition++; } @@ -734,5 +810,10 @@ private interface CheckpointFieldExtractor */ @Nullable DeltaLakeTransactionLogEntry getEntry(ConnectorSession session, int pagePosition, Block... blocks); + + default int getRequiredChannels() + { + return 1; + } } } diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointSchemaManager.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointSchemaManager.java index c2ab94fe4adbc..2bb692b695ca2 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointSchemaManager.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointSchemaManager.java @@ -121,7 +121,8 @@ public RowType getAddEntryType( ProtocolEntry protocolEntry, Predicate addStatsMinMaxColumnFilter, boolean requireWriteStatsAsJson, - boolean requireWriteStatsAsStruct) + boolean requireWriteStatsAsStruct, + boolean usePartitionValues) { List allColumns = extractSchema(metadataEntry, protocolEntry, typeManager); List minMaxColumns = columnsWithStats(metadataEntry, protocolEntry, typeManager); @@ -158,7 +159,9 @@ public RowType getAddEntryType( MapType stringMap = (MapType) typeManager.getType(TypeSignature.mapType(VARCHAR.getTypeSignature(), VARCHAR.getTypeSignature())); ImmutableList.Builder addFields = ImmutableList.builder(); addFields.add(RowType.field("path", VARCHAR)); - addFields.add(RowType.field("partitionValues", stringMap)); + if (usePartitionValues) { + addFields.add(RowType.field("partitionValues", stringMap)); + } addFields.add(RowType.field("size", BIGINT)); addFields.add(RowType.field("modificationTime", BIGINT)); addFields.add(RowType.field("dataChange", BOOLEAN)); @@ -183,6 +186,15 @@ public RowType getAddEntryType( return RowType.from(addFields.build()); } + public RowType getAddEntryPartitionValuesType() + { + ImmutableList.Builder addFields = ImmutableList.builder(); + MapType stringMap = (MapType) typeManager.getType(TypeSignature.mapType(VARCHAR.getTypeSignature(), VARCHAR.getTypeSignature())); + addFields.add(RowType.field("partitionValues", stringMap)); + + return RowType.from(addFields.build()); + } + private static RowType.Field buildNullCountType(Optional columnName, Type columnType) { if (columnType instanceof RowType rowType) { diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointWriter.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointWriter.java index 0cc8c1041431d..0ca4c2e5efdc3 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointWriter.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointWriter.java @@ -120,7 +120,8 @@ public void write(CheckpointEntries entries, TrinoOutputFile outputFile) entries.getProtocolEntry(), alwaysTrue(), writeStatsAsJson, - writeStatsAsStruct); + writeStatsAsStruct, + true); RowType removeEntryType = checkpointSchemaManager.getRemoveEntryType(); List columnNames = ImmutableList.of( From b554d65e5032cb25578b6ac6e6bf61132c0b1c85 Mon Sep 17 00:00:00 2001 From: Marius Grama Date: Mon, 27 Nov 2023 18:04:29 +0100 Subject: [PATCH 065/350] Compute Parquet types for `add` entries only once per checkpoint file --- .../checkpoint/CheckpointEntryIterator.java | 24 ++++++++++++------- 1 file changed, 16 insertions(+), 8 deletions(-) diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointEntryIterator.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointEntryIterator.java index 03915c854caf0..9e88bfd940296 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointEntryIterator.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointEntryIterator.java @@ -144,6 +144,8 @@ public String getColumnName() private final Optional txnType; private final Optional addType; private final Optional addPartitionValuesType; + private final Optional addDeletionVectorType; + private final Optional addParsedStatsFieldType; private final Optional removeType; private final Optional metadataType; private final Optional protocolType; @@ -233,12 +235,23 @@ public CheckpointEntryIterator( txnType = getParquetType(fields, TRANSACTION); addType = getAddParquetTypeContainingField(fields, "path"); addPartitionValuesType = getAddParquetTypeContainingField(fields, "partitionValues"); + addDeletionVectorType = addType.flatMap(type -> getOptionalFieldType(type, "deletionVector")); + addParsedStatsFieldType = addType.flatMap(type -> getOptionalFieldType(type, "stats_parsed")); removeType = getParquetType(fields, REMOVE); metadataType = getParquetType(fields, METADATA); protocolType = getParquetType(fields, PROTOCOL); commitType = getParquetType(fields, COMMIT); } + private static Optional getOptionalFieldType(RowType type, String fieldName) + { + return type.getFields().stream() + .filter(field -> field.getName().orElseThrow().equals(fieldName)) + .collect(toOptional()) + .map(RowType.Field::getType) + .map(RowType.class::cast); + } + private Optional getAddParquetTypeContainingField(Set fields, String fieldName) { return fields.contains(ADD) ? @@ -550,16 +563,11 @@ public DeltaLakeTransactionLogEntry getEntry(ConnectorSession session, int pageP Optional deletionVector = Optional.empty(); if (deletionVectorsEnabled) { deletionVector = Optional.ofNullable(addReader.getRow("deletionVector")) - .map(row -> { - RowType.Field deletionVectorField = addType.orElseThrow().getFields().stream().filter(field -> field.getName().orElseThrow().equals("deletionVector")).collect(onlyElement()); - return parseDeletionVectorFromParquet(session, row, (RowType) deletionVectorField.getType()); - }); + .map(row -> parseDeletionVectorFromParquet(session, row, addDeletionVectorType.orElseThrow())); } - Optional parsedStats = Optional.ofNullable(addReader.getRow("stats_parsed")).map(row -> { - RowType.Field parsedStatsField = addType.orElseThrow().getFields().stream().filter(field -> field.getName().orElseThrow().equals("stats_parsed")).collect(onlyElement()); - return parseStatisticsFromParquet(session, row, (RowType) parsedStatsField.getType()); - }); + Optional parsedStats = Optional.ofNullable(addReader.getRow("stats_parsed")) + .map(row -> parseStatisticsFromParquet(session, row, addParsedStatsFieldType.orElseThrow())); Optional stats = Optional.empty(); if (parsedStats.isEmpty()) { stats = Optional.ofNullable(addReader.getString("stats")); From 814aa57702f9eed9a17515413dc4082e45345aba Mon Sep 17 00:00:00 2001 From: David Phillips Date: Sun, 26 Nov 2023 16:26:53 -0600 Subject: [PATCH 066/350] Encapsulate usage of alternate header name config --- .../dispatcher/QueuedStatementResource.java | 6 +--- .../HttpRequestSessionContextFactory.java | 23 ++++++------ .../java/io/trino/server/QueryResource.java | 12 +++---- .../trino/server/QueryStateInfoResource.java | 9 ++--- .../ResourceSecurityDynamicFeature.java | 15 +++----- .../io/trino/server/ui/UiQueryResource.java | 11 +++--- .../io/trino/server/ui/WorkerResource.java | 8 ++--- .../TestHttpRequestSessionContextFactory.java | 35 +++++++++---------- .../server/TestQuerySessionSupplier.java | 17 ++++----- .../server/security/TestResourceSecurity.java | 5 +-- .../java/io/trino/server/ui/TestWebUi.java | 5 +-- 11 files changed, 62 insertions(+), 84 deletions(-) diff --git a/core/trino-main/src/main/java/io/trino/dispatcher/QueuedStatementResource.java b/core/trino-main/src/main/java/io/trino/dispatcher/QueuedStatementResource.java index b4cd5713344cd..e4562422b4810 100644 --- a/core/trino-main/src/main/java/io/trino/dispatcher/QueuedStatementResource.java +++ b/core/trino-main/src/main/java/io/trino/dispatcher/QueuedStatementResource.java @@ -32,7 +32,6 @@ import io.trino.execution.QueryManagerConfig; import io.trino.execution.QueryState; import io.trino.server.HttpRequestSessionContextFactory; -import io.trino.server.ProtocolConfig; import io.trino.server.ServerConfig; import io.trino.server.SessionContext; import io.trino.server.protocol.QueryInfoUrlFactory; @@ -120,7 +119,6 @@ public class QueuedStatementResource private final ScheduledExecutorService timeoutExecutor; private final boolean compressionEnabled; - private final Optional alternateHeaderName; private final QueryManager queryManager; @Inject @@ -131,7 +129,6 @@ public QueuedStatementResource( DispatchExecutor executor, QueryInfoUrlFactory queryInfoUrlTemplate, ServerConfig serverConfig, - ProtocolConfig protocolConfig, QueryManagerConfig queryManagerConfig) { this.sessionContextFactory = requireNonNull(sessionContextFactory, "sessionContextFactory is null"); @@ -141,7 +138,6 @@ public QueuedStatementResource( this.timeoutExecutor = executor.getScheduledExecutor(); this.queryInfoUrlFactory = requireNonNull(queryInfoUrlTemplate, "queryInfoUrlTemplate is null"); this.compressionEnabled = serverConfig.isQueryResultsCompressionEnabled(); - this.alternateHeaderName = protocolConfig.getAlternateHeaderName(); queryManager = new QueryManager(queryManagerConfig.getClientTimeout()); } @@ -185,7 +181,7 @@ private Query registerQuery(String statement, HttpServletRequest servletRequest, MultivaluedMap headers = httpHeaders.getRequestHeaders(); - SessionContext sessionContext = sessionContextFactory.createSessionContext(headers, alternateHeaderName, remoteAddress, identity); + SessionContext sessionContext = sessionContextFactory.createSessionContext(headers, remoteAddress, identity); Query query = new Query(statement, sessionContext, dispatchManager, queryInfoUrlFactory, tracer); queryManager.registerQuery(query); diff --git a/core/trino-main/src/main/java/io/trino/server/HttpRequestSessionContextFactory.java b/core/trino-main/src/main/java/io/trino/server/HttpRequestSessionContextFactory.java index cf821e7622105..854a69d5d3035 100644 --- a/core/trino-main/src/main/java/io/trino/server/HttpRequestSessionContextFactory.java +++ b/core/trino-main/src/main/java/io/trino/server/HttpRequestSessionContextFactory.java @@ -75,10 +75,17 @@ public class HttpRequestSessionContextFactory private final Metadata metadata; private final GroupProvider groupProvider; private final AccessControl accessControl; + private final Optional alternateHeaderName; @Inject - public HttpRequestSessionContextFactory(PreparedStatementEncoder preparedStatementEncoder, Metadata metadata, GroupProvider groupProvider, AccessControl accessControl) + public HttpRequestSessionContextFactory( + PreparedStatementEncoder preparedStatementEncoder, + Metadata metadata, + GroupProvider groupProvider, + AccessControl accessControl, + ProtocolConfig protocolConfig) { + this.alternateHeaderName = protocolConfig.getAlternateHeaderName(); this.preparedStatementEncoder = requireNonNull(preparedStatementEncoder, "preparedStatementEncoder is null"); this.metadata = requireNonNull(metadata, "metadata is null"); this.groupProvider = requireNonNull(groupProvider, "groupProvider is null"); @@ -87,7 +94,6 @@ public HttpRequestSessionContextFactory(PreparedStatementEncoder preparedStateme public SessionContext createSessionContext( MultivaluedMap headers, - Optional alternateHeaderName, Optional remoteAddress, Optional authenticatedIdentity) throws WebApplicationException @@ -184,21 +190,14 @@ else if (nameParts.size() == 2) { clientInfo); } - public Identity extractAuthorizedIdentity( - HttpServletRequest servletRequest, - HttpHeaders httpHeaders, - Optional alternateHeaderName) + public Identity extractAuthorizedIdentity(HttpServletRequest servletRequest, HttpHeaders httpHeaders) { return extractAuthorizedIdentity( Optional.ofNullable((Identity) servletRequest.getAttribute(AUTHENTICATED_IDENTITY)), - httpHeaders.getRequestHeaders(), - alternateHeaderName); + httpHeaders.getRequestHeaders()); } - public Identity extractAuthorizedIdentity( - Optional optionalAuthenticatedIdentity, - MultivaluedMap headers, - Optional alternateHeaderName) + public Identity extractAuthorizedIdentity(Optional optionalAuthenticatedIdentity, MultivaluedMap headers) throws AccessDeniedException { ProtocolHeaders protocolHeaders; diff --git a/core/trino-main/src/main/java/io/trino/server/QueryResource.java b/core/trino-main/src/main/java/io/trino/server/QueryResource.java index 0640059d25361..ae38cfabda018 100644 --- a/core/trino-main/src/main/java/io/trino/server/QueryResource.java +++ b/core/trino-main/src/main/java/io/trino/server/QueryResource.java @@ -58,15 +58,13 @@ public class QueryResource private final DispatchManager dispatchManager; private final AccessControl accessControl; private final HttpRequestSessionContextFactory sessionContextFactory; - private final Optional alternateHeaderName; @Inject - public QueryResource(DispatchManager dispatchManager, AccessControl accessControl, HttpRequestSessionContextFactory sessionContextFactory, ProtocolConfig protocolConfig) + public QueryResource(DispatchManager dispatchManager, AccessControl accessControl, HttpRequestSessionContextFactory sessionContextFactory) { this.dispatchManager = requireNonNull(dispatchManager, "dispatchManager is null"); this.accessControl = requireNonNull(accessControl, "accessControl is null"); this.sessionContextFactory = requireNonNull(sessionContextFactory, "sessionContextFactory is null"); - this.alternateHeaderName = protocolConfig.getAlternateHeaderName(); } @ResourceSecurity(AUTHENTICATED_USER) @@ -76,7 +74,7 @@ public List getAllQueryInfo(@QueryParam("state") String stateFil QueryState expectedState = stateFilter == null ? null : QueryState.valueOf(stateFilter.toUpperCase(Locale.ENGLISH)); List queries = dispatchManager.getQueries(); - queries = filterQueries(sessionContextFactory.extractAuthorizedIdentity(servletRequest, httpHeaders, alternateHeaderName), queries, accessControl); + queries = filterQueries(sessionContextFactory.extractAuthorizedIdentity(servletRequest, httpHeaders), queries, accessControl); ImmutableList.Builder builder = ImmutableList.builder(); for (BasicQueryInfo queryInfo : queries) { @@ -99,7 +97,7 @@ public Response getQueryInfo(@PathParam("queryId") QueryId queryId, @Context Htt return Response.status(Status.GONE).build(); } try { - checkCanViewQueryOwnedBy(sessionContextFactory.extractAuthorizedIdentity(servletRequest, httpHeaders, alternateHeaderName), queryInfo.get().getSession().toIdentity(), accessControl); + checkCanViewQueryOwnedBy(sessionContextFactory.extractAuthorizedIdentity(servletRequest, httpHeaders), queryInfo.get().getSession().toIdentity(), accessControl); return Response.ok(queryInfo.get()).build(); } catch (AccessDeniedException e) { @@ -116,7 +114,7 @@ public void cancelQuery(@PathParam("queryId") QueryId queryId, @Context HttpServ try { BasicQueryInfo queryInfo = dispatchManager.getQueryInfo(queryId); - checkCanKillQueryOwnedBy(sessionContextFactory.extractAuthorizedIdentity(servletRequest, httpHeaders, alternateHeaderName), queryInfo.getSession().toIdentity(), accessControl); + checkCanKillQueryOwnedBy(sessionContextFactory.extractAuthorizedIdentity(servletRequest, httpHeaders), queryInfo.getSession().toIdentity(), accessControl); dispatchManager.cancelQuery(queryId); } catch (AccessDeniedException e) { @@ -149,7 +147,7 @@ private Response failQuery(QueryId queryId, TrinoException queryException, HttpS try { BasicQueryInfo queryInfo = dispatchManager.getQueryInfo(queryId); - checkCanKillQueryOwnedBy(sessionContextFactory.extractAuthorizedIdentity(servletRequest, httpHeaders, alternateHeaderName), queryInfo.getSession().toIdentity(), accessControl); + checkCanKillQueryOwnedBy(sessionContextFactory.extractAuthorizedIdentity(servletRequest, httpHeaders), queryInfo.getSession().toIdentity(), accessControl); // check before killing to provide the proper error code (this is racy) if (queryInfo.getState().isDone()) { diff --git a/core/trino-main/src/main/java/io/trino/server/QueryStateInfoResource.java b/core/trino-main/src/main/java/io/trino/server/QueryStateInfoResource.java index 63997ff0e8c75..df07b0bff9ee3 100644 --- a/core/trino-main/src/main/java/io/trino/server/QueryStateInfoResource.java +++ b/core/trino-main/src/main/java/io/trino/server/QueryStateInfoResource.java @@ -56,21 +56,18 @@ public class QueryStateInfoResource private final ResourceGroupManager resourceGroupManager; private final AccessControl accessControl; private final HttpRequestSessionContextFactory sessionContextFactory; - private final Optional alternateHeaderName; @Inject public QueryStateInfoResource( DispatchManager dispatchManager, ResourceGroupManager resourceGroupManager, AccessControl accessControl, - HttpRequestSessionContextFactory sessionContextFactory, - ProtocolConfig protocolConfig) + HttpRequestSessionContextFactory sessionContextFactory) { this.dispatchManager = requireNonNull(dispatchManager, "dispatchManager is null"); this.resourceGroupManager = requireNonNull(resourceGroupManager, "resourceGroupManager is null"); this.accessControl = requireNonNull(accessControl, "accessControl is null"); this.sessionContextFactory = requireNonNull(sessionContextFactory, "sessionContextFactory is null"); - this.alternateHeaderName = protocolConfig.getAlternateHeaderName(); } @ResourceSecurity(AUTHENTICATED_USER) @@ -79,7 +76,7 @@ public QueryStateInfoResource( public List getQueryStateInfos(@QueryParam("user") String user, @Context HttpServletRequest servletRequest, @Context HttpHeaders httpHeaders) { List queryInfos = dispatchManager.getQueries(); - queryInfos = filterQueries(sessionContextFactory.extractAuthorizedIdentity(servletRequest, httpHeaders, alternateHeaderName), queryInfos, accessControl); + queryInfos = filterQueries(sessionContextFactory.extractAuthorizedIdentity(servletRequest, httpHeaders), queryInfos, accessControl); if (!isNullOrEmpty(user)) { queryInfos = queryInfos.stream() @@ -115,7 +112,7 @@ public QueryStateInfo getQueryStateInfo(@PathParam("queryId") String queryId, @C { try { BasicQueryInfo queryInfo = dispatchManager.getQueryInfo(new QueryId(queryId)); - checkCanViewQueryOwnedBy(sessionContextFactory.extractAuthorizedIdentity(servletRequest, httpHeaders, alternateHeaderName), queryInfo.getSession().toIdentity(), accessControl); + checkCanViewQueryOwnedBy(sessionContextFactory.extractAuthorizedIdentity(servletRequest, httpHeaders), queryInfo.getSession().toIdentity(), accessControl); return getQueryStateInfo(queryInfo); } catch (AccessDeniedException e) { diff --git a/core/trino-main/src/main/java/io/trino/server/security/ResourceSecurityDynamicFeature.java b/core/trino-main/src/main/java/io/trino/server/security/ResourceSecurityDynamicFeature.java index 2ffe53bdd71ad..a5e106ebeac15 100644 --- a/core/trino-main/src/main/java/io/trino/server/security/ResourceSecurityDynamicFeature.java +++ b/core/trino-main/src/main/java/io/trino/server/security/ResourceSecurityDynamicFeature.java @@ -17,7 +17,6 @@ import io.trino.security.AccessControl; import io.trino.server.HttpRequestSessionContextFactory; import io.trino.server.InternalAuthenticationManager; -import io.trino.server.ProtocolConfig; import io.trino.server.security.ResourceSecurity.AccessType; import io.trino.server.ui.WebUiAuthenticationFilter; import io.trino.spi.TrinoException; @@ -54,7 +53,6 @@ public class ResourceSecurityDynamicFeature private final HttpRequestSessionContextFactory sessionContextFactory; private final Optional fixedManagementUser; private final boolean fixedManagementUserForHttps; - private final Optional alternateHeaderName; @Inject public ResourceSecurityDynamicFeature( @@ -64,8 +62,7 @@ public ResourceSecurityDynamicFeature( InternalAuthenticationManager internalAuthenticationManager, AccessControl accessControl, HttpRequestSessionContextFactory sessionContextFactory, - SecurityConfig securityConfig, - ProtocolConfig protocolConfig) + SecurityConfig securityConfig) { this.resourceAccessType = requireNonNull(resourceAccessType, "resourceAccessType is null"); this.authenticationFilter = requireNonNull(authenticationFilter, "authenticationFilter is null"); @@ -75,7 +72,6 @@ public ResourceSecurityDynamicFeature( this.sessionContextFactory = requireNonNull(sessionContextFactory, "sessionContextFactory is null"); this.fixedManagementUser = securityConfig.getFixedManagementUser(); this.fixedManagementUserForHttps = securityConfig.isFixedManagementUserForHttps(); - this.alternateHeaderName = protocolConfig.getAlternateHeaderName(); } @Override @@ -97,7 +93,7 @@ public void configure(ResourceInfo resourceInfo, FeatureContext context) case MANAGEMENT_READ: case MANAGEMENT_WRITE: context.register(new ManagementAuthenticationFilter(fixedManagementUser, fixedManagementUserForHttps, authenticationFilter)); - context.register(new ManagementAuthorizationFilter(accessControl, sessionContextFactory, accessType == MANAGEMENT_READ, alternateHeaderName)); + context.register(new ManagementAuthorizationFilter(accessControl, sessionContextFactory, accessType == MANAGEMENT_READ)); context.register(new DisposeIdentityResponseFilter()); return; case INTERNAL_ONLY: @@ -141,14 +137,12 @@ private static class ManagementAuthorizationFilter private final AccessControl accessControl; private final HttpRequestSessionContextFactory sessionContextFactory; private final boolean read; - private final Optional alternateHeaderName; - public ManagementAuthorizationFilter(AccessControl accessControl, HttpRequestSessionContextFactory sessionContextFactory, boolean read, Optional alternateHeaderName) + public ManagementAuthorizationFilter(AccessControl accessControl, HttpRequestSessionContextFactory sessionContextFactory, boolean read) { this.accessControl = requireNonNull(accessControl, "accessControl is null"); this.sessionContextFactory = requireNonNull(sessionContextFactory, "sessionContextFactory is null"); this.read = read; - this.alternateHeaderName = requireNonNull(alternateHeaderName, "alternateHeaderName is null"); } @Override @@ -161,8 +155,7 @@ public void filter(ContainerRequestContext request) try { Identity identity = sessionContextFactory.extractAuthorizedIdentity( Optional.ofNullable((Identity) request.getProperty(AUTHENTICATED_IDENTITY)), - request.getHeaders(), - alternateHeaderName); + request.getHeaders()); if (read) { accessControl.checkCanReadSystemInformation(identity); } diff --git a/core/trino-main/src/main/java/io/trino/server/ui/UiQueryResource.java b/core/trino-main/src/main/java/io/trino/server/ui/UiQueryResource.java index 40531ed228030..9ed0e1f6fac07 100644 --- a/core/trino-main/src/main/java/io/trino/server/ui/UiQueryResource.java +++ b/core/trino-main/src/main/java/io/trino/server/ui/UiQueryResource.java @@ -21,7 +21,6 @@ import io.trino.security.AccessControl; import io.trino.server.BasicQueryInfo; import io.trino.server.HttpRequestSessionContextFactory; -import io.trino.server.ProtocolConfig; import io.trino.server.security.ResourceSecurity; import io.trino.spi.QueryId; import io.trino.spi.TrinoException; @@ -57,15 +56,13 @@ public class UiQueryResource private final DispatchManager dispatchManager; private final AccessControl accessControl; private final HttpRequestSessionContextFactory sessionContextFactory; - private final Optional alternateHeaderName; @Inject - public UiQueryResource(DispatchManager dispatchManager, AccessControl accessControl, HttpRequestSessionContextFactory sessionContextFactory, ProtocolConfig protocolConfig) + public UiQueryResource(DispatchManager dispatchManager, AccessControl accessControl, HttpRequestSessionContextFactory sessionContextFactory) { this.dispatchManager = requireNonNull(dispatchManager, "dispatchManager is null"); this.accessControl = requireNonNull(accessControl, "accessControl is null"); this.sessionContextFactory = requireNonNull(sessionContextFactory, "sessionContextFactory is null"); - this.alternateHeaderName = protocolConfig.getAlternateHeaderName(); } @ResourceSecurity(WEB_UI) @@ -75,7 +72,7 @@ public List getAllQueryInfo(@QueryParam("state") String s QueryState expectedState = stateFilter == null ? null : QueryState.valueOf(stateFilter.toUpperCase(Locale.ENGLISH)); List queries = dispatchManager.getQueries(); - queries = filterQueries(sessionContextFactory.extractAuthorizedIdentity(servletRequest, httpHeaders, alternateHeaderName), queries, accessControl); + queries = filterQueries(sessionContextFactory.extractAuthorizedIdentity(servletRequest, httpHeaders), queries, accessControl); ImmutableList.Builder builder = ImmutableList.builder(); for (BasicQueryInfo queryInfo : queries) { @@ -96,7 +93,7 @@ public Response getQueryInfo(@PathParam("queryId") QueryId queryId, @Context Htt Optional queryInfo = dispatchManager.getFullQueryInfo(queryId); if (queryInfo.isPresent()) { try { - checkCanViewQueryOwnedBy(sessionContextFactory.extractAuthorizedIdentity(servletRequest, httpHeaders, alternateHeaderName), queryInfo.get().getSession().toIdentity(), accessControl); + checkCanViewQueryOwnedBy(sessionContextFactory.extractAuthorizedIdentity(servletRequest, httpHeaders), queryInfo.get().getSession().toIdentity(), accessControl); return Response.ok(queryInfo.get()).build(); } catch (AccessDeniedException e) { @@ -129,7 +126,7 @@ private Response failQuery(QueryId queryId, TrinoException queryException, HttpS try { BasicQueryInfo queryInfo = dispatchManager.getQueryInfo(queryId); - checkCanKillQueryOwnedBy(sessionContextFactory.extractAuthorizedIdentity(servletRequest, httpHeaders, alternateHeaderName), queryInfo.getSession().toIdentity(), accessControl); + checkCanKillQueryOwnedBy(sessionContextFactory.extractAuthorizedIdentity(servletRequest, httpHeaders), queryInfo.getSession().toIdentity(), accessControl); // check before killing to provide the proper error code (this is racy) if (queryInfo.getState().isDone()) { diff --git a/core/trino-main/src/main/java/io/trino/server/ui/WorkerResource.java b/core/trino-main/src/main/java/io/trino/server/ui/WorkerResource.java index bf03403eed123..df7e119e25c73 100644 --- a/core/trino-main/src/main/java/io/trino/server/ui/WorkerResource.java +++ b/core/trino-main/src/main/java/io/trino/server/ui/WorkerResource.java @@ -28,7 +28,6 @@ import io.trino.security.AccessControl; import io.trino.server.ForWorkerInfo; import io.trino.server.HttpRequestSessionContextFactory; -import io.trino.server.ProtocolConfig; import io.trino.server.security.ResourceSecurity; import io.trino.spi.Node; import io.trino.spi.QueryId; @@ -71,7 +70,6 @@ public class WorkerResource private final AccessControl accessControl; private final HttpClient httpClient; private final HttpRequestSessionContextFactory sessionContextFactory; - private final Optional alternateHeaderName; @Inject public WorkerResource( @@ -79,15 +77,13 @@ public WorkerResource( InternalNodeManager nodeManager, AccessControl accessControl, @ForWorkerInfo HttpClient httpClient, - HttpRequestSessionContextFactory sessionContextFactory, - ProtocolConfig protocolConfig) + HttpRequestSessionContextFactory sessionContextFactory) { this.dispatchManager = requireNonNull(dispatchManager, "dispatchManager is null"); this.nodeManager = requireNonNull(nodeManager, "nodeManager is null"); this.accessControl = requireNonNull(accessControl, "accessControl is null"); this.httpClient = requireNonNull(httpClient, "httpClient is null"); this.sessionContextFactory = requireNonNull(sessionContextFactory, "sessionContextFactory is null"); - this.alternateHeaderName = protocolConfig.getAlternateHeaderName(); } @ResourceSecurity(WEB_UI) @@ -119,7 +115,7 @@ public Response getThreads( Optional queryInfo = dispatchManager.getFullQueryInfo(queryId); if (queryInfo.isPresent()) { try { - checkCanViewQueryOwnedBy(sessionContextFactory.extractAuthorizedIdentity(servletRequest, httpHeaders, alternateHeaderName), queryInfo.get().getSession().toIdentity(), accessControl); + checkCanViewQueryOwnedBy(sessionContextFactory.extractAuthorizedIdentity(servletRequest, httpHeaders), queryInfo.get().getSession().toIdentity(), accessControl); return proxyJsonResponse(nodeId, "v1/task/" + task); } catch (AccessDeniedException e) { diff --git a/core/trino-main/src/test/java/io/trino/server/TestHttpRequestSessionContextFactory.java b/core/trino-main/src/test/java/io/trino/server/TestHttpRequestSessionContextFactory.java index 6e62951548188..8407e30b8b23c 100644 --- a/core/trino-main/src/test/java/io/trino/server/TestHttpRequestSessionContextFactory.java +++ b/core/trino-main/src/test/java/io/trino/server/TestHttpRequestSessionContextFactory.java @@ -40,12 +40,6 @@ public class TestHttpRequestSessionContextFactory { - private static final HttpRequestSessionContextFactory SESSION_CONTEXT_FACTORY = new HttpRequestSessionContextFactory( - new PreparedStatementEncoder(new ProtocolConfig()), - createTestMetadataManager(), - ImmutableSet::of, - new AllowAllAccessControl()); - @Test public void testSessionContext() { @@ -76,9 +70,8 @@ private static void assertSessionContext(ProtocolHeaders protocolHeaders) .put(protocolHeaders.requestExtraCredential(), "test.token.abc=xyz") .build()); - SessionContext context = SESSION_CONTEXT_FACTORY.createSessionContext( + SessionContext context = sessionContextFactory(protocolHeaders).createSessionContext( headers, - Optional.of(protocolHeaders.getProtocolName()), Optional.of("testRemote"), Optional.empty()); assertThat(context.getSource().orElse(null)).isEqualTo("testSource"); @@ -118,31 +111,27 @@ private static void assertMappedUser(ProtocolHeaders protocolHeaders) MultivaluedMap userHeaders = new GuavaMultivaluedMap<>(ImmutableListMultimap.of(protocolHeaders.requestUser(), "testUser")); MultivaluedMap emptyHeaders = new MultivaluedHashMap<>(); - SessionContext context = SESSION_CONTEXT_FACTORY.createSessionContext( + SessionContext context = sessionContextFactory(protocolHeaders).createSessionContext( userHeaders, - Optional.of(protocolHeaders.getProtocolName()), Optional.of("testRemote"), Optional.empty()); assertThat(context.getIdentity()).isEqualTo(Identity.forUser("testUser").withGroups(ImmutableSet.of("testUser")).build()); - context = SESSION_CONTEXT_FACTORY.createSessionContext( + context = sessionContextFactory(protocolHeaders).createSessionContext( emptyHeaders, - Optional.of(protocolHeaders.getProtocolName()), Optional.of("testRemote"), Optional.of(Identity.forUser("mappedUser").withGroups(ImmutableSet.of("test")).build())); assertThat(context.getIdentity()).isEqualTo(Identity.forUser("mappedUser").withGroups(ImmutableSet.of("test", "mappedUser")).build()); - context = SESSION_CONTEXT_FACTORY.createSessionContext( + context = sessionContextFactory(protocolHeaders).createSessionContext( userHeaders, - Optional.of(protocolHeaders.getProtocolName()), Optional.of("testRemote"), Optional.of(Identity.ofUser("mappedUser"))); assertThat(context.getIdentity()).isEqualTo(Identity.forUser("testUser").withGroups(ImmutableSet.of("testUser")).build()); assertThatThrownBy( - () -> SESSION_CONTEXT_FACTORY.createSessionContext( + () -> sessionContextFactory(protocolHeaders).createSessionContext( emptyHeaders, - Optional.of(protocolHeaders.getProtocolName()), Optional.of("testRemote"), Optional.empty())) .isInstanceOf(WebApplicationException.class) @@ -171,12 +160,22 @@ private static void assertPreparedStatementsHeaderDoesNotParse(ProtocolHeaders p .build()); assertThatThrownBy( - () -> SESSION_CONTEXT_FACTORY.createSessionContext( + () -> sessionContextFactory(protocolHeaders).createSessionContext( headers, - Optional.of(protocolHeaders.getProtocolName()), Optional.of("testRemote"), Optional.empty())) .isInstanceOf(WebApplicationException.class) .hasMessageMatching("Invalid " + protocolHeaders.requestPreparedStatement() + " header: line 1:1: mismatched input 'abcdefg'. Expecting: .*"); } + + private static HttpRequestSessionContextFactory sessionContextFactory(ProtocolHeaders headers) + { + return new HttpRequestSessionContextFactory( + new PreparedStatementEncoder(new ProtocolConfig()), + createTestMetadataManager(), + ImmutableSet::of, + new AllowAllAccessControl(), + new ProtocolConfig() + .setAlternateHeaderName(headers.getProtocolName())); + } } diff --git a/core/trino-main/src/test/java/io/trino/server/TestQuerySessionSupplier.java b/core/trino-main/src/test/java/io/trino/server/TestQuerySessionSupplier.java index 7cb8c3fd41bce..f75523215f747 100644 --- a/core/trino-main/src/test/java/io/trino/server/TestQuerySessionSupplier.java +++ b/core/trino-main/src/test/java/io/trino/server/TestQuerySessionSupplier.java @@ -71,12 +71,13 @@ public class TestQuerySessionSupplier new PreparedStatementEncoder(new ProtocolConfig()), createTestMetadataManager(), ImmutableSet::of, - new AllowAllAccessControl()); + new AllowAllAccessControl(), + new ProtocolConfig()); @Test public void testCreateSession() { - SessionContext context = SESSION_CONTEXT_FACTORY.createSessionContext(TEST_HEADERS, Optional.empty(), Optional.of("testRemote"), Optional.empty()); + SessionContext context = SESSION_CONTEXT_FACTORY.createSessionContext(TEST_HEADERS, Optional.of("testRemote"), Optional.empty()); QuerySessionSupplier sessionSupplier = createSessionSupplier(new SqlEnvironmentConfig()); Session session = sessionSupplier.createSession(new QueryId("test_query_id"), Span.getInvalid(), context); @@ -106,14 +107,14 @@ public void testCreateSession() public void testEmptyClientTags() { MultivaluedMap headers1 = new GuavaMultivaluedMap<>(ImmutableListMultimap.of(TRINO_HEADERS.requestUser(), "testUser")); - SessionContext context1 = SESSION_CONTEXT_FACTORY.createSessionContext(headers1, Optional.empty(), Optional.of("remoteAddress"), Optional.empty()); + SessionContext context1 = SESSION_CONTEXT_FACTORY.createSessionContext(headers1, Optional.of("remoteAddress"), Optional.empty()); assertThat(context1.getClientTags()).isEqualTo(ImmutableSet.of()); MultivaluedMap headers2 = new GuavaMultivaluedMap<>(ImmutableListMultimap.builder() .put(TRINO_HEADERS.requestUser(), "testUser") .put(TRINO_HEADERS.requestClientTags(), "") .build()); - SessionContext context2 = SESSION_CONTEXT_FACTORY.createSessionContext(headers2, Optional.empty(), Optional.of("remoteAddress"), Optional.empty()); + SessionContext context2 = SESSION_CONTEXT_FACTORY.createSessionContext(headers2, Optional.of("remoteAddress"), Optional.empty()); assertThat(context2.getClientTags()).isEqualTo(ImmutableSet.of()); } @@ -124,11 +125,11 @@ public void testClientCapabilities() .put(TRINO_HEADERS.requestUser(), "testUser") .put(TRINO_HEADERS.requestClientCapabilities(), "foo, bar") .build()); - SessionContext context1 = SESSION_CONTEXT_FACTORY.createSessionContext(headers1, Optional.empty(), Optional.of("remoteAddress"), Optional.empty()); + SessionContext context1 = SESSION_CONTEXT_FACTORY.createSessionContext(headers1, Optional.of("remoteAddress"), Optional.empty()); assertThat(context1.getClientCapabilities()).isEqualTo(ImmutableSet.of("foo", "bar")); MultivaluedMap headers2 = new GuavaMultivaluedMap<>(ImmutableListMultimap.of(TRINO_HEADERS.requestUser(), "testUser")); - SessionContext context2 = SESSION_CONTEXT_FACTORY.createSessionContext(headers2, Optional.empty(), Optional.of("remoteAddress"), Optional.empty()); + SessionContext context2 = SESSION_CONTEXT_FACTORY.createSessionContext(headers2, Optional.of("remoteAddress"), Optional.empty()); assertThat(context2.getClientCapabilities()).isEqualTo(ImmutableSet.of()); } @@ -139,7 +140,7 @@ public void testInvalidTimeZone() .put(TRINO_HEADERS.requestUser(), "testUser") .put(TRINO_HEADERS.requestTimeZone(), "unknown_timezone") .build()); - SessionContext context = SESSION_CONTEXT_FACTORY.createSessionContext(headers, Optional.empty(), Optional.of("remoteAddress"), Optional.empty()); + SessionContext context = SESSION_CONTEXT_FACTORY.createSessionContext(headers, Optional.of("remoteAddress"), Optional.empty()); QuerySessionSupplier sessionSupplier = createSessionSupplier(new SqlEnvironmentConfig()); assertThatThrownBy(() -> sessionSupplier.createSession(new QueryId("test_query_id"), Span.getInvalid(), context)) .isInstanceOf(TrinoException.class) @@ -230,7 +231,7 @@ public void testDefaultCatalogAndSchema() private static Session createSession(ListMultimap headers, SqlEnvironmentConfig config) { MultivaluedMap headerMap = new GuavaMultivaluedMap<>(headers); - SessionContext context = SESSION_CONTEXT_FACTORY.createSessionContext(headerMap, Optional.empty(), Optional.of("testRemote"), Optional.empty()); + SessionContext context = SESSION_CONTEXT_FACTORY.createSessionContext(headerMap, Optional.of("testRemote"), Optional.empty()); QuerySessionSupplier sessionSupplier = createSessionSupplier(config); return sessionSupplier.createSession(new QueryId("test_query_id"), Span.getInvalid(), context); } diff --git a/core/trino-main/src/test/java/io/trino/server/security/TestResourceSecurity.java b/core/trino-main/src/test/java/io/trino/server/security/TestResourceSecurity.java index e46f873524460..2452cfc8958ec 100644 --- a/core/trino-main/src/test/java/io/trino/server/security/TestResourceSecurity.java +++ b/core/trino-main/src/test/java/io/trino/server/security/TestResourceSecurity.java @@ -1186,7 +1186,8 @@ public TestResource(AccessControl accessControl) new PreparedStatementEncoder(new ProtocolConfig()), createTestMetadataManager(), user -> ImmutableSet.of(), - accessControl); + accessControl, + new ProtocolConfig()); } @ResourceSecurity(AUTHENTICATED_USER) @@ -1207,7 +1208,7 @@ public jakarta.ws.rs.core.Response webUiIdentity(@Context HttpServletRequest ser public jakarta.ws.rs.core.Response echoIdentity(HttpServletRequest servletRequest, HttpHeaders httpHeaders) { - Identity identity = sessionContextFactory.extractAuthorizedIdentity(servletRequest, httpHeaders, Optional.empty()); + Identity identity = sessionContextFactory.extractAuthorizedIdentity(servletRequest, httpHeaders); return jakarta.ws.rs.core.Response.ok() .header("user", identity.getUser()) .header("principal", identity.getPrincipal().map(Principal::getName).orElse(null)) diff --git a/core/trino-main/src/test/java/io/trino/server/ui/TestWebUi.java b/core/trino-main/src/test/java/io/trino/server/ui/TestWebUi.java index 838fd26d7f953..7fa2ad2b3610f 100644 --- a/core/trino-main/src/test/java/io/trino/server/ui/TestWebUi.java +++ b/core/trino-main/src/test/java/io/trino/server/ui/TestWebUi.java @@ -423,14 +423,15 @@ public TestResource(AccessControl accessControl) new PreparedStatementEncoder(new ProtocolConfig()), createTestMetadataManager(), ImmutableSet::of, - accessControl); + accessControl, + new ProtocolConfig()); } @ResourceSecurity(WEB_UI) @GET public jakarta.ws.rs.core.Response echoToken(@Context HttpServletRequest servletRequest, @Context HttpHeaders httpHeaders) { - Identity identity = sessionContextFactory.extractAuthorizedIdentity(servletRequest, httpHeaders, Optional.empty()); + Identity identity = sessionContextFactory.extractAuthorizedIdentity(servletRequest, httpHeaders); return jakarta.ws.rs.core.Response.ok() .header("user", identity.getUser()) .build(); From 4865d030ef636e42736e08b76eba5e4ceb23e442 Mon Sep 17 00:00:00 2001 From: David Phillips Date: Sun, 26 Nov 2023 16:41:22 -0600 Subject: [PATCH 067/350] Encapsulate authenticated identity servlet attribute --- .../dispatcher/QueuedStatementResource.java | 7 ++++--- .../HttpRequestSessionContextFactory.java | 6 ++---- .../io/trino/server/ServletSecurityUtils.java | 20 ++++++++++++++++++- .../ResourceSecurityDynamicFeature.java | 10 +++------- .../java/io/trino/server/ui/TestWebUi.java | 4 ++-- 5 files changed, 30 insertions(+), 17 deletions(-) diff --git a/core/trino-main/src/main/java/io/trino/dispatcher/QueuedStatementResource.java b/core/trino-main/src/main/java/io/trino/dispatcher/QueuedStatementResource.java index e4562422b4810..f2feb98f73574 100644 --- a/core/trino-main/src/main/java/io/trino/dispatcher/QueuedStatementResource.java +++ b/core/trino-main/src/main/java/io/trino/dispatcher/QueuedStatementResource.java @@ -84,7 +84,8 @@ import static io.airlift.jaxrs.AsyncResponseHandler.bindAsyncResponse; import static io.trino.execution.QueryState.FAILED; import static io.trino.execution.QueryState.QUEUED; -import static io.trino.server.HttpRequestSessionContextFactory.AUTHENTICATED_IDENTITY; +import static io.trino.server.ServletSecurityUtils.authenticatedIdentity; +import static io.trino.server.ServletSecurityUtils.clearAuthenticatedIdentity; import static io.trino.server.protocol.QueryInfoUrlFactory.getQueryInfoUri; import static io.trino.server.protocol.Slug.Context.EXECUTING_QUERY; import static io.trino.server.protocol.Slug.Context.QUEUED_QUERY; @@ -174,7 +175,7 @@ public Response postStatement( private Query registerQuery(String statement, HttpServletRequest servletRequest, HttpHeaders httpHeaders) { Optional remoteAddress = Optional.ofNullable(servletRequest.getRemoteAddr()); - Optional identity = Optional.ofNullable((Identity) servletRequest.getAttribute(AUTHENTICATED_IDENTITY)); + Optional identity = authenticatedIdentity(servletRequest); if (identity.flatMap(Identity::getPrincipal).map(InternalPrincipal.class::isInstance).orElse(false)) { throw badRequest(FORBIDDEN, "Internal communication can not be used to start a query"); } @@ -186,7 +187,7 @@ private Query registerQuery(String statement, HttpServletRequest servletRequest, queryManager.registerQuery(query); // let authentication filter know that identity lifecycle has been handed off - servletRequest.setAttribute(AUTHENTICATED_IDENTITY, null); + clearAuthenticatedIdentity(servletRequest); return query; } diff --git a/core/trino-main/src/main/java/io/trino/server/HttpRequestSessionContextFactory.java b/core/trino-main/src/main/java/io/trino/server/HttpRequestSessionContextFactory.java index 854a69d5d3035..775d663fe26bd 100644 --- a/core/trino-main/src/main/java/io/trino/server/HttpRequestSessionContextFactory.java +++ b/core/trino-main/src/main/java/io/trino/server/HttpRequestSessionContextFactory.java @@ -60,6 +60,7 @@ import static com.google.common.collect.ImmutableMap.toImmutableMap; import static com.google.common.net.HttpHeaders.USER_AGENT; import static io.trino.client.ProtocolHeaders.detectProtocol; +import static io.trino.server.ServletSecurityUtils.authenticatedIdentity; import static io.trino.spi.security.AccessDeniedException.denySetRole; import static java.lang.String.format; import static java.nio.charset.StandardCharsets.UTF_8; @@ -69,7 +70,6 @@ public class HttpRequestSessionContextFactory { private static final Splitter DOT_SPLITTER = Splitter.on('.'); - public static final String AUTHENTICATED_IDENTITY = "trino.authenticated-identity"; private final PreparedStatementEncoder preparedStatementEncoder; private final Metadata metadata; @@ -192,9 +192,7 @@ else if (nameParts.size() == 2) { public Identity extractAuthorizedIdentity(HttpServletRequest servletRequest, HttpHeaders httpHeaders) { - return extractAuthorizedIdentity( - Optional.ofNullable((Identity) servletRequest.getAttribute(AUTHENTICATED_IDENTITY)), - httpHeaders.getRequestHeaders()); + return extractAuthorizedIdentity(authenticatedIdentity(servletRequest), httpHeaders.getRequestHeaders()); } public Identity extractAuthorizedIdentity(Optional optionalAuthenticatedIdentity, MultivaluedMap headers) diff --git a/core/trino-main/src/main/java/io/trino/server/ServletSecurityUtils.java b/core/trino-main/src/main/java/io/trino/server/ServletSecurityUtils.java index e25e3be88760e..ebd3169b370f9 100644 --- a/core/trino-main/src/main/java/io/trino/server/ServletSecurityUtils.java +++ b/core/trino-main/src/main/java/io/trino/server/ServletSecurityUtils.java @@ -15,6 +15,7 @@ import io.trino.spi.security.BasicPrincipal; import io.trino.spi.security.Identity; +import jakarta.servlet.http.HttpServletRequest; import jakarta.ws.rs.container.ContainerRequestContext; import jakarta.ws.rs.core.Response; import jakarta.ws.rs.core.Response.ResponseBuilder; @@ -23,14 +24,16 @@ import java.security.Principal; import java.util.Collection; +import java.util.Optional; import static com.google.common.net.MediaType.PLAIN_TEXT_UTF_8; -import static io.trino.server.HttpRequestSessionContextFactory.AUTHENTICATED_IDENTITY; import static jakarta.ws.rs.core.HttpHeaders.WWW_AUTHENTICATE; import static jakarta.ws.rs.core.Response.Status.UNAUTHORIZED; public final class ServletSecurityUtils { + private static final String AUTHENTICATED_IDENTITY = "trino.authenticated-identity"; + private ServletSecurityUtils() {} public static void sendErrorMessage(ContainerRequestContext request, Status errorCode, String errorMessage) @@ -62,6 +65,21 @@ private static ResponseBuilder errorResponse(Status errorCode, String errorMessa .entity(errorMessage); } + public static Optional authenticatedIdentity(ContainerRequestContext request) + { + return Optional.ofNullable((Identity) request.getProperty(AUTHENTICATED_IDENTITY)); + } + + public static Optional authenticatedIdentity(HttpServletRequest request) + { + return Optional.ofNullable((Identity) request.getAttribute(AUTHENTICATED_IDENTITY)); + } + + public static void clearAuthenticatedIdentity(HttpServletRequest request) + { + request.setAttribute(AUTHENTICATED_IDENTITY, null); + } + public static void setAuthenticatedIdentity(ContainerRequestContext request, String username) { setAuthenticatedIdentity(request, Identity.forUser(username) diff --git a/core/trino-main/src/main/java/io/trino/server/security/ResourceSecurityDynamicFeature.java b/core/trino-main/src/main/java/io/trino/server/security/ResourceSecurityDynamicFeature.java index a5e106ebeac15..9296ffdb24e3d 100644 --- a/core/trino-main/src/main/java/io/trino/server/security/ResourceSecurityDynamicFeature.java +++ b/core/trino-main/src/main/java/io/trino/server/security/ResourceSecurityDynamicFeature.java @@ -36,7 +36,7 @@ import java.util.Optional; -import static io.trino.server.HttpRequestSessionContextFactory.AUTHENTICATED_IDENTITY; +import static io.trino.server.ServletSecurityUtils.authenticatedIdentity; import static io.trino.server.ServletSecurityUtils.setAuthenticatedIdentity; import static io.trino.server.security.ResourceSecurity.AccessType.MANAGEMENT_READ; import static io.trino.spi.StandardErrorCode.SERVER_STARTING_UP; @@ -153,9 +153,7 @@ public void filter(ContainerRequestContext request) } try { - Identity identity = sessionContextFactory.extractAuthorizedIdentity( - Optional.ofNullable((Identity) request.getProperty(AUTHENTICATED_IDENTITY)), - request.getHeaders()); + Identity identity = sessionContextFactory.extractAuthorizedIdentity(authenticatedIdentity(request), request.getHeaders()); if (read) { accessControl.checkCanReadSystemInformation(identity); } @@ -207,9 +205,7 @@ private static class DisposeIdentityResponseFilter public void filter(ContainerRequestContext request, ContainerResponseContext response) { // destroy identity if identity is still attached to the request - Optional.ofNullable(request.getProperty(AUTHENTICATED_IDENTITY)) - .map(Identity.class::cast) - .ifPresent(Identity::destroy); + authenticatedIdentity(request).ifPresent(Identity::destroy); } } } diff --git a/core/trino-main/src/test/java/io/trino/server/ui/TestWebUi.java b/core/trino-main/src/test/java/io/trino/server/ui/TestWebUi.java index 7fa2ad2b3610f..5494ac654c52f 100644 --- a/core/trino-main/src/test/java/io/trino/server/ui/TestWebUi.java +++ b/core/trino-main/src/test/java/io/trino/server/ui/TestWebUi.java @@ -100,7 +100,7 @@ import static io.jsonwebtoken.security.Keys.hmacShaKeyFor; import static io.trino.client.OkHttpUtil.setupSsl; import static io.trino.metadata.MetadataManager.createTestMetadataManager; -import static io.trino.server.HttpRequestSessionContextFactory.AUTHENTICATED_IDENTITY; +import static io.trino.server.ServletSecurityUtils.authenticatedIdentity; import static io.trino.server.security.ResourceSecurity.AccessType.WEB_UI; import static io.trino.server.security.jwt.JwtUtil.newJwtBuilder; import static io.trino.server.security.oauth2.OAuth2CallbackResource.CALLBACK_ENDPOINT; @@ -1400,7 +1400,7 @@ private static class AuthenticatedIdentityCapturingFilter public synchronized void filter(ContainerRequestContext request) throws IOException { - Optional identity = Optional.ofNullable((Identity) request.getProperty(AUTHENTICATED_IDENTITY)); + Optional identity = authenticatedIdentity(request); if (identity.map(Identity::getUser).filter(not(""::equals)).isPresent()) { if (authenticatedIdentity == null) { authenticatedIdentity = identity.get(); From 403ad890b97c45bebf752044a238375d16cddfd7 Mon Sep 17 00:00:00 2001 From: David Phillips Date: Sun, 26 Nov 2023 16:53:05 -0600 Subject: [PATCH 068/350] Move TransactionBuilder to testing package --- .../src/main/java/io/trino/testing/LocalQueryRunner.java | 2 +- .../trino/{transaction => testing}/TransactionBuilder.java | 5 ++++- .../src/test/java/io/trino/cost/TestCostCalculator.java | 2 +- .../test/java/io/trino/cost/TestFilterStatsCalculator.java | 2 +- .../test/java/io/trino/cost/TestScalarStatsCalculator.java | 2 +- .../java/io/trino/metadata/TestingFunctionResolution.java | 2 +- .../java/io/trino/security/TestAccessControlManager.java | 2 +- .../io/trino/security/TestFileBasedSystemAccessControl.java | 2 +- .../src/test/java/io/trino/sql/ExpressionTestUtils.java | 2 +- .../test/java/io/trino/sql/TestExpressionInterpreter.java | 2 +- .../src/test/java/io/trino/sql/analyzer/TestAnalyzer.java | 2 +- .../trino/sql/planner/TestConnectorExpressionTranslator.java | 2 +- .../test/java/io/trino/sql/planner/TestDomainTranslator.java | 2 +- .../trino/sql/planner/TestEffectivePredicateExtractor.java | 2 +- .../test/java/io/trino/sql/planner/TestLiteralEncoder.java | 2 +- .../io/trino/sql/planner/TestPlanFragmentPartitionCount.java | 2 +- .../iterative/rule/TestCanonicalizeExpressionRewriter.java | 2 +- .../sql/planner/optimizations/TestExpressionEquivalence.java | 2 +- .../src/test/java/io/trino/sql/query/QueryAssertions.java | 2 +- .../src/test/java/io/trino/sql/routine/TestSqlFunctions.java | 2 +- .../java/io/trino/sql/routine/TestSqlRoutineAnalyzer.java | 2 +- .../java/io/trino/plugin/hive/BaseHiveConnectorTest.java | 2 +- .../io/trino/plugin/iceberg/BaseIcebergConnectorTest.java | 2 +- .../io/trino/plugin/kinesis/TestMinimalFunctionality.java | 2 +- .../test/java/io/trino/plugin/kinesis/TestRecordAccess.java | 2 +- .../java/io/trino/plugin/redis/TestMinimalFunctionality.java | 2 +- .../faulttolerant/TestOverridePartitionCountRecursively.java | 2 +- .../java/io/trino/testing/AbstractTestQueryFramework.java | 3 +-- .../java/io/trino/testing/AbstractTestingTrinoClient.java | 2 +- .../java/io/trino/testing/statistics/MetricComparator.java | 2 +- .../io/trino/tests/TestGetTableStatisticsOperations.java | 2 +- .../src/test/java/io/trino/tests/TestMetadataManager.java | 2 +- 32 files changed, 35 insertions(+), 33 deletions(-) rename core/trino-main/src/main/java/io/trino/{transaction => testing}/TransactionBuilder.java (97%) diff --git a/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java b/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java index 289a7fff840cc..6b558283e7ac7 100644 --- a/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java +++ b/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java @@ -253,7 +253,7 @@ import static io.trino.sql.planner.LogicalPlanner.Stage.OPTIMIZED_AND_VALIDATED; import static io.trino.sql.planner.optimizations.PlanNodeSearcher.searchFrom; import static io.trino.sql.testing.TreeAssertions.assertFormattedSql; -import static io.trino.transaction.TransactionBuilder.transaction; +import static io.trino.testing.TransactionBuilder.transaction; import static io.trino.version.EmbedVersion.testingVersionEmbedder; import static java.util.Objects.requireNonNull; import static java.util.concurrent.Executors.newCachedThreadPool; diff --git a/core/trino-main/src/main/java/io/trino/transaction/TransactionBuilder.java b/core/trino-main/src/main/java/io/trino/testing/TransactionBuilder.java similarity index 97% rename from core/trino-main/src/main/java/io/trino/transaction/TransactionBuilder.java rename to core/trino-main/src/main/java/io/trino/testing/TransactionBuilder.java index d2a1ec3e50fd1..d2345917c421b 100644 --- a/core/trino-main/src/main/java/io/trino/transaction/TransactionBuilder.java +++ b/core/trino-main/src/main/java/io/trino/testing/TransactionBuilder.java @@ -11,13 +11,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.transaction; +package io.trino.testing; import io.trino.Session; import io.trino.execution.QueryIdGenerator; import io.trino.metadata.Metadata; import io.trino.security.AccessControl; import io.trino.spi.transaction.IsolationLevel; +import io.trino.transaction.TransactionId; +import io.trino.transaction.TransactionInfo; +import io.trino.transaction.TransactionManager; import java.util.function.Consumer; import java.util.function.Function; diff --git a/core/trino-main/src/test/java/io/trino/cost/TestCostCalculator.java b/core/trino-main/src/test/java/io/trino/cost/TestCostCalculator.java index 1a2f485762238..afb47bb2c7bb8 100644 --- a/core/trino-main/src/test/java/io/trino/cost/TestCostCalculator.java +++ b/core/trino-main/src/test/java/io/trino/cost/TestCostCalculator.java @@ -76,7 +76,7 @@ import static io.trino.sql.planner.plan.ExchangeNode.replicatedExchange; import static io.trino.testing.TestingHandles.TEST_CATALOG_NAME; import static io.trino.testing.TestingSession.testSessionBuilder; -import static io.trino.transaction.TransactionBuilder.transaction; +import static io.trino.testing.TransactionBuilder.transaction; import static java.lang.String.format; import static java.util.Objects.requireNonNull; import static org.assertj.core.api.Assertions.assertThat; diff --git a/core/trino-main/src/test/java/io/trino/cost/TestFilterStatsCalculator.java b/core/trino-main/src/test/java/io/trino/cost/TestFilterStatsCalculator.java index 2f52d31636708..40c953ebe282d 100644 --- a/core/trino-main/src/test/java/io/trino/cost/TestFilterStatsCalculator.java +++ b/core/trino-main/src/test/java/io/trino/cost/TestFilterStatsCalculator.java @@ -38,7 +38,7 @@ import static io.trino.sql.planner.TypeAnalyzer.createTestingTypeAnalyzer; import static io.trino.sql.planner.iterative.rule.test.PlanBuilder.expression; import static io.trino.testing.TestingSession.testSessionBuilder; -import static io.trino.transaction.TransactionBuilder.transaction; +import static io.trino.testing.TransactionBuilder.transaction; import static java.lang.Double.NEGATIVE_INFINITY; import static java.lang.Double.NaN; import static java.lang.Double.POSITIVE_INFINITY; diff --git a/core/trino-main/src/test/java/io/trino/cost/TestScalarStatsCalculator.java b/core/trino-main/src/test/java/io/trino/cost/TestScalarStatsCalculator.java index c2dac31de5ec4..4a12f759ac59d 100644 --- a/core/trino-main/src/test/java/io/trino/cost/TestScalarStatsCalculator.java +++ b/core/trino-main/src/test/java/io/trino/cost/TestScalarStatsCalculator.java @@ -44,7 +44,7 @@ import static io.trino.sql.analyzer.TypeSignatureTranslator.toSqlType; import static io.trino.sql.planner.TypeAnalyzer.createTestingTypeAnalyzer; import static io.trino.testing.TestingSession.testSessionBuilder; -import static io.trino.transaction.TransactionBuilder.transaction; +import static io.trino.testing.TransactionBuilder.transaction; import static java.lang.Double.NEGATIVE_INFINITY; import static java.lang.Double.POSITIVE_INFINITY; diff --git a/core/trino-main/src/test/java/io/trino/metadata/TestingFunctionResolution.java b/core/trino-main/src/test/java/io/trino/metadata/TestingFunctionResolution.java index a79adfb3f9b4c..bb1746f7bdc8f 100644 --- a/core/trino-main/src/test/java/io/trino/metadata/TestingFunctionResolution.java +++ b/core/trino-main/src/test/java/io/trino/metadata/TestingFunctionResolution.java @@ -39,8 +39,8 @@ import static io.trino.SessionTestUtils.TEST_SESSION; import static io.trino.sql.planner.TestingPlannerContext.plannerContextBuilder; +import static io.trino.testing.TransactionBuilder.transaction; import static io.trino.transaction.InMemoryTransactionManager.createTestTransactionManager; -import static io.trino.transaction.TransactionBuilder.transaction; import static java.util.Objects.requireNonNull; public class TestingFunctionResolution diff --git a/core/trino-main/src/test/java/io/trino/security/TestAccessControlManager.java b/core/trino-main/src/test/java/io/trino/security/TestAccessControlManager.java index 7545f5d5227e5..961353ef7d3fc 100644 --- a/core/trino-main/src/test/java/io/trino/security/TestAccessControlManager.java +++ b/core/trino-main/src/test/java/io/trino/security/TestAccessControlManager.java @@ -67,8 +67,8 @@ import static io.trino.spi.security.AccessDeniedException.denySelectTable; import static io.trino.testing.TestingEventListenerManager.emptyEventListenerManager; import static io.trino.testing.TestingHandles.TEST_CATALOG_NAME; +import static io.trino.testing.TransactionBuilder.transaction; import static io.trino.transaction.InMemoryTransactionManager.createTestTransactionManager; -import static io.trino.transaction.TransactionBuilder.transaction; import static java.nio.file.Files.createTempFile; import static java.nio.file.StandardOpenOption.CREATE; import static java.nio.file.StandardOpenOption.TRUNCATE_EXISTING; diff --git a/core/trino-main/src/test/java/io/trino/security/TestFileBasedSystemAccessControl.java b/core/trino-main/src/test/java/io/trino/security/TestFileBasedSystemAccessControl.java index 010c930212bfc..2cb2f6533d9d5 100644 --- a/core/trino-main/src/test/java/io/trino/security/TestFileBasedSystemAccessControl.java +++ b/core/trino-main/src/test/java/io/trino/security/TestFileBasedSystemAccessControl.java @@ -50,8 +50,8 @@ import static io.trino.spi.security.PrincipalType.USER; import static io.trino.spi.security.Privilege.SELECT; import static io.trino.testing.TestingEventListenerManager.emptyEventListenerManager; +import static io.trino.testing.TransactionBuilder.transaction; import static io.trino.transaction.InMemoryTransactionManager.createTestTransactionManager; -import static io.trino.transaction.TransactionBuilder.transaction; import static java.lang.Thread.sleep; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; diff --git a/core/trino-main/src/test/java/io/trino/sql/ExpressionTestUtils.java b/core/trino-main/src/test/java/io/trino/sql/ExpressionTestUtils.java index 008dd7f0e21ec..5be0d4ab4667c 100644 --- a/core/trino-main/src/test/java/io/trino/sql/ExpressionTestUtils.java +++ b/core/trino-main/src/test/java/io/trino/sql/ExpressionTestUtils.java @@ -43,7 +43,7 @@ import static io.trino.sql.analyzer.SemanticExceptions.semanticException; import static io.trino.sql.analyzer.TypeSignatureTranslator.toSqlType; import static io.trino.sql.planner.TypeAnalyzer.createTestingTypeAnalyzer; -import static io.trino.transaction.TransactionBuilder.transaction; +import static io.trino.testing.TransactionBuilder.transaction; public final class ExpressionTestUtils { diff --git a/core/trino-main/src/test/java/io/trino/sql/TestExpressionInterpreter.java b/core/trino-main/src/test/java/io/trino/sql/TestExpressionInterpreter.java index 70fa383e6af33..71b8383a61a6c 100644 --- a/core/trino-main/src/test/java/io/trino/sql/TestExpressionInterpreter.java +++ b/core/trino-main/src/test/java/io/trino/sql/TestExpressionInterpreter.java @@ -71,8 +71,8 @@ import static io.trino.sql.ExpressionUtils.rewriteIdentifiersToSymbolReferences; import static io.trino.sql.planner.TestingPlannerContext.plannerContextBuilder; import static io.trino.sql.planner.TypeAnalyzer.createTestingTypeAnalyzer; +import static io.trino.testing.TransactionBuilder.transaction; import static io.trino.testing.assertions.TrinoExceptionAssert.assertTrinoExceptionThrownBy; -import static io.trino.transaction.TransactionBuilder.transaction; import static io.trino.type.DateTimes.scaleEpochMillisToMicros; import static io.trino.type.IntervalDayTimeType.INTERVAL_DAY_TIME; import static java.lang.String.format; diff --git a/core/trino-main/src/test/java/io/trino/sql/analyzer/TestAnalyzer.java b/core/trino-main/src/test/java/io/trino/sql/analyzer/TestAnalyzer.java index fb8f3dac141fd..503f5ba2dbec0 100644 --- a/core/trino-main/src/test/java/io/trino/sql/analyzer/TestAnalyzer.java +++ b/core/trino-main/src/test/java/io/trino/sql/analyzer/TestAnalyzer.java @@ -199,8 +199,8 @@ import static io.trino.testing.TestingAccessControlManager.privilege; import static io.trino.testing.TestingEventListenerManager.emptyEventListenerManager; import static io.trino.testing.TestingSession.testSessionBuilder; +import static io.trino.testing.TransactionBuilder.transaction; import static io.trino.testing.assertions.TrinoExceptionAssert.assertTrinoExceptionThrownBy; -import static io.trino.transaction.TransactionBuilder.transaction; import static io.trino.type.InternalTypeManager.TESTING_TYPE_MANAGER; import static java.lang.String.format; import static java.util.Collections.emptyList; diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/TestConnectorExpressionTranslator.java b/core/trino-main/src/test/java/io/trino/sql/planner/TestConnectorExpressionTranslator.java index fa755e6a50ca5..0db62980c9c59 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/TestConnectorExpressionTranslator.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/TestConnectorExpressionTranslator.java @@ -86,7 +86,7 @@ import static io.trino.sql.planner.ConnectorExpressionTranslator.translate; import static io.trino.sql.planner.TestingPlannerContext.PLANNER_CONTEXT; import static io.trino.sql.planner.TypeAnalyzer.createTestingTypeAnalyzer; -import static io.trino.transaction.TransactionBuilder.transaction; +import static io.trino.testing.TransactionBuilder.transaction; import static io.trino.type.JoniRegexpType.JONI_REGEXP; import static io.trino.type.LikeFunctions.likePattern; import static io.trino.type.LikePatternType.LIKE_PATTERN; diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/TestDomainTranslator.java b/core/trino-main/src/test/java/io/trino/sql/planner/TestDomainTranslator.java index 4bad8335dab96..0259b1d52ff71 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/TestDomainTranslator.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/TestDomainTranslator.java @@ -98,7 +98,7 @@ import static io.trino.sql.tree.ComparisonExpression.Operator.LESS_THAN_OR_EQUAL; import static io.trino.sql.tree.ComparisonExpression.Operator.NOT_EQUAL; import static io.trino.testing.TestingConnectorSession.SESSION; -import static io.trino.transaction.TransactionBuilder.transaction; +import static io.trino.testing.TransactionBuilder.transaction; import static io.trino.type.ColorType.COLOR; import static io.trino.type.LikeFunctions.LIKE_FUNCTION_NAME; import static io.trino.type.LikeFunctions.LIKE_PATTERN_FUNCTION_NAME; diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/TestEffectivePredicateExtractor.java b/core/trino-main/src/test/java/io/trino/sql/planner/TestEffectivePredicateExtractor.java index d46dcfd2abd71..1244478cc0ed8 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/TestEffectivePredicateExtractor.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/TestEffectivePredicateExtractor.java @@ -113,8 +113,8 @@ import static io.trino.sql.tree.BooleanLiteral.TRUE_LITERAL; import static io.trino.sql.tree.ComparisonExpression.Operator.EQUAL; import static io.trino.testing.TestingHandles.TEST_CATALOG_HANDLE; +import static io.trino.testing.TransactionBuilder.transaction; import static io.trino.tests.BogusType.BOGUS; -import static io.trino.transaction.TransactionBuilder.transaction; import static io.trino.type.UnknownType.UNKNOWN; import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_METHOD; diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/TestLiteralEncoder.java b/core/trino-main/src/test/java/io/trino/sql/planner/TestLiteralEncoder.java index 05ce1f93e8173..ce290fac07f5d 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/TestLiteralEncoder.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/TestLiteralEncoder.java @@ -72,7 +72,7 @@ import static io.trino.sql.ExpressionUtils.isEffectivelyLiteral; import static io.trino.sql.SqlFormatter.formatSql; import static io.trino.sql.planner.TestingPlannerContext.PLANNER_CONTEXT; -import static io.trino.transaction.TransactionBuilder.transaction; +import static io.trino.testing.TransactionBuilder.transaction; import static io.trino.type.CodePointsType.CODE_POINTS; import static io.trino.type.JoniRegexpType.JONI_REGEXP; import static io.trino.type.JsonPathType.JSON_PATH; diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/TestPlanFragmentPartitionCount.java b/core/trino-main/src/test/java/io/trino/sql/planner/TestPlanFragmentPartitionCount.java index ddcae1df4180e..5aeffcb41522d 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/TestPlanFragmentPartitionCount.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/TestPlanFragmentPartitionCount.java @@ -42,7 +42,7 @@ import static io.trino.sql.planner.plan.JoinNode.Type.INNER; import static io.trino.testing.TestingHandles.TEST_CATALOG_NAME; import static io.trino.testing.TestingSession.testSessionBuilder; -import static io.trino.transaction.TransactionBuilder.transaction; +import static io.trino.testing.TransactionBuilder.transaction; import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestCanonicalizeExpressionRewriter.java b/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestCanonicalizeExpressionRewriter.java index cfa726c2bbad3..9680dec3d94a3 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestCanonicalizeExpressionRewriter.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestCanonicalizeExpressionRewriter.java @@ -39,8 +39,8 @@ import static io.trino.sql.planner.TestingPlannerContext.plannerContextBuilder; import static io.trino.sql.planner.TypeAnalyzer.createTestingTypeAnalyzer; import static io.trino.sql.planner.iterative.rule.CanonicalizeExpressionRewriter.rewrite; +import static io.trino.testing.TransactionBuilder.transaction; import static io.trino.transaction.InMemoryTransactionManager.createTestTransactionManager; -import static io.trino.transaction.TransactionBuilder.transaction; public class TestCanonicalizeExpressionRewriter { diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestExpressionEquivalence.java b/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestExpressionEquivalence.java index cdc9179eab7b6..076705032a2e8 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestExpressionEquivalence.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestExpressionEquivalence.java @@ -41,7 +41,7 @@ import static io.trino.sql.planner.SymbolsExtractor.extractUnique; import static io.trino.sql.planner.TestingPlannerContext.plannerContextBuilder; import static io.trino.sql.planner.TypeAnalyzer.createTestingTypeAnalyzer; -import static io.trino.transaction.TransactionBuilder.transaction; +import static io.trino.testing.TransactionBuilder.transaction; import static java.lang.String.format; import static java.util.function.Function.identity; import static java.util.stream.Collectors.toMap; diff --git a/core/trino-main/src/test/java/io/trino/sql/query/QueryAssertions.java b/core/trino-main/src/test/java/io/trino/sql/query/QueryAssertions.java index e50954ddbba7d..75d5579809431 100644 --- a/core/trino-main/src/test/java/io/trino/sql/query/QueryAssertions.java +++ b/core/trino-main/src/test/java/io/trino/sql/query/QueryAssertions.java @@ -64,7 +64,7 @@ import static io.trino.sql.query.QueryAssertions.QueryAssert.newQueryAssert; import static io.trino.testing.TestingHandles.TEST_CATALOG_NAME; import static io.trino.testing.TestingSession.testSessionBuilder; -import static io.trino.transaction.TransactionBuilder.transaction; +import static io.trino.testing.TransactionBuilder.transaction; import static java.lang.String.format; import static java.util.Objects.requireNonNull; import static java.util.stream.Collectors.toList; diff --git a/core/trino-main/src/test/java/io/trino/sql/routine/TestSqlFunctions.java b/core/trino-main/src/test/java/io/trino/sql/routine/TestSqlFunctions.java index 8807b83c3c398..e87783adf57ac 100644 --- a/core/trino-main/src/test/java/io/trino/sql/routine/TestSqlFunctions.java +++ b/core/trino-main/src/test/java/io/trino/sql/routine/TestSqlFunctions.java @@ -49,8 +49,8 @@ import static io.trino.spi.type.VarcharType.VARCHAR; import static io.trino.sql.planner.TestingPlannerContext.plannerContextBuilder; import static io.trino.testing.TestingSession.testSessionBuilder; +import static io.trino.testing.TransactionBuilder.transaction; import static io.trino.transaction.InMemoryTransactionManager.createTestTransactionManager; -import static io.trino.transaction.TransactionBuilder.transaction; import static io.trino.type.UnknownType.UNKNOWN; import static java.lang.Math.floor; import static org.assertj.core.api.Assertions.assertThat; diff --git a/core/trino-main/src/test/java/io/trino/sql/routine/TestSqlRoutineAnalyzer.java b/core/trino-main/src/test/java/io/trino/sql/routine/TestSqlRoutineAnalyzer.java index 4d8c325926481..e0cddbc75523d 100644 --- a/core/trino-main/src/test/java/io/trino/sql/routine/TestSqlRoutineAnalyzer.java +++ b/core/trino-main/src/test/java/io/trino/sql/routine/TestSqlRoutineAnalyzer.java @@ -33,8 +33,8 @@ import static io.trino.spi.StandardErrorCode.TYPE_MISMATCH; import static io.trino.sql.planner.TestingPlannerContext.plannerContextBuilder; import static io.trino.testing.TestingSession.testSession; +import static io.trino.testing.TransactionBuilder.transaction; import static io.trino.testing.assertions.TrinoExceptionAssert.assertTrinoExceptionThrownBy; -import static io.trino.transaction.TransactionBuilder.transaction; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.from; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java index 27d90ae44a4b0..337bb255d4029 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java @@ -178,7 +178,7 @@ import static io.trino.testing.TestingAccessControlManager.privilege; import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.testing.TestingSession.testSessionBuilder; -import static io.trino.transaction.TransactionBuilder.transaction; +import static io.trino.testing.TransactionBuilder.transaction; import static io.trino.type.InternalTypeManager.TESTING_TYPE_MANAGER; import static java.lang.String.format; import static java.lang.String.join; diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java index 6fe1ac5518f8e..6821a6aaeb990 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java @@ -132,8 +132,8 @@ import static io.trino.testing.TestingConnectorSession.SESSION; import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.testing.TestingSession.testSessionBuilder; +import static io.trino.testing.TransactionBuilder.transaction; import static io.trino.testing.assertions.Assert.assertEventually; -import static io.trino.transaction.TransactionBuilder.transaction; import static java.lang.String.format; import static java.lang.String.join; import static java.nio.charset.StandardCharsets.UTF_8; diff --git a/plugin/trino-kinesis/src/test/java/io/trino/plugin/kinesis/TestMinimalFunctionality.java b/plugin/trino-kinesis/src/test/java/io/trino/plugin/kinesis/TestMinimalFunctionality.java index 18a3297bf2c73..cfa8372decd0f 100644 --- a/plugin/trino-kinesis/src/test/java/io/trino/plugin/kinesis/TestMinimalFunctionality.java +++ b/plugin/trino-kinesis/src/test/java/io/trino/plugin/kinesis/TestMinimalFunctionality.java @@ -45,7 +45,7 @@ import java.util.stream.Stream; import static io.trino.spi.type.TimeZoneKey.UTC_KEY; -import static io.trino.transaction.TransactionBuilder.transaction; +import static io.trino.testing.TransactionBuilder.transaction; import static java.nio.charset.StandardCharsets.UTF_8; import static java.util.Locale.ENGLISH; import static org.assertj.core.api.Assertions.assertThat; diff --git a/plugin/trino-kinesis/src/test/java/io/trino/plugin/kinesis/TestRecordAccess.java b/plugin/trino-kinesis/src/test/java/io/trino/plugin/kinesis/TestRecordAccess.java index 4ee7baffd22e7..0030cd70d5af1 100644 --- a/plugin/trino-kinesis/src/test/java/io/trino/plugin/kinesis/TestRecordAccess.java +++ b/plugin/trino-kinesis/src/test/java/io/trino/plugin/kinesis/TestRecordAccess.java @@ -43,7 +43,7 @@ import java.util.zip.GZIPOutputStream; import static io.trino.testing.TestingSession.testSessionBuilder; -import static io.trino.transaction.TransactionBuilder.transaction; +import static io.trino.testing.TransactionBuilder.transaction; import static java.lang.String.format; import static java.nio.charset.StandardCharsets.UTF_8; import static org.assertj.core.api.Assertions.assertThat; diff --git a/plugin/trino-redis/src/test/java/io/trino/plugin/redis/TestMinimalFunctionality.java b/plugin/trino-redis/src/test/java/io/trino/plugin/redis/TestMinimalFunctionality.java index ac67e6d9d1086..6ebcdf32c74ee 100644 --- a/plugin/trino-redis/src/test/java/io/trino/plugin/redis/TestMinimalFunctionality.java +++ b/plugin/trino-redis/src/test/java/io/trino/plugin/redis/TestMinimalFunctionality.java @@ -22,7 +22,7 @@ import java.util.Map; import java.util.Optional; -import static io.trino.transaction.TransactionBuilder.transaction; +import static io.trino.testing.TransactionBuilder.transaction; import static java.lang.String.format; import static org.assertj.core.api.Assertions.assertThat; diff --git a/testing/trino-faulttolerant-tests/src/test/java/io/trino/faulttolerant/TestOverridePartitionCountRecursively.java b/testing/trino-faulttolerant-tests/src/test/java/io/trino/faulttolerant/TestOverridePartitionCountRecursively.java index e73ec67fafe46..98c6fb53f33bd 100644 --- a/testing/trino-faulttolerant-tests/src/test/java/io/trino/faulttolerant/TestOverridePartitionCountRecursively.java +++ b/testing/trino-faulttolerant-tests/src/test/java/io/trino/faulttolerant/TestOverridePartitionCountRecursively.java @@ -57,8 +57,8 @@ import static io.trino.sql.planner.SystemPartitioningHandle.SINGLE_DISTRIBUTION; import static io.trino.sql.planner.SystemPartitioningHandle.SOURCE_DISTRIBUTION; import static io.trino.sql.planner.TopologicalOrderSubPlanVisitor.sortPlanInTopologicalOrder; +import static io.trino.testing.TransactionBuilder.transaction; import static io.trino.tpch.TpchTable.getTables; -import static io.trino.transaction.TransactionBuilder.transaction; import static java.util.Objects.requireNonNull; import static org.assertj.core.api.Assertions.assertThat; diff --git a/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestQueryFramework.java b/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestQueryFramework.java index 5b6af5f28b3d5..e4fb18e592aaf 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestQueryFramework.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestQueryFramework.java @@ -53,7 +53,6 @@ import io.trino.sql.tree.ExplainType; import io.trino.testing.TestingAccessControlManager.TestingPrivilege; import io.trino.testng.services.ReportBadTestAnnotations; -import io.trino.transaction.TransactionBuilder; import io.trino.util.AutoCloseableCloser; import org.assertj.core.api.AssertProvider; import org.intellij.lang.annotations.Language; @@ -80,8 +79,8 @@ import static io.trino.execution.querystats.PlanOptimizersStatsCollector.createPlanOptimizersStatsCollector; import static io.trino.sql.SqlFormatter.formatSql; import static io.trino.sql.planner.OptimizerConfig.JoinReorderingStrategy; +import static io.trino.testing.TransactionBuilder.transaction; import static io.trino.testing.assertions.Assert.assertEventually; -import static io.trino.transaction.TransactionBuilder.transaction; import static java.lang.String.format; import static java.util.Collections.emptyList; import static java.util.concurrent.TimeUnit.SECONDS; diff --git a/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestingTrinoClient.java b/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestingTrinoClient.java index 806ba333ecb0c..f2a9cb46267a6 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestingTrinoClient.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestingTrinoClient.java @@ -48,7 +48,7 @@ import static io.trino.spi.session.ResourceEstimates.EXECUTION_TIME; import static io.trino.spi.session.ResourceEstimates.PEAK_MEMORY; import static io.trino.testing.TestingStatementClientFactory.DEFAULT_STATEMENT_FACTORY; -import static io.trino.transaction.TransactionBuilder.transaction; +import static io.trino.testing.TransactionBuilder.transaction; import static java.util.Objects.requireNonNull; public abstract class AbstractTestingTrinoClient diff --git a/testing/trino-testing/src/main/java/io/trino/testing/statistics/MetricComparator.java b/testing/trino-testing/src/main/java/io/trino/testing/statistics/MetricComparator.java index 40c4e59d63692..0cd60854a5f9d 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/statistics/MetricComparator.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/statistics/MetricComparator.java @@ -28,7 +28,7 @@ import static com.google.common.collect.ImmutableList.toImmutableList; import static com.google.common.collect.Iterables.getOnlyElement; -import static io.trino.transaction.TransactionBuilder.transaction; +import static io.trino.testing.TransactionBuilder.transaction; import static java.lang.String.format; import static java.util.stream.Collectors.joining; diff --git a/testing/trino-tests/src/test/java/io/trino/tests/TestGetTableStatisticsOperations.java b/testing/trino-tests/src/test/java/io/trino/tests/TestGetTableStatisticsOperations.java index e50a7c4307f9c..8864318a11ea1 100644 --- a/testing/trino-tests/src/test/java/io/trino/tests/TestGetTableStatisticsOperations.java +++ b/testing/trino-tests/src/test/java/io/trino/tests/TestGetTableStatisticsOperations.java @@ -33,7 +33,7 @@ import static io.trino.execution.querystats.PlanOptimizersStatsCollector.createPlanOptimizersStatsCollector; import static io.trino.sql.planner.LogicalPlanner.Stage.OPTIMIZED_AND_VALIDATED; import static io.trino.testing.TestingSession.testSessionBuilder; -import static io.trino.transaction.TransactionBuilder.transaction; +import static io.trino.testing.TransactionBuilder.transaction; import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; diff --git a/testing/trino-tests/src/test/java/io/trino/tests/TestMetadataManager.java b/testing/trino-tests/src/test/java/io/trino/tests/TestMetadataManager.java index c40dec07c4c8a..aaae7d41b06cf 100644 --- a/testing/trino-tests/src/test/java/io/trino/tests/TestMetadataManager.java +++ b/testing/trino-tests/src/test/java/io/trino/tests/TestMetadataManager.java @@ -30,9 +30,9 @@ import io.trino.spi.connector.SchemaTableName; import io.trino.testing.DistributedQueryRunner; import io.trino.testing.TestingSessionContext; +import io.trino.testing.TransactionBuilder; import io.trino.tests.tpch.TpchQueryRunnerBuilder; import io.trino.tracing.TracingMetadata; -import io.trino.transaction.TransactionBuilder; import org.intellij.lang.annotations.Language; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; From c6da488a4640b7b0197b405e459def22df37ee01 Mon Sep 17 00:00:00 2001 From: David Phillips Date: Sun, 26 Nov 2023 16:58:02 -0600 Subject: [PATCH 069/350] Move EmbedVersion to util package --- .../execution/executor/timesharing/TimeSharingTaskExecutor.java | 2 +- core/trino-main/src/main/java/io/trino/server/Server.java | 2 +- .../src/main/java/io/trino/server/ServerMainModule.java | 2 +- .../src/main/java/io/trino/testing/LocalQueryRunner.java | 2 +- .../src/main/java/io/trino/testing/TestingConnectorContext.java | 2 +- .../src/main/java/io/trino/{version => util}/EmbedVersion.java | 2 +- .../test/java/io/trino/execution/BaseTestSqlTaskManager.java | 2 +- .../trino/execution/TestSqlTaskManagerRaceWithCatalogPrune.java | 2 +- .../io/trino/execution/TestSqlTaskManagerThreadPerDriver.java | 2 +- .../java/io/trino/execution/TestTaskExecutorStuckSplits.java | 2 +- .../executor/dedicated/TestThreadPerDriverTaskExecutor.java | 2 +- .../test/java/io/trino/{version => util}/TestEmbedVersion.java | 2 +- 12 files changed, 12 insertions(+), 12 deletions(-) rename core/trino-main/src/main/java/io/trino/{version => util}/EmbedVersion.java (99%) rename core/trino-main/src/test/java/io/trino/{version => util}/TestEmbedVersion.java (99%) diff --git a/core/trino-main/src/main/java/io/trino/execution/executor/timesharing/TimeSharingTaskExecutor.java b/core/trino-main/src/main/java/io/trino/execution/executor/timesharing/TimeSharingTaskExecutor.java index 4536d9437af13..8a49ca436cd61 100644 --- a/core/trino-main/src/main/java/io/trino/execution/executor/timesharing/TimeSharingTaskExecutor.java +++ b/core/trino-main/src/main/java/io/trino/execution/executor/timesharing/TimeSharingTaskExecutor.java @@ -73,7 +73,7 @@ import static io.airlift.concurrent.Threads.threadsNamed; import static io.airlift.tracing.Tracing.noopTracer; import static io.trino.execution.executor.timesharing.MultilevelSplitQueue.computeLevel; -import static io.trino.version.EmbedVersion.testingVersionEmbedder; +import static io.trino.util.EmbedVersion.testingVersionEmbedder; import static java.lang.Math.min; import static java.lang.String.format; import static java.util.Objects.requireNonNull; diff --git a/core/trino-main/src/main/java/io/trino/server/Server.java b/core/trino-main/src/main/java/io/trino/server/Server.java index c90c1221af699..a2510514c9155 100644 --- a/core/trino-main/src/main/java/io/trino/server/Server.java +++ b/core/trino-main/src/main/java/io/trino/server/Server.java @@ -63,7 +63,7 @@ import io.trino.server.security.oauth2.OAuth2Client; import io.trino.spi.connector.CatalogHandle; import io.trino.transaction.TransactionManagerModule; -import io.trino.version.EmbedVersion; +import io.trino.util.EmbedVersion; import org.weakref.jmx.guice.MBeanModule; import java.io.IOException; diff --git a/core/trino-main/src/main/java/io/trino/server/ServerMainModule.java b/core/trino-main/src/main/java/io/trino/server/ServerMainModule.java index ad76199fe4b19..98b6a2ddb34fd 100644 --- a/core/trino-main/src/main/java/io/trino/server/ServerMainModule.java +++ b/core/trino-main/src/main/java/io/trino/server/ServerMainModule.java @@ -153,8 +153,8 @@ import io.trino.type.TypeOperatorsCache; import io.trino.type.TypeSignatureDeserializer; import io.trino.type.TypeSignatureKeyDeserializer; +import io.trino.util.EmbedVersion; import io.trino.util.FinalizerService; -import io.trino.version.EmbedVersion; import jakarta.annotation.PreDestroy; import java.util.List; diff --git a/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java b/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java index 6b558283e7ac7..9b7b2a3e7349f 100644 --- a/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java +++ b/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java @@ -254,7 +254,7 @@ import static io.trino.sql.planner.optimizations.PlanNodeSearcher.searchFrom; import static io.trino.sql.testing.TreeAssertions.assertFormattedSql; import static io.trino.testing.TransactionBuilder.transaction; -import static io.trino.version.EmbedVersion.testingVersionEmbedder; +import static io.trino.util.EmbedVersion.testingVersionEmbedder; import static java.util.Objects.requireNonNull; import static java.util.concurrent.Executors.newCachedThreadPool; import static java.util.concurrent.Executors.newScheduledThreadPool; diff --git a/core/trino-main/src/main/java/io/trino/testing/TestingConnectorContext.java b/core/trino-main/src/main/java/io/trino/testing/TestingConnectorContext.java index d7ff18054488c..a74f99807a950 100644 --- a/core/trino-main/src/main/java/io/trino/testing/TestingConnectorContext.java +++ b/core/trino-main/src/main/java/io/trino/testing/TestingConnectorContext.java @@ -31,7 +31,7 @@ import io.trino.spi.type.TypeManager; import io.trino.spi.type.TypeOperators; import io.trino.sql.gen.JoinCompiler; -import io.trino.version.EmbedVersion; +import io.trino.util.EmbedVersion; import static io.trino.spi.connector.MetadataProvider.NOOP_METADATA_PROVIDER; import static io.trino.testing.TestingHandles.TEST_CATALOG_HANDLE; diff --git a/core/trino-main/src/main/java/io/trino/version/EmbedVersion.java b/core/trino-main/src/main/java/io/trino/util/EmbedVersion.java similarity index 99% rename from core/trino-main/src/main/java/io/trino/version/EmbedVersion.java rename to core/trino-main/src/main/java/io/trino/util/EmbedVersion.java index 07402220f6b37..774a8c672e523 100644 --- a/core/trino-main/src/main/java/io/trino/version/EmbedVersion.java +++ b/core/trino-main/src/main/java/io/trino/util/EmbedVersion.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.version; +package io.trino.util; import com.google.common.collect.ImmutableMap; import com.google.inject.Inject; diff --git a/core/trino-main/src/test/java/io/trino/execution/BaseTestSqlTaskManager.java b/core/trino-main/src/test/java/io/trino/execution/BaseTestSqlTaskManager.java index 3323dcef7baac..cdae4c641db38 100644 --- a/core/trino-main/src/test/java/io/trino/execution/BaseTestSqlTaskManager.java +++ b/core/trino-main/src/test/java/io/trino/execution/BaseTestSqlTaskManager.java @@ -47,7 +47,7 @@ import io.trino.spi.exchange.ExchangeId; import io.trino.spiller.LocalSpillManager; import io.trino.spiller.NodeSpillConfig; -import io.trino.version.EmbedVersion; +import io.trino.util.EmbedVersion; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; diff --git a/core/trino-main/src/test/java/io/trino/execution/TestSqlTaskManagerRaceWithCatalogPrune.java b/core/trino-main/src/test/java/io/trino/execution/TestSqlTaskManagerRaceWithCatalogPrune.java index 7fe6e9e4b997b..81eac9704bb3c 100644 --- a/core/trino-main/src/test/java/io/trino/execution/TestSqlTaskManagerRaceWithCatalogPrune.java +++ b/core/trino-main/src/test/java/io/trino/execution/TestSqlTaskManagerRaceWithCatalogPrune.java @@ -53,7 +53,7 @@ import io.trino.testing.TestingSession; import io.trino.transaction.NoOpTransactionManager; import io.trino.transaction.TransactionInfo; -import io.trino.version.EmbedVersion; +import io.trino.util.EmbedVersion; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.TestInstance; diff --git a/core/trino-main/src/test/java/io/trino/execution/TestSqlTaskManagerThreadPerDriver.java b/core/trino-main/src/test/java/io/trino/execution/TestSqlTaskManagerThreadPerDriver.java index bbec2769cdc57..08327f8c00823 100644 --- a/core/trino-main/src/test/java/io/trino/execution/TestSqlTaskManagerThreadPerDriver.java +++ b/core/trino-main/src/test/java/io/trino/execution/TestSqlTaskManagerThreadPerDriver.java @@ -19,7 +19,7 @@ import io.trino.execution.executor.dedicated.ThreadPerDriverTaskExecutor; import io.trino.execution.executor.scheduler.FairScheduler; -import static io.trino.version.EmbedVersion.testingVersionEmbedder; +import static io.trino.util.EmbedVersion.testingVersionEmbedder; public class TestSqlTaskManagerThreadPerDriver extends BaseTestSqlTaskManager diff --git a/core/trino-main/src/test/java/io/trino/execution/TestTaskExecutorStuckSplits.java b/core/trino-main/src/test/java/io/trino/execution/TestTaskExecutorStuckSplits.java index 6f305b10174b0..379e7afe6b726 100644 --- a/core/trino-main/src/test/java/io/trino/execution/TestTaskExecutorStuckSplits.java +++ b/core/trino-main/src/test/java/io/trino/execution/TestTaskExecutorStuckSplits.java @@ -36,7 +36,7 @@ import io.trino.spi.connector.CatalogHandle; import io.trino.spiller.LocalSpillManager; import io.trino.spiller.NodeSpillConfig; -import io.trino.version.EmbedVersion; +import io.trino.util.EmbedVersion; import org.junit.jupiter.api.Test; import java.util.List; diff --git a/core/trino-main/src/test/java/io/trino/execution/executor/dedicated/TestThreadPerDriverTaskExecutor.java b/core/trino-main/src/test/java/io/trino/execution/executor/dedicated/TestThreadPerDriverTaskExecutor.java index 2648bfbc581b3..ff6318e466010 100644 --- a/core/trino-main/src/test/java/io/trino/execution/executor/dedicated/TestThreadPerDriverTaskExecutor.java +++ b/core/trino-main/src/test/java/io/trino/execution/executor/dedicated/TestThreadPerDriverTaskExecutor.java @@ -39,7 +39,7 @@ import java.util.function.Function; import static io.airlift.tracing.Tracing.noopTracer; -import static io.trino.version.EmbedVersion.testingVersionEmbedder; +import static io.trino.util.EmbedVersion.testingVersionEmbedder; import static java.util.concurrent.TimeUnit.MILLISECONDS; import static org.assertj.core.api.Assertions.assertThat; diff --git a/core/trino-main/src/test/java/io/trino/version/TestEmbedVersion.java b/core/trino-main/src/test/java/io/trino/util/TestEmbedVersion.java similarity index 99% rename from core/trino-main/src/test/java/io/trino/version/TestEmbedVersion.java rename to core/trino-main/src/test/java/io/trino/util/TestEmbedVersion.java index 678ccd9faf6d7..82c8345d46260 100644 --- a/core/trino-main/src/test/java/io/trino/version/TestEmbedVersion.java +++ b/core/trino-main/src/test/java/io/trino/util/TestEmbedVersion.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.version; +package io.trino.util; import org.junit.jupiter.api.Test; From 526e2e2f62a6a00374897917bc4928ce379cdb40 Mon Sep 17 00:00:00 2001 From: David Phillips Date: Sun, 26 Nov 2023 17:01:55 -0600 Subject: [PATCH 070/350] Move IndexManager to operator.index package --- .../main/java/io/trino/{ => operator}/index/IndexManager.java | 2 +- .../src/main/java/io/trino/server/ServerMainModule.java | 2 +- .../main/java/io/trino/sql/planner/LocalExecutionPlanner.java | 2 +- .../src/main/java/io/trino/testing/LocalQueryRunner.java | 2 +- .../src/test/java/io/trino/execution/TaskTestUtils.java | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) rename core/trino-main/src/main/java/io/trino/{ => operator}/index/IndexManager.java (98%) diff --git a/core/trino-main/src/main/java/io/trino/index/IndexManager.java b/core/trino-main/src/main/java/io/trino/operator/index/IndexManager.java similarity index 98% rename from core/trino-main/src/main/java/io/trino/index/IndexManager.java rename to core/trino-main/src/main/java/io/trino/operator/index/IndexManager.java index c204fbe804498..cd8b74ff7714b 100644 --- a/core/trino-main/src/main/java/io/trino/index/IndexManager.java +++ b/core/trino-main/src/main/java/io/trino/operator/index/IndexManager.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.index; +package io.trino.operator.index; import com.google.inject.Inject; import io.trino.Session; diff --git a/core/trino-main/src/main/java/io/trino/server/ServerMainModule.java b/core/trino-main/src/main/java/io/trino/server/ServerMainModule.java index 98b6a2ddb34fd..96c8f407d58a3 100644 --- a/core/trino-main/src/main/java/io/trino/server/ServerMainModule.java +++ b/core/trino-main/src/main/java/io/trino/server/ServerMainModule.java @@ -57,7 +57,6 @@ import io.trino.execution.scheduler.NodeSchedulerConfig; import io.trino.execution.scheduler.TopologyAwareNodeSelectorModule; import io.trino.execution.scheduler.UniformNodeSelectorModule; -import io.trino.index.IndexManager; import io.trino.memory.LocalMemoryManager; import io.trino.memory.LocalMemoryManagerExporter; import io.trino.memory.MemoryInfo; @@ -94,6 +93,7 @@ import io.trino.operator.PagesIndexPageSorter; import io.trino.operator.RetryPolicy; import io.trino.operator.index.IndexJoinLookupStats; +import io.trino.operator.index.IndexManager; import io.trino.operator.scalar.json.JsonExistsFunction; import io.trino.operator.scalar.json.JsonQueryFunction; import io.trino.operator.scalar.json.JsonValueFunction; diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/LocalExecutionPlanner.java b/core/trino-main/src/main/java/io/trino/sql/planner/LocalExecutionPlanner.java index 6de51b87e18f4..f8d56ee3b5865 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/LocalExecutionPlanner.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/LocalExecutionPlanner.java @@ -43,7 +43,6 @@ import io.trino.execution.TaskManagerConfig; import io.trino.execution.buffer.OutputBuffer; import io.trino.execution.buffer.PagesSerdeFactory; -import io.trino.index.IndexManager; import io.trino.metadata.MergeHandle; import io.trino.metadata.Metadata; import io.trino.metadata.ResolvedFunction; @@ -115,6 +114,7 @@ import io.trino.operator.index.IndexBuildDriverFactoryProvider; import io.trino.operator.index.IndexJoinLookupStats; import io.trino.operator.index.IndexLookupSourceFactory; +import io.trino.operator.index.IndexManager; import io.trino.operator.index.IndexSourceOperator; import io.trino.operator.join.HashBuilderOperator.HashBuilderOperatorFactory; import io.trino.operator.join.JoinBridgeManager; diff --git a/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java b/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java index 9b7b2a3e7349f..e922227cca368 100644 --- a/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java +++ b/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java @@ -77,7 +77,6 @@ import io.trino.execution.scheduler.NodeSchedulerConfig; import io.trino.execution.scheduler.UniformNodeSelectorFactory; import io.trino.execution.warnings.WarningCollector; -import io.trino.index.IndexManager; import io.trino.memory.MemoryManagerConfig; import io.trino.memory.NodeMemoryConfig; import io.trino.metadata.AnalyzePropertyManager; @@ -121,6 +120,7 @@ import io.trino.operator.PagesIndexPageSorter; import io.trino.operator.TaskContext; import io.trino.operator.index.IndexJoinLookupStats; +import io.trino.operator.index.IndexManager; import io.trino.operator.scalar.json.JsonExistsFunction; import io.trino.operator.scalar.json.JsonQueryFunction; import io.trino.operator.scalar.json.JsonValueFunction; diff --git a/core/trino-main/src/test/java/io/trino/execution/TaskTestUtils.java b/core/trino-main/src/test/java/io/trino/execution/TaskTestUtils.java index 321a329998211..0bfc83a4b6c74 100644 --- a/core/trino-main/src/test/java/io/trino/execution/TaskTestUtils.java +++ b/core/trino-main/src/test/java/io/trino/execution/TaskTestUtils.java @@ -29,11 +29,11 @@ import io.trino.execution.scheduler.NodeScheduler; import io.trino.execution.scheduler.NodeSchedulerConfig; import io.trino.execution.scheduler.UniformNodeSelectorFactory; -import io.trino.index.IndexManager; import io.trino.metadata.InMemoryNodeManager; import io.trino.metadata.Split; import io.trino.operator.PagesIndex; import io.trino.operator.index.IndexJoinLookupStats; +import io.trino.operator.index.IndexManager; import io.trino.spi.connector.CatalogHandle; import io.trino.spiller.GenericSpillerFactory; import io.trino.split.PageSinkManager; From 5c60383520fa5f36359d85fea99caf50d379f6ec Mon Sep 17 00:00:00 2001 From: David Phillips Date: Tue, 28 Nov 2023 15:40:10 -0800 Subject: [PATCH 071/350] Wrap arguments in DistributedQueryRunner --- .../trino/testing/DistributedQueryRunner.java | 37 +++++++++++++++---- 1 file changed, 30 insertions(+), 7 deletions(-) diff --git a/testing/trino-testing/src/main/java/io/trino/testing/DistributedQueryRunner.java b/testing/trino-testing/src/main/java/io/trino/testing/DistributedQueryRunner.java index 36f485b11153c..f5abc10b1c41f 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/DistributedQueryRunner.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/DistributedQueryRunner.java @@ -55,7 +55,6 @@ import io.trino.sql.parser.SqlParser; import io.trino.sql.planner.NodePartitioningManager; import io.trino.sql.planner.Plan; -import io.trino.sql.tree.Statement; import io.trino.testing.containers.OpenTracingCollector; import io.trino.transaction.TransactionManager; import org.intellij.lang.annotations.Language; @@ -147,7 +146,15 @@ private DistributedQueryRunner( closer.register(() -> extraCloseables.forEach(DistributedQueryRunner::closeUnchecked)); log.info("Created TestingDiscoveryServer in %s", nanosSince(discoveryStart)); - registerNewWorker = () -> createServer(false, extraProperties, environment, additionalModule, baseDataDir, Optional.empty(), Optional.of(ImmutableList.of()), ImmutableList.of()); + registerNewWorker = () -> createServer( + false, + extraProperties, + environment, + additionalModule, + baseDataDir, + Optional.empty(), + Optional.of(ImmutableList.of()), + ImmutableList.of()); int coordinatorCount = backupCoordinatorProperties.isEmpty() ? 1 : 2; checkArgument(nodeCount >= coordinatorCount, "nodeCount includes coordinator(s) count, so must be at least %s, got: %s", coordinatorCount, nodeCount); @@ -166,7 +173,16 @@ private DistributedQueryRunner( extraCoordinatorProperties.put("web-ui.user", "admin"); } - coordinator = createServer(true, extraCoordinatorProperties, environment, additionalModule, baseDataDir, systemAccessControlConfiguration, systemAccessControls, eventListeners); + coordinator = createServer( + true, + extraCoordinatorProperties, + environment, + additionalModule, + baseDataDir, + systemAccessControlConfiguration, + systemAccessControls, + eventListeners); + if (backupCoordinatorProperties.isPresent()) { Map extraBackupCoordinatorProperties = new HashMap<>(); extraBackupCoordinatorProperties.putAll(extraProperties); @@ -195,7 +211,11 @@ private DistributedQueryRunner( } // copy session using property manager in coordinator - defaultSession = defaultSession.toSessionRepresentation().toSession(coordinator.getSessionPropertyManager(), defaultSession.getIdentity().getExtraCredentials(), defaultSession.getExchangeEncryptionKey()); + defaultSession = defaultSession.toSessionRepresentation().toSession( + coordinator.getSessionPropertyManager(), + defaultSession.getIdentity().getExtraCredentials(), + defaultSession.getExchangeEncryptionKey()); + this.trinoClient = closer.register(testingTrinoClientFactory.create(coordinator, defaultSession)); ensureNodesGloballyVisible(); @@ -529,9 +549,12 @@ public Plan createPlan(Session session, String sql) // session must be in a transaction registered with the transaction manager in this query runner getTransactionManager().getTransactionInfo(session.getRequiredTransactionId()); - SqlParser sqlParser = coordinator.getInstance(Key.get(SqlParser.class)); - Statement statement = sqlParser.createStatement(sql); - return coordinator.getQueryExplainer().getLogicalPlan(session, statement, ImmutableList.of(), WarningCollector.NOOP, createPlanOptimizersStatsCollector()); + return coordinator.getQueryExplainer().getLogicalPlan( + session, + coordinator.getInstance(Key.get(SqlParser.class)).createStatement(sql), + ImmutableList.of(), + WarningCollector.NOOP, + createPlanOptimizersStatsCollector()); } public Plan getQueryPlan(QueryId queryId) From 281614d4ef95d51564642b4bff8332385e0e8faa Mon Sep 17 00:00:00 2001 From: David Phillips Date: Tue, 28 Nov 2023 15:47:38 -0800 Subject: [PATCH 072/350] Simplify backup coordinator creation --- .../trino/testing/DistributedQueryRunner.java | 29 ++++++++----------- 1 file changed, 12 insertions(+), 17 deletions(-) diff --git a/testing/trino-testing/src/main/java/io/trino/testing/DistributedQueryRunner.java b/testing/trino-testing/src/main/java/io/trino/testing/DistributedQueryRunner.java index f5abc10b1c41f..c0bcd56090a8a 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/DistributedQueryRunner.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/DistributedQueryRunner.java @@ -183,23 +183,18 @@ private DistributedQueryRunner( systemAccessControls, eventListeners); - if (backupCoordinatorProperties.isPresent()) { - Map extraBackupCoordinatorProperties = new HashMap<>(); - extraBackupCoordinatorProperties.putAll(extraProperties); - extraBackupCoordinatorProperties.putAll(backupCoordinatorProperties.get()); - backupCoordinator = Optional.of(createServer( - true, - extraBackupCoordinatorProperties, - environment, - additionalModule, - baseDataDir, - systemAccessControlConfiguration, - systemAccessControls, - eventListeners)); - } - else { - backupCoordinator = Optional.empty(); - } + backupCoordinator = backupCoordinatorProperties.map(properties -> createServer( + true, + ImmutableMap.builder() + .putAll(extraProperties) + .putAll(properties) + .buildOrThrow(), + environment, + additionalModule, + baseDataDir, + systemAccessControlConfiguration, + systemAccessControls, + eventListeners)); } catch (Exception e) { try { From ee6407e98c8bef9fabd221bd1120617d39065084 Mon Sep 17 00:00:00 2001 From: David Phillips Date: Sat, 2 Dec 2023 16:48:11 -0800 Subject: [PATCH 073/350] Use testing OpenTelemetryExtension --- ...stIcebergGetTableStatisticsOperations.java | 30 ++++----------- .../TestGetTableStatisticsOperations.java | 37 ++++++------------- 2 files changed, 19 insertions(+), 48 deletions(-) diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergGetTableStatisticsOperations.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergGetTableStatisticsOperations.java index b31ca0a442a16..36bdc4ba43ae7 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergGetTableStatisticsOperations.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergGetTableStatisticsOperations.java @@ -14,10 +14,8 @@ package io.trino.plugin.iceberg; import com.google.common.collect.ImmutableMap; -import io.opentelemetry.sdk.testing.exporter.InMemorySpanExporter; -import io.opentelemetry.sdk.trace.SdkTracerProvider; +import io.opentelemetry.sdk.testing.junit5.OpenTelemetryExtension; import io.opentelemetry.sdk.trace.data.SpanData; -import io.opentelemetry.sdk.trace.export.SimpleSpanProcessor; import io.trino.metadata.InternalFunctionBundle; import io.trino.plugin.hive.metastore.Database; import io.trino.plugin.hive.metastore.HiveMetastore; @@ -32,6 +30,7 @@ import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.extension.RegisterExtension; import org.junit.jupiter.api.parallel.Execution; import java.io.IOException; @@ -57,22 +56,18 @@ public class TestIcebergGetTableStatisticsOperations extends AbstractTestQueryFramework { + @RegisterExtension + static final OpenTelemetryExtension TELEMETRY = OpenTelemetryExtension.create(); + private LocalQueryRunner localQueryRunner; - private InMemorySpanExporter spanExporter; private Path metastoreDir; @Override protected QueryRunner createQueryRunner() throws Exception { - spanExporter = closeAfterClass(InMemorySpanExporter.create()); - - SdkTracerProvider tracerProvider = SdkTracerProvider.builder() - .addSpanProcessor(SimpleSpanProcessor.create(spanExporter)) - .build(); - localQueryRunner = LocalQueryRunner.builder(testSessionBuilder().build()) - .withMetadataDecorator(metadata -> new TracingMetadata(tracerProvider.get("test"), metadata)) + .withMetadataDecorator(metadata -> new TracingMetadata(TELEMETRY.getOpenTelemetry().getTracer("test"), metadata)) .build(); localQueryRunner.installPlugin(new TpchPlugin()); localQueryRunner.createCatalog("tpch", "tpch", ImmutableMap.of()); @@ -113,16 +108,9 @@ public void tearDown() localQueryRunner.close(); } - private void resetCounters() - { - spanExporter.reset(); - } - @Test public void testTwoWayJoin() { - resetCounters(); - planDistributedQuery("SELECT * " + "FROM iceberg.tiny.orders o, iceberg.tiny.lineitem l " + "WHERE o.orderkey = l.orderkey"); @@ -132,8 +120,6 @@ public void testTwoWayJoin() @Test public void testThreeWayJoin() { - resetCounters(); - planDistributedQuery("SELECT * " + "FROM iceberg.tiny.customer c, iceberg.tiny.orders o, iceberg.tiny.lineitem l " + "WHERE o.orderkey = l.orderkey AND c.custkey = o.custkey"); @@ -151,9 +137,9 @@ private void planDistributedQuery(@Language("SQL") String sql) createPlanOptimizersStatsCollector())); } - private long getTableStatisticsMethodInvocations() + private static long getTableStatisticsMethodInvocations() { - return spanExporter.getFinishedSpanItems().stream() + return TELEMETRY.getSpans().stream() .map(SpanData::getName) .filter(name -> name.equals("Metadata.getTableStatistics")) .count(); diff --git a/testing/trino-tests/src/test/java/io/trino/tests/TestGetTableStatisticsOperations.java b/testing/trino-tests/src/test/java/io/trino/tests/TestGetTableStatisticsOperations.java index 8864318a11ea1..73591178989e8 100644 --- a/testing/trino-tests/src/test/java/io/trino/tests/TestGetTableStatisticsOperations.java +++ b/testing/trino-tests/src/test/java/io/trino/tests/TestGetTableStatisticsOperations.java @@ -14,21 +14,20 @@ package io.trino.tests; import com.google.common.collect.ImmutableMap; -import io.opentelemetry.sdk.testing.exporter.InMemorySpanExporter; -import io.opentelemetry.sdk.trace.SdkTracerProvider; +import io.opentelemetry.sdk.testing.junit5.OpenTelemetryExtension; import io.opentelemetry.sdk.trace.data.SpanData; -import io.opentelemetry.sdk.trace.export.SimpleSpanProcessor; import io.trino.execution.warnings.WarningCollector; import io.trino.plugin.tpch.TpchPlugin; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.LocalQueryRunner; import io.trino.testing.QueryRunner; -import io.trino.testng.services.ManageTestResources; import io.trino.tracing.TracingMetadata; import org.intellij.lang.annotations.Language; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.parallel.Execution; import static io.trino.execution.querystats.PlanOptimizersStatsCollector.createPlanOptimizersStatsCollector; import static io.trino.sql.planner.LogicalPlanner.Stage.OPTIMIZED_AND_VALIDATED; @@ -36,28 +35,24 @@ import static io.trino.testing.TransactionBuilder.transaction; import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; @TestInstance(PER_CLASS) +@Execution(SAME_THREAD) public class TestGetTableStatisticsOperations extends AbstractTestQueryFramework { - @ManageTestResources.Suppress(because = "Not a TestNG test class") + @RegisterExtension + static final OpenTelemetryExtension TELEMETRY = OpenTelemetryExtension.create(); + private LocalQueryRunner localQueryRunner; - @ManageTestResources.Suppress(because = "Not a TestNG test class") - private InMemorySpanExporter spanExporter; @Override protected QueryRunner createQueryRunner() throws Exception { - spanExporter = closeAfterClass(InMemorySpanExporter.create()); - - SdkTracerProvider tracerProvider = SdkTracerProvider.builder() - .addSpanProcessor(SimpleSpanProcessor.create(spanExporter)) - .build(); - localQueryRunner = LocalQueryRunner.builder(testSessionBuilder().build()) - .withMetadataDecorator(metadata -> new TracingMetadata(tracerProvider.get("test"), metadata)) + .withMetadataDecorator(metadata -> new TracingMetadata(TELEMETRY.getOpenTelemetry().getTracer("test"), metadata)) .build(); localQueryRunner.installPlugin(new TpchPlugin()); localQueryRunner.createCatalog("tpch", "tpch", ImmutableMap.of()); @@ -69,19 +64,11 @@ public void tearDown() { localQueryRunner.close(); localQueryRunner = null; - spanExporter = null; - } - - private void resetCounters() - { - spanExporter.reset(); } @Test public void testTwoWayJoin() { - resetCounters(); - planDistributedQuery("SELECT * " + "FROM tpch.tiny.orders o, tpch.tiny.lineitem l " + "WHERE o.orderkey = l.orderkey"); @@ -91,8 +78,6 @@ public void testTwoWayJoin() @Test public void testThreeWayJoin() { - resetCounters(); - planDistributedQuery("SELECT * " + "FROM tpch.tiny.customer c, tpch.tiny.orders o, tpch.tiny.lineitem l " + "WHERE o.orderkey = l.orderkey AND c.custkey = o.custkey"); @@ -107,9 +92,9 @@ private void planDistributedQuery(@Language("SQL") String sql) }); } - private long getTableStatisticsMethodInvocations() + private static long getTableStatisticsMethodInvocations() { - return spanExporter.getFinishedSpanItems().stream() + return TELEMETRY.getSpans().stream() .map(SpanData::getName) .filter(name -> name.equals("Metadata.getTableStatistics")) .count(); From f163b59e65dab9683d14bb588763eb391e6e36ca Mon Sep 17 00:00:00 2001 From: David Phillips Date: Sat, 2 Dec 2023 16:20:37 -0800 Subject: [PATCH 074/350] Collect spans for queries in DistributedQueryRunner --- core/trino-main/pom.xml | 5 +++++ .../server/testing/TestingTrinoServer.java | 12 ++++++++++ .../trino/testing/DistributedQueryRunner.java | 22 +++++++++++++------ 3 files changed, 32 insertions(+), 7 deletions(-) diff --git a/core/trino-main/pom.xml b/core/trino-main/pom.xml index 73b9cd58314f0..1e0a8050611be 100644 --- a/core/trino-main/pom.xml +++ b/core/trino-main/pom.xml @@ -238,6 +238,11 @@ opentelemetry-context + + io.opentelemetry + opentelemetry-sdk-trace + + io.trino re2j diff --git a/core/trino-main/src/main/java/io/trino/server/testing/TestingTrinoServer.java b/core/trino-main/src/main/java/io/trino/server/testing/TestingTrinoServer.java index 672b7497b3527..d702f91516e39 100644 --- a/core/trino-main/src/main/java/io/trino/server/testing/TestingTrinoServer.java +++ b/core/trino-main/src/main/java/io/trino/server/testing/TestingTrinoServer.java @@ -40,6 +40,7 @@ import io.airlift.openmetrics.JmxOpenMetricsModule; import io.airlift.tracetoken.TraceTokenModule; import io.airlift.tracing.TracingModule; +import io.opentelemetry.sdk.trace.SpanProcessor; import io.trino.connector.CatalogManagerModule; import io.trino.connector.ConnectorName; import io.trino.connector.ConnectorServicesProvider; @@ -128,6 +129,7 @@ import static com.google.common.base.Preconditions.checkState; import static com.google.common.io.MoreFiles.deleteRecursively; import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; +import static com.google.inject.multibindings.Multibinder.newSetBinder; import static com.google.inject.util.Modules.EMPTY_MODULE; import static io.airlift.concurrent.MoreFutures.getFutureValue; import static java.lang.Integer.parseInt; @@ -230,6 +232,7 @@ private TestingTrinoServer( Optional discoveryUri, Module additionalModule, Optional baseDataDir, + Optional spanProcessor, Optional systemAccessControlConfiguration, Optional> systemAccessControls, List eventListeners) @@ -302,6 +305,7 @@ private TestingTrinoServer( binder.bind(GracefulShutdownHandler.class).in(Scopes.SINGLETON); binder.bind(ProcedureTester.class).in(Scopes.SINGLETON); binder.bind(ExchangeManagerRegistry.class).in(Scopes.SINGLETON); + spanProcessor.ifPresent(processor -> newSetBinder(binder, SpanProcessor.class).addBinding().toInstance(processor)); }); if (coordinator) { @@ -716,6 +720,7 @@ public static class Builder private Optional discoveryUri = Optional.empty(); private Module additionalModule = EMPTY_MODULE; private Optional baseDataDir = Optional.empty(); + private Optional spanProcessor = Optional.empty(); private Optional systemAccessControlConfiguration = Optional.empty(); private Optional> systemAccessControls = Optional.of(ImmutableList.of()); private List eventListeners = ImmutableList.of(); @@ -756,6 +761,12 @@ public Builder setBaseDataDir(Optional baseDataDir) return this; } + public Builder setSpanProcessor(Optional spanProcessor) + { + this.spanProcessor = requireNonNull(spanProcessor, "spanProcessor is null"); + return this; + } + public Builder setSystemAccessControlConfiguration(Optional systemAccessControlConfiguration) { this.systemAccessControlConfiguration = requireNonNull(systemAccessControlConfiguration, "systemAccessControlConfiguration is null"); @@ -788,6 +799,7 @@ public TestingTrinoServer build() discoveryUri, additionalModule, baseDataDir, + spanProcessor, systemAccessControlConfiguration, systemAccessControls, eventListeners); diff --git a/testing/trino-testing/src/main/java/io/trino/testing/DistributedQueryRunner.java b/testing/trino-testing/src/main/java/io/trino/testing/DistributedQueryRunner.java index c0bcd56090a8a..5c62cb78a6e10 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/DistributedQueryRunner.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/DistributedQueryRunner.java @@ -22,6 +22,10 @@ import io.airlift.discovery.server.testing.TestingDiscoveryServer; import io.airlift.log.Logger; import io.airlift.log.Logging; +import io.opentelemetry.sdk.testing.exporter.InMemorySpanExporter; +import io.opentelemetry.sdk.trace.SpanProcessor; +import io.opentelemetry.sdk.trace.data.SpanData; +import io.opentelemetry.sdk.trace.export.SimpleSpanProcessor; import io.trino.Session; import io.trino.Session.SessionBuilder; import io.trino.connector.CoordinatorDynamicCatalogManager; @@ -101,6 +105,7 @@ public class DistributedQueryRunner private TestingTrinoServer coordinator; private Optional backupCoordinator; private Runnable registerNewWorker; + private final InMemorySpanExporter spanExporter = InMemorySpanExporter.create(); private final List servers = new CopyOnWriteArrayList<>(); private final List functionBundles = new CopyOnWriteArrayList<>(ImmutableList.of(AbstractTestQueries.CUSTOM_FUNCTIONS)); private final List plugins = new CopyOnWriteArrayList<>(); @@ -234,6 +239,7 @@ private TestingTrinoServer createServer( environment, additionalModule, baseDataDir, + Optional.of(SimpleSpanProcessor.create(spanExporter)), systemAccessControlConfiguration, systemAccessControls, eventListeners)); @@ -262,6 +268,7 @@ private static TestingTrinoServer createTestingTrinoServer( String environment, Module additionalModule, Optional baseDataDir, + Optional spanProcessor, Optional systemAccessControlConfiguration, Optional> systemAccessControls, List eventListeners) @@ -296,6 +303,7 @@ private static TestingTrinoServer createTestingTrinoServer( .setDiscoveryUri(discoveryUri) .setAdditionalModule(additionalModule) .setBaseDataDir(baseDataDir) + .setSpanProcessor(spanProcessor) .setSystemAccessControlConfiguration(systemAccessControlConfiguration) .setSystemAccessControls(systemAccessControls) .setEventListeners(eventListeners) @@ -329,6 +337,11 @@ public TestingTrinoClient getClient() return trinoClient; } + public List getSpans() + { + return spanExporter.getFinishedSpanItems(); + } + @Override public int getNodeCount() { @@ -500,13 +513,7 @@ public boolean tableExists(Session session, String table) @Override public MaterializedResult execute(@Language("SQL") String sql) { - lock.readLock().lock(); - try { - return trinoClient.execute(sql).getResult(); - } - finally { - lock.readLock().unlock(); - } + return execute(getDefaultSession(), sql); } @Override @@ -519,6 +526,7 @@ public MaterializedResultWithQueryId executeWithQueryId(Session session, @Langua { lock.readLock().lock(); try { + spanExporter.reset(); ResultWithQueryId result = trinoClient.execute(session, sql); return new MaterializedResultWithQueryId(result.getQueryId(), result.getResult()); } From fbf7c854921a9a1f3724c3e57515bb37aca1a86c Mon Sep 17 00:00:00 2001 From: David Phillips Date: Wed, 6 Dec 2023 12:34:50 +0800 Subject: [PATCH 075/350] Use static imports for TestingHiveUtils --- .../java/io/trino/plugin/hive/BaseHiveConnectorTest.java | 5 +++-- .../src/test/java/io/trino/plugin/hive/HiveQueryRunner.java | 3 ++- .../java/io/trino/plugin/hive/TestParquetPageSkipping.java | 3 ++- .../trino/plugin/hive/fs/BaseCachingDirectoryListerTest.java | 4 ++-- .../cache/TestCachingHiveMetastoreWithQueryRunner.java | 4 ++-- .../hive/optimizer/TestConnectorPushdownRulesWithHive.java | 4 ++-- .../java/io/trino/plugin/hive/optimizer/TestHivePlans.java | 4 ++-- .../optimizer/TestHiveProjectionPushdownIntoTableScan.java | 4 ++-- .../io/trino/plugin/hive/orc/TestHiveOrcWithShortZoneId.java | 5 +++-- .../hive/parquet/TestReadingTimeLogicalAnnotation.java | 4 ++-- .../io/trino/plugin/iceberg/TestIcebergMetadataListing.java | 4 ++-- 11 files changed, 24 insertions(+), 20 deletions(-) diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java index 337bb255d4029..a977ce6a996c1 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java @@ -150,6 +150,7 @@ import static io.trino.plugin.hive.HiveTableProperties.PARTITIONED_BY_PROPERTY; import static io.trino.plugin.hive.HiveTableProperties.STORAGE_FORMAT_PROPERTY; import static io.trino.plugin.hive.HiveType.toHiveType; +import static io.trino.plugin.hive.TestingHiveUtils.getConnectorService; import static io.trino.plugin.hive.util.HiveUtil.columnExtraInfo; import static io.trino.spi.security.Identity.ofUser; import static io.trino.spi.security.SelectedRole.Type.ROLE; @@ -8673,7 +8674,7 @@ public void testTimestampWithTimeZone() String tableLocation = getTableLocation("test_timestamptz_base"); // TIMESTAMP WITH LOCAL TIME ZONE is not mapped to any Trino type, so we need to create the metastore entry manually - HiveMetastore metastore = TestingHiveUtils.getConnectorService(getDistributedQueryRunner(), HiveMetastoreFactory.class) + HiveMetastore metastore = getConnectorService(getDistributedQueryRunner(), HiveMetastoreFactory.class) .createMetastore(Optional.of(getSession().getIdentity().toConnectorIdentity(catalog))); metastore.createTable( new Table( @@ -9248,7 +9249,7 @@ private String getTableLocation(String tableName) private TrinoFileSystem getTrinoFileSystem() { - return TestingHiveUtils.getConnectorService(getQueryRunner(), TrinoFileSystemFactory.class).create(ConnectorIdentity.ofUser("test")); + return getConnectorService(getQueryRunner(), TrinoFileSystemFactory.class).create(ConnectorIdentity.ofUser("test")); } @Override diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveQueryRunner.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveQueryRunner.java index cbccbdfb70b98..229d3277d0336 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveQueryRunner.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveQueryRunner.java @@ -50,6 +50,7 @@ import static com.google.inject.util.Modules.EMPTY_MODULE; import static io.airlift.log.Level.WARN; import static io.airlift.units.Duration.nanosSince; +import static io.trino.plugin.hive.TestingHiveUtils.getConnectorService; import static io.trino.plugin.hive.security.HiveSecurityModule.ALLOW_ALL; import static io.trino.plugin.hive.security.HiveSecurityModule.SQL_STANDARD; import static io.trino.plugin.tpch.ColumnNaming.SIMPLIFIED; @@ -295,7 +296,7 @@ public DistributedQueryRunner build() private void populateData(DistributedQueryRunner queryRunner) { - HiveMetastore metastore = TestingHiveUtils.getConnectorService(queryRunner, HiveMetastoreFactory.class) + HiveMetastore metastore = getConnectorService(queryRunner, HiveMetastoreFactory.class) .createMetastore(Optional.empty()); if (metastore.getDatabase(TPCH_SCHEMA).isEmpty()) { metastore.createDatabase(createDatabaseMetastoreObject(TPCH_SCHEMA, initialSchemasLocationBase)); diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestParquetPageSkipping.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestParquetPageSkipping.java index 40fc1166921a5..97a89cd27a785 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestParquetPageSkipping.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestParquetPageSkipping.java @@ -41,6 +41,7 @@ import static com.google.common.collect.MoreCollectors.onlyElement; import static io.trino.parquet.reader.ParquetReader.COLUMN_INDEX_ROWS_FILTERED; +import static io.trino.plugin.hive.TestingHiveUtils.getConnectorService; import static io.trino.testing.QueryAssertions.assertEqualsIgnoreOrder; import static io.trino.testing.TestingNames.randomNameSuffix; import static java.lang.String.format; @@ -62,7 +63,7 @@ protected QueryRunner createQueryRunner() "parquet.max-buffer-size", "1MB")) .build(); - fileSystem = TestingHiveUtils.getConnectorService(queryRunner, TrinoFileSystemFactory.class) + fileSystem = getConnectorService(queryRunner, TrinoFileSystemFactory.class) .create(ConnectorIdentity.ofUser("test")); return queryRunner; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/fs/BaseCachingDirectoryListerTest.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/fs/BaseCachingDirectoryListerTest.java index d1f2a84e22e11..7090236230076 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/fs/BaseCachingDirectoryListerTest.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/fs/BaseCachingDirectoryListerTest.java @@ -17,7 +17,6 @@ import com.google.common.collect.ImmutableMap; import io.trino.filesystem.Location; import io.trino.plugin.hive.HiveQueryRunner; -import io.trino.plugin.hive.TestingHiveUtils; import io.trino.plugin.hive.metastore.HiveMetastore; import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.metastore.PrincipalPrivileges; @@ -34,6 +33,7 @@ import java.util.Optional; import static io.trino.plugin.hive.HiveQueryRunner.TPCH_SCHEMA; +import static io.trino.plugin.hive.TestingHiveUtils.getConnectorService; import static java.lang.String.format; import static org.assertj.core.api.Assertions.assertThat; @@ -59,7 +59,7 @@ protected QueryRunner createQueryRunner(Map properties) .setDirectoryLister(directoryLister) .build(); - metastore = TestingHiveUtils.getConnectorService(queryRunner, HiveMetastoreFactory.class) + metastore = getConnectorService(queryRunner, HiveMetastoreFactory.class) .createMetastore(Optional.empty()); return queryRunner; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastoreWithQueryRunner.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastoreWithQueryRunner.java index a76e5c944498d..93b184573fd70 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastoreWithQueryRunner.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastoreWithQueryRunner.java @@ -19,7 +19,6 @@ import com.google.inject.Key; import io.trino.Session; import io.trino.plugin.hive.HiveQueryRunner; -import io.trino.plugin.hive.TestingHiveUtils; import io.trino.plugin.hive.metastore.HiveMetastore; import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.metastore.RawHiveMetastoreFactory; @@ -36,6 +35,7 @@ import java.util.Optional; import static com.google.common.base.Verify.verify; +import static io.trino.plugin.hive.TestingHiveUtils.getConnectorService; import static io.trino.spi.security.SelectedRole.Type.ROLE; import static io.trino.testing.TestingSession.testSessionBuilder; import static java.util.Collections.nCopies; @@ -74,7 +74,7 @@ protected QueryRunner createQueryRunner() "hive.metastore-refresh-interval", "10m")) .build(); - rawMetastore = TestingHiveUtils.getConnectorService(queryRunner, Key.get(HiveMetastoreFactory.class, RawHiveMetastoreFactory.class)) + rawMetastore = getConnectorService(queryRunner, Key.get(HiveMetastoreFactory.class, RawHiveMetastoreFactory.class)) .createMetastore(Optional.empty()); queryRunner.execute(ADMIN, "CREATE SCHEMA " + SCHEMA); diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/optimizer/TestConnectorPushdownRulesWithHive.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/optimizer/TestConnectorPushdownRulesWithHive.java index 8a0a58523560d..444348b5753ac 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/optimizer/TestConnectorPushdownRulesWithHive.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/optimizer/TestConnectorPushdownRulesWithHive.java @@ -24,7 +24,6 @@ import io.trino.plugin.hive.HiveTableHandle; import io.trino.plugin.hive.HiveTransactionHandle; import io.trino.plugin.hive.TestingHiveConnectorFactory; -import io.trino.plugin.hive.TestingHiveUtils; import io.trino.plugin.hive.metastore.Database; import io.trino.plugin.hive.metastore.HiveMetastore; import io.trino.plugin.hive.metastore.HiveMetastoreFactory; @@ -64,6 +63,7 @@ import static io.trino.plugin.hive.HiveQueryRunner.HIVE_CATALOG; import static io.trino.plugin.hive.HiveType.HIVE_INT; import static io.trino.plugin.hive.HiveType.toHiveType; +import static io.trino.plugin.hive.TestingHiveUtils.getConnectorService; import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.spi.type.IntegerType.INTEGER; import static io.trino.spi.type.RowType.field; @@ -108,7 +108,7 @@ protected Optional createLocalQueryRunner() queryRunner.createCatalog(HIVE_CATALOG, new TestingHiveConnectorFactory(baseDir.toPath()), ImmutableMap.of()); catalogHandle = queryRunner.getCatalogHandle(HIVE_CATALOG); - metastore = TestingHiveUtils.getConnectorService(queryRunner, HiveMetastoreFactory.class) + metastore = getConnectorService(queryRunner, HiveMetastoreFactory.class) .createMetastore(Optional.empty()); metastore.createDatabase(Database.builder() diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/optimizer/TestHivePlans.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/optimizer/TestHivePlans.java index b7ea19adbf174..be611290ca3b2 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/optimizer/TestHivePlans.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/optimizer/TestHivePlans.java @@ -15,7 +15,6 @@ import io.trino.Session; import io.trino.plugin.hive.TestingHiveConnectorFactory; -import io.trino.plugin.hive.TestingHiveUtils; import io.trino.plugin.hive.metastore.Database; import io.trino.plugin.hive.metastore.HiveMetastore; import io.trino.plugin.hive.metastore.HiveMetastoreFactory; @@ -40,6 +39,7 @@ import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; import static io.trino.SystemSessionProperties.JOIN_DISTRIBUTION_TYPE; import static io.trino.SystemSessionProperties.JOIN_REORDERING_STRATEGY; +import static io.trino.plugin.hive.TestingHiveUtils.getConnectorService; import static io.trino.sql.planner.assertions.PlanMatchPattern.exchange; import static io.trino.sql.planner.assertions.PlanMatchPattern.filter; import static io.trino.sql.planner.assertions.PlanMatchPattern.join; @@ -78,7 +78,7 @@ protected LocalQueryRunner createLocalQueryRunner() LocalQueryRunner queryRunner = LocalQueryRunner.create(HIVE_SESSION); queryRunner.createCatalog(HIVE_CATALOG_NAME, new TestingHiveConnectorFactory(baseDir.toPath()), Map.of("hive.max-partitions-for-eager-load", "5")); - HiveMetastore metastore = TestingHiveUtils.getConnectorService(queryRunner, HiveMetastoreFactory.class) + HiveMetastore metastore = getConnectorService(queryRunner, HiveMetastoreFactory.class) .createMetastore(Optional.empty()); metastore.createDatabase(Database.builder() diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/optimizer/TestHiveProjectionPushdownIntoTableScan.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/optimizer/TestHiveProjectionPushdownIntoTableScan.java index e32faaa3a1c1c..2acac74db4843 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/optimizer/TestHiveProjectionPushdownIntoTableScan.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/optimizer/TestHiveProjectionPushdownIntoTableScan.java @@ -22,7 +22,6 @@ import io.trino.plugin.hive.HiveColumnHandle; import io.trino.plugin.hive.HiveTableHandle; import io.trino.plugin.hive.TestingHiveConnectorFactory; -import io.trino.plugin.hive.TestingHiveUtils; import io.trino.plugin.hive.metastore.Database; import io.trino.plugin.hive.metastore.HiveMetastore; import io.trino.plugin.hive.metastore.HiveMetastoreFactory; @@ -45,6 +44,7 @@ import static com.google.common.io.MoreFiles.deleteRecursively; import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; import static io.trino.plugin.hive.TestHiveReaderProjectionsUtil.createProjectedColumnHandle; +import static io.trino.plugin.hive.TestingHiveUtils.getConnectorService; import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.sql.planner.assertions.PlanMatchPattern.any; import static io.trino.sql.planner.assertions.PlanMatchPattern.anyTree; @@ -84,7 +84,7 @@ protected LocalQueryRunner createLocalQueryRunner() LocalQueryRunner queryRunner = LocalQueryRunner.create(HIVE_SESSION); queryRunner.createCatalog(HIVE_CATALOG_NAME, new TestingHiveConnectorFactory(baseDir.toPath()), ImmutableMap.of()); - HiveMetastore metastore = TestingHiveUtils.getConnectorService(queryRunner, HiveMetastoreFactory.class) + HiveMetastore metastore = getConnectorService(queryRunner, HiveMetastoreFactory.class) .createMetastore(Optional.empty()); metastore.createDatabase(Database.builder() diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/orc/TestHiveOrcWithShortZoneId.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/orc/TestHiveOrcWithShortZoneId.java index 0d4a88097c054..c231a370e9937 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/orc/TestHiveOrcWithShortZoneId.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/orc/TestHiveOrcWithShortZoneId.java @@ -19,7 +19,6 @@ import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.plugin.hive.HiveQueryRunner; -import io.trino.plugin.hive.TestingHiveUtils; import io.trino.spi.security.ConnectorIdentity; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.DistributedQueryRunner; @@ -31,6 +30,8 @@ import java.net.URL; import java.util.UUID; +import static io.trino.plugin.hive.TestingHiveUtils.getConnectorService; + public class TestHiveOrcWithShortZoneId extends AbstractTestQueryFramework { @@ -46,7 +47,7 @@ protected QueryRunner createQueryRunner() URL resourceLocation = Resources.getResource("with_short_zone_id/data/data.orc"); - TrinoFileSystem fileSystem = TestingHiveUtils.getConnectorService(queryRunner, TrinoFileSystemFactory.class) + TrinoFileSystem fileSystem = getConnectorService(queryRunner, TrinoFileSystemFactory.class) .create(ConnectorIdentity.ofUser("test")); Location tempDir = Location.of("local:///temp_" + UUID.randomUUID()); diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestReadingTimeLogicalAnnotation.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestReadingTimeLogicalAnnotation.java index 2810a7161ec21..2808a6da3de73 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestReadingTimeLogicalAnnotation.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestReadingTimeLogicalAnnotation.java @@ -18,7 +18,6 @@ import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.plugin.hive.HiveQueryRunner; -import io.trino.plugin.hive.TestingHiveUtils; import io.trino.spi.security.ConnectorIdentity; import io.trino.sql.query.QueryAssertions; import io.trino.testing.DistributedQueryRunner; @@ -28,6 +27,7 @@ import java.net.URL; import java.util.UUID; +import static io.trino.plugin.hive.TestingHiveUtils.getConnectorService; import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.testing.MaterializedResult.resultBuilder; import static org.assertj.core.api.Assertions.assertThat; @@ -42,7 +42,7 @@ public void testReadingTimeLogicalAnnotationAsBigInt() QueryAssertions assertions = new QueryAssertions(queryRunner)) { URL resourceLocation = Resources.getResource("parquet_file_with_time_logical_annotation/time-micros.parquet"); - TrinoFileSystem fileSystem = TestingHiveUtils.getConnectorService(queryRunner, TrinoFileSystemFactory.class) + TrinoFileSystem fileSystem = getConnectorService(queryRunner, TrinoFileSystemFactory.class) .create(ConnectorIdentity.ofUser("test")); Location tempDir = Location.of("local:///temp_" + UUID.randomUUID()); diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetadataListing.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetadataListing.java index 8c3831435bde8..317b31e396609 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetadataListing.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetadataListing.java @@ -18,7 +18,6 @@ import io.trino.metadata.MaterializedViewDefinition; import io.trino.metadata.QualifiedObjectName; import io.trino.plugin.hive.TestingHivePlugin; -import io.trino.plugin.hive.TestingHiveUtils; import io.trino.plugin.hive.metastore.HiveMetastore; import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.spi.connector.SchemaTableName; @@ -36,6 +35,7 @@ import java.io.File; import java.util.Optional; +import static io.trino.plugin.hive.TestingHiveUtils.getConnectorService; import static io.trino.spi.security.SelectedRole.Type.ROLE; import static io.trino.testing.TestingSession.testSessionBuilder; import static org.assertj.core.api.Assertions.assertThat; @@ -66,7 +66,7 @@ protected DistributedQueryRunner createQueryRunner() queryRunner.installPlugin(new TestingHivePlugin(baseDir.toPath())); queryRunner.createCatalog("hive", "hive", ImmutableMap.of("hive.security", "sql-standard")); - metastore = TestingHiveUtils.getConnectorService(queryRunner, HiveMetastoreFactory.class) + metastore = getConnectorService(queryRunner, HiveMetastoreFactory.class) .createMetastore(Optional.empty()); return queryRunner; From 4e547de001862a0635223e2d2d7bdc907d5e55c4 Mon Sep 17 00:00:00 2001 From: David Phillips Date: Wed, 6 Dec 2023 12:32:02 +0800 Subject: [PATCH 076/350] Cleanup warnings in HiveQueryRunner --- .../io/trino/plugin/hive/HiveQueryRunner.java | 18 ++++++------------ 1 file changed, 6 insertions(+), 12 deletions(-) diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveQueryRunner.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveQueryRunner.java index 229d3277d0336..90e1fe1e27f20 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveQueryRunner.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveQueryRunner.java @@ -105,7 +105,6 @@ public static class Builder> private ImmutableMap.Builder hiveProperties = ImmutableMap.builder(); private List> initialTables = ImmutableList.of(); private Optional initialSchemasLocationBase = Optional.empty(); - private Function initialTablesSessionMutator = Function.identity(); private Optional> metastore = Optional.empty(); private Optional openTelemetry = Optional.empty(); private Module module = EMPTY_MODULE; @@ -162,13 +161,6 @@ public SELF setInitialSchemasLocationBase(String initialSchemasLocationBase) return self(); } - @CanIgnoreReturnValue - public SELF setInitialTablesSessionMutator(Function initialTablesSessionMutator) - { - this.initialTablesSessionMutator = requireNonNull(initialTablesSessionMutator, "initialTablesSessionMutator is null"); - return self(); - } - @CanIgnoreReturnValue public SELF setMetastore(Function metastore) { @@ -218,6 +210,7 @@ public SELF setCreateTpchSchemas(boolean createTpchSchemas) return self(); } + @SuppressWarnings("unused") @CanIgnoreReturnValue public SELF setTpchColumnNaming(ColumnNaming tpchColumnNaming) { @@ -225,6 +218,7 @@ public SELF setTpchColumnNaming(ColumnNaming tpchColumnNaming) return self(); } + @SuppressWarnings("unused") @CanIgnoreReturnValue public SELF setTpchDecimalTypeMapping(DecimalTypeMapping tpchDecimalTypeMapping) { @@ -294,19 +288,19 @@ public DistributedQueryRunner build() } } - private void populateData(DistributedQueryRunner queryRunner) + private void populateData(QueryRunner queryRunner) { HiveMetastore metastore = getConnectorService(queryRunner, HiveMetastoreFactory.class) .createMetastore(Optional.empty()); if (metastore.getDatabase(TPCH_SCHEMA).isEmpty()) { metastore.createDatabase(createDatabaseMetastoreObject(TPCH_SCHEMA, initialSchemasLocationBase)); - Session session = initialTablesSessionMutator.apply(queryRunner.getDefaultSession()); + Session session = queryRunner.getDefaultSession(); copyTpchTables(queryRunner, "tpch", TINY_SCHEMA_NAME, session, initialTables); } if (tpchBucketedCatalogEnabled && metastore.getDatabase(TPCH_BUCKETED_SCHEMA).isEmpty()) { metastore.createDatabase(createDatabaseMetastoreObject(TPCH_BUCKETED_SCHEMA, initialSchemasLocationBase)); - Session session = initialTablesSessionMutator.apply(createBucketedSession(Optional.empty())); + Session session = createBucketedSession(Optional.empty()); copyTpchTablesBucketed(queryRunner, "tpch", TINY_SCHEMA_NAME, session, initialTables, tpchColumnNaming); } } @@ -410,7 +404,7 @@ public static void main(String[] args) baseDataDir = Optional.of(path); } - DistributedQueryRunner queryRunner = HiveQueryRunner.builder() + DistributedQueryRunner queryRunner = builder() .setExtraProperties(ImmutableMap.of("http-server.http.port", "8080")) .setHiveProperties(ImmutableMap.of("hive.security", ALLOW_ALL)) .setSkipTimezoneSetup(true) From a05b2f7b31f9e1075e129552f6a692579aca8d12 Mon Sep 17 00:00:00 2001 From: David Phillips Date: Wed, 6 Dec 2023 12:58:14 +0800 Subject: [PATCH 077/350] Remove unnecessary config constant --- .../src/main/java/io/trino/plugin/hive/HiveConfig.java | 3 +-- .../test/java/io/trino/plugin/iceberg/TestIcebergPlugin.java | 3 +-- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveConfig.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveConfig.java index 0f54cc3af1ecb..a67a4ca9d6ae3 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveConfig.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveConfig.java @@ -68,7 +68,6 @@ public class HiveConfig public static final String CONFIGURATION_HIVE_PARTITION_PROJECTION_ENABLED = "hive.partition-projection-enabled"; private static final Splitter SPLITTER = Splitter.on(',').trimResults().omitEmptyStrings(); - public static final String HIVE_VIEWS_ENABLED = "hive.hive-views.enabled"; private boolean singleStatementWritesOnly; @@ -785,7 +784,7 @@ public boolean isTranslateHiveViews() } @LegacyConfig({"hive.views-execution.enabled", "hive.translate-hive-views"}) - @Config(HIVE_VIEWS_ENABLED) + @Config("hive.hive-views.enabled") @ConfigDescription("Experimental: Allow translation of Hive views into Trino views") public HiveConfig setTranslateHiveViews(boolean translateHiveViews) { diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergPlugin.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergPlugin.java index 775dad7252973..0f055ebed57d7 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergPlugin.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergPlugin.java @@ -25,7 +25,6 @@ import java.util.Map; import static com.google.common.collect.Iterables.getOnlyElement; -import static io.trino.plugin.hive.HiveConfig.HIVE_VIEWS_ENABLED; import static org.assertj.core.api.Assertions.assertThatThrownBy; public class TestIcebergPlugin @@ -214,7 +213,7 @@ public void testIcebergPluginFailsWhenIncorrectPropertyProvided() "test", Map.of( "iceberg.catalog.type", "HIVE_METASTORE", - HIVE_VIEWS_ENABLED, "true", + "hive.hive-views.enabled", "true", "hive.metastore.uri", "thrift://foo:1234", "bootstrap.quiet", "true"), new TestingConnectorContext()) From f5ba0252c7bb2cd2371a089471bc388ae105e033 Mon Sep 17 00:00:00 2001 From: David Phillips Date: Sat, 2 Dec 2023 16:22:57 -0800 Subject: [PATCH 078/350] Add tracing wrapper in metastore factories --- .../file/FileHiveMetastoreFactory.java | 14 +++++++++++--- .../glue/GlueHiveMetastoreFactory.java | 19 +++++-------------- .../metastore/glue/GlueMetastoreModule.java | 2 +- .../thrift/BridgingHiveMetastoreFactory.java | 9 +++++++-- 4 files changed, 24 insertions(+), 20 deletions(-) diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/file/FileHiveMetastoreFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/file/FileHiveMetastoreFactory.java index 6c74be568e154..27a028706aa21 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/file/FileHiveMetastoreFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/file/FileHiveMetastoreFactory.java @@ -14,11 +14,13 @@ package io.trino.plugin.hive.metastore.file; import com.google.inject.Inject; +import io.opentelemetry.api.trace.Tracer; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.plugin.hive.HideDeltaLakeTables; import io.trino.plugin.hive.NodeVersion; import io.trino.plugin.hive.metastore.HiveMetastore; import io.trino.plugin.hive.metastore.HiveMetastoreFactory; +import io.trino.plugin.hive.metastore.tracing.TracingHiveMetastore; import io.trino.spi.security.ConnectorIdentity; import java.util.Optional; @@ -26,13 +28,19 @@ public class FileHiveMetastoreFactory implements HiveMetastoreFactory { - private final FileHiveMetastore metastore; + private final HiveMetastore metastore; @Inject - public FileHiveMetastoreFactory(NodeVersion nodeVersion, TrinoFileSystemFactory fileSystemFactory, @HideDeltaLakeTables boolean hideDeltaLakeTables, FileHiveMetastoreConfig config) + public FileHiveMetastoreFactory( + NodeVersion nodeVersion, + TrinoFileSystemFactory fileSystemFactory, + @HideDeltaLakeTables boolean hideDeltaLakeTables, + FileHiveMetastoreConfig config, + Tracer tracer) { // file metastore does not support impersonation, so just create a single shared instance - metastore = new FileHiveMetastore(nodeVersion, fileSystemFactory, hideDeltaLakeTables, config); + metastore = new TracingHiveMetastore(tracer, + new FileHiveMetastore(nodeVersion, fileSystemFactory, hideDeltaLakeTables, config)); } @Override diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueHiveMetastoreFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueHiveMetastoreFactory.java index 9fb979be961a7..6e37d604852dc 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueHiveMetastoreFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueHiveMetastoreFactory.java @@ -14,33 +14,24 @@ package io.trino.plugin.hive.metastore.glue; import com.google.inject.Inject; +import io.opentelemetry.api.trace.Tracer; import io.trino.plugin.hive.metastore.HiveMetastore; import io.trino.plugin.hive.metastore.HiveMetastoreFactory; +import io.trino.plugin.hive.metastore.tracing.TracingHiveMetastore; import io.trino.spi.security.ConnectorIdentity; -import org.weakref.jmx.Flatten; -import org.weakref.jmx.Managed; import java.util.Optional; -import static java.util.Objects.requireNonNull; - public class GlueHiveMetastoreFactory implements HiveMetastoreFactory { - private final GlueHiveMetastore metastore; + private final HiveMetastore metastore; // Glue metastore does not support impersonation, so just use single shared instance @Inject - public GlueHiveMetastoreFactory(GlueHiveMetastore metastore) + public GlueHiveMetastoreFactory(GlueHiveMetastore metastore, Tracer tracer) { - this.metastore = requireNonNull(metastore, "metastore is null"); - } - - @Flatten - @Managed - public GlueHiveMetastore getMetastore() - { - return metastore; + this.metastore = new TracingHiveMetastore(tracer, metastore); } @Override diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueMetastoreModule.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueMetastoreModule.java index ea4e4cb1e6da2..cb40ea0b3be18 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueMetastoreModule.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueMetastoreModule.java @@ -69,7 +69,7 @@ protected void setup(Binder binder) binder.bind(GlueHiveMetastoreFactory.class).in(Scopes.SINGLETON); binder.bind(Key.get(GlueMetastoreStats.class, ForGlueHiveMetastore.class)).toInstance(new GlueMetastoreStats()); binder.bind(AWSGlueAsync.class).toProvider(HiveGlueClientProvider.class).in(Scopes.SINGLETON); - newExporter(binder).export(GlueHiveMetastoreFactory.class).as(generator -> generator.generatedNameOf(GlueHiveMetastore.class)); + newExporter(binder).export(GlueHiveMetastore.class).withGeneratedName(); binder.bind(Key.get(boolean.class, AllowHiveTableRename.class)).toInstance(false); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/BridgingHiveMetastoreFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/BridgingHiveMetastoreFactory.java index 140abcd519e94..1dc2ff2c5a51a 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/BridgingHiveMetastoreFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/BridgingHiveMetastoreFactory.java @@ -14,8 +14,10 @@ package io.trino.plugin.hive.metastore.thrift; import com.google.inject.Inject; +import io.opentelemetry.api.trace.Tracer; import io.trino.plugin.hive.metastore.HiveMetastore; import io.trino.plugin.hive.metastore.HiveMetastoreFactory; +import io.trino.plugin.hive.metastore.tracing.TracingHiveMetastore; import io.trino.spi.security.ConnectorIdentity; import java.util.Optional; @@ -26,11 +28,13 @@ public class BridgingHiveMetastoreFactory implements HiveMetastoreFactory { private final ThriftMetastoreFactory thriftMetastoreFactory; + private final Tracer tracer; @Inject - public BridgingHiveMetastoreFactory(ThriftMetastoreFactory thriftMetastoreFactory) + public BridgingHiveMetastoreFactory(ThriftMetastoreFactory thriftMetastoreFactory, Tracer tracer) { this.thriftMetastoreFactory = requireNonNull(thriftMetastoreFactory, "thriftMetastore is null"); + this.tracer = requireNonNull(tracer, "tracer is null"); } @Override @@ -42,6 +46,7 @@ public boolean isImpersonationEnabled() @Override public HiveMetastore createMetastore(Optional identity) { - return new BridgingHiveMetastore(thriftMetastoreFactory.createMetastore(identity)); + return new TracingHiveMetastore(tracer, + new BridgingHiveMetastore(thriftMetastoreFactory.createMetastore(identity))); } } From 72953c2f63209181b74dd1e614b17b9071db6c83 Mon Sep 17 00:00:00 2001 From: David Phillips Date: Mon, 27 Nov 2023 17:19:25 -0800 Subject: [PATCH 079/350] Move metastore method enum to top level --- ...DeltaLakePerTransactionMetastoreCache.java | 2 +- ...estDeltaLakeMetastoreAccessOperations.java | 12 +-- .../CountingAccessHiveMetastore.java | 96 +++++++++---------- .../hive/metastore/MetastoreMethod.java | 41 ++++++++ .../TestHiveMetastoreAccessOperations.java | 18 ++-- ...astoreMetadataQueriesAccessOperations.java | 24 ++--- .../TestIcebergMetastoreAccessOperations.java | 14 +-- 7 files changed, 120 insertions(+), 87 deletions(-) create mode 100644 plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/MetastoreMethod.java diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakePerTransactionMetastoreCache.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakePerTransactionMetastoreCache.java index de8ba048e2a40..cd4e58f6b9f37 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakePerTransactionMetastoreCache.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakePerTransactionMetastoreCache.java @@ -36,7 +36,7 @@ import java.util.Optional; import static io.trino.plugin.deltalake.DeltaLakeQueryRunner.DELTA_CATALOG; -import static io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Method.GET_TABLE; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_TABLE; import static io.trino.plugin.hive.metastore.file.TestingFileHiveMetastore.createTestingFileHiveMetastore; import static io.trino.testing.TestingSession.testSessionBuilder; import static java.util.Objects.requireNonNull; diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/TestDeltaLakeMetastoreAccessOperations.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/TestDeltaLakeMetastoreAccessOperations.java index b29b715bd254b..812fdf94ed671 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/TestDeltaLakeMetastoreAccessOperations.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/TestDeltaLakeMetastoreAccessOperations.java @@ -35,12 +35,12 @@ import java.nio.file.Path; import java.util.Optional; -import static io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Method.CREATE_TABLE; -import static io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Method.DROP_TABLE; -import static io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Method.GET_ALL_DATABASES; -import static io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Method.GET_ALL_TABLES_FROM_DATABASE; -import static io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Method.GET_DATABASE; -import static io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Method.GET_TABLE; +import static io.trino.plugin.hive.metastore.MetastoreMethod.CREATE_TABLE; +import static io.trino.plugin.hive.metastore.MetastoreMethod.DROP_TABLE; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_ALL_DATABASES; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_ALL_TABLES_FROM_DATABASE; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_DATABASE; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_TABLE; import static io.trino.plugin.hive.metastore.file.TestingFileHiveMetastore.createTestingFileHiveMetastore; import static io.trino.testing.TestingSession.testSessionBuilder; import static java.util.Objects.requireNonNull; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/CountingAccessHiveMetastore.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/CountingAccessHiveMetastore.java index a1737c3ad01f1..08cf3c7834b66 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/CountingAccessHiveMetastore.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/CountingAccessHiveMetastore.java @@ -36,51 +36,43 @@ import java.util.Set; import java.util.function.Function; -import static io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Method.GET_ALL_RELATION_TYPES; -import static io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Method.GET_ALL_TABLES; -import static io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Method.GET_ALL_VIEWS; -import static io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Method.GET_RELATION_TYPES_FROM_DATABASE; +import static io.trino.plugin.hive.metastore.MetastoreMethod.ADD_PARTITIONS; +import static io.trino.plugin.hive.metastore.MetastoreMethod.CREATE_DATABASE; +import static io.trino.plugin.hive.metastore.MetastoreMethod.CREATE_TABLE; +import static io.trino.plugin.hive.metastore.MetastoreMethod.DROP_DATABASE; +import static io.trino.plugin.hive.metastore.MetastoreMethod.DROP_TABLE; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_ALL_DATABASES; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_ALL_RELATION_TYPES; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_ALL_TABLES; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_ALL_TABLES_FROM_DATABASE; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_ALL_VIEWS; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_ALL_VIEWS_FROM_DATABASE; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_DATABASE; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_PARTITION; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_PARTITIONS_BY_NAMES; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_PARTITION_NAMES_BY_FILTER; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_PARTITION_STATISTICS; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_RELATION_TYPES_FROM_DATABASE; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_TABLE; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_TABLES_WITH_PARAMETER; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_TABLE_STATISTICS; +import static io.trino.plugin.hive.metastore.MetastoreMethod.REPLACE_TABLE; +import static io.trino.plugin.hive.metastore.MetastoreMethod.UPDATE_PARTITION_STATISTICS; +import static io.trino.plugin.hive.metastore.MetastoreMethod.UPDATE_TABLE_STATISTICS; @ThreadSafe public class CountingAccessHiveMetastore implements HiveMetastore { - public enum Method - { - CREATE_DATABASE, - DROP_DATABASE, - CREATE_TABLE, - GET_ALL_DATABASES, - GET_DATABASE, - GET_TABLE, - GET_ALL_TABLES, - GET_ALL_TABLES_FROM_DATABASE, - GET_RELATION_TYPES_FROM_DATABASE, - GET_ALL_RELATION_TYPES, - GET_TABLES_WITH_PARAMETER, - GET_TABLE_STATISTICS, - GET_ALL_VIEWS, - GET_ALL_VIEWS_FROM_DATABASE, - UPDATE_TABLE_STATISTICS, - ADD_PARTITIONS, - GET_PARTITION_NAMES_BY_FILTER, - GET_PARTITIONS_BY_NAMES, - GET_PARTITION, - GET_PARTITION_STATISTICS, - UPDATE_PARTITION_STATISTICS, - REPLACE_TABLE, - DROP_TABLE, - } - private final HiveMetastore delegate; - private final ConcurrentHashMultiset methodInvocations = ConcurrentHashMultiset.create(); + private final ConcurrentHashMultiset methodInvocations = ConcurrentHashMultiset.create(); public CountingAccessHiveMetastore(HiveMetastore delegate) { this.delegate = delegate; } - public Multiset getMethodInvocations() + public Multiset getMethodInvocations() { return ImmutableMultiset.copyOf(methodInvocations); } @@ -93,7 +85,7 @@ public void resetCounters() @Override public Optional
getTable(String databaseName, String tableName) { - methodInvocations.add(Method.GET_TABLE); + methodInvocations.add(GET_TABLE); return delegate.getTable(databaseName, tableName); } @@ -107,28 +99,28 @@ public Set getSupportedColumnStatistics(Type type) @Override public List getAllDatabases() { - methodInvocations.add(Method.GET_ALL_DATABASES); + methodInvocations.add(GET_ALL_DATABASES); return delegate.getAllDatabases(); } @Override public Optional getDatabase(String databaseName) { - methodInvocations.add(Method.GET_DATABASE); + methodInvocations.add(GET_DATABASE); return delegate.getDatabase(databaseName); } @Override public List getTablesWithParameter(String databaseName, String parameterKey, String parameterValue) { - methodInvocations.add(Method.GET_TABLES_WITH_PARAMETER); + methodInvocations.add(GET_TABLES_WITH_PARAMETER); return delegate.getTablesWithParameter(databaseName, parameterKey, parameterValue); } @Override public List getAllViews(String databaseName) { - methodInvocations.add(Method.GET_ALL_VIEWS_FROM_DATABASE); + methodInvocations.add(GET_ALL_VIEWS_FROM_DATABASE); return delegate.getAllViews(databaseName); } @@ -145,14 +137,14 @@ public Optional> getAllViews() @Override public void createDatabase(Database database) { - methodInvocations.add(Method.CREATE_DATABASE); + methodInvocations.add(CREATE_DATABASE); delegate.createDatabase(database); } @Override public void dropDatabase(String databaseName, boolean deleteData) { - methodInvocations.add(Method.DROP_DATABASE); + methodInvocations.add(DROP_DATABASE); delegate.dropDatabase(databaseName, deleteData); } @@ -171,21 +163,21 @@ public void setDatabaseOwner(String databaseName, HivePrincipal principal) @Override public void createTable(Table table, PrincipalPrivileges principalPrivileges) { - methodInvocations.add(Method.CREATE_TABLE); + methodInvocations.add(CREATE_TABLE); delegate.createTable(table, principalPrivileges); } @Override public void dropTable(String databaseName, String tableName, boolean deleteData) { - methodInvocations.add(Method.DROP_TABLE); + methodInvocations.add(DROP_TABLE); delegate.dropTable(databaseName, tableName, deleteData); } @Override public void replaceTable(String databaseName, String tableName, Table newTable, PrincipalPrivileges principalPrivileges) { - methodInvocations.add(Method.REPLACE_TABLE); + methodInvocations.add(REPLACE_TABLE); delegate.replaceTable(databaseName, tableName, newTable, principalPrivileges); } @@ -234,7 +226,7 @@ public void dropColumn(String databaseName, String tableName, String columnName) @Override public Optional getPartition(Table table, List partitionValues) { - methodInvocations.add(Method.GET_PARTITION); + methodInvocations.add(GET_PARTITION); return delegate.getPartition(table, partitionValues); } @@ -244,21 +236,21 @@ public Optional> getPartitionNamesByFilter(String databaseName, List columnNames, TupleDomain partitionKeysFilter) { - methodInvocations.add(Method.GET_PARTITION_NAMES_BY_FILTER); + methodInvocations.add(GET_PARTITION_NAMES_BY_FILTER); return delegate.getPartitionNamesByFilter(databaseName, tableName, columnNames, partitionKeysFilter); } @Override public Map> getPartitionsByNames(Table table, List partitionNames) { - methodInvocations.add(Method.GET_PARTITIONS_BY_NAMES); + methodInvocations.add(GET_PARTITIONS_BY_NAMES); return delegate.getPartitionsByNames(table, partitionNames); } @Override public void addPartitions(String databaseName, String tableName, List partitions) { - methodInvocations.add(Method.ADD_PARTITIONS); + methodInvocations.add(ADD_PARTITIONS); delegate.addPartitions(databaseName, tableName, partitions); } @@ -331,14 +323,14 @@ public Set listTablePrivileges(String databaseName, String ta @Override public PartitionStatistics getTableStatistics(Table table) { - methodInvocations.add(Method.GET_TABLE_STATISTICS); + methodInvocations.add(GET_TABLE_STATISTICS); return delegate.getTableStatistics(table); } @Override public Map getPartitionStatistics(Table table, List partitions) { - methodInvocations.add(Method.GET_PARTITION_STATISTICS); + methodInvocations.add(GET_PARTITION_STATISTICS); return delegate.getPartitionStatistics(table, partitions); } @@ -348,21 +340,21 @@ public void updateTableStatistics(String databaseName, AcidTransaction transaction, Function update) { - methodInvocations.add(Method.UPDATE_TABLE_STATISTICS); + methodInvocations.add(UPDATE_TABLE_STATISTICS); delegate.updateTableStatistics(databaseName, tableName, transaction, update); } @Override public void updatePartitionStatistics(Table table, Map> updates) { - methodInvocations.add(Method.UPDATE_PARTITION_STATISTICS); + methodInvocations.add(UPDATE_PARTITION_STATISTICS); delegate.updatePartitionStatistics(table, updates); } @Override public List getAllTables(String databaseName) { - methodInvocations.add(Method.GET_ALL_TABLES_FROM_DATABASE); + methodInvocations.add(GET_ALL_TABLES_FROM_DATABASE); return delegate.getAllTables(databaseName); } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/MetastoreMethod.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/MetastoreMethod.java new file mode 100644 index 0000000000000..fb7ce0e7db18e --- /dev/null +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/MetastoreMethod.java @@ -0,0 +1,41 @@ +/* + * 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.hive.metastore; + +public enum MetastoreMethod +{ + CREATE_DATABASE, + DROP_DATABASE, + CREATE_TABLE, + GET_ALL_DATABASES, + GET_DATABASE, + GET_TABLE, + GET_ALL_TABLES, + GET_ALL_TABLES_FROM_DATABASE, + GET_RELATION_TYPES_FROM_DATABASE, + GET_ALL_RELATION_TYPES, + GET_TABLES_WITH_PARAMETER, + GET_TABLE_STATISTICS, + GET_ALL_VIEWS, + GET_ALL_VIEWS_FROM_DATABASE, + UPDATE_TABLE_STATISTICS, + ADD_PARTITIONS, + GET_PARTITION_NAMES_BY_FILTER, + GET_PARTITIONS_BY_NAMES, + GET_PARTITION, + GET_PARTITION_STATISTICS, + UPDATE_PARTITION_STATISTICS, + REPLACE_TABLE, + DROP_TABLE, +} diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreAccessOperations.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreAccessOperations.java index b24a9f15e267b..03b6c746e87a0 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreAccessOperations.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreAccessOperations.java @@ -29,15 +29,15 @@ import java.io.File; -import static io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Method.CREATE_TABLE; -import static io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Method.GET_DATABASE; -import static io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Method.GET_PARTITIONS_BY_NAMES; -import static io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Method.GET_PARTITION_NAMES_BY_FILTER; -import static io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Method.GET_PARTITION_STATISTICS; -import static io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Method.GET_TABLE; -import static io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Method.GET_TABLE_STATISTICS; -import static io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Method.UPDATE_PARTITION_STATISTICS; -import static io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Method.UPDATE_TABLE_STATISTICS; +import static io.trino.plugin.hive.metastore.MetastoreMethod.CREATE_TABLE; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_DATABASE; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_PARTITIONS_BY_NAMES; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_PARTITION_NAMES_BY_FILTER; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_PARTITION_STATISTICS; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_TABLE; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_TABLE_STATISTICS; +import static io.trino.plugin.hive.metastore.MetastoreMethod.UPDATE_PARTITION_STATISTICS; +import static io.trino.plugin.hive.metastore.MetastoreMethod.UPDATE_TABLE_STATISTICS; import static io.trino.plugin.hive.metastore.file.TestingFileHiveMetastore.createTestingFileHiveMetastore; import static io.trino.testing.TestingSession.testSessionBuilder; import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreMetadataQueriesAccessOperations.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreMetadataQueriesAccessOperations.java index 4c73ecf68734e..79d2670a87514 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreMetadataQueriesAccessOperations.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreMetadataQueriesAccessOperations.java @@ -23,10 +23,10 @@ import io.trino.plugin.hive.containers.HiveHadoop; import io.trino.plugin.hive.metastore.Column; import io.trino.plugin.hive.metastore.CountingAccessHiveMetastore; -import io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Method; import io.trino.plugin.hive.metastore.CountingAccessHiveMetastoreUtil; import io.trino.plugin.hive.metastore.Database; import io.trino.plugin.hive.metastore.HiveMetastore; +import io.trino.plugin.hive.metastore.MetastoreMethod; import io.trino.plugin.hive.metastore.Table; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.DistributedQueryRunner; @@ -42,14 +42,14 @@ import static io.trino.plugin.hive.HiveStorageFormat.PARQUET; import static io.trino.plugin.hive.TableType.MANAGED_TABLE; import static io.trino.plugin.hive.TestingThriftHiveMetastoreBuilder.testingThriftHiveMetastoreBuilder; -import static io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Method.GET_ALL_DATABASES; -import static io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Method.GET_ALL_RELATION_TYPES; -import static io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Method.GET_ALL_TABLES; -import static io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Method.GET_ALL_TABLES_FROM_DATABASE; -import static io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Method.GET_ALL_VIEWS; -import static io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Method.GET_ALL_VIEWS_FROM_DATABASE; -import static io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Method.GET_RELATION_TYPES_FROM_DATABASE; -import static io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Method.GET_TABLE; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_ALL_DATABASES; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_ALL_RELATION_TYPES; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_ALL_TABLES; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_ALL_TABLES_FROM_DATABASE; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_ALL_VIEWS; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_ALL_VIEWS_FROM_DATABASE; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_RELATION_TYPES_FROM_DATABASE; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_TABLE; import static io.trino.plugin.hive.metastore.PrincipalPrivileges.NO_PRIVILEGES; import static io.trino.plugin.hive.metastore.StorageFormat.fromHiveStorageFormat; import static io.trino.testing.TestingSession.testSessionBuilder; @@ -151,7 +151,7 @@ public void testSelectSchemasWithLikeOverSchemaName() @Test public void testSelectTablesWithoutPredicate() { - Multiset tables = ImmutableMultiset.builder() + Multiset tables = ImmutableMultiset.builder() .add(GET_ALL_RELATION_TYPES) .build(); assertMetastoreInvocations("SELECT * FROM information_schema.tables", tables); @@ -206,7 +206,7 @@ public void testSelectTablesWithFilterByTableName() .add(GET_ALL_RELATION_TYPES) .build()); - Multiset tables = ImmutableMultiset.builder() + Multiset tables = ImmutableMultiset.builder() .add(GET_ALL_RELATION_TYPES) .build(); assertMetastoreInvocations("SELECT * FROM system.jdbc.tables WHERE table_name = 'test_table_0'", tables); @@ -309,7 +309,7 @@ public void testSelectViewsWithLikeOverTableName() @Test public void testSelectColumnsWithoutPredicate() { - ImmutableMultiset tables = ImmutableMultiset.builder() + ImmutableMultiset tables = ImmutableMultiset.builder() .add(GET_ALL_TABLES) .add(GET_ALL_VIEWS) .addCopies(GET_TABLE, TEST_ALL_TABLES_COUNT) diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetastoreAccessOperations.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetastoreAccessOperations.java index a19861c2f78b6..990794a5371a4 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetastoreAccessOperations.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetastoreAccessOperations.java @@ -32,13 +32,13 @@ import java.util.Optional; import static com.google.inject.util.Modules.EMPTY_MODULE; -import static io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Method.CREATE_TABLE; -import static io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Method.DROP_TABLE; -import static io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Method.GET_ALL_TABLES_FROM_DATABASE; -import static io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Method.GET_DATABASE; -import static io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Method.GET_TABLE; -import static io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Method.GET_TABLES_WITH_PARAMETER; -import static io.trino.plugin.hive.metastore.CountingAccessHiveMetastore.Method.REPLACE_TABLE; +import static io.trino.plugin.hive.metastore.MetastoreMethod.CREATE_TABLE; +import static io.trino.plugin.hive.metastore.MetastoreMethod.DROP_TABLE; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_ALL_TABLES_FROM_DATABASE; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_DATABASE; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_TABLE; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_TABLES_WITH_PARAMETER; +import static io.trino.plugin.hive.metastore.MetastoreMethod.REPLACE_TABLE; import static io.trino.plugin.hive.metastore.file.TestingFileHiveMetastore.createTestingFileHiveMetastore; import static io.trino.plugin.iceberg.IcebergSessionProperties.COLLECT_EXTENDED_STATISTICS_ON_WRITE; import static io.trino.plugin.iceberg.TableType.DATA; From 0ab49b4385410bb17373b9bea3777229ca76ea18 Mon Sep 17 00:00:00 2001 From: David Phillips Date: Mon, 27 Nov 2023 17:29:12 -0800 Subject: [PATCH 080/350] Fix type safety for counting metastore tests --- ...DeltaLakePerTransactionMetastoreCache.java | 7 +- ...estDeltaLakeMetastoreAccessOperations.java | 29 +++---- .../CountingAccessHiveMetastoreUtil.java | 2 +- .../TestHiveMetastoreAccessOperations.java | 47 ++++++----- ...astoreMetadataQueriesAccessOperations.java | 66 +++++++-------- .../TestIcebergMetastoreAccessOperations.java | 83 ++++++++++--------- 6 files changed, 119 insertions(+), 115 deletions(-) diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakePerTransactionMetastoreCache.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakePerTransactionMetastoreCache.java index cd4e58f6b9f37..3dffcbc7d2802 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakePerTransactionMetastoreCache.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakePerTransactionMetastoreCache.java @@ -24,6 +24,7 @@ import io.trino.plugin.hive.metastore.CountingAccessHiveMetastore; import io.trino.plugin.hive.metastore.CountingAccessHiveMetastoreUtil; import io.trino.plugin.hive.metastore.HiveMetastoreFactory; +import io.trino.plugin.hive.metastore.MetastoreMethod; import io.trino.plugin.hive.metastore.RawHiveMetastoreFactory; import io.trino.plugin.hive.metastore.file.FileHiveMetastore; import io.trino.plugin.tpch.TpchPlugin; @@ -110,7 +111,7 @@ public void testPerTransactionHiveMetastoreCachingEnabled() try (DistributedQueryRunner queryRunner = createQueryRunner(true)) { // Verify cache works; we expect only two calls to `getTable` because we have two tables in a query. assertMetastoreInvocations(queryRunner, "SELECT * FROM nation JOIN region ON nation.regionkey = region.regionkey", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .addCopies(GET_TABLE, 2) .build()); } @@ -122,13 +123,13 @@ public void testPerTransactionHiveMetastoreCachingDisabled() { try (DistributedQueryRunner queryRunner = createQueryRunner(false)) { assertMetastoreInvocations(queryRunner, "SELECT * FROM nation JOIN region ON nation.regionkey = region.regionkey", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .addCopies(GET_TABLE, 2) .build()); } } - private void assertMetastoreInvocations(QueryRunner queryRunner, @Language("SQL") String query, Multiset expectedInvocations) + private void assertMetastoreInvocations(QueryRunner queryRunner, @Language("SQL") String query, Multiset expectedInvocations) { CountingAccessHiveMetastoreUtil.assertMetastoreInvocations(metastore, queryRunner, queryRunner.getDefaultSession(), query, expectedInvocations); } diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/TestDeltaLakeMetastoreAccessOperations.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/TestDeltaLakeMetastoreAccessOperations.java index 812fdf94ed671..d739bba8baf9c 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/TestDeltaLakeMetastoreAccessOperations.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/TestDeltaLakeMetastoreAccessOperations.java @@ -25,6 +25,7 @@ import io.trino.plugin.hive.metastore.CountingAccessHiveMetastore; import io.trino.plugin.hive.metastore.CountingAccessHiveMetastoreUtil; import io.trino.plugin.hive.metastore.HiveMetastoreFactory; +import io.trino.plugin.hive.metastore.MetastoreMethod; import io.trino.plugin.hive.metastore.RawHiveMetastoreFactory; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.DistributedQueryRunner; @@ -97,7 +98,7 @@ protected void setup(Binder binder) public void testCreateTable() { assertMetastoreInvocations("CREATE TABLE test_create (id VARCHAR, age INT)", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(CREATE_TABLE) .add(GET_DATABASE) .add(GET_TABLE) @@ -108,7 +109,7 @@ public void testCreateTable() public void testCreateTableAsSelect() { assertMetastoreInvocations("CREATE TABLE test_ctas AS SELECT 1 AS age", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_DATABASE) .add(CREATE_TABLE) .add(GET_TABLE) @@ -121,7 +122,7 @@ public void testSelect() assertUpdate("CREATE TABLE test_select_from (id VARCHAR, age INT)"); assertMetastoreInvocations("SELECT * FROM test_select_from", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_TABLE) .build()); } @@ -132,7 +133,7 @@ public void testSelectWithFilter() assertUpdate("CREATE TABLE test_select_from_where AS SELECT 2 as age", 1); assertMetastoreInvocations("SELECT * FROM test_select_from_where WHERE age = 2", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_TABLE) .build()); } @@ -144,7 +145,7 @@ public void testSelectFromView() assertUpdate("CREATE VIEW test_select_view_view AS SELECT id, age FROM test_select_view_table"); assertMetastoreInvocations("SELECT * FROM test_select_view_view", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .addCopies(GET_TABLE, 2) .build()); } @@ -156,7 +157,7 @@ public void testSelectFromViewWithFilter() assertUpdate("CREATE VIEW test_select_view_where_view AS SELECT age FROM test_select_view_where_table"); assertMetastoreInvocations("SELECT * FROM test_select_view_where_view WHERE age = 2", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .addCopies(GET_TABLE, 2) .build()); } @@ -195,7 +196,7 @@ public void testJoin() assertUpdate("CREATE TABLE test_join_t2 AS SELECT 'name1' as name, 'id1' AS id", 1); assertMetastoreInvocations("SELECT name, age FROM test_join_t1 JOIN test_join_t2 ON test_join_t2.id = test_join_t1.id", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .addCopies(GET_TABLE, 2) .build()); } @@ -206,7 +207,7 @@ public void testSelfJoin() assertUpdate("CREATE TABLE test_self_join_table AS SELECT 2 as age, 0 parent, 3 AS id", 1); assertMetastoreInvocations("SELECT child.age, parent.age FROM test_self_join_table child JOIN test_self_join_table parent ON child.parent = parent.id", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_TABLE) .build()); } @@ -217,7 +218,7 @@ public void testExplainSelect() assertUpdate("CREATE TABLE test_explain AS SELECT 2 as age", 1); assertMetastoreInvocations("EXPLAIN SELECT * FROM test_explain", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_TABLE) .build()); } @@ -228,7 +229,7 @@ public void testShowStatsForTable() assertUpdate("CREATE TABLE test_show_stats AS SELECT 2 as age", 1); assertMetastoreInvocations("SHOW STATS FOR test_show_stats", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_TABLE) .build()); } @@ -239,7 +240,7 @@ public void testShowStatsForTableWithFilter() assertUpdate("CREATE TABLE test_show_stats_with_filter AS SELECT 2 as age", 1); assertMetastoreInvocations("SHOW STATS FOR (SELECT * FROM test_show_stats_with_filter where age >= 2)", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_TABLE) .build()); } @@ -250,7 +251,7 @@ public void testDropTable() assertUpdate("CREATE TABLE test_drop_table AS SELECT 20050910 as a_number", 1); assertMetastoreInvocations("DROP TABLE test_drop_table", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_TABLE) .add(DROP_TABLE) .build()); @@ -260,13 +261,13 @@ public void testDropTable() public void testShowTables() { assertMetastoreInvocations("SHOW TABLES", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_ALL_DATABASES) .add(GET_ALL_TABLES_FROM_DATABASE) .build()); } - private void assertMetastoreInvocations(@Language("SQL") String query, Multiset expectedInvocations) + private void assertMetastoreInvocations(@Language("SQL") String query, Multiset expectedInvocations) { assertUpdate("CALL system.flush_metadata_cache()"); diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/CountingAccessHiveMetastoreUtil.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/CountingAccessHiveMetastoreUtil.java index 46a8a8a969f66..b6c0873800184 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/CountingAccessHiveMetastoreUtil.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/CountingAccessHiveMetastoreUtil.java @@ -29,7 +29,7 @@ public static void assertMetastoreInvocations( QueryRunner queryRunner, Session session, @Language("SQL") String query, - Multiset expectedInvocations) + Multiset expectedInvocations) { metastore.resetCounters(); queryRunner.execute(session, query); diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreAccessOperations.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreAccessOperations.java index 03b6c746e87a0..126202d7398fb 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreAccessOperations.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreAccessOperations.java @@ -20,6 +20,7 @@ import io.trino.plugin.hive.TestingHivePlugin; import io.trino.plugin.hive.metastore.CountingAccessHiveMetastore; import io.trino.plugin.hive.metastore.CountingAccessHiveMetastoreUtil; +import io.trino.plugin.hive.metastore.MetastoreMethod; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.DistributedQueryRunner; import io.trino.testing.QueryRunner; @@ -73,7 +74,7 @@ protected QueryRunner createQueryRunner() public void testUse() { assertMetastoreInvocations("USE " + getSession().getSchema().orElseThrow(), - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_DATABASE) .build()); } @@ -82,7 +83,7 @@ public void testUse() public void testCreateTable() { assertMetastoreInvocations("CREATE TABLE test_create(id VARCHAR, age INT)", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(CREATE_TABLE) .add(GET_DATABASE) .add(GET_TABLE) @@ -94,7 +95,7 @@ public void testCreateTable() public void testCreateTableAsSelect() { assertMetastoreInvocations("CREATE TABLE test_ctas AS SELECT 1 AS age", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_DATABASE) .add(CREATE_TABLE) .add(GET_TABLE) @@ -108,7 +109,7 @@ public void testSelect() assertUpdate("CREATE TABLE test_select_from(id VARCHAR, age INT)"); assertMetastoreInvocations("SELECT * FROM test_select_from", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_TABLE) .build()); } @@ -119,7 +120,7 @@ public void testSelectPartitionedTable() assertUpdate("CREATE TABLE test_select_partition WITH (partitioned_by = ARRAY['part']) AS SELECT 1 AS data, 10 AS part", 1); assertMetastoreInvocations("SELECT * FROM test_select_partition", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_TABLE) .add(GET_PARTITION_NAMES_BY_FILTER) .add(GET_PARTITIONS_BY_NAMES) @@ -127,7 +128,7 @@ public void testSelectPartitionedTable() assertUpdate("INSERT INTO test_select_partition SELECT 2 AS data, 20 AS part", 1); assertMetastoreInvocations("SELECT * FROM test_select_partition", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_TABLE) .add(GET_PARTITION_NAMES_BY_FILTER) .add(GET_PARTITIONS_BY_NAMES) @@ -135,7 +136,7 @@ public void testSelectPartitionedTable() // Specify a specific partition assertMetastoreInvocations("SELECT * FROM test_select_partition WHERE part = 10", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_TABLE) .add(GET_PARTITION_NAMES_BY_FILTER) .add(GET_PARTITIONS_BY_NAMES) @@ -148,7 +149,7 @@ public void testSelectWithFilter() assertUpdate("CREATE TABLE test_select_from_where AS SELECT 2 AS age", 1); assertMetastoreInvocations("SELECT * FROM test_select_from_where WHERE age = 2", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_TABLE) .build()); } @@ -160,7 +161,7 @@ public void testSelectFromView() assertUpdate("CREATE VIEW test_select_view_view AS SELECT id, age FROM test_select_view_table"); assertMetastoreInvocations("SELECT * FROM test_select_view_view", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .addCopies(GET_TABLE, 2) .build()); } @@ -172,7 +173,7 @@ public void testSelectFromViewWithFilter() assertUpdate("CREATE VIEW test_select_view_where_view AS SELECT age FROM test_select_view_where_table"); assertMetastoreInvocations("SELECT * FROM test_select_view_where_view WHERE age = 2", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .addCopies(GET_TABLE, 2) .build()); } @@ -184,7 +185,7 @@ public void testJoin() assertUpdate("CREATE TABLE test_join_t2 AS SELECT 'name1' AS name, 'id1' AS id", 1); assertMetastoreInvocations("SELECT name, age FROM test_join_t1 JOIN test_join_t2 ON test_join_t2.id = test_join_t1.id", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .addCopies(GET_TABLE, 2) .addCopies(GET_TABLE_STATISTICS, 2) .build()); @@ -196,7 +197,7 @@ public void testSelfJoin() assertUpdate("CREATE TABLE test_self_join_table AS SELECT 2 AS age, 0 parent, 3 AS id", 1); assertMetastoreInvocations("SELECT child.age, parent.age FROM test_self_join_table child JOIN test_self_join_table parent ON child.parent = parent.id", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_TABLE) .add(GET_TABLE_STATISTICS) .build()); @@ -208,7 +209,7 @@ public void testExplainSelect() assertUpdate("CREATE TABLE test_explain AS SELECT 2 AS age", 1); assertMetastoreInvocations("EXPLAIN SELECT * FROM test_explain", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_TABLE) .add(GET_TABLE_STATISTICS) .build()); @@ -220,7 +221,7 @@ public void testDescribe() assertUpdate("CREATE TABLE test_describe(id VARCHAR, age INT)"); assertMetastoreInvocations("DESCRIBE test_describe", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_DATABASE) .add(GET_TABLE) .build()); @@ -232,7 +233,7 @@ public void testShowStatsForTable() assertUpdate("CREATE TABLE test_show_stats AS SELECT 2 AS age", 1); assertMetastoreInvocations("SHOW STATS FOR test_show_stats", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_TABLE) .add(GET_TABLE_STATISTICS) .build()); @@ -244,7 +245,7 @@ public void testShowStatsForTableWithFilter() assertUpdate("CREATE TABLE test_show_stats_with_filter AS SELECT 2 AS age", 1); assertMetastoreInvocations("SHOW STATS FOR (SELECT * FROM test_show_stats_with_filter where age >= 2)", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_TABLE) .add(GET_TABLE_STATISTICS) .build()); @@ -256,7 +257,7 @@ public void testAnalyze() assertUpdate("CREATE TABLE test_analyze AS SELECT 2 AS age", 1); assertMetastoreInvocations("ANALYZE test_analyze", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_TABLE) .add(UPDATE_TABLE_STATISTICS) .build()); @@ -268,7 +269,7 @@ public void testAnalyzePartitionedTable() assertUpdate("CREATE TABLE test_analyze_partition WITH (partitioned_by = ARRAY['part']) AS SELECT 1 AS data, 10 AS part", 1); assertMetastoreInvocations("ANALYZE test_analyze_partition", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .addCopies(GET_TABLE, 1) .add(GET_PARTITION_NAMES_BY_FILTER) .add(GET_PARTITIONS_BY_NAMES) @@ -279,7 +280,7 @@ public void testAnalyzePartitionedTable() assertUpdate("INSERT INTO test_analyze_partition SELECT 2 AS data, 20 AS part", 1); assertMetastoreInvocations("ANALYZE test_analyze_partition", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_TABLE) .add(GET_PARTITION_NAMES_BY_FILTER) .add(GET_PARTITIONS_BY_NAMES) @@ -294,7 +295,7 @@ public void testDropStats() assertUpdate("CREATE TABLE drop_stats AS SELECT 2 AS age", 1); assertMetastoreInvocations("CALL system.drop_stats('test_schema', 'drop_stats')", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_TABLE) .add(UPDATE_TABLE_STATISTICS) .build()); @@ -306,7 +307,7 @@ public void testDropStatsPartitionedTable() assertUpdate("CREATE TABLE drop_stats_partition WITH (partitioned_by = ARRAY['part']) AS SELECT 1 AS data, 10 AS part", 1); assertMetastoreInvocations("CALL system.drop_stats('test_schema', 'drop_stats_partition')", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_TABLE) .add(GET_PARTITION_NAMES_BY_FILTER) .add(UPDATE_PARTITION_STATISTICS) @@ -315,14 +316,14 @@ public void testDropStatsPartitionedTable() assertUpdate("INSERT INTO drop_stats_partition SELECT 2 AS data, 20 AS part", 1); assertMetastoreInvocations("CALL system.drop_stats('test_schema', 'drop_stats_partition')", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_TABLE) .add(GET_PARTITION_NAMES_BY_FILTER) .addCopies(UPDATE_PARTITION_STATISTICS, 2) .build()); } - private void assertMetastoreInvocations(@Language("SQL") String query, Multiset expectedInvocations) + private void assertMetastoreInvocations(@Language("SQL") String query, Multiset expectedInvocations) { CountingAccessHiveMetastoreUtil.assertMetastoreInvocations(metastore, getQueryRunner(), getQueryRunner().getDefaultSession(), query, expectedInvocations); } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreMetadataQueriesAccessOperations.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreMetadataQueriesAccessOperations.java index 79d2670a87514..eba36f5754dbd 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreMetadataQueriesAccessOperations.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreMetadataQueriesAccessOperations.java @@ -170,12 +170,12 @@ public void testSelectTablesWithFilterBySchema() { assertMetastoreInvocations( "SELECT * FROM information_schema.tables WHERE table_schema = 'test_schema_0'", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_RELATION_TYPES_FROM_DATABASE) .build()); assertMetastoreInvocations( "SELECT * FROM system.jdbc.tables WHERE table_schem = 'test_schema_0'", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_RELATION_TYPES_FROM_DATABASE) .build()); } @@ -185,13 +185,13 @@ public void testSelectTablesWithLikeOverSchema() { assertMetastoreInvocations( "SELECT * FROM information_schema.tables WHERE table_schema LIKE 'test%'", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_ALL_DATABASES) .add(GET_ALL_RELATION_TYPES) .build()); assertMetastoreInvocations( "SELECT * FROM system.jdbc.tables WHERE table_schem LIKE 'test%'", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_ALL_RELATION_TYPES) .build()); } @@ -201,7 +201,7 @@ public void testSelectTablesWithFilterByTableName() { assertMetastoreInvocations( "SELECT * FROM information_schema.tables WHERE table_name = 'test_table_0'", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_ALL_DATABASES) .add(GET_ALL_RELATION_TYPES) .build()); @@ -219,13 +219,13 @@ public void testSelectTablesWithLikeOverTableName() { assertMetastoreInvocations( "SELECT * FROM information_schema.tables WHERE table_name LIKE 'test%'", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_ALL_DATABASES) .add(GET_ALL_RELATION_TYPES) .build()); assertMetastoreInvocations( "SELECT * FROM system.jdbc.tables WHERE table_name LIKE 'test%'", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_ALL_RELATION_TYPES) .build()); } @@ -236,7 +236,7 @@ public void testSelectViewsWithoutPredicate() assertMetastoreInvocations("SELECT * FROM information_schema.views", ImmutableMultiset.of(GET_ALL_VIEWS)); assertMetastoreInvocations( "SELECT * FROM system.jdbc.tables WHERE table_type = 'VIEW'", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_ALL_RELATION_TYPES) .build()); } @@ -253,7 +253,7 @@ public void testSelectViewsWithFilterBySchema() { assertMetastoreInvocations("SELECT * FROM information_schema.views WHERE table_schema = 'test_schema_0'", ImmutableMultiset.of(GET_ALL_VIEWS_FROM_DATABASE)); assertMetastoreInvocations("SELECT * FROM system.jdbc.tables WHERE table_type = 'VIEW' AND table_schem = 'test_schema_0'", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_RELATION_TYPES_FROM_DATABASE) .build()); } @@ -263,13 +263,13 @@ public void testSelectViewsWithLikeOverSchema() { assertMetastoreInvocations( "SELECT * FROM information_schema.views WHERE table_schema LIKE 'test%'", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_ALL_DATABASES) .add(GET_ALL_VIEWS) .build()); assertMetastoreInvocations( "SELECT * FROM system.jdbc.tables WHERE table_type = 'VIEW' AND table_schem LIKE 'test%'", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_ALL_RELATION_TYPES) .build()); } @@ -279,13 +279,13 @@ public void testSelectViewsWithFilterByTableName() { assertMetastoreInvocations( "SELECT * FROM information_schema.views WHERE table_name = 'test_table_0'", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_ALL_DATABASES) .add(GET_ALL_VIEWS) .build()); assertMetastoreInvocations( "SELECT * FROM system.jdbc.tables WHERE table_type = 'VIEW' AND table_name = 'test_table_0'", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_ALL_RELATION_TYPES) .build()); } @@ -295,13 +295,13 @@ public void testSelectViewsWithLikeOverTableName() { assertMetastoreInvocations( "SELECT * FROM information_schema.views WHERE table_name LIKE 'test%'", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_ALL_DATABASES) .add(GET_ALL_VIEWS) .build()); assertMetastoreInvocations( "SELECT * FROM system.jdbc.tables WHERE table_type = 'VIEW' AND table_name LIKE 'test%'", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_ALL_RELATION_TYPES) .build()); } @@ -329,25 +329,25 @@ public void testSelectColumnsFilterByInformationSchema() public void testSelectColumnsFilterBySchema() { assertMetastoreInvocations("SELECT * FROM information_schema.columns WHERE table_schema = 'test_schema_0'", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_ALL_TABLES_FROM_DATABASE) .add(GET_ALL_VIEWS_FROM_DATABASE) .addCopies(GET_TABLE, TEST_TABLES_IN_SCHEMA_COUNT) .build()); assertMetastoreInvocations("SELECT * FROM system.jdbc.columns WHERE table_schem = 'test_schema_0'", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_ALL_TABLES_FROM_DATABASE) .add(GET_ALL_VIEWS_FROM_DATABASE) .addCopies(GET_TABLE, TEST_TABLES_IN_SCHEMA_COUNT) .build()); assertMetastoreInvocations("SELECT * FROM system.jdbc.columns WHERE table_schem LIKE 'test\\_schema\\_0' ESCAPE '\\'", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_ALL_TABLES_FROM_DATABASE) .add(GET_ALL_VIEWS_FROM_DATABASE) .addCopies(GET_TABLE, TEST_TABLES_IN_SCHEMA_COUNT) .build()); assertMetastoreInvocations("SELECT * FROM system.jdbc.columns WHERE table_schem LIKE 'test_schema_0' ESCAPE '\\'", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_ALL_DATABASES) .add(GET_ALL_TABLES_FROM_DATABASE) .add(GET_ALL_VIEWS_FROM_DATABASE) @@ -360,7 +360,7 @@ public void testSelectColumnsWithLikeOverSchema() { assertMetastoreInvocations( "SELECT * FROM information_schema.columns WHERE table_schema LIKE 'test%'", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_ALL_DATABASES) .add(GET_ALL_TABLES) .add(GET_ALL_VIEWS) @@ -368,7 +368,7 @@ public void testSelectColumnsWithLikeOverSchema() .build()); assertMetastoreInvocations( "SELECT * FROM system.jdbc.columns WHERE table_schem LIKE 'test%'", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_ALL_DATABASES) .addCopies(GET_ALL_TABLES_FROM_DATABASE, TEST_SCHEMAS_COUNT) .addCopies(GET_TABLE, TEST_ALL_TABLES_COUNT) @@ -380,7 +380,7 @@ public void testSelectColumnsFilterByTableName() { assertMetastoreInvocations( "SELECT * FROM information_schema.columns WHERE table_name = 'test_table_0'", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_ALL_DATABASES) .add(GET_ALL_TABLES) .add(GET_ALL_VIEWS) @@ -390,21 +390,21 @@ public void testSelectColumnsFilterByTableName() .build()); assertMetastoreInvocations( "SELECT * FROM system.jdbc.columns WHERE table_name = 'test_table_0'", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_ALL_DATABASES) .add(GET_ALL_TABLES_FROM_DATABASE) .addCopies(GET_TABLE, TEST_SCHEMAS_COUNT + 1) .build()); assertMetastoreInvocations( "SELECT * FROM system.jdbc.columns WHERE table_name LIKE 'test\\_table\\_0' ESCAPE '\\'", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_ALL_DATABASES) .add(GET_ALL_TABLES_FROM_DATABASE) .addCopies(GET_TABLE, TEST_SCHEMAS_COUNT + 1) .build()); assertMetastoreInvocations( "SELECT * FROM system.jdbc.columns WHERE table_name LIKE 'test_table_0' ESCAPE '\\'", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_ALL_DATABASES) .add(GET_ALL_TABLES_FROM_DATABASE) .addCopies(GET_ALL_TABLES_FROM_DATABASE, TEST_SCHEMAS_COUNT) @@ -416,14 +416,14 @@ public void testSelectColumnsFilterByTableName() public void testSelectColumnsWithLikeOverTableName() { assertMetastoreInvocations("SELECT * FROM information_schema.columns WHERE table_name LIKE 'test%'", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_ALL_DATABASES) .add(GET_ALL_TABLES) .add(GET_ALL_VIEWS) .addCopies(GET_TABLE, TEST_ALL_TABLES_COUNT) .build()); assertMetastoreInvocations("SELECT * FROM system.jdbc.columns WHERE table_name LIKE 'test%'", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_ALL_DATABASES) .addCopies(GET_ALL_TABLES_FROM_DATABASE, TEST_SCHEMAS_COUNT + 1) .addCopies(GET_TABLE, TEST_ALL_TABLES_COUNT) @@ -435,14 +435,14 @@ public void testSelectColumnsFilterByColumn() { assertMetastoreInvocations( "SELECT * FROM information_schema.columns WHERE column_name = 'name'", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_ALL_TABLES) .add(GET_ALL_VIEWS) .addCopies(GET_TABLE, TEST_ALL_TABLES_COUNT) .build()); assertMetastoreInvocations( "SELECT * FROM system.jdbc.columns WHERE column_name = 'name'", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_ALL_TABLES) .add(GET_ALL_VIEWS) .addCopies(GET_TABLE, TEST_ALL_TABLES_COUNT) @@ -454,7 +454,7 @@ public void testSelectColumnsWithLikeOverColumn() { assertMetastoreInvocations( "SELECT * FROM information_schema.columns WHERE column_name LIKE 'n%'", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_ALL_DATABASES) .add(GET_ALL_TABLES) .add(GET_ALL_VIEWS) @@ -462,7 +462,7 @@ public void testSelectColumnsWithLikeOverColumn() .build()); assertMetastoreInvocations( "SELECT * FROM system.jdbc.columns WHERE column_name LIKE 'n%'", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_ALL_TABLES) .add(GET_ALL_VIEWS) .addCopies(GET_TABLE, TEST_ALL_TABLES_COUNT) @@ -476,14 +476,14 @@ public void testSelectColumnsFilterByTableAndSchema() assertMetastoreInvocations("SELECT * FROM system.jdbc.columns WHERE table_schem = 'test_schema_0' AND table_name = 'test_table_0'", ImmutableMultiset.of(GET_TABLE)); assertMetastoreInvocations("SELECT * FROM system.jdbc.columns WHERE table_schem LIKE 'test\\_schema\\_0' ESCAPE '\\' AND table_name LIKE 'test\\_table\\_0' ESCAPE '\\'", ImmutableMultiset.of(GET_TABLE)); assertMetastoreInvocations("SELECT * FROM system.jdbc.columns WHERE table_schem LIKE 'test_schema_0' ESCAPE '\\' AND table_name LIKE 'test_table_0' ESCAPE '\\'", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_ALL_DATABASES) .add(GET_ALL_TABLES_FROM_DATABASE) .add(GET_TABLE) .build()); } - private void assertMetastoreInvocations(@Language("SQL") String query, Multiset expectedInvocations) + private void assertMetastoreInvocations(@Language("SQL") String query, Multiset expectedInvocations) { CountingAccessHiveMetastoreUtil.assertMetastoreInvocations(metastore, getQueryRunner(), getQueryRunner().getDefaultSession(), query, expectedInvocations); } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetastoreAccessOperations.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetastoreAccessOperations.java index 990794a5371a4..61b592b6eca3a 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetastoreAccessOperations.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetastoreAccessOperations.java @@ -18,6 +18,7 @@ import io.trino.Session; import io.trino.plugin.hive.metastore.CountingAccessHiveMetastore; import io.trino.plugin.hive.metastore.CountingAccessHiveMetastoreUtil; +import io.trino.plugin.hive.metastore.MetastoreMethod; import io.trino.plugin.iceberg.catalog.file.TestingIcebergFileMetastoreCatalogModule; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.DistributedQueryRunner; @@ -97,7 +98,7 @@ public void testUse() .setSchema(Optional.empty()) .build(); assertMetastoreInvocations(session, "USE %s.%s".formatted(catalog, schema), - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_DATABASE) .build()); } @@ -106,7 +107,7 @@ public void testUse() public void testCreateTable() { assertMetastoreInvocations("CREATE TABLE test_create (id VARCHAR, age INT)", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(CREATE_TABLE) .add(GET_DATABASE) .add(GET_TABLE) @@ -117,13 +118,13 @@ public void testCreateTable() public void testCreateOrReplaceTable() { assertMetastoreInvocations("CREATE OR REPLACE TABLE test_create_or_replace (id VARCHAR, age INT)", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(CREATE_TABLE) .add(GET_DATABASE) .add(GET_TABLE) .build()); assertMetastoreInvocations("CREATE OR REPLACE TABLE test_create_or_replace (id VARCHAR, age INT)", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_DATABASE) .add(REPLACE_TABLE) .add(GET_TABLE) @@ -136,7 +137,7 @@ public void testCreateTableAsSelect() assertMetastoreInvocations( withStatsOnWrite(getSession(), false), "CREATE TABLE test_ctas AS SELECT 1 AS age", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_DATABASE) .add(CREATE_TABLE) .add(GET_TABLE) @@ -145,7 +146,7 @@ public void testCreateTableAsSelect() assertMetastoreInvocations( withStatsOnWrite(getSession(), true), "CREATE TABLE test_ctas_with_stats AS SELECT 1 AS age", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_DATABASE) .add(CREATE_TABLE) .addCopies(GET_TABLE, 4) @@ -158,7 +159,7 @@ public void testCreateOrReplaceTableAsSelect() { assertMetastoreInvocations( "CREATE OR REPLACE TABLE test_cortas AS SELECT 1 AS age", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_DATABASE) .add(CREATE_TABLE) .addCopies(GET_TABLE, 4) @@ -167,7 +168,7 @@ public void testCreateOrReplaceTableAsSelect() assertMetastoreInvocations( "CREATE OR REPLACE TABLE test_cortas AS SELECT 1 AS age", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_DATABASE) .addCopies(GET_TABLE, 3) .addCopies(REPLACE_TABLE, 2) @@ -180,7 +181,7 @@ public void testSelect() assertUpdate("CREATE TABLE test_select_from (id VARCHAR, age INT)"); assertMetastoreInvocations("SELECT * FROM test_select_from", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_TABLE) .build()); } @@ -191,7 +192,7 @@ public void testSelectWithFilter() assertUpdate("CREATE TABLE test_select_from_where AS SELECT 2 as age", 1); assertMetastoreInvocations("SELECT * FROM test_select_from_where WHERE age = 2", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_TABLE) .build()); } @@ -203,7 +204,7 @@ public void testSelectFromView() assertUpdate("CREATE VIEW test_select_view_view AS SELECT id, age FROM test_select_view_table"); assertMetastoreInvocations("SELECT * FROM test_select_view_view", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .addCopies(GET_TABLE, 2) .build()); } @@ -215,7 +216,7 @@ public void testSelectFromViewWithFilter() assertUpdate("CREATE VIEW test_select_view_where_view AS SELECT age FROM test_select_view_where_table"); assertMetastoreInvocations("SELECT * FROM test_select_view_where_view WHERE age = 2", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .addCopies(GET_TABLE, 2) .build()); } @@ -227,7 +228,7 @@ public void testSelectFromMaterializedView() assertUpdate("CREATE MATERIALIZED VIEW test_select_mview_view AS SELECT id, age FROM test_select_mview_table"); assertMetastoreInvocations("SELECT * FROM test_select_mview_view", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .addCopies(GET_TABLE, 2) .build()); } @@ -239,7 +240,7 @@ public void testSelectFromMaterializedViewWithFilter() assertUpdate("CREATE MATERIALIZED VIEW test_select_mview_where_view AS SELECT age FROM test_select_mview_where_table"); assertMetastoreInvocations("SELECT * FROM test_select_mview_where_view WHERE age = 2", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .addCopies(GET_TABLE, 2) .build()); } @@ -251,7 +252,7 @@ public void testRefreshMaterializedView() assertUpdate("CREATE MATERIALIZED VIEW test_refresh_mview_view AS SELECT id, age FROM test_refresh_mview_table"); assertMetastoreInvocations("REFRESH MATERIALIZED VIEW test_refresh_mview_view", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .addCopies(GET_TABLE, 2) .addCopies(REPLACE_TABLE, 1) .build()); @@ -264,7 +265,7 @@ public void testJoin() assertUpdate("CREATE TABLE test_join_t2 AS SELECT 'name1' as name, 'id1' AS id", 1); assertMetastoreInvocations("SELECT name, age FROM test_join_t1 JOIN test_join_t2 ON test_join_t2.id = test_join_t1.id", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .addCopies(GET_TABLE, 2) .build()); } @@ -275,7 +276,7 @@ public void testSelfJoin() assertUpdate("CREATE TABLE test_self_join_table AS SELECT 2 as age, 0 parent, 3 AS id", 1); assertMetastoreInvocations("SELECT child.age, parent.age FROM test_self_join_table child JOIN test_self_join_table parent ON child.parent = parent.id", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_TABLE) .build()); } @@ -286,7 +287,7 @@ public void testExplainSelect() assertUpdate("CREATE TABLE test_explain AS SELECT 2 as age", 1); assertMetastoreInvocations("EXPLAIN SELECT * FROM test_explain", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_TABLE) .build()); } @@ -297,7 +298,7 @@ public void testShowStatsForTable() assertUpdate("CREATE TABLE test_show_stats AS SELECT 2 as age", 1); assertMetastoreInvocations("SHOW STATS FOR test_show_stats", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_TABLE) .build()); } @@ -308,7 +309,7 @@ public void testShowStatsForTableWithFilter() assertUpdate("CREATE TABLE test_show_stats_with_filter AS SELECT 2 as age", 1); assertMetastoreInvocations("SHOW STATS FOR (SELECT * FROM test_show_stats_with_filter where age >= 2)", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_TABLE) .build()); } @@ -320,37 +321,37 @@ public void testSelectSystemTable() // select from $history assertMetastoreInvocations("SELECT * FROM \"test_select_snapshots$history\"", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .addCopies(GET_TABLE, 1) .build()); // select from $snapshots assertMetastoreInvocations("SELECT * FROM \"test_select_snapshots$snapshots\"", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .addCopies(GET_TABLE, 1) .build()); // select from $manifests assertMetastoreInvocations("SELECT * FROM \"test_select_snapshots$manifests\"", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .addCopies(GET_TABLE, 1) .build()); // select from $partitions assertMetastoreInvocations("SELECT * FROM \"test_select_snapshots$partitions\"", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .addCopies(GET_TABLE, 1) .build()); // select from $files assertMetastoreInvocations("SELECT * FROM \"test_select_snapshots$files\"", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .addCopies(GET_TABLE, 1) .build()); // select from $properties assertMetastoreInvocations("SELECT * FROM \"test_select_snapshots$properties\"", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .addCopies(GET_TABLE, 1) .build()); @@ -368,7 +369,7 @@ public void testUnregisterTable() assertUpdate("CREATE TABLE test_unregister_table AS SELECT 2 as age", 1); assertMetastoreInvocations("CALL system.unregister_table(CURRENT_SCHEMA, 'test_unregister_table')", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_DATABASE) .add(GET_TABLE) .add(DROP_TABLE) @@ -396,7 +397,7 @@ public void testInformationSchemaColumns(int tables) // Bulk retrieval assertMetastoreInvocations(session, "SELECT * FROM information_schema.columns WHERE table_schema = CURRENT_SCHEMA AND table_name LIKE 'test_select_i_s_columns%'", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_ALL_TABLES_FROM_DATABASE) .addCopies(GET_TABLE, tables * 2) .addCopies(GET_TABLES_WITH_PARAMETER, 2) @@ -404,13 +405,13 @@ public void testInformationSchemaColumns(int tables) // Pointed lookup assertMetastoreInvocations(session, "SELECT * FROM information_schema.columns WHERE table_schema = CURRENT_SCHEMA AND table_name = 'test_select_i_s_columns0'", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_TABLE) .build()); // Pointed lookup via DESCRIBE (which does some additional things before delegating to information_schema.columns) assertMetastoreInvocations(session, "DESCRIBE test_select_i_s_columns0", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_DATABASE) .add(GET_TABLE) .build()); @@ -442,7 +443,7 @@ public void testSystemMetadataTableComments(int tables) // Bulk retrieval assertMetastoreInvocations(session, "SELECT * FROM system.metadata.table_comments WHERE schema_name = CURRENT_SCHEMA AND table_name LIKE 'test_select_s_m_t_comments%'", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_ALL_TABLES_FROM_DATABASE) .addCopies(GET_TABLE, tables * 2) .addCopies(GET_TABLES_WITH_PARAMETER, 2) @@ -450,7 +451,7 @@ public void testSystemMetadataTableComments(int tables) // Bulk retrieval for two schemas assertMetastoreInvocations(session, "SELECT * FROM system.metadata.table_comments WHERE schema_name IN (CURRENT_SCHEMA, 'non_existent') AND table_name LIKE 'test_select_s_m_t_comments%'", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .addCopies(GET_ALL_TABLES_FROM_DATABASE, 2) .addCopies(GET_TABLES_WITH_PARAMETER, 4) .addCopies(GET_TABLE, tables * 2) @@ -458,7 +459,7 @@ public void testSystemMetadataTableComments(int tables) // Pointed lookup assertMetastoreInvocations(session, "SELECT * FROM system.metadata.table_comments WHERE schema_name = CURRENT_SCHEMA AND table_name = 'test_select_s_m_t_comments0'", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .addCopies(GET_TABLE, 1) .build()); @@ -497,34 +498,34 @@ public void testSystemMetadataMaterializedViews() // Bulk retrieval assertMetastoreInvocations(session, "SELECT * FROM system.metadata.materialized_views WHERE schema_name = CURRENT_SCHEMA", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_TABLES_WITH_PARAMETER) .addCopies(GET_TABLE, 4) .build()); // Bulk retrieval without selecting freshness assertMetastoreInvocations(session, "SELECT schema_name, name FROM system.metadata.materialized_views WHERE schema_name = CURRENT_SCHEMA", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_TABLES_WITH_PARAMETER) .addCopies(GET_TABLE, 2) .build()); // Bulk retrieval for two schemas assertMetastoreInvocations(session, "SELECT * FROM system.metadata.materialized_views WHERE schema_name IN (CURRENT_SCHEMA, 'non_existent')", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .addCopies(GET_TABLES_WITH_PARAMETER, 2) .addCopies(GET_TABLE, 4) .build()); // Pointed lookup assertMetastoreInvocations(session, "SELECT * FROM system.metadata.materialized_views WHERE schema_name = CURRENT_SCHEMA AND name = 'mv1'", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .addCopies(GET_TABLE, 3) .build()); // Pointed lookup without selecting freshness assertMetastoreInvocations(session, "SELECT schema_name, name FROM system.metadata.materialized_views WHERE schema_name = CURRENT_SCHEMA AND name = 'mv1'", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_TABLE) .build()); @@ -535,18 +536,18 @@ public void testSystemMetadataMaterializedViews() public void testShowTables() { assertMetastoreInvocations("SHOW TABLES", - ImmutableMultiset.builder() + ImmutableMultiset.builder() .add(GET_DATABASE) .add(GET_ALL_TABLES_FROM_DATABASE) .build()); } - private void assertMetastoreInvocations(@Language("SQL") String query, Multiset expectedInvocations) + private void assertMetastoreInvocations(@Language("SQL") String query, Multiset expectedInvocations) { assertMetastoreInvocations(getSession(), query, expectedInvocations); } - private void assertMetastoreInvocations(Session session, @Language("SQL") String query, Multiset expectedInvocations) + private void assertMetastoreInvocations(Session session, @Language("SQL") String query, Multiset expectedInvocations) { CountingAccessHiveMetastoreUtil.assertMetastoreInvocations(metastore, getQueryRunner(), session, query, expectedInvocations); } From f1d0a4412ca90a8bbbe14e50bde2b0c2d26347e9 Mon Sep 17 00:00:00 2001 From: David Phillips Date: Sat, 2 Dec 2023 16:52:35 -0800 Subject: [PATCH 081/350] Create Hive metastore for testing using properties --- .../hive/InternalHiveConnectorFactory.java | 7 ++- .../io/trino/plugin/hive/HiveQueryRunner.java | 17 +++---- .../hive/TestingHiveConnectorFactory.java | 10 ++-- .../trino/plugin/hive/TestingHivePlugin.java | 11 ++--- .../s3/TestS3FileSystemAccessOperations.java | 47 ++++++++----------- 5 files changed, 38 insertions(+), 54 deletions(-) diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/InternalHiveConnectorFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/InternalHiveConnectorFactory.java index 6dda13819bece..6d10eb61cb10e 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/InternalHiveConnectorFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/InternalHiveConnectorFactory.java @@ -80,7 +80,7 @@ private InternalHiveConnectorFactory() {} public static Connector createConnector(String catalogName, Map config, ConnectorContext context, Module module) { - return createConnector(catalogName, config, context, module, Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty()); + return createConnector(catalogName, config, context, module, Optional.empty(), Optional.empty(), Optional.empty()); } public static Connector createConnector( @@ -90,7 +90,6 @@ public static Connector createConnector( Module module, Optional metastore, Optional fileSystemFactory, - Optional openTelemetry, Optional directoryLister) { requireNonNull(config, "config is null"); @@ -110,11 +109,11 @@ public static Connector createConnector( new HiveSecurityModule(), fileSystemFactory .map(factory -> (Module) binder -> binder.bind(TrinoFileSystemFactory.class).toInstance(factory)) - .orElseGet(() -> new FileSystemModule(catalogName, context.getNodeManager(), openTelemetry.orElse(context.getOpenTelemetry()))), + .orElseGet(() -> new FileSystemModule(catalogName, context.getNodeManager(), context.getOpenTelemetry())), new HiveProcedureModule(), new MBeanServerModule(), binder -> { - binder.bind(OpenTelemetry.class).toInstance(openTelemetry.orElse(context.getOpenTelemetry())); + binder.bind(OpenTelemetry.class).toInstance(context.getOpenTelemetry()); binder.bind(Tracer.class).toInstance(context.getTracer()); binder.bind(NodeVersion.class).toInstance(new NodeVersion(context.getNodeManager().getCurrentNode().getVersion())); binder.bind(NodeManager.class).toInstance(context.getNodeManager()); diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveQueryRunner.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveQueryRunner.java index 90e1fe1e27f20..5993731129f01 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveQueryRunner.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveQueryRunner.java @@ -19,7 +19,6 @@ import com.google.inject.Module; import io.airlift.log.Logger; import io.airlift.log.Logging; -import io.opentelemetry.api.OpenTelemetry; import io.trino.Session; import io.trino.metadata.QualifiedObjectName; import io.trino.plugin.hive.fs.DirectoryLister; @@ -106,7 +105,6 @@ public static class Builder> private List> initialTables = ImmutableList.of(); private Optional initialSchemasLocationBase = Optional.empty(); private Optional> metastore = Optional.empty(); - private Optional openTelemetry = Optional.empty(); private Module module = EMPTY_MODULE; private Optional directoryLister = Optional.empty(); private boolean tpcdsCatalogEnabled; @@ -168,13 +166,6 @@ public SELF setMetastore(Function metasto return self(); } - @CanIgnoreReturnValue - public SELF setOpenTelemetry(OpenTelemetry openTelemetry) - { - this.openTelemetry = Optional.of(openTelemetry); - return self(); - } - @CanIgnoreReturnValue public SELF setModule(Module module) { @@ -247,7 +238,13 @@ public DistributedQueryRunner build() Optional metastore = this.metastore.map(factory -> factory.apply(queryRunner)); Path dataDir = queryRunner.getCoordinator().getBaseDataDir().resolve("hive_data"); - queryRunner.installPlugin(new TestingHivePlugin(dataDir, metastore, openTelemetry, module, directoryLister)); + + if (metastore.isEmpty() && !hiveProperties.buildOrThrow().containsKey("hive.metastore")) { + hiveProperties.put("hive.metastore", "file"); + hiveProperties.put("hive.metastore.catalog.dir", queryRunner.getCoordinator().getBaseDataDir().resolve("hive_data").toString()); + } + + queryRunner.installPlugin(new TestingHivePlugin(dataDir, metastore, module, directoryLister)); Map hiveProperties = new HashMap<>(); if (!skipTimezoneSetup) { diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestingHiveConnectorFactory.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestingHiveConnectorFactory.java index c0dcb429488ce..b6b5761570d8e 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestingHiveConnectorFactory.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestingHiveConnectorFactory.java @@ -15,7 +15,6 @@ import com.google.common.collect.ImmutableMap; import com.google.inject.Module; -import io.opentelemetry.api.OpenTelemetry; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.filesystem.local.LocalFileSystemFactory; import io.trino.plugin.hive.fs.DirectoryLister; @@ -39,25 +38,22 @@ public class TestingHiveConnectorFactory implements ConnectorFactory { private final Optional metastore; - private final Optional openTelemetry; private final Module module; private final Optional directoryLister; public TestingHiveConnectorFactory(Path localFileSystemRootPath) { - this(localFileSystemRootPath, Optional.empty(), Optional.empty(), EMPTY_MODULE, Optional.empty()); + this(localFileSystemRootPath, Optional.empty(), EMPTY_MODULE, Optional.empty()); } @Deprecated public TestingHiveConnectorFactory( Path localFileSystemRootPath, Optional metastore, - Optional openTelemetry, Module module, Optional directoryLister) { this.metastore = requireNonNull(metastore, "metastore is null"); - this.openTelemetry = requireNonNull(openTelemetry, "openTelemetry is null"); localFileSystemRootPath.toFile().mkdirs(); this.module = binder -> { @@ -85,6 +81,8 @@ public Connector create(String catalogName, Map config, Connecto if (metastore.isEmpty() && !config.containsKey("hive.metastore")) { configBuilder.put("hive.metastore", "file"); } - return createConnector(catalogName, configBuilder.buildOrThrow(), context, module, metastore, Optional.empty(), openTelemetry, directoryLister); + return createConnector(catalogName, configBuilder.buildOrThrow(), context, module, metastore, + Optional.empty(), + directoryLister); } } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestingHivePlugin.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestingHivePlugin.java index 202c5ef949a08..64733255a667d 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestingHivePlugin.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestingHivePlugin.java @@ -15,7 +15,6 @@ import com.google.common.collect.ImmutableList; import com.google.inject.Module; -import io.opentelemetry.api.OpenTelemetry; import io.trino.plugin.hive.fs.DirectoryLister; import io.trino.plugin.hive.metastore.HiveMetastore; import io.trino.spi.Plugin; @@ -32,27 +31,25 @@ public class TestingHivePlugin { private final Path localFileSystemRootPath; private final Optional metastore; - private final Optional openTelemetry; private final Module module; private final Optional directoryLister; public TestingHivePlugin(Path localFileSystemRootPath) { - this(localFileSystemRootPath, Optional.empty(), Optional.empty(), EMPTY_MODULE, Optional.empty()); + this(localFileSystemRootPath, Optional.empty(), EMPTY_MODULE, Optional.empty()); } @Deprecated public TestingHivePlugin(Path localFileSystemRootPath, HiveMetastore metastore) { - this(localFileSystemRootPath, Optional.of(metastore), Optional.empty(), EMPTY_MODULE, Optional.empty()); + this(localFileSystemRootPath, Optional.of(metastore), EMPTY_MODULE, Optional.empty()); } @Deprecated - public TestingHivePlugin(Path localFileSystemRootPath, Optional metastore, Optional openTelemetry, Module module, Optional directoryLister) + public TestingHivePlugin(Path localFileSystemRootPath, Optional metastore, Module module, Optional directoryLister) { this.localFileSystemRootPath = requireNonNull(localFileSystemRootPath, "localFileSystemRootPath is null"); this.metastore = requireNonNull(metastore, "metastore is null"); - this.openTelemetry = requireNonNull(openTelemetry, "openTelemetry is null"); this.module = requireNonNull(module, "module is null"); this.directoryLister = requireNonNull(directoryLister, "directoryLister is null"); } @@ -60,6 +57,6 @@ public TestingHivePlugin(Path localFileSystemRootPath, Optional m @Override public Iterable getConnectorFactories() { - return ImmutableList.of(new TestingHiveConnectorFactory(localFileSystemRootPath, metastore, openTelemetry, module, directoryLister)); + return ImmutableList.of(new TestingHiveConnectorFactory(localFileSystemRootPath, metastore, module, directoryLister)); } } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/TestS3FileSystemAccessOperations.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/TestS3FileSystemAccessOperations.java index b044fe9e7b260..4e41f64437152 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/TestS3FileSystemAccessOperations.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/TestS3FileSystemAccessOperations.java @@ -18,12 +18,7 @@ import com.google.common.collect.ImmutableMultiset; import com.google.common.collect.Multiset; import io.airlift.units.DataSize; -import io.opentelemetry.api.OpenTelemetry; -import io.opentelemetry.sdk.OpenTelemetrySdk; -import io.opentelemetry.sdk.testing.exporter.InMemorySpanExporter; -import io.opentelemetry.sdk.trace.SdkTracerProvider; import io.opentelemetry.sdk.trace.data.SpanData; -import io.opentelemetry.sdk.trace.export.SimpleSpanProcessor; import io.trino.Session; import io.trino.plugin.hive.HiveQueryRunner; import io.trino.testing.AbstractTestQueryFramework; @@ -39,7 +34,7 @@ import java.util.List; import java.util.Map; -import java.util.Optional; +import java.util.function.Predicate; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.collect.Maps.uniqueIndex; @@ -60,7 +55,6 @@ public class TestS3FileSystemAccessOperations private static final String BUCKET = "test-bucket"; private Minio minio; - private InMemorySpanExporter spanExporter; @Override protected QueryRunner createQueryRunner() @@ -70,16 +64,6 @@ protected QueryRunner createQueryRunner() minio.start(); minio.createBucket(BUCKET); - spanExporter = closeAfterClass(InMemorySpanExporter.create()); - - SdkTracerProvider tracerProvider = SdkTracerProvider.builder() - .addSpanProcessor(SimpleSpanProcessor.create(spanExporter)) - .build(); - - OpenTelemetry openTelemetry = OpenTelemetrySdk.builder() - .setTracerProvider(tracerProvider) - .build(); - return HiveQueryRunner.builder() .setHiveProperties(ImmutableMap.builder() .put("hive.metastore.disable-location-checks", "true") @@ -91,8 +75,9 @@ protected QueryRunner createQueryRunner() .put("s3.endpoint", minio.getMinioAddress()) .put("s3.path-style-access", "true") .put("hive.non-managed-table-writes-enabled", "true") + .put("hive.metastore", "file") + .put("hive.metastore.catalog.dir", "s3://%s/catalog".formatted(BUCKET)) .buildOrThrow()) - .setOpenTelemetry(openTelemetry) .setInitialSchemasLocationBase("s3://" + BUCKET) .build(); } @@ -101,7 +86,6 @@ protected QueryRunner createQueryRunner() public void tearDown() { // closed by closeAfterClass - spanExporter = null; minio = null; } @@ -181,25 +165,34 @@ private void assertFileSystemAccesses(@Language("SQL") String query, Multiset expectedAccesses) { DistributedQueryRunner queryRunner = getDistributedQueryRunner(); - spanExporter.reset(); queryRunner.executeWithQueryId(session, query); - assertMultisetsEqual(getOperations(), expectedAccesses); + assertMultisetsEqual(getOperations(queryRunner.getSpans()), expectedAccesses); } - private Multiset getOperations() + private static Multiset getOperations(List items) { - List items = spanExporter.getFinishedSpanItems(); Map spansById = uniqueIndex(items, SpanData::getSpanId); return items.stream() .filter(span -> span.getName().startsWith("S3.")) - .filter(span -> Optional.ofNullable(span.getParentSpanId()) - .map(spansById::get) - .map(parent -> !parent.getName().startsWith("HiveMetastore.")) - .orElse(true)) + .filter(span -> !hasAncestor(span, spansById, parent -> parent.getName().startsWith("HiveMetastore."))) .map(SpanData::getName) .collect(toCollection(HashMultiset::create)); } + private static boolean hasAncestor(SpanData span, Map spansById, Predicate predicate) + { + while (true) { + SpanData parent = spansById.get(span.getParentSpanId()); + if (parent == null) { + return false; + } + if (predicate.test(parent)) { + return true; + } + span = parent; + } + } + private static int occurrences(StorageFormat tableType, int orcValue, int parquetValue) { checkArgument(!(orcValue == parquetValue), "No need to use Occurrences when ORC and Parquet"); From 11a024fd08b23725eee8090fa20e3665c7de21d5 Mon Sep 17 00:00:00 2001 From: David Phillips Date: Sun, 3 Dec 2023 10:10:21 +0800 Subject: [PATCH 082/350] Make metastore method names unique --- ...HiveMetastoreBackedDeltaLakeMetastore.java | 2 +- ...estDeltaLakeMetastoreAccessOperations.java | 4 +- .../plugin/hive/HiveMetastoreClosure.java | 20 +++--- .../metastore/ForwardingHiveMetastore.java | 20 +++--- .../plugin/hive/metastore/HiveMetastore.java | 10 +-- .../SemiTransactionalHiveMetastore.java | 12 ++-- .../metastore/cache/CachingHiveMetastore.java | 20 +++--- .../metastore/file/FileHiveMetastore.java | 16 ++--- .../metastore/glue/GlueHiveMetastore.java | 8 +-- .../thrift/BridgingHiveMetastore.java | 12 ++-- .../tracing/TracingHiveMetastore.java | 30 ++++---- .../hive/procedure/DropStatsProcedure.java | 4 +- .../trino/plugin/hive/AbstractTestHive.java | 18 ++--- .../plugin/hive/AbstractTestHiveLocal.java | 2 +- .../CountingAccessHiveMetastore.java | 26 +++---- .../hive/metastore/MetastoreMethod.java | 15 +++- .../hive/metastore/TestMetastoreMethod.java | 70 +++++++++++++++++++ .../cache/TestCachingHiveMetastore.java | 14 ++-- .../metastore/glue/TestGlueHiveMetastore.java | 8 +-- ...astoreMetadataQueriesAccessOperations.java | 44 ++++++------ .../io/trino/plugin/hudi/HudiMetadata.java | 2 +- .../iceberg/catalog/hms/TrinoHiveCatalog.java | 2 +- .../iceberg/TestIcebergMetadataListing.java | 2 +- .../TestIcebergMetastoreAccessOperations.java | 10 +-- 24 files changed, 225 insertions(+), 146 deletions(-) create mode 100644 plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/TestMetastoreMethod.java diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/metastore/HiveMetastoreBackedDeltaLakeMetastore.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/metastore/HiveMetastoreBackedDeltaLakeMetastore.java index df3a1c1a8f150..854512b93352c 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/metastore/HiveMetastoreBackedDeltaLakeMetastore.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/metastore/HiveMetastoreBackedDeltaLakeMetastore.java @@ -63,7 +63,7 @@ public List getAllTables(String databaseName) // it would be nice to filter out non-Delta tables; however, we can not call // metastore.getTablesWithParameter(schema, TABLE_PROVIDER_PROP, TABLE_PROVIDER_VALUE), because that property // contains a dot and must be compared case-insensitive - return delegate.getAllTables(databaseName); + return delegate.getTables(databaseName); } @Override diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/TestDeltaLakeMetastoreAccessOperations.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/TestDeltaLakeMetastoreAccessOperations.java index d739bba8baf9c..473dd5f6179b8 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/TestDeltaLakeMetastoreAccessOperations.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/TestDeltaLakeMetastoreAccessOperations.java @@ -39,9 +39,9 @@ import static io.trino.plugin.hive.metastore.MetastoreMethod.CREATE_TABLE; import static io.trino.plugin.hive.metastore.MetastoreMethod.DROP_TABLE; import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_ALL_DATABASES; -import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_ALL_TABLES_FROM_DATABASE; import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_DATABASE; import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_TABLE; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_TABLES; import static io.trino.plugin.hive.metastore.file.TestingFileHiveMetastore.createTestingFileHiveMetastore; import static io.trino.testing.TestingSession.testSessionBuilder; import static java.util.Objects.requireNonNull; @@ -263,7 +263,7 @@ public void testShowTables() assertMetastoreInvocations("SHOW TABLES", ImmutableMultiset.builder() .add(GET_ALL_DATABASES) - .add(GET_ALL_TABLES_FROM_DATABASE) + .add(GET_TABLES) .build()); } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetastoreClosure.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetastoreClosure.java index cdbeee24b11a9..13d05eb634043 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetastoreClosure.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetastoreClosure.java @@ -140,13 +140,13 @@ public void updateTableStatistics(String databaseName, delegate.updateTableStatistics(databaseName, tableName, transaction, update); } - public void updatePartitionStatistics(String databaseName, + public void updatePartitionsStatistics(String databaseName, String tableName, String partitionName, Function update) { Table table = getExistingTable(databaseName, tableName); - delegate.updatePartitionStatistics(table, partitionName, update); + delegate.updatePartitionsStatistics(table, partitionName, update); } public void updatePartitionStatistics(String databaseName, String tableName, Map> updates) @@ -155,9 +155,9 @@ public void updatePartitionStatistics(String databaseName, String tableName, Map delegate.updatePartitionStatistics(table, updates); } - public List getAllTables(String databaseName) + public List getTables(String databaseName) { - return delegate.getAllTables(databaseName); + return delegate.getTables(databaseName); } public Optional> getAllTables() @@ -170,14 +170,14 @@ public Map getRelationTypes(String databaseName) return delegate.getRelationTypes(databaseName); } - public Optional> getRelationTypes() + public Optional> getAllRelationTypes() { - return delegate.getRelationTypes(); + return delegate.getAllRelationTypes(); } - public List getAllViews(String databaseName) + public List getViews(String databaseName) { - return delegate.getAllViews(databaseName); + return delegate.getViews(databaseName); } public Optional> getAllViews() @@ -453,9 +453,9 @@ public boolean functionExists(SchemaFunctionName name, String signatureToken) return delegate.functionExists(name.getSchemaName(), name.getFunctionName(), signatureToken); } - public Collection getFunctions(String schemaName) + public Collection getAllFunctions(String schemaName) { - return delegate.getFunctions(schemaName); + return delegate.getAllFunctions(schemaName); } public Collection getFunctions(SchemaFunctionName name) diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/ForwardingHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/ForwardingHiveMetastore.java index 2a4c7a2701d60..b6374f7138c20 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/ForwardingHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/ForwardingHiveMetastore.java @@ -95,12 +95,12 @@ public void updateTableStatistics( } @Override - public void updatePartitionStatistics( + public void updatePartitionsStatistics( Table table, String partitionName, Function update) { - delegate.updatePartitionStatistics(table, partitionName, update); + delegate.updatePartitionsStatistics(table, partitionName, update); } @Override @@ -112,9 +112,9 @@ public void updatePartitionStatistics( } @Override - public List getAllTables(String databaseName) + public List getTables(String databaseName) { - return delegate.getAllTables(databaseName); + return delegate.getTables(databaseName); } @Override @@ -130,9 +130,9 @@ public Map getRelationTypes(String databaseName) } @Override - public Optional> getRelationTypes() + public Optional> getAllRelationTypes() { - return delegate.getRelationTypes(); + return delegate.getAllRelationTypes(); } @Override @@ -142,9 +142,9 @@ public List getTablesWithParameter(String databaseName, String parameter } @Override - public List getAllViews(String databaseName) + public List getViews(String databaseName) { - return delegate.getAllViews(databaseName); + return delegate.getViews(databaseName); } @Override @@ -472,9 +472,9 @@ public boolean functionExists(String databaseName, String functionName, String s } @Override - public Collection getFunctions(String databaseName) + public Collection getAllFunctions(String databaseName) { - return delegate.getFunctions(databaseName); + return delegate.getAllFunctions(databaseName); } @Override diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/HiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/HiveMetastore.java index 2a821d23f29ee..c852b8adf77d9 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/HiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/HiveMetastore.java @@ -56,14 +56,14 @@ public interface HiveMetastore void updateTableStatistics(String databaseName, String tableName, AcidTransaction transaction, Function update); - default void updatePartitionStatistics(Table table, String partitionName, Function update) + default void updatePartitionsStatistics(Table table, String partitionName, Function update) { updatePartitionStatistics(table, ImmutableMap.of(partitionName, update)); } void updatePartitionStatistics(Table table, Map> updates); - List getAllTables(String databaseName); + List getTables(String databaseName); /** * @return List of tables, views and materialized views names from all schemas or Optional.empty if operation is not supported @@ -75,14 +75,14 @@ default void updatePartitionStatistics(Table table, String partitionName, Functi /** * @return empty if operation is not supported */ - Optional> getRelationTypes(); + Optional> getAllRelationTypes(); List getTablesWithParameter(String databaseName, String parameterKey, String parameterValue); /** * Lists views and materialized views from given database. */ - List getAllViews(String databaseName); + List getViews(String databaseName); /** * @return List of views including materialized views names from all schemas or Optional.empty if operation is not supported @@ -244,7 +244,7 @@ default void alterTransactionalTable(Table table, long transactionId, long write boolean functionExists(String databaseName, String functionName, String signatureToken); - Collection getFunctions(String databaseName); + Collection getAllFunctions(String databaseName); Collection getFunctions(String databaseName, String functionName); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/SemiTransactionalHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/SemiTransactionalHiveMetastore.java index cabc1de51b45a..838071d7a5de0 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/SemiTransactionalHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/SemiTransactionalHiveMetastore.java @@ -240,7 +240,7 @@ public synchronized List getAllTables(String databaseName) if (!tableActions.isEmpty()) { throw new UnsupportedOperationException("Listing all tables after adding/dropping/altering tables/views in a transaction is not supported"); } - return delegate.getAllTables(databaseName); + return delegate.getTables(databaseName); } public synchronized Optional> getAllTables() @@ -267,7 +267,7 @@ public synchronized Optional> getRelationType if (!tableActions.isEmpty()) { throw new UnsupportedOperationException("Listing all relations after adding/dropping/altering tables/views in a transaction is not supported"); } - return delegate.getRelationTypes(); + return delegate.getAllRelationTypes(); } public synchronized Optional
getTable(String databaseName, String tableName) @@ -441,7 +441,7 @@ public synchronized List getAllViews(String databaseName) if (!tableActions.isEmpty()) { throw new UnsupportedOperationException("Listing all tables after adding/dropping/altering tables/views in a transaction is not supported"); } - return delegate.getAllViews(databaseName); + return delegate.getViews(databaseName); } public synchronized Optional> getAllViews() @@ -1236,7 +1236,7 @@ public synchronized boolean functionExists(SchemaFunctionName name, String signa public synchronized Collection getFunctions(String schemaName) { checkReadable(); - return delegate.getFunctions(schemaName); + return delegate.getAllFunctions(schemaName); } public synchronized Collection getFunctions(SchemaFunctionName name) @@ -3365,7 +3365,7 @@ public UpdateStatisticsOperation(SchemaTableName tableName, Optional par public void run(HiveMetastoreClosure metastore, AcidTransaction transaction) { if (partitionName.isPresent()) { - metastore.updatePartitionStatistics(tableName.getSchemaName(), tableName.getTableName(), partitionName.get(), this::updateStatistics); + metastore.updatePartitionsStatistics(tableName.getSchemaName(), tableName.getTableName(), partitionName.get(), this::updateStatistics); } else { metastore.updateTableStatistics(tableName.getSchemaName(), tableName.getTableName(), transaction, this::updateStatistics); @@ -3379,7 +3379,7 @@ public void undo(HiveMetastoreClosure metastore, AcidTransaction transaction) return; } if (partitionName.isPresent()) { - metastore.updatePartitionStatistics(tableName.getSchemaName(), tableName.getTableName(), partitionName.get(), this::resetStatistics); + metastore.updatePartitionsStatistics(tableName.getSchemaName(), tableName.getTableName(), partitionName.get(), this::resetStatistics); } else { metastore.updateTableStatistics(tableName.getSchemaName(), tableName.getTableName(), transaction, this::resetStatistics); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/CachingHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/CachingHiveMetastore.java index 4c7bc9c86f839..a0ea2cfea32ce 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/CachingHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/CachingHiveMetastore.java @@ -559,10 +559,10 @@ public void updateTableStatistics(String databaseName, } @Override - public void updatePartitionStatistics(Table table, String partitionName, Function update) + public void updatePartitionsStatistics(Table table, String partitionName, Function update) { try { - delegate.updatePartitionStatistics(table, partitionName, update); + delegate.updatePartitionsStatistics(table, partitionName, update); } finally { HivePartitionName hivePartitionName = hivePartitionName(hiveTableName(table.getDatabaseName(), table.getTableName()), partitionName); @@ -589,7 +589,7 @@ public void updatePartitionStatistics(Table table, Map getAllTables(String databaseName) + public List getTables(String databaseName) { Map relationTypes = relationTypesCache.getIfPresent(databaseName); if (relationTypes != null) { @@ -600,7 +600,7 @@ public List getAllTables(String databaseName) private List loadAllTables(String databaseName) { - return delegate.getAllTables(databaseName); + return delegate.getTables(databaseName); } @Override @@ -630,14 +630,14 @@ private Map loadRelationTypes(String databaseName) } @Override - public Optional> getRelationTypes() + public Optional> getAllRelationTypes() { return getOptional(allRelationTypesCache, SingletonCacheKey.INSTANCE); } private Optional> loadRelationTypes() { - return delegate.getRelationTypes(); + return delegate.getAllRelationTypes(); } @Override @@ -653,14 +653,14 @@ private List loadTablesMatchingParameter(TablesWithParameterCacheKey key } @Override - public List getAllViews(String databaseName) + public List getViews(String databaseName) { return get(viewNamesCache, databaseName); } private List loadAllViews(String databaseName) { - return delegate.getAllViews(databaseName); + return delegate.getViews(databaseName); } @Override @@ -1204,9 +1204,9 @@ public boolean functionExists(String databaseName, String functionName, String s } @Override - public Collection getFunctions(String databaseName) + public Collection getAllFunctions(String databaseName) { - return delegate.getFunctions(databaseName); + return delegate.getAllFunctions(databaseName); } @Override diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/file/FileHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/file/FileHiveMetastore.java index 95d195032f728..efd6ab1b7d22b 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/file/FileHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/file/FileHiveMetastore.java @@ -227,7 +227,7 @@ public synchronized void dropDatabase(String databaseName, boolean deleteData) databaseName = databaseName.toLowerCase(ENGLISH); getRequiredDatabase(databaseName); - if (!getAllTables(databaseName).isEmpty()) { + if (!getTables(databaseName).isEmpty()) { throw new TrinoException(HIVE_METASTORE_ERROR, "Database " + databaseName + " is not empty"); } @@ -532,7 +532,7 @@ public synchronized void updatePartitionStatistics(Table table, Map getAllTables(String databaseName) + public synchronized List getTables(String databaseName) { return listAllTables(databaseName).stream() .filter(hideDeltaLakeTables @@ -551,13 +551,13 @@ public Optional> getAllTables() public synchronized Map getRelationTypes(String databaseName) { ImmutableMap.Builder relationTypes = ImmutableMap.builder(); - getAllTables(databaseName).forEach(name -> relationTypes.put(name, RelationType.TABLE)); - getAllViews(databaseName).forEach(name -> relationTypes.put(name, RelationType.VIEW)); + getTables(databaseName).forEach(name -> relationTypes.put(name, RelationType.TABLE)); + getViews(databaseName).forEach(name -> relationTypes.put(name, RelationType.VIEW)); return relationTypes.buildKeepingLast(); } @Override - public Optional> getRelationTypes() + public Optional> getAllRelationTypes() { return Optional.empty(); } @@ -627,9 +627,9 @@ private List doListAllTables(String databaseName) } @Override - public synchronized List getAllViews(String databaseName) + public synchronized List getViews(String databaseName) { - return getAllTables(databaseName).stream() + return getTables(databaseName).stream() .map(tableName -> getTable(databaseName, tableName)) .filter(Optional::isPresent) .map(Optional::get) @@ -1271,7 +1271,7 @@ public synchronized boolean functionExists(String databaseName, String functionN } @Override - public synchronized Collection getFunctions(String databaseName) + public synchronized Collection getAllFunctions(String databaseName) { return getFunctions(databaseName, Optional.empty()); } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueHiveMetastore.java index e1efe5654fcd5..e7ba1fab181c7 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueHiveMetastore.java @@ -424,7 +424,7 @@ private void updatePartitionStatisticsBatch(Table table, Map getAllTables(String databaseName) + public List getTables(String databaseName) { return getTableNames(databaseName, tableFilter); } @@ -461,7 +461,7 @@ public Map getRelationTypes(String databaseName) } @Override - public Optional> getRelationTypes() + public Optional> getAllRelationTypes() { return Optional.empty(); } @@ -473,7 +473,7 @@ public List getTablesWithParameter(String databaseName, String parameter } @Override - public List getAllViews(String databaseName) + public List getViews(String databaseName) { return getTableNames(databaseName, SOME_KIND_OF_VIEW_FILTER); } @@ -1273,7 +1273,7 @@ public boolean functionExists(String databaseName, String functionName, String s } @Override - public Collection getFunctions(String databaseName) + public Collection getAllFunctions(String databaseName) { return getFunctionsByPattern(databaseName, "trino__.*"); } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/BridgingHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/BridgingHiveMetastore.java index 5cd62ce4babf1..443aa3c04ac9c 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/BridgingHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/BridgingHiveMetastore.java @@ -145,7 +145,7 @@ public void updatePartitionStatistics(Table table, Map getAllTables(String databaseName) + public List getTables(String databaseName) { return delegate.getAllTables(databaseName); } @@ -154,8 +154,8 @@ public List getAllTables(String databaseName) public Map getRelationTypes(String databaseName) { ImmutableMap.Builder relationTypes = ImmutableMap.builder(); - getAllTables(databaseName).forEach(name -> relationTypes.put(name, RelationType.TABLE)); - getAllViews(databaseName).forEach(name -> relationTypes.put(name, RelationType.VIEW)); + getTables(databaseName).forEach(name -> relationTypes.put(name, RelationType.TABLE)); + getViews(databaseName).forEach(name -> relationTypes.put(name, RelationType.VIEW)); return relationTypes.buildKeepingLast(); } @@ -166,7 +166,7 @@ public List getTablesWithParameter(String databaseName, String parameter } @Override - public List getAllViews(String databaseName) + public List getViews(String databaseName) { return delegate.getAllViews(databaseName); } @@ -178,7 +178,7 @@ public Optional> getAllTables() } @Override - public Optional> getRelationTypes() + public Optional> getAllRelationTypes() { return getAllTables().flatMap(relations -> getAllViews().map(views -> { ImmutableMap.Builder relationTypes = ImmutableMap.builder(); @@ -611,7 +611,7 @@ public boolean functionExists(String databaseName, String functionName, String s } @Override - public Collection getFunctions(String databaseName) + public Collection getAllFunctions(String databaseName) { return getFunctionsByPattern(databaseName, "trino__*"); } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/tracing/TracingHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/tracing/TracingHiveMetastore.java index c5d9fb8b95b8f..18373dd1b45ec 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/tracing/TracingHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/tracing/TracingHiveMetastore.java @@ -149,14 +149,14 @@ public void updateTableStatistics(String databaseName, String tableName, AcidTra } @Override - public void updatePartitionStatistics(Table table, String partitionName, Function update) + public void updatePartitionsStatistics(Table table, String partitionName, Function update) { - Span span = tracer.spanBuilder("HiveMetastore.updatePartitionStatistics") + Span span = tracer.spanBuilder("HiveMetastore.updatePartitionsStatistics") .setAttribute(SCHEMA, table.getDatabaseName()) .setAttribute(TABLE, table.getTableName()) .setAttribute(PARTITION, partitionName) .startSpan(); - withTracing(span, () -> delegate.updatePartitionStatistics(table, partitionName, update)); + withTracing(span, () -> delegate.updatePartitionsStatistics(table, partitionName, update)); } @Override @@ -170,13 +170,13 @@ public void updatePartitionStatistics(Table table, Map getAllTables(String databaseName) + public List getTables(String databaseName) { - Span span = tracer.spanBuilder("HiveMetastore.getAllTables") + Span span = tracer.spanBuilder("HiveMetastore.getTables") .setAttribute(SCHEMA, databaseName) .startSpan(); return withTracing(span, () -> { - List tables = delegate.getAllTables(databaseName); + List tables = delegate.getTables(databaseName); span.setAttribute(TABLE_RESPONSE_COUNT, tables.size()); return tables; }); @@ -208,12 +208,12 @@ public Map getRelationTypes(String databaseName) } @Override - public Optional> getRelationTypes() + public Optional> getAllRelationTypes() { - Span span = tracer.spanBuilder("HiveMetastore.getRelations") + Span span = tracer.spanBuilder("HiveMetastore.getAllRelationTypes") .startSpan(); return withTracing(span, () -> { - Optional> relationTypes = delegate.getRelationTypes(); + Optional> relationTypes = delegate.getAllRelationTypes(); relationTypes.ifPresent(map -> span.setAttribute(TABLE_RESPONSE_COUNT, map.size())); return relationTypes; }); @@ -234,13 +234,13 @@ public List getTablesWithParameter(String databaseName, String parameter } @Override - public List getAllViews(String databaseName) + public List getViews(String databaseName) { - Span span = tracer.spanBuilder("HiveMetastore.getAllViews") + Span span = tracer.spanBuilder("HiveMetastore.getViews") .setAttribute(SCHEMA, databaseName) .startSpan(); return withTracing(span, () -> { - List views = delegate.getAllViews(databaseName); + List views = delegate.getViews(databaseName); span.setAttribute(TABLE_RESPONSE_COUNT, views.size()); return views; }); @@ -702,13 +702,13 @@ public boolean functionExists(String databaseName, String functionName, String s } @Override - public Collection getFunctions(String databaseName) + public Collection getAllFunctions(String databaseName) { - Span span = tracer.spanBuilder("HiveMetastore.getFunctions") + Span span = tracer.spanBuilder("HiveMetastore.getAllFunctions") .setAttribute(SCHEMA, databaseName) .startSpan(); return withTracing(span, () -> { - Collection functions = delegate.getFunctions(databaseName); + Collection functions = delegate.getAllFunctions(databaseName); span.setAttribute(FUNCTION_RESPONSE_COUNT, functions.size()); return functions; }); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/DropStatsProcedure.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/DropStatsProcedure.java index be2e8e2954412..157bde3e95b90 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/DropStatsProcedure.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/DropStatsProcedure.java @@ -125,7 +125,7 @@ private void doDropStats(ConnectorSession session, ConnectorAccessControl access .collect(toImmutableList()); validatePartitions(partitionStringValues, partitionColumns); - partitionStringValues.forEach(values -> metastore.updatePartitionStatistics( + partitionStringValues.forEach(values -> metastore.updatePartitionsStatistics( schema, table, makePartName(partitionColumns, values), @@ -144,7 +144,7 @@ private void doDropStats(ConnectorSession session, ConnectorAccessControl access else { // the table is partitioned; remove stats for every partition metastore.getPartitionNamesByFilter(handle.getSchemaName(), handle.getTableName(), partitionColumns, TupleDomain.all()) - .ifPresent(partitions -> partitions.forEach(partitionName -> metastore.updatePartitionStatistics( + .ifPresent(partitions -> partitions.forEach(partitionName -> metastore.updatePartitionsStatistics( schema, table, partitionName, diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHive.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHive.java index f5c2468d71370..c5a1c2d041c06 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHive.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHive.java @@ -3628,8 +3628,8 @@ protected void createDummyPartitionedTable(SchemaTableName tableName, List new PartitionWithStatistics(createDummyPartition(table, partitionName), partitionName, PartitionStatistics.empty())) .collect(toImmutableList()); metastoreClient.addPartitions(tableName.getSchemaName(), tableName.getTableName(), partitions); - metastoreClient.updatePartitionStatistics(tableName.getSchemaName(), tableName.getTableName(), firstPartitionName, currentStatistics -> ZERO_TABLE_STATISTICS); - metastoreClient.updatePartitionStatistics(tableName.getSchemaName(), tableName.getTableName(), secondPartitionName, currentStatistics -> ZERO_TABLE_STATISTICS); + metastoreClient.updatePartitionsStatistics(tableName.getSchemaName(), tableName.getTableName(), firstPartitionName, currentStatistics -> ZERO_TABLE_STATISTICS); + metastoreClient.updatePartitionsStatistics(tableName.getSchemaName(), tableName.getTableName(), secondPartitionName, currentStatistics -> ZERO_TABLE_STATISTICS); } protected void testUpdatePartitionStatistics( @@ -3653,11 +3653,11 @@ protected void testUpdatePartitionStatistics( for (int i = 0; i < firstPartitionStatistics.size(); i++) { PartitionStatistics statisticsPartition1 = firstPartitionStatistics.get(i); PartitionStatistics statisticsPartition2 = secondPartitionStatistics.get(i); - metastoreClient.updatePartitionStatistics(tableName.getSchemaName(), tableName.getTableName(), firstPartitionName, actualStatistics -> { + metastoreClient.updatePartitionsStatistics(tableName.getSchemaName(), tableName.getTableName(), firstPartitionName, actualStatistics -> { assertThat(actualStatistics).isEqualTo(expectedStatisticsPartition1.get()); return statisticsPartition1; }); - metastoreClient.updatePartitionStatistics(tableName.getSchemaName(), tableName.getTableName(), secondPartitionName, actualStatistics -> { + metastoreClient.updatePartitionsStatistics(tableName.getSchemaName(), tableName.getTableName(), secondPartitionName, actualStatistics -> { assertThat(actualStatistics).isEqualTo(expectedStatisticsPartition2.get()); return statisticsPartition2; }); @@ -3669,11 +3669,11 @@ protected void testUpdatePartitionStatistics( assertThat(metastoreClient.getPartitionStatistics(tableName.getSchemaName(), tableName.getTableName(), ImmutableSet.of(firstPartitionName, secondPartitionName))) .isEqualTo(ImmutableMap.of(firstPartitionName, expectedStatisticsPartition1.get(), secondPartitionName, expectedStatisticsPartition2.get())); - metastoreClient.updatePartitionStatistics(tableName.getSchemaName(), tableName.getTableName(), firstPartitionName, currentStatistics -> { + metastoreClient.updatePartitionsStatistics(tableName.getSchemaName(), tableName.getTableName(), firstPartitionName, currentStatistics -> { assertThat(currentStatistics).isEqualTo(expectedStatisticsPartition1.get()); return initialStatistics; }); - metastoreClient.updatePartitionStatistics(tableName.getSchemaName(), tableName.getTableName(), secondPartitionName, currentStatistics -> { + metastoreClient.updatePartitionsStatistics(tableName.getSchemaName(), tableName.getTableName(), secondPartitionName, currentStatistics -> { assertThat(currentStatistics).isEqualTo(expectedStatisticsPartition2.get()); return initialStatistics; }); @@ -3798,8 +3798,8 @@ protected void testPartitionStatisticsSampling(List columns, Par try { createDummyPartitionedTable(tableName, columns); HiveMetastoreClosure metastoreClient = new HiveMetastoreClosure(getMetastoreClient(), TESTING_TYPE_MANAGER, false); - metastoreClient.updatePartitionStatistics(tableName.getSchemaName(), tableName.getTableName(), "ds=2016-01-01", actualStatistics -> statistics); - metastoreClient.updatePartitionStatistics(tableName.getSchemaName(), tableName.getTableName(), "ds=2016-01-02", actualStatistics -> statistics); + metastoreClient.updatePartitionsStatistics(tableName.getSchemaName(), tableName.getTableName(), "ds=2016-01-01", actualStatistics -> statistics); + metastoreClient.updatePartitionsStatistics(tableName.getSchemaName(), tableName.getTableName(), "ds=2016-01-02", actualStatistics -> statistics); try (Transaction transaction = newTransaction()) { ConnectorSession session = newSession(); @@ -4911,7 +4911,7 @@ private void eraseStatistics(SchemaTableName schemaTableName) .map(Optional::get) .collect(toImmutableList()); for (Partition partition : partitions) { - metastoreClient.updatePartitionStatistics( + metastoreClient.updatePartitionsStatistics( table, makePartName(partitionColumns, partition.getValues()), statistics -> new PartitionStatistics(createEmptyStatistics(), ImmutableMap.of())); diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHiveLocal.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHiveLocal.java index a2cd60052b204..10d4b3130cdb8 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHiveLocal.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHiveLocal.java @@ -177,7 +177,7 @@ public void cleanup() throws IOException { try { - for (String tableName : metastoreClient.getAllTables(database)) { + for (String tableName : metastoreClient.getTables(database)) { metastoreClient.dropTable(database, tableName, true); } metastoreClient.dropDatabase(testDbName, true); diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/CountingAccessHiveMetastore.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/CountingAccessHiveMetastore.java index 08cf3c7834b66..8900fb7aade2b 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/CountingAccessHiveMetastore.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/CountingAccessHiveMetastore.java @@ -44,18 +44,18 @@ import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_ALL_DATABASES; import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_ALL_RELATION_TYPES; import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_ALL_TABLES; -import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_ALL_TABLES_FROM_DATABASE; import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_ALL_VIEWS; -import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_ALL_VIEWS_FROM_DATABASE; import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_DATABASE; import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_PARTITION; import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_PARTITIONS_BY_NAMES; import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_PARTITION_NAMES_BY_FILTER; import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_PARTITION_STATISTICS; -import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_RELATION_TYPES_FROM_DATABASE; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_RELATION_TYPES; import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_TABLE; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_TABLES; import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_TABLES_WITH_PARAMETER; import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_TABLE_STATISTICS; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_VIEWS; import static io.trino.plugin.hive.metastore.MetastoreMethod.REPLACE_TABLE; import static io.trino.plugin.hive.metastore.MetastoreMethod.UPDATE_PARTITION_STATISTICS; import static io.trino.plugin.hive.metastore.MetastoreMethod.UPDATE_TABLE_STATISTICS; @@ -118,10 +118,10 @@ public List getTablesWithParameter(String databaseName, String parameter } @Override - public List getAllViews(String databaseName) + public List getViews(String databaseName) { - methodInvocations.add(GET_ALL_VIEWS_FROM_DATABASE); - return delegate.getAllViews(databaseName); + methodInvocations.add(GET_VIEWS); + return delegate.getViews(databaseName); } @Override @@ -352,10 +352,10 @@ public void updatePartitionStatistics(Table table, Map getAllTables(String databaseName) + public List getTables(String databaseName) { - methodInvocations.add(GET_ALL_TABLES_FROM_DATABASE); - return delegate.getAllTables(databaseName); + methodInvocations.add(GET_TABLES); + return delegate.getTables(databaseName); } @Override @@ -371,14 +371,14 @@ public Optional> getAllTables() @Override public Map getRelationTypes(String databaseName) { - methodInvocations.add(GET_RELATION_TYPES_FROM_DATABASE); + methodInvocations.add(GET_RELATION_TYPES); return delegate.getRelationTypes(databaseName); } @Override - public Optional> getRelationTypes() + public Optional> getAllRelationTypes() { - Optional> relationTypes = delegate.getRelationTypes(); + Optional> relationTypes = delegate.getAllRelationTypes(); if (relationTypes.isPresent()) { methodInvocations.add(GET_ALL_RELATION_TYPES); } @@ -392,7 +392,7 @@ public boolean functionExists(String databaseName, String functionName, String s } @Override - public Collection getFunctions(String databaseName) + public Collection getAllFunctions(String databaseName) { throw new UnsupportedOperationException(); } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/MetastoreMethod.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/MetastoreMethod.java index fb7ce0e7db18e..30c6461a532b8 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/MetastoreMethod.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/MetastoreMethod.java @@ -13,6 +13,9 @@ */ package io.trino.plugin.hive.metastore; +import static com.google.common.base.CaseFormat.LOWER_CAMEL; +import static com.google.common.base.CaseFormat.UPPER_UNDERSCORE; + public enum MetastoreMethod { CREATE_DATABASE, @@ -22,13 +25,13 @@ public enum MetastoreMethod GET_DATABASE, GET_TABLE, GET_ALL_TABLES, - GET_ALL_TABLES_FROM_DATABASE, - GET_RELATION_TYPES_FROM_DATABASE, + GET_TABLES, + GET_RELATION_TYPES, GET_ALL_RELATION_TYPES, GET_TABLES_WITH_PARAMETER, GET_TABLE_STATISTICS, GET_ALL_VIEWS, - GET_ALL_VIEWS_FROM_DATABASE, + GET_VIEWS, UPDATE_TABLE_STATISTICS, ADD_PARTITIONS, GET_PARTITION_NAMES_BY_FILTER, @@ -38,4 +41,10 @@ public enum MetastoreMethod UPDATE_PARTITION_STATISTICS, REPLACE_TABLE, DROP_TABLE, + /**/; + + public static MetastoreMethod fromMethodName(String name) + { + return valueOf(LOWER_CAMEL.to(UPPER_UNDERSCORE, name)); + } } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/TestMetastoreMethod.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/TestMetastoreMethod.java new file mode 100644 index 0000000000000..88c02c4e6a207 --- /dev/null +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/TestMetastoreMethod.java @@ -0,0 +1,70 @@ +/* + * 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.hive.metastore; + +import org.junit.jupiter.api.Test; + +import java.lang.reflect.Method; +import java.util.Set; +import java.util.stream.Stream; + +import static com.google.common.base.CaseFormat.LOWER_CAMEL; +import static com.google.common.base.CaseFormat.UPPER_UNDERSCORE; +import static com.google.common.collect.ImmutableSetMultimap.toImmutableSetMultimap; +import static io.trino.plugin.hive.metastore.MetastoreMethod.DROP_TABLE; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_ALL_VIEWS; +import static io.trino.plugin.hive.metastore.MetastoreMethod.fromMethodName; +import static java.util.function.Function.identity; +import static java.util.stream.Collectors.toSet; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +class TestMetastoreMethod +{ + @Test + void testFromMethodName() + { + assertThat(fromMethodName("getAllViews")).isEqualTo(GET_ALL_VIEWS); + assertThat(fromMethodName("dropTable")).isEqualTo(DROP_TABLE); + } + + @Test + void testFromMethodNameInvalid() + { + assertThatThrownBy(() -> fromMethodName("doesNotExist")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("No enum constant io.trino.plugin.hive.metastore.MetastoreMethod.DOES_NOT_EXIST"); + } + + @Test + void testEnumNamesMapToMethods() + { + Set methodNames = Stream.of(HiveMetastore.class.getMethods()) + .map(Method::getName) + .collect(toSet()); + + for (MetastoreMethod method : MetastoreMethod.values()) { + assertThat(methodNames).contains(UPPER_UNDERSCORE.to(LOWER_CAMEL, method.name())); + } + } + + @Test + void testMethodNamesUnique() + { + Stream.of(HiveMetastore.class.getMethods()) + .collect(toImmutableSetMultimap(Method::getName, identity())) + .asMap().values().forEach(methods -> + assertThat(methods).hasSize(1)); + } +} diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastore.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastore.java index a9fe3a4f988a3..ba9cf74e0f27c 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastore.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastore.java @@ -167,7 +167,7 @@ public void testCachingWithOnlyPartitionsCacheEnabled() .usesCache(); assertThatCachingWithDisabledPartitionCache() - .whenExecuting(testedMetastore -> testedMetastore.getAllTables(TEST_DATABASE)) + .whenExecuting(testedMetastore -> testedMetastore.getTables(TEST_DATABASE)) .usesCache(); assertThatCachingWithDisabledPartitionCache() @@ -216,16 +216,16 @@ public void testGetAllDatabases() public void testGetAllTable() { assertThat(mockClient.getAccessCount()).isEqualTo(0); - assertThat(metastore.getAllTables(TEST_DATABASE)).isEqualTo(ImmutableList.of(TEST_TABLE)); + assertThat(metastore.getTables(TEST_DATABASE)).isEqualTo(ImmutableList.of(TEST_TABLE)); assertThat(mockClient.getAccessCount()).isEqualTo(1); - assertThat(metastore.getAllTables(TEST_DATABASE)).isEqualTo(ImmutableList.of(TEST_TABLE)); + assertThat(metastore.getTables(TEST_DATABASE)).isEqualTo(ImmutableList.of(TEST_TABLE)); assertThat(mockClient.getAccessCount()).isEqualTo(1); assertThat(metastore.getTableNamesStats().getRequestCount()).isEqualTo(2); assertThat(metastore.getTableNamesStats().getHitRate()).isEqualTo(0.5); metastore.flushCache(); - assertThat(metastore.getAllTables(TEST_DATABASE)).isEqualTo(ImmutableList.of(TEST_TABLE)); + assertThat(metastore.getTables(TEST_DATABASE)).isEqualTo(ImmutableList.of(TEST_TABLE)); assertThat(mockClient.getAccessCount()).isEqualTo(2); assertThat(metastore.getTableNamesStats().getRequestCount()).isEqualTo(3); assertThat(metastore.getTableNamesStats().getHitRate()).isEqualTo(1.0 / 3); @@ -239,7 +239,7 @@ public void testBatchGetAllTable() assertThat(mockClient.getAccessCount()).isEqualTo(1); assertThat(metastore.getAllTables()).isEqualTo(Optional.of(ImmutableList.of(TEST_SCHEMA_TABLE))); assertThat(mockClient.getAccessCount()).isEqualTo(1); - assertThat(metastore.getAllTables(TEST_DATABASE)).isEqualTo(ImmutableList.of(TEST_TABLE)); + assertThat(metastore.getTables(TEST_DATABASE)).isEqualTo(ImmutableList.of(TEST_TABLE)); assertThat(mockClient.getAccessCount()).isEqualTo(2); assertThat(metastore.getAllTableNamesStats().getRequestCount()).isEqualTo(2); assertThat(metastore.getAllTableNamesStats().getHitRate()).isEqualTo(.5); @@ -255,7 +255,7 @@ public void testBatchGetAllTable() @Test public void testInvalidDbGetAllTAbles() { - assertThat(metastore.getAllTables(BAD_DATABASE).isEmpty()).isTrue(); + assertThat(metastore.getTables(BAD_DATABASE).isEmpty()).isTrue(); } @Test @@ -929,7 +929,7 @@ public void testUpdatePartitionStatistics() Table table = hiveMetastoreClosure.getTable(TEST_DATABASE, TEST_TABLE).orElseThrow(); assertThat(mockClient.getAccessCount()).isEqualTo(1); - hiveMetastoreClosure.updatePartitionStatistics(table.getDatabaseName(), table.getTableName(), TEST_PARTITION1, identity()); + hiveMetastoreClosure.updatePartitionsStatistics(table.getDatabaseName(), table.getTableName(), TEST_PARTITION1, identity()); assertThat(mockClient.getAccessCount()).isEqualTo(5); } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/glue/TestGlueHiveMetastore.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/glue/TestGlueHiveMetastore.java index 5a1586a6eb6a1..6d95ddbb5deab 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/glue/TestGlueHiveMetastore.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/glue/TestGlueHiveMetastore.java @@ -395,8 +395,8 @@ public void testGetPartitionsWithFilterUsingReservedKeywordsAsColumnName() .map(partitionName -> new PartitionWithStatistics(createDummyPartition(table, partitionName), partitionName, PartitionStatistics.empty())) .collect(toImmutableList()); metastoreClient.addPartitions(tableName.getSchemaName(), tableName.getTableName(), partitions); - metastoreClient.updatePartitionStatistics(tableName.getSchemaName(), tableName.getTableName(), partitionName1, currentStatistics -> ZERO_TABLE_STATISTICS); - metastoreClient.updatePartitionStatistics(tableName.getSchemaName(), tableName.getTableName(), partitionName2, currentStatistics -> ZERO_TABLE_STATISTICS); + metastoreClient.updatePartitionsStatistics(tableName.getSchemaName(), tableName.getTableName(), partitionName1, currentStatistics -> ZERO_TABLE_STATISTICS); + metastoreClient.updatePartitionsStatistics(tableName.getSchemaName(), tableName.getTableName(), partitionName2, currentStatistics -> ZERO_TABLE_STATISTICS); Optional> partitionNames = metastoreClient.getPartitionNamesByFilter( tableName.getSchemaName(), @@ -1261,7 +1261,7 @@ public void testStatisticsPartitionedTableColumnModification() GlueHiveMetastore metastoreClient = (GlueHiveMetastore) getMetastoreClient(); double countBefore = metastoreClient.getStats().getBatchUpdatePartition().getTime().getAllTime().getCount(); - metastore.updatePartitionStatistics(tableName.getSchemaName(), tableName.getTableName(), "ds=2016-01-01", actualStatistics -> partitionStatistics); + metastore.updatePartitionsStatistics(tableName.getSchemaName(), tableName.getTableName(), "ds=2016-01-01", actualStatistics -> partitionStatistics); assertThat(metastoreClient.getStats().getBatchUpdatePartition().getTime().getAllTime().getCount()).isEqualTo(countBefore + 1); PartitionStatistics tableStatistics = new PartitionStatistics(createEmptyStatistics(), Map.of()); @@ -1554,7 +1554,7 @@ private void createDummyPartitionedTable(SchemaTableName tableName, List metastoreClient.updatePartitionStatistics( + partitionName -> metastoreClient.updatePartitionsStatistics( tableName.getSchemaName(), tableName.getTableName(), partitionName, currentStatistics -> ZERO_TABLE_STATISTICS)); } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreMetadataQueriesAccessOperations.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreMetadataQueriesAccessOperations.java index eba36f5754dbd..930c462cb6cd2 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreMetadataQueriesAccessOperations.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreMetadataQueriesAccessOperations.java @@ -45,11 +45,11 @@ import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_ALL_DATABASES; import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_ALL_RELATION_TYPES; import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_ALL_TABLES; -import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_ALL_TABLES_FROM_DATABASE; import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_ALL_VIEWS; -import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_ALL_VIEWS_FROM_DATABASE; -import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_RELATION_TYPES_FROM_DATABASE; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_RELATION_TYPES; import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_TABLE; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_TABLES; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_VIEWS; import static io.trino.plugin.hive.metastore.PrincipalPrivileges.NO_PRIVILEGES; import static io.trino.plugin.hive.metastore.StorageFormat.fromHiveStorageFormat; import static io.trino.testing.TestingSession.testSessionBuilder; @@ -171,12 +171,12 @@ public void testSelectTablesWithFilterBySchema() assertMetastoreInvocations( "SELECT * FROM information_schema.tables WHERE table_schema = 'test_schema_0'", ImmutableMultiset.builder() - .add(GET_RELATION_TYPES_FROM_DATABASE) + .add(GET_RELATION_TYPES) .build()); assertMetastoreInvocations( "SELECT * FROM system.jdbc.tables WHERE table_schem = 'test_schema_0'", ImmutableMultiset.builder() - .add(GET_RELATION_TYPES_FROM_DATABASE) + .add(GET_RELATION_TYPES) .build()); } @@ -251,10 +251,10 @@ public void testSelectViewsWithFilterByInformationSchema() @Test public void testSelectViewsWithFilterBySchema() { - assertMetastoreInvocations("SELECT * FROM information_schema.views WHERE table_schema = 'test_schema_0'", ImmutableMultiset.of(GET_ALL_VIEWS_FROM_DATABASE)); + assertMetastoreInvocations("SELECT * FROM information_schema.views WHERE table_schema = 'test_schema_0'", ImmutableMultiset.of(GET_VIEWS)); assertMetastoreInvocations("SELECT * FROM system.jdbc.tables WHERE table_type = 'VIEW' AND table_schem = 'test_schema_0'", ImmutableMultiset.builder() - .add(GET_RELATION_TYPES_FROM_DATABASE) + .add(GET_RELATION_TYPES) .build()); } @@ -330,27 +330,27 @@ public void testSelectColumnsFilterBySchema() { assertMetastoreInvocations("SELECT * FROM information_schema.columns WHERE table_schema = 'test_schema_0'", ImmutableMultiset.builder() - .add(GET_ALL_TABLES_FROM_DATABASE) - .add(GET_ALL_VIEWS_FROM_DATABASE) + .add(GET_TABLES) + .add(GET_VIEWS) .addCopies(GET_TABLE, TEST_TABLES_IN_SCHEMA_COUNT) .build()); assertMetastoreInvocations("SELECT * FROM system.jdbc.columns WHERE table_schem = 'test_schema_0'", ImmutableMultiset.builder() - .add(GET_ALL_TABLES_FROM_DATABASE) - .add(GET_ALL_VIEWS_FROM_DATABASE) + .add(GET_TABLES) + .add(GET_VIEWS) .addCopies(GET_TABLE, TEST_TABLES_IN_SCHEMA_COUNT) .build()); assertMetastoreInvocations("SELECT * FROM system.jdbc.columns WHERE table_schem LIKE 'test\\_schema\\_0' ESCAPE '\\'", ImmutableMultiset.builder() - .add(GET_ALL_TABLES_FROM_DATABASE) - .add(GET_ALL_VIEWS_FROM_DATABASE) + .add(GET_TABLES) + .add(GET_VIEWS) .addCopies(GET_TABLE, TEST_TABLES_IN_SCHEMA_COUNT) .build()); assertMetastoreInvocations("SELECT * FROM system.jdbc.columns WHERE table_schem LIKE 'test_schema_0' ESCAPE '\\'", ImmutableMultiset.builder() .add(GET_ALL_DATABASES) - .add(GET_ALL_TABLES_FROM_DATABASE) - .add(GET_ALL_VIEWS_FROM_DATABASE) + .add(GET_TABLES) + .add(GET_VIEWS) .addCopies(GET_TABLE, TEST_TABLES_IN_SCHEMA_COUNT) .build()); } @@ -370,7 +370,7 @@ public void testSelectColumnsWithLikeOverSchema() "SELECT * FROM system.jdbc.columns WHERE table_schem LIKE 'test%'", ImmutableMultiset.builder() .add(GET_ALL_DATABASES) - .addCopies(GET_ALL_TABLES_FROM_DATABASE, TEST_SCHEMAS_COUNT) + .addCopies(GET_TABLES, TEST_SCHEMAS_COUNT) .addCopies(GET_TABLE, TEST_ALL_TABLES_COUNT) .build()); } @@ -392,22 +392,22 @@ public void testSelectColumnsFilterByTableName() "SELECT * FROM system.jdbc.columns WHERE table_name = 'test_table_0'", ImmutableMultiset.builder() .add(GET_ALL_DATABASES) - .add(GET_ALL_TABLES_FROM_DATABASE) + .add(GET_TABLES) .addCopies(GET_TABLE, TEST_SCHEMAS_COUNT + 1) .build()); assertMetastoreInvocations( "SELECT * FROM system.jdbc.columns WHERE table_name LIKE 'test\\_table\\_0' ESCAPE '\\'", ImmutableMultiset.builder() .add(GET_ALL_DATABASES) - .add(GET_ALL_TABLES_FROM_DATABASE) + .add(GET_TABLES) .addCopies(GET_TABLE, TEST_SCHEMAS_COUNT + 1) .build()); assertMetastoreInvocations( "SELECT * FROM system.jdbc.columns WHERE table_name LIKE 'test_table_0' ESCAPE '\\'", ImmutableMultiset.builder() .add(GET_ALL_DATABASES) - .add(GET_ALL_TABLES_FROM_DATABASE) - .addCopies(GET_ALL_TABLES_FROM_DATABASE, TEST_SCHEMAS_COUNT) + .add(GET_TABLES) + .addCopies(GET_TABLES, TEST_SCHEMAS_COUNT) .addCopies(GET_TABLE, TEST_SCHEMAS_COUNT) .build()); } @@ -425,7 +425,7 @@ public void testSelectColumnsWithLikeOverTableName() assertMetastoreInvocations("SELECT * FROM system.jdbc.columns WHERE table_name LIKE 'test%'", ImmutableMultiset.builder() .add(GET_ALL_DATABASES) - .addCopies(GET_ALL_TABLES_FROM_DATABASE, TEST_SCHEMAS_COUNT + 1) + .addCopies(GET_TABLES, TEST_SCHEMAS_COUNT + 1) .addCopies(GET_TABLE, TEST_ALL_TABLES_COUNT) .build()); } @@ -478,7 +478,7 @@ public void testSelectColumnsFilterByTableAndSchema() assertMetastoreInvocations("SELECT * FROM system.jdbc.columns WHERE table_schem LIKE 'test_schema_0' ESCAPE '\\' AND table_name LIKE 'test_table_0' ESCAPE '\\'", ImmutableMultiset.builder() .add(GET_ALL_DATABASES) - .add(GET_ALL_TABLES_FROM_DATABASE) + .add(GET_TABLES) .add(GET_TABLE) .build()); } 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 fbbc5e09cb4e7..152ca336bd8b0 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 @@ -205,7 +205,7 @@ public List listTables(ConnectorSession session, Optional tableNames = ImmutableList.builder(); for (String schemaName : listSchemas(session, optionalSchemaName)) { - for (String tableName : metastore.getAllTables(schemaName)) { + for (String tableName : metastore.getTables(schemaName)) { tableNames.add(new SchemaTableName(schemaName, tableName)); } } diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/TrinoHiveCatalog.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/TrinoHiveCatalog.java index 702fd0ccd699b..6d0027e1c26df 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/TrinoHiveCatalog.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/TrinoHiveCatalog.java @@ -360,7 +360,7 @@ public List listTables(ConnectorSession session, Optional tablesListBuilder = ImmutableSet.builder(); for (String schemaName : listNamespaces(session, namespace)) { - metastore.getAllTables(schemaName).forEach(tableName -> tablesListBuilder.add(new SchemaTableName(schemaName, tableName))); + metastore.getTables(schemaName).forEach(tableName -> tablesListBuilder.add(new SchemaTableName(schemaName, tableName))); } return tablesListBuilder.build().asList(); } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetadataListing.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetadataListing.java index 317b31e396609..b34ca4a5e5e8d 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetadataListing.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetadataListing.java @@ -102,7 +102,7 @@ public void tearDown() @Test public void testTableListing() { - assertThat(metastore.getAllTables("test_schema")) + assertThat(metastore.getTables("test_schema")) .containsExactlyInAnyOrder( "iceberg_table1", "iceberg_table2", diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetastoreAccessOperations.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetastoreAccessOperations.java index 61b592b6eca3a..7854a9a2778de 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetastoreAccessOperations.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetastoreAccessOperations.java @@ -35,9 +35,9 @@ import static com.google.inject.util.Modules.EMPTY_MODULE; import static io.trino.plugin.hive.metastore.MetastoreMethod.CREATE_TABLE; import static io.trino.plugin.hive.metastore.MetastoreMethod.DROP_TABLE; -import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_ALL_TABLES_FROM_DATABASE; import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_DATABASE; import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_TABLE; +import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_TABLES; import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_TABLES_WITH_PARAMETER; import static io.trino.plugin.hive.metastore.MetastoreMethod.REPLACE_TABLE; import static io.trino.plugin.hive.metastore.file.TestingFileHiveMetastore.createTestingFileHiveMetastore; @@ -398,7 +398,7 @@ public void testInformationSchemaColumns(int tables) // Bulk retrieval assertMetastoreInvocations(session, "SELECT * FROM information_schema.columns WHERE table_schema = CURRENT_SCHEMA AND table_name LIKE 'test_select_i_s_columns%'", ImmutableMultiset.builder() - .add(GET_ALL_TABLES_FROM_DATABASE) + .add(GET_TABLES) .addCopies(GET_TABLE, tables * 2) .addCopies(GET_TABLES_WITH_PARAMETER, 2) .build()); @@ -444,7 +444,7 @@ public void testSystemMetadataTableComments(int tables) // Bulk retrieval assertMetastoreInvocations(session, "SELECT * FROM system.metadata.table_comments WHERE schema_name = CURRENT_SCHEMA AND table_name LIKE 'test_select_s_m_t_comments%'", ImmutableMultiset.builder() - .add(GET_ALL_TABLES_FROM_DATABASE) + .add(GET_TABLES) .addCopies(GET_TABLE, tables * 2) .addCopies(GET_TABLES_WITH_PARAMETER, 2) .build()); @@ -452,7 +452,7 @@ public void testSystemMetadataTableComments(int tables) // Bulk retrieval for two schemas assertMetastoreInvocations(session, "SELECT * FROM system.metadata.table_comments WHERE schema_name IN (CURRENT_SCHEMA, 'non_existent') AND table_name LIKE 'test_select_s_m_t_comments%'", ImmutableMultiset.builder() - .addCopies(GET_ALL_TABLES_FROM_DATABASE, 2) + .addCopies(GET_TABLES, 2) .addCopies(GET_TABLES_WITH_PARAMETER, 4) .addCopies(GET_TABLE, tables * 2) .build()); @@ -538,7 +538,7 @@ public void testShowTables() assertMetastoreInvocations("SHOW TABLES", ImmutableMultiset.builder() .add(GET_DATABASE) - .add(GET_ALL_TABLES_FROM_DATABASE) + .add(GET_TABLES) .build()); } From 458bfd7dbff3ac3dafc72b06bba61b7b18179e79 Mon Sep 17 00:00:00 2001 From: David Phillips Date: Tue, 5 Dec 2023 17:07:49 +0800 Subject: [PATCH 083/350] Use tracing for metastore access tests --- ...DeltaLakePerTransactionMetastoreCache.java | 77 +--- ...estDeltaLakeMetastoreAccessOperations.java | 57 +-- .../CountingAccessHiveMetastore.java | 423 ------------------ ...oreUtil.java => MetastoreInvocations.java} | 28 +- .../hive/metastore/MetastoreMethod.java | 1 + .../TestHiveMetastoreAccessOperations.java | 43 +- ...astoreMetadataQueriesAccessOperations.java | 79 ++-- .../TestIcebergMetastoreAccessOperations.java | 32 +- 8 files changed, 101 insertions(+), 639 deletions(-) delete mode 100644 plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/CountingAccessHiveMetastore.java rename plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/{CountingAccessHiveMetastoreUtil.java => MetastoreInvocations.java} (50%) diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakePerTransactionMetastoreCache.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakePerTransactionMetastoreCache.java index 3dffcbc7d2802..244e00944e939 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakePerTransactionMetastoreCache.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakePerTransactionMetastoreCache.java @@ -16,65 +16,34 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMultiset; import com.google.common.collect.Multiset; -import com.google.inject.Binder; -import com.google.inject.Key; -import io.airlift.configuration.AbstractConfigurationAwareModule; -import io.trino.Session; import io.trino.plugin.base.util.Closables; -import io.trino.plugin.hive.metastore.CountingAccessHiveMetastore; -import io.trino.plugin.hive.metastore.CountingAccessHiveMetastoreUtil; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.metastore.MetastoreMethod; -import io.trino.plugin.hive.metastore.RawHiveMetastoreFactory; -import io.trino.plugin.hive.metastore.file.FileHiveMetastore; -import io.trino.plugin.tpch.TpchPlugin; import io.trino.testing.DistributedQueryRunner; -import io.trino.testing.QueryRunner; import org.intellij.lang.annotations.Language; import org.junit.jupiter.api.Test; -import java.nio.file.Path; -import java.util.Optional; +import java.util.HashMap; +import java.util.Map; import static io.trino.plugin.deltalake.DeltaLakeQueryRunner.DELTA_CATALOG; +import static io.trino.plugin.deltalake.DeltaLakeQueryRunner.createDeltaLakeQueryRunner; +import static io.trino.plugin.hive.metastore.MetastoreInvocations.assertMetastoreInvocationsForQuery; import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_TABLE; -import static io.trino.plugin.hive.metastore.file.TestingFileHiveMetastore.createTestingFileHiveMetastore; -import static io.trino.testing.TestingSession.testSessionBuilder; -import static java.util.Objects.requireNonNull; public class TestDeltaLakePerTransactionMetastoreCache { - private CountingAccessHiveMetastore metastore; - - private DistributedQueryRunner createQueryRunner(boolean enablePerTransactionHiveMetastoreCaching) + private static DistributedQueryRunner createQueryRunner(boolean enablePerTransactionHiveMetastoreCaching) throws Exception { - Session session = testSessionBuilder() - .setCatalog(DELTA_CATALOG) - .setSchema("default") - .build(); + Map deltaLakeProperties = new HashMap<>(); + deltaLakeProperties.put("delta.register-table-procedure.enabled", "true"); + if (!enablePerTransactionHiveMetastoreCaching) { + // almost disable the cache; 0 is not allowed as config property value + deltaLakeProperties.put("delta.per-transaction-metastore-cache-maximum-size", "1"); + } - DistributedQueryRunner queryRunner = DistributedQueryRunner.builder(session).build(); + DistributedQueryRunner queryRunner = createDeltaLakeQueryRunner(DELTA_CATALOG, ImmutableMap.of(), deltaLakeProperties); try { - Path dataDirectory = queryRunner.getCoordinator().getBaseDataDir().resolve("file-metastore"); - FileHiveMetastore fileMetastore = createTestingFileHiveMetastore(dataDirectory.toFile()); - metastore = new CountingAccessHiveMetastore(fileMetastore); - queryRunner.installPlugin(new TestingDeltaLakePlugin(dataDirectory, Optional.empty(), Optional.empty(), new CountingAccessMetastoreModule(metastore))); - - ImmutableMap.Builder deltaLakeProperties = ImmutableMap.builder(); - deltaLakeProperties.put("hive.metastore", "test"); // use test value so we do not get clash with default bindings) - deltaLakeProperties.put("delta.register-table-procedure.enabled", "true"); - if (!enablePerTransactionHiveMetastoreCaching) { - // almost disable the cache; 0 is not allowed as config property value - deltaLakeProperties.put("delta.per-transaction-metastore-cache-maximum-size", "1"); - } - - queryRunner.createCatalog(DELTA_CATALOG, "delta_lake", deltaLakeProperties.buildOrThrow()); - queryRunner.execute("CREATE SCHEMA " + session.getSchema().orElseThrow()); - - queryRunner.installPlugin(new TpchPlugin()); - queryRunner.createCatalog("tpch", "tpch"); - queryRunner.execute("CREATE TABLE nation AS SELECT * FROM tpch.tiny.nation"); queryRunner.execute("CREATE TABLE region AS SELECT * FROM tpch.tiny.region"); } @@ -86,24 +55,6 @@ private DistributedQueryRunner createQueryRunner(boolean enablePerTransactionHiv return queryRunner; } - private static class CountingAccessMetastoreModule - extends AbstractConfigurationAwareModule - { - private final CountingAccessHiveMetastore metastore; - - public CountingAccessMetastoreModule(CountingAccessHiveMetastore metastore) - { - this.metastore = requireNonNull(metastore, "metastore is null"); - } - - @Override - protected void setup(Binder binder) - { - binder.bind(HiveMetastoreFactory.class).annotatedWith(RawHiveMetastoreFactory.class).toInstance(HiveMetastoreFactory.ofInstance(metastore)); - binder.bind(Key.get(boolean.class, AllowDeltaLakeManagedTableRename.class)).toInstance(false); - } - } - @Test public void testPerTransactionHiveMetastoreCachingEnabled() throws Exception @@ -129,8 +80,8 @@ public void testPerTransactionHiveMetastoreCachingDisabled() } } - private void assertMetastoreInvocations(QueryRunner queryRunner, @Language("SQL") String query, Multiset expectedInvocations) + private static void assertMetastoreInvocations(DistributedQueryRunner queryRunner, @Language("SQL") String query, Multiset expectedInvocations) { - CountingAccessHiveMetastoreUtil.assertMetastoreInvocations(metastore, queryRunner, queryRunner.getDefaultSession(), query, expectedInvocations); + assertMetastoreInvocationsForQuery(queryRunner, queryRunner.getDefaultSession(), query, expectedInvocations); } } diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/TestDeltaLakeMetastoreAccessOperations.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/TestDeltaLakeMetastoreAccessOperations.java index 473dd5f6179b8..8c8354c3603e2 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/TestDeltaLakeMetastoreAccessOperations.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/TestDeltaLakeMetastoreAccessOperations.java @@ -16,84 +16,39 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMultiset; import com.google.common.collect.Multiset; -import com.google.inject.Binder; -import com.google.inject.Key; -import io.airlift.configuration.AbstractConfigurationAwareModule; -import io.trino.Session; -import io.trino.plugin.deltalake.AllowDeltaLakeManagedTableRename; -import io.trino.plugin.deltalake.TestingDeltaLakePlugin; -import io.trino.plugin.hive.metastore.CountingAccessHiveMetastore; -import io.trino.plugin.hive.metastore.CountingAccessHiveMetastoreUtil; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.metastore.MetastoreMethod; -import io.trino.plugin.hive.metastore.RawHiveMetastoreFactory; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.DistributedQueryRunner; import org.intellij.lang.annotations.Language; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.parallel.Execution; -import java.nio.file.Path; -import java.util.Optional; +import java.util.Map; +import static io.trino.plugin.deltalake.DeltaLakeQueryRunner.DELTA_CATALOG; +import static io.trino.plugin.deltalake.DeltaLakeQueryRunner.createDeltaLakeQueryRunner; +import static io.trino.plugin.hive.metastore.MetastoreInvocations.assertMetastoreInvocationsForQuery; import static io.trino.plugin.hive.metastore.MetastoreMethod.CREATE_TABLE; import static io.trino.plugin.hive.metastore.MetastoreMethod.DROP_TABLE; import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_ALL_DATABASES; import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_DATABASE; import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_TABLE; import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_TABLES; -import static io.trino.plugin.hive.metastore.file.TestingFileHiveMetastore.createTestingFileHiveMetastore; -import static io.trino.testing.TestingSession.testSessionBuilder; -import static java.util.Objects.requireNonNull; import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; @Execution(SAME_THREAD) // metastore invocation counters shares mutable state so can't be run from many threads simultaneously public class TestDeltaLakeMetastoreAccessOperations extends AbstractTestQueryFramework { - private static final Session TEST_SESSION = testSessionBuilder() - .setCatalog("delta") - .setSchema("test_schema") - .build(); - - private CountingAccessHiveMetastore metastore; - @Override protected DistributedQueryRunner createQueryRunner() throws Exception { - DistributedQueryRunner queryRunner = DistributedQueryRunner.builder(TEST_SESSION).build(); - - Path baseDir = queryRunner.getCoordinator().getBaseDataDir().resolve("delta_lake"); - metastore = new CountingAccessHiveMetastore(createTestingFileHiveMetastore(baseDir.toFile())); - - queryRunner.installPlugin(new TestingDeltaLakePlugin(baseDir, Optional.empty(), Optional.empty(), new CountingAccessMetastoreModule(metastore))); - ImmutableMap.Builder deltaLakeProperties = ImmutableMap.builder(); - deltaLakeProperties.put("hive.metastore", "test"); // use test value so we do not get clash with default bindings) - queryRunner.createCatalog("delta", "delta_lake", deltaLakeProperties.buildOrThrow()); - + DistributedQueryRunner queryRunner = createDeltaLakeQueryRunner(DELTA_CATALOG, ImmutableMap.of(), Map.of()); queryRunner.execute("CREATE SCHEMA test_schema"); return queryRunner; } - private static class CountingAccessMetastoreModule - extends AbstractConfigurationAwareModule - { - private final CountingAccessHiveMetastore metastore; - - public CountingAccessMetastoreModule(CountingAccessHiveMetastore metastore) - { - this.metastore = requireNonNull(metastore, "metastore is null"); - } - - @Override - protected void setup(Binder binder) - { - binder.bind(HiveMetastoreFactory.class).annotatedWith(RawHiveMetastoreFactory.class).toInstance(HiveMetastoreFactory.ofInstance(metastore)); - binder.bind(Key.get(boolean.class, AllowDeltaLakeManagedTableRename.class)).toInstance(false); - } - } - @Test public void testCreateTable() { @@ -271,6 +226,6 @@ private void assertMetastoreInvocations(@Language("SQL") String query, Multiset< { assertUpdate("CALL system.flush_metadata_cache()"); - CountingAccessHiveMetastoreUtil.assertMetastoreInvocations(metastore, getQueryRunner(), getSession(), query, expectedInvocations); + assertMetastoreInvocationsForQuery(getDistributedQueryRunner(), getSession(), query, expectedInvocations); } } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/CountingAccessHiveMetastore.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/CountingAccessHiveMetastore.java deleted file mode 100644 index 8900fb7aade2b..0000000000000 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/CountingAccessHiveMetastore.java +++ /dev/null @@ -1,423 +0,0 @@ -/* - * 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.hive.metastore; - -import com.google.common.collect.ConcurrentHashMultiset; -import com.google.common.collect.ImmutableMultiset; -import com.google.common.collect.Multiset; -import com.google.errorprone.annotations.ThreadSafe; -import io.trino.plugin.hive.HiveColumnStatisticType; -import io.trino.plugin.hive.HiveType; -import io.trino.plugin.hive.PartitionStatistics; -import io.trino.plugin.hive.acid.AcidTransaction; -import io.trino.plugin.hive.metastore.HivePrivilegeInfo.HivePrivilege; -import io.trino.spi.connector.RelationType; -import io.trino.spi.connector.SchemaTableName; -import io.trino.spi.function.LanguageFunction; -import io.trino.spi.predicate.TupleDomain; -import io.trino.spi.security.RoleGrant; -import io.trino.spi.type.Type; - -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.function.Function; - -import static io.trino.plugin.hive.metastore.MetastoreMethod.ADD_PARTITIONS; -import static io.trino.plugin.hive.metastore.MetastoreMethod.CREATE_DATABASE; -import static io.trino.plugin.hive.metastore.MetastoreMethod.CREATE_TABLE; -import static io.trino.plugin.hive.metastore.MetastoreMethod.DROP_DATABASE; -import static io.trino.plugin.hive.metastore.MetastoreMethod.DROP_TABLE; -import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_ALL_DATABASES; -import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_ALL_RELATION_TYPES; -import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_ALL_TABLES; -import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_ALL_VIEWS; -import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_DATABASE; -import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_PARTITION; -import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_PARTITIONS_BY_NAMES; -import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_PARTITION_NAMES_BY_FILTER; -import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_PARTITION_STATISTICS; -import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_RELATION_TYPES; -import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_TABLE; -import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_TABLES; -import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_TABLES_WITH_PARAMETER; -import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_TABLE_STATISTICS; -import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_VIEWS; -import static io.trino.plugin.hive.metastore.MetastoreMethod.REPLACE_TABLE; -import static io.trino.plugin.hive.metastore.MetastoreMethod.UPDATE_PARTITION_STATISTICS; -import static io.trino.plugin.hive.metastore.MetastoreMethod.UPDATE_TABLE_STATISTICS; - -@ThreadSafe -public class CountingAccessHiveMetastore - implements HiveMetastore -{ - private final HiveMetastore delegate; - private final ConcurrentHashMultiset methodInvocations = ConcurrentHashMultiset.create(); - - public CountingAccessHiveMetastore(HiveMetastore delegate) - { - this.delegate = delegate; - } - - public Multiset getMethodInvocations() - { - return ImmutableMultiset.copyOf(methodInvocations); - } - - public void resetCounters() - { - methodInvocations.clear(); - } - - @Override - public Optional
getTable(String databaseName, String tableName) - { - methodInvocations.add(GET_TABLE); - return delegate.getTable(databaseName, tableName); - } - - @Override - public Set getSupportedColumnStatistics(Type type) - { - // No need to count that, since it's a pure local operation. - return delegate.getSupportedColumnStatistics(type); - } - - @Override - public List getAllDatabases() - { - methodInvocations.add(GET_ALL_DATABASES); - return delegate.getAllDatabases(); - } - - @Override - public Optional getDatabase(String databaseName) - { - methodInvocations.add(GET_DATABASE); - return delegate.getDatabase(databaseName); - } - - @Override - public List getTablesWithParameter(String databaseName, String parameterKey, String parameterValue) - { - methodInvocations.add(GET_TABLES_WITH_PARAMETER); - return delegate.getTablesWithParameter(databaseName, parameterKey, parameterValue); - } - - @Override - public List getViews(String databaseName) - { - methodInvocations.add(GET_VIEWS); - return delegate.getViews(databaseName); - } - - @Override - public Optional> getAllViews() - { - Optional> allViews = delegate.getAllViews(); - if (allViews.isPresent()) { - methodInvocations.add(GET_ALL_VIEWS); - } - return allViews; - } - - @Override - public void createDatabase(Database database) - { - methodInvocations.add(CREATE_DATABASE); - delegate.createDatabase(database); - } - - @Override - public void dropDatabase(String databaseName, boolean deleteData) - { - methodInvocations.add(DROP_DATABASE); - delegate.dropDatabase(databaseName, deleteData); - } - - @Override - public void renameDatabase(String databaseName, String newDatabaseName) - { - throw new UnsupportedOperationException(); - } - - @Override - public void setDatabaseOwner(String databaseName, HivePrincipal principal) - { - throw new UnsupportedOperationException(); - } - - @Override - public void createTable(Table table, PrincipalPrivileges principalPrivileges) - { - methodInvocations.add(CREATE_TABLE); - delegate.createTable(table, principalPrivileges); - } - - @Override - public void dropTable(String databaseName, String tableName, boolean deleteData) - { - methodInvocations.add(DROP_TABLE); - delegate.dropTable(databaseName, tableName, deleteData); - } - - @Override - public void replaceTable(String databaseName, String tableName, Table newTable, PrincipalPrivileges principalPrivileges) - { - methodInvocations.add(REPLACE_TABLE); - delegate.replaceTable(databaseName, tableName, newTable, principalPrivileges); - } - - @Override - public void renameTable(String databaseName, String tableName, String newDatabaseName, String newTableName) - { - throw new UnsupportedOperationException(); - } - - @Override - public void commentTable(String databaseName, String tableName, Optional comment) - { - throw new UnsupportedOperationException(); - } - - @Override - public void setTableOwner(String databaseName, String tableName, HivePrincipal principal) - { - throw new UnsupportedOperationException(); - } - - @Override - public void commentColumn(String databaseName, String tableName, String columnName, Optional comment) - { - throw new UnsupportedOperationException(); - } - - @Override - public void addColumn(String databaseName, String tableName, String columnName, HiveType columnType, String columnComment) - { - throw new UnsupportedOperationException(); - } - - @Override - public void renameColumn(String databaseName, String tableName, String oldColumnName, String newColumnName) - { - throw new UnsupportedOperationException(); - } - - @Override - public void dropColumn(String databaseName, String tableName, String columnName) - { - throw new UnsupportedOperationException(); - } - - @Override - public Optional getPartition(Table table, List partitionValues) - { - methodInvocations.add(GET_PARTITION); - return delegate.getPartition(table, partitionValues); - } - - @Override - public Optional> getPartitionNamesByFilter(String databaseName, - String tableName, - List columnNames, - TupleDomain partitionKeysFilter) - { - methodInvocations.add(GET_PARTITION_NAMES_BY_FILTER); - return delegate.getPartitionNamesByFilter(databaseName, tableName, columnNames, partitionKeysFilter); - } - - @Override - public Map> getPartitionsByNames(Table table, List partitionNames) - { - methodInvocations.add(GET_PARTITIONS_BY_NAMES); - return delegate.getPartitionsByNames(table, partitionNames); - } - - @Override - public void addPartitions(String databaseName, String tableName, List partitions) - { - methodInvocations.add(ADD_PARTITIONS); - delegate.addPartitions(databaseName, tableName, partitions); - } - - @Override - public void dropPartition(String databaseName, String tableName, List parts, boolean deleteData) - { - throw new UnsupportedOperationException(); - } - - @Override - public void alterPartition(String databaseName, String tableName, PartitionWithStatistics partition) - { - throw new UnsupportedOperationException(); - } - - @Override - public void createRole(String role, String grantor) - { - throw new UnsupportedOperationException(); - } - - @Override - public void dropRole(String role) - { - throw new UnsupportedOperationException(); - } - - @Override - public Set listRoles() - { - throw new UnsupportedOperationException(); - } - - @Override - public void grantRoles(Set roles, Set grantees, boolean adminOption, HivePrincipal grantor) - { - throw new UnsupportedOperationException(); - } - - @Override - public void revokeRoles(Set roles, Set grantees, boolean adminOption, HivePrincipal grantor) - { - throw new UnsupportedOperationException(); - } - - @Override - public Set listRoleGrants(HivePrincipal principal) - { - throw new UnsupportedOperationException(); - } - - @Override - public void grantTablePrivileges(String databaseName, String tableName, String tableOwner, HivePrincipal grantee, HivePrincipal grantor, Set privileges, boolean grantOption) - { - throw new UnsupportedOperationException(); - } - - @Override - public void revokeTablePrivileges(String databaseName, String tableName, String tableOwner, HivePrincipal grantee, HivePrincipal grantor, Set privileges, boolean grantOption) - { - throw new UnsupportedOperationException(); - } - - @Override - public Set listTablePrivileges(String databaseName, String tableName, Optional tableOwner, Optional principal) - { - throw new UnsupportedOperationException(); - } - - @Override - public PartitionStatistics getTableStatistics(Table table) - { - methodInvocations.add(GET_TABLE_STATISTICS); - return delegate.getTableStatistics(table); - } - - @Override - public Map getPartitionStatistics(Table table, List partitions) - { - methodInvocations.add(GET_PARTITION_STATISTICS); - return delegate.getPartitionStatistics(table, partitions); - } - - @Override - public void updateTableStatistics(String databaseName, - String tableName, - AcidTransaction transaction, - Function update) - { - methodInvocations.add(UPDATE_TABLE_STATISTICS); - delegate.updateTableStatistics(databaseName, tableName, transaction, update); - } - - @Override - public void updatePartitionStatistics(Table table, Map> updates) - { - methodInvocations.add(UPDATE_PARTITION_STATISTICS); - delegate.updatePartitionStatistics(table, updates); - } - - @Override - public List getTables(String databaseName) - { - methodInvocations.add(GET_TABLES); - return delegate.getTables(databaseName); - } - - @Override - public Optional> getAllTables() - { - Optional> allTables = delegate.getAllTables(); - if (allTables.isPresent()) { - methodInvocations.add(GET_ALL_TABLES); - } - return allTables; - } - - @Override - public Map getRelationTypes(String databaseName) - { - methodInvocations.add(GET_RELATION_TYPES); - return delegate.getRelationTypes(databaseName); - } - - @Override - public Optional> getAllRelationTypes() - { - Optional> relationTypes = delegate.getAllRelationTypes(); - if (relationTypes.isPresent()) { - methodInvocations.add(GET_ALL_RELATION_TYPES); - } - return relationTypes; - } - - @Override - public boolean functionExists(String databaseName, String functionName, String signatureToken) - { - throw new UnsupportedOperationException(); - } - - @Override - public Collection getAllFunctions(String databaseName) - { - throw new UnsupportedOperationException(); - } - - @Override - public Collection getFunctions(String databaseName, String functionName) - { - throw new UnsupportedOperationException(); - } - - @Override - public void createFunction(String databaseName, String functionName, LanguageFunction function) - { - throw new UnsupportedOperationException(); - } - - @Override - public void replaceFunction(String databaseName, String functionName, LanguageFunction function) - { - throw new UnsupportedOperationException(); - } - - @Override - public void dropFunction(String databaseName, String functionName, String signatureToken) - { - throw new UnsupportedOperationException(); - } -} diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/CountingAccessHiveMetastoreUtil.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/MetastoreInvocations.java similarity index 50% rename from plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/CountingAccessHiveMetastoreUtil.java rename to plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/MetastoreInvocations.java index b6c0873800184..551bfb9542e8c 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/CountingAccessHiveMetastoreUtil.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/MetastoreInvocations.java @@ -14,25 +14,37 @@ package io.trino.plugin.hive.metastore; import com.google.common.collect.Multiset; +import io.opentelemetry.sdk.trace.data.SpanData; import io.trino.Session; -import io.trino.testing.QueryRunner; +import io.trino.testing.DistributedQueryRunner; import org.intellij.lang.annotations.Language; +import static com.google.common.collect.ImmutableMultiset.toImmutableMultiset; import static io.trino.testing.MultisetAssertions.assertMultisetsEqual; -public final class CountingAccessHiveMetastoreUtil +public final class MetastoreInvocations { - private CountingAccessHiveMetastoreUtil() {} + private static final String TRACE_PREFIX = "HiveMetastore."; - public static void assertMetastoreInvocations( - CountingAccessHiveMetastore metastore, - QueryRunner queryRunner, + private MetastoreInvocations() {} + + public static void assertMetastoreInvocationsForQuery( + DistributedQueryRunner queryRunner, Session session, @Language("SQL") String query, Multiset expectedInvocations) { - metastore.resetCounters(); queryRunner.execute(session, query); - assertMultisetsEqual(metastore.getMethodInvocations(), expectedInvocations); + + Multiset invocations = queryRunner.getSpans().stream() + .map(SpanData::getName) + .filter(name -> name.startsWith(TRACE_PREFIX)) + .map(name -> name.substring(TRACE_PREFIX.length())) + .filter(name -> !name.equals("listRoleGrants")) + .filter(name -> !name.equals("listTablePrivileges")) + .map(MetastoreMethod::fromMethodName) + .collect(toImmutableMultiset()); + + assertMultisetsEqual(invocations, expectedInvocations); } } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/MetastoreMethod.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/MetastoreMethod.java index 30c6461a532b8..71cfe96731bc8 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/MetastoreMethod.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/MetastoreMethod.java @@ -39,6 +39,7 @@ public enum MetastoreMethod GET_PARTITION, GET_PARTITION_STATISTICS, UPDATE_PARTITION_STATISTICS, + UPDATE_PARTITIONS_STATISTICS, REPLACE_TABLE, DROP_TABLE, /**/; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreAccessOperations.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreAccessOperations.java index 126202d7398fb..3b816687bd400 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreAccessOperations.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreAccessOperations.java @@ -13,23 +13,17 @@ */ package io.trino.plugin.hive.metastore.thrift; -import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMultiset; import com.google.common.collect.Multiset; -import io.trino.Session; -import io.trino.plugin.hive.TestingHivePlugin; -import io.trino.plugin.hive.metastore.CountingAccessHiveMetastore; -import io.trino.plugin.hive.metastore.CountingAccessHiveMetastoreUtil; +import io.trino.plugin.hive.HiveQueryRunner; import io.trino.plugin.hive.metastore.MetastoreMethod; import io.trino.testing.AbstractTestQueryFramework; -import io.trino.testing.DistributedQueryRunner; import io.trino.testing.QueryRunner; import org.intellij.lang.annotations.Language; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.parallel.Execution; -import java.io.File; - +import static io.trino.plugin.hive.metastore.MetastoreInvocations.assertMetastoreInvocationsForQuery; import static io.trino.plugin.hive.metastore.MetastoreMethod.CREATE_TABLE; import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_DATABASE; import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_PARTITIONS_BY_NAMES; @@ -37,37 +31,20 @@ import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_PARTITION_STATISTICS; import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_TABLE; import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_TABLE_STATISTICS; +import static io.trino.plugin.hive.metastore.MetastoreMethod.UPDATE_PARTITIONS_STATISTICS; import static io.trino.plugin.hive.metastore.MetastoreMethod.UPDATE_PARTITION_STATISTICS; import static io.trino.plugin.hive.metastore.MetastoreMethod.UPDATE_TABLE_STATISTICS; -import static io.trino.plugin.hive.metastore.file.TestingFileHiveMetastore.createTestingFileHiveMetastore; -import static io.trino.testing.TestingSession.testSessionBuilder; import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; @Execution(SAME_THREAD)// metastore invocation counters shares mutable state so can't be run from many threads simultaneously public class TestHiveMetastoreAccessOperations extends AbstractTestQueryFramework { - private static final Session TEST_SESSION = testSessionBuilder() - .setCatalog("hive") - .setSchema("test_schema") - .build(); - - private CountingAccessHiveMetastore metastore; - @Override protected QueryRunner createQueryRunner() throws Exception { - DistributedQueryRunner queryRunner = DistributedQueryRunner.builder(TEST_SESSION).build(); - - File baseDir = queryRunner.getCoordinator().getBaseDataDir().resolve("hive").toFile(); - metastore = new CountingAccessHiveMetastore(createTestingFileHiveMetastore(baseDir)); - - queryRunner.installPlugin(new TestingHivePlugin(baseDir.toPath(), metastore)); - queryRunner.createCatalog("hive", "hive", ImmutableMap.of()); - - queryRunner.execute("CREATE SCHEMA test_schema"); - return queryRunner; + return HiveQueryRunner.create(); } @Test @@ -294,7 +271,7 @@ public void testDropStats() { assertUpdate("CREATE TABLE drop_stats AS SELECT 2 AS age", 1); - assertMetastoreInvocations("CALL system.drop_stats('test_schema', 'drop_stats')", + assertMetastoreInvocations("CALL system.drop_stats(CURRENT_SCHEMA, 'drop_stats')", ImmutableMultiset.builder() .add(GET_TABLE) .add(UPDATE_TABLE_STATISTICS) @@ -306,25 +283,25 @@ public void testDropStatsPartitionedTable() { assertUpdate("CREATE TABLE drop_stats_partition WITH (partitioned_by = ARRAY['part']) AS SELECT 1 AS data, 10 AS part", 1); - assertMetastoreInvocations("CALL system.drop_stats('test_schema', 'drop_stats_partition')", + assertMetastoreInvocations("CALL system.drop_stats(CURRENT_SCHEMA, 'drop_stats_partition')", ImmutableMultiset.builder() .add(GET_TABLE) .add(GET_PARTITION_NAMES_BY_FILTER) - .add(UPDATE_PARTITION_STATISTICS) + .add(UPDATE_PARTITIONS_STATISTICS) .build()); assertUpdate("INSERT INTO drop_stats_partition SELECT 2 AS data, 20 AS part", 1); - assertMetastoreInvocations("CALL system.drop_stats('test_schema', 'drop_stats_partition')", + assertMetastoreInvocations("CALL system.drop_stats(CURRENT_SCHEMA, 'drop_stats_partition')", ImmutableMultiset.builder() .add(GET_TABLE) .add(GET_PARTITION_NAMES_BY_FILTER) - .addCopies(UPDATE_PARTITION_STATISTICS, 2) + .addCopies(UPDATE_PARTITIONS_STATISTICS, 2) .build()); } private void assertMetastoreInvocations(@Language("SQL") String query, Multiset expectedInvocations) { - CountingAccessHiveMetastoreUtil.assertMetastoreInvocations(metastore, getQueryRunner(), getQueryRunner().getDefaultSession(), query, expectedInvocations); + assertMetastoreInvocationsForQuery(getDistributedQueryRunner(), getSession(), query, expectedInvocations); } } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreMetadataQueriesAccessOperations.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreMetadataQueriesAccessOperations.java index 930c462cb6cd2..a47c41c580fe2 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreMetadataQueriesAccessOperations.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreMetadataQueriesAccessOperations.java @@ -14,18 +14,17 @@ package io.trino.plugin.hive.metastore.thrift; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMultiset; import com.google.common.collect.Multiset; -import io.trino.plugin.hive.HiveConfig; +import io.airlift.log.Logger; +import io.trino.Session; +import io.trino.plugin.hive.HiveQueryRunner; import io.trino.plugin.hive.HiveType; -import io.trino.plugin.hive.TestingHivePlugin; import io.trino.plugin.hive.containers.HiveHadoop; import io.trino.plugin.hive.metastore.Column; -import io.trino.plugin.hive.metastore.CountingAccessHiveMetastore; -import io.trino.plugin.hive.metastore.CountingAccessHiveMetastoreUtil; import io.trino.plugin.hive.metastore.Database; import io.trino.plugin.hive.metastore.HiveMetastore; +import io.trino.plugin.hive.metastore.HiveMetastoreFactory; import io.trino.plugin.hive.metastore.MetastoreMethod; import io.trino.plugin.hive.metastore.Table; import io.trino.testing.AbstractTestQueryFramework; @@ -39,9 +38,11 @@ import java.util.Map; import java.util.Optional; +import static io.airlift.units.Duration.nanosSince; import static io.trino.plugin.hive.HiveStorageFormat.PARQUET; import static io.trino.plugin.hive.TableType.MANAGED_TABLE; -import static io.trino.plugin.hive.TestingThriftHiveMetastoreBuilder.testingThriftHiveMetastoreBuilder; +import static io.trino.plugin.hive.TestingHiveUtils.getConnectorService; +import static io.trino.plugin.hive.metastore.MetastoreInvocations.assertMetastoreInvocationsForQuery; import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_ALL_DATABASES; import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_ALL_RELATION_TYPES; import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_ALL_TABLES; @@ -59,13 +60,19 @@ public class TestHiveMetastoreMetadataQueriesAccessOperations extends AbstractTestQueryFramework { + private static final Logger log = Logger.get(TestHiveMetastoreMetadataQueriesAccessOperations.class); + private static final int MAX_PREFIXES_COUNT = 20; private static final int TEST_SCHEMAS_COUNT = MAX_PREFIXES_COUNT + 1; private static final int TEST_TABLES_IN_SCHEMA_COUNT = MAX_PREFIXES_COUNT + 3; private static final int TEST_ALL_TABLES_COUNT = TEST_SCHEMAS_COUNT * TEST_TABLES_IN_SCHEMA_COUNT; + private static final Session SESSION = testSessionBuilder() + .setCatalog("hive") + .setSchema(Optional.empty()) + .build(); + private HiveHadoop hiveHadoop; - private CountingAccessHiveMetastore metastore; @Override protected QueryRunner createQueryRunner() @@ -74,16 +81,38 @@ protected QueryRunner createQueryRunner() hiveHadoop = HiveHadoop.builder().build(); hiveHadoop.start(); - HiveMetastore thriftMetastore = new BridgingHiveMetastore(testingThriftHiveMetastoreBuilder() - .metastoreClient(hiveHadoop.getHiveMetastoreEndpoint()) - .thriftMetastoreConfig(new ThriftMetastoreConfig() - .setBatchMetadataFetchEnabled(true) - .setDeleteFilesOnDrop(true)) - .hiveConfig(new HiveConfig().setTranslateHiveViews(true)) - .build()); + DistributedQueryRunner queryRunner = HiveQueryRunner.builder(SESSION) + // metadata queries do not use workers + .setNodeCount(1) + .addCoordinatorProperty("optimizer.experimental-max-prefetched-information-schema-prefixes", Integer.toString(MAX_PREFIXES_COUNT)) + .addHiveProperty("hive.metastore", "thrift") + .addHiveProperty("hive.metastore.uri", "thrift://" + hiveHadoop.getHiveMetastoreEndpoint()) + .addHiveProperty("hive.metastore.thrift.batch-fetch.enabled", "true") + .addHiveProperty("hive.hive-views.enabled", "true") + .setCreateTpchSchemas(false) + .build(); + + try { + long start = System.nanoTime(); + createTestingTables(queryRunner); + log.info("Created testing tables in %s", nanosSince(start)); + } + catch (RuntimeException e) { + queryRunner.close(); + throw e; + } + + return queryRunner; + } + + private static void createTestingTables(QueryRunner queryRunner) + { + HiveMetastore metastore = getConnectorService(queryRunner, HiveMetastoreFactory.class) + .createMetastore(Optional.empty()); + for (int databaseId = 0; databaseId < TEST_SCHEMAS_COUNT; databaseId++) { String databaseName = "test_schema_" + databaseId; - thriftMetastore.createDatabase(Database.builder() + metastore.createDatabase(Database.builder() .setDatabaseName(databaseName) .setOwnerName(Optional.empty()) .setOwnerType(Optional.empty()) @@ -100,25 +129,9 @@ protected QueryRunner createQueryRunner() .setOwner(Optional.empty()); table.getStorageBuilder() .setStorageFormat(fromHiveStorageFormat(PARQUET)); - thriftMetastore.createTable(table.build(), NO_PRIVILEGES); + metastore.createTable(table.build(), NO_PRIVILEGES); } } - - DistributedQueryRunner queryRunner = DistributedQueryRunner.builder( - testSessionBuilder() - .setCatalog("hive") - .setSchema(Optional.empty()) - .build()) - // metadata queries do not use workers - .setNodeCount(1) - .addCoordinatorProperty("optimizer.experimental-max-prefetched-information-schema-prefixes", Integer.toString(MAX_PREFIXES_COUNT)) - .build(); - - metastore = new CountingAccessHiveMetastore(thriftMetastore); - - queryRunner.installPlugin(new TestingHivePlugin(queryRunner.getCoordinator().getBaseDataDir().resolve("hive_data"), metastore)); - queryRunner.createCatalog("hive", "hive", ImmutableMap.of()); - return queryRunner; } @AfterAll @@ -485,6 +498,6 @@ public void testSelectColumnsFilterByTableAndSchema() private void assertMetastoreInvocations(@Language("SQL") String query, Multiset expectedInvocations) { - CountingAccessHiveMetastoreUtil.assertMetastoreInvocations(metastore, getQueryRunner(), getQueryRunner().getDefaultSession(), query, expectedInvocations); + assertMetastoreInvocationsForQuery(getDistributedQueryRunner(), getQueryRunner().getDefaultSession(), query, expectedInvocations); } } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetastoreAccessOperations.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetastoreAccessOperations.java index 7854a9a2778de..b79b2cecce2aa 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetastoreAccessOperations.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetastoreAccessOperations.java @@ -16,10 +16,7 @@ import com.google.common.collect.ImmutableMultiset; import com.google.common.collect.Multiset; import io.trino.Session; -import io.trino.plugin.hive.metastore.CountingAccessHiveMetastore; -import io.trino.plugin.hive.metastore.CountingAccessHiveMetastoreUtil; import io.trino.plugin.hive.metastore.MetastoreMethod; -import io.trino.plugin.iceberg.catalog.file.TestingIcebergFileMetastoreCatalogModule; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.DistributedQueryRunner; import org.intellij.lang.annotations.Language; @@ -29,10 +26,9 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.MethodSource; -import java.io.File; import java.util.Optional; -import static com.google.inject.util.Modules.EMPTY_MODULE; +import static io.trino.plugin.hive.metastore.MetastoreInvocations.assertMetastoreInvocationsForQuery; import static io.trino.plugin.hive.metastore.MetastoreMethod.CREATE_TABLE; import static io.trino.plugin.hive.metastore.MetastoreMethod.DROP_TABLE; import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_DATABASE; @@ -40,7 +36,6 @@ import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_TABLES; import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_TABLES_WITH_PARAMETER; import static io.trino.plugin.hive.metastore.MetastoreMethod.REPLACE_TABLE; -import static io.trino.plugin.hive.metastore.file.TestingFileHiveMetastore.createTestingFileHiveMetastore; import static io.trino.plugin.iceberg.IcebergSessionProperties.COLLECT_EXTENDED_STATISTICS_ON_WRITE; import static io.trino.plugin.iceberg.TableType.DATA; import static io.trino.plugin.iceberg.TableType.FILES; @@ -52,7 +47,6 @@ import static io.trino.plugin.iceberg.TableType.REFS; import static io.trino.plugin.iceberg.TableType.SNAPSHOTS; import static io.trino.testing.TestingNames.randomNameSuffix; -import static io.trino.testing.TestingSession.testSessionBuilder; import static org.assertj.core.api.Assertions.assertThat; @Execution(ExecutionMode.SAME_THREAD) // metastore invocation counters shares mutable state so can't be run from many threads simultaneously @@ -60,32 +54,14 @@ public class TestIcebergMetastoreAccessOperations extends AbstractTestQueryFramework { private static final int MAX_PREFIXES_COUNT = 10; - private static final Session TEST_SESSION = testSessionBuilder() - .setCatalog("iceberg") - .setSchema("test_schema") - .build(); - - private CountingAccessHiveMetastore metastore; @Override protected DistributedQueryRunner createQueryRunner() throws Exception { - DistributedQueryRunner queryRunner = DistributedQueryRunner.builder(TEST_SESSION) + return IcebergQueryRunner.builder() .addCoordinatorProperty("optimizer.experimental-max-prefetched-information-schema-prefixes", Integer.toString(MAX_PREFIXES_COUNT)) .build(); - - File baseDir = queryRunner.getCoordinator().getBaseDataDir().resolve("iceberg_data").toFile(); - metastore = new CountingAccessHiveMetastore(createTestingFileHiveMetastore(baseDir)); - queryRunner.installPlugin(new TestingIcebergPlugin( - baseDir.toPath(), - Optional.of(new TestingIcebergFileMetastoreCatalogModule(metastore)), - Optional.empty(), - EMPTY_MODULE)); - queryRunner.createCatalog("iceberg", "iceberg"); - - queryRunner.execute("CREATE SCHEMA test_schema"); - return queryRunner; } @Test @@ -356,7 +332,7 @@ public void testSelectSystemTable() .build()); assertQueryFails("SELECT * FROM \"test_select_snapshots$materialized_view_storage\"", - "Table 'test_schema.test_select_snapshots\\$materialized_view_storage' not found"); + "Table 'tpch.test_select_snapshots\\$materialized_view_storage' not found"); // This test should get updated if a new system table is added. assertThat(TableType.values()) @@ -549,7 +525,7 @@ private void assertMetastoreInvocations(@Language("SQL") String query, Multiset< private void assertMetastoreInvocations(Session session, @Language("SQL") String query, Multiset expectedInvocations) { - CountingAccessHiveMetastoreUtil.assertMetastoreInvocations(metastore, getQueryRunner(), session, query, expectedInvocations); + assertMetastoreInvocationsForQuery(getDistributedQueryRunner(), session, query, expectedInvocations); } private static Session withStatsOnWrite(Session session, boolean enabled) From aac6edf6be2ca27d4989c1e2d3d1b60d55a93fb7 Mon Sep 17 00:00:00 2001 From: David Phillips Date: Wed, 6 Dec 2023 13:51:57 +0800 Subject: [PATCH 084/350] Remove unused test exclusion --- plugin/trino-hive-hadoop2/pom.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/plugin/trino-hive-hadoop2/pom.xml b/plugin/trino-hive-hadoop2/pom.xml index 790c47ff3e7a0..7f94db9ac7926 100644 --- a/plugin/trino-hive-hadoop2/pom.xml +++ b/plugin/trino-hive-hadoop2/pom.xml @@ -237,7 +237,6 @@ **/TestHiveFileSystemAbfsAccessKey.java **/TestHiveFileSystemAbfsOAuth.java **/TestHiveFileSystemAdl.java - **/TestHiveAzure.java From 8b3f8cf27d80414082d5cec9409bbefaee7c6a9f Mon Sep 17 00:00:00 2001 From: David Phillips Date: Wed, 6 Dec 2023 13:59:19 +0800 Subject: [PATCH 085/350] Remove legacy Hive file system tests --- .github/workflows/ci.yml | 47 - .../bin/run_hive_abfs_access_key_tests.sh | 46 - .../bin/run_hive_abfs_oauth_tests.sh | 49 - .../bin/run_hive_adl_tests.sh | 47 - .../bin/run_hive_wasb_tests.sh | 46 - .../core-site.xml.abfs-access-key-template | 40 - .../files/core-site.xml.abfs-oauth-template | 60 -- .../conf/files/core-site.xml.adl-template | 54 - .../conf/files/core-site.xml.wasb-template | 40 - plugin/trino-hive-hadoop2/pom.xml | 70 -- .../hive/AbstractTestHiveFileSystemAbfs.java | 114 --- .../hive/TestHiveFileSystemAbfsAccessKey.java | 48 - .../hive/TestHiveFileSystemAbfsOAuth.java | 53 - .../plugin/hive/TestHiveFileSystemAdl.java | 162 --- .../plugin/hive/TestHiveFileSystemS3.java | 262 ----- .../plugin/hive/TestHiveFileSystemWasb.java | 81 -- .../hive/AbstractTestHiveFileSystem.java | 942 ------------------ .../plugin/hive/HiveFileSystemTestUtils.java | 137 --- 18 files changed, 2298 deletions(-) delete mode 100755 plugin/trino-hive-hadoop2/bin/run_hive_abfs_access_key_tests.sh delete mode 100755 plugin/trino-hive-hadoop2/bin/run_hive_abfs_oauth_tests.sh delete mode 100755 plugin/trino-hive-hadoop2/bin/run_hive_adl_tests.sh delete mode 100755 plugin/trino-hive-hadoop2/bin/run_hive_wasb_tests.sh delete mode 100644 plugin/trino-hive-hadoop2/conf/files/core-site.xml.abfs-access-key-template delete mode 100644 plugin/trino-hive-hadoop2/conf/files/core-site.xml.abfs-oauth-template delete mode 100644 plugin/trino-hive-hadoop2/conf/files/core-site.xml.adl-template delete mode 100644 plugin/trino-hive-hadoop2/conf/files/core-site.xml.wasb-template delete mode 100644 plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/AbstractTestHiveFileSystemAbfs.java delete mode 100644 plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/TestHiveFileSystemAbfsAccessKey.java delete mode 100644 plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/TestHiveFileSystemAbfsOAuth.java delete mode 100644 plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/TestHiveFileSystemAdl.java delete mode 100644 plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/TestHiveFileSystemS3.java delete mode 100644 plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/TestHiveFileSystemWasb.java delete mode 100644 plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHiveFileSystem.java delete mode 100644 plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveFileSystemTestUtils.java diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index cd365660d38a9..e3d7b3dbfa5c9 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -333,53 +333,6 @@ jobs: if [ "${AWS_ACCESS_KEY_ID}" != "" ]; then $MAVEN test ${MAVEN_TEST} -pl :trino-hive -P aws-tests fi - - name: Run Hive Azure ABFS Access Key Tests - if: matrix.config != 'config-empty' # Hive 1.x does not support Azure storage - env: - ABFS_CONTAINER: ${{ secrets.AZURE_ABFS_CONTAINER }} - ABFS_ACCOUNT: ${{ secrets.AZURE_ABFS_ACCOUNT }} - ABFS_ACCESS_KEY: ${{ secrets.AZURE_ABFS_ACCESSKEY }} - run: | - if [ "${ABFS_CONTAINER}" != "" ]; then - source plugin/trino-hive-hadoop2/conf/hive-tests-${{ matrix.config }}.sh && - plugin/trino-hive-hadoop2/bin/run_hive_abfs_access_key_tests.sh - fi - - name: Run Hive Azure ABFS OAuth Tests - if: matrix.config != 'config-empty' # Hive 1.x does not support Azure storage - env: - ABFS_CONTAINER: ${{ secrets.AZURE_ABFS_CONTAINER }} - ABFS_ACCOUNT: ${{ secrets.AZURE_ABFS_ACCOUNT }} - ABFS_OAUTH_ENDPOINT: ${{ secrets.AZURE_ABFS_OAUTH_ENDPOINT }} - ABFS_OAUTH_CLIENTID: ${{ secrets.AZURE_ABFS_OAUTH_CLIENTID }} - ABFS_OAUTH_SECRET: ${{ secrets.AZURE_ABFS_OAUTH_SECRET }} - run: | - if [ -n "$ABFS_CONTAINER" ]; then - source plugin/trino-hive-hadoop2/conf/hive-tests-${{ matrix.config }}.sh && - plugin/trino-hive-hadoop2/bin/run_hive_abfs_oauth_tests.sh - fi - - name: Run Hive Azure WASB Tests - if: matrix.config != 'config-empty' # Hive 1.x does not support Azure storage - env: - WASB_CONTAINER: ${{ secrets.AZURE_WASB_CONTAINER }} - WASB_ACCOUNT: ${{ secrets.AZURE_WASB_ACCOUNT }} - WASB_ACCESS_KEY: ${{ secrets.AZURE_WASB_ACCESSKEY }} - run: | - if [ "${WASB_CONTAINER}" != "" ]; then - source plugin/trino-hive-hadoop2/conf/hive-tests-${{ matrix.config }}.sh && - plugin/trino-hive-hadoop2/bin/run_hive_wasb_tests.sh - fi - - name: Run Hive Azure ADL Tests - if: matrix.config != 'config-empty' # Hive 1.x does not support Azure storage - env: - ADL_NAME: ${{ secrets.AZURE_ADL_NAME }} - ADL_CLIENT_ID: ${{ secrets.AZURE_ADL_CLIENTID }} - ADL_CREDENTIAL: ${{ secrets.AZURE_ADL_CREDENTIAL }} - ADL_REFRESH_URL: ${{ secrets.AZURE_ADL_REFRESHURL }} - run: | - if [ "${ADL_NAME}" != "" ]; then - source plugin/trino-hive-hadoop2/conf/hive-tests-${{ matrix.config }}.sh && - plugin/trino-hive-hadoop2/bin/run_hive_adl_tests.sh - fi - name: Upload test results uses: actions/upload-artifact@v3 # Upload all test reports only on failure, because the artifacts are large diff --git a/plugin/trino-hive-hadoop2/bin/run_hive_abfs_access_key_tests.sh b/plugin/trino-hive-hadoop2/bin/run_hive_abfs_access_key_tests.sh deleted file mode 100755 index 60c0fd1ec5dc0..0000000000000 --- a/plugin/trino-hive-hadoop2/bin/run_hive_abfs_access_key_tests.sh +++ /dev/null @@ -1,46 +0,0 @@ -#!/usr/bin/env bash - -set -euo pipefail -x - -. "${BASH_SOURCE%/*}/common.sh" - -abort_if_not_gib_impacted - -check_vars ABFS_CONTAINER ABFS_ACCOUNT ABFS_ACCESS_KEY - -cleanup_hadoop_docker_containers -start_hadoop_docker_containers - -test_directory="$(date '+%Y%m%d-%H%M%S')-$(uuidgen | sha1sum | cut -b 1-6)" - -# insert Azure credentials -deploy_core_site_xml core-site.xml.abfs-access-key-template \ - ABFS_ACCESS_KEY ABFS_ACCOUNT - -# restart hive-server2 to apply changes in core-site.xml -exec_in_hadoop_master_container supervisorctl restart hive-server2 -retry check_hadoop - -create_test_tables "abfs://${ABFS_CONTAINER}@${ABFS_ACCOUNT}.dfs.core.windows.net/${test_directory}" - -stop_unnecessary_hadoop_services - -# run product tests -pushd $PROJECT_ROOT -set +e -./mvnw ${MAVEN_TEST:--B} -pl :trino-hive-hadoop2 test -P test-hive-hadoop2-abfs-access-key \ - -DHADOOP_USER_NAME=hive \ - -Dhive.hadoop2.metastoreHost=localhost \ - -Dhive.hadoop2.metastorePort=9083 \ - -Dhive.hadoop2.databaseName=default \ - -Dhive.hadoop2.abfs.container=${ABFS_CONTAINER} \ - -Dhive.hadoop2.abfs.account=${ABFS_ACCOUNT} \ - -Dhive.hadoop2.abfs.accessKey=${ABFS_ACCESS_KEY} \ - -Dhive.hadoop2.abfs.testDirectory="${test_directory}" -EXIT_CODE=$? -set -e -popd - -cleanup_hadoop_docker_containers - -exit ${EXIT_CODE} diff --git a/plugin/trino-hive-hadoop2/bin/run_hive_abfs_oauth_tests.sh b/plugin/trino-hive-hadoop2/bin/run_hive_abfs_oauth_tests.sh deleted file mode 100755 index 87fe5e34291cb..0000000000000 --- a/plugin/trino-hive-hadoop2/bin/run_hive_abfs_oauth_tests.sh +++ /dev/null @@ -1,49 +0,0 @@ -#!/usr/bin/env bash -set -euxo pipefail - -. "${BASH_SOURCE%/*}/common.sh" - -abort_if_not_gib_impacted - -check_vars ABFS_ACCOUNT ABFS_CONTAINER \ - ABFS_OAUTH_ENDPOINT ABFS_OAUTH_CLIENTID ABFS_OAUTH_SECRET - -test_directory="$(date '+%Y%m%d-%H%M%S')-$(uuidgen | sha1sum | cut -b 1-6)" - -cleanup_hadoop_docker_containers -start_hadoop_docker_containers - -# insert Azure credentials -deploy_core_site_xml core-site.xml.abfs-oauth-template \ - ABFS_ACCOUNT ABFS_CONTAINER \ - ABFS_OAUTH_ENDPOINT ABFS_OAUTH_CLIENTID ABFS_OAUTH_SECRET - -# restart hive-server2 to apply changes in core-site.xml -exec_in_hadoop_master_container supervisorctl restart hive-server2 -retry check_hadoop - -create_test_tables \ - "abfs://$ABFS_CONTAINER@$ABFS_ACCOUNT.dfs.core.windows.net/$test_directory" - -stop_unnecessary_hadoop_services - -pushd $PROJECT_ROOT -set +e -./mvnw ${MAVEN_TEST:--B} -pl :trino-hive-hadoop2 test -P test-hive-hadoop2-abfs-oauth \ - -DHADOOP_USER_NAME=hive \ - -Dhive.hadoop2.metastoreHost=localhost \ - -Dhive.hadoop2.metastorePort=9083 \ - -Dhive.hadoop2.databaseName=default \ - -Dtest.hive.azure.abfs.container="$ABFS_CONTAINER" \ - -Dtest.hive.azure.abfs.storage-account="$ABFS_ACCOUNT" \ - -Dtest.hive.azure.abfs.test-directory="$test_directory" \ - -Dtest.hive.azure.abfs.oauth.endpoint="$ABFS_OAUTH_ENDPOINT" \ - -Dtest.hive.azure.abfs.oauth.client-id="$ABFS_OAUTH_CLIENTID" \ - -Dtest.hive.azure.abfs.oauth.secret="$ABFS_OAUTH_SECRET" -EXIT_CODE=$? -set -e -popd - -cleanup_hadoop_docker_containers - -exit ${EXIT_CODE} diff --git a/plugin/trino-hive-hadoop2/bin/run_hive_adl_tests.sh b/plugin/trino-hive-hadoop2/bin/run_hive_adl_tests.sh deleted file mode 100755 index c064eb923fa78..0000000000000 --- a/plugin/trino-hive-hadoop2/bin/run_hive_adl_tests.sh +++ /dev/null @@ -1,47 +0,0 @@ -#!/usr/bin/env bash - -set -euo pipefail -x - -. "${BASH_SOURCE%/*}/common.sh" - -abort_if_not_gib_impacted - -check_vars ADL_NAME ADL_CLIENT_ID ADL_CREDENTIAL ADL_REFRESH_URL - -cleanup_hadoop_docker_containers -start_hadoop_docker_containers - -test_directory="$(date '+%Y%m%d-%H%M%S')-$(uuidgen | sha1sum | cut -b 1-6)" - -# insert Azure credentials -deploy_core_site_xml core-site.xml.adl-template \ - ADL_CLIENT_ID ADL_CREDENTIAL ADL_REFRESH_URL - -# restart hive-server2 to apply changes in core-site.xml -exec_in_hadoop_master_container supervisorctl restart hive-server2 -retry check_hadoop - -create_test_tables "adl://${ADL_NAME}.azuredatalakestore.net/${test_directory}" - -stop_unnecessary_hadoop_services - -# run product tests -pushd $PROJECT_ROOT -set +e -./mvnw ${MAVEN_TEST:--B} -pl :trino-hive-hadoop2 test -P test-hive-hadoop2-adl \ - -DHADOOP_USER_NAME=hive \ - -Dhive.hadoop2.metastoreHost=localhost \ - -Dhive.hadoop2.metastorePort=9083 \ - -Dhive.hadoop2.databaseName=default \ - -Dhive.hadoop2.adl.name=${ADL_NAME} \ - -Dhive.hadoop2.adl.clientId=${ADL_CLIENT_ID} \ - -Dhive.hadoop2.adl.credential=${ADL_CREDENTIAL} \ - -Dhive.hadoop2.adl.refreshUrl=${ADL_REFRESH_URL} \ - -Dhive.hadoop2.adl.testDirectory=${test_directory} -EXIT_CODE=$? -set -e -popd - -cleanup_hadoop_docker_containers - -exit ${EXIT_CODE} diff --git a/plugin/trino-hive-hadoop2/bin/run_hive_wasb_tests.sh b/plugin/trino-hive-hadoop2/bin/run_hive_wasb_tests.sh deleted file mode 100755 index 40c19ddf8c472..0000000000000 --- a/plugin/trino-hive-hadoop2/bin/run_hive_wasb_tests.sh +++ /dev/null @@ -1,46 +0,0 @@ -#!/usr/bin/env bash - -set -euo pipefail -x - -. "${BASH_SOURCE%/*}/common.sh" - -abort_if_not_gib_impacted - -check_vars WASB_CONTAINER WASB_ACCOUNT WASB_ACCESS_KEY - -cleanup_hadoop_docker_containers -start_hadoop_docker_containers - -test_directory="$(date '+%Y%m%d-%H%M%S')-$(uuidgen | sha1sum | cut -b 1-6)" - -# insert Azure credentials -deploy_core_site_xml core-site.xml.wasb-template \ - WASB_ACCESS_KEY WASB_ACCOUNT - -# restart hive-server2 to apply changes in core-site.xml -exec_in_hadoop_master_container supervisorctl restart hive-server2 -retry check_hadoop - -create_test_tables "wasb://${WASB_CONTAINER}@${WASB_ACCOUNT}.blob.core.windows.net/${test_directory}" - -stop_unnecessary_hadoop_services - -# run product tests -pushd $PROJECT_ROOT -set +e -./mvnw ${MAVEN_TEST:--B} -pl :trino-hive-hadoop2 test -P test-hive-hadoop2-wasb \ - -DHADOOP_USER_NAME=hive \ - -Dhive.hadoop2.metastoreHost=localhost \ - -Dhive.hadoop2.metastorePort=9083 \ - -Dhive.hadoop2.databaseName=default \ - -Dhive.hadoop2.wasb.container=${WASB_CONTAINER} \ - -Dhive.hadoop2.wasb.account=${WASB_ACCOUNT} \ - -Dhive.hadoop2.wasb.accessKey=${WASB_ACCESS_KEY} \ - -Dhive.hadoop2.wasb.testDirectory=${test_directory} -EXIT_CODE=$? -set -e -popd - -cleanup_hadoop_docker_containers - -exit ${EXIT_CODE} diff --git a/plugin/trino-hive-hadoop2/conf/files/core-site.xml.abfs-access-key-template b/plugin/trino-hive-hadoop2/conf/files/core-site.xml.abfs-access-key-template deleted file mode 100644 index 7f807e26e0c1f..0000000000000 --- a/plugin/trino-hive-hadoop2/conf/files/core-site.xml.abfs-access-key-template +++ /dev/null @@ -1,40 +0,0 @@ - - - - - - - hadoop.proxyuser.hive.hosts - * - - - - hadoop.proxyuser.hive.groups - * - - - - fs.defaultFS - hdfs://hadoop-master:9000 - - - - fs.azure.account.key.%ABFS_ACCOUNT%.dfs.core.windows.net - %ABFS_ACCESS_KEY% - - diff --git a/plugin/trino-hive-hadoop2/conf/files/core-site.xml.abfs-oauth-template b/plugin/trino-hive-hadoop2/conf/files/core-site.xml.abfs-oauth-template deleted file mode 100644 index 48869661ea995..0000000000000 --- a/plugin/trino-hive-hadoop2/conf/files/core-site.xml.abfs-oauth-template +++ /dev/null @@ -1,60 +0,0 @@ - - - - - - hadoop.proxyuser.hive.hosts - * - - - - hadoop.proxyuser.hive.groups - * - - - - fs.defaultFS - hdfs://hadoop-master:9000 - - - - - fs.azure.account.auth.type - OAuth - - - - fs.azure.account.oauth.provider.type - org.apache.hadoop.fs.azurebfs.oauth2.ClientCredsTokenProvider - - - - fs.azure.account.oauth2.client.id - %ABFS_OAUTH_CLIENTID% - - - - fs.azure.account.oauth2.client.secret - %ABFS_OAUTH_SECRET% - - - - fs.azure.account.oauth2.client.endpoint - %ABFS_OAUTH_ENDPOINT% - - diff --git a/plugin/trino-hive-hadoop2/conf/files/core-site.xml.adl-template b/plugin/trino-hive-hadoop2/conf/files/core-site.xml.adl-template deleted file mode 100644 index bb0fa9f47e3cb..0000000000000 --- a/plugin/trino-hive-hadoop2/conf/files/core-site.xml.adl-template +++ /dev/null @@ -1,54 +0,0 @@ - - - - - - hadoop.proxyuser.hive.hosts - * - - - - hadoop.proxyuser.hive.groups - * - - - - fs.defaultFS - hdfs://hadoop-master:9000 - - - - fs.adl.oauth2.access.token.provider.type - ClientCredential - - - - fs.adl.oauth2.client.id - %ADL_CLIENT_ID% - - - - fs.adl.oauth2.credential - %ADL_CREDENTIAL% - - - - fs.adl.oauth2.refresh.url - %ADL_REFRESH_URL% - - diff --git a/plugin/trino-hive-hadoop2/conf/files/core-site.xml.wasb-template b/plugin/trino-hive-hadoop2/conf/files/core-site.xml.wasb-template deleted file mode 100644 index 74f28465c0a62..0000000000000 --- a/plugin/trino-hive-hadoop2/conf/files/core-site.xml.wasb-template +++ /dev/null @@ -1,40 +0,0 @@ - - - - - - - hadoop.proxyuser.hive.hosts - * - - - - hadoop.proxyuser.hive.groups - * - - - - fs.defaultFS - hdfs://hadoop-master:9000 - - - - fs.azure.account.key.%WASB_ACCOUNT%.blob.core.windows.net - %WASB_ACCESS_KEY% - - diff --git a/plugin/trino-hive-hadoop2/pom.xml b/plugin/trino-hive-hadoop2/pom.xml index 7f94db9ac7926..535dca91b021c 100644 --- a/plugin/trino-hive-hadoop2/pom.xml +++ b/plugin/trino-hive-hadoop2/pom.xml @@ -232,11 +232,6 @@ **/TestHive.java **/TestHiveThriftMetastoreWithS3.java - **/TestHiveFileSystemS3.java - **/TestHiveFileSystemWasb.java - **/TestHiveFileSystemAbfsAccessKey.java - **/TestHiveFileSystemAbfsOAuth.java - **/TestHiveFileSystemAdl.java @@ -269,71 +264,6 @@ **/TestHiveThriftMetastoreWithS3.java - **/TestHiveFileSystemS3.java - - - - - - - - test-hive-hadoop2-wasb - - - - org.apache.maven.plugins - maven-surefire-plugin - - - **/TestHiveFileSystemWasb.java - - - - - - - - test-hive-hadoop2-abfs-access-key - - - - org.apache.maven.plugins - maven-surefire-plugin - - - **/TestHiveFileSystemAbfsAccessKey.java - - - - - - - - test-hive-hadoop2-abfs-oauth - - - - org.apache.maven.plugins - maven-surefire-plugin - - - **/TestHiveFileSystemAbfsOAuth.java - - - - - - - - test-hive-hadoop2-adl - - - - org.apache.maven.plugins - maven-surefire-plugin - - - **/TestHiveFileSystemAdl.java diff --git a/plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/AbstractTestHiveFileSystemAbfs.java b/plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/AbstractTestHiveFileSystemAbfs.java deleted file mode 100644 index 8daba40b523bd..0000000000000 --- a/plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/AbstractTestHiveFileSystemAbfs.java +++ /dev/null @@ -1,114 +0,0 @@ -/* - * 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.hive; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; -import io.trino.hdfs.ConfigurationInitializer; -import io.trino.hdfs.DynamicHdfsConfiguration; -import io.trino.hdfs.HdfsConfig; -import io.trino.hdfs.HdfsConfiguration; -import io.trino.hdfs.HdfsConfigurationInitializer; -import io.trino.hdfs.azure.HiveAzureConfig; -import io.trino.hdfs.azure.TrinoAzureConfigurationInitializer; -import io.trino.plugin.hive.AbstractTestHive.Transaction; -import io.trino.spi.connector.ColumnMetadata; -import io.trino.spi.connector.ConnectorTableMetadata; -import io.trino.spi.connector.SchemaTableName; -import org.apache.hadoop.fs.Path; - -import java.util.Map; -import java.util.Optional; - -import static com.google.common.base.Preconditions.checkArgument; -import static io.trino.plugin.hive.HiveTableProperties.BUCKETED_BY_PROPERTY; -import static io.trino.plugin.hive.HiveTableProperties.BUCKET_COUNT_PROPERTY; -import static io.trino.plugin.hive.HiveTableProperties.EXTERNAL_LOCATION_PROPERTY; -import static io.trino.plugin.hive.HiveTableProperties.SKIP_FOOTER_LINE_COUNT; -import static io.trino.plugin.hive.HiveTableProperties.SKIP_HEADER_LINE_COUNT; -import static io.trino.plugin.hive.HiveTableProperties.SORTED_BY_PROPERTY; -import static io.trino.plugin.hive.HiveTableProperties.STORAGE_FORMAT_PROPERTY; -import static io.trino.spi.type.BigintType.BIGINT; -import static java.lang.String.format; -import static org.assertj.core.util.Strings.isNullOrEmpty; - -public abstract class AbstractTestHiveFileSystemAbfs - extends AbstractTestHiveFileSystem -{ - protected String account; - protected String container; - protected String testDirectory; - - protected static String checkParameter(String value, String name) - { - checkArgument(!isNullOrEmpty(value), "expected non-empty %s", name); - return value; - } - - protected void setup(String host, int port, String databaseName, String container, String account, String testDirectory) - { - this.container = checkParameter(container, "container"); - this.account = checkParameter(account, "account"); - this.testDirectory = checkParameter(testDirectory, "test directory"); - super.setup( - checkParameter(host, "host"), - port, - checkParameter(databaseName, "database name"), - createHdfsConfiguration()); - } - - @Override - protected void onSetupComplete() - { - ensureTableExists(table, "trino_test_external_fs", ImmutableMap.of()); - ensureTableExists(tableWithHeader, "trino_test_external_fs_with_header", ImmutableMap.of(SKIP_HEADER_LINE_COUNT, 1)); - ensureTableExists(tableWithHeaderAndFooter, "trino_test_external_fs_with_header_and_footer", ImmutableMap.of(SKIP_HEADER_LINE_COUNT, 2, SKIP_FOOTER_LINE_COUNT, 2)); - } - - private void ensureTableExists(SchemaTableName table, String tableDirectoryName, Map tableProperties) - { - try (Transaction transaction = newTransaction()) { - ConnectorTableMetadata tableMetadata = new ConnectorTableMetadata( - table, - ImmutableList.of(new ColumnMetadata("t_bigint", BIGINT)), - ImmutableMap.builder() - .putAll(tableProperties) - .put(STORAGE_FORMAT_PROPERTY, HiveStorageFormat.TEXTFILE) - .put(EXTERNAL_LOCATION_PROPERTY, getBasePath().toString() + "/" + tableDirectoryName) - .put(BUCKET_COUNT_PROPERTY, 0) - .put(BUCKETED_BY_PROPERTY, ImmutableList.of()) - .put(SORTED_BY_PROPERTY, ImmutableList.of()) - .buildOrThrow()); - if (!transaction.getMetadata().listTables(newSession(), Optional.of(table.getSchemaName())).contains(table)) { - transaction.getMetadata().createTable(newSession(), tableMetadata, false); - } - transaction.commit(); - } - } - - protected abstract HiveAzureConfig getConfig(); - - private HdfsConfiguration createHdfsConfiguration() - { - ConfigurationInitializer initializer = new TrinoAzureConfigurationInitializer(getConfig()); - return new DynamicHdfsConfiguration(new HdfsConfigurationInitializer(new HdfsConfig(), ImmutableSet.of(initializer)), ImmutableSet.of()); - } - - @Override - protected Path getBasePath() - { - return new Path(format("abfs://%s@%s.dfs.core.windows.net/%s/", container, account, testDirectory)); - } -} diff --git a/plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/TestHiveFileSystemAbfsAccessKey.java b/plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/TestHiveFileSystemAbfsAccessKey.java deleted file mode 100644 index 539fc8ffcc47b..0000000000000 --- a/plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/TestHiveFileSystemAbfsAccessKey.java +++ /dev/null @@ -1,48 +0,0 @@ -/* - * 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.hive; - -import io.trino.hdfs.azure.HiveAzureConfig; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.TestInstance; - -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; - -@TestInstance(PER_CLASS) -public class TestHiveFileSystemAbfsAccessKey - extends AbstractTestHiveFileSystemAbfs -{ - private String accessKey; - - @BeforeAll - public void setup() - { - this.accessKey = checkParameter(System.getProperty("hive.hadoop2.abfs.accessKey"), "access key"); - super.setup( - System.getProperty("hive.hadoop2.metastoreHost"), - Integer.getInteger("hive.hadoop2.metastorePort"), - System.getProperty("hive.hadoop2.databaseName"), - System.getProperty("hive.hadoop2.abfs.container"), - System.getProperty("hive.hadoop2.abfs.account"), - System.getProperty("hive.hadoop2.abfs.testDirectory")); - } - - @Override - protected HiveAzureConfig getConfig() - { - return new HiveAzureConfig() - .setAbfsAccessKey(accessKey) - .setAbfsStorageAccount(account); - } -} diff --git a/plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/TestHiveFileSystemAbfsOAuth.java b/plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/TestHiveFileSystemAbfsOAuth.java deleted file mode 100644 index 36adb3a9db31b..0000000000000 --- a/plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/TestHiveFileSystemAbfsOAuth.java +++ /dev/null @@ -1,53 +0,0 @@ -/* - * 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.hive; - -import io.trino.hdfs.azure.HiveAzureConfig; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.TestInstance; - -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; - -@TestInstance(PER_CLASS) -public class TestHiveFileSystemAbfsOAuth - extends AbstractTestHiveFileSystemAbfs -{ - private String endpoint; - private String clientId; - private String secret; - - @BeforeAll - public void setup() - { - this.endpoint = checkParameter(System.getProperty("test.hive.azure.abfs.oauth.endpoint"), "endpoint"); - this.clientId = checkParameter(System.getProperty("test.hive.azure.abfs.oauth.client-id"), "client ID"); - this.secret = checkParameter(System.getProperty("test.hive.azure.abfs.oauth.secret"), "secret"); - super.setup( - System.getProperty("hive.hadoop2.metastoreHost"), - Integer.getInteger("hive.hadoop2.metastorePort"), - System.getProperty("hive.hadoop2.databaseName"), - System.getProperty("test.hive.azure.abfs.container"), - System.getProperty("test.hive.azure.abfs.storage-account"), - System.getProperty("test.hive.azure.abfs.test-directory")); - } - - @Override - protected HiveAzureConfig getConfig() - { - return new HiveAzureConfig() - .setAbfsOAuthClientEndpoint(endpoint) - .setAbfsOAuthClientId(clientId) - .setAbfsOAuthClientSecret(secret); - } -} diff --git a/plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/TestHiveFileSystemAdl.java b/plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/TestHiveFileSystemAdl.java deleted file mode 100644 index 525bf50631065..0000000000000 --- a/plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/TestHiveFileSystemAdl.java +++ /dev/null @@ -1,162 +0,0 @@ -/* - * 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.hive; - -import com.google.common.collect.ImmutableSet; -import io.trino.hdfs.ConfigurationInitializer; -import io.trino.hdfs.DynamicHdfsConfiguration; -import io.trino.hdfs.HdfsConfig; -import io.trino.hdfs.HdfsConfiguration; -import io.trino.hdfs.HdfsConfigurationInitializer; -import io.trino.hdfs.azure.HiveAzureConfig; -import io.trino.hdfs.azure.TrinoAzureConfigurationInitializer; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; - -import java.io.FileNotFoundException; -import java.util.UUID; - -import static com.google.common.base.Preconditions.checkArgument; -import static java.lang.String.format; -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.assertj.core.util.Strings.isNullOrEmpty; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; - -@TestInstance(PER_CLASS) -public class TestHiveFileSystemAdl - extends AbstractTestHiveFileSystem -{ - private String dataLakeName; - private String clientId; - private String credential; - private String refreshUrl; - private String testDirectory; - - @BeforeAll - public void setup() - { - String host = System.getProperty("hive.hadoop2.metastoreHost"); - int port = Integer.getInteger("hive.hadoop2.metastorePort"); - String databaseName = System.getProperty("hive.hadoop2.databaseName"); - String dataLakeName = System.getProperty("hive.hadoop2.adl.name"); - String clientId = System.getProperty("hive.hadoop2.adl.clientId"); - String credential = System.getProperty("hive.hadoop2.adl.credential"); - String refreshUrl = System.getProperty("hive.hadoop2.adl.refreshUrl"); - String testDirectory = System.getProperty("hive.hadoop2.adl.testDirectory"); - - checkArgument(!isNullOrEmpty(host), "expected non empty host"); - checkArgument(!isNullOrEmpty(databaseName), "expected non empty databaseName"); - checkArgument(!isNullOrEmpty(dataLakeName), "expected non empty dataLakeName"); - checkArgument(!isNullOrEmpty(clientId), "expected non empty clientId"); - checkArgument(!isNullOrEmpty(credential), "expected non empty credential"); - checkArgument(!isNullOrEmpty(refreshUrl), "expected non empty refreshUrl"); - checkArgument(!isNullOrEmpty(testDirectory), "expected non empty testDirectory"); - - this.dataLakeName = dataLakeName; - this.clientId = clientId; - this.credential = credential; - this.refreshUrl = refreshUrl; - this.testDirectory = testDirectory; - - super.setup(host, port, databaseName, createHdfsConfiguration()); - } - - private HdfsConfiguration createHdfsConfiguration() - { - ConfigurationInitializer azureConfig = new TrinoAzureConfigurationInitializer(new HiveAzureConfig() - .setAdlClientId(clientId) - .setAdlCredential(credential) - .setAdlRefreshUrl(refreshUrl)); - return new DynamicHdfsConfiguration(new HdfsConfigurationInitializer(new HdfsConfig(), ImmutableSet.of(azureConfig)), ImmutableSet.of()); - } - - @Override - protected Path getBasePath() - { - return new Path(format("adl://%s.azuredatalakestore.net/%s/", dataLakeName, testDirectory)); - } - - @Override - @Test - public void testRename() - throws Exception - { - Path basePath = new Path(getBasePath(), UUID.randomUUID().toString()); - FileSystem fs = hdfsEnvironment.getFileSystem(TESTING_CONTEXT, basePath); - assertThat(fs.exists(basePath)).isFalse(); - - // create file foo.txt - Path path = new Path(basePath, "foo.txt"); - assertThat(fs.createNewFile(path)).isTrue(); - assertThat(fs.exists(path)).isTrue(); - - // rename foo.txt to bar.txt when bar does not exist - Path newPath = new Path(basePath, "bar.txt"); - assertThat(fs.exists(newPath)).isFalse(); - assertThat(fs.rename(path, newPath)).isTrue(); - assertThat(fs.exists(path)).isFalse(); - assertThat(fs.exists(newPath)).isTrue(); - - // rename foo.txt to foo.txt when foo.txt does not exist - // This fails with error no such file in ADLFileSystem - assertThatThrownBy(() -> fs.rename(path, path)) - .isInstanceOf(FileNotFoundException.class); - - // create file foo.txt and rename to existing bar.txt - assertThat(fs.createNewFile(path)).isTrue(); - assertThat(fs.rename(path, newPath)).isFalse(); - - // rename foo.txt to foo.txt when foo.txt exists - // This returns true in ADLFileSystem - assertThat(fs.rename(path, path)).isTrue(); - - // delete foo.txt - assertThat(fs.delete(path, false)).isTrue(); - assertThat(fs.exists(path)).isFalse(); - - // create directory source with file - Path source = new Path(basePath, "source"); - assertThat(fs.createNewFile(new Path(source, "test.txt"))).isTrue(); - - // rename source to non-existing target - Path target = new Path(basePath, "target"); - assertThat(fs.exists(target)).isFalse(); - assertThat(fs.rename(source, target)).isTrue(); - assertThat(fs.exists(source)).isFalse(); - assertThat(fs.exists(target)).isTrue(); - - // create directory source with file - assertThat(fs.createNewFile(new Path(source, "test.txt"))).isTrue(); - - // rename source to existing target - assertThat(fs.rename(source, target)).isTrue(); - assertThat(fs.exists(source)).isFalse(); - target = new Path(target, "source"); - assertThat(fs.exists(target)).isTrue(); - assertThat(fs.exists(new Path(target, "test.txt"))).isTrue(); - - // delete target - target = new Path(basePath, "target"); - assertThat(fs.exists(target)).isTrue(); - assertThat(fs.delete(target, true)).isTrue(); - assertThat(fs.exists(target)).isFalse(); - - // cleanup - fs.delete(basePath, true); - } -} diff --git a/plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/TestHiveFileSystemS3.java b/plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/TestHiveFileSystemS3.java deleted file mode 100644 index 5ff770ac39ff5..0000000000000 --- a/plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/TestHiveFileSystemS3.java +++ /dev/null @@ -1,262 +0,0 @@ -/* - * 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.hive; - -import com.amazonaws.auth.AWSStaticCredentialsProvider; -import com.amazonaws.auth.BasicAWSCredentials; -import com.amazonaws.client.builder.AwsClientBuilder; -import com.amazonaws.services.s3.AmazonS3; -import com.amazonaws.services.s3.AmazonS3Client; -import com.amazonaws.services.s3.model.ObjectMetadata; -import com.amazonaws.services.s3.model.PutObjectRequest; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Streams; -import com.google.common.net.MediaType; -import io.trino.filesystem.Location; -import io.trino.filesystem.TrinoFileSystem; -import io.trino.filesystem.hdfs.HdfsFileSystemFactory; -import io.trino.hdfs.ConfigurationInitializer; -import io.trino.hdfs.DynamicHdfsConfiguration; -import io.trino.hdfs.HdfsConfig; -import io.trino.hdfs.HdfsConfiguration; -import io.trino.hdfs.HdfsConfigurationInitializer; -import io.trino.hdfs.TrinoHdfsFileSystemStats; -import io.trino.hdfs.s3.HiveS3Config; -import io.trino.hdfs.s3.TrinoS3ConfigurationInitializer; -import io.trino.plugin.hive.fs.FileSystemDirectoryLister; -import io.trino.plugin.hive.fs.HiveFileIterator; -import io.trino.plugin.hive.fs.TrinoFileStatus; -import io.trino.plugin.hive.metastore.Column; -import io.trino.plugin.hive.metastore.StorageFormat; -import io.trino.plugin.hive.metastore.Table; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; - -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.Optional; - -import static com.google.common.base.Preconditions.checkArgument; -import static io.trino.plugin.hive.HiveTestUtils.SESSION; -import static io.trino.plugin.hive.HiveType.HIVE_LONG; -import static io.trino.plugin.hive.HiveType.HIVE_STRING; -import static java.io.InputStream.nullInputStream; -import static java.lang.String.format; -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.util.Strings.isNullOrEmpty; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; - -@TestInstance(PER_CLASS) -public class TestHiveFileSystemS3 - extends AbstractTestHiveFileSystem -{ - private static final MediaType DIRECTORY_MEDIA_TYPE = MediaType.create("application", "x-directory"); - private String awsAccessKey; - private String awsSecretKey; - private String writableBucket; - private String testDirectory; - private AmazonS3 s3Client; - - @BeforeAll - public void setup() - { - String host = System.getProperty("hive.hadoop2.metastoreHost"); - int port = Integer.getInteger("hive.hadoop2.metastorePort"); - String databaseName = System.getProperty("hive.hadoop2.databaseName"); - String s3endpoint = System.getProperty("hive.hadoop2.s3.endpoint"); - String awsAccessKey = System.getProperty("hive.hadoop2.s3.awsAccessKey"); - String awsSecretKey = System.getProperty("hive.hadoop2.s3.awsSecretKey"); - String writableBucket = System.getProperty("hive.hadoop2.s3.writableBucket"); - String testDirectory = System.getProperty("hive.hadoop2.s3.testDirectory"); - - checkArgument(!isNullOrEmpty(host), "Expected non empty host"); - checkArgument(!isNullOrEmpty(databaseName), "Expected non empty databaseName"); - checkArgument(!isNullOrEmpty(awsAccessKey), "Expected non empty awsAccessKey"); - checkArgument(!isNullOrEmpty(awsSecretKey), "Expected non empty awsSecretKey"); - checkArgument(!isNullOrEmpty(s3endpoint), "Expected non empty s3endpoint"); - checkArgument(!isNullOrEmpty(writableBucket), "Expected non empty writableBucket"); - checkArgument(!isNullOrEmpty(testDirectory), "Expected non empty testDirectory"); - this.awsAccessKey = awsAccessKey; - this.awsSecretKey = awsSecretKey; - this.writableBucket = writableBucket; - this.testDirectory = testDirectory; - - s3Client = AmazonS3Client.builder() - .withEndpointConfiguration(new AwsClientBuilder.EndpointConfiguration(s3endpoint, null)) - .withCredentials(new AWSStaticCredentialsProvider(new BasicAWSCredentials(awsAccessKey, awsSecretKey))) - .build(); - - setup(host, port, databaseName, createHdfsConfiguration()); - } - - private HdfsConfiguration createHdfsConfiguration() - { - ConfigurationInitializer s3Config = new TrinoS3ConfigurationInitializer(new HiveS3Config() - .setS3AwsAccessKey(awsAccessKey) - .setS3AwsSecretKey(awsSecretKey)); - HdfsConfigurationInitializer initializer = new HdfsConfigurationInitializer(new HdfsConfig(), ImmutableSet.of(s3Config)); - return new DynamicHdfsConfiguration(initializer, ImmutableSet.of()); - } - - @Override - protected Path getBasePath() - { - // HDP 3.1 does not understand s3:// out of the box. - return new Path(format("s3a://%s/%s/", writableBucket, testDirectory)); - } - - @Test - public void testIgnoreHadoopFolderMarker() - throws Exception - { - Path basePath = getBasePath(); - FileSystem fs = hdfsEnvironment.getFileSystem(TESTING_CONTEXT, basePath); - - String markerFileName = "test_table_$folder$"; - Path filePath = new Path(basePath, markerFileName); - fs.create(filePath).close(); - - assertThat(Arrays.stream(fs.listStatus(basePath)).anyMatch(file -> file.getPath().getName().equalsIgnoreCase(markerFileName))).isFalse(); - } - - /** - * Tests the same functionality like {@link #testFileIteratorPartitionedListing()} with the - * setup done by native {@link AmazonS3} - */ - @Test - public void testFileIteratorPartitionedListingNativeS3Client() - throws Exception - { - Table.Builder tableBuilder = Table.builder() - .setDatabaseName(table.getSchemaName()) - .setTableName(table.getTableName()) - .setDataColumns(ImmutableList.of(new Column("data", HIVE_LONG, Optional.empty(), Map.of()))) - .setPartitionColumns(ImmutableList.of(new Column("part", HIVE_STRING, Optional.empty(), Map.of()))) - .setOwner(Optional.empty()) - .setTableType("fake"); - tableBuilder.getStorageBuilder() - .setStorageFormat(StorageFormat.fromHiveStorageFormat(HiveStorageFormat.CSV)); - Table fakeTable = tableBuilder.build(); - - Path basePath = new Path(getBasePath(), "test-file-iterator-partitioned-listing-native-setup"); - FileSystem fs = hdfsEnvironment.getFileSystem(TESTING_CONTEXT, basePath); - TrinoFileSystem trinoFileSystem = new HdfsFileSystemFactory(hdfsEnvironment, new TrinoHdfsFileSystemStats()).create(SESSION); - fs.mkdirs(basePath); - String basePrefix = basePath.toUri().getPath().substring(1); - - // Expected file system tree: - // test-file-iterator-partitioned-listing-native-setup/ - // .hidden/ - // nested-file-in-hidden.txt - // part=simple/ - // _hidden-file.txt - // plain-file.txt - // part=nested/ - // parent/ - // _nested-hidden-file.txt - // nested-file.txt - // part=plus+sign/ - // plus-file.txt - // part=percent%sign/ - // percent-file.txt - // part=url%20encoded/ - // url-encoded-file.txt - // part=level1|level2/ - // pipe-file.txt - // parent1/ - // parent2/ - // deeply-nested-file.txt - // part=level1 | level2/ - // pipe-blanks-file.txt - // empty-directory/ - // .hidden-in-base.txt - - createFile(writableBucket, format("%s/.hidden/nested-file-in-hidden.txt", basePrefix)); - createFile(writableBucket, format("%s/part=simple/_hidden-file.txt", basePrefix)); - createFile(writableBucket, format("%s/part=simple/plain-file.txt", basePrefix)); - createFile(writableBucket, format("%s/part=nested/parent/_nested-hidden-file.txt", basePrefix)); - createFile(writableBucket, format("%s/part=nested/parent/nested-file.txt", basePrefix)); - createFile(writableBucket, format("%s/part=plus+sign/plus-file.txt", basePrefix)); - createFile(writableBucket, format("%s/part=percent%%sign/percent-file.txt", basePrefix)); - createFile(writableBucket, format("%s/part=url%%20encoded/url-encoded-file.txt", basePrefix)); - createFile(writableBucket, format("%s/part=level1|level2/pipe-file.txt", basePrefix)); - createFile(writableBucket, format("%s/part=level1|level2/parent1/parent2/deeply-nested-file.txt", basePrefix)); - createFile(writableBucket, format("%s/part=level1 | level2/pipe-blanks-file.txt", basePrefix)); - createDirectory(writableBucket, format("%s/empty-directory/", basePrefix)); - createFile(writableBucket, format("%s/.hidden-in-base.txt", basePrefix)); - - // List recursively through hive file iterator - HiveFileIterator recursiveIterator = new HiveFileIterator( - fakeTable, - Location.of(basePath.toString()), - trinoFileSystem, - new FileSystemDirectoryLister(), - HiveFileIterator.NestedDirectoryPolicy.RECURSE); - - List recursiveListing = Streams.stream(recursiveIterator) - .map(TrinoFileStatus::getPath) - .toList(); - // Should not include directories, or files underneath hidden directories - assertThat(recursiveListing).containsExactlyInAnyOrder( - format("%s/part=simple/plain-file.txt", basePath), - format("%s/part=nested/parent/nested-file.txt", basePath), - format("%s/part=plus+sign/plus-file.txt", basePath), - format("%s/part=percent%%sign/percent-file.txt", basePath), - format("%s/part=url%%20encoded/url-encoded-file.txt", basePath), - format("%s/part=level1|level2/pipe-file.txt", basePath), - format("%s/part=level1|level2/parent1/parent2/deeply-nested-file.txt", basePath), - format("%s/part=level1 | level2/pipe-blanks-file.txt", basePath)); - - HiveFileIterator shallowIterator = new HiveFileIterator( - fakeTable, - Location.of(basePath.toString()), - trinoFileSystem, - new FileSystemDirectoryLister(), - HiveFileIterator.NestedDirectoryPolicy.IGNORED); - List shallowListing = Streams.stream(shallowIterator) - .map(TrinoFileStatus::getPath) - .map(Path::new) - .toList(); - // Should not include any hidden files, folders, or nested files - assertThat(shallowListing).isEmpty(); - } - - protected void createDirectory(String bucketName, String key) - { - // create meta-data for your folder and set content-length to 0 - ObjectMetadata metadata = new ObjectMetadata(); - metadata.setContentLength(0); - metadata.setContentType(DIRECTORY_MEDIA_TYPE.toString()); - // create a PutObjectRequest passing the folder name suffixed by / - if (!key.endsWith("/")) { - key += "/"; - } - PutObjectRequest putObjectRequest = new PutObjectRequest(bucketName, key, nullInputStream(), metadata); - // send request to S3 to create folder - s3Client.putObject(putObjectRequest); - } - - protected void createFile(String bucketName, String key) - { - ObjectMetadata metadata = new ObjectMetadata(); - metadata.setContentLength(0); - PutObjectRequest putObjectRequest = new PutObjectRequest(bucketName, key, nullInputStream(), metadata); - s3Client.putObject(putObjectRequest); - } -} diff --git a/plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/TestHiveFileSystemWasb.java b/plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/TestHiveFileSystemWasb.java deleted file mode 100644 index 08d9eda5c3e46..0000000000000 --- a/plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/TestHiveFileSystemWasb.java +++ /dev/null @@ -1,81 +0,0 @@ -/* - * 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.hive; - -import com.google.common.collect.ImmutableSet; -import io.trino.hdfs.ConfigurationInitializer; -import io.trino.hdfs.DynamicHdfsConfiguration; -import io.trino.hdfs.HdfsConfig; -import io.trino.hdfs.HdfsConfiguration; -import io.trino.hdfs.HdfsConfigurationInitializer; -import io.trino.hdfs.azure.HiveAzureConfig; -import io.trino.hdfs.azure.TrinoAzureConfigurationInitializer; -import org.apache.hadoop.fs.Path; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.TestInstance; - -import static com.google.common.base.Preconditions.checkArgument; -import static java.lang.String.format; -import static org.assertj.core.util.Strings.isNullOrEmpty; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; - -@TestInstance(PER_CLASS) -public class TestHiveFileSystemWasb - extends AbstractTestHiveFileSystem -{ - private String container; - private String account; - private String accessKey; - private String testDirectory; - - @BeforeAll - public void setup() - { - String host = System.getProperty("hive.hadoop2.metastoreHost"); - int port = Integer.getInteger("hive.hadoop2.metastorePort"); - String databaseName = System.getProperty("hive.hadoop2.databaseName"); - String container = System.getProperty("hive.hadoop2.wasb.container"); - String account = System.getProperty("hive.hadoop2.wasb.account"); - String accessKey = System.getProperty("hive.hadoop2.wasb.accessKey"); - String testDirectory = System.getProperty("hive.hadoop2.wasb.testDirectory"); - - checkArgument(!isNullOrEmpty(host), "expected non empty host"); - checkArgument(!isNullOrEmpty(databaseName), "expected non empty databaseName"); - checkArgument(!isNullOrEmpty(container), "expected non empty container"); - checkArgument(!isNullOrEmpty(account), "expected non empty account"); - checkArgument(!isNullOrEmpty(accessKey), "expected non empty accessKey"); - checkArgument(!isNullOrEmpty(testDirectory), "expected non empty testDirectory"); - - this.container = container; - this.account = account; - this.accessKey = accessKey; - this.testDirectory = testDirectory; - - super.setup(host, port, databaseName, createHdfsConfiguration()); - } - - private HdfsConfiguration createHdfsConfiguration() - { - ConfigurationInitializer wasbConfig = new TrinoAzureConfigurationInitializer(new HiveAzureConfig() - .setWasbAccessKey(accessKey) - .setWasbStorageAccount(account)); - return new DynamicHdfsConfiguration(new HdfsConfigurationInitializer(new HdfsConfig(), ImmutableSet.of(wasbConfig)), ImmutableSet.of()); - } - - @Override - protected Path getBasePath() - { - return new Path(format("wasb://%s@%s.blob.core.windows.net/%s/", container, account, testDirectory)); - } -} diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHiveFileSystem.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHiveFileSystem.java deleted file mode 100644 index 6010dfd0b3e58..0000000000000 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHiveFileSystem.java +++ /dev/null @@ -1,942 +0,0 @@ -/* - * 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.hive; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Streams; -import com.google.common.net.HostAndPort; -import io.airlift.concurrent.BoundedExecutor; -import io.airlift.json.JsonCodec; -import io.airlift.slice.Slice; -import io.airlift.stats.CounterStat; -import io.trino.filesystem.Location; -import io.trino.filesystem.TrinoFileSystem; -import io.trino.filesystem.hdfs.HdfsFileSystemFactory; -import io.trino.hdfs.HdfsConfig; -import io.trino.hdfs.HdfsConfiguration; -import io.trino.hdfs.HdfsContext; -import io.trino.hdfs.HdfsEnvironment; -import io.trino.hdfs.TrinoHdfsFileSystemStats; -import io.trino.hdfs.authentication.NoHdfsAuthentication; -import io.trino.operator.GroupByHashPageIndexerFactory; -import io.trino.plugin.base.CatalogName; -import io.trino.plugin.hive.AbstractTestHive.Transaction; -import io.trino.plugin.hive.fs.FileSystemDirectoryLister; -import io.trino.plugin.hive.fs.HiveFileIterator; -import io.trino.plugin.hive.fs.TransactionScopeCachingDirectoryListerFactory; -import io.trino.plugin.hive.fs.TrinoFileStatus; -import io.trino.plugin.hive.metastore.Column; -import io.trino.plugin.hive.metastore.Database; -import io.trino.plugin.hive.metastore.ForwardingHiveMetastore; -import io.trino.plugin.hive.metastore.HiveMetastore; -import io.trino.plugin.hive.metastore.HiveMetastoreConfig; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; -import io.trino.plugin.hive.metastore.PrincipalPrivileges; -import io.trino.plugin.hive.metastore.StorageFormat; -import io.trino.plugin.hive.metastore.Table; -import io.trino.plugin.hive.metastore.thrift.BridgingHiveMetastore; -import io.trino.plugin.hive.security.SqlStandardAccessControlMetadata; -import io.trino.spi.connector.ColumnHandle; -import io.trino.spi.connector.ColumnMetadata; -import io.trino.spi.connector.ConnectorInsertTableHandle; -import io.trino.spi.connector.ConnectorMetadata; -import io.trino.spi.connector.ConnectorOutputTableHandle; -import io.trino.spi.connector.ConnectorPageSink; -import io.trino.spi.connector.ConnectorPageSinkProvider; -import io.trino.spi.connector.ConnectorPageSource; -import io.trino.spi.connector.ConnectorPageSourceProvider; -import io.trino.spi.connector.ConnectorSession; -import io.trino.spi.connector.ConnectorSplit; -import io.trino.spi.connector.ConnectorSplitManager; -import io.trino.spi.connector.ConnectorSplitSource; -import io.trino.spi.connector.ConnectorTableHandle; -import io.trino.spi.connector.ConnectorTableMetadata; -import io.trino.spi.connector.DynamicFilter; -import io.trino.spi.connector.SchemaTableName; -import io.trino.spi.connector.TableNotFoundException; -import io.trino.spi.predicate.TupleDomain; -import io.trino.spi.security.ConnectorIdentity; -import io.trino.spi.type.TypeOperators; -import io.trino.sql.gen.JoinCompiler; -import io.trino.testing.MaterializedResult; -import io.trino.testing.TestingNodeManager; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; -import org.junit.jupiter.api.parallel.Execution; - -import java.io.BufferedReader; -import java.io.IOException; -import java.io.InputStream; -import java.io.InputStreamReader; -import java.io.OutputStream; -import java.io.UncheckedIOException; -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.ScheduledExecutorService; - -import static com.google.common.collect.ImmutableList.toImmutableList; -import static com.google.common.collect.Iterables.getOnlyElement; -import static com.google.common.util.concurrent.MoreExecutors.newDirectExecutorService; -import static io.airlift.concurrent.MoreFutures.getFutureValue; -import static io.airlift.concurrent.Threads.daemonThreadsNamed; -import static io.trino.hdfs.FileSystemUtils.getRawFileSystem; -import static io.trino.plugin.hive.AbstractTestHive.createTableProperties; -import static io.trino.plugin.hive.AbstractTestHive.filterNonHiddenColumnHandles; -import static io.trino.plugin.hive.AbstractTestHive.filterNonHiddenColumnMetadata; -import static io.trino.plugin.hive.AbstractTestHive.getAllSplits; -import static io.trino.plugin.hive.AbstractTestHive.getSplits; -import static io.trino.plugin.hive.HiveTableProperties.EXTERNAL_LOCATION_PROPERTY; -import static io.trino.plugin.hive.HiveTestUtils.HDFS_FILE_SYSTEM_STATS; -import static io.trino.plugin.hive.HiveTestUtils.PAGE_SORTER; -import static io.trino.plugin.hive.HiveTestUtils.SESSION; -import static io.trino.plugin.hive.HiveTestUtils.getDefaultHiveFileWriterFactories; -import static io.trino.plugin.hive.HiveTestUtils.getDefaultHivePageSourceFactories; -import static io.trino.plugin.hive.HiveTestUtils.getHiveSessionProperties; -import static io.trino.plugin.hive.HiveTestUtils.getTypes; -import static io.trino.plugin.hive.HiveType.HIVE_LONG; -import static io.trino.plugin.hive.HiveType.HIVE_STRING; -import static io.trino.plugin.hive.TestingThriftHiveMetastoreBuilder.testingThriftHiveMetastoreBuilder; -import static io.trino.plugin.hive.metastore.PrincipalPrivileges.NO_PRIVILEGES; -import static io.trino.spi.connector.MetadataProvider.NOOP_METADATA_PROVIDER; -import static io.trino.spi.connector.RetryMode.NO_RETRIES; -import static io.trino.spi.type.BigintType.BIGINT; -import static io.trino.testing.MaterializedResult.materializeSourceDataStream; -import static io.trino.testing.QueryAssertions.assertEqualsIgnoreOrder; -import static io.trino.testing.TestingNames.randomNameSuffix; -import static io.trino.testing.TestingPageSinkId.TESTING_PAGE_SINK_ID; -import static io.trino.type.InternalTypeManager.TESTING_TYPE_MANAGER; -import static java.nio.charset.StandardCharsets.UTF_8; -import static java.util.Locale.ENGLISH; -import static java.util.UUID.randomUUID; -import static java.util.concurrent.Executors.newCachedThreadPool; -import static java.util.concurrent.Executors.newScheduledThreadPool; -import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; - -@TestInstance(PER_CLASS) -@Execution(CONCURRENT) -public abstract class AbstractTestHiveFileSystem -{ - protected static final HdfsContext TESTING_CONTEXT = new HdfsContext(ConnectorIdentity.ofUser("test")); - - protected String database; - protected SchemaTableName table; - protected SchemaTableName tableWithHeader; - protected SchemaTableName tableWithHeaderAndFooter; - protected SchemaTableName temporaryCreateTable; - protected SchemaTableName temporaryCreateTableWithExternalLocation; - - protected HdfsEnvironment hdfsEnvironment; - protected LocationService locationService; - protected TestingHiveMetastore metastoreClient; - protected HiveMetadataFactory metadataFactory; - protected HiveTransactionManager transactionManager; - protected ConnectorSplitManager splitManager; - protected ConnectorPageSinkProvider pageSinkProvider; - protected ConnectorPageSourceProvider pageSourceProvider; - - private ExecutorService executor; - private HiveConfig config; - private ScheduledExecutorService heartbeatService; - - @BeforeAll - public void setUp() - { - executor = newCachedThreadPool(daemonThreadsNamed("hive-%s")); - heartbeatService = newScheduledThreadPool(1); - } - - @AfterAll - public void tearDown() - { - if (executor != null) { - executor.shutdownNow(); - executor = null; - } - if (heartbeatService != null) { - heartbeatService.shutdownNow(); - heartbeatService = null; - } - } - - protected abstract Path getBasePath(); - - protected void onSetupComplete() {} - - protected void setup(String host, int port, String databaseName, HdfsConfiguration hdfsConfiguration) - { - database = databaseName; - table = new SchemaTableName(database, "trino_test_external_fs"); - tableWithHeader = new SchemaTableName(database, "trino_test_external_fs_with_header"); - tableWithHeaderAndFooter = new SchemaTableName(database, "trino_test_external_fs_with_header_and_footer"); - - String random = randomUUID().toString().toLowerCase(ENGLISH).replace("-", ""); - temporaryCreateTable = new SchemaTableName(database, "tmp_trino_test_create_" + random); - temporaryCreateTableWithExternalLocation = new SchemaTableName(database, "tmp_trino_test_create_external" + random); - - config = new HiveConfig() - .setWritesToNonManagedTablesEnabled(true); - - HivePartitionManager hivePartitionManager = new HivePartitionManager(config); - - hdfsEnvironment = new HdfsEnvironment(hdfsConfiguration, new HdfsConfig(), new NoHdfsAuthentication()); - metastoreClient = new TestingHiveMetastore( - new BridgingHiveMetastore( - testingThriftHiveMetastoreBuilder() - .metastoreClient(HostAndPort.fromParts(host, port)) - .hiveConfig(config) - .fileSystemFactory(new HdfsFileSystemFactory(hdfsEnvironment, HDFS_FILE_SYSTEM_STATS)) - .build()), - getBasePath(), - hdfsEnvironment); - HdfsFileSystemFactory fileSystemFactory = new HdfsFileSystemFactory(hdfsEnvironment, HDFS_FILE_SYSTEM_STATS); - locationService = new HiveLocationService(fileSystemFactory, config); - JsonCodec partitionUpdateCodec = JsonCodec.jsonCodec(PartitionUpdate.class); - metadataFactory = new HiveMetadataFactory( - new CatalogName("hive"), - config, - new HiveMetastoreConfig(), - HiveMetastoreFactory.ofInstance(metastoreClient), - getDefaultHiveFileWriterFactories(config, hdfsEnvironment), - fileSystemFactory, - hivePartitionManager, - newDirectExecutorService(), - heartbeatService, - TESTING_TYPE_MANAGER, - NOOP_METADATA_PROVIDER, - locationService, - partitionUpdateCodec, - new NodeVersion("test_version"), - new NoneHiveRedirectionsProvider(), - ImmutableSet.of( - new PartitionsSystemTableProvider(hivePartitionManager, TESTING_TYPE_MANAGER), - new PropertiesSystemTableProvider()), - new DefaultHiveMaterializedViewMetadataFactory(), - SqlStandardAccessControlMetadata::new, - new FileSystemDirectoryLister(), - new TransactionScopeCachingDirectoryListerFactory(config), - true); - transactionManager = new HiveTransactionManager(metadataFactory); - splitManager = new HiveSplitManager( - transactionManager, - hivePartitionManager, - fileSystemFactory, - new BoundedExecutor(executor, config.getMaxSplitIteratorThreads()), - new CounterStat(), - config.getMaxOutstandingSplits(), - config.getMaxOutstandingSplitsSize(), - config.getMinPartitionBatchSize(), - config.getMaxPartitionBatchSize(), - config.getMaxInitialSplits(), - config.getSplitLoaderConcurrency(), - config.getMaxSplitsPerSecond(), - config.getRecursiveDirWalkerEnabled(), - TESTING_TYPE_MANAGER, - config.getMaxPartitionsPerScan()); - pageSinkProvider = new HivePageSinkProvider( - getDefaultHiveFileWriterFactories(config, hdfsEnvironment), - fileSystemFactory, - PAGE_SORTER, - HiveMetastoreFactory.ofInstance(metastoreClient), - new GroupByHashPageIndexerFactory(new JoinCompiler(new TypeOperators())), - TESTING_TYPE_MANAGER, - config, - new SortingFileWriterConfig(), - locationService, - partitionUpdateCodec, - new TestingNodeManager("fake-environment"), - new HiveEventClient(), - getHiveSessionProperties(config), - new HiveWriterStats()); - pageSourceProvider = new HivePageSourceProvider( - TESTING_TYPE_MANAGER, - config, - getDefaultHivePageSourceFactories(hdfsEnvironment, config)); - - onSetupComplete(); - } - - protected ConnectorSession newSession() - { - return HiveFileSystemTestUtils.newSession(config); - } - - protected Transaction newTransaction() - { - return HiveFileSystemTestUtils.newTransaction(transactionManager); - } - - protected MaterializedResult readTable(SchemaTableName tableName) - throws IOException - { - return HiveFileSystemTestUtils.readTable(tableName, transactionManager, config, pageSourceProvider, splitManager); - } - - @Test - public void testGetRecords() - throws Exception - { - assertEqualsIgnoreOrder( - readTable(table), - MaterializedResult.resultBuilder(newSession(), BIGINT) - .row(3L).row(14L).row(15L) // test_table.csv - .row(92L).row(65L).row(35L) // test_table.csv.gz - .row(89L).row(79L).row(32L) // test_table.csv.bz2 - .row(38L).row(46L).row(26L) // test_table.csv.lz4 - .build()); - } - - @Test - public void testGetRecordsWithHeader() - throws IOException - { - assertEqualsIgnoreOrder( - readTable(tableWithHeader), - MaterializedResult.resultBuilder(newSession(), BIGINT) - .row(2L).row(71L).row(82L) // test_table_with_header.csv - .row(81L).row(82L).row(84L) // test_table_with_header.csv.gz - .row(59L).row(4L).row(52L) // test_table_with_header.csv.bz2 - .row(35L).row(36L).row(2L) // test_table_with_header.csv.lz4 - .build()); - } - - @Test - public void testGetRecordsWithHeaderAndFooter() - throws IOException - { - assertEqualsIgnoreOrder( - readTable(tableWithHeaderAndFooter), - MaterializedResult.resultBuilder(newSession(), BIGINT) - .row(1L).row(41L).row(42L) // test_table_with_header_and_footer.csv - .row(13L).row(56L).row(23L) // test_table_with_header_and_footer.csv.gz - .row(73L).row(9L).row(50L) // test_table_with_header_and_footer.csv.bz2 - .row(48L).row(80L).row(16L) // test_table_with_header_and_footer.csv.lz4 - .build()); - } - - @Test - public void testGetFileStatus() - throws Exception - { - Path basePath = getBasePath(); - Path tablePath = new Path(basePath, "trino_test_external_fs"); - Path filePath = new Path(tablePath, "test_table.csv"); - FileSystem fs = hdfsEnvironment.getFileSystem(TESTING_CONTEXT, basePath); - - assertThat(fs.getFileStatus(basePath).isDirectory()) - .describedAs("basePath should be considered a directory") - .isTrue(); - assertThat(fs.getFileStatus(tablePath).isDirectory()) - .describedAs("tablePath should be considered a directory") - .isTrue(); - assertThat(fs.getFileStatus(filePath).isFile()) - .describedAs("filePath should be considered a file") - .isTrue(); - assertThat(fs.getFileStatus(filePath).isDirectory()) - .describedAs("filePath should not be considered a directory") - .isFalse(); - assertThat(fs.exists(new Path(basePath, "foo-" + randomUUID()))) - .describedAs("foo-random path should be found not to exist") - .isFalse(); - assertThat(fs.exists(new Path(basePath, "foo"))) - .describedAs("foo path should be found not to exist") - .isFalse(); - } - - @Test - public void testRename() - throws Exception - { - Path basePath = new Path(getBasePath(), randomUUID().toString()); - FileSystem fs = hdfsEnvironment.getFileSystem(TESTING_CONTEXT, basePath); - assertThat(fs.exists(basePath)).isFalse(); - - // create file foo.txt - Path path = new Path(basePath, "foo.txt"); - assertThat(fs.createNewFile(path)).isTrue(); - assertThat(fs.exists(path)).isTrue(); - - // rename foo.txt to bar.txt when bar does not exist - Path newPath = new Path(basePath, "bar.txt"); - assertThat(fs.exists(newPath)).isFalse(); - assertThat(fs.rename(path, newPath)).isTrue(); - assertThat(fs.exists(path)).isFalse(); - assertThat(fs.exists(newPath)).isTrue(); - - // rename foo.txt to foo.txt when foo.txt does not exist - assertThat(fs.rename(path, path)).isFalse(); - - // create file foo.txt and rename to existing bar.txt - assertThat(fs.createNewFile(path)).isTrue(); - assertThat(fs.rename(path, newPath)).isFalse(); - - // rename foo.txt to foo.txt when foo.txt exists - assertThat(fs.rename(path, path)).isEqualTo(getRawFileSystem(fs) instanceof AzureBlobFileSystem); - - // delete foo.txt - assertThat(fs.delete(path, false)).isTrue(); - assertThat(fs.exists(path)).isFalse(); - - // create directory source with file - Path source = new Path(basePath, "source"); - assertThat(fs.createNewFile(new Path(source, "test.txt"))).isTrue(); - - // rename source to non-existing target - Path target = new Path(basePath, "target"); - assertThat(fs.exists(target)).isFalse(); - assertThat(fs.rename(source, target)).isTrue(); - assertThat(fs.exists(source)).isFalse(); - assertThat(fs.exists(target)).isTrue(); - - // create directory source with file - assertThat(fs.createNewFile(new Path(source, "test.txt"))).isTrue(); - - // rename source to existing target - assertThat(fs.rename(source, target)).isTrue(); - assertThat(fs.exists(source)).isFalse(); - target = new Path(target, "source"); - assertThat(fs.exists(target)).isTrue(); - assertThat(fs.exists(new Path(target, "test.txt"))).isTrue(); - - // delete target - target = new Path(basePath, "target"); - assertThat(fs.exists(target)).isTrue(); - assertThat(fs.delete(target, true)).isTrue(); - assertThat(fs.exists(target)).isFalse(); - - // cleanup - fs.delete(basePath, true); - } - - @Test - public void testFileIteratorListing() - throws Exception - { - Table.Builder tableBuilder = Table.builder() - .setDatabaseName(table.getSchemaName()) - .setTableName(table.getTableName()) - .setDataColumns(ImmutableList.of(new Column("one", HIVE_LONG, Optional.empty(), Map.of()))) - .setPartitionColumns(ImmutableList.of()) - .setOwner(Optional.empty()) - .setTableType("fake"); - tableBuilder.getStorageBuilder() - .setStorageFormat(StorageFormat.fromHiveStorageFormat(HiveStorageFormat.CSV)); - Table fakeTable = tableBuilder.build(); - - // Expected file system tree: - // test-file-iterator-listing/ - // .hidden/ - // nested-file-in-hidden.txt - // parent/ - // _nested-hidden-file.txt - // nested-file.txt - // empty-directory/ - // .hidden-in-base.txt - // base-path-file.txt - Path basePath = new Path(getBasePath(), "test-file-iterator-listing"); - FileSystem fs = hdfsEnvironment.getFileSystem(TESTING_CONTEXT, basePath); - TrinoFileSystem trinoFileSystem = new HdfsFileSystemFactory(hdfsEnvironment, new TrinoHdfsFileSystemStats()).create(SESSION); - fs.mkdirs(basePath); - - // create file in hidden folder - Path fileInHiddenParent = new Path(new Path(basePath, ".hidden"), "nested-file-in-hidden.txt"); - fs.createNewFile(fileInHiddenParent); - // create hidden file in non-hidden folder - Path nestedHiddenFile = new Path(new Path(basePath, "parent"), "_nested-hidden-file.txt"); - fs.createNewFile(nestedHiddenFile); - // create file in non-hidden folder - Path nestedFile = new Path(new Path(basePath, "parent"), "nested-file.txt"); - fs.createNewFile(nestedFile); - // create file in base path - Path baseFile = new Path(basePath, "base-path-file.txt"); - fs.createNewFile(baseFile); - // create hidden file in base path - Path hiddenBase = new Path(basePath, ".hidden-in-base.txt"); - fs.createNewFile(hiddenBase); - // create empty subdirectory - Path emptyDirectory = new Path(basePath, "empty-directory"); - fs.mkdirs(emptyDirectory); - - // List recursively through hive file iterator - HiveFileIterator recursiveIterator = new HiveFileIterator( - fakeTable, - Location.of(basePath.toString()), - trinoFileSystem, - new FileSystemDirectoryLister(), - HiveFileIterator.NestedDirectoryPolicy.RECURSE); - - List recursiveListing = Streams.stream(recursiveIterator) - .map(TrinoFileStatus::getPath) - .map(Path::new) - .toList(); - // Should not include directories, or files underneath hidden directories - assertEqualsIgnoreOrder(recursiveListing, ImmutableList.of(nestedFile, baseFile)); - - HiveFileIterator shallowIterator = new HiveFileIterator( - fakeTable, - Location.of(basePath.toString()), - trinoFileSystem, - new FileSystemDirectoryLister(), - HiveFileIterator.NestedDirectoryPolicy.IGNORED); - List shallowListing = Streams.stream(shallowIterator) - .map(TrinoFileStatus::getPath) - .map(Path::new) - .toList(); - // Should not include any hidden files, folders, or nested files - assertEqualsIgnoreOrder(shallowListing, ImmutableList.of(baseFile)); - } - - @Test - public void testFileIteratorPartitionedListing() - throws Exception - { - Table.Builder tableBuilder = Table.builder() - .setDatabaseName(table.getSchemaName()) - .setTableName(table.getTableName()) - .setDataColumns(ImmutableList.of(new Column("data", HIVE_LONG, Optional.empty(), Map.of()))) - .setPartitionColumns(ImmutableList.of(new Column("part", HIVE_STRING, Optional.empty(), Map.of()))) - .setOwner(Optional.empty()) - .setTableType("fake"); - tableBuilder.getStorageBuilder() - .setStorageFormat(StorageFormat.fromHiveStorageFormat(HiveStorageFormat.CSV)); - Table fakeTable = tableBuilder.build(); - - // Expected file system tree: - // test-file-iterator-partitioned-listing/ - // .hidden/ - // nested-file-in-hidden.txt - // part=simple/ - // _hidden-file.txt - // plain-file.txt - // part=nested/ - // parent/ - // _nested-hidden-file.txt - // nested-file.txt - // part=plus+sign/ - // plus-file.txt - // part=percent%sign/ - // percent-file.txt - // part=url%20encoded/ - // url-encoded-file.txt - // part=level1|level2/ - // pipe-file.txt - // parent1/ - // parent2/ - // deeply-nested-file.txt - // part=level1 | level2/ - // pipe-blanks-file.txt - // empty-directory/ - // .hidden-in-base.txt - Path basePath = new Path(getBasePath(), "test-file-iterator-partitioned-listing"); - FileSystem fs = hdfsEnvironment.getFileSystem(TESTING_CONTEXT, basePath); - TrinoFileSystem trinoFileSystem = new HdfsFileSystemFactory(hdfsEnvironment, new TrinoHdfsFileSystemStats()).create(SESSION); - fs.mkdirs(basePath); - - // create file in hidden folder - Path fileInHiddenParent = new Path(new Path(basePath, ".hidden"), "nested-file-in-hidden.txt"); - fs.createNewFile(fileInHiddenParent); - // create hidden file in non-hidden folder - Path hiddenFileUnderPartitionSimple = new Path(new Path(basePath, "part=simple"), "_hidden-file.txt"); - fs.createNewFile(hiddenFileUnderPartitionSimple); - // create file in `part=simple` non-hidden folder - Path plainFilePartitionSimple = new Path(new Path(basePath, "part=simple"), "plain-file.txt"); - fs.createNewFile(plainFilePartitionSimple); - Path nestedFilePartitionNested = new Path(new Path(new Path(basePath, "part=nested"), "parent"), "nested-file.txt"); - fs.createNewFile(nestedFilePartitionNested); - // create hidden file in non-hidden folder - Path nestedHiddenFilePartitionNested = new Path(new Path(new Path(basePath, "part=nested"), "parent"), "_nested-hidden-file.txt"); - fs.createNewFile(nestedHiddenFilePartitionNested); - // create file in `part=plus+sign` non-hidden folder (which contains `+` special character) - Path plainFilePartitionPlusSign = new Path(new Path(basePath, "part=plus+sign"), "plus-file.txt"); - fs.createNewFile(plainFilePartitionPlusSign); - // create file in `part=percent%sign` non-hidden folder (which contains `%` special character) - Path plainFilePartitionPercentSign = new Path(new Path(basePath, "part=percent%sign"), "percent-file.txt"); - fs.createNewFile(plainFilePartitionPercentSign); - // create file in `part=url%20encoded` non-hidden folder (which contains `%` special character) - Path plainFilePartitionUrlEncoded = new Path(new Path(basePath, "part=url%20encoded"), "url-encoded-file.txt"); - fs.createNewFile(plainFilePartitionUrlEncoded); - // create file in `part=level1|level2` non-hidden folder (which contains `|` special character) - Path plainFilePartitionPipeSign = new Path(new Path(basePath, "part=level1|level2"), "pipe-file.txt"); - fs.createNewFile(plainFilePartitionPipeSign); - Path deeplyNestedFilePartitionPipeSign = new Path(new Path(new Path(new Path(basePath, "part=level1|level2"), "parent1"), "parent2"), "deeply-nested-file.txt"); - fs.createNewFile(deeplyNestedFilePartitionPipeSign); - // create file in `part=level1 | level2` non-hidden folder (which contains `|` and blank space special characters) - Path plainFilePartitionPipeSignBlanks = new Path(new Path(basePath, "part=level1 | level2"), "pipe-blanks-file.txt"); - fs.createNewFile(plainFilePartitionPipeSignBlanks); - - // create empty subdirectory - Path emptyDirectory = new Path(basePath, "empty-directory"); - fs.mkdirs(emptyDirectory); - // create hidden file in base path - Path hiddenBase = new Path(basePath, ".hidden-in-base.txt"); - fs.createNewFile(hiddenBase); - - // List recursively through hive file iterator - HiveFileIterator recursiveIterator = new HiveFileIterator( - fakeTable, - Location.of(basePath.toString()), - trinoFileSystem, - new FileSystemDirectoryLister(), - HiveFileIterator.NestedDirectoryPolicy.RECURSE); - - List recursiveListing = Streams.stream(recursiveIterator) - .map(TrinoFileStatus::getPath) - .map(Path::new) - .toList(); - // Should not include directories, or files underneath hidden directories - assertThat(recursiveListing).containsExactlyInAnyOrder( - plainFilePartitionSimple, - nestedFilePartitionNested, - plainFilePartitionPlusSign, - plainFilePartitionPercentSign, - plainFilePartitionUrlEncoded, - plainFilePartitionPipeSign, - deeplyNestedFilePartitionPipeSign, - plainFilePartitionPipeSignBlanks); - - HiveFileIterator shallowIterator = new HiveFileIterator( - fakeTable, - Location.of(basePath.toString()), - trinoFileSystem, - new FileSystemDirectoryLister(), - HiveFileIterator.NestedDirectoryPolicy.IGNORED); - List shallowListing = Streams.stream(shallowIterator) - .map(TrinoFileStatus::getPath) - .map(Path::new) - .toList(); - // Should not include any hidden files, folders, or nested files - assertThat(shallowListing).isEmpty(); - } - - @Test - public void testDirectoryWithTrailingSpace() - throws Exception - { - Path basePath = new Path(getBasePath(), randomUUID().toString()); - FileSystem fs = hdfsEnvironment.getFileSystem(TESTING_CONTEXT, basePath); - assertThat(fs.exists(basePath)).isFalse(); - - Path path = new Path(new Path(basePath, "dir_with_space "), "foo.txt"); - try (OutputStream outputStream = fs.create(path)) { - outputStream.write("test".getBytes(UTF_8)); - } - assertThat(fs.exists(path)).isTrue(); - - try (InputStream inputStream = fs.open(path)) { - String content = new BufferedReader(new InputStreamReader(inputStream, UTF_8)).readLine(); - assertThat(content).isEqualTo("test"); - } - - fs.delete(basePath, true); - } - - @Test - public void testTableCreation() - throws Exception - { - for (HiveStorageFormat storageFormat : HiveStorageFormat.values()) { - if (storageFormat == HiveStorageFormat.CSV) { - // CSV supports only unbounded VARCHAR type - continue; - } - if (storageFormat == HiveStorageFormat.REGEX) { - // REGEX format is read-only - continue; - } - createTable(temporaryCreateTable, storageFormat); - dropTable(temporaryCreateTable); - } - } - - @Test - public void testTableCreationExternalLocation() - throws Exception - { - for (HiveStorageFormat storageFormat : HiveStorageFormat.values()) { - if (storageFormat == HiveStorageFormat.CSV) { - // CSV supports only unbounded VARCHAR type - continue; - } - if (storageFormat == HiveStorageFormat.REGEX) { - // REGEX format is read-only - continue; - } - createExternalTableOnNonExistingPath(temporaryCreateTableWithExternalLocation, storageFormat); - dropTable(temporaryCreateTableWithExternalLocation); - } - } - - private void createTable(SchemaTableName tableName, HiveStorageFormat storageFormat) - throws Exception - { - List columns = ImmutableList.of(new ColumnMetadata("id", BIGINT)); - - MaterializedResult data = MaterializedResult.resultBuilder(newSession(), BIGINT) - .row(1L) - .row(3L) - .row(2L) - .build(); - - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorSession session = newSession(); - - // begin creating the table - ConnectorTableMetadata tableMetadata = new ConnectorTableMetadata(tableName, columns, createTableProperties(storageFormat)); - ConnectorOutputTableHandle outputHandle = metadata.beginCreateTable(session, tableMetadata, Optional.empty(), NO_RETRIES); - - // write the records - ConnectorPageSink sink = pageSinkProvider.createPageSink(transaction.getTransactionHandle(), session, outputHandle, TESTING_PAGE_SINK_ID); - sink.appendPage(data.toPage()); - Collection fragments = getFutureValue(sink.finish()); - - // commit the table - metadata.finishCreateTable(session, outputHandle, fragments, ImmutableList.of()); - - transaction.commit(); - - // Hack to work around the metastore not being configured for S3 or other FS. - // The metastore tries to validate the location when creating the - // table, which fails without explicit configuration for file system. - // We work around that by using a dummy location when creating the - // table and update it here to the correct location. - Location location = locationService.getTableWriteInfo(((HiveOutputTableHandle) outputHandle).getLocationHandle(), false).targetPath(); - metastoreClient.updateTableLocation(database, tableName.getTableName(), location.toString()); - } - - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorSession session = newSession(); - - // load the new table - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); - List columnHandles = filterNonHiddenColumnHandles(metadata.getColumnHandles(session, tableHandle).values()); - - // verify the metadata - ConnectorTableMetadata tableMetadata = metadata.getTableMetadata(session, getTableHandle(metadata, tableName)); - assertThat(filterNonHiddenColumnMetadata(tableMetadata.getColumns())).isEqualTo(columns); - - // verify the data - metadata.beginQuery(session); - ConnectorSplitSource splitSource = getSplits(splitManager, transaction, session, tableHandle); - ConnectorSplit split = getOnlyElement(getAllSplits(splitSource)); - - try (ConnectorPageSource pageSource = pageSourceProvider.createPageSource(transaction.getTransactionHandle(), session, split, tableHandle, columnHandles, DynamicFilter.EMPTY)) { - MaterializedResult result = materializeSourceDataStream(session, pageSource, getTypes(columnHandles)); - assertEqualsIgnoreOrder(result.getMaterializedRows(), data.getMaterializedRows()); - } - - metadata.cleanupQuery(session); - } - } - - private void createExternalTableOnNonExistingPath(SchemaTableName tableName, HiveStorageFormat storageFormat) - throws Exception - { - List columns = ImmutableList.of(new ColumnMetadata("id", BIGINT)); - String externalLocation = getBasePath() + "/external_" + randomNameSuffix(); - - MaterializedResult data = MaterializedResult.resultBuilder(newSession(), BIGINT) - .row(1L) - .row(3L) - .row(2L) - .build(); - - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorSession session = newSession(); - - Map tableProperties = ImmutableMap.builder() - .putAll(createTableProperties(storageFormat)) - .put(EXTERNAL_LOCATION_PROPERTY, externalLocation) - .buildOrThrow(); - - // begin creating the table - ConnectorTableMetadata tableMetadata = new ConnectorTableMetadata(tableName, columns, tableProperties); - metadata.createTable(session, tableMetadata, true); - - transaction.commit(); - - // Hack to work around the metastore not being configured for S3 or other FS. - // The metastore tries to validate the location when creating the - // table, which fails without explicit configuration for file system. - // We work around that by using a dummy location when creating the - // table and update it here to the correct location. - Location location = locationService.getTableWriteInfo(new LocationHandle(externalLocation, externalLocation, LocationHandle.WriteMode.DIRECT_TO_TARGET_NEW_DIRECTORY), false).targetPath(); - metastoreClient.updateTableLocation(database, tableName.getTableName(), location.toString()); - } - - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorSession session = newSession(); - - ConnectorTableHandle connectorTableHandle = getTableHandle(metadata, tableName); - ConnectorInsertTableHandle outputHandle = metadata.beginInsert(session, connectorTableHandle, ImmutableList.of(), NO_RETRIES); - - ConnectorPageSink sink = pageSinkProvider.createPageSink(transaction.getTransactionHandle(), session, outputHandle, TESTING_PAGE_SINK_ID); - sink.appendPage(data.toPage()); - Collection fragments = getFutureValue(sink.finish()); - - metadata.finishInsert(session, outputHandle, fragments, ImmutableList.of()); - transaction.commit(); - } - - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorSession session = newSession(); - - // load the new table - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); - List columnHandles = filterNonHiddenColumnHandles(metadata.getColumnHandles(session, tableHandle).values()); - - // verify the metadata - ConnectorTableMetadata tableMetadata = metadata.getTableMetadata(session, getTableHandle(metadata, tableName)); - assertThat(filterNonHiddenColumnMetadata(tableMetadata.getColumns())).isEqualTo(columns); - assertThat(tableMetadata.getProperties()).containsEntry("external_location", externalLocation); - - // verify the data - metadata.beginQuery(session); - ConnectorSplitSource splitSource = getSplits(splitManager, transaction, session, tableHandle); - ConnectorSplit split = getOnlyElement(getAllSplits(splitSource)); - - try (ConnectorPageSource pageSource = pageSourceProvider.createPageSource(transaction.getTransactionHandle(), session, split, tableHandle, columnHandles, DynamicFilter.EMPTY)) { - MaterializedResult result = materializeSourceDataStream(session, pageSource, getTypes(columnHandles)); - assertEqualsIgnoreOrder(result.getMaterializedRows(), data.getMaterializedRows()); - } - - metadata.cleanupQuery(session); - } - } - - private void dropTable(SchemaTableName table) - { - try (Transaction transaction = newTransaction()) { - transaction.getMetastore().dropTable(newSession(), table.getSchemaName(), table.getTableName()); - transaction.commit(); - } - } - - private ConnectorTableHandle getTableHandle(ConnectorMetadata metadata, SchemaTableName tableName) - { - return HiveFileSystemTestUtils.getTableHandle(metadata, tableName, newSession()); - } - - public static class TestingHiveMetastore - extends ForwardingHiveMetastore - { - private final Path basePath; - private final HdfsEnvironment hdfsEnvironment; - - public TestingHiveMetastore(HiveMetastore delegate, Path basePath, HdfsEnvironment hdfsEnvironment) - { - super(delegate); - this.basePath = basePath; - this.hdfsEnvironment = hdfsEnvironment; - } - - @Override - public Optional getDatabase(String databaseName) - { - return super.getDatabase(databaseName) - .map(database -> Database.builder(database) - .setLocation(Optional.of(basePath.toString())) - .build()); - } - - @Override - public void createTable(Table table, PrincipalPrivileges privileges) - { - // hack to work around the metastore not being configured for S3 or other FS - Table.Builder tableBuilder = Table.builder(table); - tableBuilder.getStorageBuilder().setLocation("/"); - super.createTable(tableBuilder.build(), privileges); - } - - @Override - public void dropTable(String databaseName, String tableName, boolean deleteData) - { - try { - Table table = getTable(databaseName, tableName) - .orElseThrow(() -> new TableNotFoundException(new SchemaTableName(databaseName, tableName))); - - // hack to work around the metastore not being configured for S3 or other FS - List locations = listAllDataPaths(databaseName, tableName); - - Table.Builder tableBuilder = Table.builder(table); - tableBuilder.getStorageBuilder().setLocation("/"); - - // drop table - replaceTable(databaseName, tableName, tableBuilder.build(), NO_PRIVILEGES); - super.dropTable(databaseName, tableName, false); - - // drop data - if (deleteData) { - for (String location : locations) { - Path path = new Path(location); - hdfsEnvironment.getFileSystem(TESTING_CONTEXT, path).delete(path, true); - } - } - } - catch (IOException e) { - throw new UncheckedIOException(e); - } - } - - public void updateTableLocation(String databaseName, String tableName, String location) - { - Table table = getTable(databaseName, tableName) - .orElseThrow(() -> new TableNotFoundException(new SchemaTableName(databaseName, tableName))); - Table.Builder tableBuilder = Table.builder(table); - tableBuilder.getStorageBuilder().setLocation(location); - - // NOTE: this clears the permissions - replaceTable(databaseName, tableName, tableBuilder.build(), NO_PRIVILEGES); - } - - private List listAllDataPaths(String schemaName, String tableName) - { - ImmutableList.Builder locations = ImmutableList.builder(); - Table table = getTable(schemaName, tableName).get(); - List partitionColumnNames = table.getPartitionColumns().stream().map(Column::getName).collect(toImmutableList()); - if (table.getStorage().getLocation() != null) { - // For partitioned table, there should be nothing directly under this directory. - // But including this location in the set makes the directory content assert more - // extensive, which is desirable. - locations.add(table.getStorage().getLocation()); - } - - Optional> partitionNames = getPartitionNamesByFilter(schemaName, tableName, partitionColumnNames, TupleDomain.all()); - if (partitionNames.isPresent()) { - getPartitionsByNames(table, partitionNames.get()).values().stream() - .map(Optional::get) - .map(partition -> partition.getStorage().getLocation()) - .filter(location -> !location.startsWith(table.getStorage().getLocation())) - .forEach(locations::add); - } - - return locations.build(); - } - } -} diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveFileSystemTestUtils.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveFileSystemTestUtils.java deleted file mode 100644 index a43d782d67f97..0000000000000 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveFileSystemTestUtils.java +++ /dev/null @@ -1,137 +0,0 @@ -/* - * 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.hive; - -import com.google.common.collect.ImmutableList; -import io.trino.plugin.hive.AbstractTestHive.HiveTransaction; -import io.trino.plugin.hive.AbstractTestHive.Transaction; -import io.trino.spi.connector.ColumnHandle; -import io.trino.spi.connector.ConnectorMetadata; -import io.trino.spi.connector.ConnectorPageSource; -import io.trino.spi.connector.ConnectorPageSourceProvider; -import io.trino.spi.connector.ConnectorSession; -import io.trino.spi.connector.ConnectorSplit; -import io.trino.spi.connector.ConnectorSplitManager; -import io.trino.spi.connector.ConnectorSplitSource; -import io.trino.spi.connector.ConnectorTableHandle; -import io.trino.spi.connector.DynamicFilter; -import io.trino.spi.connector.SchemaTableName; -import io.trino.spi.type.Type; -import io.trino.testing.MaterializedResult; -import io.trino.testing.MaterializedRow; - -import java.io.Closeable; -import java.io.IOException; -import java.util.List; -import java.util.stream.IntStream; - -import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.collect.ImmutableList.toImmutableList; -import static io.trino.plugin.hive.AbstractTestHive.getAllSplits; -import static io.trino.plugin.hive.AbstractTestHive.getSplits; -import static io.trino.plugin.hive.HiveTestUtils.getHiveSession; -import static io.trino.plugin.hive.HiveTestUtils.getTypes; -import static io.trino.testing.MaterializedResult.materializeSourceDataStream; - -public class HiveFileSystemTestUtils -{ - private HiveFileSystemTestUtils() {} - - public static MaterializedResult readTable(SchemaTableName tableName, HiveTransactionManager transactionManager, - HiveConfig config, ConnectorPageSourceProvider pageSourceProvider, - ConnectorSplitManager splitManager) - throws IOException - { - ConnectorMetadata metadata = null; - ConnectorSession session = null; - ConnectorSplitSource splitSource = null; - - try (Transaction transaction = newTransaction(transactionManager)) { - metadata = transaction.getMetadata(); - session = newSession(config); - - ConnectorTableHandle table = getTableHandle(metadata, tableName, session); - List columnHandles = ImmutableList.copyOf(metadata.getColumnHandles(session, table).values()); - - metadata.beginQuery(session); - splitSource = getSplits(splitManager, transaction, session, table); - - List allTypes = getTypes(columnHandles); - List dataTypes = getTypes(columnHandles.stream() - .filter(columnHandle -> !((HiveColumnHandle) columnHandle).isHidden()) - .collect(toImmutableList())); - MaterializedResult.Builder result = MaterializedResult.resultBuilder(session, dataTypes); - - List splits = getAllSplits(splitSource); - for (ConnectorSplit split : splits) { - try (ConnectorPageSource pageSource = pageSourceProvider.createPageSource( - transaction.getTransactionHandle(), - session, - split, - table, - columnHandles, - DynamicFilter.EMPTY)) { - MaterializedResult pageSourceResult = materializeSourceDataStream(session, pageSource, allTypes); - for (MaterializedRow row : pageSourceResult.getMaterializedRows()) { - Object[] dataValues = IntStream.range(0, row.getFieldCount()) - .filter(channel -> !((HiveColumnHandle) columnHandles.get(channel)).isHidden()) - .mapToObj(row::getField) - .toArray(); - result.row(dataValues); - } - } - } - return result.build(); - } - finally { - cleanUpQuery(metadata, session); - closeQuietly(splitSource); - } - } - - public static ConnectorTableHandle getTableHandle(ConnectorMetadata metadata, SchemaTableName tableName, ConnectorSession session) - { - ConnectorTableHandle handle = metadata.getTableHandle(session, tableName); - checkArgument(handle != null, "table not found: %s", tableName); - return handle; - } - - public static ConnectorSession newSession(HiveConfig config) - { - return getHiveSession(config); - } - - public static Transaction newTransaction(HiveTransactionManager transactionManager) - { - return new HiveTransaction(transactionManager); - } - - private static void closeQuietly(Closeable closeable) - { - try { - if (closeable != null) { - closeable.close(); - } - } - catch (IOException ignored) { - } - } - - private static void cleanUpQuery(ConnectorMetadata metadata, ConnectorSession session) - { - if (metadata != null && session != null) { - metadata.cleanupQuery(session); - } - } -} From 6e7a84706e68ad47d713824fec7564f85e57f25f Mon Sep 17 00:00:00 2001 From: David Phillips Date: Wed, 6 Dec 2023 14:05:03 +0800 Subject: [PATCH 086/350] Remove legacy Hive S3 tests --- .github/workflows/ci.yml | 16 -- .../bin/run_hive_s3_tests.sh | 75 ------- .../conf/files/hadoop-put.sh | 8 - plugin/trino-hive-hadoop2/pom.xml | 17 -- .../hive/TestHiveThriftMetastoreWithS3.java | 202 ------------------ .../resources/s3/hive-core-site.template.xml | 43 ---- 6 files changed, 361 deletions(-) delete mode 100755 plugin/trino-hive-hadoop2/bin/run_hive_s3_tests.sh delete mode 100755 plugin/trino-hive-hadoop2/conf/files/hadoop-put.sh delete mode 100644 plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/TestHiveThriftMetastoreWithS3.java delete mode 100644 plugin/trino-hive-hadoop2/src/test/resources/s3/hive-core-site.template.xml diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index e3d7b3dbfa5c9..8fd233feea2bb 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -306,22 +306,6 @@ jobs: run: | source plugin/trino-hive-hadoop2/conf/hive-tests-${{ matrix.config }}.sh && plugin/trino-hive-hadoop2/bin/run_hive_tests.sh - - name: Run Hive S3 Tests - env: - AWS_ACCESS_KEY_ID: ${{ secrets.TRINO_AWS_ACCESS_KEY_ID }} - AWS_SECRET_ACCESS_KEY: ${{ secrets.TRINO_AWS_SECRET_ACCESS_KEY }} - AWS_REGION: ${{ vars.TRINO_AWS_REGION }} - S3_BUCKET: ${{ vars.TRINO_S3_BUCKET }} - S3_BUCKET_ENDPOINT: "https://s3.${{ vars.TRINO_AWS_REGION }}.amazonaws.com" - run: | - if [ "${AWS_ACCESS_KEY_ID}" != "" ]; then - source plugin/trino-hive-hadoop2/conf/hive-tests-${{ matrix.config }}.sh && - plugin/trino-hive-hadoop2/bin/run_hive_s3_tests.sh - if [ matrix.config == 'config-hdp3' ]; then - # JsonSerde class needed for the S3 Select JSON tests is only available on hdp3. - plugin/trino-hive-hadoop2/bin/run_hive_s3_select_json_tests.sh - fi - fi - name: Run Hive AWS Tests env: AWS_ACCESS_KEY_ID: ${{ secrets.TRINO_AWS_ACCESS_KEY_ID }} diff --git a/plugin/trino-hive-hadoop2/bin/run_hive_s3_tests.sh b/plugin/trino-hive-hadoop2/bin/run_hive_s3_tests.sh deleted file mode 100755 index 82fa2b7fd6e6c..0000000000000 --- a/plugin/trino-hive-hadoop2/bin/run_hive_s3_tests.sh +++ /dev/null @@ -1,75 +0,0 @@ -#!/usr/bin/env bash - -set -euo pipefail -x - -. "${BASH_SOURCE%/*}/common.sh" - -abort_if_not_gib_impacted - -check_vars S3_BUCKET S3_BUCKET_ENDPOINT AWS_REGION \ - AWS_ACCESS_KEY_ID AWS_SECRET_ACCESS_KEY - -cleanup_hadoop_docker_containers -start_hadoop_docker_containers - -test_directory="$(date '+%Y%m%d-%H%M%S')-$(uuidgen | sha1sum | cut -b 1-6)" - -# insert AWS credentials -deploy_core_site_xml core-site.xml.s3-template \ - AWS_ACCESS_KEY_ID AWS_SECRET_ACCESS_KEY S3_BUCKET_ENDPOINT - -# create test tables -# can't use create_test_tables because the first table is created with different commands -table_path="s3a://${S3_BUCKET}/${test_directory}/trino_test_external_fs/" -exec_in_hadoop_master_container hadoop fs -mkdir -p "${table_path}" -exec_in_hadoop_master_container /docker/files/hadoop-put.sh /docker/files/test_table.csv{,.gz,.bz2,.lz4} "${table_path}" -exec_in_hadoop_master_container sudo -Eu hive beeline -u jdbc:hive2://localhost:10000/default -n hive -e " - CREATE EXTERNAL TABLE trino_test_external_fs(t_bigint bigint) - STORED AS TEXTFILE - LOCATION '${table_path}'" - -table_path="s3a://${S3_BUCKET}/${test_directory}/trino_test_external_fs_with_header/" -exec_in_hadoop_master_container hadoop fs -mkdir -p "${table_path}" -exec_in_hadoop_master_container hadoop fs -put -f /docker/files/test_table_with_header.csv{,.gz,.bz2,.lz4} "${table_path}" -exec_in_hadoop_master_container /usr/bin/hive -e " - CREATE EXTERNAL TABLE trino_test_external_fs_with_header(t_bigint bigint) - STORED AS TEXTFILE - LOCATION '${table_path}' - TBLPROPERTIES ('skip.header.line.count'='1')" - -table_path="s3a://${S3_BUCKET}/${test_directory}/trino_test_external_fs_with_header_and_footer/" -exec_in_hadoop_master_container hadoop fs -mkdir -p "${table_path}" -exec_in_hadoop_master_container hadoop fs -put -f /docker/files/test_table_with_header_and_footer.csv{,.gz,.bz2,.lz4} "${table_path}" -exec_in_hadoop_master_container /usr/bin/hive -e " - CREATE EXTERNAL TABLE trino_test_external_fs_with_header_and_footer(t_bigint bigint) - STORED AS TEXTFILE - LOCATION '${table_path}' - TBLPROPERTIES ('skip.header.line.count'='2', 'skip.footer.line.count'='2')" - -stop_unnecessary_hadoop_services - -# restart hive-metastore to apply S3 changes in core-site.xml -docker exec "$(hadoop_master_container)" supervisorctl restart hive-metastore -retry check_hadoop - -# run product tests -pushd "${PROJECT_ROOT}" -set +e -./mvnw ${MAVEN_TEST:--B} -pl :trino-hive-hadoop2 test -P test-hive-hadoop2-s3 \ - -DHADOOP_USER_NAME=hive \ - -Dhive.hadoop2.metastoreHost=localhost \ - -Dhive.hadoop2.metastorePort=9083 \ - -Dhive.hadoop2.databaseName=default \ - -Dhive.hadoop2.s3.endpoint="${S3_BUCKET_ENDPOINT}" \ - -Dhive.hadoop2.s3.region="${AWS_REGION}" \ - -Dhive.hadoop2.s3.awsAccessKey="${AWS_ACCESS_KEY_ID}" \ - -Dhive.hadoop2.s3.awsSecretKey="${AWS_SECRET_ACCESS_KEY}" \ - -Dhive.hadoop2.s3.writableBucket="${S3_BUCKET}" \ - -Dhive.hadoop2.s3.testDirectory="${test_directory}" -EXIT_CODE=$? -set -e -popd - -cleanup_hadoop_docker_containers - -exit "${EXIT_CODE}" diff --git a/plugin/trino-hive-hadoop2/conf/files/hadoop-put.sh b/plugin/trino-hive-hadoop2/conf/files/hadoop-put.sh deleted file mode 100755 index 33a7431af06e4..0000000000000 --- a/plugin/trino-hive-hadoop2/conf/files/hadoop-put.sh +++ /dev/null @@ -1,8 +0,0 @@ -#!/bin/bash - -set -xeuo pipefail - -# Hadoop 3 without -d (don't create _COPYING_ temporary file) requires additional S3 permissions -# Hadoop 2 doesn't have '-d' switch -hadoop fs -put -f -d "$@" || -hadoop fs -put -f "$@" diff --git a/plugin/trino-hive-hadoop2/pom.xml b/plugin/trino-hive-hadoop2/pom.xml index 535dca91b021c..eea081792b71a 100644 --- a/plugin/trino-hive-hadoop2/pom.xml +++ b/plugin/trino-hive-hadoop2/pom.xml @@ -231,7 +231,6 @@ **/TestHive.java - **/TestHiveThriftMetastoreWithS3.java @@ -254,21 +253,5 @@ - - test-hive-hadoop2-s3 - - - - org.apache.maven.plugins - maven-surefire-plugin - - - **/TestHiveThriftMetastoreWithS3.java - - - - - - diff --git a/plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/TestHiveThriftMetastoreWithS3.java b/plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/TestHiveThriftMetastoreWithS3.java deleted file mode 100644 index d46a24a7e5157..0000000000000 --- a/plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/TestHiveThriftMetastoreWithS3.java +++ /dev/null @@ -1,202 +0,0 @@ -/* - * 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.hive; - -import com.amazonaws.auth.AWSStaticCredentialsProvider; -import com.amazonaws.auth.BasicAWSCredentials; -import com.amazonaws.client.builder.AwsClientBuilder; -import com.amazonaws.services.s3.AmazonS3; -import com.amazonaws.services.s3.AmazonS3Client; -import com.amazonaws.services.s3.model.S3ObjectSummary; -import com.google.common.collect.ImmutableMap; -import com.google.common.io.Resources; -import io.trino.plugin.hive.containers.HiveHadoop; -import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreConfig; -import io.trino.plugin.hive.s3.S3HiveQueryRunner; -import io.trino.testing.AbstractTestQueryFramework; -import io.trino.testing.QueryRunner; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; - -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.attribute.PosixFilePermissions; -import java.util.List; - -import static io.trino.testing.TestingNames.randomNameSuffix; -import static java.nio.charset.StandardCharsets.UTF_8; -import static java.util.Objects.requireNonNull; -import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; - -@TestInstance(PER_CLASS) -public class TestHiveThriftMetastoreWithS3 - extends AbstractTestQueryFramework -{ - private final String s3endpoint; - private final String s3Region; - private final String awsAccessKey; - private final String awsSecretKey; - private final String writableBucket; - private final String schemaName; - private final Path hadoopCoreSiteXmlTempFile; - private final AmazonS3 s3Client; - - public TestHiveThriftMetastoreWithS3() - throws IOException - { - this.s3endpoint = requireNonNull(System.getProperty("hive.hadoop2.s3.endpoint"), "hive.hadoop2.s3.endpoint is null"); - this.s3Region = requireNonNull(System.getProperty("hive.hadoop2.s3.region"), "hive.hadoop2.s3.region is null"); - this.awsAccessKey = requireNonNull(System.getProperty("hive.hadoop2.s3.awsAccessKey"), "hive.hadoop2.s3.awsAccessKey is null"); - this.awsSecretKey = requireNonNull(System.getProperty("hive.hadoop2.s3.awsSecretKey"), "hive.hadoop2.s3.awsSecretKey is null"); - this.writableBucket = requireNonNull(System.getProperty("hive.hadoop2.s3.writableBucket"), "hive.hadoop2.s3.writableBucket is null"); - this.schemaName = "test_thrift_s3_" + randomNameSuffix(); - - String coreSiteXmlContent = Resources.toString(Resources.getResource("s3/hive-core-site.template.xml"), UTF_8) - .replace("%S3_BUCKET_ENDPOINT%", s3endpoint) - .replace("%AWS_ACCESS_KEY_ID%", awsAccessKey) - .replace("%AWS_SECRET_ACCESS_KEY%", awsSecretKey); - - hadoopCoreSiteXmlTempFile = Files.createTempFile("core-site", ".xml", PosixFilePermissions.asFileAttribute(PosixFilePermissions.fromString("rw-r--r--"))); - hadoopCoreSiteXmlTempFile.toFile().deleteOnExit(); - Files.writeString(hadoopCoreSiteXmlTempFile, coreSiteXmlContent); - - s3Client = AmazonS3Client.builder() - .withEndpointConfiguration(new AwsClientBuilder.EndpointConfiguration(s3endpoint, null)) - .withCredentials(new AWSStaticCredentialsProvider(new BasicAWSCredentials(awsAccessKey, awsSecretKey))) - .build(); - } - - @Override - protected QueryRunner createQueryRunner() - throws Exception - { - HiveHadoop hiveHadoop = HiveHadoop.builder() - .withFilesToMount(ImmutableMap.of("/etc/hadoop/conf/core-site.xml", hadoopCoreSiteXmlTempFile.normalize().toAbsolutePath().toString())) - .build(); - hiveHadoop.start(); - - return S3HiveQueryRunner.builder() - .setHiveMetastoreEndpoint(hiveHadoop.getHiveMetastoreEndpoint()) - .setS3Endpoint(s3endpoint) - .setS3Region(s3Region) - .setS3AccessKey(awsAccessKey) - .setS3SecretKey(awsSecretKey) - .setBucketName(writableBucket) - .setCreateTpchSchemas(false) - .setThriftMetastoreConfig(new ThriftMetastoreConfig().setDeleteFilesOnDrop(true)) - .setHiveProperties(ImmutableMap.of("hive.allow-register-partition-procedure", "true")) - .build(); - } - - @BeforeAll - public void setUp() - { - String schemaLocation = "s3a://%s/%s".formatted(writableBucket, schemaName); - assertUpdate("CREATE SCHEMA " + schemaName + " WITH (location = '" + schemaLocation + "')"); - } - - @AfterAll - public void tearDown() - { - assertUpdate("DROP SCHEMA IF EXISTS " + schemaName); - } - - @Test - public void testRecreateTable() - { - String tableName = "test_recreate_table_" + randomNameSuffix(); - String schemaTableName = "%s.%s".formatted(schemaName, tableName); - String tableLocation = "%s/%s".formatted(schemaName, tableName); - - // Creating a new table generates special empty file on S3 (not MinIO) - assertUpdate("CREATE TABLE " + schemaTableName + "(col int)"); - try { - assertUpdate("INSERT INTO " + schemaTableName + " VALUES (1)", 1); - assertThat(getS3ObjectSummaries(tableLocation)).hasSize(2); // directory + file - - // DROP TABLE with Thrift metastore on S3 (not MinIO) leaves some files - // when 'hive.metastore.thrift.delete-files-on-drop' config property is false. - // Then, the subsequent CREATE TABLE throws "Target directory for table 'xxx' already exists" - assertUpdate("DROP TABLE " + schemaTableName); - assertThat(getS3ObjectSummaries(tableLocation)).hasSize(0); - - assertUpdate("CREATE TABLE " + schemaTableName + "(col int)"); - } - finally { - assertUpdate("DROP TABLE IF EXISTS " + schemaTableName); - } - } - - @Test - public void testRecreatePartition() - { - String tableName = "test_recreate_partition_" + randomNameSuffix(); - String schemaTableName = "%s.%s".formatted(schemaName, tableName); - String partitionLocation = "%s/%s/part=1".formatted(schemaName, tableName); - - assertUpdate("CREATE TABLE " + schemaTableName + "(col int, part int) WITH (partitioned_by = ARRAY['part'])"); - try { - // Creating an empty partition generates special empty file on S3 (not MinIO) - assertUpdate("CALL system.create_empty_partition('%s', '%s', ARRAY['part'], ARRAY['1'])".formatted(schemaName, tableName)); - assertUpdate("INSERT INTO " + schemaTableName + " VALUES (1, 1)", 1); - assertQuery("SELECT * FROM " + schemaTableName, "VALUES (1, 1)"); - - assertThat(getS3ObjectSummaries(partitionLocation)).hasSize(2); // directory + file - - // DELETE with Thrift metastore on S3 (not MinIO) leaves some files - // when 'hive.metastore.thrift.delete-files-on-drop' config property is false. - // Then, the subsequent SELECT doesn't return an empty row - assertUpdate("DELETE FROM " + schemaTableName); - assertThat(getS3ObjectSummaries(partitionLocation)).hasSize(0); - - assertUpdate("CALL system.create_empty_partition('%s', '%s', ARRAY['part'], ARRAY['1'])".formatted(schemaName, tableName)); - assertQueryReturnsEmptyResult("SELECT * FROM " + schemaTableName); - } - finally { - assertUpdate("DROP TABLE " + schemaTableName); - } - } - - @Test - public void testUnregisterPartitionNotRemoveData() - { - // Verify unregister_partition procedure doesn't remove physical data even when 'hive.metastore.thrift.delete-files-on-drop' config property is true - String tableName = "test_recreate_partition_" + randomNameSuffix(); - String schemaTableName = "%s.%s".formatted(schemaName, tableName); - - assertUpdate("CREATE TABLE " + schemaTableName + "(col int, part int) WITH (partitioned_by = ARRAY['part'])"); - try { - assertUpdate("INSERT INTO " + schemaTableName + " VALUES (1, 1)", 1); - assertQuery("SELECT * FROM " + schemaTableName, "VALUES (1, 1)"); - - assertUpdate("CALL system.unregister_partition('%s', '%s', ARRAY['part'], ARRAY['1'])".formatted(schemaName, tableName)); - assertQueryReturnsEmptyResult("SELECT * FROM " + schemaTableName); - - assertUpdate("CALL system.register_partition('%s', '%s', ARRAY['part'], ARRAY['1'])".formatted(schemaName, tableName)); - assertQuery("SELECT * FROM " + schemaTableName, "VALUES (1, 1)"); - } - finally { - assertUpdate("DROP TABLE " + schemaTableName); - } - } - - private List getS3ObjectSummaries(String prefix) - { - return s3Client.listObjectsV2(writableBucket, prefix).getObjectSummaries(); - } -} diff --git a/plugin/trino-hive-hadoop2/src/test/resources/s3/hive-core-site.template.xml b/plugin/trino-hive-hadoop2/src/test/resources/s3/hive-core-site.template.xml deleted file mode 100644 index a3dc6ad47d4b0..0000000000000 --- a/plugin/trino-hive-hadoop2/src/test/resources/s3/hive-core-site.template.xml +++ /dev/null @@ -1,43 +0,0 @@ - - - - fs.defaultFS - hdfs://hadoop-master:9000 - - - - fs.s3a.endpoint - %S3_BUCKET_ENDPOINT% - - - - fs.s3.awsAccessKeyId - %AWS_ACCESS_KEY_ID% - - - - fs.s3.awsSecretAccessKey - %AWS_SECRET_ACCESS_KEY% - - - - fs.s3a.access.key - %AWS_ACCESS_KEY_ID% - - - - fs.s3a.secret.key - %AWS_SECRET_ACCESS_KEY% - - - - - hadoop.proxyuser.hive.hosts - * - - - - hadoop.proxyuser.hive.groups - * - - From 36f0aa5e398ec532f657a24ffc570024d61b67e3 Mon Sep 17 00:00:00 2001 From: David Phillips Date: Wed, 6 Dec 2023 16:12:30 +0800 Subject: [PATCH 087/350] Remove legacy Hive tests --- .github/workflows/ci.yml | 4 - plugin/trino-hive-hadoop2/bin/common.sh | 214 - .../trino-hive-hadoop2/bin/run_hive_tests.sh | 35 - plugin/trino-hive-hadoop2/bin/start_hive.sh | 21 - .../conf/docker-compose.yml | 22 - .../conf/files/core-site.xml.s3-template | 61 - .../conf/files/test_table.csv | 3 - .../conf/files/test_table.csv.bz2 | Bin 43 -> 0 bytes .../conf/files/test_table.csv.gz | Bin 29 -> 0 bytes .../conf/files/test_table.csv.lz4 | Bin 18 -> 0 bytes .../conf/files/test_table.json | 2 - .../conf/files/test_table.json.bz2 | Bin 87 -> 0 bytes .../conf/files/test_table.json.gz | Bin 73 -> 0 bytes ...table_csv_scan_range_select_pushdown_1.csv | 100 - ...table_csv_scan_range_select_pushdown_2.csv | 100 - ...table_csv_scan_range_select_pushdown_3.csv | 100 - ...ble_json_scan_range_select_pushdown_1.json | 100 - ...ble_json_scan_range_select_pushdown_2.json | 100 - ...ble_json_scan_range_select_pushdown_3.json | 100 - .../files/test_table_with_comma_delimiter.csv | 3 - .../test_table_with_comma_delimiter.csv.bz2 | Bin 51 -> 0 bytes .../test_table_with_comma_delimiter.csv.gz | Bin 78 -> 0 bytes .../conf/files/test_table_with_header.csv | 4 - .../conf/files/test_table_with_header.csv.bz2 | Bin 44 -> 0 bytes .../conf/files/test_table_with_header.csv.gz | Bin 31 -> 0 bytes .../conf/files/test_table_with_header.csv.lz4 | Bin 19 -> 0 bytes .../test_table_with_header_and_footer.csv | 7 - .../test_table_with_header_and_footer.csv.bz2 | Bin 48 -> 0 bytes .../test_table_with_header_and_footer.csv.gz | Bin 37 -> 0 bytes .../test_table_with_header_and_footer.csv.lz4 | Bin 27 -> 0 bytes .../files/test_table_with_pipe_delimiter.csv | 3 - .../test_table_with_pipe_delimiter.csv.bz2 | Bin 54 -> 0 bytes .../test_table_with_pipe_delimiter.csv.gz | Bin 78 -> 0 bytes .../conf/files/tez-site.xml | 100 - plugin/trino-hive-hadoop2/conf/files/words | 100 - .../conf/hive-tests-config-apache-hive3.sh | 1 - .../conf/hive-tests-config-hdp3.sh | 1 - .../conf/hive-tests-defaults.sh | 11 - plugin/trino-hive-hadoop2/pom.xml | 38 - .../java/io/trino/plugin/hive/TestHive.java | 184 - .../trino/plugin/hive/AbstractTestHive.java | 6368 ----------------- .../plugin/hive/AbstractTestHiveLocal.java | 351 - .../plugin/hive/TestHiveFileMetastore.java | 84 - .../metastore/glue/TestGlueHiveMetastore.java | 1605 ----- .../spark_bucketed_nation/._SUCCESS.crc | Bin 8 -> 0 bytes ...513-2b2e82ec274f_00000.c000.snappy.orc.crc | Bin 24 -> 0 bytes ...513-2b2e82ec274f_00001.c000.snappy.orc.crc | Bin 20 -> 0 bytes ...513-2b2e82ec274f_00002.c000.snappy.orc.crc | Bin 20 -> 0 bytes .../resources/spark_bucketed_nation/_SUCCESS | 0 ...85-a513-2b2e82ec274f_00000.c000.snappy.orc | Bin 1865 -> 0 bytes ...85-a513-2b2e82ec274f_00001.c000.snappy.orc | Bin 1480 -> 0 bytes ...85-a513-2b2e82ec274f_00002.c000.snappy.orc | Bin 1200 -> 0 bytes 52 files changed, 9822 deletions(-) delete mode 100644 plugin/trino-hive-hadoop2/bin/common.sh delete mode 100755 plugin/trino-hive-hadoop2/bin/run_hive_tests.sh delete mode 100755 plugin/trino-hive-hadoop2/bin/start_hive.sh delete mode 100644 plugin/trino-hive-hadoop2/conf/docker-compose.yml delete mode 100644 plugin/trino-hive-hadoop2/conf/files/core-site.xml.s3-template delete mode 100644 plugin/trino-hive-hadoop2/conf/files/test_table.csv delete mode 100644 plugin/trino-hive-hadoop2/conf/files/test_table.csv.bz2 delete mode 100644 plugin/trino-hive-hadoop2/conf/files/test_table.csv.gz delete mode 100644 plugin/trino-hive-hadoop2/conf/files/test_table.csv.lz4 delete mode 100644 plugin/trino-hive-hadoop2/conf/files/test_table.json delete mode 100644 plugin/trino-hive-hadoop2/conf/files/test_table.json.bz2 delete mode 100644 plugin/trino-hive-hadoop2/conf/files/test_table.json.gz delete mode 100644 plugin/trino-hive-hadoop2/conf/files/test_table_csv_scan_range_select_pushdown_1.csv delete mode 100644 plugin/trino-hive-hadoop2/conf/files/test_table_csv_scan_range_select_pushdown_2.csv delete mode 100644 plugin/trino-hive-hadoop2/conf/files/test_table_csv_scan_range_select_pushdown_3.csv delete mode 100644 plugin/trino-hive-hadoop2/conf/files/test_table_json_scan_range_select_pushdown_1.json delete mode 100644 plugin/trino-hive-hadoop2/conf/files/test_table_json_scan_range_select_pushdown_2.json delete mode 100644 plugin/trino-hive-hadoop2/conf/files/test_table_json_scan_range_select_pushdown_3.json delete mode 100644 plugin/trino-hive-hadoop2/conf/files/test_table_with_comma_delimiter.csv delete mode 100644 plugin/trino-hive-hadoop2/conf/files/test_table_with_comma_delimiter.csv.bz2 delete mode 100644 plugin/trino-hive-hadoop2/conf/files/test_table_with_comma_delimiter.csv.gz delete mode 100644 plugin/trino-hive-hadoop2/conf/files/test_table_with_header.csv delete mode 100644 plugin/trino-hive-hadoop2/conf/files/test_table_with_header.csv.bz2 delete mode 100644 plugin/trino-hive-hadoop2/conf/files/test_table_with_header.csv.gz delete mode 100644 plugin/trino-hive-hadoop2/conf/files/test_table_with_header.csv.lz4 delete mode 100644 plugin/trino-hive-hadoop2/conf/files/test_table_with_header_and_footer.csv delete mode 100644 plugin/trino-hive-hadoop2/conf/files/test_table_with_header_and_footer.csv.bz2 delete mode 100644 plugin/trino-hive-hadoop2/conf/files/test_table_with_header_and_footer.csv.gz delete mode 100644 plugin/trino-hive-hadoop2/conf/files/test_table_with_header_and_footer.csv.lz4 delete mode 100644 plugin/trino-hive-hadoop2/conf/files/test_table_with_pipe_delimiter.csv delete mode 100644 plugin/trino-hive-hadoop2/conf/files/test_table_with_pipe_delimiter.csv.bz2 delete mode 100644 plugin/trino-hive-hadoop2/conf/files/test_table_with_pipe_delimiter.csv.gz delete mode 100644 plugin/trino-hive-hadoop2/conf/files/tez-site.xml delete mode 100644 plugin/trino-hive-hadoop2/conf/files/words delete mode 100644 plugin/trino-hive-hadoop2/conf/hive-tests-config-apache-hive3.sh delete mode 100644 plugin/trino-hive-hadoop2/conf/hive-tests-config-hdp3.sh delete mode 100644 plugin/trino-hive-hadoop2/conf/hive-tests-defaults.sh delete mode 100644 plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/TestHive.java delete mode 100644 plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHive.java delete mode 100644 plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHiveLocal.java delete mode 100644 plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileMetastore.java delete mode 100644 plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/glue/TestGlueHiveMetastore.java delete mode 100644 plugin/trino-hive/src/test/resources/spark_bucketed_nation/._SUCCESS.crc delete mode 100644 plugin/trino-hive/src/test/resources/spark_bucketed_nation/.part-00000-0a72da44-98a5-4b85-a513-2b2e82ec274f_00000.c000.snappy.orc.crc delete mode 100644 plugin/trino-hive/src/test/resources/spark_bucketed_nation/.part-00000-0a72da44-98a5-4b85-a513-2b2e82ec274f_00001.c000.snappy.orc.crc delete mode 100644 plugin/trino-hive/src/test/resources/spark_bucketed_nation/.part-00000-0a72da44-98a5-4b85-a513-2b2e82ec274f_00002.c000.snappy.orc.crc delete mode 100644 plugin/trino-hive/src/test/resources/spark_bucketed_nation/_SUCCESS delete mode 100644 plugin/trino-hive/src/test/resources/spark_bucketed_nation/part-00000-0a72da44-98a5-4b85-a513-2b2e82ec274f_00000.c000.snappy.orc delete mode 100644 plugin/trino-hive/src/test/resources/spark_bucketed_nation/part-00000-0a72da44-98a5-4b85-a513-2b2e82ec274f_00001.c000.snappy.orc delete mode 100644 plugin/trino-hive/src/test/resources/spark_bucketed_nation/part-00000-0a72da44-98a5-4b85-a513-2b2e82ec274f_00002.c000.snappy.orc diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 8fd233feea2bb..42c188ebb8c73 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -302,10 +302,6 @@ jobs: run: | export MAVEN_OPTS="${MAVEN_INSTALL_OPTS}" $MAVEN clean install ${MAVEN_FAST_INSTALL} ${MAVEN_GIB} -Dgib.logImpactedTo=gib-impacted.log -am -pl :trino-hive-hadoop2 - - name: Run Hive Tests - run: | - source plugin/trino-hive-hadoop2/conf/hive-tests-${{ matrix.config }}.sh && - plugin/trino-hive-hadoop2/bin/run_hive_tests.sh - name: Run Hive AWS Tests env: AWS_ACCESS_KEY_ID: ${{ secrets.TRINO_AWS_ACCESS_KEY_ID }} diff --git a/plugin/trino-hive-hadoop2/bin/common.sh b/plugin/trino-hive-hadoop2/bin/common.sh deleted file mode 100644 index 534bc9020ec75..0000000000000 --- a/plugin/trino-hive-hadoop2/bin/common.sh +++ /dev/null @@ -1,214 +0,0 @@ -#!/usr/bin/env bash - -function retry() { - local END - local EXIT_CODE - - END=$(($(date +%s) + 600)) - - while (( $(date +%s) < $END )); do - set +e - "$@" - EXIT_CODE=$? - set -e - - if [[ ${EXIT_CODE} == 0 ]]; then - break - fi - sleep 5 - done - - return ${EXIT_CODE} -} - -function hadoop_master_container() { - docker-compose -f "${DOCKER_COMPOSE_LOCATION}" ps -q hadoop-master | grep . -} - -function hadoop_master_ip() { - HADOOP_MASTER_CONTAINER=$(hadoop_master_container) - docker inspect -f '{{range .NetworkSettings.Networks}}{{.IPAddress}}{{end}}' $HADOOP_MASTER_CONTAINER -} - -function check_hadoop() { - HADOOP_MASTER_CONTAINER=$(hadoop_master_container) - docker exec ${HADOOP_MASTER_CONTAINER} supervisorctl status hive-server2 | grep -i running &> /dev/null && - docker exec ${HADOOP_MASTER_CONTAINER} supervisorctl status hive-metastore | grep -i running &> /dev/null && - docker exec ${HADOOP_MASTER_CONTAINER} netstat -lpn | grep -i 0.0.0.0:10000 &> /dev/null && - docker exec ${HADOOP_MASTER_CONTAINER} netstat -lpn | grep -i 0.0.0.0:9083 &> /dev/null -} - -function exec_in_hadoop_master_container() { - HADOOP_MASTER_CONTAINER=$(hadoop_master_container) - docker exec ${HADOOP_MASTER_CONTAINER} "$@" -} - -function stop_unnecessary_hadoop_services() { - HADOOP_MASTER_CONTAINER=$(hadoop_master_container) - docker exec ${HADOOP_MASTER_CONTAINER} supervisorctl status - docker exec ${HADOOP_MASTER_CONTAINER} supervisorctl stop yarn-resourcemanager - docker exec ${HADOOP_MASTER_CONTAINER} supervisorctl stop yarn-nodemanager -} - -# Expands docker compose file paths files into the format "-f $1 -f $2 ...." -# Arguments: -# $1, $2, ...: A list of docker-compose files used to start/stop containers -function expand_compose_args() { - local files=( "${@}" ) - local compose_args="" - for file in ${files[@]}; do - compose_args+=" -f ${file}" - done - echo "${compose_args}" -} - -function cleanup_docker_containers() { - local compose_args="$(expand_compose_args "$@")" - # stop containers started with "up" - docker-compose ${compose_args} down --remove-orphans - - # docker logs processes are being terminated as soon as docker container are stopped - # wait for docker logs termination - wait -} - -function cleanup_hadoop_docker_containers() { - cleanup_docker_containers "${DOCKER_COMPOSE_LOCATION}" -} - -function termination_handler() { - set +e - cleanup_docker_containers "$@" - exit 130 -} - -# Check that all arguments are the names of non-empty variables. -function check_vars() { - ( # Subshell to preserve xtrace - set +x # Disable xtrace to make the messages printed clear - local failing=0 - for arg; do - if [[ ! -v "${arg}" ]]; then - echo "error: Variable not set: ${arg}" >&2 - failing=1 - elif [[ -z "${!arg}" ]]; then - echo "error: Variable is empty: ${arg}" >&2 - failing=1 - fi - done - return "$failing" - ) -} - -SCRIPT_DIR="${BASH_SOURCE%/*}" -INTEGRATION_TESTS_ROOT="${SCRIPT_DIR}/.." -PROJECT_ROOT="${INTEGRATION_TESTS_ROOT}/../.." -DOCKER_COMPOSE_LOCATION="${INTEGRATION_TESTS_ROOT}/conf/docker-compose.yml" -source "${INTEGRATION_TESTS_ROOT}/conf/hive-tests-defaults.sh" - -# check docker and docker compose installation -docker-compose version -docker version - -# extract proxy IP -if [ -n "${DOCKER_MACHINE_NAME:-}" ] -then - PROXY=`docker-machine ip` -else - PROXY=127.0.0.1 -fi - -# Starts containers based on multiple docker compose locations -# Arguments: -# $1, $2, ...: A list of docker-compose files used to start containers -function start_docker_containers() { - local compose_args="$(expand_compose_args $@)" - # Purposefully don't surround ${compose_args} with quotes so that docker-compose infers multiple arguments - # stop already running containers - docker-compose ${compose_args} down || true - - # catch terminate signals - # trap arguments are not expanded until the trap is called, so they must be in a global variable - TRAP_ARGS="$@" - trap 'termination_handler $TRAP_ARGS' INT TERM - - # pull docker images - if [[ "${CONTINUOUS_INTEGRATION:-false}" == 'true' ]]; then - retry docker-compose ${compose_args} pull --quiet - fi - - # start containers - docker-compose ${compose_args} up -d -} - -function start_hadoop_docker_containers() { - start_docker_containers "${DOCKER_COMPOSE_LOCATION}" - - # start docker logs for hadoop container - docker-compose -f "${DOCKER_COMPOSE_LOCATION}" logs --no-color hadoop-master & - - # wait until hadoop processes is started - retry check_hadoop -} - -# $1 = base URI for table names -function create_test_tables() { - local table_name table_path - local base_path="${1:?create_test_tables requires an argument}" - base_path="${base_path%/}" # remove trailing slash - - table_name="trino_test_external_fs" - table_path="$base_path/$table_name/" - exec_in_hadoop_master_container hadoop fs -mkdir -p "${table_path}" - exec_in_hadoop_master_container hadoop fs -copyFromLocal -f /docker/files/test_table.csv{,.gz,.bz2,.lz4} "${table_path}" - exec_in_hadoop_master_container /usr/bin/hive -e "CREATE EXTERNAL TABLE $table_name(t_bigint bigint) LOCATION '${table_path}'" - - table_name="trino_test_external_fs_with_header" - table_path="$base_path/$table_name/" - exec_in_hadoop_master_container hadoop fs -mkdir -p "${table_path}" - exec_in_hadoop_master_container hadoop fs -copyFromLocal -f /docker/files/test_table_with_header.csv{,.gz,.bz2,.lz4} "${table_path}" - exec_in_hadoop_master_container /usr/bin/hive -e " - CREATE EXTERNAL TABLE $table_name(t_bigint bigint) - STORED AS TEXTFILE - LOCATION '${table_path}' - TBLPROPERTIES ('skip.header.line.count'='1')" - - table_name="trino_test_external_fs_with_header_and_footer" - table_path="$base_path/$table_name/" - exec_in_hadoop_master_container hadoop fs -mkdir -p "${table_path}" - exec_in_hadoop_master_container hadoop fs -copyFromLocal -f /docker/files/test_table_with_header_and_footer.csv{,.gz,.bz2,.lz4} "${table_path}" - exec_in_hadoop_master_container /usr/bin/hive -e " - CREATE EXTERNAL TABLE $table_name(t_bigint bigint) - STORED AS TEXTFILE - LOCATION '${table_path}' - TBLPROPERTIES ('skip.header.line.count'='2', 'skip.footer.line.count'='2')" -} - -# $1 = basename of core-site.xml template -# other arguments are names of variables to substitute in the file -function deploy_core_site_xml() { - local template="${1:?deploy_core_site_xml expects at least one argument}" - shift - local args=() - local name value - for name; do - shift - value="${!name//\\/\\\\}" # escape \ as \\ - value="${value//|/\\|}" # escape | as \| - args+=(-e "s|%$name%|$value|g") - done - exec_in_hadoop_master_container bash -c \ - 'sed "${@:2}" "/docker/files/$1" > /etc/hadoop/conf/core-site.xml' \ - bash "$template" "${args[@]}" -} - -# Checks if Gitflow Incremental Builder (GIB) is enabled and the trino-hive-hadoop2 module should be build and/or tested -function abort_if_not_gib_impacted() { - local module=plugin/trino-hive-hadoop2 - local impacted_log=gib-impacted.log - if [ -f "$impacted_log" ] && ! grep -q "^${module}$" "$impacted_log"; then - echo >&2 "Module $module not present in $impacted_log, exiting" - exit 0 - fi - return 0 -} diff --git a/plugin/trino-hive-hadoop2/bin/run_hive_tests.sh b/plugin/trino-hive-hadoop2/bin/run_hive_tests.sh deleted file mode 100755 index e7a82e9cf5004..0000000000000 --- a/plugin/trino-hive-hadoop2/bin/run_hive_tests.sh +++ /dev/null @@ -1,35 +0,0 @@ -#!/usr/bin/env bash - -set -euo pipefail -x - -. "${BASH_SOURCE%/*}/common.sh" - -abort_if_not_gib_impacted - -cleanup_hadoop_docker_containers -start_hadoop_docker_containers - -# generate test data -exec_in_hadoop_master_container sudo -Eu hive beeline -u jdbc:hive2://localhost:10000/default -n hive -f /docker/sql/create-test.sql - -stop_unnecessary_hadoop_services - -HADOOP_MASTER_IP=$(hadoop_master_ip) - -# run product tests -pushd "${PROJECT_ROOT}" -set +e -./mvnw ${MAVEN_TEST:--B} -pl :trino-hive-hadoop2 test -P test-hive-hadoop2 \ - -DHADOOP_USER_NAME=hive \ - -Dtest.metastore=localhost:9083 \ - -Dtest.database=default \ - -Dhive.metastore.thrift.client.socks-proxy="${PROXY}:1180" \ - -Dhive.hdfs.socks-proxy="${PROXY}:1180" \ - -Dhadoop-master-ip="${HADOOP_MASTER_IP}" -EXIT_CODE=$? -set -e -popd - -cleanup_hadoop_docker_containers - -exit "${EXIT_CODE}" diff --git a/plugin/trino-hive-hadoop2/bin/start_hive.sh b/plugin/trino-hive-hadoop2/bin/start_hive.sh deleted file mode 100755 index 82179fd6eb697..0000000000000 --- a/plugin/trino-hive-hadoop2/bin/start_hive.sh +++ /dev/null @@ -1,21 +0,0 @@ -#!/usr/bin/env bash - -set -euo pipefail - -. "${BASH_SOURCE%/*}/common.sh" - -cleanup_hadoop_docker_containers -start_hadoop_docker_containers - -HADOOP_MASTER_IP=$(hadoop_master_ip) - -# get short version of container ID (as shown by "docker ps") -CONTAINER=$(echo "${HADOOP_MASTER_CONTAINER}" | cut -b1-12) - -echo -echo "Proxy: ${PROXY}:1180" -echo "Hadoop: ${HADOOP_MASTER_IP}" -echo "Docker: ${CONTAINER}" -echo -echo "docker exec -it ${CONTAINER} bash" -echo diff --git a/plugin/trino-hive-hadoop2/conf/docker-compose.yml b/plugin/trino-hive-hadoop2/conf/docker-compose.yml deleted file mode 100644 index fe9414fe33ed4..0000000000000 --- a/plugin/trino-hive-hadoop2/conf/docker-compose.yml +++ /dev/null @@ -1,22 +0,0 @@ -version: '2' -services: - hadoop-master: - hostname: hadoop-master - image: '${HADOOP_BASE_IMAGE}:${DOCKER_IMAGES_VERSION}' - ports: - - '1180:1180' - - '8020:8020' # Default hadoop namenode port - - '8042:8042' - - '8088:8088' - - '9000:9000' # Default hadoop namenode port - - '9083:9083' # Metastore Thrift - - '9864:9864' # DataNode Web UI since Hadoop 3 - - '9870:9870' # NameNode Web UI since Hadoop 3 - - '10000:10000' # HiveServer2 - - '19888:19888' - - '50070:50070' # NameNode Web UI prior to Hadoop 3 - - '50075:50075' # DataNode Web UI prior to Hadoop 3 - volumes: - - ../../trino-hive/src/test/sql:/docker/sql:ro - - ./files:/docker/files:ro - - ./files/tez-site.xml:/etc/tez/conf/tez-site.xml:ro diff --git a/plugin/trino-hive-hadoop2/conf/files/core-site.xml.s3-template b/plugin/trino-hive-hadoop2/conf/files/core-site.xml.s3-template deleted file mode 100644 index 984026e39e380..0000000000000 --- a/plugin/trino-hive-hadoop2/conf/files/core-site.xml.s3-template +++ /dev/null @@ -1,61 +0,0 @@ - - - - - - - hadoop.proxyuser.hive.hosts - * - - - - hadoop.proxyuser.hive.groups - * - - - - fs.defaultFS - hdfs://hadoop-master:9000 - - - - fs.s3.awsAccessKeyId - %AWS_ACCESS_KEY_ID% - - - - fs.s3.awsSecretAccessKey - %AWS_SECRET_ACCESS_KEY% - - - - fs.s3a.access.key - %AWS_ACCESS_KEY_ID% - - - - fs.s3a.secret.key - %AWS_SECRET_ACCESS_KEY% - - - - fs.s3a.endpoint - %S3_BUCKET_ENDPOINT% - - - diff --git a/plugin/trino-hive-hadoop2/conf/files/test_table.csv b/plugin/trino-hive-hadoop2/conf/files/test_table.csv deleted file mode 100644 index 0628eaa1bdc28..0000000000000 --- a/plugin/trino-hive-hadoop2/conf/files/test_table.csv +++ /dev/null @@ -1,3 +0,0 @@ -3 -14 -15 diff --git a/plugin/trino-hive-hadoop2/conf/files/test_table.csv.bz2 b/plugin/trino-hive-hadoop2/conf/files/test_table.csv.bz2 deleted file mode 100644 index d9d6b339f90497567e1c5c7af3e705c1e5b58024..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 43 ycmZ>Y%CIzaj8qGb?Ak1r&cMLz!N4FO@j!t=v4Mw~U2%8!PNg{-9#${y21pp250sjC1 diff --git a/plugin/trino-hive-hadoop2/conf/files/test_table.json b/plugin/trino-hive-hadoop2/conf/files/test_table.json deleted file mode 100644 index 6173ff8aae879..0000000000000 --- a/plugin/trino-hive-hadoop2/conf/files/test_table.json +++ /dev/null @@ -1,2 +0,0 @@ -{"col_1":2, "col_2":4} -{"col_1":5, "col_2":6} diff --git a/plugin/trino-hive-hadoop2/conf/files/test_table.json.bz2 b/plugin/trino-hive-hadoop2/conf/files/test_table.json.bz2 deleted file mode 100644 index 6b90f2081e35b18e9228caf1cab8d1d501376773..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 87 zcmV-d0I2^$T4*^jL0KkKSqQZU0RR9j+kgNNPy~Ij004*tfC?Z0P^L^wgChc}dZR*Z tMrid)wawfll8w!A_YHDp(2tTwcj$T`p+i1fpKBk*+>uTcBmyl#z(DZ`AVB~C diff --git a/plugin/trino-hive-hadoop2/conf/files/test_table.json.gz b/plugin/trino-hive-hadoop2/conf/files/test_table.json.gz deleted file mode 100644 index ae46357976038069a4496cbac7144a0358a4d6c7..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 73 zcmV-P0Ji@hiwFo{BH3a919W9`bYFB~Vr*qDYIARH0IOC?&d-TARI)PHQ2;ZIl&p-5 fYq_dn;zkxo5=KZ8hQY%CIzaj8qGbESM*3&A`BRgMmSSfu)o|fx(c6kw<-HRmTP86SupM-kInpq9UW- Ha_#^CUC$5* diff --git a/plugin/trino-hive-hadoop2/conf/files/test_table_with_comma_delimiter.csv.gz b/plugin/trino-hive-hadoop2/conf/files/test_table_with_comma_delimiter.csv.gz deleted file mode 100644 index 2d8c9cb91edd8be4f5baa9914bc278425a840a4e..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 78 zcmb2|=HTEHT$IGXT#{N`5?_*-l#?1?o>`I+pPZkYn;4&xnvY%CIzaj8qGb+*)B_$-uzm!N4G3rl7!JaDXG(c|quF?wv|=G(4jA$jCGR00aaK AhX4Qo diff --git a/plugin/trino-hive-hadoop2/conf/files/test_table_with_header.csv.gz b/plugin/trino-hive-hadoop2/conf/files/test_table_with_header.csv.gz deleted file mode 100644 index 5466eaa771c0e37e364ba7c45098485cbc4478f5..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 31 gcmb2|=3oE=VdE#841nm-CWD7e46?s{4{!s;0f|8gTmS$7 diff --git a/plugin/trino-hive-hadoop2/conf/files/test_table_with_header.csv.lz4 b/plugin/trino-hive-hadoop2/conf/files/test_table_with_header.csv.lz4 deleted file mode 100644 index 746baf57149b9fb2011d61961b0dcb2f32fefbbf..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 19 acmZQzVBlh4VBlV0$YpHGWo*V}#03Bl1p*QP diff --git a/plugin/trino-hive-hadoop2/conf/files/test_table_with_header_and_footer.csv b/plugin/trino-hive-hadoop2/conf/files/test_table_with_header_and_footer.csv deleted file mode 100644 index c25ef79c3a630..0000000000000 --- a/plugin/trino-hive-hadoop2/conf/files/test_table_with_header_and_footer.csv +++ /dev/null @@ -1,7 +0,0 @@ -1 -2 -1 -41 -42 -4 -8 diff --git a/plugin/trino-hive-hadoop2/conf/files/test_table_with_header_and_footer.csv.bz2 b/plugin/trino-hive-hadoop2/conf/files/test_table_with_header_and_footer.csv.bz2 deleted file mode 100644 index a23e84d6c1882cb0f1b4d5678329078bcdf9eb82..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 48 zcmZ>Y%CIzaj8qGboXZz6gMop?gMmSy?tub>5?h-hgMovssz#~dqjw(5xrBrlo)%^R E05cE`EC2ui diff --git a/plugin/trino-hive-hadoop2/conf/files/test_table_with_header_and_footer.csv.gz b/plugin/trino-hive-hadoop2/conf/files/test_table_with_header_and_footer.csv.gz deleted file mode 100644 index 7a439a4d1b0e87cb9ec60a88677e78a09b87984b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 37 ocmb2|=3oE=VdE!8kBm&78JRvdGJ0n8aMJ@Oh9!^R#R&ps0nE4zTmS$7 diff --git a/plugin/trino-hive-hadoop2/conf/files/test_table_with_header_and_footer.csv.lz4 b/plugin/trino-hive-hadoop2/conf/files/test_table_with_header_and_footer.csv.lz4 deleted file mode 100644 index d337bd0bb853827030cec5004b39bcb29a4b9b95..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 27 icmZQzU=U=u0 diff --git a/plugin/trino-hive-hadoop2/conf/files/test_table_with_pipe_delimiter.csv b/plugin/trino-hive-hadoop2/conf/files/test_table_with_pipe_delimiter.csv deleted file mode 100644 index 0cc012bf882a3..0000000000000 --- a/plugin/trino-hive-hadoop2/conf/files/test_table_with_pipe_delimiter.csv +++ /dev/null @@ -1,3 +0,0 @@ -1|2 -3|4 -55|66 diff --git a/plugin/trino-hive-hadoop2/conf/files/test_table_with_pipe_delimiter.csv.bz2 b/plugin/trino-hive-hadoop2/conf/files/test_table_with_pipe_delimiter.csv.bz2 deleted file mode 100644 index df138bc6d194a6bf87b341e321dd654e30303146..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 54 zcmZ>Y%CIzaj8qGbydUs#E&~J0i3SFN$_ETA3Jit}2AVIvDxaLf>#w4AdY6%}c#*3` Izyy#w0Pr6Zl>h($ diff --git a/plugin/trino-hive-hadoop2/conf/files/test_table_with_pipe_delimiter.csv.gz b/plugin/trino-hive-hadoop2/conf/files/test_table_with_pipe_delimiter.csv.gz deleted file mode 100644 index 6634c19f33456ab008deb5170308f2cc07880d5c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 78 zcmb2|=HR&0yda5zxg@o?B)%juDJM0)JhLPtz96$8H9jRZCo?y*B(*3$y(+UnFS)pk i!PscMp~-_yD-0hQ8Jey(+`i!n6T{q_&&-7w7#IMh^BzF} diff --git a/plugin/trino-hive-hadoop2/conf/files/tez-site.xml b/plugin/trino-hive-hadoop2/conf/files/tez-site.xml deleted file mode 100644 index 8f340b5611e9f..0000000000000 --- a/plugin/trino-hive-hadoop2/conf/files/tez-site.xml +++ /dev/null @@ -1,100 +0,0 @@ - - - - - - - tez.lib.uris.ignore - false - - - tez.lib.uris - file:///usr/hdp/current/tez-client/lib/tez.tar.gz - - - tez.am.mode.session - false - - - tez.am.acl.enabled - false - - - tez.am.log.level - WARN - - - tez.task.log.level - WARN - - - tez.runtime.io.sort.mb - 8 - - - tez.am.max.app.attempts - 1 - - - tez.am.task.max.failed.attempts - 1 - - - tez.shuffle-vertex-manager.min-src-fraction - 0.10 - - - tez.shuffle-vertex-manager.max-src-fraction - 1.00 - - - tez.am.launch.cmd-opts - -server -Djava.net.preferIPv4Stack=true -XX:+UseParallelGC -Dhadoop.metrics.log.level=WARN - - - tez.am.resource.memory.mb - 512 - - - tez.task.launch.cmd-opts - -server -Djava.net.preferIPv4Stack=true -XX:+UseParallelGC -Dhadoop.metrics.log.level=WARN - - - tez.task.resource.memory.mb - 512 - - - tez.task.resource.cpu.vcores - 1 - - - tez.runtime.sort.threads - 1 - - - tez.runtime.io.sort.factor - 100 - - - tez.runtime.shuffle.memory-to-memory.enable - false - - - tez.runtime.optimize.local.fetch - true - - - hive.tez.container.size - 2048 - - diff --git a/plugin/trino-hive-hadoop2/conf/files/words b/plugin/trino-hive-hadoop2/conf/files/words deleted file mode 100644 index 6d01e61127052..0000000000000 --- a/plugin/trino-hive-hadoop2/conf/files/words +++ /dev/null @@ -1,100 +0,0 @@ -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x -x diff --git a/plugin/trino-hive-hadoop2/conf/hive-tests-config-apache-hive3.sh b/plugin/trino-hive-hadoop2/conf/hive-tests-config-apache-hive3.sh deleted file mode 100644 index 8e05591f9118f..0000000000000 --- a/plugin/trino-hive-hadoop2/conf/hive-tests-config-apache-hive3.sh +++ /dev/null @@ -1 +0,0 @@ -export HADOOP_BASE_IMAGE="ghcr.io/trinodb/testing/hive3.1-hive" diff --git a/plugin/trino-hive-hadoop2/conf/hive-tests-config-hdp3.sh b/plugin/trino-hive-hadoop2/conf/hive-tests-config-hdp3.sh deleted file mode 100644 index c736e171caba5..0000000000000 --- a/plugin/trino-hive-hadoop2/conf/hive-tests-config-hdp3.sh +++ /dev/null @@ -1 +0,0 @@ -export HADOOP_BASE_IMAGE="ghcr.io/trinodb/testing/hdp3.1-hive" diff --git a/plugin/trino-hive-hadoop2/conf/hive-tests-defaults.sh b/plugin/trino-hive-hadoop2/conf/hive-tests-defaults.sh deleted file mode 100644 index cc5a3030aee03..0000000000000 --- a/plugin/trino-hive-hadoop2/conf/hive-tests-defaults.sh +++ /dev/null @@ -1,11 +0,0 @@ -#!/usr/bin/env bash - -DEFAULT_DOCKER_VERSION=$(./mvnw help:evaluate -Dexpression=dep.docker.images.version -q -DforceStdout) - -if [ -z "$DEFAULT_DOCKER_VERSION" ]; -then - >&2 echo "Could not read dep.docker.images.version from parent POM" - exit 1 -fi - -export DOCKER_IMAGES_VERSION=${DOCKER_IMAGES_VERSION:-$DEFAULT_DOCKER_VERSION} diff --git a/plugin/trino-hive-hadoop2/pom.xml b/plugin/trino-hive-hadoop2/pom.xml index eea081792b71a..54baeb2832c83 100644 --- a/plugin/trino-hive-hadoop2/pom.xml +++ b/plugin/trino-hive-hadoop2/pom.xml @@ -216,42 +216,4 @@ test - - - - default - - true - - - - - org.apache.maven.plugins - maven-surefire-plugin - - - **/TestHive.java - - - - - - - - test-hive-hadoop2 - - - - org.apache.maven.plugins - maven-surefire-plugin - - - **/TestHive.java - - - - - - - diff --git a/plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/TestHive.java b/plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/TestHive.java deleted file mode 100644 index 2ae7c145603d2..0000000000000 --- a/plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/TestHive.java +++ /dev/null @@ -1,184 +0,0 @@ -/* - * 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.hive; - -import com.google.common.collect.ImmutableList; -import com.google.common.net.HostAndPort; -import io.trino.spi.connector.ConnectorMetadata; -import io.trino.spi.connector.SchemaTableName; -import io.trino.spi.connector.SchemaTablePrefix; -import org.apache.hadoop.net.NetUtils; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.junit.jupiter.api.Assumptions.abort; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; - -@TestInstance(PER_CLASS) -public class TestHive - extends AbstractTestHive -{ - @BeforeAll - public void initialize() - { - String metastore = System.getProperty("test.metastore"); - String database = System.getProperty("test.database"); - String hadoopMasterIp = System.getProperty("hadoop-master-ip"); - if (hadoopMasterIp != null) { - // Even though Hadoop is accessed by proxy, Hadoop still tries to resolve hadoop-master - // (e.g: in: NameNodeProxies.createProxy) - // This adds a static resolution for hadoop-master to docker container internal ip - NetUtils.addStaticResolution("hadoop-master", hadoopMasterIp); - } - - setup(HostAndPort.fromString(metastore), database); - } - - @Test - @Override - public void testHideDeltaLakeTables() - { - assertThatThrownBy(super::testHideDeltaLakeTables) - .hasMessageMatching("(?s)\n" + - "Expecting\n" + - " \\[.*\\b(\\w+.tmp_trino_test_trino_delta_lake_table_\\w+)\\b.*]\n" + - "not to contain\n" + - " \\[\\1]\n" + - "but found.*"); - - abort("not supported"); - } - - @Test - public void testHiveViewsHaveNoColumns() - { - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - assertThat(listTableColumns(metadata, newSession(), new SchemaTablePrefix(view.getSchemaName(), view.getTableName()))) - .isEmpty(); - } - } - - @Test - public void testHiveViewTranslationError() - { - try (Transaction transaction = newTransaction()) { - assertThatThrownBy(() -> transaction.getMetadata().getView(newSession(), view)) - .isInstanceOf(HiveViewNotSupportedException.class) - .hasMessageContaining("Hive views are not supported"); - - // TODO: combine this with tests for successful translation (currently in TestHiveViews product test) - } - } - - @Test - @Override - public void testUpdateBasicPartitionStatistics() - throws Exception - { - SchemaTableName tableName = temporaryTable("update_basic_partition_statistics"); - try { - createDummyPartitionedTable(tableName, STATISTICS_PARTITIONED_TABLE_COLUMNS); - // When the table has partitions, but row count statistics are set to zero, we treat this case as empty - // statistics to avoid underestimation in the CBO. This scenario may be caused when other engines are - // used to ingest data into partitioned hive tables. - testUpdatePartitionStatistics( - tableName, - EMPTY_ROWCOUNT_STATISTICS, - ImmutableList.of(BASIC_STATISTICS_1, BASIC_STATISTICS_2), - ImmutableList.of(BASIC_STATISTICS_2, BASIC_STATISTICS_1)); - } - finally { - dropTable(tableName); - } - } - - @Test - @Override - public void testUpdatePartitionColumnStatistics() - throws Exception - { - SchemaTableName tableName = temporaryTable("update_partition_column_statistics"); - try { - createDummyPartitionedTable(tableName, STATISTICS_PARTITIONED_TABLE_COLUMNS); - // When the table has partitions, but row count statistics are set to zero, we treat this case as empty - // statistics to avoid underestimation in the CBO. This scenario may be caused when other engines are - // used to ingest data into partitioned hive tables. - testUpdatePartitionStatistics( - tableName, - EMPTY_ROWCOUNT_STATISTICS, - ImmutableList.of(STATISTICS_1_1, STATISTICS_1_2, STATISTICS_2), - ImmutableList.of(STATISTICS_1_2, STATISTICS_1_1, STATISTICS_2)); - } - finally { - dropTable(tableName); - } - } - - @Test - @Override - public void testUpdatePartitionColumnStatisticsEmptyOptionalFields() - throws Exception - { - SchemaTableName tableName = temporaryTable("update_partition_column_statistics"); - try { - createDummyPartitionedTable(tableName, STATISTICS_PARTITIONED_TABLE_COLUMNS); - // When the table has partitions, but row count statistics are set to zero, we treat this case as empty - // statistics to avoid underestimation in the CBO. This scenario may be caused when other engines are - // used to ingest data into partitioned hive tables. - testUpdatePartitionStatistics( - tableName, - EMPTY_ROWCOUNT_STATISTICS, - ImmutableList.of(STATISTICS_EMPTY_OPTIONAL_FIELDS), - ImmutableList.of(STATISTICS_EMPTY_OPTIONAL_FIELDS)); - } - finally { - dropTable(tableName); - } - } - - @Test - @Override - public void testStorePartitionWithStatistics() - throws Exception - { - // When the table has partitions, but row count statistics are set to zero, we treat this case as empty - // statistics to avoid underestimation in the CBO. This scenario may be caused when other engines are - // used to ingest data into partitioned hive tables. - testStorePartitionWithStatistics(STATISTICS_PARTITIONED_TABLE_COLUMNS, STATISTICS_1, STATISTICS_2, STATISTICS_1_1, EMPTY_ROWCOUNT_STATISTICS); - } - - @Test - @Override - public void testDataColumnProperties() - { - // Column properties are currently not supported in ThriftHiveMetastore - assertThatThrownBy(super::testDataColumnProperties) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Persisting column properties is not supported: Column{name=id, type=bigint}"); - } - - @Test - @Override - public void testPartitionColumnProperties() - { - // Column properties are currently not supported in ThriftHiveMetastore - assertThatThrownBy(super::testPartitionColumnProperties) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Persisting column properties is not supported: Column{name=part_key, type=varchar(256)}"); - } -} diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHive.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHive.java deleted file mode 100644 index c5a1c2d041c06..0000000000000 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHive.java +++ /dev/null @@ -1,6368 +0,0 @@ -/* - * 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.hive; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableMultimap; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Sets; -import com.google.common.net.HostAndPort; -import io.airlift.json.JsonCodec; -import io.airlift.log.Logger; -import io.airlift.slice.Slice; -import io.airlift.stats.CounterStat; -import io.airlift.units.DataSize; -import io.airlift.units.Duration; -import io.trino.filesystem.Location; -import io.trino.filesystem.TrinoFileSystem; -import io.trino.filesystem.TrinoFileSystemFactory; -import io.trino.filesystem.hdfs.HdfsFileSystemFactory; -import io.trino.hdfs.HdfsContext; -import io.trino.hdfs.HdfsEnvironment; -import io.trino.operator.GroupByHashPageIndexerFactory; -import io.trino.plugin.base.CatalogName; -import io.trino.plugin.base.metrics.LongCount; -import io.trino.plugin.hive.LocationService.WriteInfo; -import io.trino.plugin.hive.fs.DirectoryLister; -import io.trino.plugin.hive.fs.RemoteIterator; -import io.trino.plugin.hive.fs.TransactionScopeCachingDirectoryListerFactory; -import io.trino.plugin.hive.fs.TrinoFileStatus; -import io.trino.plugin.hive.fs.TrinoFileStatusRemoteIterator; -import io.trino.plugin.hive.line.LinePageSource; -import io.trino.plugin.hive.metastore.Column; -import io.trino.plugin.hive.metastore.HiveColumnStatistics; -import io.trino.plugin.hive.metastore.HiveMetastore; -import io.trino.plugin.hive.metastore.HiveMetastoreFactory; -import io.trino.plugin.hive.metastore.HivePrincipal; -import io.trino.plugin.hive.metastore.HivePrivilegeInfo; -import io.trino.plugin.hive.metastore.HivePrivilegeInfo.HivePrivilege; -import io.trino.plugin.hive.metastore.Partition; -import io.trino.plugin.hive.metastore.PartitionWithStatistics; -import io.trino.plugin.hive.metastore.PrincipalPrivileges; -import io.trino.plugin.hive.metastore.SemiTransactionalHiveMetastore; -import io.trino.plugin.hive.metastore.SortingColumn; -import io.trino.plugin.hive.metastore.StorageFormat; -import io.trino.plugin.hive.metastore.Table; -import io.trino.plugin.hive.metastore.cache.CachingHiveMetastore; -import io.trino.plugin.hive.metastore.cache.CachingHiveMetastoreConfig; -import io.trino.plugin.hive.metastore.thrift.BridgingHiveMetastore; -import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreConfig; -import io.trino.plugin.hive.orc.OrcPageSource; -import io.trino.plugin.hive.parquet.ParquetPageSource; -import io.trino.plugin.hive.rcfile.RcFilePageSource; -import io.trino.plugin.hive.security.SqlStandardAccessControlMetadata; -import io.trino.spi.Page; -import io.trino.spi.TrinoException; -import io.trino.spi.block.Block; -import io.trino.spi.connector.Assignment; -import io.trino.spi.connector.CatalogSchemaTableName; -import io.trino.spi.connector.ColumnHandle; -import io.trino.spi.connector.ColumnMetadata; -import io.trino.spi.connector.ConnectorBucketNodeMap; -import io.trino.spi.connector.ConnectorInsertTableHandle; -import io.trino.spi.connector.ConnectorMaterializedViewDefinition; -import io.trino.spi.connector.ConnectorMetadata; -import io.trino.spi.connector.ConnectorNodePartitioningProvider; -import io.trino.spi.connector.ConnectorOutputTableHandle; -import io.trino.spi.connector.ConnectorPageSink; -import io.trino.spi.connector.ConnectorPageSinkProvider; -import io.trino.spi.connector.ConnectorPageSource; -import io.trino.spi.connector.ConnectorPageSourceProvider; -import io.trino.spi.connector.ConnectorPartitioningHandle; -import io.trino.spi.connector.ConnectorSession; -import io.trino.spi.connector.ConnectorSplit; -import io.trino.spi.connector.ConnectorSplitManager; -import io.trino.spi.connector.ConnectorSplitSource; -import io.trino.spi.connector.ConnectorTableHandle; -import io.trino.spi.connector.ConnectorTableLayout; -import io.trino.spi.connector.ConnectorTableMetadata; -import io.trino.spi.connector.ConnectorTableProperties; -import io.trino.spi.connector.ConnectorTransactionHandle; -import io.trino.spi.connector.ConnectorViewDefinition; -import io.trino.spi.connector.ConnectorViewDefinition.ViewColumn; -import io.trino.spi.connector.Constraint; -import io.trino.spi.connector.ConstraintApplicationResult; -import io.trino.spi.connector.DiscretePredicates; -import io.trino.spi.connector.DynamicFilter; -import io.trino.spi.connector.ProjectionApplicationResult; -import io.trino.spi.connector.SchemaTableName; -import io.trino.spi.connector.SchemaTablePrefix; -import io.trino.spi.connector.SortingProperty; -import io.trino.spi.connector.TableColumnsMetadata; -import io.trino.spi.connector.TableNotFoundException; -import io.trino.spi.connector.TableScanRedirectApplicationResult; -import io.trino.spi.connector.ViewNotFoundException; -import io.trino.spi.expression.ConnectorExpression; -import io.trino.spi.expression.FieldDereference; -import io.trino.spi.expression.Variable; -import io.trino.spi.metrics.Metrics; -import io.trino.spi.predicate.Domain; -import io.trino.spi.predicate.NullableValue; -import io.trino.spi.predicate.Range; -import io.trino.spi.predicate.TupleDomain; -import io.trino.spi.predicate.ValueSet; -import io.trino.spi.statistics.ColumnStatistics; -import io.trino.spi.statistics.TableStatistics; -import io.trino.spi.type.ArrayType; -import io.trino.spi.type.CharType; -import io.trino.spi.type.MapType; -import io.trino.spi.type.NamedTypeSignature; -import io.trino.spi.type.RowFieldName; -import io.trino.spi.type.RowType; -import io.trino.spi.type.SqlDate; -import io.trino.spi.type.SqlTimestamp; -import io.trino.spi.type.SqlTimestampWithTimeZone; -import io.trino.spi.type.SqlVarbinary; -import io.trino.spi.type.Type; -import io.trino.spi.type.TypeId; -import io.trino.spi.type.TypeOperators; -import io.trino.spi.type.VarcharType; -import io.trino.sql.gen.JoinCompiler; -import io.trino.testing.MaterializedResult; -import io.trino.testing.MaterializedRow; -import io.trino.testing.TestingConnectorSession; -import io.trino.testing.TestingNodeManager; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.assertj.core.api.InstanceOfAssertFactories; -import org.joda.time.DateTime; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Disabled; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; -import org.junit.jupiter.api.parallel.Execution; - -import java.io.IOException; -import java.io.OutputStream; -import java.math.BigDecimal; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.util.Arrays; -import java.util.Collection; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.OptionalDouble; -import java.util.OptionalInt; -import java.util.OptionalLong; -import java.util.Set; -import java.util.UUID; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ThreadLocalRandom; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Function; -import java.util.function.Predicate; -import java.util.stream.IntStream; -import java.util.stream.LongStream; - -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.collect.ImmutableList.toImmutableList; -import static com.google.common.collect.ImmutableMap.toImmutableMap; -import static com.google.common.collect.ImmutableSet.toImmutableSet; -import static com.google.common.collect.Iterables.concat; -import static com.google.common.collect.Iterables.getOnlyElement; -import static com.google.common.collect.Lists.newArrayList; -import static com.google.common.collect.Lists.reverse; -import static com.google.common.collect.Maps.uniqueIndex; -import static com.google.common.collect.MoreCollectors.onlyElement; -import static com.google.common.collect.Sets.difference; -import static com.google.common.collect.Streams.stream; -import static com.google.common.hash.Hashing.sha256; -import static com.google.common.io.MoreFiles.deleteRecursively; -import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; -import static io.airlift.concurrent.MoreFutures.getFutureValue; -import static io.airlift.concurrent.Threads.daemonThreadsNamed; -import static io.airlift.slice.Slices.utf8Slice; -import static io.airlift.testing.Assertions.assertGreaterThan; -import static io.airlift.testing.Assertions.assertGreaterThanOrEqual; -import static io.airlift.testing.Assertions.assertInstanceOf; -import static io.airlift.testing.Assertions.assertLessThanOrEqual; -import static io.airlift.units.DataSize.Unit.KILOBYTE; -import static io.trino.parquet.reader.ParquetReader.PARQUET_CODEC_METRIC_PREFIX; -import static io.trino.plugin.hive.AbstractTestHive.TransactionDeleteInsertTestTag.COMMIT; -import static io.trino.plugin.hive.AbstractTestHive.TransactionDeleteInsertTestTag.ROLLBACK_AFTER_APPEND_PAGE; -import static io.trino.plugin.hive.AbstractTestHive.TransactionDeleteInsertTestTag.ROLLBACK_AFTER_BEGIN_INSERT; -import static io.trino.plugin.hive.AbstractTestHive.TransactionDeleteInsertTestTag.ROLLBACK_AFTER_DELETE; -import static io.trino.plugin.hive.AbstractTestHive.TransactionDeleteInsertTestTag.ROLLBACK_AFTER_FINISH_INSERT; -import static io.trino.plugin.hive.AbstractTestHive.TransactionDeleteInsertTestTag.ROLLBACK_AFTER_SINK_FINISH; -import static io.trino.plugin.hive.AbstractTestHive.TransactionDeleteInsertTestTag.ROLLBACK_RIGHT_AWAY; -import static io.trino.plugin.hive.HiveBasicStatistics.createEmptyStatistics; -import static io.trino.plugin.hive.HiveBasicStatistics.createZeroStatistics; -import static io.trino.plugin.hive.HiveColumnHandle.BUCKET_COLUMN_NAME; -import static io.trino.plugin.hive.HiveColumnHandle.ColumnType.PARTITION_KEY; -import static io.trino.plugin.hive.HiveColumnHandle.bucketColumnHandle; -import static io.trino.plugin.hive.HiveColumnHandle.createBaseColumn; -import static io.trino.plugin.hive.HiveErrorCode.HIVE_INVALID_BUCKET_FILES; -import static io.trino.plugin.hive.HiveErrorCode.HIVE_INVALID_PARTITION_VALUE; -import static io.trino.plugin.hive.HiveErrorCode.HIVE_PARTITION_SCHEMA_MISMATCH; -import static io.trino.plugin.hive.HiveMetadata.TRINO_QUERY_ID_NAME; -import static io.trino.plugin.hive.HiveMetadata.TRINO_VERSION_NAME; -import static io.trino.plugin.hive.HiveStorageFormat.AVRO; -import static io.trino.plugin.hive.HiveStorageFormat.CSV; -import static io.trino.plugin.hive.HiveStorageFormat.JSON; -import static io.trino.plugin.hive.HiveStorageFormat.ORC; -import static io.trino.plugin.hive.HiveStorageFormat.PARQUET; -import static io.trino.plugin.hive.HiveStorageFormat.RCBINARY; -import static io.trino.plugin.hive.HiveStorageFormat.RCTEXT; -import static io.trino.plugin.hive.HiveStorageFormat.REGEX; -import static io.trino.plugin.hive.HiveStorageFormat.SEQUENCEFILE; -import static io.trino.plugin.hive.HiveStorageFormat.TEXTFILE; -import static io.trino.plugin.hive.HiveTableProperties.BUCKETED_BY_PROPERTY; -import static io.trino.plugin.hive.HiveTableProperties.BUCKET_COUNT_PROPERTY; -import static io.trino.plugin.hive.HiveTableProperties.EXTERNAL_LOCATION_PROPERTY; -import static io.trino.plugin.hive.HiveTableProperties.PARTITIONED_BY_PROPERTY; -import static io.trino.plugin.hive.HiveTableProperties.SORTED_BY_PROPERTY; -import static io.trino.plugin.hive.HiveTableProperties.STORAGE_FORMAT_PROPERTY; -import static io.trino.plugin.hive.HiveTableProperties.TRANSACTIONAL; -import static io.trino.plugin.hive.HiveTestUtils.HDFS_ENVIRONMENT; -import static io.trino.plugin.hive.HiveTestUtils.HDFS_FILE_SYSTEM_FACTORY; -import static io.trino.plugin.hive.HiveTestUtils.HDFS_FILE_SYSTEM_STATS; -import static io.trino.plugin.hive.HiveTestUtils.PAGE_SORTER; -import static io.trino.plugin.hive.HiveTestUtils.SESSION; -import static io.trino.plugin.hive.HiveTestUtils.arrayType; -import static io.trino.plugin.hive.HiveTestUtils.getDefaultHiveFileWriterFactories; -import static io.trino.plugin.hive.HiveTestUtils.getDefaultHivePageSourceFactories; -import static io.trino.plugin.hive.HiveTestUtils.getHiveSession; -import static io.trino.plugin.hive.HiveTestUtils.getHiveSessionProperties; -import static io.trino.plugin.hive.HiveTestUtils.getTypes; -import static io.trino.plugin.hive.HiveTestUtils.mapType; -import static io.trino.plugin.hive.HiveTestUtils.rowType; -import static io.trino.plugin.hive.HiveType.HIVE_INT; -import static io.trino.plugin.hive.HiveType.HIVE_LONG; -import static io.trino.plugin.hive.HiveType.HIVE_STRING; -import static io.trino.plugin.hive.HiveType.toHiveType; -import static io.trino.plugin.hive.LocationHandle.WriteMode.STAGE_AND_MOVE_TO_TARGET_DIRECTORY; -import static io.trino.plugin.hive.TableType.MANAGED_TABLE; -import static io.trino.plugin.hive.TestingThriftHiveMetastoreBuilder.testingThriftHiveMetastoreBuilder; -import static io.trino.plugin.hive.acid.AcidTransaction.NO_ACID_TRANSACTION; -import static io.trino.plugin.hive.metastore.HiveColumnStatistics.createBinaryColumnStatistics; -import static io.trino.plugin.hive.metastore.HiveColumnStatistics.createBooleanColumnStatistics; -import static io.trino.plugin.hive.metastore.HiveColumnStatistics.createDateColumnStatistics; -import static io.trino.plugin.hive.metastore.HiveColumnStatistics.createDecimalColumnStatistics; -import static io.trino.plugin.hive.metastore.HiveColumnStatistics.createDoubleColumnStatistics; -import static io.trino.plugin.hive.metastore.HiveColumnStatistics.createIntegerColumnStatistics; -import static io.trino.plugin.hive.metastore.HiveColumnStatistics.createStringColumnStatistics; -import static io.trino.plugin.hive.metastore.PrincipalPrivileges.NO_PRIVILEGES; -import static io.trino.plugin.hive.metastore.SortingColumn.Order.ASCENDING; -import static io.trino.plugin.hive.metastore.SortingColumn.Order.DESCENDING; -import static io.trino.plugin.hive.metastore.StorageFormat.fromHiveStorageFormat; -import static io.trino.plugin.hive.metastore.cache.CachingHiveMetastore.createCachingHiveMetastore; -import static io.trino.plugin.hive.orc.OrcPageSource.ORC_CODEC_METRIC_PREFIX; -import static io.trino.plugin.hive.util.HiveBucketing.BucketingVersion.BUCKETING_V1; -import static io.trino.plugin.hive.util.HiveUtil.DELTA_LAKE_PROVIDER; -import static io.trino.plugin.hive.util.HiveUtil.ICEBERG_TABLE_TYPE_NAME; -import static io.trino.plugin.hive.util.HiveUtil.ICEBERG_TABLE_TYPE_VALUE; -import static io.trino.plugin.hive.util.HiveUtil.SPARK_TABLE_PROVIDER_KEY; -import static io.trino.plugin.hive.util.HiveUtil.columnExtraInfo; -import static io.trino.plugin.hive.util.HiveUtil.toPartitionValues; -import static io.trino.plugin.hive.util.HiveWriteUtils.getTableDefaultLocation; -import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; -import static io.trino.spi.StandardErrorCode.TRANSACTION_CONFLICT; -import static io.trino.spi.connector.MetadataProvider.NOOP_METADATA_PROVIDER; -import static io.trino.spi.connector.RetryMode.NO_RETRIES; -import static io.trino.spi.connector.SortOrder.ASC_NULLS_FIRST; -import static io.trino.spi.connector.SortOrder.DESC_NULLS_LAST; -import static io.trino.spi.security.PrincipalType.USER; -import static io.trino.spi.type.BigintType.BIGINT; -import static io.trino.spi.type.BooleanType.BOOLEAN; -import static io.trino.spi.type.CharType.createCharType; -import static io.trino.spi.type.DateType.DATE; -import static io.trino.spi.type.DecimalType.createDecimalType; -import static io.trino.spi.type.DoubleType.DOUBLE; -import static io.trino.spi.type.HyperLogLogType.HYPER_LOG_LOG; -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.TimestampType.TIMESTAMP_MILLIS; -import static io.trino.spi.type.TimestampWithTimeZoneType.TIMESTAMP_TZ_MILLIS; -import static io.trino.spi.type.TinyintType.TINYINT; -import static io.trino.spi.type.VarbinaryType.VARBINARY; -import static io.trino.spi.type.VarcharType.VARCHAR; -import static io.trino.spi.type.VarcharType.createUnboundedVarcharType; -import static io.trino.spi.type.VarcharType.createVarcharType; -import static io.trino.testing.DateTimeTestingUtils.sqlTimestampOf; -import static io.trino.testing.MaterializedResult.materializeSourceDataStream; -import static io.trino.testing.QueryAssertions.assertEqualsIgnoreOrder; -import static io.trino.testing.TestingNames.randomNameSuffix; -import static io.trino.testing.TestingPageSinkId.TESTING_PAGE_SINK_ID; -import static io.trino.testing.assertions.TrinoExceptionAssert.assertTrinoExceptionThrownBy; -import static io.trino.type.InternalTypeManager.TESTING_TYPE_MANAGER; -import static java.lang.Float.floatToRawIntBits; -import static java.lang.Math.toIntExact; -import static java.lang.String.format; -import static java.nio.charset.StandardCharsets.UTF_8; -import static java.nio.file.Files.createTempDirectory; -import static java.util.Locale.ENGLISH; -import static java.util.Objects.requireNonNull; -import static java.util.concurrent.Executors.newCachedThreadPool; -import static java.util.concurrent.Executors.newScheduledThreadPool; -import static java.util.concurrent.TimeUnit.MILLISECONDS; -import static java.util.concurrent.TimeUnit.MINUTES; -import static java.util.concurrent.TimeUnit.SECONDS; -import static java.util.stream.Collectors.toList; -import static org.apache.hadoop.hive.common.FileUtils.makePartName; -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.assertj.core.api.Fail.fail; -import static org.assertj.core.data.Offset.offset; -import static org.joda.time.DateTimeZone.UTC; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; - -@TestInstance(PER_CLASS) -@Execution(SAME_THREAD) // staging directory is shared mutable state -public abstract class AbstractTestHive -{ - private static final Logger log = Logger.get(AbstractTestHive.class); - - protected static final String TEMPORARY_TABLE_PREFIX = "tmp_trino_test_"; - - protected static final String INVALID_DATABASE = "totally_invalid_database_name"; - protected static final String INVALID_TABLE = "totally_invalid_table_name"; - - protected static final String TEST_SERVER_VERSION = "test_version"; - - private static final Type ARRAY_TYPE = arrayType(createUnboundedVarcharType()); - private static final Type MAP_TYPE = mapType(createUnboundedVarcharType(), BIGINT); - private static final Type ROW_TYPE = rowType(ImmutableList.of( - new NamedTypeSignature(Optional.of(new RowFieldName("f_string")), createUnboundedVarcharType().getTypeSignature()), - new NamedTypeSignature(Optional.of(new RowFieldName("f_bigint")), BIGINT.getTypeSignature()), - new NamedTypeSignature(Optional.of(new RowFieldName("f_boolean")), BOOLEAN.getTypeSignature()))); - - private static final List CREATE_TABLE_COLUMNS = ImmutableList.builder() - .add(new ColumnMetadata("id", BIGINT)) - .add(new ColumnMetadata("t_string", createUnboundedVarcharType())) - .add(new ColumnMetadata("t_tinyint", TINYINT)) - .add(new ColumnMetadata("t_smallint", SMALLINT)) - .add(new ColumnMetadata("t_integer", INTEGER)) - .add(new ColumnMetadata("t_bigint", BIGINT)) - .add(new ColumnMetadata("t_float", REAL)) - .add(new ColumnMetadata("t_double", DOUBLE)) - .add(new ColumnMetadata("t_boolean", BOOLEAN)) - .add(new ColumnMetadata("t_array", ARRAY_TYPE)) - .add(new ColumnMetadata("t_map", MAP_TYPE)) - .add(new ColumnMetadata("t_row", ROW_TYPE)) - .build(); - - private static final MaterializedResult CREATE_TABLE_DATA = - MaterializedResult.resultBuilder(SESSION, BIGINT, createUnboundedVarcharType(), TINYINT, SMALLINT, INTEGER, BIGINT, REAL, DOUBLE, BOOLEAN, ARRAY_TYPE, MAP_TYPE, ROW_TYPE) - .row(1L, "hello", (byte) 45, (short) 345, 234, 123L, -754.1985f, 43.5, true, ImmutableList.of("apple", "banana"), ImmutableMap.of("one", 1L, "two", 2L), ImmutableList.of("true", 1L, true)) - .row(2L, null, null, null, null, null, null, null, null, null, null, null) - .row(3L, "bye", (byte) 46, (short) 346, 345, 456L, 754.2008f, 98.1, false, ImmutableList.of("ape", "bear"), ImmutableMap.of("three", 3L, "four", 4L), ImmutableList.of("false", 0L, false)) - .build(); - - protected static final List CREATE_TABLE_COLUMNS_PARTITIONED = ImmutableList.builder() - .addAll(CREATE_TABLE_COLUMNS) - .add(new ColumnMetadata("ds", createUnboundedVarcharType())) - .build(); - - protected static final Set COLUMN_NAMES_PARTITIONED = CREATE_TABLE_COLUMNS_PARTITIONED.stream().map(ColumnMetadata::getName).collect(toImmutableSet()); - - protected static final Predicate PARTITION_COLUMN_FILTER = columnName -> columnName.equals("ds") || columnName.startsWith("part_"); - - private static final MaterializedResult CREATE_TABLE_PARTITIONED_DATA = new MaterializedResult( - CREATE_TABLE_DATA.getMaterializedRows().stream() - .map(row -> new MaterializedRow(row.getPrecision(), newArrayList(concat(row.getFields(), ImmutableList.of("2015-07-0" + row.getField(0)))))) - .collect(toList()), - ImmutableList.builder() - .addAll(CREATE_TABLE_DATA.getTypes()) - .add(createUnboundedVarcharType()) - .build()); - - private static final String CREATE_TABLE_PARTITIONED_DATA_2ND_PARTITION_VALUE = "2015-07-04"; - - private static final MaterializedResult CREATE_TABLE_PARTITIONED_DATA_2ND = - MaterializedResult.resultBuilder(SESSION, BIGINT, createUnboundedVarcharType(), TINYINT, SMALLINT, INTEGER, BIGINT, REAL, DOUBLE, BOOLEAN, ARRAY_TYPE, MAP_TYPE, ROW_TYPE, createUnboundedVarcharType()) - .row(4L, "hello", (byte) 45, (short) 345, 234, 123L, 754.1985f, 43.5, true, ImmutableList.of("apple", "banana"), ImmutableMap.of("one", 1L, "two", 2L), ImmutableList.of("true", 1L, true), CREATE_TABLE_PARTITIONED_DATA_2ND_PARTITION_VALUE) - .row(5L, null, null, null, null, null, null, null, null, null, null, null, CREATE_TABLE_PARTITIONED_DATA_2ND_PARTITION_VALUE) - .row(6L, "bye", (byte) 46, (short) 346, 345, 456L, -754.2008f, 98.1, false, ImmutableList.of("ape", "bear"), ImmutableMap.of("three", 3L, "four", 4L), ImmutableList.of("false", 0L, false), CREATE_TABLE_PARTITIONED_DATA_2ND_PARTITION_VALUE) - .build(); - - private static final List MISMATCH_SCHEMA_PRIMITIVE_COLUMN_BEFORE = ImmutableList.builder() - .add(new ColumnMetadata("tinyint_to_smallint", TINYINT)) - .add(new ColumnMetadata("tinyint_to_integer", TINYINT)) - .add(new ColumnMetadata("tinyint_to_bigint", TINYINT)) - .add(new ColumnMetadata("smallint_to_integer", SMALLINT)) - .add(new ColumnMetadata("smallint_to_bigint", SMALLINT)) - .add(new ColumnMetadata("integer_to_bigint", INTEGER)) - .add(new ColumnMetadata("integer_to_varchar", INTEGER)) - .add(new ColumnMetadata("varchar_to_integer", createUnboundedVarcharType())) - .add(new ColumnMetadata("float_to_double", REAL)) - .add(new ColumnMetadata("varchar_to_drop_in_row", createUnboundedVarcharType())) - .build(); - - private static final List MISMATCH_SCHEMA_TABLE_BEFORE = ImmutableList.builder() - .addAll(MISMATCH_SCHEMA_PRIMITIVE_COLUMN_BEFORE) - .add(new ColumnMetadata("struct_to_struct", toRowType(MISMATCH_SCHEMA_PRIMITIVE_COLUMN_BEFORE))) - .add(new ColumnMetadata("list_to_list", arrayType(toRowType(MISMATCH_SCHEMA_PRIMITIVE_COLUMN_BEFORE)))) - .add(new ColumnMetadata("map_to_map", mapType(MISMATCH_SCHEMA_PRIMITIVE_COLUMN_BEFORE.get(1).getType(), toRowType(MISMATCH_SCHEMA_PRIMITIVE_COLUMN_BEFORE)))) - .add(new ColumnMetadata("ds", createUnboundedVarcharType())) - .build(); - - private static RowType toRowType(List columns) - { - return rowType(columns.stream() - .map(col -> new NamedTypeSignature(Optional.of(new RowFieldName(format("f_%s", col.getName()))), col.getType().getTypeSignature())) - .collect(toImmutableList())); - } - - private static final MaterializedResult MISMATCH_SCHEMA_PRIMITIVE_FIELDS_DATA_BEFORE = - MaterializedResult.resultBuilder(SESSION, TINYINT, TINYINT, TINYINT, SMALLINT, SMALLINT, INTEGER, INTEGER, createUnboundedVarcharType(), REAL, createUnboundedVarcharType()) - .row((byte) -11, (byte) 12, (byte) -13, (short) 14, (short) 15, -16, 17, "2147483647", 18.0f, "2016-08-01") - .row((byte) 21, (byte) -22, (byte) 23, (short) -24, (short) 25, 26, -27, "asdf", -28.0f, "2016-08-02") - .row((byte) -31, (byte) -32, (byte) 33, (short) 34, (short) -35, 36, 37, "-923", 39.5f, "2016-08-03") - .row(null, (byte) 42, (byte) 43, (short) 44, (short) -45, 46, 47, "2147483648", 49.5f, "2016-08-03") - .build(); - - private static final MaterializedResult MISMATCH_SCHEMA_TABLE_DATA_BEFORE = - MaterializedResult.resultBuilder(SESSION, MISMATCH_SCHEMA_TABLE_BEFORE.stream().map(ColumnMetadata::getType).collect(toImmutableList())) - .rows(MISMATCH_SCHEMA_PRIMITIVE_FIELDS_DATA_BEFORE.getMaterializedRows() - .stream() - .map(materializedRow -> { - List result = materializedRow.getFields(); - List rowResult = materializedRow.getFields(); - result.add(rowResult); - result.add(Arrays.asList(rowResult, null, rowResult)); - result.add(ImmutableMap.of(rowResult.get(1), rowResult)); - result.add(rowResult.get(9)); - return new MaterializedRow(materializedRow.getPrecision(), result); - }).collect(toImmutableList())) - .build(); - - private static final List MISMATCH_SCHEMA_PRIMITIVE_COLUMN_AFTER = ImmutableList.builder() - .add(new ColumnMetadata("tinyint_to_smallint", SMALLINT)) - .add(new ColumnMetadata("tinyint_to_integer", INTEGER)) - .add(new ColumnMetadata("tinyint_to_bigint", BIGINT)) - .add(new ColumnMetadata("smallint_to_integer", INTEGER)) - .add(new ColumnMetadata("smallint_to_bigint", BIGINT)) - .add(new ColumnMetadata("integer_to_bigint", BIGINT)) - .add(new ColumnMetadata("integer_to_varchar", createUnboundedVarcharType())) - .add(new ColumnMetadata("varchar_to_integer", INTEGER)) - .add(new ColumnMetadata("float_to_double", DOUBLE)) - .add(new ColumnMetadata("varchar_to_drop_in_row", createUnboundedVarcharType())) - .build(); - - private static final Type MISMATCH_SCHEMA_ROW_TYPE_APPEND = toRowType(ImmutableList.builder() - .addAll(MISMATCH_SCHEMA_PRIMITIVE_COLUMN_AFTER) - .add(new ColumnMetadata(format("%s_append", MISMATCH_SCHEMA_PRIMITIVE_COLUMN_AFTER.get(0).getName()), MISMATCH_SCHEMA_PRIMITIVE_COLUMN_AFTER.get(0).getType())) - .build()); - private static final Type MISMATCH_SCHEMA_ROW_TYPE_DROP = toRowType(MISMATCH_SCHEMA_PRIMITIVE_COLUMN_AFTER.subList(0, MISMATCH_SCHEMA_PRIMITIVE_COLUMN_AFTER.size() - 1)); - - private static final List MISMATCH_SCHEMA_TABLE_AFTER = ImmutableList.builder() - .addAll(MISMATCH_SCHEMA_PRIMITIVE_COLUMN_AFTER) - .add(new ColumnMetadata("struct_to_struct", MISMATCH_SCHEMA_ROW_TYPE_APPEND)) - .add(new ColumnMetadata("list_to_list", arrayType(MISMATCH_SCHEMA_ROW_TYPE_APPEND))) - .add(new ColumnMetadata("map_to_map", mapType(MISMATCH_SCHEMA_PRIMITIVE_COLUMN_AFTER.get(1).getType(), MISMATCH_SCHEMA_ROW_TYPE_DROP))) - .add(new ColumnMetadata("ds", createUnboundedVarcharType())) - .build(); - - private static final MaterializedResult MISMATCH_SCHEMA_PRIMITIVE_FIELDS_DATA_AFTER = - MaterializedResult.resultBuilder(SESSION, SMALLINT, INTEGER, BIGINT, INTEGER, BIGINT, BIGINT, createUnboundedVarcharType(), INTEGER, DOUBLE, createUnboundedVarcharType()) - .row((short) -11, 12, -13L, 14, 15L, -16L, "17", 2147483647, 18.0, "2016-08-01") - .row((short) 21, -22, 23L, -24, 25L, 26L, "-27", null, -28.0, "2016-08-02") - .row((short) -31, -32, 33L, 34, -35L, 36L, "37", -923, 39.5, "2016-08-03") - .row(null, 42, 43L, 44, -45L, 46L, "47", null, 49.5, "2016-08-03") - .build(); - - private static final MaterializedResult MISMATCH_SCHEMA_TABLE_DATA_AFTER = - MaterializedResult.resultBuilder(SESSION, MISMATCH_SCHEMA_TABLE_AFTER.stream().map(ColumnMetadata::getType).collect(toImmutableList())) - .rows(MISMATCH_SCHEMA_PRIMITIVE_FIELDS_DATA_AFTER.getMaterializedRows() - .stream() - .map(materializedRow -> { - List result = materializedRow.getFields(); - List appendFieldRowResult = materializedRow.getFields(); - appendFieldRowResult.add(null); - List dropFieldRowResult = materializedRow.getFields().subList(0, materializedRow.getFields().size() - 1); - result.add(appendFieldRowResult); - result.add(Arrays.asList(appendFieldRowResult, null, appendFieldRowResult)); - result.add(ImmutableMap.of(result.get(1), dropFieldRowResult)); - result.add(result.get(9)); - return new MaterializedRow(materializedRow.getPrecision(), result); - }).collect(toImmutableList())) - .build(); - - protected Set createTableFormats = difference( - ImmutableSet.copyOf(HiveStorageFormat.values()), - // exclude formats that change table schema with serde and read-only formats - ImmutableSet.of(AVRO, CSV, REGEX)); - - private static final JoinCompiler JOIN_COMPILER = new JoinCompiler(new TypeOperators()); - - protected static final List STATISTICS_TABLE_COLUMNS = ImmutableList.builder() - .add(new ColumnMetadata("t_boolean", BOOLEAN)) - .add(new ColumnMetadata("t_bigint", BIGINT)) - .add(new ColumnMetadata("t_integer", INTEGER)) - .add(new ColumnMetadata("t_smallint", SMALLINT)) - .add(new ColumnMetadata("t_tinyint", TINYINT)) - .add(new ColumnMetadata("t_double", DOUBLE)) - .add(new ColumnMetadata("t_float", REAL)) - .add(new ColumnMetadata("t_string", createUnboundedVarcharType())) - .add(new ColumnMetadata("t_varchar", createVarcharType(100))) - .add(new ColumnMetadata("t_char", createCharType(5))) - .add(new ColumnMetadata("t_varbinary", VARBINARY)) - .add(new ColumnMetadata("t_date", DATE)) - .add(new ColumnMetadata("t_timestamp", TIMESTAMP_MILLIS)) - .add(new ColumnMetadata("t_short_decimal", createDecimalType(5, 2))) - .add(new ColumnMetadata("t_long_decimal", createDecimalType(20, 3))) - .build(); - - protected static final List STATISTICS_PARTITIONED_TABLE_COLUMNS = ImmutableList.builder() - .addAll(STATISTICS_TABLE_COLUMNS) - .add(new ColumnMetadata("ds", VARCHAR)) - .build(); - - protected static final PartitionStatistics ZERO_TABLE_STATISTICS = new PartitionStatistics(createZeroStatistics(), ImmutableMap.of()); - protected static final PartitionStatistics EMPTY_ROWCOUNT_STATISTICS = ZERO_TABLE_STATISTICS.withBasicStatistics(ZERO_TABLE_STATISTICS.getBasicStatistics().withEmptyRowCount()); - protected static final PartitionStatistics BASIC_STATISTICS_1 = new PartitionStatistics(new HiveBasicStatistics(0, 20, 3, 0), ImmutableMap.of()); - protected static final PartitionStatistics BASIC_STATISTICS_2 = new PartitionStatistics(new HiveBasicStatistics(0, 30, 2, 0), ImmutableMap.of()); - - protected static final PartitionStatistics STATISTICS_1 = - new PartitionStatistics( - BASIC_STATISTICS_1.getBasicStatistics(), - ImmutableMap.builder() - .put("t_boolean", createBooleanColumnStatistics(OptionalLong.of(5), OptionalLong.of(6), OptionalLong.of(3))) - .put("t_bigint", createIntegerColumnStatistics(OptionalLong.of(1234L), OptionalLong.of(5678L), OptionalLong.of(2), OptionalLong.of(5))) - .put("t_integer", createIntegerColumnStatistics(OptionalLong.of(123L), OptionalLong.of(567L), OptionalLong.of(3), OptionalLong.of(4))) - .put("t_smallint", createIntegerColumnStatistics(OptionalLong.of(12L), OptionalLong.of(56L), OptionalLong.of(2), OptionalLong.of(6))) - .put("t_tinyint", createIntegerColumnStatistics(OptionalLong.of(1L), OptionalLong.of(2L), OptionalLong.of(1), OptionalLong.of(3))) - .put("t_double", createDoubleColumnStatistics(OptionalDouble.of(1234.25), OptionalDouble.of(5678.58), OptionalLong.of(7), OptionalLong.of(8))) - .put("t_float", createDoubleColumnStatistics(OptionalDouble.of(123.25), OptionalDouble.of(567.58), OptionalLong.of(9), OptionalLong.of(10))) - .put("t_string", createStringColumnStatistics(OptionalLong.of(10), OptionalLong.of(50), OptionalLong.of(3), OptionalLong.of(7))) - .put("t_varchar", createStringColumnStatistics(OptionalLong.of(100), OptionalLong.of(230), OptionalLong.of(5), OptionalLong.of(3))) - .put("t_char", createStringColumnStatistics(OptionalLong.of(5), OptionalLong.of(50), OptionalLong.of(1), OptionalLong.of(4))) - .put("t_varbinary", createBinaryColumnStatistics(OptionalLong.of(4), OptionalLong.of(50), OptionalLong.of(1))) - .put("t_date", createDateColumnStatistics(Optional.of(LocalDate.ofEpochDay(1)), Optional.of(LocalDate.ofEpochDay(2)), OptionalLong.of(7), OptionalLong.of(6))) - .put("t_timestamp", createIntegerColumnStatistics(OptionalLong.of(1234567L), OptionalLong.of(71234567L), OptionalLong.of(7), OptionalLong.of(5))) - .put("t_short_decimal", createDecimalColumnStatistics(Optional.of(new BigDecimal(10)), Optional.of(new BigDecimal(12)), OptionalLong.of(3), OptionalLong.of(5))) - .put("t_long_decimal", createDecimalColumnStatistics(Optional.of(new BigDecimal("12345678901234567.123")), Optional.of(new BigDecimal("81234567890123456.123")), OptionalLong.of(2), OptionalLong.of(1))) - .buildOrThrow()); - - protected static final PartitionStatistics STATISTICS_1_1 = - new PartitionStatistics( - new HiveBasicStatistics(OptionalLong.of(0), OptionalLong.of(15), OptionalLong.empty(), OptionalLong.of(0)), - STATISTICS_1.getColumnStatistics().entrySet() - .stream() - .filter(entry -> entry.getKey().hashCode() % 2 == 0) - .collect(toImmutableMap(Map.Entry::getKey, Map.Entry::getValue))); - - protected static final PartitionStatistics STATISTICS_1_2 = - new PartitionStatistics( - new HiveBasicStatistics(OptionalLong.of(0), OptionalLong.of(15), OptionalLong.of(3), OptionalLong.of(0)), - STATISTICS_1.getColumnStatistics().entrySet() - .stream() - .filter(entry -> entry.getKey().hashCode() % 2 == 1) - .collect(toImmutableMap(Map.Entry::getKey, Map.Entry::getValue))); - - protected static final PartitionStatistics STATISTICS_2 = - new PartitionStatistics( - BASIC_STATISTICS_2.getBasicStatistics(), - ImmutableMap.builder() - .put("t_boolean", createBooleanColumnStatistics(OptionalLong.of(4), OptionalLong.of(3), OptionalLong.of(2))) - .put("t_bigint", createIntegerColumnStatistics(OptionalLong.of(2345L), OptionalLong.of(6789L), OptionalLong.of(4), OptionalLong.of(7))) - .put("t_integer", createIntegerColumnStatistics(OptionalLong.of(234L), OptionalLong.of(678L), OptionalLong.of(5), OptionalLong.of(6))) - .put("t_smallint", createIntegerColumnStatistics(OptionalLong.of(23L), OptionalLong.of(65L), OptionalLong.of(7), OptionalLong.of(5))) - .put("t_tinyint", createIntegerColumnStatistics(OptionalLong.of(3L), OptionalLong.of(12L), OptionalLong.of(2), OptionalLong.of(3))) - .put("t_double", createDoubleColumnStatistics(OptionalDouble.of(2345.25), OptionalDouble.of(6785.58), OptionalLong.of(6), OptionalLong.of(3))) - .put("t_float", createDoubleColumnStatistics(OptionalDouble.of(235.25), OptionalDouble.of(676.58), OptionalLong.of(7), OptionalLong.of(11))) - .put("t_string", createStringColumnStatistics(OptionalLong.of(301), OptionalLong.of(600), OptionalLong.of(2), OptionalLong.of(6))) - .put("t_varchar", createStringColumnStatistics(OptionalLong.of(99), OptionalLong.of(223), OptionalLong.of(7), OptionalLong.of(1))) - .put("t_char", createStringColumnStatistics(OptionalLong.of(6), OptionalLong.of(60), OptionalLong.of(0), OptionalLong.of(3))) - .put("t_varbinary", createBinaryColumnStatistics(OptionalLong.of(2), OptionalLong.of(10), OptionalLong.of(2))) - .put("t_date", createDateColumnStatistics(Optional.of(LocalDate.ofEpochDay(2)), Optional.of(LocalDate.ofEpochDay(3)), OptionalLong.of(8), OptionalLong.of(7))) - .put("t_timestamp", createIntegerColumnStatistics(OptionalLong.of(2345671L), OptionalLong.of(12345677L), OptionalLong.of(9), OptionalLong.of(1))) - .put("t_short_decimal", createDecimalColumnStatistics(Optional.of(new BigDecimal(11)), Optional.of(new BigDecimal(14)), OptionalLong.of(5), OptionalLong.of(7))) - .put("t_long_decimal", createDecimalColumnStatistics(Optional.of(new BigDecimal("71234567890123456.123")), Optional.of(new BigDecimal("78123456789012345.123")), OptionalLong.of(2), OptionalLong.of(1))) - .buildOrThrow()); - - protected static final PartitionStatistics STATISTICS_EMPTY_OPTIONAL_FIELDS = - new PartitionStatistics( - new HiveBasicStatistics(OptionalLong.of(0), OptionalLong.of(20), OptionalLong.empty(), OptionalLong.of(0)), - ImmutableMap.builder() - .put("t_boolean", createBooleanColumnStatistics(OptionalLong.of(4), OptionalLong.of(3), OptionalLong.of(2))) - .put("t_bigint", createIntegerColumnStatistics(OptionalLong.empty(), OptionalLong.empty(), OptionalLong.of(4), OptionalLong.of(7))) - .put("t_integer", createIntegerColumnStatistics(OptionalLong.empty(), OptionalLong.empty(), OptionalLong.of(5), OptionalLong.of(6))) - .put("t_smallint", createIntegerColumnStatistics(OptionalLong.empty(), OptionalLong.empty(), OptionalLong.of(7), OptionalLong.of(5))) - .put("t_tinyint", createIntegerColumnStatistics(OptionalLong.empty(), OptionalLong.empty(), OptionalLong.of(2), OptionalLong.of(3))) - .put("t_double", createDoubleColumnStatistics(OptionalDouble.empty(), OptionalDouble.empty(), OptionalLong.of(6), OptionalLong.of(3))) - .put("t_float", createDoubleColumnStatistics(OptionalDouble.empty(), OptionalDouble.empty(), OptionalLong.of(7), OptionalLong.of(11))) - .put("t_string", createStringColumnStatistics(OptionalLong.of(0), OptionalLong.of(0), OptionalLong.of(2), OptionalLong.of(6))) - .put("t_varchar", createStringColumnStatistics(OptionalLong.of(0), OptionalLong.of(0), OptionalLong.of(7), OptionalLong.of(1))) - .put("t_char", createStringColumnStatistics(OptionalLong.of(0), OptionalLong.of(0), OptionalLong.of(0), OptionalLong.of(3))) - .put("t_varbinary", createBinaryColumnStatistics(OptionalLong.of(0), OptionalLong.of(0), OptionalLong.of(2))) - // https://issues.apache.org/jira/browse/HIVE-20098 - // .put("t_date", createDateColumnStatistics(Optional.empty(), Optional.empty(), OptionalLong.of(8), OptionalLong.of(7))) - .put("t_timestamp", createIntegerColumnStatistics(OptionalLong.empty(), OptionalLong.empty(), OptionalLong.of(9), OptionalLong.of(1))) - .put("t_short_decimal", createDecimalColumnStatistics(Optional.empty(), Optional.empty(), OptionalLong.of(5), OptionalLong.of(7))) - .put("t_long_decimal", createDecimalColumnStatistics(Optional.empty(), Optional.empty(), OptionalLong.of(2), OptionalLong.of(1))) - .buildOrThrow()); - - protected String database; - protected SchemaTableName tablePartitionFormat; - protected SchemaTableName tableUnpartitioned; - protected SchemaTableName tablePartitionedWithNull; - protected SchemaTableName tableOffline; - protected SchemaTableName tableNotReadable; - protected SchemaTableName view; - protected SchemaTableName invalidTable; - protected SchemaTableName tableBucketedStringInt; - protected SchemaTableName tableBucketedBigintBoolean; - protected SchemaTableName tableBucketedDoubleFloat; - protected SchemaTableName tablePartitionSchemaChange; - protected SchemaTableName tablePartitionSchemaChangeNonCanonical; - - protected ConnectorTableHandle invalidTableHandle; - - protected ColumnHandle dsColumn; - protected ColumnHandle fileFormatColumn; - protected ColumnHandle dummyColumn; - protected ColumnHandle intColumn; - protected ColumnHandle pStringColumn; - protected ColumnHandle pIntegerColumn; - - protected ConnectorTableProperties tablePartitionFormatProperties; - protected List tablePartitionFormatPartitions; - protected List tableUnpartitionedPartitions; - - protected HdfsEnvironment hdfsEnvironment; - protected LocationService locationService; - - protected CountingDirectoryLister countingDirectoryLister; - protected HiveMetadataFactory metadataFactory; - protected HiveTransactionManager transactionManager; - protected HiveMetastore metastoreClient; - protected ConnectorSplitManager splitManager; - protected ConnectorPageSourceProvider pageSourceProvider; - protected ConnectorPageSinkProvider pageSinkProvider; - protected ConnectorNodePartitioningProvider nodePartitioningProvider; - protected ExecutorService executor; - - private ScheduledExecutorService heartbeatService; - private java.nio.file.Path temporaryStagingDirectory; - - protected final Set materializedViews = Sets.newConcurrentHashSet(); - - @BeforeAll - public void setupClass() - throws Exception - { - executor = newCachedThreadPool(daemonThreadsNamed("hive-%s")); - heartbeatService = newScheduledThreadPool(1); - // Use separate staging directory for each test class to prevent intermittent failures coming from test parallelism - temporaryStagingDirectory = createTempDirectory("trino-staging-"); - } - - @AfterAll - public void tearDown() - { - if (executor != null) { - executor.shutdownNow(); - executor = null; - } - if (heartbeatService != null) { - heartbeatService.shutdownNow(); - heartbeatService = null; - } - if (temporaryStagingDirectory != null) { - try { - deleteRecursively(temporaryStagingDirectory, ALLOW_INSECURE); - } - catch (Exception e) { - log.warn(e, "Error deleting %s", temporaryStagingDirectory); - } - } - } - - protected void setupHive(String databaseName) - { - database = databaseName; - tablePartitionFormat = new SchemaTableName(database, "trino_test_partition_format"); - tableUnpartitioned = new SchemaTableName(database, "trino_test_unpartitioned"); - tablePartitionedWithNull = new SchemaTableName(database, "trino_test_partitioned_with_null"); - tableOffline = new SchemaTableName(database, "trino_test_offline"); - tableNotReadable = new SchemaTableName(database, "trino_test_not_readable"); - view = new SchemaTableName(database, "trino_test_view"); - invalidTable = new SchemaTableName(database, INVALID_TABLE); - tableBucketedStringInt = new SchemaTableName(database, "trino_test_bucketed_by_string_int"); - tableBucketedBigintBoolean = new SchemaTableName(database, "trino_test_bucketed_by_bigint_boolean"); - tableBucketedDoubleFloat = new SchemaTableName(database, "trino_test_bucketed_by_double_float"); - tablePartitionSchemaChange = new SchemaTableName(database, "trino_test_partition_schema_change"); - tablePartitionSchemaChangeNonCanonical = new SchemaTableName(database, "trino_test_partition_schema_change_non_canonical"); - - invalidTableHandle = new HiveTableHandle(database, INVALID_TABLE, ImmutableMap.of(), ImmutableList.of(), ImmutableList.of(), Optional.empty()); - - dsColumn = createBaseColumn("ds", -1, HIVE_STRING, VARCHAR, PARTITION_KEY, Optional.empty()); - fileFormatColumn = createBaseColumn("file_format", -1, HIVE_STRING, VARCHAR, PARTITION_KEY, Optional.empty()); - dummyColumn = createBaseColumn("dummy", -1, HIVE_INT, INTEGER, PARTITION_KEY, Optional.empty()); - intColumn = createBaseColumn("t_int", -1, HIVE_INT, INTEGER, PARTITION_KEY, Optional.empty()); - pStringColumn = createBaseColumn("p_string", -1, HIVE_STRING, VARCHAR, PARTITION_KEY, Optional.empty()); - pIntegerColumn = createBaseColumn("p_integer", -1, HIVE_INT, INTEGER, PARTITION_KEY, Optional.empty()); - - List partitionColumns = ImmutableList.of(dsColumn, fileFormatColumn, dummyColumn); - tablePartitionFormatPartitions = ImmutableList.builder() - .add(new HivePartition(tablePartitionFormat, - "ds=2012-12-29/file_format=textfile/dummy=1", - ImmutableMap.builder() - .put(dsColumn, NullableValue.of(createUnboundedVarcharType(), utf8Slice("2012-12-29"))) - .put(fileFormatColumn, NullableValue.of(createUnboundedVarcharType(), utf8Slice("textfile"))) - .put(dummyColumn, NullableValue.of(INTEGER, 1L)) - .buildOrThrow())) - .add(new HivePartition(tablePartitionFormat, - "ds=2012-12-29/file_format=sequencefile/dummy=2", - ImmutableMap.builder() - .put(dsColumn, NullableValue.of(createUnboundedVarcharType(), utf8Slice("2012-12-29"))) - .put(fileFormatColumn, NullableValue.of(createUnboundedVarcharType(), utf8Slice("sequencefile"))) - .put(dummyColumn, NullableValue.of(INTEGER, 2L)) - .buildOrThrow())) - .add(new HivePartition(tablePartitionFormat, - "ds=2012-12-29/file_format=rctext/dummy=3", - ImmutableMap.builder() - .put(dsColumn, NullableValue.of(createUnboundedVarcharType(), utf8Slice("2012-12-29"))) - .put(fileFormatColumn, NullableValue.of(createUnboundedVarcharType(), utf8Slice("rctext"))) - .put(dummyColumn, NullableValue.of(INTEGER, 3L)) - .buildOrThrow())) - .add(new HivePartition(tablePartitionFormat, - "ds=2012-12-29/file_format=rcbinary/dummy=4", - ImmutableMap.builder() - .put(dsColumn, NullableValue.of(createUnboundedVarcharType(), utf8Slice("2012-12-29"))) - .put(fileFormatColumn, NullableValue.of(createUnboundedVarcharType(), utf8Slice("rcbinary"))) - .put(dummyColumn, NullableValue.of(INTEGER, 4L)) - .buildOrThrow())) - .build(); - tableUnpartitionedPartitions = ImmutableList.of(new HivePartition(tableUnpartitioned)); - tablePartitionFormatProperties = new ConnectorTableProperties( - TupleDomain.withColumnDomains(ImmutableMap.of( - dsColumn, Domain.create(ValueSet.ofRanges(Range.equal(createUnboundedVarcharType(), utf8Slice("2012-12-29"))), false), - fileFormatColumn, Domain.create(ValueSet.ofRanges(Range.equal(createUnboundedVarcharType(), utf8Slice("textfile")), Range.equal(createUnboundedVarcharType(), utf8Slice("sequencefile")), Range.equal(createUnboundedVarcharType(), utf8Slice("rctext")), Range.equal(createUnboundedVarcharType(), utf8Slice("rcbinary"))), false), - dummyColumn, Domain.create(ValueSet.ofRanges(Range.equal(INTEGER, 1L), Range.equal(INTEGER, 2L), Range.equal(INTEGER, 3L), Range.equal(INTEGER, 4L)), false))), - Optional.empty(), - Optional.of(new DiscretePredicates(partitionColumns, ImmutableList.of( - TupleDomain.withColumnDomains(ImmutableMap.of( - dsColumn, Domain.create(ValueSet.ofRanges(Range.equal(createUnboundedVarcharType(), utf8Slice("2012-12-29"))), false), - fileFormatColumn, Domain.create(ValueSet.ofRanges(Range.equal(createUnboundedVarcharType(), utf8Slice("textfile"))), false), - dummyColumn, Domain.create(ValueSet.ofRanges(Range.equal(INTEGER, 1L)), false))), - TupleDomain.withColumnDomains(ImmutableMap.of( - dsColumn, Domain.create(ValueSet.ofRanges(Range.equal(createUnboundedVarcharType(), utf8Slice("2012-12-29"))), false), - fileFormatColumn, Domain.create(ValueSet.ofRanges(Range.equal(createUnboundedVarcharType(), utf8Slice("sequencefile"))), false), - dummyColumn, Domain.create(ValueSet.ofRanges(Range.equal(INTEGER, 2L)), false))), - TupleDomain.withColumnDomains(ImmutableMap.of( - dsColumn, Domain.create(ValueSet.ofRanges(Range.equal(createUnboundedVarcharType(), utf8Slice("2012-12-29"))), false), - fileFormatColumn, Domain.create(ValueSet.ofRanges(Range.equal(createUnboundedVarcharType(), utf8Slice("rctext"))), false), - dummyColumn, Domain.create(ValueSet.ofRanges(Range.equal(INTEGER, 3L)), false))), - TupleDomain.withColumnDomains(ImmutableMap.of( - dsColumn, Domain.create(ValueSet.ofRanges(Range.equal(createUnboundedVarcharType(), utf8Slice("2012-12-29"))), false), - fileFormatColumn, Domain.create(ValueSet.ofRanges(Range.equal(createUnboundedVarcharType(), utf8Slice("rcbinary"))), false), - dummyColumn, Domain.create(ValueSet.ofRanges(Range.equal(INTEGER, 4L)), false)))))), - ImmutableList.of()); - } - - protected final void setup(HostAndPort metastoreAddress, String databaseName) - { - HiveConfig hiveConfig = getHiveConfig() - .setParquetTimeZone("UTC") - .setRcfileTimeZone("UTC"); - - hdfsEnvironment = HDFS_ENVIRONMENT; - - CachingHiveMetastoreConfig cachingHiveMetastoreConfig = new CachingHiveMetastoreConfig(); - HiveMetastore metastore = createCachingHiveMetastore( - new BridgingHiveMetastore(testingThriftHiveMetastoreBuilder() - .metastoreClient(metastoreAddress) - .hiveConfig(hiveConfig) - .thriftMetastoreConfig(new ThriftMetastoreConfig() - .setAssumeCanonicalPartitionKeys(true)) - .fileSystemFactory(new HdfsFileSystemFactory(hdfsEnvironment, HDFS_FILE_SYSTEM_STATS)) - .build()), - new Duration(1, MINUTES), - new Duration(1, MINUTES), - Optional.of(new Duration(15, SECONDS)), - executor, - 10000, - CachingHiveMetastore.StatsRecording.ENABLED, - cachingHiveMetastoreConfig.isCacheMissing(), - cachingHiveMetastoreConfig.isPartitionCacheEnabled()); - - setup(databaseName, hiveConfig, metastore, hdfsEnvironment); - } - - protected final void setup(String databaseName, HiveConfig hiveConfig, HiveMetastore hiveMetastore, HdfsEnvironment hdfsConfiguration) - { - setupHive(databaseName); - - metastoreClient = hiveMetastore; - hdfsEnvironment = hdfsConfiguration; - HivePartitionManager partitionManager = new HivePartitionManager(hiveConfig); - HdfsFileSystemFactory fileSystemFactory = new HdfsFileSystemFactory(hdfsEnvironment, HDFS_FILE_SYSTEM_STATS); - locationService = new HiveLocationService(fileSystemFactory, hiveConfig); - JsonCodec partitionUpdateCodec = JsonCodec.jsonCodec(PartitionUpdate.class); - countingDirectoryLister = new CountingDirectoryLister(); - metadataFactory = new HiveMetadataFactory( - new CatalogName("hive"), - HiveMetastoreFactory.ofInstance(metastoreClient), - getDefaultHiveFileWriterFactories(hiveConfig, hdfsEnvironment), - fileSystemFactory, - partitionManager, - 10, - 10, - 10, - 100_000, - false, - false, - false, - true, - true, - false, - false, - 1000, - Optional.empty(), - true, - TESTING_TYPE_MANAGER, - NOOP_METADATA_PROVIDER, - locationService, - partitionUpdateCodec, - executor, - heartbeatService, - TEST_SERVER_VERSION, - (session, tableHandle) -> { - if (!tableHandle.getTableName().contains("apply_redirection_tester")) { - return Optional.empty(); - } - return Optional.of(new TableScanRedirectApplicationResult( - new CatalogSchemaTableName("hive", databaseName, "mock_redirection_target"), - ImmutableMap.of(), - TupleDomain.all())); - }, - ImmutableSet.of( - new PartitionsSystemTableProvider(partitionManager, TESTING_TYPE_MANAGER), - new PropertiesSystemTableProvider()), - metastore -> new NoneHiveMaterializedViewMetadata() - { - @Override - public List listMaterializedViews(ConnectorSession session, Optional schemaName) - { - return materializedViews.stream() - .filter(schemaName - .>map(name -> mvName -> mvName.getSchemaName().equals(name)) - .orElse(mvName -> true)) - .collect(toImmutableList()); - } - - @Override - public Optional getMaterializedView(ConnectorSession session, SchemaTableName viewName) - { - if (!viewName.getTableName().contains("materialized_view_tester")) { - return Optional.empty(); - } - return Optional.of(new ConnectorMaterializedViewDefinition( - "dummy_view_sql", - Optional.empty(), - Optional.empty(), - Optional.empty(), - ImmutableList.of(new ConnectorMaterializedViewDefinition.Column("abc", TypeId.of("type"), Optional.empty())), - Optional.of(java.time.Duration.ZERO), - Optional.empty(), - Optional.of("alice"), - ImmutableList.of(), - ImmutableMap.of())); - } - }, - SqlStandardAccessControlMetadata::new, - countingDirectoryLister, - new TransactionScopeCachingDirectoryListerFactory(hiveConfig), - false, - true, - HiveTimestampPrecision.DEFAULT_PRECISION); - transactionManager = new HiveTransactionManager(metadataFactory); - splitManager = new HiveSplitManager( - transactionManager, - partitionManager, - fileSystemFactory, - executor, - new CounterStat(), - 100, - hiveConfig.getMaxOutstandingSplitsSize(), - hiveConfig.getMinPartitionBatchSize(), - hiveConfig.getMaxPartitionBatchSize(), - hiveConfig.getMaxInitialSplits(), - hiveConfig.getSplitLoaderConcurrency(), - hiveConfig.getMaxSplitsPerSecond(), - false, - TESTING_TYPE_MANAGER, - hiveConfig.getMaxPartitionsPerScan()); - pageSinkProvider = new HivePageSinkProvider( - getDefaultHiveFileWriterFactories(hiveConfig, hdfsEnvironment), - fileSystemFactory, - PAGE_SORTER, - HiveMetastoreFactory.ofInstance(metastoreClient), - new GroupByHashPageIndexerFactory(JOIN_COMPILER), - TESTING_TYPE_MANAGER, - getHiveConfig(), - getSortingFileWriterConfig(), - locationService, - partitionUpdateCodec, - new TestingNodeManager("fake-environment"), - new HiveEventClient(), - getHiveSessionProperties(hiveConfig), - new HiveWriterStats()); - pageSourceProvider = new HivePageSourceProvider( - TESTING_TYPE_MANAGER, - hiveConfig, - getDefaultHivePageSourceFactories(hdfsEnvironment, hiveConfig)); - nodePartitioningProvider = new HiveNodePartitioningProvider( - new TestingNodeManager("fake-environment"), - TESTING_TYPE_MANAGER); - } - - /** - * Allow subclass to change default configuration. - */ - protected HiveConfig getHiveConfig() - { - return new HiveConfig() - .setTemporaryStagingDirectoryPath(temporaryStagingDirectory.resolve("temp_path_").toAbsolutePath().toString()); - } - - protected SortingFileWriterConfig getSortingFileWriterConfig() - { - return new SortingFileWriterConfig() - .setMaxOpenSortFiles(10) - .setWriterSortBufferSize(DataSize.of(100, KILOBYTE)); - } - - protected ConnectorSession newSession() - { - return newSession(ImmutableMap.of()); - } - - protected ConnectorSession newSession(Map propertyValues) - { - return TestingConnectorSession.builder() - .setPropertyMetadata(getHiveSessionProperties(getHiveConfig()).getSessionProperties()) - .setPropertyValues(propertyValues) - .build(); - } - - protected Transaction newTransaction() - { - return new HiveTransaction(transactionManager); - } - - protected interface Transaction - extends AutoCloseable - { - ConnectorMetadata getMetadata(); - - SemiTransactionalHiveMetastore getMetastore(); - - ConnectorTransactionHandle getTransactionHandle(); - - void commit(); - - void rollback(); - - @Override - void close(); - } - - static class HiveTransaction - implements Transaction - { - private final HiveTransactionManager transactionManager; - private final ConnectorTransactionHandle transactionHandle; - private boolean closed; - - public HiveTransaction(HiveTransactionManager transactionManager) - { - this.transactionManager = requireNonNull(transactionManager, "transactionManager is null"); - this.transactionHandle = new HiveTransactionHandle(false); - transactionManager.begin(transactionHandle); - getMetastore().testOnlyThrowOnCleanupFailures(); - } - - @Override - public ConnectorMetadata getMetadata() - { - return transactionManager.get(transactionHandle, SESSION.getIdentity()); - } - - @Override - public SemiTransactionalHiveMetastore getMetastore() - { - return transactionManager.get(transactionHandle, SESSION.getIdentity()).getMetastore(); - } - - @Override - public ConnectorTransactionHandle getTransactionHandle() - { - return transactionHandle; - } - - @Override - public void commit() - { - checkState(!closed); - closed = true; - transactionManager.commit(transactionHandle); - } - - @Override - public void rollback() - { - checkState(!closed); - closed = true; - transactionManager.rollback(transactionHandle); - } - - @Override - public void close() - { - if (!closed) { - try { - getMetastore().testOnlyCheckIsReadOnly(); // transactions in this test with writes in it must explicitly commit or rollback - } - finally { - rollback(); - } - } - } - } - - @Test - public void testGetDatabaseNames() - { - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - List databases = metadata.listSchemaNames(newSession()); - assertThat(databases).contains(database); - } - } - - @Test - public void testGetTableNames() - { - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - List tables = metadata.listTables(newSession(), Optional.of(database)); - assertThat(tables).contains(tablePartitionFormat); - assertThat(tables).contains(tableUnpartitioned); - } - } - - @Test - public void testGetAllTableNames() - { - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - List tables = metadata.listTables(newSession(), Optional.empty()); - assertThat(tables).contains(tablePartitionFormat); - assertThat(tables).contains(tableUnpartitioned); - } - } - - @Test - public void testGetAllTableColumns() - { - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - Map> allColumns = listTableColumns(metadata, newSession(), new SchemaTablePrefix()); - assertThat(allColumns).containsKey(tablePartitionFormat); - assertThat(allColumns).containsKey(tableUnpartitioned); - } - } - - @Test - public void testGetAllTableColumnsInSchema() - { - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - Map> allColumns = listTableColumns(metadata, newSession(), new SchemaTablePrefix(database)); - assertThat(allColumns).containsKey(tablePartitionFormat); - assertThat(allColumns).containsKey(tableUnpartitioned); - } - } - - @Test - public void testListUnknownSchema() - { - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorSession session = newSession(); - assertThat(metadata.getTableHandle(session, new SchemaTableName(INVALID_DATABASE, INVALID_TABLE))).isNull(); - assertThat(metadata.listTables(session, Optional.of(INVALID_DATABASE))).isEqualTo(ImmutableList.of()); - assertThat(listTableColumns(metadata, session, new SchemaTablePrefix(INVALID_DATABASE, INVALID_TABLE))).isEqualTo(ImmutableMap.of()); - assertThat(metadata.listViews(session, Optional.of(INVALID_DATABASE))).isEqualTo(ImmutableList.of()); - assertThat(metadata.getViews(session, Optional.of(INVALID_DATABASE))).isEqualTo(ImmutableMap.of()); - assertThat(metadata.getView(session, new SchemaTableName(INVALID_DATABASE, INVALID_TABLE))).isEqualTo(Optional.empty()); - } - } - - @Test - public void testGetPartitions() - throws Exception - { - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorTableHandle tableHandle = getTableHandle(metadata, tablePartitionFormat); - tableHandle = applyFilter(metadata, tableHandle, Constraint.alwaysTrue()); - ConnectorTableProperties properties = metadata.getTableProperties(newSession(), tableHandle); - assertExpectedTableProperties(properties, tablePartitionFormatProperties); - assertExpectedPartitions(tableHandle, tablePartitionFormatPartitions); - } - } - - @Test - public void testGetPartitionsWithBindings() - { - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorTableHandle tableHandle = getTableHandle(metadata, tablePartitionFormat); - Constraint constraint = new Constraint(TupleDomain.withColumnDomains(ImmutableMap.of(intColumn, Domain.singleValue(BIGINT, 5L)))); - tableHandle = applyFilter(metadata, tableHandle, constraint); - ConnectorTableProperties properties = metadata.getTableProperties(newSession(), tableHandle); - assertExpectedTableProperties(properties, tablePartitionFormatProperties); - assertExpectedPartitions(tableHandle, tablePartitionFormatPartitions); - } - } - - @Test - public void testGetPartitionsWithFilter() - { - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorTableHandle tableHandle = getTableHandle(metadata, tablePartitionedWithNull); - - Domain varcharSomeValue = Domain.singleValue(VARCHAR, utf8Slice("abc")); - Domain varcharOnlyNull = Domain.onlyNull(VARCHAR); - Domain varcharNotNull = Domain.notNull(VARCHAR); - - Domain integerSomeValue = Domain.singleValue(INTEGER, 123L); - Domain integerOnlyNull = Domain.onlyNull(INTEGER); - Domain integerNotNull = Domain.notNull(INTEGER); - - // all - assertThat(getPartitionNamesByFilter(metadata, tableHandle, new Constraint(TupleDomain.all()))) - .containsOnly( - "p_string=__HIVE_DEFAULT_PARTITION__/p_integer=__HIVE_DEFAULT_PARTITION__", - "p_string=abc/p_integer=123", - "p_string=def/p_integer=456"); - - // is some value - assertThat(getPartitionNamesByFilter(metadata, tableHandle, pStringColumn, varcharSomeValue)) - .containsOnly("p_string=abc/p_integer=123"); - assertThat(getPartitionNamesByFilter(metadata, tableHandle, pIntegerColumn, integerSomeValue)) - .containsOnly("p_string=abc/p_integer=123"); - - // IS NULL - assertThat(getPartitionNamesByFilter(metadata, tableHandle, pStringColumn, varcharOnlyNull)) - .containsOnly("p_string=__HIVE_DEFAULT_PARTITION__/p_integer=__HIVE_DEFAULT_PARTITION__"); - assertThat(getPartitionNamesByFilter(metadata, tableHandle, pIntegerColumn, integerOnlyNull)) - .containsOnly("p_string=__HIVE_DEFAULT_PARTITION__/p_integer=__HIVE_DEFAULT_PARTITION__"); - - // IS NOT NULL - assertThat(getPartitionNamesByFilter(metadata, tableHandle, pStringColumn, varcharNotNull)) - .containsOnly("p_string=abc/p_integer=123", "p_string=def/p_integer=456"); - assertThat(getPartitionNamesByFilter(metadata, tableHandle, pIntegerColumn, integerNotNull)) - .containsOnly("p_string=abc/p_integer=123", "p_string=def/p_integer=456"); - - // IS NULL OR is some value - assertThat(getPartitionNamesByFilter(metadata, tableHandle, pStringColumn, varcharOnlyNull.union(varcharSomeValue))) - .containsOnly("p_string=__HIVE_DEFAULT_PARTITION__/p_integer=__HIVE_DEFAULT_PARTITION__", "p_string=abc/p_integer=123"); - assertThat(getPartitionNamesByFilter(metadata, tableHandle, pIntegerColumn, integerOnlyNull.union(integerSomeValue))) - .containsOnly("p_string=__HIVE_DEFAULT_PARTITION__/p_integer=__HIVE_DEFAULT_PARTITION__", "p_string=abc/p_integer=123"); - - // IS NOT NULL AND is NOT some value - assertThat(getPartitionNamesByFilter(metadata, tableHandle, pStringColumn, varcharSomeValue.complement().intersect(varcharNotNull))) - .containsOnly("p_string=def/p_integer=456"); - assertThat(getPartitionNamesByFilter(metadata, tableHandle, pIntegerColumn, integerSomeValue.complement().intersect(integerNotNull))) - .containsOnly("p_string=def/p_integer=456"); - - // IS NULL OR is NOT some value - assertThat(getPartitionNamesByFilter(metadata, tableHandle, pStringColumn, varcharSomeValue.complement())) - .containsOnly("p_string=__HIVE_DEFAULT_PARTITION__/p_integer=__HIVE_DEFAULT_PARTITION__", "p_string=def/p_integer=456"); - assertThat(getPartitionNamesByFilter(metadata, tableHandle, pIntegerColumn, integerSomeValue.complement())) - .containsOnly("p_string=__HIVE_DEFAULT_PARTITION__/p_integer=__HIVE_DEFAULT_PARTITION__", "p_string=def/p_integer=456"); - } - } - - private Set getPartitionNamesByFilter(ConnectorMetadata metadata, ConnectorTableHandle tableHandle, ColumnHandle columnHandle, Domain domain) - { - return getPartitionNamesByFilter(metadata, tableHandle, new Constraint(TupleDomain.withColumnDomains(ImmutableMap.of(columnHandle, domain)))); - } - - private Set getPartitionNamesByFilter(ConnectorMetadata metadata, ConnectorTableHandle tableHandle, Constraint constraint) - { - return applyFilter(metadata, tableHandle, constraint) - .getPartitions().orElseThrow(() -> new IllegalStateException("No partitions")) - .stream() - .map(HivePartition::getPartitionId) - .collect(toImmutableSet()); - } - - @Test - public void testMismatchSchemaTable() - throws Exception - { - for (HiveStorageFormat storageFormat : createTableFormats) { - // TODO: fix coercion for JSON - if (storageFormat == JSON) { - continue; - } - SchemaTableName temporaryMismatchSchemaTable = temporaryTable("mismatch_schema"); - try { - doTestMismatchSchemaTable( - temporaryMismatchSchemaTable, - storageFormat, - MISMATCH_SCHEMA_TABLE_BEFORE, - MISMATCH_SCHEMA_TABLE_DATA_BEFORE, - MISMATCH_SCHEMA_TABLE_AFTER, - MISMATCH_SCHEMA_TABLE_DATA_AFTER); - } - finally { - dropTable(temporaryMismatchSchemaTable); - } - } - } - - protected void doTestMismatchSchemaTable( - SchemaTableName schemaTableName, - HiveStorageFormat storageFormat, - List tableBefore, - MaterializedResult dataBefore, - List tableAfter, - MaterializedResult dataAfter) - throws Exception - { - String schemaName = schemaTableName.getSchemaName(); - String tableName = schemaTableName.getTableName(); - - doCreateEmptyTable(schemaTableName, storageFormat, tableBefore); - - // insert the data - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorTableHandle tableHandle = getTableHandle(metadata, schemaTableName); - - ConnectorInsertTableHandle insertTableHandle = metadata.beginInsert(session, tableHandle, ImmutableList.of(), NO_RETRIES); - ConnectorPageSink sink = pageSinkProvider.createPageSink(transaction.getTransactionHandle(), session, insertTableHandle, TESTING_PAGE_SINK_ID); - sink.appendPage(dataBefore.toPage()); - Collection fragments = getFutureValue(sink.finish()); - - metadata.finishInsert(session, insertTableHandle, fragments, ImmutableList.of()); - - transaction.commit(); - } - - // load the table and verify the data - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - ConnectorMetadata metadata = transaction.getMetadata(); - metadata.beginQuery(session); - ConnectorTableHandle tableHandle = getTableHandle(metadata, schemaTableName); - - List columnHandles = metadata.getColumnHandles(session, tableHandle).values().stream() - .filter(columnHandle -> !((HiveColumnHandle) columnHandle).isHidden()) - .collect(toList()); - - MaterializedResult result = readTable(transaction, tableHandle, columnHandles, session, TupleDomain.all(), OptionalInt.empty(), Optional.empty()); - assertEqualsIgnoreOrder(result.getMaterializedRows(), dataBefore.getMaterializedRows()); - transaction.commit(); - } - - // alter the table schema - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - PrincipalPrivileges principalPrivileges = testingPrincipalPrivilege(session); - Table oldTable = transaction.getMetastore().getTable(schemaName, tableName).get(); - List dataColumns = tableAfter.stream() - .filter(columnMetadata -> !columnMetadata.getName().equals("ds")) - .map(columnMetadata -> new Column(columnMetadata.getName(), toHiveType(columnMetadata.getType()), Optional.empty(), Map.of())) - .collect(toList()); - Table.Builder newTable = Table.builder(oldTable) - .setDataColumns(dataColumns); - - transaction.getMetastore().replaceTable(schemaName, tableName, newTable.build(), principalPrivileges); - - transaction.commit(); - } - - // load the altered table and verify the data - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - ConnectorMetadata metadata = transaction.getMetadata(); - metadata.beginQuery(session); - ConnectorTableHandle tableHandle = getTableHandle(metadata, schemaTableName); - List columnHandles = metadata.getColumnHandles(session, tableHandle).values().stream() - .filter(columnHandle -> !((HiveColumnHandle) columnHandle).isHidden()) - .collect(toList()); - - MaterializedResult result = readTable(transaction, tableHandle, columnHandles, session, TupleDomain.all(), OptionalInt.empty(), Optional.empty()); - assertEqualsIgnoreOrder(result.getMaterializedRows(), dataAfter.getMaterializedRows()); - - transaction.commit(); - } - - // insertions to the partitions with type mismatches should fail - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorTableHandle tableHandle = getTableHandle(metadata, schemaTableName); - - ConnectorInsertTableHandle insertTableHandle = metadata.beginInsert(session, tableHandle, ImmutableList.of(), NO_RETRIES); - ConnectorPageSink sink = pageSinkProvider.createPageSink(transaction.getTransactionHandle(), session, insertTableHandle, TESTING_PAGE_SINK_ID); - sink.appendPage(dataAfter.toPage()); - Collection fragments = getFutureValue(sink.finish()); - - metadata.finishInsert(session, insertTableHandle, fragments, ImmutableList.of()); - - transaction.commit(); - - fail("expected exception"); - } - catch (TrinoException e) { - // expected - assertThat(e.getErrorCode()).isEqualTo(HIVE_PARTITION_SCHEMA_MISMATCH.toErrorCode()); - } - } - - protected void assertExpectedTableProperties(ConnectorTableProperties actualProperties, ConnectorTableProperties expectedProperties) - { - assertThat(actualProperties.getPredicate()).isEqualTo(expectedProperties.getPredicate()); - assertThat(actualProperties.getDiscretePredicates().isPresent()).isEqualTo(expectedProperties.getDiscretePredicates().isPresent()); - actualProperties.getDiscretePredicates().ifPresent(actual -> { - DiscretePredicates expected = expectedProperties.getDiscretePredicates().get(); - assertThat(actual.getColumns()).isEqualTo(expected.getColumns()); - assertEqualsIgnoreOrder(actual.getPredicates(), expected.getPredicates()); - }); - assertThat(actualProperties.getLocalProperties()).isEqualTo(expectedProperties.getLocalProperties()); - } - - protected void assertExpectedPartitions(ConnectorTableHandle table, Iterable expectedPartitions) - { - Iterable actualPartitions = ((HiveTableHandle) table).getPartitions().orElseThrow(AssertionError::new); - Map actualById = uniqueIndex(actualPartitions, HivePartition::getPartitionId); - Map expectedById = uniqueIndex(expectedPartitions, HivePartition::getPartitionId); - - assertThat(actualById).isEqualTo(expectedById); - - // HivePartition.equals doesn't compare all the fields, so let's check them - for (Map.Entry expected : expectedById.entrySet()) { - HivePartition actualPartition = actualById.get(expected.getKey()); - HivePartition expectedPartition = expected.getValue(); - assertThat(actualPartition.getPartitionId()).isEqualTo(expectedPartition.getPartitionId()); - assertThat(actualPartition.getKeys()).isEqualTo(expectedPartition.getKeys()); - assertThat(actualPartition.getTableName()).isEqualTo(expectedPartition.getTableName()); - } - } - - @Test - public void testGetPartitionNamesUnpartitioned() - { - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableUnpartitioned); - tableHandle = applyFilter(metadata, tableHandle, Constraint.alwaysTrue()); - ConnectorTableProperties properties = metadata.getTableProperties(newSession(), tableHandle); - assertExpectedTableProperties(properties, new ConnectorTableProperties()); - assertExpectedPartitions(tableHandle, tableUnpartitionedPartitions); - } - } - - @Test - public void testGetTableSchemaPartitionFormat() - { - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorTableMetadata tableMetadata = metadata.getTableMetadata(newSession(), getTableHandle(metadata, tablePartitionFormat)); - Map map = uniqueIndex(tableMetadata.getColumns(), ColumnMetadata::getName); - - assertPrimitiveField(map, "t_string", createUnboundedVarcharType(), false); - assertPrimitiveField(map, "t_tinyint", TINYINT, false); - assertPrimitiveField(map, "t_smallint", SMALLINT, false); - assertPrimitiveField(map, "t_int", INTEGER, false); - assertPrimitiveField(map, "t_bigint", BIGINT, false); - assertPrimitiveField(map, "t_float", REAL, false); - assertPrimitiveField(map, "t_double", DOUBLE, false); - assertPrimitiveField(map, "t_boolean", BOOLEAN, false); - assertPrimitiveField(map, "ds", createUnboundedVarcharType(), true); - assertPrimitiveField(map, "file_format", createUnboundedVarcharType(), true); - assertPrimitiveField(map, "dummy", INTEGER, true); - } - } - - @Test - public void testGetTableSchemaUnpartitioned() - { - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableUnpartitioned); - ConnectorTableMetadata tableMetadata = metadata.getTableMetadata(newSession(), tableHandle); - Map map = uniqueIndex(tableMetadata.getColumns(), ColumnMetadata::getName); - - assertPrimitiveField(map, "t_string", createUnboundedVarcharType(), false); - assertPrimitiveField(map, "t_tinyint", TINYINT, false); - } - } - - @Test - public void testGetTableSchemaOffline() - { - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - Map> columns = listTableColumns(metadata, newSession(), tableOffline.toSchemaTablePrefix()); - assertThat(columns.size()).isEqualTo(1); - Map map = uniqueIndex(getOnlyElement(columns.values()), ColumnMetadata::getName); - - assertPrimitiveField(map, "t_string", createUnboundedVarcharType(), false); - } - } - - @Test - public void testGetTableSchemaNotReadablePartition() - { - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableNotReadable); - ConnectorTableMetadata tableMetadata = metadata.getTableMetadata(newSession(), tableHandle); - Map map = uniqueIndex(tableMetadata.getColumns(), ColumnMetadata::getName); - - assertPrimitiveField(map, "t_string", createUnboundedVarcharType(), false); - } - } - - @Test - public void testGetTableSchemaException() - { - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - assertThat(metadata.getTableHandle(newSession(), invalidTable)).isNull(); - } - } - - @Test - public void testGetTableStatsBucketedStringInt() - { - assertTableStatsComputed( - tableBucketedStringInt, - ImmutableSet.of( - "t_bigint", - "t_boolean", - "t_double", - "t_float", - "t_int", - "t_smallint", - "t_string", - "t_tinyint", - "ds")); - } - - @Test - public void testGetTableStatsUnpartitioned() - { - assertTableStatsComputed( - tableUnpartitioned, - ImmutableSet.of("t_string", "t_tinyint")); - } - - private void assertTableStatsComputed( - SchemaTableName tableName, - Set expectedColumnStatsColumns) - { - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorSession session = newSession(); - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); - - // first check if table handle with only one projected column will return this column stats - String firstColumnName = expectedColumnStatsColumns.iterator().next(); - verifyTableStatisticsWithColumns(metadata, session, applyProjection(metadata, session, tableHandle, firstColumnName), ImmutableSet.of(firstColumnName)); - - verifyTableStatisticsWithColumns(metadata, session, tableHandle, expectedColumnStatsColumns); - } - } - - private static ConnectorTableHandle applyProjection(ConnectorMetadata metadata, ConnectorSession session, ConnectorTableHandle tableHandle, String columnName) - { - Map columnHandles = metadata.getColumnHandles(session, tableHandle); - HiveColumnHandle firstColumn = (HiveColumnHandle) columnHandles.get(columnName); - return metadata.applyProjection( - session, - tableHandle, - ImmutableList.of(new Variable("c1", firstColumn.getBaseType())), - ImmutableMap.of("c1", firstColumn)) - .orElseThrow() - .getHandle(); - } - - private static void verifyTableStatisticsWithColumns( - ConnectorMetadata metadata, - ConnectorSession session, - ConnectorTableHandle tableHandle, - Set expectedColumnStatsColumns) - { - TableStatistics tableStatistics = metadata.getTableStatistics(session, tableHandle); - - assertThat(tableStatistics.getRowCount().isUnknown()) - .describedAs("row count is unknown") - .isFalse(); - - Map columnsStatistics = tableStatistics - .getColumnStatistics() - .entrySet() - .stream() - .collect( - toImmutableMap( - entry -> ((HiveColumnHandle) entry.getKey()).getName(), - Map.Entry::getValue)); - - assertThat(columnsStatistics.keySet()) - .describedAs("columns with statistics") - .isEqualTo(expectedColumnStatsColumns); - - Map columnHandles = metadata.getColumnHandles(session, tableHandle); - columnsStatistics.forEach((columnName, columnStatistics) -> { - ColumnHandle columnHandle = columnHandles.get(columnName); - Type columnType = metadata.getColumnMetadata(session, tableHandle, columnHandle).getType(); - - assertThat(columnStatistics.getNullsFraction().isUnknown()) - .describedAs("unknown nulls fraction for " + columnName) - .isFalse(); - - assertThat(columnStatistics.getDistinctValuesCount().isUnknown()) - .describedAs("unknown distinct values count for " + columnName) - .isFalse(); - - if (columnType instanceof VarcharType) { - assertThat(columnStatistics.getDataSize().isUnknown()) - .describedAs("unknown data size for " + columnName) - .isFalse(); - } - else { - assertThat(columnStatistics.getDataSize().isUnknown()) - .describedAs("unknown data size for" + columnName) - .isTrue(); - } - }); - } - - @Test - public void testGetPartitionSplitsBatch() - { - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorSession session = newSession(); - metadata.beginQuery(session); - - ConnectorTableHandle tableHandle = getTableHandle(metadata, tablePartitionFormat); - ConnectorSplitSource splitSource = getSplits(splitManager, transaction, session, tableHandle); - - assertThat(getSplitCount(splitSource)).isEqualTo(tablePartitionFormatPartitions.size()); - } - } - - @Test - public void testGetPartitionSplitsBatchUnpartitioned() - { - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorSession session = newSession(); - metadata.beginQuery(session); - - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableUnpartitioned); - ConnectorSplitSource splitSource = getSplits(splitManager, transaction, session, tableHandle); - - assertThat(getSplitCount(splitSource)).isEqualTo(1); - } - } - - @Test - public void testPerTransactionDirectoryListerCache() - throws Exception - { - long initListCount = countingDirectoryLister.getListCount(); - SchemaTableName tableName = temporaryTable("per_transaction_listing_cache_test"); - List columns = ImmutableList.of(new Column("test", HIVE_STRING, Optional.empty(), Map.of())); - createEmptyTable(tableName, ORC, columns, ImmutableList.of()); - try { - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorSession session = newSession(); - metadata.beginQuery(session); - - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); - getAllSplits(getSplits(splitManager, transaction, session, tableHandle)); - - // directory should be listed initially - assertThat(countingDirectoryLister.getListCount()).isEqualTo(initListCount + 1); - - // directory content should be cached - getAllSplits(getSplits(splitManager, transaction, session, tableHandle)); - assertThat(countingDirectoryLister.getListCount()).isEqualTo(initListCount + 1); - } - - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorSession session = newSession(); - metadata.beginQuery(session); - - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); - getAllSplits(getSplits(splitManager, transaction, session, tableHandle)); - - // directory should be listed again in new transaction - assertThat(countingDirectoryLister.getListCount()).isEqualTo(initListCount + 2); - } - } - finally { - dropTable(tableName); - } - } - - @Test - public void testGetPartitionSplitsBatchInvalidTable() - { - assertThatThrownBy(() -> { - try (Transaction transaction = newTransaction()) { - getSplits(splitManager, transaction, newSession(), invalidTableHandle); - } - }).isInstanceOf(TableNotFoundException.class); - } - - @Test - public void testGetPartitionTableOffline() - { - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - try { - getTableHandle(metadata, tableOffline); - fail("expected TableOfflineException"); - } - catch (TableOfflineException e) { - assertThat(e.getTableName()).isEqualTo(tableOffline); - } - } - } - - @Test - public void testGetPartitionSplitsTableNotReadablePartition() - { - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorSession session = newSession(); - - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableNotReadable); - assertThat(tableHandle).isNotNull(); - - try { - getSplitCount(getSplits(splitManager, transaction, session, tableHandle)); - fail("Expected HiveNotReadableException"); - } - catch (HiveNotReadableException e) { - assertThat(e).hasMessageMatching("Table '.*\\.trino_test_not_readable' is not readable: reason for not readable"); - assertThat(e.getTableName()).isEqualTo(tableNotReadable); - assertThat(e.getPartition()).isEqualTo(Optional.empty()); - } - } - } - - @Test - public void testBucketedTableStringInt() - throws Exception - { - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorSession session = newSession(); - metadata.beginQuery(session); - - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableBucketedStringInt); - List columnHandles = ImmutableList.copyOf(metadata.getColumnHandles(session, tableHandle).values()); - Map columnIndex = indexColumns(columnHandles); - - assertTableIsBucketed(tableHandle, transaction, session); - - String testString = "test"; - Integer testInt = 13; - Short testSmallint = 12; - - // Reverse the order of bindings as compared to bucketing order - ImmutableMap bindings = ImmutableMap.builder() - .put(columnHandles.get(columnIndex.get("t_int")), NullableValue.of(INTEGER, (long) testInt)) - .put(columnHandles.get(columnIndex.get("t_string")), NullableValue.of(createUnboundedVarcharType(), utf8Slice(testString))) - .put(columnHandles.get(columnIndex.get("t_smallint")), NullableValue.of(SMALLINT, (long) testSmallint)) - .buildOrThrow(); - - MaterializedResult result = readTable(transaction, tableHandle, columnHandles, session, TupleDomain.fromFixedValues(bindings), OptionalInt.of(1), Optional.empty()); - - boolean rowFound = false; - for (MaterializedRow row : result) { - if (testString.equals(row.getField(columnIndex.get("t_string"))) && - testInt.equals(row.getField(columnIndex.get("t_int"))) && - testSmallint.equals(row.getField(columnIndex.get("t_smallint")))) { - rowFound = true; - } - } - assertThat(rowFound).isTrue(); - } - } - - @SuppressWarnings("ConstantConditions") - @Test - public void testBucketedTableBigintBoolean() - throws Exception - { - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorSession session = newSession(); - metadata.beginQuery(session); - - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableBucketedBigintBoolean); - List columnHandles = ImmutableList.copyOf(metadata.getColumnHandles(session, tableHandle).values()); - Map columnIndex = indexColumns(columnHandles); - - assertTableIsBucketed(tableHandle, transaction, session); - ConnectorTableProperties properties = metadata.getTableProperties( - newSession(ImmutableMap.of("propagate_table_scan_sorting_properties", true)), - tableHandle); - // trino_test_bucketed_by_bigint_boolean does not define sorting, therefore local properties is empty - assertThat(properties.getLocalProperties().isEmpty()).isTrue(); - assertThat(metadata.getTableProperties(newSession(), tableHandle).getLocalProperties().isEmpty()).isTrue(); - - String testString = "test"; - Long testBigint = 89L; - Boolean testBoolean = true; - - ImmutableMap bindings = ImmutableMap.builder() - .put(columnHandles.get(columnIndex.get("t_string")), NullableValue.of(createUnboundedVarcharType(), utf8Slice(testString))) - .put(columnHandles.get(columnIndex.get("t_bigint")), NullableValue.of(BIGINT, testBigint)) - .put(columnHandles.get(columnIndex.get("t_boolean")), NullableValue.of(BOOLEAN, testBoolean)) - .buildOrThrow(); - - MaterializedResult result = readTable(transaction, tableHandle, columnHandles, session, TupleDomain.fromFixedValues(bindings), OptionalInt.of(1), Optional.empty()); - - boolean rowFound = false; - for (MaterializedRow row : result) { - if (testString.equals(row.getField(columnIndex.get("t_string"))) && - testBigint.equals(row.getField(columnIndex.get("t_bigint"))) && - testBoolean.equals(row.getField(columnIndex.get("t_boolean")))) { - rowFound = true; - break; - } - } - assertThat(rowFound).isTrue(); - } - } - - @Test - public void testBucketedTableDoubleFloat() - throws Exception - { - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorSession session = newSession(); - metadata.beginQuery(session); - - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableBucketedDoubleFloat); - List columnHandles = ImmutableList.copyOf(metadata.getColumnHandles(session, tableHandle).values()); - Map columnIndex = indexColumns(columnHandles); - - assertTableIsBucketed(tableHandle, transaction, session); - - float testFloatValue = 87.1f; - double testDoubleValue = 88.2; - - ImmutableMap bindings = ImmutableMap.builder() - .put(columnHandles.get(columnIndex.get("t_float")), NullableValue.of(REAL, (long) floatToRawIntBits(testFloatValue))) - .put(columnHandles.get(columnIndex.get("t_double")), NullableValue.of(DOUBLE, testDoubleValue)) - .buildOrThrow(); - - MaterializedResult result = readTable(transaction, tableHandle, columnHandles, session, TupleDomain.fromFixedValues(bindings), OptionalInt.of(1), Optional.empty()); - assertThat(result).anyMatch(row -> testFloatValue == (float) row.getField(columnIndex.get("t_float")) - && testDoubleValue == (double) row.getField(columnIndex.get("t_double"))); - } - } - - @Test - public void testBucketedTableEvolutionWithDifferentReadBucketCount() - throws Exception - { - for (HiveStorageFormat storageFormat : createTableFormats) { - SchemaTableName temporaryBucketEvolutionTable = temporaryTable("bucket_evolution"); - try { - doTestBucketedTableEvolutionWithDifferentReadCount(storageFormat, temporaryBucketEvolutionTable); - } - finally { - dropTable(temporaryBucketEvolutionTable); - } - } - } - - private void doTestBucketedTableEvolutionWithDifferentReadCount(HiveStorageFormat storageFormat, SchemaTableName tableName) - throws Exception - { - int rowCount = 100; - int bucketCount = 16; - - // Produce a table with a partition with bucket count different but compatible with the table bucket count - createEmptyTable( - tableName, - storageFormat, - ImmutableList.of( - new Column("id", HIVE_LONG, Optional.empty(), Map.of()), - new Column("name", HIVE_STRING, Optional.empty(), Map.of())), - ImmutableList.of(new Column("pk", HIVE_STRING, Optional.empty(), Map.of())), - Optional.of(new HiveBucketProperty(ImmutableList.of("id"), BUCKETING_V1, 4, ImmutableList.of()))); - // write a 4-bucket partition - MaterializedResult.Builder bucket8Builder = MaterializedResult.resultBuilder(SESSION, BIGINT, VARCHAR, VARCHAR); - IntStream.range(0, rowCount).forEach(i -> bucket8Builder.row((long) i, String.valueOf(i), "four")); - insertData(tableName, bucket8Builder.build()); - - // Alter the bucket count to 16 - alterBucketProperty(tableName, Optional.of(new HiveBucketProperty(ImmutableList.of("id"), BUCKETING_V1, bucketCount, ImmutableList.of()))); - - MaterializedResult result; - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorSession session = newSession(); - metadata.beginQuery(session); - - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); - - // read entire table - List columnHandles = ImmutableList.copyOf(metadata.getColumnHandles(session, tableHandle).values()); - - List splits = getAllSplits(getSplits(splitManager, transaction, session, tableHandle)); - assertThat(splits.size()).isEqualTo(16); - - ImmutableList.Builder allRows = ImmutableList.builder(); - for (ConnectorSplit split : splits) { - try (ConnectorPageSource pageSource = pageSourceProvider.createPageSource(transaction.getTransactionHandle(), session, split, tableHandle, columnHandles, DynamicFilter.EMPTY)) { - MaterializedResult intermediateResult = materializeSourceDataStream(session, pageSource, getTypes(columnHandles)); - allRows.addAll(intermediateResult.getMaterializedRows()); - } - } - result = new MaterializedResult(allRows.build(), getTypes(columnHandles)); - - assertThat(result.getRowCount()).isEqualTo(rowCount); - - Map columnIndex = indexColumns(columnHandles); - int nameColumnIndex = columnIndex.get("name"); - int bucketColumnIndex = columnIndex.get(BUCKET_COLUMN_NAME); - for (MaterializedRow row : result.getMaterializedRows()) { - String name = (String) row.getField(nameColumnIndex); - int bucket = (int) row.getField(bucketColumnIndex); - - assertThat(bucket).isEqualTo(Integer.parseInt(name) % bucketCount); - } - } - } - - @Test - public void testBucketedTableEvolution() - throws Exception - { - for (HiveStorageFormat storageFormat : createTableFormats) { - SchemaTableName temporaryBucketEvolutionTable = temporaryTable("bucket_evolution"); - try { - doTestBucketedTableEvolution(storageFormat, temporaryBucketEvolutionTable); - } - finally { - dropTable(temporaryBucketEvolutionTable); - } - } - } - - private void doTestBucketedTableEvolution(HiveStorageFormat storageFormat, SchemaTableName tableName) - throws Exception - { - int rowCount = 100; - - // - // Produce a table with 8 buckets. - // The table has 3 partitions of 3 different bucket count (4, 8, 16). - createEmptyTable( - tableName, - storageFormat, - ImmutableList.of( - new Column("id", HIVE_LONG, Optional.empty(), Map.of()), - new Column("name", HIVE_STRING, Optional.empty(), Map.of())), - ImmutableList.of(new Column("pk", HIVE_STRING, Optional.empty(), Map.of())), - Optional.of(new HiveBucketProperty(ImmutableList.of("id"), BUCKETING_V1, 4, ImmutableList.of()))); - // write a 4-bucket partition - MaterializedResult.Builder bucket4Builder = MaterializedResult.resultBuilder(SESSION, BIGINT, VARCHAR, VARCHAR); - IntStream.range(0, rowCount).forEach(i -> bucket4Builder.row((long) i, String.valueOf(i), "four")); - insertData(tableName, bucket4Builder.build()); - // write a 16-bucket partition - alterBucketProperty(tableName, Optional.of(new HiveBucketProperty(ImmutableList.of("id"), BUCKETING_V1, 16, ImmutableList.of()))); - MaterializedResult.Builder bucket16Builder = MaterializedResult.resultBuilder(SESSION, BIGINT, VARCHAR, VARCHAR); - IntStream.range(0, rowCount).forEach(i -> bucket16Builder.row((long) i, String.valueOf(i), "sixteen")); - insertData(tableName, bucket16Builder.build()); - // write an 8-bucket partition - alterBucketProperty(tableName, Optional.of(new HiveBucketProperty(ImmutableList.of("id"), BUCKETING_V1, 8, ImmutableList.of()))); - MaterializedResult.Builder bucket8Builder = MaterializedResult.resultBuilder(SESSION, BIGINT, VARCHAR, VARCHAR); - IntStream.range(0, rowCount).forEach(i -> bucket8Builder.row((long) i, String.valueOf(i), "eight")); - insertData(tableName, bucket8Builder.build()); - - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorSession session = newSession(); - metadata.beginQuery(session); - - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); - - // read entire table - List columnHandles = ImmutableList.copyOf(metadata.getColumnHandles(session, tableHandle).values()); - MaterializedResult result = readTable( - transaction, - tableHandle, - columnHandles, - session, - TupleDomain.all(), - OptionalInt.empty(), - Optional.empty()); - assertBucketTableEvolutionResult(result, columnHandles, ImmutableSet.of(0, 1, 2, 3, 4, 5, 6, 7), rowCount); - - // read single bucket (table/logical bucket) - result = readTable( - transaction, - tableHandle, - columnHandles, - session, - TupleDomain.fromFixedValues(ImmutableMap.of(bucketColumnHandle(), NullableValue.of(INTEGER, 6L))), - OptionalInt.empty(), - Optional.empty()); - assertBucketTableEvolutionResult(result, columnHandles, ImmutableSet.of(6), rowCount); - - // read single bucket, without selecting the bucketing column (i.e. id column) - columnHandles = metadata.getColumnHandles(session, tableHandle).values().stream() - .filter(columnHandle -> !"id".equals(((HiveColumnHandle) columnHandle).getName())) - .collect(toImmutableList()); - result = readTable( - transaction, - tableHandle, - columnHandles, - session, - TupleDomain.fromFixedValues(ImmutableMap.of(bucketColumnHandle(), NullableValue.of(INTEGER, 6L))), - OptionalInt.empty(), - Optional.empty()); - assertBucketTableEvolutionResult(result, columnHandles, ImmutableSet.of(6), rowCount); - } - } - - private static void assertBucketTableEvolutionResult(MaterializedResult result, List columnHandles, Set bucketIds, int rowCount) - { - // Assert that only elements in the specified bucket shows up, and each element shows up 3 times. - int bucketCount = 8; - Set expectedIds = LongStream.range(0, rowCount) - .filter(x -> bucketIds.contains(toIntExact(x % bucketCount))) - .boxed() - .collect(toImmutableSet()); - - // assert that content from all three buckets are the same - Map columnIndex = indexColumns(columnHandles); - OptionalInt idColumnIndex = columnIndex.containsKey("id") ? OptionalInt.of(columnIndex.get("id")) : OptionalInt.empty(); - int nameColumnIndex = columnIndex.get("name"); - int bucketColumnIndex = columnIndex.get(BUCKET_COLUMN_NAME); - Map idCount = new HashMap<>(); - for (MaterializedRow row : result.getMaterializedRows()) { - String name = (String) row.getField(nameColumnIndex); - int bucket = (int) row.getField(bucketColumnIndex); - idCount.compute(Long.parseLong(name), (key, oldValue) -> oldValue == null ? 1 : oldValue + 1); - assertThat(bucket).isEqualTo(Integer.parseInt(name) % bucketCount); - if (idColumnIndex.isPresent()) { - long id = (long) row.getField(idColumnIndex.getAsInt()); - assertThat(Integer.parseInt(name)).isEqualTo(id); - } - } - assertThat((int) idCount.values().stream() - .distinct() - .collect(onlyElement())).isEqualTo(3); - assertThat(idCount.keySet()).isEqualTo(expectedIds); - } - - @Test - public void testBucketedSortedTableEvolution() - throws Exception - { - SchemaTableName temporaryTable = temporaryTable("test_bucket_sorting_evolution"); - try { - doTestBucketedSortedTableEvolution(temporaryTable); - } - finally { - dropTable(temporaryTable); - } - } - - private void doTestBucketedSortedTableEvolution(SchemaTableName tableName) - throws Exception - { - int rowCount = 100; - // Create table and populate it with 3 partitions with different sort orders but same bucketing - createEmptyTable( - tableName, - ORC, - ImmutableList.of( - new Column("id", HIVE_LONG, Optional.empty(), Map.of()), - new Column("name", HIVE_STRING, Optional.empty(), Map.of())), - ImmutableList.of(new Column("pk", HIVE_STRING, Optional.empty(), Map.of())), - Optional.of(new HiveBucketProperty( - ImmutableList.of("id"), - BUCKETING_V1, - 4, - ImmutableList.of(new SortingColumn("id", ASCENDING), new SortingColumn("name", ASCENDING))))); - // write a 4-bucket partition sorted by id, name - MaterializedResult.Builder sortedByIdNameBuilder = MaterializedResult.resultBuilder(SESSION, BIGINT, VARCHAR, VARCHAR); - IntStream.range(0, rowCount).forEach(i -> sortedByIdNameBuilder.row((long) i, String.valueOf(i), "sorted_by_id_name")); - insertData(tableName, sortedByIdNameBuilder.build()); - - // write a 4-bucket partition sorted by name - alterBucketProperty(tableName, Optional.of(new HiveBucketProperty( - ImmutableList.of("id"), - BUCKETING_V1, - 4, - ImmutableList.of(new SortingColumn("name", ASCENDING))))); - MaterializedResult.Builder sortedByNameBuilder = MaterializedResult.resultBuilder(SESSION, BIGINT, VARCHAR, VARCHAR); - IntStream.range(0, rowCount).forEach(i -> sortedByNameBuilder.row((long) i, String.valueOf(i), "sorted_by_name")); - insertData(tableName, sortedByNameBuilder.build()); - - // write a 4-bucket partition sorted by id - alterBucketProperty(tableName, Optional.of(new HiveBucketProperty( - ImmutableList.of("id"), - BUCKETING_V1, - 4, - ImmutableList.of(new SortingColumn("id", ASCENDING))))); - MaterializedResult.Builder sortedByIdBuilder = MaterializedResult.resultBuilder(SESSION, BIGINT, VARCHAR, VARCHAR); - IntStream.range(0, rowCount).forEach(i -> sortedByIdBuilder.row((long) i, String.valueOf(i), "sorted_by_id")); - insertData(tableName, sortedByIdBuilder.build()); - - ConnectorTableHandle tableHandle; - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorSession session = newSession(); - metadata.beginQuery(session); - tableHandle = getTableHandle(metadata, tableName); - - // read entire table - List columnHandles = ImmutableList.copyOf(metadata.getColumnHandles(session, tableHandle).values()); - MaterializedResult result = readTable(transaction, tableHandle, columnHandles, session, TupleDomain.all(), OptionalInt.empty(), Optional.empty()); - assertThat(result.getRowCount()).isEqualTo(300); - } - - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorSession session = newSession(ImmutableMap.of("propagate_table_scan_sorting_properties", true)); - metadata.beginQuery(session); - Map columnHandles = metadata.getColumnHandles(session, tableHandle); - // verify local sorting property - ConnectorTableProperties properties = metadata.getTableProperties(session, tableHandle); - assertThat(properties.getLocalProperties()).isEqualTo(ImmutableList.of( - new SortingProperty<>(columnHandles.get("id"), ASC_NULLS_FIRST))); - - // read on a entire table should fail with exception - assertThatThrownBy(() -> readTable(transaction, tableHandle, ImmutableList.copyOf(columnHandles.values()), session, TupleDomain.all(), OptionalInt.empty(), Optional.empty())) - .isInstanceOf(TrinoException.class) - .hasMessage("Hive table (%s) sorting by [id] is not compatible with partition (pk=sorted_by_name) sorting by [name]." + - " This restriction can be avoided by disabling propagate_table_scan_sorting_properties.", tableName); - - // read only the partitions with sorting that is compatible to table sorting - MaterializedResult result = readTable( - transaction, - tableHandle, - ImmutableList.copyOf(columnHandles.values()), - session, - TupleDomain.withColumnDomains(ImmutableMap.of( - columnHandles.get("pk"), - Domain.create(ValueSet.of(VARCHAR, utf8Slice("sorted_by_id_name"), utf8Slice("sorted_by_id")), false))), - OptionalInt.empty(), - Optional.empty()); - assertThat(result.getRowCount()).isEqualTo(200); - } - } - - @Test - public void testBucketedTableValidation() - throws Exception - { - for (HiveStorageFormat storageFormat : createTableFormats) { - SchemaTableName table = temporaryTable("bucket_validation"); - try { - doTestBucketedTableValidation(storageFormat, table); - } - finally { - dropTable(table); - } - } - } - - private void doTestBucketedTableValidation(HiveStorageFormat storageFormat, SchemaTableName tableName) - throws Exception - { - prepareInvalidBuckets(storageFormat, tableName); - - // read succeeds when validation is disabled - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorSession session = newSession(ImmutableMap.of("validate_bucketing", false)); - metadata.beginQuery(session); - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); - List columnHandles = filterNonHiddenColumnHandles(metadata.getColumnHandles(session, tableHandle).values()); - MaterializedResult result = readTable(transaction, tableHandle, columnHandles, session, TupleDomain.all(), OptionalInt.empty(), Optional.of(storageFormat)); - assertThat(result.getRowCount()).isEqualTo(87); // fewer rows due to deleted file - } - - // read fails due to validation failure - assertReadFailsWithMessageMatching(storageFormat, tableName, "Hive table is corrupt\\. File '.*/000002_0_.*' is for bucket 2, but contains a row for bucket 5."); - } - - private void prepareInvalidBuckets(HiveStorageFormat storageFormat, SchemaTableName tableName) - throws Exception - { - createEmptyTable( - tableName, - storageFormat, - ImmutableList.of( - new Column("id", HIVE_LONG, Optional.empty(), Map.of()), - new Column("name", HIVE_STRING, Optional.empty(), Map.of())), - ImmutableList.of(), - Optional.of(new HiveBucketProperty(ImmutableList.of("id"), BUCKETING_V1, 8, ImmutableList.of()))); - - MaterializedResult.Builder dataBuilder = MaterializedResult.resultBuilder(SESSION, BIGINT, VARCHAR); - for (long id = 0; id < 100; id++) { - dataBuilder.row(id, String.valueOf(id)); - } - insertData(tableName, dataBuilder.build()); - - try (Transaction transaction = newTransaction()) { - Set files = listAllDataFiles(transaction, tableName.getSchemaName(), tableName.getTableName()); - - Path bucket2 = files.stream() - .map(Path::new) - .filter(path -> path.getName().startsWith("000002_0_")) - .collect(onlyElement()); - - Path bucket5 = files.stream() - .map(Path::new) - .filter(path -> path.getName().startsWith("000005_0_")) - .collect(onlyElement()); - - HdfsContext context = new HdfsContext(newSession()); - FileSystem fileSystem = hdfsEnvironment.getFileSystem(context, bucket2); - fileSystem.delete(bucket2, false); - fileSystem.rename(bucket5, bucket2); - } - } - - protected void assertReadFailsWithMessageMatching(HiveStorageFormat storageFormat, SchemaTableName tableName, String regex) - { - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorSession session = newSession(); - metadata.beginQuery(session); - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); - List columnHandles = filterNonHiddenColumnHandles(metadata.getColumnHandles(session, tableHandle).values()); - assertTrinoExceptionThrownBy( - () -> readTable(transaction, tableHandle, columnHandles, session, TupleDomain.all(), OptionalInt.empty(), Optional.of(storageFormat))) - .hasErrorCode(HIVE_INVALID_BUCKET_FILES) - .hasMessageMatching(regex); - } - } - - private void assertTableIsBucketed(ConnectorTableHandle tableHandle, Transaction transaction, ConnectorSession session) - { - // the bucketed test tables should have ~32 splits - List splits = getAllSplits(tableHandle, transaction, session); - assertThat(splits.size()).as("splits.size()") - .isBetween(31, 32); - - // verify all paths are unique - Set paths = new HashSet<>(); - for (ConnectorSplit split : splits) { - assertThat(paths.add(((HiveSplit) split).getPath())).isTrue(); - } - } - - @Test - public void testGetRecords() - throws Exception - { - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorSession session = newSession(); - metadata.beginQuery(session); - - ConnectorTableHandle tableHandle = getTableHandle(metadata, tablePartitionFormat); - ConnectorTableMetadata tableMetadata = metadata.getTableMetadata(session, tableHandle); - List columnHandles = ImmutableList.copyOf(metadata.getColumnHandles(session, tableHandle).values()); - Map columnIndex = indexColumns(columnHandles); - - List splits = getAllSplits(tableHandle, transaction, session); - assertThat(splits.size()).isEqualTo(tablePartitionFormatPartitions.size()); - - for (ConnectorSplit split : splits) { - HiveSplit hiveSplit = (HiveSplit) split; - - List partitionKeys = hiveSplit.getPartitionKeys(); - String ds = partitionKeys.get(0).getValue(); - String fileFormat = partitionKeys.get(1).getValue(); - HiveStorageFormat fileType = HiveStorageFormat.valueOf(fileFormat.toUpperCase(ENGLISH)); - int dummyPartition = Integer.parseInt(partitionKeys.get(2).getValue()); - - long rowNumber = 0; - long completedBytes = 0; - try (ConnectorPageSource pageSource = pageSourceProvider.createPageSource(transaction.getTransactionHandle(), session, hiveSplit, tableHandle, columnHandles, DynamicFilter.EMPTY)) { - MaterializedResult result = materializeSourceDataStream(session, pageSource, getTypes(columnHandles)); - - assertPageSourceType(pageSource, fileType); - - for (MaterializedRow row : result) { - try { - assertValueTypes(row, tableMetadata.getColumns()); - } - catch (RuntimeException e) { - throw new RuntimeException("row " + rowNumber, e); - } - - rowNumber++; - Object value; - - value = row.getField(columnIndex.get("t_string")); - if (rowNumber % 19 == 0) { - assertThat(value).isNull(); - } - else if (rowNumber % 19 == 1) { - assertThat(value).isEqualTo(""); - } - else { - assertThat(value).isEqualTo("test"); - } - - assertThat(row.getField(columnIndex.get("t_tinyint"))).isEqualTo((byte) (1 + rowNumber)); - assertThat(row.getField(columnIndex.get("t_smallint"))).isEqualTo((short) (2 + rowNumber)); - assertThat(row.getField(columnIndex.get("t_int"))).isEqualTo(3 + (int) rowNumber); - - if (rowNumber % 13 == 0) { - assertThat(row.getField(columnIndex.get("t_bigint"))).isNull(); - } - else { - assertThat(row.getField(columnIndex.get("t_bigint"))).isEqualTo(4 + rowNumber); - } - - assertThat((Float) row.getField(columnIndex.get("t_float"))).isCloseTo(5.1f + rowNumber, offset(0.001f)); - assertThat(row.getField(columnIndex.get("t_double"))).isEqualTo(6.2 + rowNumber); - - if (rowNumber % 3 == 2) { - assertThat(row.getField(columnIndex.get("t_boolean"))).isNull(); - } - else { - assertThat(row.getField(columnIndex.get("t_boolean"))).isEqualTo(rowNumber % 3 != 0); - } - - assertThat(row.getField(columnIndex.get("ds"))).isEqualTo(ds); - assertThat(row.getField(columnIndex.get("file_format"))).isEqualTo(fileFormat); - assertThat(row.getField(columnIndex.get("dummy"))).isEqualTo(dummyPartition); - - long newCompletedBytes = pageSource.getCompletedBytes(); - assertThat(newCompletedBytes >= completedBytes).isTrue(); - assertThat(newCompletedBytes <= hiveSplit.getLength()).isTrue(); - completedBytes = newCompletedBytes; - } - - assertThat(completedBytes <= hiveSplit.getLength()).isTrue(); - assertThat(rowNumber).isEqualTo(100); - } - } - } - } - - @Test - public void testGetPartialRecords() - throws Exception - { - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorSession session = newSession(); - metadata.beginQuery(session); - - ConnectorTableHandle tableHandle = getTableHandle(metadata, tablePartitionFormat); - List columnHandles = ImmutableList.copyOf(metadata.getColumnHandles(session, tableHandle).values()); - Map columnIndex = indexColumns(columnHandles); - - List splits = getAllSplits(tableHandle, transaction, session); - assertThat(splits.size()).isEqualTo(tablePartitionFormatPartitions.size()); - - for (ConnectorSplit split : splits) { - HiveSplit hiveSplit = (HiveSplit) split; - - List partitionKeys = hiveSplit.getPartitionKeys(); - String ds = partitionKeys.get(0).getValue(); - String fileFormat = partitionKeys.get(1).getValue(); - HiveStorageFormat fileType = HiveStorageFormat.valueOf(fileFormat.toUpperCase(ENGLISH)); - int dummyPartition = Integer.parseInt(partitionKeys.get(2).getValue()); - - long rowNumber = 0; - try (ConnectorPageSource pageSource = pageSourceProvider.createPageSource(transaction.getTransactionHandle(), session, hiveSplit, tableHandle, columnHandles, DynamicFilter.EMPTY)) { - assertPageSourceType(pageSource, fileType); - MaterializedResult result = materializeSourceDataStream(session, pageSource, getTypes(columnHandles)); - for (MaterializedRow row : result) { - rowNumber++; - - assertThat(row.getField(columnIndex.get("t_double"))).isEqualTo(6.2 + rowNumber); - assertThat(row.getField(columnIndex.get("ds"))).isEqualTo(ds); - assertThat(row.getField(columnIndex.get("file_format"))).isEqualTo(fileFormat); - assertThat(row.getField(columnIndex.get("dummy"))).isEqualTo(dummyPartition); - } - } - assertThat(rowNumber).isEqualTo(100); - } - } - } - - @Test - public void testGetRecordsUnpartitioned() - throws Exception - { - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorSession session = newSession(); - metadata.beginQuery(session); - - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableUnpartitioned); - List columnHandles = ImmutableList.copyOf(metadata.getColumnHandles(session, tableHandle).values()); - Map columnIndex = indexColumns(columnHandles); - - List splits = getAllSplits(tableHandle, transaction, session); - assertThat(splits).hasSameSizeAs(tableUnpartitionedPartitions); - - for (ConnectorSplit split : splits) { - HiveSplit hiveSplit = (HiveSplit) split; - - assertThat(hiveSplit.getPartitionKeys()).isEqualTo(ImmutableList.of()); - - long rowNumber = 0; - try (ConnectorPageSource pageSource = pageSourceProvider.createPageSource(transaction.getTransactionHandle(), session, split, tableHandle, columnHandles, DynamicFilter.EMPTY)) { - assertPageSourceType(pageSource, TEXTFILE); - MaterializedResult result = materializeSourceDataStream(session, pageSource, getTypes(columnHandles)); - - for (MaterializedRow row : result) { - rowNumber++; - - if (rowNumber % 19 == 0) { - assertThat(row.getField(columnIndex.get("t_string"))).isNull(); - } - else if (rowNumber % 19 == 1) { - assertThat(row.getField(columnIndex.get("t_string"))).isEqualTo(""); - } - else { - assertThat(row.getField(columnIndex.get("t_string"))).isEqualTo("unpartitioned"); - } - - assertThat(row.getField(columnIndex.get("t_tinyint"))).isEqualTo((byte) (1 + rowNumber)); - } - } - assertThat(rowNumber).isEqualTo(100); - } - } - } - - @Test - public void testPartitionSchemaMismatch() - { - assertThatThrownBy(() -> { - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorTableHandle table = getTableHandle(metadata, tablePartitionSchemaChange); - ConnectorSession session = newSession(); - metadata.beginQuery(session); - readTable(transaction, table, ImmutableList.of(dsColumn), session, TupleDomain.all(), OptionalInt.empty(), Optional.empty()); - } - }) - .isInstanceOf(TrinoException.class) - .hasMessageMatching(".*The column 't_data' in table '.*\\.trino_test_partition_schema_change' is declared as type 'float', but partition 'ds=2012-12-29' declared column 't_data' as type 'string'."); - } - - // TODO coercion of non-canonical values should be supported - @Test - @Disabled - public void testPartitionSchemaNonCanonical() - throws Exception - { - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - ConnectorMetadata metadata = transaction.getMetadata(); - - ConnectorTableHandle table = getTableHandle(metadata, tablePartitionSchemaChangeNonCanonical); - ColumnHandle column = metadata.getColumnHandles(session, table).get("t_boolean"); - - Constraint constraint = new Constraint(TupleDomain.fromFixedValues(ImmutableMap.of(column, NullableValue.of(BOOLEAN, false)))); - table = applyFilter(metadata, table, constraint); - HivePartition partition = getOnlyElement(((HiveTableHandle) table).getPartitions().orElseThrow(AssertionError::new)); - assertThat(getPartitionId(partition)).isEqualTo("t_boolean=0"); - - ConnectorSplitSource splitSource = getSplits(splitManager, transaction, session, table); - ConnectorSplit split = getOnlyElement(getAllSplits(splitSource)); - - ImmutableList columnHandles = ImmutableList.of(column); - try (ConnectorPageSource ignored = pageSourceProvider.createPageSource(transaction.getTransactionHandle(), session, split, table, columnHandles, DynamicFilter.EMPTY)) { - fail("expected exception"); - } - catch (TrinoException e) { - assertThat(e.getErrorCode()).isEqualTo(HIVE_INVALID_PARTITION_VALUE.toErrorCode()); - } - } - } - - @Test - public void testTypesTextFile() - throws Exception - { - assertGetRecords("trino_test_types_textfile", TEXTFILE); - } - - @Test - public void testTypesSequenceFile() - throws Exception - { - assertGetRecords("trino_test_types_sequencefile", SEQUENCEFILE); - } - - @Test - public void testTypesRcText() - throws Exception - { - assertGetRecords("trino_test_types_rctext", RCTEXT); - } - - @Test - public void testTypesRcBinary() - throws Exception - { - assertGetRecords("trino_test_types_rcbinary", RCBINARY); - } - - @Test - public void testTypesOrc() - throws Exception - { - assertGetRecords("trino_test_types_orc", ORC); - } - - @Test - public void testTypesParquet() - throws Exception - { - assertGetRecords("trino_test_types_parquet", PARQUET); - } - - @Test - public void testEmptyTextFile() - throws Exception - { - assertEmptyFile(TEXTFILE); - } - - @Test - public void testEmptySequenceFile() - throws Exception - { - assertEmptyFile(SEQUENCEFILE); - } - - @Test - public void testEmptyRcTextFile() - throws Exception - { - assertEmptyFile(RCTEXT); - } - - @Test - public void testEmptyRcBinaryFile() - throws Exception - { - assertEmptyFile(RCBINARY); - } - - @Test - public void testEmptyOrcFile() - throws Exception - { - assertEmptyFile(ORC); - } - - private void assertEmptyFile(HiveStorageFormat format) - throws Exception - { - SchemaTableName tableName = temporaryTable("empty_file"); - try { - List columns = ImmutableList.of(new Column("test", HIVE_STRING, Optional.empty(), Map.of())); - createEmptyTable(tableName, format, columns, ImmutableList.of()); - - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - ConnectorMetadata metadata = transaction.getMetadata(); - metadata.beginQuery(session); - - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); - List columnHandles = filterNonHiddenColumnHandles(metadata.getColumnHandles(session, tableHandle).values()); - - Table table = transaction.getMetastore() - .getTable(tableName.getSchemaName(), tableName.getTableName()) - .orElseThrow(AssertionError::new); - - // verify directory is empty - HdfsContext context = new HdfsContext(session); - Path location = new Path(table.getStorage().getLocation()); - assertThat(listDirectory(context, location).isEmpty()).isTrue(); - - // read table with empty directory - readTable(transaction, tableHandle, columnHandles, session, TupleDomain.all(), OptionalInt.of(0), Optional.of(ORC)); - - // create empty file - FileSystem fileSystem = hdfsEnvironment.getFileSystem(context, location); - assertThat(fileSystem.createNewFile(new Path(location, "empty-file"))).isTrue(); - assertThat(listDirectory(context, location)).isEqualTo(ImmutableList.of("empty-file")); - - // read table with empty file - MaterializedResult result = readTable(transaction, tableHandle, columnHandles, session, TupleDomain.all(), OptionalInt.of(0), Optional.empty()); - assertThat(result.getRowCount()).isEqualTo(0); - } - } - finally { - dropTable(tableName); - } - } - - @Test - public void testRenameTable() - { - SchemaTableName temporaryRenameTableOld = temporaryTable("rename_old"); - SchemaTableName temporaryRenameTableNew = temporaryTable("rename_new"); - try { - createDummyTable(temporaryRenameTableOld); - - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - ConnectorMetadata metadata = transaction.getMetadata(); - - metadata.renameTable(session, getTableHandle(metadata, temporaryRenameTableOld), temporaryRenameTableNew); - transaction.commit(); - } - - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - ConnectorMetadata metadata = transaction.getMetadata(); - - assertThat(metadata.getTableHandle(session, temporaryRenameTableOld)).isNull(); - assertThat(metadata.getTableHandle(session, temporaryRenameTableNew)).isNotNull(); - } - } - finally { - dropTable(temporaryRenameTableOld); - dropTable(temporaryRenameTableNew); - } - } - - @Test - public void testTableCreation() - throws Exception - { - for (HiveStorageFormat storageFormat : createTableFormats) { - SchemaTableName temporaryCreateTable = temporaryTable("create"); - try { - doCreateTable(temporaryCreateTable, storageFormat); - } - finally { - dropTable(temporaryCreateTable); - } - } - } - - @Test - public void testTableCreationWithTrailingSpaceInLocation() - throws Exception - { - SchemaTableName tableName = temporaryTable("test_table_creation_with_trailing_space_in_location_" + randomNameSuffix()); - String tableDefaultLocationWithTrailingSpace = null; - try { - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - SemiTransactionalHiveMetastore metastore = transaction.getMetastore(); - TrinoFileSystem fileSystem = HDFS_FILE_SYSTEM_FACTORY.create(session); - - // Write data - tableDefaultLocationWithTrailingSpace = getTableDefaultLocation(metastore, fileSystem, tableName.getSchemaName(), tableName.getTableName()) + " "; - Path dataFilePath = new Path(tableDefaultLocationWithTrailingSpace, "foo.txt"); - FileSystem fs = hdfsEnvironment.getFileSystem(new HdfsContext(session), new Path(tableDefaultLocationWithTrailingSpace)); - try (OutputStream outputStream = fs.create(dataFilePath)) { - outputStream.write("hello\u0001world\nbye\u0001world".getBytes(UTF_8)); - } - - // create table - ConnectorTableMetadata tableMetadata = new ConnectorTableMetadata( - tableName, - ImmutableList.builder() - .add(new ColumnMetadata("t_string1", VARCHAR)) - .add(new ColumnMetadata("t_string2", VARCHAR)) - .build(), - ImmutableMap.builder() - .putAll(createTableProperties(TEXTFILE, ImmutableList.of())) - .put(EXTERNAL_LOCATION_PROPERTY, tableDefaultLocationWithTrailingSpace) - .buildOrThrow()); - - ConnectorMetadata metadata = transaction.getMetadata(); - metadata.createTable(session, tableMetadata, false); - - transaction.commit(); - } - - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - ConnectorMetadata metadata = transaction.getMetadata(); - metadata.beginQuery(session); - - // verify the data - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); - List columnHandles = filterNonHiddenColumnHandles(metadata.getColumnHandles(session, tableHandle).values()); - MaterializedResult result = readTable(transaction, tableHandle, columnHandles, session, TupleDomain.all(), OptionalInt.empty(), Optional.of(TEXTFILE)); - assertEqualsIgnoreOrder( - result.getMaterializedRows(), - MaterializedResult.resultBuilder(SESSION, VARCHAR, VARCHAR) - .row("hello", "world") - .row("bye", "world") - .build()); - } - } - finally { - dropTable(tableName); - if (tableDefaultLocationWithTrailingSpace != null) { - FileSystem fs = hdfsEnvironment.getFileSystem(new HdfsContext(SESSION), new Path(tableDefaultLocationWithTrailingSpace)); - fs.delete(new Path(tableDefaultLocationWithTrailingSpace), true); - } - } - } - - @Test - public void testTableCreationRollback() - throws Exception - { - SchemaTableName temporaryCreateRollbackTable = temporaryTable("create_rollback"); - try { - Location stagingPathRoot; - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - ConnectorMetadata metadata = transaction.getMetadata(); - - // begin creating the table - ConnectorTableMetadata tableMetadata = new ConnectorTableMetadata(temporaryCreateRollbackTable, CREATE_TABLE_COLUMNS, createTableProperties(RCBINARY)); - - ConnectorOutputTableHandle outputHandle = metadata.beginCreateTable(session, tableMetadata, Optional.empty(), NO_RETRIES); - - // write the data - ConnectorPageSink sink = pageSinkProvider.createPageSink(transaction.getTransactionHandle(), session, outputHandle, TESTING_PAGE_SINK_ID); - sink.appendPage(CREATE_TABLE_DATA.toPage()); - getFutureValue(sink.finish()); - - // verify we have data files - stagingPathRoot = getStagingPathRoot(outputHandle); - HdfsContext context = new HdfsContext(session); - assertThat(listAllDataFiles(context, stagingPathRoot).isEmpty()).isFalse(); - - // rollback the table - transaction.rollback(); - } - - // verify all files have been deleted - HdfsContext context = new HdfsContext(newSession()); - assertThat(listAllDataFiles(context, stagingPathRoot).isEmpty()).isTrue(); - - // verify table is not in the metastore - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - ConnectorMetadata metadata = transaction.getMetadata(); - assertThat(metadata.getTableHandle(session, temporaryCreateRollbackTable)).isNull(); - } - } - finally { - dropTable(temporaryCreateRollbackTable); - } - } - - @Test - public void testTableCreationIgnoreExisting() - { - List columns = ImmutableList.of(new Column("dummy", HiveType.valueOf("uniontype"), Optional.empty(), Map.of())); - SchemaTableName schemaTableName = temporaryTable("create"); - ConnectorSession session = newSession(); - String schemaName = schemaTableName.getSchemaName(); - String tableName = schemaTableName.getTableName(); - PrincipalPrivileges privileges = testingPrincipalPrivilege(session); - Location targetPath; - try { - try (Transaction transaction = newTransaction()) { - LocationService locationService = getLocationService(); - targetPath = locationService.forNewTable(transaction.getMetastore(), session, schemaName, tableName); - Table table = createSimpleTable(schemaTableName, columns, session, targetPath, "q1"); - transaction.getMetastore() - .createTable(session, table, privileges, Optional.empty(), Optional.empty(), false, ZERO_TABLE_STATISTICS, false); - Optional
tableHandle = transaction.getMetastore().getTable(schemaName, tableName); - assertThat(tableHandle.isPresent()).isTrue(); - transaction.commit(); - } - - // try creating it again from another transaction with ignoreExisting=false - try (Transaction transaction = newTransaction()) { - Table table = createSimpleTable(schemaTableName, columns, session, targetPath.appendSuffix("_2"), "q2"); - transaction.getMetastore() - .createTable(session, table, privileges, Optional.empty(), Optional.empty(), false, ZERO_TABLE_STATISTICS, false); - transaction.commit(); - fail("Expected exception"); - } - catch (TrinoException e) { - assertInstanceOf(e, TableAlreadyExistsException.class); - } - - // try creating it again from another transaction with ignoreExisting=true - try (Transaction transaction = newTransaction()) { - Table table = createSimpleTable(schemaTableName, columns, session, targetPath.appendSuffix("_3"), "q3"); - transaction.getMetastore() - .createTable(session, table, privileges, Optional.empty(), Optional.empty(), true, ZERO_TABLE_STATISTICS, false); - transaction.commit(); - } - - // at this point the table should exist, now try creating the table again with a different table definition - columns = ImmutableList.of(new Column("new_column", HiveType.valueOf("string"), Optional.empty(), Map.of())); - try (Transaction transaction = newTransaction()) { - Table table = createSimpleTable(schemaTableName, columns, session, targetPath.appendSuffix("_4"), "q4"); - transaction.getMetastore() - .createTable(session, table, privileges, Optional.empty(), Optional.empty(), true, ZERO_TABLE_STATISTICS, false); - transaction.commit(); - fail("Expected exception"); - } - catch (TrinoException e) { - assertThat(e.getErrorCode()).isEqualTo(TRANSACTION_CONFLICT.toErrorCode()); - assertThat(e.getMessage()).isEqualTo(format("Table already exists with a different schema: '%s'", schemaTableName.getTableName())); - } - } - finally { - dropTable(schemaTableName); - } - } - - private static Table createSimpleTable(SchemaTableName schemaTableName, List columns, ConnectorSession session, Location targetPath, String queryId) - { - String tableOwner = session.getUser(); - String schemaName = schemaTableName.getSchemaName(); - String tableName = schemaTableName.getTableName(); - return Table.builder() - .setDatabaseName(schemaName) - .setTableName(tableName) - .setOwner(Optional.of(tableOwner)) - .setTableType(MANAGED_TABLE.name()) - .setParameters(ImmutableMap.of( - TRINO_VERSION_NAME, TEST_SERVER_VERSION, - TRINO_QUERY_ID_NAME, queryId)) - .setDataColumns(columns) - .withStorage(storage -> storage - .setLocation(targetPath.toString()) - .setStorageFormat(fromHiveStorageFormat(ORC)) - .setSerdeParameters(ImmutableMap.of())) - .build(); - } - - @Test - public void testBucketSortedTables() - throws Exception - { - SchemaTableName table = temporaryTable("create_sorted"); - try { - doTestBucketSortedTables(table); - } - finally { - dropTable(table); - } - } - - private void doTestBucketSortedTables(SchemaTableName table) - throws IOException - { - int bucketCount = 3; - int expectedRowCount = 0; - - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - ConnectorMetadata metadata = transaction.getMetadata(); - - // begin creating the table - ConnectorTableMetadata tableMetadata = new ConnectorTableMetadata( - table, - ImmutableList.builder() - .add(new ColumnMetadata("id", VARCHAR)) - .add(new ColumnMetadata("value_asc", VARCHAR)) - .add(new ColumnMetadata("value_desc", BIGINT)) - .add(new ColumnMetadata("ds", VARCHAR)) - .build(), - ImmutableMap.builder() - .put(STORAGE_FORMAT_PROPERTY, RCBINARY) - .put(PARTITIONED_BY_PROPERTY, ImmutableList.of("ds")) - .put(BUCKETED_BY_PROPERTY, ImmutableList.of("id")) - .put(BUCKET_COUNT_PROPERTY, bucketCount) - .put(SORTED_BY_PROPERTY, ImmutableList.builder() - .add(new SortingColumn("value_asc", ASCENDING)) - .add(new SortingColumn("value_desc", DESCENDING)) - .build()) - .buildOrThrow()); - - ConnectorOutputTableHandle outputHandle = metadata.beginCreateTable(session, tableMetadata, Optional.empty(), NO_RETRIES); - - // write the data - ConnectorPageSink sink = pageSinkProvider.createPageSink(transaction.getTransactionHandle(), session, outputHandle, TESTING_PAGE_SINK_ID); - List types = tableMetadata.getColumns().stream() - .map(ColumnMetadata::getType) - .collect(toList()); - ThreadLocalRandom random = ThreadLocalRandom.current(); - for (int i = 0; i < 50; i++) { - MaterializedResult.Builder builder = MaterializedResult.resultBuilder(session, types); - for (int j = 0; j < 1000; j++) { - builder.row( - sha256().hashLong(random.nextLong()).toString(), - "test" + random.nextInt(100), - random.nextLong(100_000), - "2018-04-01"); - expectedRowCount++; - } - sink.appendPage(builder.build().toPage()); - } - - HdfsContext context = new HdfsContext(session); - HiveConfig config = getHiveConfig(); - // verify we have enough temporary files per bucket to require multiple passes - Location stagingPathRoot; - if (config.isTemporaryStagingDirectoryEnabled()) { - stagingPathRoot = Location.of(config.getTemporaryStagingDirectoryPath() - .replace("${USER}", context.getIdentity().getUser())); - } - else { - stagingPathRoot = getStagingPathRoot(outputHandle); - } - assertThat(listAllDataFiles(context, stagingPathRoot)) - .filteredOn(file -> file.contains(".tmp-sort.")) - .size().isGreaterThan(bucketCount * getSortingFileWriterConfig().getMaxOpenSortFiles() * 2); - - // finish the write - Collection fragments = getFutureValue(sink.finish()); - - // verify there are no temporary files - for (String file : listAllDataFiles(context, stagingPathRoot)) { - assertThat(file).doesNotContain(".tmp-sort."); - } - - // finish creating table - metadata.finishCreateTable(session, outputHandle, fragments, ImmutableList.of()); - - transaction.commit(); - } - - // verify that bucket files are sorted - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorSession session = newSession(); - metadata.beginQuery(session); - - ConnectorTableHandle tableHandle = getTableHandle(metadata, table); - List columnHandles = ImmutableList.copyOf(metadata.getColumnHandles(session, tableHandle).values()); - - // verify local sorting property - ConnectorTableProperties properties = metadata.getTableProperties( - newSession(ImmutableMap.of( - "propagate_table_scan_sorting_properties", true, - "bucket_execution_enabled", false)), - tableHandle); - Map columnIndex = indexColumns(columnHandles); - assertThat(properties.getLocalProperties()).isEqualTo(ImmutableList.of( - new SortingProperty<>(columnHandles.get(columnIndex.get("value_asc")), ASC_NULLS_FIRST), - new SortingProperty<>(columnHandles.get(columnIndex.get("value_desc")), DESC_NULLS_LAST))); - assertThat(metadata.getTableProperties(newSession(), tableHandle).getLocalProperties()).isEmpty(); - - List splits = getAllSplits(tableHandle, transaction, session); - assertThat(splits).hasSize(bucketCount); - - int actualRowCount = 0; - for (ConnectorSplit split : splits) { - try (ConnectorPageSource pageSource = pageSourceProvider.createPageSource(transaction.getTransactionHandle(), session, split, tableHandle, columnHandles, DynamicFilter.EMPTY)) { - String lastValueAsc = null; - long lastValueDesc = -1; - - while (!pageSource.isFinished()) { - Page page = pageSource.getNextPage(); - if (page == null) { - continue; - } - for (int i = 0; i < page.getPositionCount(); i++) { - Block blockAsc = page.getBlock(1); - Block blockDesc = page.getBlock(2); - assertThat(blockAsc.isNull(i)).isFalse(); - assertThat(blockDesc.isNull(i)).isFalse(); - - String valueAsc = VARCHAR.getSlice(blockAsc, i).toStringUtf8(); - if (lastValueAsc != null) { - assertGreaterThanOrEqual(valueAsc, lastValueAsc); - if (valueAsc.equals(lastValueAsc)) { - long valueDesc = BIGINT.getLong(blockDesc, i); - if (lastValueDesc != -1) { - assertLessThanOrEqual(valueDesc, lastValueDesc); - } - lastValueDesc = valueDesc; - } - else { - lastValueDesc = -1; - } - } - lastValueAsc = valueAsc; - actualRowCount++; - } - } - } - } - assertThat(actualRowCount).isEqualTo(expectedRowCount); - } - } - - @Test - public void testInsert() - throws Exception - { - for (HiveStorageFormat storageFormat : createTableFormats) { - SchemaTableName temporaryInsertTable = temporaryTable("insert"); - try { - doInsert(storageFormat, temporaryInsertTable); - } - finally { - dropTable(temporaryInsertTable); - } - } - } - - @Test - public void testInsertOverwriteUnpartitioned() - throws Exception - { - SchemaTableName table = temporaryTable("insert_overwrite"); - try { - doInsertOverwriteUnpartitioned(table); - } - finally { - dropTable(table); - } - } - - @Test - public void testInsertIntoNewPartition() - throws Exception - { - for (HiveStorageFormat storageFormat : createTableFormats) { - SchemaTableName temporaryInsertIntoNewPartitionTable = temporaryTable("insert_new_partitioned"); - try { - doInsertIntoNewPartition(storageFormat, temporaryInsertIntoNewPartitionTable); - } - finally { - dropTable(temporaryInsertIntoNewPartitionTable); - } - } - } - - @Test - public void testInsertIntoExistingPartition() - throws Exception - { - for (HiveStorageFormat storageFormat : createTableFormats) { - SchemaTableName temporaryInsertIntoExistingPartitionTable = temporaryTable("insert_existing_partitioned"); - try { - doInsertIntoExistingPartition(storageFormat, temporaryInsertIntoExistingPartitionTable); - } - finally { - dropTable(temporaryInsertIntoExistingPartitionTable); - } - } - } - - @Test - public void testInsertIntoExistingPartitionEmptyStatistics() - throws Exception - { - for (HiveStorageFormat storageFormat : createTableFormats) { - SchemaTableName temporaryInsertIntoExistingPartitionTable = temporaryTable("insert_existing_partitioned_empty_statistics"); - try { - doInsertIntoExistingPartitionEmptyStatistics(storageFormat, temporaryInsertIntoExistingPartitionTable); - } - finally { - dropTable(temporaryInsertIntoExistingPartitionTable); - } - } - } - - @Test - public void testInsertUnsupportedWriteType() - throws Exception - { - SchemaTableName temporaryInsertUnsupportedWriteType = temporaryTable("insert_unsupported_type"); - try { - doInsertUnsupportedWriteType(ORC, temporaryInsertUnsupportedWriteType); - } - finally { - dropTable(temporaryInsertUnsupportedWriteType); - } - } - - @Test - public void testMetadataDelete() - throws Exception - { - for (HiveStorageFormat storageFormat : createTableFormats) { - SchemaTableName temporaryMetadataDeleteTable = temporaryTable("metadata_delete"); - try { - doTestMetadataDelete(storageFormat, temporaryMetadataDeleteTable); - } - finally { - dropTable(temporaryMetadataDeleteTable); - } - } - } - - @Test - public void testEmptyTableCreation() - throws Exception - { - for (HiveStorageFormat storageFormat : createTableFormats) { - SchemaTableName temporaryCreateEmptyTable = temporaryTable("create_empty"); - try { - doCreateEmptyTable(temporaryCreateEmptyTable, storageFormat, CREATE_TABLE_COLUMNS); - } - finally { - dropTable(temporaryCreateEmptyTable); - } - } - } - - @Test - public void testCreateEmptyTableShouldNotCreateStagingDirectory() - throws IOException - { - for (HiveStorageFormat storageFormat : createTableFormats) { - SchemaTableName temporaryCreateEmptyTable = temporaryTable("create_empty"); - try { - List columns = ImmutableList.of(new Column("test", HIVE_STRING, Optional.empty(), Map.of())); - try (Transaction transaction = newTransaction()) { - String temporaryStagingPrefix = "hive-temporary-staging-prefix-" + UUID.randomUUID().toString().toLowerCase(ENGLISH).replace("-", ""); - ConnectorSession session = newSession(); - String tableOwner = session.getUser(); - String schemaName = temporaryCreateEmptyTable.getSchemaName(); - String tableName = temporaryCreateEmptyTable.getTableName(); - HiveConfig hiveConfig = getHiveConfig() - .setTemporaryStagingDirectoryPath(temporaryStagingPrefix) - .setTemporaryStagingDirectoryEnabled(true); - TrinoFileSystemFactory fileSystemFactory = new HdfsFileSystemFactory(hdfsEnvironment, HDFS_FILE_SYSTEM_STATS); - LocationService locationService = new HiveLocationService(fileSystemFactory, hiveConfig); - Location targetPath = locationService.forNewTable(transaction.getMetastore(), session, schemaName, tableName); - Table.Builder tableBuilder = Table.builder() - .setDatabaseName(schemaName) - .setTableName(tableName) - .setOwner(Optional.of(tableOwner)) - .setTableType(MANAGED_TABLE.name()) - .setParameters(ImmutableMap.of( - TRINO_VERSION_NAME, TEST_SERVER_VERSION, - TRINO_QUERY_ID_NAME, session.getQueryId())) - .setDataColumns(columns); - tableBuilder.getStorageBuilder() - .setLocation(targetPath.toString()) - .setStorageFormat(StorageFormat.create(storageFormat.getSerde(), storageFormat.getInputFormat(), storageFormat.getOutputFormat())); - transaction.getMetastore().createTable( - session, - tableBuilder.build(), - testingPrincipalPrivilege(tableOwner, session.getUser()), - Optional.empty(), - Optional.empty(), - true, - ZERO_TABLE_STATISTICS, - false); - transaction.commit(); - - HdfsContext context = new HdfsContext(session); - Path temporaryRoot = new Path(targetPath.toString(), temporaryStagingPrefix); - FileSystem fileSystem = hdfsEnvironment.getFileSystem(context, temporaryRoot); - assertThat(fileSystem.exists(temporaryRoot)) - .describedAs(format("Temporary staging directory %s is created.", temporaryRoot)) - .isFalse(); - } - } - finally { - dropTable(temporaryCreateEmptyTable); - } - } - } - - @Test - public void testViewCreation() - { - SchemaTableName temporaryCreateView = temporaryTable("create_view"); - try { - verifyViewCreation(temporaryCreateView); - } - finally { - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - metadata.dropView(newSession(), temporaryCreateView); - transaction.commit(); - } - catch (RuntimeException e) { - // this usually occurs because the view was not created - } - } - } - - @Test - public void testCreateTableUnsupportedType() - { - for (HiveStorageFormat storageFormat : createTableFormats) { - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - ConnectorMetadata metadata = transaction.getMetadata(); - List columns = ImmutableList.of(new ColumnMetadata("dummy", HYPER_LOG_LOG)); - ConnectorTableMetadata tableMetadata = new ConnectorTableMetadata(invalidTable, columns, createTableProperties(storageFormat)); - metadata.beginCreateTable(session, tableMetadata, Optional.empty(), NO_RETRIES); - fail("create table with unsupported type should fail for storage format " + storageFormat); - } - catch (TrinoException e) { - assertThat(e.getErrorCode()).isEqualTo(NOT_SUPPORTED.toErrorCode()); - } - } - } - - @Test - public void testHideDeltaLakeTables() - { - ConnectorSession session = newSession(); - SchemaTableName tableName = temporaryTable("trino_delta_lake_table"); - - Table.Builder table = Table.builder() - .setDatabaseName(tableName.getSchemaName()) - .setTableName(tableName.getTableName()) - .setOwner(Optional.of(session.getUser())) - .setTableType(MANAGED_TABLE.name()) - .setPartitionColumns(List.of(new Column("a_partition_column", HIVE_INT, Optional.empty(), Map.of()))) - .setDataColumns(List.of(new Column("a_column", HIVE_STRING, Optional.empty(), Map.of()))) - .setParameter(SPARK_TABLE_PROVIDER_KEY, DELTA_LAKE_PROVIDER); - table.getStorageBuilder() - .setStorageFormat(fromHiveStorageFormat(PARQUET)) - .setLocation(getTableDefaultLocation( - metastoreClient.getDatabase(tableName.getSchemaName()).orElseThrow(), - new HdfsFileSystemFactory(hdfsEnvironment, HDFS_FILE_SYSTEM_STATS).create(session), - tableName.getSchemaName(), - tableName.getTableName()).toString()); - metastoreClient.createTable(table.build(), NO_PRIVILEGES); - - try { - // Verify the table was created as a Delta Lake table - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - metadata.beginQuery(session); - assertThatThrownBy(() -> getTableHandle(metadata, tableName)) - .hasMessage(format("Cannot query Delta Lake table '%s'", tableName)); - } - - // Verify the hidden `$properties` and `$partitions` Delta Lake table handle can't be obtained within the hive connector - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - metadata.beginQuery(session); - SchemaTableName propertiesTableName = new SchemaTableName(tableName.getSchemaName(), format("%s$properties", tableName.getTableName())); - assertThat(metadata.getSystemTable(newSession(), propertiesTableName)).isEmpty(); - SchemaTableName partitionsTableName = new SchemaTableName(tableName.getSchemaName(), format("%s$partitions", tableName.getTableName())); - assertThat(metadata.getSystemTable(newSession(), partitionsTableName)).isEmpty(); - } - - // Assert that table is hidden - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - - // TODO (https://github.com/trinodb/trino/issues/5426) these assertions should use information_schema instead of metadata directly, - // as information_schema or MetadataManager may apply additional logic - - // list all tables - assertThat(metadata.listTables(session, Optional.empty())) - .doesNotContain(tableName); - - // list all tables in a schema - assertThat(metadata.listTables(session, Optional.of(tableName.getSchemaName()))) - .doesNotContain(tableName); - - // list all columns in a schema - assertThat(listTableColumns(metadata, session, new SchemaTablePrefix(tableName.getSchemaName())).keySet()) - .doesNotContain(tableName); - - // list all columns in a table - assertThat(listTableColumns(metadata, session, new SchemaTablePrefix(tableName.getSchemaName(), tableName.getTableName())).keySet()) - .doesNotContain(tableName); - } - } - finally { - // Clean up - metastoreClient.dropTable(tableName.getSchemaName(), tableName.getTableName(), true); - } - } - - @Test - public void testDisallowQueryingOfIcebergTables() - { - ConnectorSession session = newSession(); - SchemaTableName tableName = temporaryTable("trino_iceberg_table"); - - Table.Builder table = Table.builder() - .setDatabaseName(tableName.getSchemaName()) - .setTableName(tableName.getTableName()) - .setOwner(Optional.of(session.getUser())) - .setTableType(MANAGED_TABLE.name()) - .setPartitionColumns(List.of(new Column("a_partition_column", HIVE_INT, Optional.empty(), Map.of()))) - .setDataColumns(List.of(new Column("a_column", HIVE_STRING, Optional.empty(), Map.of()))) - .setParameter(ICEBERG_TABLE_TYPE_NAME, ICEBERG_TABLE_TYPE_VALUE); - table.getStorageBuilder() - .setStorageFormat(fromHiveStorageFormat(PARQUET)) - .setLocation(getTableDefaultLocation( - metastoreClient.getDatabase(tableName.getSchemaName()).orElseThrow(), - new HdfsFileSystemFactory(hdfsEnvironment, HDFS_FILE_SYSTEM_STATS).create(session), - tableName.getSchemaName(), - tableName.getTableName()).toString()); - metastoreClient.createTable(table.build(), NO_PRIVILEGES); - - try { - // Verify that the table was created as a Iceberg table can't be queried in hive - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - metadata.beginQuery(session); - assertThatThrownBy(() -> getTableHandle(metadata, tableName)) - .hasMessage(format("Cannot query Iceberg table '%s'", tableName)); - } - - // Verify the hidden `$properties` and `$partitions` hive system tables table handle can't be obtained for the Iceberg tables - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - metadata.beginQuery(session); - SchemaTableName propertiesTableName = new SchemaTableName(tableName.getSchemaName(), format("%s$properties", tableName.getTableName())); - assertThat(metadata.getSystemTable(newSession(), propertiesTableName)).isEmpty(); - SchemaTableName partitionsTableName = new SchemaTableName(tableName.getSchemaName(), format("%s$partitions", tableName.getTableName())); - assertThat(metadata.getSystemTable(newSession(), partitionsTableName)).isEmpty(); - } - } - finally { - // Clean up - metastoreClient.dropTable(tableName.getSchemaName(), tableName.getTableName(), true); - } - } - - @Test - public void testUpdateBasicTableStatistics() - throws Exception - { - SchemaTableName tableName = temporaryTable("update_basic_table_statistics"); - try { - doCreateEmptyTable(tableName, ORC, STATISTICS_TABLE_COLUMNS); - testUpdateTableStatistics(tableName, ZERO_TABLE_STATISTICS, BASIC_STATISTICS_1, BASIC_STATISTICS_2); - } - finally { - dropTable(tableName); - } - } - - @Test - public void testUpdateTableColumnStatistics() - throws Exception - { - SchemaTableName tableName = temporaryTable("update_table_column_statistics"); - try { - doCreateEmptyTable(tableName, ORC, STATISTICS_TABLE_COLUMNS); - testUpdateTableStatistics(tableName, ZERO_TABLE_STATISTICS, STATISTICS_1_1, STATISTICS_1_2, STATISTICS_2); - } - finally { - dropTable(tableName); - } - } - - @Test - public void testUpdateTableColumnStatisticsEmptyOptionalFields() - throws Exception - { - SchemaTableName tableName = temporaryTable("update_table_column_statistics_empty_optional_fields"); - try { - doCreateEmptyTable(tableName, ORC, STATISTICS_TABLE_COLUMNS); - testUpdateTableStatistics(tableName, ZERO_TABLE_STATISTICS, STATISTICS_EMPTY_OPTIONAL_FIELDS); - } - finally { - dropTable(tableName); - } - } - - protected void testUpdateTableStatistics(SchemaTableName tableName, PartitionStatistics initialStatistics, PartitionStatistics... statistics) - { - HiveMetastoreClosure metastoreClient = new HiveMetastoreClosure(getMetastoreClient(), TESTING_TYPE_MANAGER, false); - assertThat(metastoreClient.getTableStatistics(tableName.getSchemaName(), tableName.getTableName(), Optional.empty())) - .isEqualTo(initialStatistics); - - AtomicReference expectedStatistics = new AtomicReference<>(initialStatistics); - for (PartitionStatistics partitionStatistics : statistics) { - metastoreClient.updateTableStatistics(tableName.getSchemaName(), tableName.getTableName(), NO_ACID_TRANSACTION, actualStatistics -> { - assertThat(actualStatistics).isEqualTo(expectedStatistics.get()); - return partitionStatistics; - }); - assertThat(metastoreClient.getTableStatistics(tableName.getSchemaName(), tableName.getTableName(), Optional.empty())) - .isEqualTo(partitionStatistics); - expectedStatistics.set(partitionStatistics); - } - - assertThat(metastoreClient.getTableStatistics(tableName.getSchemaName(), tableName.getTableName(), Optional.empty())) - .isEqualTo(expectedStatistics.get()); - - metastoreClient.updateTableStatistics(tableName.getSchemaName(), tableName.getTableName(), NO_ACID_TRANSACTION, actualStatistics -> { - assertThat(actualStatistics).isEqualTo(expectedStatistics.get()); - return initialStatistics; - }); - - assertThat(metastoreClient.getTableStatistics(tableName.getSchemaName(), tableName.getTableName(), Optional.empty())) - .isEqualTo(initialStatistics); - } - - @Test - public void testUpdateBasicPartitionStatistics() - throws Exception - { - SchemaTableName tableName = temporaryTable("update_basic_partition_statistics"); - try { - createDummyPartitionedTable(tableName, STATISTICS_PARTITIONED_TABLE_COLUMNS); - testUpdatePartitionStatistics( - tableName, - ZERO_TABLE_STATISTICS, - ImmutableList.of(BASIC_STATISTICS_1, BASIC_STATISTICS_2), - ImmutableList.of(BASIC_STATISTICS_2, BASIC_STATISTICS_1)); - } - finally { - dropTable(tableName); - } - } - - @Test - public void testUpdatePartitionColumnStatistics() - throws Exception - { - SchemaTableName tableName = temporaryTable("update_partition_column_statistics"); - try { - createDummyPartitionedTable(tableName, STATISTICS_PARTITIONED_TABLE_COLUMNS); - testUpdatePartitionStatistics( - tableName, - ZERO_TABLE_STATISTICS, - ImmutableList.of(STATISTICS_1_1, STATISTICS_1_2, STATISTICS_2), - ImmutableList.of(STATISTICS_1_2, STATISTICS_1_1, STATISTICS_2)); - } - finally { - dropTable(tableName); - } - } - - @Test - public void testUpdatePartitionColumnStatisticsEmptyOptionalFields() - throws Exception - { - SchemaTableName tableName = temporaryTable("update_partition_column_statistics"); - try { - createDummyPartitionedTable(tableName, STATISTICS_PARTITIONED_TABLE_COLUMNS); - testUpdatePartitionStatistics( - tableName, - ZERO_TABLE_STATISTICS, - ImmutableList.of(STATISTICS_EMPTY_OPTIONAL_FIELDS), - ImmutableList.of(STATISTICS_EMPTY_OPTIONAL_FIELDS)); - } - finally { - dropTable(tableName); - } - } - - @Test - public void testDataColumnProperties() - throws Exception - { - SchemaTableName tableName = temporaryTable("test_column_properties"); - HiveMetastoreClosure metastoreClient = new HiveMetastoreClosure(getMetastoreClient(), TESTING_TYPE_MANAGER, false); - try { - doCreateEmptyTable(tableName, ORC, List.of(new ColumnMetadata("id", BIGINT), new ColumnMetadata("part_key", createVarcharType(256)))); - - Table table = metastoreClient.getTable(tableName.getSchemaName(), tableName.getTableName()).orElseThrow(); - assertThat(table.getDataColumns()) - .singleElement() - .extracting(Column::getProperties, InstanceOfAssertFactories.MAP) - .isEmpty(); - assertThat(table.getPartitionColumns()) - .singleElement() - .extracting(Column::getProperties, InstanceOfAssertFactories.MAP) - .isEmpty(); - - String columnPropertyValue = "data column value ,;.!??? \" ' {} [] non-printable \000 \001 spaces \n\r\t\f hiragana だ emoji 🤷‍♂️ x"; - metastoreClient.replaceTable( - tableName.getSchemaName(), - tableName.getTableName(), - Table.builder(table) - .setDataColumns(List.of(new Column("id", HIVE_LONG, Optional.empty(), Map.of("data prop", columnPropertyValue)))) - .build(), - NO_PRIVILEGES); - - table = metastoreClient.getTable(tableName.getSchemaName(), tableName.getTableName()).orElseThrow(); - assertThat(table.getDataColumns()) - .singleElement() - .extracting(Column::getProperties, InstanceOfAssertFactories.MAP) - .isEqualTo(Map.of("data prop", columnPropertyValue)); - assertThat(table.getPartitionColumns()) - .singleElement() - .extracting(Column::getProperties, InstanceOfAssertFactories.MAP) - .isEmpty(); - } - finally { - dropTable(tableName); - } - } - - @Test - public void testPartitionColumnProperties() - throws Exception - { - SchemaTableName tableName = temporaryTable("test_column_properties"); - HiveMetastoreClosure metastoreClient = new HiveMetastoreClosure(getMetastoreClient(), TESTING_TYPE_MANAGER, false); - try { - doCreateEmptyTable(tableName, ORC, List.of(new ColumnMetadata("id", BIGINT), new ColumnMetadata("part_key", createVarcharType(256)))); - - Table table = metastoreClient.getTable(tableName.getSchemaName(), tableName.getTableName()).orElseThrow(); - assertThat(table.getDataColumns()) - .singleElement() - .extracting(Column::getProperties, InstanceOfAssertFactories.MAP) - .isEmpty(); - assertThat(table.getPartitionColumns()) - .singleElement() - .extracting(Column::getProperties, InstanceOfAssertFactories.MAP) - .isEmpty(); - - String columnPropertyValue = "partition column value ,;.!??? \" ' {} [] non-printable \000 \001 spaces \n\r\t\f hiragana だ emoji 🤷‍♂️ x"; - metastoreClient.replaceTable( - tableName.getSchemaName(), - tableName.getTableName(), - Table.builder(table) - .setPartitionColumns(List.of(new Column("part_key", HiveType.valueOf("varchar(256)"), Optional.empty(), Map.of("partition prop", columnPropertyValue)))) - .build(), - NO_PRIVILEGES); - - table = metastoreClient.getTable(tableName.getSchemaName(), tableName.getTableName()).orElseThrow(); - assertThat(table.getDataColumns()) - .singleElement() - .extracting(Column::getProperties, InstanceOfAssertFactories.MAP) - .isEmpty(); - assertThat(table.getPartitionColumns()) - .singleElement() - .extracting(Column::getProperties, InstanceOfAssertFactories.MAP) - .isEqualTo(Map.of("partition prop", columnPropertyValue)); - } - finally { - dropTable(tableName); - } - } - - @Test - public void testInputInfoWhenTableIsPartitioned() - throws Exception - { - SchemaTableName tableName = temporaryTable("test_input_info_with_partitioned_table"); - try { - createDummyPartitionedTable(tableName, STATISTICS_PARTITIONED_TABLE_COLUMNS); - assertInputInfo(tableName, new HiveInputInfo(ImmutableList.of(), true, Optional.of("ORC"))); - } - finally { - dropTable(tableName); - } - } - - @Test - public void testInputInfoWhenTableIsNotPartitioned() - { - SchemaTableName tableName = temporaryTable("test_input_info_without_partitioned_table"); - try { - createDummyTable(tableName); - assertInputInfo(tableName, new HiveInputInfo(ImmutableList.of(), false, Optional.of("TEXTFILE"))); - } - finally { - dropTable(tableName); - } - } - - @Test - public void testInputInfoWithParquetTableFormat() - { - SchemaTableName tableName = temporaryTable("test_input_info_with_parquet_table_format"); - try { - createDummyTable(tableName, PARQUET); - assertInputInfo(tableName, new HiveInputInfo(ImmutableList.of(), false, Optional.of("PARQUET"))); - } - finally { - dropTable(tableName); - } - } - - private void assertInputInfo(SchemaTableName tableName, HiveInputInfo expectedInputInfo) - { - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - ConnectorMetadata metadata = transaction.getMetadata(); - HiveTableHandle tableHandle = (HiveTableHandle) metadata.getTableHandle(session, tableName); - assertThat(metadata.getInfo(tableHandle)).isEqualTo(Optional.of(expectedInputInfo)); - } - } - - /** - * During table scan, the illegal storage format for some specific table should not fail the whole table scan - */ - @Test - public void testIllegalStorageFormatDuringTableScan() - { - SchemaTableName schemaTableName = temporaryTable("test_illegal_storage_format"); - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - List columns = ImmutableList.of(new Column("pk", HIVE_STRING, Optional.empty(), Map.of())); - String tableOwner = session.getUser(); - String schemaName = schemaTableName.getSchemaName(); - String tableName = schemaTableName.getTableName(); - Location targetPath = locationService.forNewTable(transaction.getMetastore(), session, schemaName, tableName); - //create table whose storage format is null - Table.Builder tableBuilder = Table.builder() - .setDatabaseName(schemaName) - .setTableName(tableName) - .setOwner(Optional.of(tableOwner)) - .setTableType(MANAGED_TABLE.name()) - .setParameters(ImmutableMap.of( - TRINO_VERSION_NAME, TEST_SERVER_VERSION, - TRINO_QUERY_ID_NAME, session.getQueryId())) - .setDataColumns(columns) - .withStorage(storage -> storage - .setLocation(targetPath.toString()) - .setStorageFormat(StorageFormat.createNullable(null, null, null)) - .setSerdeParameters(ImmutableMap.of())); - PrincipalPrivileges principalPrivileges = testingPrincipalPrivilege(tableOwner, session.getUser()); - transaction.getMetastore().createTable(session, tableBuilder.build(), principalPrivileges, Optional.empty(), Optional.empty(), true, ZERO_TABLE_STATISTICS, false); - transaction.commit(); - } - - // We retrieve the table whose storageFormat has null serde/inputFormat/outputFormat - // to make sure it can still be retrieved instead of throwing exception. - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - Map> allColumns = listTableColumns(metadata, newSession(), new SchemaTablePrefix(schemaTableName.getSchemaName())); - assertThat(allColumns).containsKey(schemaTableName); - } - finally { - dropTable(schemaTableName); - } - } - - protected static Map> listTableColumns(ConnectorMetadata metadata, ConnectorSession session, SchemaTablePrefix prefix) - { - return stream(metadata.streamTableColumns(session, prefix)) - .collect(toImmutableMap( - TableColumnsMetadata::getTable, - tableColumns -> tableColumns.getColumns().orElseThrow(() -> new IllegalStateException("Table " + tableColumns.getTable() + " reported as redirected")))); - } - - private void createDummyTable(SchemaTableName tableName) - { - createDummyTable(tableName, TEXTFILE); - } - - private void createDummyTable(SchemaTableName tableName, HiveStorageFormat storageFormat) - { - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - ConnectorMetadata metadata = transaction.getMetadata(); - - List columns = ImmutableList.of(new ColumnMetadata("dummy", createUnboundedVarcharType())); - ConnectorTableMetadata tableMetadata = new ConnectorTableMetadata(tableName, columns, createTableProperties(storageFormat)); - ConnectorOutputTableHandle handle = metadata.beginCreateTable(session, tableMetadata, Optional.empty(), NO_RETRIES); - metadata.finishCreateTable(session, handle, ImmutableList.of(), ImmutableList.of()); - - transaction.commit(); - } - } - - protected void createDummyPartitionedTable(SchemaTableName tableName, List columns) - throws Exception - { - doCreateEmptyTable(tableName, ORC, columns); - - HiveMetastoreClosure metastoreClient = new HiveMetastoreClosure(getMetastoreClient(), TESTING_TYPE_MANAGER, false); - Table table = metastoreClient.getTable(tableName.getSchemaName(), tableName.getTableName()) - .orElseThrow(() -> new TableNotFoundException(tableName)); - - List firstPartitionValues = ImmutableList.of("2016-01-01"); - List secondPartitionValues = ImmutableList.of("2016-01-02"); - - String firstPartitionName = makePartName(ImmutableList.of("ds"), firstPartitionValues); - String secondPartitionName = makePartName(ImmutableList.of("ds"), secondPartitionValues); - - List partitions = ImmutableList.of(firstPartitionName, secondPartitionName) - .stream() - .map(partitionName -> new PartitionWithStatistics(createDummyPartition(table, partitionName), partitionName, PartitionStatistics.empty())) - .collect(toImmutableList()); - metastoreClient.addPartitions(tableName.getSchemaName(), tableName.getTableName(), partitions); - metastoreClient.updatePartitionsStatistics(tableName.getSchemaName(), tableName.getTableName(), firstPartitionName, currentStatistics -> ZERO_TABLE_STATISTICS); - metastoreClient.updatePartitionsStatistics(tableName.getSchemaName(), tableName.getTableName(), secondPartitionName, currentStatistics -> ZERO_TABLE_STATISTICS); - } - - protected void testUpdatePartitionStatistics( - SchemaTableName tableName, - PartitionStatistics initialStatistics, - List firstPartitionStatistics, - List secondPartitionStatistics) - { - verify(firstPartitionStatistics.size() == secondPartitionStatistics.size()); - - String firstPartitionName = "ds=2016-01-01"; - String secondPartitionName = "ds=2016-01-02"; - - HiveMetastoreClosure metastoreClient = new HiveMetastoreClosure(getMetastoreClient(), TESTING_TYPE_MANAGER, false); - assertThat(metastoreClient.getPartitionStatistics(tableName.getSchemaName(), tableName.getTableName(), ImmutableSet.of(firstPartitionName, secondPartitionName))) - .isEqualTo(ImmutableMap.of(firstPartitionName, initialStatistics, secondPartitionName, initialStatistics)); - - AtomicReference expectedStatisticsPartition1 = new AtomicReference<>(initialStatistics); - AtomicReference expectedStatisticsPartition2 = new AtomicReference<>(initialStatistics); - - for (int i = 0; i < firstPartitionStatistics.size(); i++) { - PartitionStatistics statisticsPartition1 = firstPartitionStatistics.get(i); - PartitionStatistics statisticsPartition2 = secondPartitionStatistics.get(i); - metastoreClient.updatePartitionsStatistics(tableName.getSchemaName(), tableName.getTableName(), firstPartitionName, actualStatistics -> { - assertThat(actualStatistics).isEqualTo(expectedStatisticsPartition1.get()); - return statisticsPartition1; - }); - metastoreClient.updatePartitionsStatistics(tableName.getSchemaName(), tableName.getTableName(), secondPartitionName, actualStatistics -> { - assertThat(actualStatistics).isEqualTo(expectedStatisticsPartition2.get()); - return statisticsPartition2; - }); - assertThat(metastoreClient.getPartitionStatistics(tableName.getSchemaName(), tableName.getTableName(), ImmutableSet.of(firstPartitionName, secondPartitionName))) - .isEqualTo(ImmutableMap.of(firstPartitionName, statisticsPartition1, secondPartitionName, statisticsPartition2)); - expectedStatisticsPartition1.set(statisticsPartition1); - expectedStatisticsPartition2.set(statisticsPartition2); - } - - assertThat(metastoreClient.getPartitionStatistics(tableName.getSchemaName(), tableName.getTableName(), ImmutableSet.of(firstPartitionName, secondPartitionName))) - .isEqualTo(ImmutableMap.of(firstPartitionName, expectedStatisticsPartition1.get(), secondPartitionName, expectedStatisticsPartition2.get())); - metastoreClient.updatePartitionsStatistics(tableName.getSchemaName(), tableName.getTableName(), firstPartitionName, currentStatistics -> { - assertThat(currentStatistics).isEqualTo(expectedStatisticsPartition1.get()); - return initialStatistics; - }); - metastoreClient.updatePartitionsStatistics(tableName.getSchemaName(), tableName.getTableName(), secondPartitionName, currentStatistics -> { - assertThat(currentStatistics).isEqualTo(expectedStatisticsPartition2.get()); - return initialStatistics; - }); - assertThat(metastoreClient.getPartitionStatistics(tableName.getSchemaName(), tableName.getTableName(), ImmutableSet.of(firstPartitionName, secondPartitionName))) - .isEqualTo(ImmutableMap.of(firstPartitionName, initialStatistics, secondPartitionName, initialStatistics)); - } - - @Test - public void testStorePartitionWithStatistics() - throws Exception - { - testStorePartitionWithStatistics(STATISTICS_PARTITIONED_TABLE_COLUMNS, STATISTICS_1, STATISTICS_2, STATISTICS_1_1, ZERO_TABLE_STATISTICS); - } - - protected void testStorePartitionWithStatistics( - List columns, - PartitionStatistics statsForAllColumns1, - PartitionStatistics statsForAllColumns2, - PartitionStatistics statsForSubsetOfColumns, - PartitionStatistics emptyStatistics) - throws Exception - { - SchemaTableName tableName = temporaryTable("store_partition_with_statistics"); - try { - doCreateEmptyTable(tableName, ORC, columns); - - HiveMetastoreClosure metastoreClient = new HiveMetastoreClosure(getMetastoreClient(), TESTING_TYPE_MANAGER, false); - Table table = metastoreClient.getTable(tableName.getSchemaName(), tableName.getTableName()) - .orElseThrow(() -> new TableNotFoundException(tableName)); - - List partitionValues = ImmutableList.of("2016-01-01"); - String partitionName = makePartName(ImmutableList.of("ds"), partitionValues); - - Partition partition = createDummyPartition(table, partitionName); - - // create partition with stats for all columns - metastoreClient.addPartitions(tableName.getSchemaName(), tableName.getTableName(), ImmutableList.of(new PartitionWithStatistics(partition, partitionName, statsForAllColumns1))); - assertThat(metastoreClient.getPartition(tableName.getSchemaName(), tableName.getTableName(), partitionValues).get().getStorage().getStorageFormat()).isEqualTo(fromHiveStorageFormat(ORC)); - assertThat(metastoreClient.getPartitionStatistics(tableName.getSchemaName(), tableName.getTableName(), ImmutableSet.of(partitionName))) - .isEqualTo(ImmutableMap.of(partitionName, statsForAllColumns1)); - - // alter the partition into one with other stats - Partition modifiedPartition = Partition.builder(partition) - .withStorage(storage -> storage - .setStorageFormat(fromHiveStorageFormat(RCBINARY)) - .setLocation(partitionTargetPath(tableName, partitionName))) - .build(); - metastoreClient.alterPartition(tableName.getSchemaName(), tableName.getTableName(), new PartitionWithStatistics(modifiedPartition, partitionName, statsForAllColumns2)); - assertThat(metastoreClient.getPartition(tableName.getSchemaName(), tableName.getTableName(), partitionValues).get().getStorage().getStorageFormat()).isEqualTo(fromHiveStorageFormat(RCBINARY)); - assertThat(metastoreClient.getPartitionStatistics(tableName.getSchemaName(), tableName.getTableName(), ImmutableSet.of(partitionName))) - .isEqualTo(ImmutableMap.of(partitionName, statsForAllColumns2)); - - // alter the partition into one with stats for only subset of columns - modifiedPartition = Partition.builder(partition) - .withStorage(storage -> storage - .setStorageFormat(fromHiveStorageFormat(TEXTFILE)) - .setLocation(partitionTargetPath(tableName, partitionName))) - .build(); - metastoreClient.alterPartition(tableName.getSchemaName(), tableName.getTableName(), new PartitionWithStatistics(modifiedPartition, partitionName, statsForSubsetOfColumns)); - assertThat(metastoreClient.getPartitionStatistics(tableName.getSchemaName(), tableName.getTableName(), ImmutableSet.of(partitionName))) - .isEqualTo(ImmutableMap.of(partitionName, statsForSubsetOfColumns)); - - // alter the partition into one without stats - modifiedPartition = Partition.builder(partition) - .withStorage(storage -> storage - .setStorageFormat(fromHiveStorageFormat(TEXTFILE)) - .setLocation(partitionTargetPath(tableName, partitionName))) - .build(); - metastoreClient.alterPartition(tableName.getSchemaName(), tableName.getTableName(), new PartitionWithStatistics(modifiedPartition, partitionName, emptyStatistics)); - assertThat(metastoreClient.getPartitionStatistics(tableName.getSchemaName(), tableName.getTableName(), ImmutableSet.of(partitionName))) - .isEqualTo(ImmutableMap.of(partitionName, emptyStatistics)); - } - finally { - dropTable(tableName); - } - } - - protected Partition createDummyPartition(Table table, String partitionName) - { - return Partition.builder() - .setDatabaseName(table.getDatabaseName()) - .setTableName(table.getTableName()) - .setColumns(table.getDataColumns()) - .setValues(toPartitionValues(partitionName)) - .withStorage(storage -> storage - .setStorageFormat(fromHiveStorageFormat(ORC)) - .setLocation(partitionTargetPath(new SchemaTableName(table.getDatabaseName(), table.getTableName()), partitionName))) - .setParameters(ImmutableMap.of( - TRINO_VERSION_NAME, "testversion", - TRINO_QUERY_ID_NAME, "20180101_123456_00001_x1y2z")) - .build(); - } - - protected String partitionTargetPath(SchemaTableName schemaTableName, String partitionName) - { - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - SemiTransactionalHiveMetastore metastore = transaction.getMetastore(); - LocationService locationService = getLocationService(); - Table table = metastore.getTable(schemaTableName.getSchemaName(), schemaTableName.getTableName()).get(); - LocationHandle handle = locationService.forExistingTable(metastore, session, table); - return locationService.getPartitionWriteInfo(handle, Optional.empty(), partitionName).targetPath().toString(); - } - } - - /** - * This test creates 2 identical partitions and verifies that the statistics projected based on - * a single partition sample are equal to the statistics computed in a fair way - */ - @Test - public void testPartitionStatisticsSampling() - throws Exception - { - testPartitionStatisticsSampling(STATISTICS_PARTITIONED_TABLE_COLUMNS, STATISTICS_1); - } - - protected void testPartitionStatisticsSampling(List columns, PartitionStatistics statistics) - throws Exception - { - SchemaTableName tableName = temporaryTable("test_partition_statistics_sampling"); - - try { - createDummyPartitionedTable(tableName, columns); - HiveMetastoreClosure metastoreClient = new HiveMetastoreClosure(getMetastoreClient(), TESTING_TYPE_MANAGER, false); - metastoreClient.updatePartitionsStatistics(tableName.getSchemaName(), tableName.getTableName(), "ds=2016-01-01", actualStatistics -> statistics); - metastoreClient.updatePartitionsStatistics(tableName.getSchemaName(), tableName.getTableName(), "ds=2016-01-02", actualStatistics -> statistics); - - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - ConnectorMetadata metadata = transaction.getMetadata(); - - ConnectorTableHandle tableHandle = metadata.getTableHandle(session, tableName); - TableStatistics unsampledStatistics = metadata.getTableStatistics(sampleSize(2), tableHandle); - TableStatistics sampledStatistics = metadata.getTableStatistics(sampleSize(1), tableHandle); - assertThat(sampledStatistics).isEqualTo(unsampledStatistics); - } - } - finally { - dropTable(tableName); - } - } - - @Test - public void testApplyProjection() - throws Exception - { - ColumnMetadata bigIntColumn0 = new ColumnMetadata("int0", BIGINT); - ColumnMetadata bigIntColumn1 = new ColumnMetadata("int1", BIGINT); - - RowType oneLevelRowType = toRowType(ImmutableList.of(bigIntColumn0, bigIntColumn1)); - ColumnMetadata oneLevelRow0 = new ColumnMetadata("onelevelrow0", oneLevelRowType); - - RowType twoLevelRowType = toRowType(ImmutableList.of(oneLevelRow0, bigIntColumn0, bigIntColumn1)); - ColumnMetadata twoLevelRow0 = new ColumnMetadata("twolevelrow0", twoLevelRowType); - - List columnsForApplyProjectionTest = ImmutableList.of(bigIntColumn0, bigIntColumn1, oneLevelRow0, twoLevelRow0); - - SchemaTableName tableName = temporaryTable("apply_projection_tester"); - doCreateEmptyTable(tableName, ORC, columnsForApplyProjectionTest); - - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); - - List columnHandles = metadata.getColumnHandles(session, tableHandle).values().stream() - .filter(columnHandle -> !((HiveColumnHandle) columnHandle).isHidden()) - .collect(toList()); - assertThat(columnHandles.size()).isEqualTo(columnsForApplyProjectionTest.size()); - - Map columnHandleMap = columnHandles.stream() - .collect(toImmutableMap(handle -> ((HiveColumnHandle) handle).getBaseColumnName(), Function.identity())); - - // Emulate symbols coming from the query plan and map them to column handles - Map columnHandlesWithSymbols = ImmutableMap.of( - "symbol_0", columnHandleMap.get("int0"), - "symbol_1", columnHandleMap.get("int1"), - "symbol_2", columnHandleMap.get("onelevelrow0"), - "symbol_3", columnHandleMap.get("twolevelrow0")); - - // Create variables for the emulated symbols - Map symbolVariableMapping = columnHandlesWithSymbols.entrySet().stream() - .collect(toImmutableMap( - Map.Entry::getKey, - e -> new Variable( - e.getKey(), - ((HiveColumnHandle) e.getValue()).getBaseType()))); - - // Create dereference expressions for testing - FieldDereference symbol2Field0 = new FieldDereference(BIGINT, symbolVariableMapping.get("symbol_2"), 0); - FieldDereference symbol3Field0 = new FieldDereference(oneLevelRowType, symbolVariableMapping.get("symbol_3"), 0); - FieldDereference symbol3Field0Field0 = new FieldDereference(BIGINT, symbol3Field0, 0); - FieldDereference symbol3Field1 = new FieldDereference(BIGINT, symbolVariableMapping.get("symbol_3"), 1); - - Map inputAssignments; - List inputProjections; - Optional> projectionResult; - List expectedProjections; - Map expectedAssignments; - - // Test projected columns pushdown to HiveTableHandle in case of all variable references - inputAssignments = getColumnHandlesFor(columnHandlesWithSymbols, ImmutableList.of("symbol_0", "symbol_1")); - inputProjections = ImmutableList.of(symbolVariableMapping.get("symbol_0"), symbolVariableMapping.get("symbol_1")); - expectedAssignments = ImmutableMap.of( - "symbol_0", BIGINT, - "symbol_1", BIGINT); - projectionResult = metadata.applyProjection(session, tableHandle, inputProjections, inputAssignments); - assertProjectionResult(projectionResult, false, inputProjections, expectedAssignments); - - // Empty result when projected column handles are same as those present in table handle - projectionResult = metadata.applyProjection(session, projectionResult.get().getHandle(), inputProjections, inputAssignments); - assertProjectionResult(projectionResult, true, ImmutableList.of(), ImmutableMap.of()); - - // Extra columns handles in HiveTableHandle should get pruned - projectionResult = metadata.applyProjection( - session, - ((HiveTableHandle) tableHandle).withProjectedColumns(ImmutableSet.copyOf(columnHandles)), - inputProjections, - inputAssignments); - assertProjectionResult(projectionResult, false, inputProjections, expectedAssignments); - - // Test projection pushdown for dereferences - inputAssignments = getColumnHandlesFor(columnHandlesWithSymbols, ImmutableList.of("symbol_2", "symbol_3")); - inputProjections = ImmutableList.of(symbol2Field0, symbol3Field0Field0, symbol3Field1); - expectedAssignments = ImmutableMap.of( - "onelevelrow0#f_int0", BIGINT, - "twolevelrow0#f_onelevelrow0#f_int0", BIGINT, - "twolevelrow0#f_int0", BIGINT); - expectedProjections = ImmutableList.of( - new Variable("onelevelrow0#f_int0", BIGINT), - new Variable("twolevelrow0#f_onelevelrow0#f_int0", BIGINT), - new Variable("twolevelrow0#f_int0", BIGINT)); - projectionResult = metadata.applyProjection(session, tableHandle, inputProjections, inputAssignments); - assertProjectionResult(projectionResult, false, expectedProjections, expectedAssignments); - - // Test reuse of virtual column handles - // Round-1: input projections [symbol_2, symbol_2.int0]. virtual handle is created for symbol_2.int0. - inputAssignments = getColumnHandlesFor(columnHandlesWithSymbols, ImmutableList.of("symbol_2")); - inputProjections = ImmutableList.of(symbol2Field0, symbolVariableMapping.get("symbol_2")); - projectionResult = metadata.applyProjection(session, tableHandle, inputProjections, inputAssignments); - expectedProjections = ImmutableList.of(new Variable("onelevelrow0#f_int0", BIGINT), symbolVariableMapping.get("symbol_2")); - expectedAssignments = ImmutableMap.of("onelevelrow0#f_int0", BIGINT, "symbol_2", oneLevelRowType); - assertProjectionResult(projectionResult, false, expectedProjections, expectedAssignments); - - // Round-2: input projections [symbol_2.int0 and onelevelrow0#f_int0]. Virtual handle is reused. - Assignment newlyCreatedColumn = getOnlyElement(projectionResult.get().getAssignments().stream() - .filter(handle -> handle.getVariable().equals("onelevelrow0#f_int0")) - .collect(toList())); - inputAssignments = ImmutableMap.builder() - .putAll(getColumnHandlesFor(columnHandlesWithSymbols, ImmutableList.of("symbol_2"))) - .put(newlyCreatedColumn.getVariable(), newlyCreatedColumn.getColumn()) - .buildOrThrow(); - inputProjections = ImmutableList.of(symbol2Field0, new Variable("onelevelrow0#f_int0", BIGINT)); - projectionResult = metadata.applyProjection(session, tableHandle, inputProjections, inputAssignments); - expectedProjections = ImmutableList.of(new Variable("onelevelrow0#f_int0", BIGINT), new Variable("onelevelrow0#f_int0", BIGINT)); - expectedAssignments = ImmutableMap.of("onelevelrow0#f_int0", BIGINT); - assertProjectionResult(projectionResult, false, expectedProjections, expectedAssignments); - } - finally { - dropTable(tableName); - } - } - - private static Map getColumnHandlesFor(Map columnHandles, List symbols) - { - return columnHandles.entrySet().stream() - .filter(e -> symbols.contains(e.getKey())) - .collect(toImmutableMap(Map.Entry::getKey, Map.Entry::getValue)); - } - - private static void assertProjectionResult(Optional> projectionResult, boolean shouldBeEmpty, List expectedProjections, Map expectedAssignments) - { - if (shouldBeEmpty) { - assertThat(projectionResult.isEmpty()) - .describedAs("expected projectionResult to be empty") - .isTrue(); - return; - } - - assertThat(projectionResult.isPresent()) - .describedAs("expected non-empty projection result") - .isTrue(); - - ProjectionApplicationResult result = projectionResult.get(); - - // Verify projections - assertThat(expectedProjections).isEqualTo(result.getProjections()); - - // Verify assignments - List assignments = result.getAssignments(); - Map actualAssignments = uniqueIndex(assignments, Assignment::getVariable); - - for (String variable : expectedAssignments.keySet()) { - Type expectedType = expectedAssignments.get(variable); - assertThat(actualAssignments).containsKey(variable); - assertThat(actualAssignments.get(variable).getType()).isEqualTo(expectedType); - assertThat(((HiveColumnHandle) actualAssignments.get(variable).getColumn()).getType()).isEqualTo(expectedType); - } - - assertThat(actualAssignments.size()).isEqualTo(expectedAssignments.size()); - assertThat(actualAssignments.values().stream().map(Assignment::getColumn).collect(toImmutableSet())).isEqualTo(((HiveTableHandle) result.getHandle()).getProjectedColumns()); - } - - @Test - public void testApplyRedirection() - throws Exception - { - SchemaTableName sourceTableName = temporaryTable("apply_redirection_tester"); - doCreateEmptyTable(sourceTableName, ORC, CREATE_TABLE_COLUMNS); - SchemaTableName tableName = temporaryTable("apply_no_redirection_tester"); - doCreateEmptyTable(tableName, ORC, CREATE_TABLE_COLUMNS); - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - ConnectorMetadata metadata = transaction.getMetadata(); - assertThat(metadata.applyTableScanRedirect(session, getTableHandle(metadata, tableName))).isEmpty(); - Optional result = metadata.applyTableScanRedirect(session, getTableHandle(metadata, sourceTableName)); - assertThat(result).isPresent(); - assertThat(result.get().getDestinationTable()) - .isEqualTo(new CatalogSchemaTableName("hive", database, "mock_redirection_target")); - } - finally { - dropTable(sourceTableName); - dropTable(tableName); - } - } - - @Test - public void testMaterializedViewMetadata() - throws Exception - { - SchemaTableName sourceTableName = temporaryTable("materialized_view_tester"); - doCreateEmptyTable(sourceTableName, ORC, CREATE_TABLE_COLUMNS); - SchemaTableName tableName = temporaryTable("mock_table"); - doCreateEmptyTable(tableName, ORC, CREATE_TABLE_COLUMNS); - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - ConnectorMetadata metadata = transaction.getMetadata(); - assertThat(metadata.getMaterializedView(session, tableName)).isEmpty(); - Optional result = metadata.getMaterializedView(session, sourceTableName); - assertThat(result).isPresent(); - assertThat(result.get().getOriginalSql()).isEqualTo("dummy_view_sql"); - } - finally { - dropTable(sourceTableName); - dropTable(tableName); - } - } - - @Test - public void testOrcPageSourceMetrics() - throws Exception - { - SchemaTableName tableName = temporaryTable("orc_page_source_metrics"); - try { - assertPageSourceMetrics(tableName, ORC, new Metrics(ImmutableMap.of(ORC_CODEC_METRIC_PREFIX + "SNAPPY", new LongCount(209)))); - } - finally { - dropTable(tableName); - } - } - - @Test - public void testParquetPageSourceMetrics() - throws Exception - { - SchemaTableName tableName = temporaryTable("parquet_page_source_metrics"); - try { - assertPageSourceMetrics(tableName, PARQUET, new Metrics(ImmutableMap.of(PARQUET_CODEC_METRIC_PREFIX + "SNAPPY", new LongCount(1157)))); - } - finally { - dropTable(tableName); - } - } - - private void assertPageSourceMetrics(SchemaTableName tableName, HiveStorageFormat storageFormat, Metrics expectedMetrics) - throws Exception - { - createEmptyTable( - tableName, - storageFormat, - ImmutableList.of( - new Column("id", HIVE_LONG, Optional.empty(), Map.of()), - new Column("name", HIVE_STRING, Optional.empty(), Map.of())), - ImmutableList.of()); - MaterializedResult.Builder inputDataBuilder = MaterializedResult.resultBuilder(SESSION, BIGINT, VARCHAR); - IntStream.range(0, 100).forEach(i -> inputDataBuilder.row((long) i, String.valueOf(i))); - insertData(tableName, inputDataBuilder.build(), ImmutableMap.of("compression_codec", "SNAPPY")); - - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorSession session = newSession(); - metadata.beginQuery(session); - - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); - - // read entire table - List columnHandles = ImmutableList.copyOf(metadata.getColumnHandles(session, tableHandle).values()); - - List splits = getAllSplits(getSplits(splitManager, transaction, session, tableHandle)); - for (ConnectorSplit split : splits) { - try (ConnectorPageSource pageSource = pageSourceProvider.createPageSource(transaction.getTransactionHandle(), session, split, tableHandle, columnHandles, DynamicFilter.EMPTY)) { - materializeSourceDataStream(session, pageSource, getTypes(columnHandles)); - assertThat(pageSource.getMetrics()).isEqualTo(expectedMetrics); - } - } - } - } - - private ConnectorSession sampleSize(int sampleSize) - { - return getHiveSession(getHiveConfig() - .setPartitionStatisticsSampleSize(sampleSize)); - } - - private void verifyViewCreation(SchemaTableName temporaryCreateView) - { - // replace works for new view - doCreateView(temporaryCreateView, true); - - // replace works for existing view - doCreateView(temporaryCreateView, true); - - // create fails for existing view - try { - doCreateView(temporaryCreateView, false); - fail("create existing should fail"); - } - catch (ViewAlreadyExistsException e) { - assertThat(e.getViewName()).isEqualTo(temporaryCreateView); - } - - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - // drop works when view exists - metadata.dropView(newSession(), temporaryCreateView); - transaction.commit(); - } - - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - assertThat(metadata.getView(newSession(), temporaryCreateView)) - .isEmpty(); - assertThat(metadata.getViews(newSession(), Optional.of(temporaryCreateView.getSchemaName()))) - .doesNotContainKey(temporaryCreateView); - assertThat(metadata.listViews(newSession(), Optional.of(temporaryCreateView.getSchemaName()))) - .doesNotContain(temporaryCreateView); - } - - // drop fails when view does not exist - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - metadata.dropView(newSession(), temporaryCreateView); - fail("drop non-existing should fail"); - } - catch (ViewNotFoundException e) { - assertThat(e.getViewName()).isEqualTo(temporaryCreateView); - } - - // create works for new view - doCreateView(temporaryCreateView, false); - } - - private void doCreateView(SchemaTableName viewName, boolean replace) - { - String viewData = "test data"; - ConnectorViewDefinition definition = new ConnectorViewDefinition( - viewData, - Optional.empty(), - Optional.empty(), - ImmutableList.of(new ViewColumn("test", BIGINT.getTypeId(), Optional.empty())), - Optional.empty(), - Optional.empty(), - true, - ImmutableList.of()); - - try (Transaction transaction = newTransaction()) { - transaction.getMetadata().createView(newSession(), viewName, definition, replace); - transaction.commit(); - } - - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - - assertThat(metadata.getView(newSession(), viewName)) - .map(ConnectorViewDefinition::getOriginalSql) - .contains(viewData); - - Map views = metadata.getViews(newSession(), Optional.of(viewName.getSchemaName())); - assertThat(views.size()).isEqualTo(1); - assertThat(views.get(viewName).getOriginalSql()).isEqualTo(definition.getOriginalSql()); - - assertThat(metadata.listViews(newSession(), Optional.of(viewName.getSchemaName()))).contains(viewName); - } - } - - protected void doCreateTable(SchemaTableName tableName, HiveStorageFormat storageFormat) - throws Exception - { - String queryId; - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - ConnectorMetadata metadata = transaction.getMetadata(); - queryId = session.getQueryId(); - - // begin creating the table - ConnectorTableMetadata tableMetadata = new ConnectorTableMetadata(tableName, CREATE_TABLE_COLUMNS, createTableProperties(storageFormat)); - - ConnectorOutputTableHandle outputHandle = metadata.beginCreateTable(session, tableMetadata, Optional.empty(), NO_RETRIES); - - // write the data - ConnectorPageSink sink = pageSinkProvider.createPageSink(transaction.getTransactionHandle(), session, outputHandle, TESTING_PAGE_SINK_ID); - sink.appendPage(CREATE_TABLE_DATA.toPage()); - Collection fragments = getFutureValue(sink.finish()); - - // verify all new files start with the unique prefix - HdfsContext context = new HdfsContext(session); - for (String filePath : listAllDataFiles(context, getStagingPathRoot(outputHandle))) { - assertThat(new Path(filePath).getName()).startsWith(session.getQueryId()); - } - - // commit the table - metadata.finishCreateTable(session, outputHandle, fragments, ImmutableList.of()); - - transaction.commit(); - } - - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - ConnectorMetadata metadata = transaction.getMetadata(); - metadata.beginQuery(session); - // load the new table - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); - List columnHandles = filterNonHiddenColumnHandles(metadata.getColumnHandles(session, tableHandle).values()); - - // verify the metadata - ConnectorTableMetadata tableMetadata = metadata.getTableMetadata(session, getTableHandle(metadata, tableName)); - assertThat(filterNonHiddenColumnMetadata(tableMetadata.getColumns())).isEqualTo(CREATE_TABLE_COLUMNS); - - // verify the data - MaterializedResult result = readTable(transaction, tableHandle, columnHandles, session, TupleDomain.all(), OptionalInt.empty(), Optional.of(storageFormat)); - assertEqualsIgnoreOrder(result.getMaterializedRows(), CREATE_TABLE_DATA.getMaterializedRows()); - - // verify the node version and query ID in table - Table table = getMetastoreClient().getTable(tableName.getSchemaName(), tableName.getTableName()).get(); - assertThat(table.getParameters()).containsEntry(TRINO_VERSION_NAME, TEST_SERVER_VERSION); - assertThat(table.getParameters()).containsEntry(TRINO_QUERY_ID_NAME, queryId); - - // verify basic statistics - HiveBasicStatistics statistics = getBasicStatisticsForTable(transaction, tableName); - assertThat(statistics.getRowCount().getAsLong()).isEqualTo(CREATE_TABLE_DATA.getRowCount()); - assertThat(statistics.getFileCount().getAsLong()).isEqualTo(1L); - assertGreaterThan(statistics.getInMemoryDataSizeInBytes().getAsLong(), 0L); - assertGreaterThan(statistics.getOnDiskDataSizeInBytes().getAsLong(), 0L); - } - } - - protected void doCreateEmptyTable(SchemaTableName tableName, HiveStorageFormat storageFormat, List createTableColumns) - throws Exception - { - List partitionedBy = createTableColumns.stream() - .map(ColumnMetadata::getName) - .filter(PARTITION_COLUMN_FILTER) - .collect(toList()); - - doCreateEmptyTable(tableName, storageFormat, createTableColumns, partitionedBy); - } - - protected void doCreateEmptyTable(SchemaTableName tableName, HiveStorageFormat storageFormat, List createTableColumns, List partitionedBy) - throws Exception - { - String queryId; - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - ConnectorMetadata metadata = transaction.getMetadata(); - metadata.beginQuery(session); - queryId = session.getQueryId(); - - ConnectorTableMetadata tableMetadata = new ConnectorTableMetadata(tableName, createTableColumns, createTableProperties(storageFormat, partitionedBy)); - metadata.createTable(session, tableMetadata, false); - transaction.commit(); - } - - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - ConnectorMetadata metadata = transaction.getMetadata(); - metadata.beginQuery(session); - - // load the new table - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); - - // verify the metadata - ConnectorTableMetadata tableMetadata = metadata.getTableMetadata(session, getTableHandle(metadata, tableName)); - - List expectedColumns = createTableColumns.stream() - .map(column -> ColumnMetadata.builder() - .setName(column.getName()) - .setType(column.getType()) - .setComment(Optional.ofNullable(column.getComment())) - .setExtraInfo(Optional.ofNullable(columnExtraInfo(partitionedBy.contains(column.getName())))) - .build()) - .collect(toList()); - assertThat(filterNonHiddenColumnMetadata(tableMetadata.getColumns())).isEqualTo(expectedColumns); - - // verify table format - Table table = transaction.getMetastore().getTable(tableName.getSchemaName(), tableName.getTableName()).get(); - assertThat(table.getStorage().getStorageFormat().getInputFormat()).isEqualTo(storageFormat.getInputFormat()); - - // verify the node version and query ID - assertThat(table.getParameters()).containsEntry(TRINO_VERSION_NAME, TEST_SERVER_VERSION); - assertThat(table.getParameters()).containsEntry(TRINO_QUERY_ID_NAME, queryId); - - // verify the table is empty - List columnHandles = filterNonHiddenColumnHandles(metadata.getColumnHandles(session, tableHandle).values()); - MaterializedResult result = readTable(transaction, tableHandle, columnHandles, session, TupleDomain.all(), OptionalInt.empty(), Optional.of(storageFormat)); - assertThat(result.getRowCount()).isEqualTo(0); - - // verify basic statistics - if (partitionedBy.isEmpty()) { - HiveBasicStatistics statistics = getBasicStatisticsForTable(transaction, tableName); - assertThat(statistics.getRowCount().getAsLong()).isEqualTo(0L); - assertThat(statistics.getFileCount().getAsLong()).isEqualTo(0L); - assertThat(statistics.getInMemoryDataSizeInBytes().getAsLong()).isEqualTo(0L); - assertThat(statistics.getOnDiskDataSizeInBytes().getAsLong()).isEqualTo(0L); - } - } - } - - private void doInsert(HiveStorageFormat storageFormat, SchemaTableName tableName) - throws Exception - { - // creating the table - doCreateEmptyTable(tableName, storageFormat, CREATE_TABLE_COLUMNS); - - MaterializedResult.Builder resultBuilder = MaterializedResult.resultBuilder(SESSION, CREATE_TABLE_DATA.getTypes()); - for (int i = 0; i < 3; i++) { - insertData(tableName, CREATE_TABLE_DATA); - - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - ConnectorMetadata metadata = transaction.getMetadata(); - metadata.beginQuery(session); - - // load the new table - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); - List columnHandles = filterNonHiddenColumnHandles(metadata.getColumnHandles(session, tableHandle).values()); - - // verify the metadata - ConnectorTableMetadata tableMetadata = metadata.getTableMetadata(session, getTableHandle(metadata, tableName)); - assertThat(filterNonHiddenColumnMetadata(tableMetadata.getColumns())).isEqualTo(CREATE_TABLE_COLUMNS); - - // verify the data - resultBuilder.rows(CREATE_TABLE_DATA.getMaterializedRows()); - MaterializedResult result = readTable(transaction, tableHandle, columnHandles, session, TupleDomain.all(), OptionalInt.empty(), Optional.empty()); - assertEqualsIgnoreOrder(result.getMaterializedRows(), resultBuilder.build().getMaterializedRows()); - - // statistics - HiveBasicStatistics tableStatistics = getBasicStatisticsForTable(transaction, tableName); - assertThat(tableStatistics.getRowCount().orElse(0)).isEqualTo(CREATE_TABLE_DATA.getRowCount() * (i + 1L)); - assertThat(tableStatistics.getFileCount().getAsLong()).isEqualTo(i + 1L); - assertGreaterThan(tableStatistics.getInMemoryDataSizeInBytes().getAsLong(), 0L); - assertGreaterThan(tableStatistics.getOnDiskDataSizeInBytes().getAsLong(), 0L); - } - } - - // test rollback - Set existingFiles; - try (Transaction transaction = newTransaction()) { - existingFiles = listAllDataFiles(transaction, tableName.getSchemaName(), tableName.getTableName()); - assertThat(existingFiles.isEmpty()).isFalse(); - } - - Location stagingPathRoot; - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - ConnectorMetadata metadata = transaction.getMetadata(); - - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); - - // "stage" insert data - ConnectorInsertTableHandle insertTableHandle = metadata.beginInsert(session, tableHandle, ImmutableList.of(), NO_RETRIES); - ConnectorPageSink sink = pageSinkProvider.createPageSink(transaction.getTransactionHandle(), session, insertTableHandle, TESTING_PAGE_SINK_ID); - sink.appendPage(CREATE_TABLE_DATA.toPage()); - sink.appendPage(CREATE_TABLE_DATA.toPage()); - Collection fragments = getFutureValue(sink.finish()); - metadata.finishInsert(session, insertTableHandle, fragments, ImmutableList.of()); - - // statistics, visible from within transaction - HiveBasicStatistics tableStatistics = getBasicStatisticsForTable(transaction, tableName); - assertThat(tableStatistics.getRowCount().getAsLong()).isEqualTo(CREATE_TABLE_DATA.getRowCount() * 5L); - - try (Transaction otherTransaction = newTransaction()) { - // statistics, not visible from outside transaction - HiveBasicStatistics otherTableStatistics = getBasicStatisticsForTable(otherTransaction, tableName); - assertThat(otherTableStatistics.getRowCount().getAsLong()).isEqualTo(CREATE_TABLE_DATA.getRowCount() * 3L); - } - - // verify we did not modify the table directory - assertThat(listAllDataFiles(transaction, tableName.getSchemaName(), tableName.getTableName())).isEqualTo(existingFiles); - - // verify all temp files start with the unique prefix - stagingPathRoot = getStagingPathRoot(insertTableHandle); - HdfsContext context = new HdfsContext(session); - Set tempFiles = listAllDataFiles(context, stagingPathRoot); - assertThat(!tempFiles.isEmpty()).isTrue(); - for (String filePath : tempFiles) { - assertThat(new Path(filePath).getName()).startsWith(session.getQueryId()); - } - - // rollback insert - transaction.rollback(); - } - - // verify temp directory is empty - HdfsContext context = new HdfsContext(newSession()); - assertThat(listAllDataFiles(context, stagingPathRoot).isEmpty()).isTrue(); - - // verify the data is unchanged - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - ConnectorMetadata metadata = transaction.getMetadata(); - metadata.beginQuery(session); - - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); - List columnHandles = filterNonHiddenColumnHandles(metadata.getColumnHandles(session, tableHandle).values()); - MaterializedResult result = readTable(transaction, tableHandle, columnHandles, session, TupleDomain.all(), OptionalInt.empty(), Optional.empty()); - assertEqualsIgnoreOrder(result.getMaterializedRows(), resultBuilder.build().getMaterializedRows()); - - // verify we did not modify the table directory - assertThat(listAllDataFiles(transaction, tableName.getSchemaName(), tableName.getTableName())).isEqualTo(existingFiles); - } - - // verify statistics unchanged - try (Transaction transaction = newTransaction()) { - HiveBasicStatistics statistics = getBasicStatisticsForTable(transaction, tableName); - assertThat(statistics.getRowCount().getAsLong()).isEqualTo(CREATE_TABLE_DATA.getRowCount() * 3L); - assertThat(statistics.getFileCount().getAsLong()).isEqualTo(3L); - } - } - - private void doInsertOverwriteUnpartitioned(SchemaTableName tableName) - throws Exception - { - // create table with data - doCreateEmptyTable(tableName, ORC, CREATE_TABLE_COLUMNS); - insertData(tableName, CREATE_TABLE_DATA); - - // overwrite table with new data - MaterializedResult.Builder overwriteDataBuilder = MaterializedResult.resultBuilder(SESSION, CREATE_TABLE_DATA.getTypes()); - MaterializedResult overwriteData = null; - - Map overwriteProperties = ImmutableMap.of("insert_existing_partitions_behavior", "OVERWRITE"); - - for (int i = 0; i < 3; i++) { - overwriteDataBuilder.rows(reverse(CREATE_TABLE_DATA.getMaterializedRows())); - overwriteData = overwriteDataBuilder.build(); - - insertData(tableName, overwriteData, overwriteProperties); - - // verify overwrite - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - ConnectorMetadata metadata = transaction.getMetadata(); - metadata.beginQuery(session); - - // load the new table - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); - List columnHandles = filterNonHiddenColumnHandles(metadata.getColumnHandles(session, tableHandle).values()); - - // verify the metadata - ConnectorTableMetadata tableMetadata = metadata.getTableMetadata(session, getTableHandle(metadata, tableName)); - assertThat(filterNonHiddenColumnMetadata(tableMetadata.getColumns())).isEqualTo(CREATE_TABLE_COLUMNS); - - // verify the data - MaterializedResult result = readTable(transaction, tableHandle, columnHandles, session, TupleDomain.all(), OptionalInt.empty(), Optional.empty()); - assertEqualsIgnoreOrder(result.getMaterializedRows(), overwriteData.getMaterializedRows()); - - // statistics - HiveBasicStatistics tableStatistics = getBasicStatisticsForTable(transaction, tableName); - assertThat(tableStatistics.getRowCount().getAsLong()).isEqualTo(overwriteData.getRowCount()); - assertThat(tableStatistics.getFileCount().getAsLong()).isEqualTo(1L); - assertGreaterThan(tableStatistics.getInMemoryDataSizeInBytes().getAsLong(), 0L); - assertGreaterThan(tableStatistics.getOnDiskDataSizeInBytes().getAsLong(), 0L); - } - } - - // test rollback - Set existingFiles; - try (Transaction transaction = newTransaction()) { - existingFiles = listAllDataFiles(transaction, tableName.getSchemaName(), tableName.getTableName()); - assertThat(existingFiles.isEmpty()).isFalse(); - } - - Location stagingPathRoot; - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(overwriteProperties); - ConnectorMetadata metadata = transaction.getMetadata(); - - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); - - // "stage" insert data - ConnectorInsertTableHandle insertTableHandle = metadata.beginInsert(session, tableHandle, ImmutableList.of(), NO_RETRIES); - ConnectorPageSink sink = pageSinkProvider.createPageSink(transaction.getTransactionHandle(), session, insertTableHandle, TESTING_PAGE_SINK_ID); - for (int i = 0; i < 4; i++) { - sink.appendPage(overwriteData.toPage()); - } - Collection fragments = getFutureValue(sink.finish()); - metadata.finishInsert(session, insertTableHandle, fragments, ImmutableList.of()); - - // statistics, visible from within transaction - HiveBasicStatistics tableStatistics = getBasicStatisticsForTable(transaction, tableName); - assertThat(tableStatistics.getRowCount().getAsLong()).isEqualTo(overwriteData.getRowCount() * 4L); - - try (Transaction otherTransaction = newTransaction()) { - // statistics, not visible from outside transaction - HiveBasicStatistics otherTableStatistics = getBasicStatisticsForTable(otherTransaction, tableName); - assertThat(otherTableStatistics.getRowCount().getAsLong()).isEqualTo(overwriteData.getRowCount()); - } - - // verify we did not modify the table directory - assertThat(listAllDataFiles(transaction, tableName.getSchemaName(), tableName.getTableName())).isEqualTo(existingFiles); - - // verify all temp files start with the unique prefix - stagingPathRoot = getStagingPathRoot(insertTableHandle); - HdfsContext context = new HdfsContext(session); - Set tempFiles = listAllDataFiles(context, stagingPathRoot); - assertThat(!tempFiles.isEmpty()).isTrue(); - for (String filePath : tempFiles) { - assertThat(new Path(filePath).getName()).startsWith(session.getQueryId()); - } - - // rollback insert - transaction.rollback(); - } - - // verify temp directory is empty - HdfsContext context = new HdfsContext(newSession()); - assertThat(listAllDataFiles(context, stagingPathRoot).isEmpty()).isTrue(); - - // verify the data is unchanged - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - ConnectorMetadata metadata = transaction.getMetadata(); - metadata.beginQuery(session); - - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); - List columnHandles = filterNonHiddenColumnHandles(metadata.getColumnHandles(session, tableHandle).values()); - MaterializedResult result = readTable(transaction, tableHandle, columnHandles, session, TupleDomain.all(), OptionalInt.empty(), Optional.empty()); - assertEqualsIgnoreOrder(result.getMaterializedRows(), overwriteData.getMaterializedRows()); - - // verify we did not modify the table directory - assertThat(listAllDataFiles(transaction, tableName.getSchemaName(), tableName.getTableName())).isEqualTo(existingFiles); - } - - // verify statistics unchanged - try (Transaction transaction = newTransaction()) { - HiveBasicStatistics statistics = getBasicStatisticsForTable(transaction, tableName); - assertThat(statistics.getRowCount().getAsLong()).isEqualTo(overwriteData.getRowCount()); - assertThat(statistics.getFileCount().getAsLong()).isEqualTo(1L); - } - } - - private Location getStagingPathRoot(ConnectorInsertTableHandle insertTableHandle) - { - HiveInsertTableHandle handle = (HiveInsertTableHandle) insertTableHandle; - WriteInfo writeInfo = getLocationService().getQueryWriteInfo(handle.getLocationHandle()); - if (writeInfo.writeMode() != STAGE_AND_MOVE_TO_TARGET_DIRECTORY) { - throw new AssertionError("writeMode is not STAGE_AND_MOVE_TO_TARGET_DIRECTORY"); - } - return writeInfo.writePath(); - } - - private Location getStagingPathRoot(ConnectorOutputTableHandle outputTableHandle) - { - HiveOutputTableHandle handle = (HiveOutputTableHandle) outputTableHandle; - return getLocationService() - .getQueryWriteInfo(handle.getLocationHandle()) - .writePath(); - } - - private Location getTargetPathRoot(ConnectorInsertTableHandle insertTableHandle) - { - HiveInsertTableHandle hiveInsertTableHandle = (HiveInsertTableHandle) insertTableHandle; - - return getLocationService() - .getQueryWriteInfo(hiveInsertTableHandle.getLocationHandle()) - .targetPath(); - } - - protected Set listAllDataFiles(Transaction transaction, String schemaName, String tableName) - throws IOException - { - HdfsContext hdfsContext = new HdfsContext(newSession()); - Set existingFiles = new HashSet<>(); - for (String location : listAllDataPaths(transaction.getMetastore(), schemaName, tableName)) { - existingFiles.addAll(listAllDataFiles(hdfsContext, Location.of(location))); - } - return existingFiles; - } - - public static List listAllDataPaths(SemiTransactionalHiveMetastore metastore, String schemaName, String tableName) - { - ImmutableList.Builder locations = ImmutableList.builder(); - Table table = metastore.getTable(schemaName, tableName).get(); - if (table.getStorage().getLocation() != null) { - // For partitioned table, there should be nothing directly under this directory. - // But including this location in the set makes the directory content assert more - // extensive, which is desirable. - locations.add(table.getStorage().getLocation()); - } - - Optional> partitionNames = metastore.getPartitionNames(schemaName, tableName); - if (partitionNames.isPresent()) { - metastore.getPartitionsByNames(schemaName, tableName, partitionNames.get()).values().stream() - .map(Optional::get) - .map(partition -> partition.getStorage().getLocation()) - .filter(location -> !location.startsWith(table.getStorage().getLocation())) - .forEach(locations::add); - } - - return locations.build(); - } - - protected Set listAllDataFiles(HdfsContext context, Location location) - throws IOException - { - Path path = new Path(location.toString()); - Set result = new HashSet<>(); - FileSystem fileSystem = hdfsEnvironment.getFileSystem(context, path); - if (fileSystem.exists(path)) { - for (FileStatus fileStatus : fileSystem.listStatus(path)) { - if (fileStatus.getPath().getName().startsWith(".trino")) { - // skip hidden files - } - else if (fileStatus.isFile()) { - result.add(fileStatus.getPath().toString()); - } - else if (fileStatus.isDirectory()) { - result.addAll(listAllDataFiles(context, Location.of(fileStatus.getPath().toString()))); - } - } - } - return result; - } - - private void doInsertIntoNewPartition(HiveStorageFormat storageFormat, SchemaTableName tableName) - throws Exception - { - // creating the table - doCreateEmptyTable(tableName, storageFormat, CREATE_TABLE_COLUMNS_PARTITIONED); - - // insert the data - String queryId = insertData(tableName, CREATE_TABLE_PARTITIONED_DATA); - - Set existingFiles; - try (Transaction transaction = newTransaction()) { - // verify partitions were created - Table table = metastoreClient.getTable(tableName.getSchemaName(), tableName.getTableName()) - .orElseThrow(() -> new TableNotFoundException(tableName)); - List partitionNames = transaction.getMetastore().getPartitionNames(tableName.getSchemaName(), tableName.getTableName()) - .orElseThrow(() -> new AssertionError("Table does not exist: " + tableName)); - assertEqualsIgnoreOrder(partitionNames, CREATE_TABLE_PARTITIONED_DATA.getMaterializedRows().stream() - .map(row -> "ds=" + row.getField(CREATE_TABLE_PARTITIONED_DATA.getTypes().size() - 1)) - .collect(toImmutableList())); - - // verify the node versions in partitions - Map> partitions = getMetastoreClient().getPartitionsByNames(table, partitionNames); - assertThat(partitions.size()).isEqualTo(partitionNames.size()); - for (String partitionName : partitionNames) { - Partition partition = partitions.get(partitionName).get(); - assertThat(partition.getParameters()).containsEntry(TRINO_VERSION_NAME, TEST_SERVER_VERSION); - assertThat(partition.getParameters()).containsEntry(TRINO_QUERY_ID_NAME, queryId); - } - - // load the new table - ConnectorSession session = newSession(); - ConnectorMetadata metadata = transaction.getMetadata(); - metadata.beginQuery(session); - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); - List columnHandles = filterNonHiddenColumnHandles(metadata.getColumnHandles(session, tableHandle).values()); - - // verify the data - MaterializedResult result = readTable(transaction, tableHandle, columnHandles, session, TupleDomain.all(), OptionalInt.empty(), Optional.of(storageFormat)); - assertEqualsIgnoreOrder(result.getMaterializedRows(), CREATE_TABLE_PARTITIONED_DATA.getMaterializedRows()); - - // test rollback - existingFiles = listAllDataFiles(transaction, tableName.getSchemaName(), tableName.getTableName()); - assertThat(existingFiles.isEmpty()).isFalse(); - - // test statistics - for (String partitionName : partitionNames) { - HiveBasicStatistics partitionStatistics = getBasicStatisticsForPartition(transaction, tableName, COLUMN_NAMES_PARTITIONED, partitionName); - assertThat(partitionStatistics.getRowCount().getAsLong()).isEqualTo(1L); - assertThat(partitionStatistics.getFileCount().getAsLong()).isEqualTo(1L); - assertGreaterThan(partitionStatistics.getInMemoryDataSizeInBytes().getAsLong(), 0L); - assertGreaterThan(partitionStatistics.getOnDiskDataSizeInBytes().getAsLong(), 0L); - } - } - - Location stagingPathRoot; - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); - - // "stage" insert data - ConnectorInsertTableHandle insertTableHandle = metadata.beginInsert(session, tableHandle, ImmutableList.of(), NO_RETRIES); - stagingPathRoot = getStagingPathRoot(insertTableHandle); - ConnectorPageSink sink = pageSinkProvider.createPageSink(transaction.getTransactionHandle(), session, insertTableHandle, TESTING_PAGE_SINK_ID); - sink.appendPage(CREATE_TABLE_PARTITIONED_DATA_2ND.toPage()); - Collection fragments = getFutureValue(sink.finish()); - metadata.finishInsert(session, insertTableHandle, fragments, ImmutableList.of()); - - // verify all temp files start with the unique prefix - HdfsContext context = new HdfsContext(session); - Set tempFiles = listAllDataFiles(context, getStagingPathRoot(insertTableHandle)); - assertThat(!tempFiles.isEmpty()).isTrue(); - for (String filePath : tempFiles) { - assertThat(new Path(filePath).getName()).startsWith(session.getQueryId()); - } - - // rollback insert - transaction.rollback(); - } - - // verify the data is unchanged - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - ConnectorMetadata metadata = transaction.getMetadata(); - metadata.beginQuery(session); - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); - List columnHandles = filterNonHiddenColumnHandles(metadata.getColumnHandles(session, tableHandle).values()); - - MaterializedResult result = readTable(transaction, tableHandle, columnHandles, session, TupleDomain.all(), OptionalInt.empty(), Optional.empty()); - assertEqualsIgnoreOrder(result.getMaterializedRows(), CREATE_TABLE_PARTITIONED_DATA.getMaterializedRows()); - - // verify we did not modify the table directory - assertThat(listAllDataFiles(transaction, tableName.getSchemaName(), tableName.getTableName())).isEqualTo(existingFiles); - - // verify temp directory is empty - HdfsContext context = new HdfsContext(session); - assertThat(listAllDataFiles(context, stagingPathRoot).isEmpty()).isTrue(); - } - } - - private void doInsertUnsupportedWriteType(HiveStorageFormat storageFormat, SchemaTableName tableName) - throws Exception - { - List columns = ImmutableList.of(new Column("dummy", HiveType.valueOf("uniontype"), Optional.empty(), Map.of())); - List partitionColumns = ImmutableList.of(new Column("name", HIVE_STRING, Optional.empty(), Map.of())); - - createEmptyTable(tableName, storageFormat, columns, partitionColumns); - - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorSession session = newSession(); - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); - - metadata.beginInsert(session, tableHandle, ImmutableList.of(), NO_RETRIES); - fail("expected failure"); - } - catch (TrinoException e) { - assertThat(e).hasMessageMatching("Inserting into Hive table .* with column type uniontype not supported"); - } - } - - private void doInsertIntoExistingPartition(HiveStorageFormat storageFormat, SchemaTableName tableName) - throws Exception - { - // creating the table - doCreateEmptyTable(tableName, storageFormat, CREATE_TABLE_COLUMNS_PARTITIONED); - - MaterializedResult.Builder resultBuilder = MaterializedResult.resultBuilder(SESSION, CREATE_TABLE_PARTITIONED_DATA.getTypes()); - for (int i = 0; i < 3; i++) { - // insert the data - insertData(tableName, CREATE_TABLE_PARTITIONED_DATA); - - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - ConnectorMetadata metadata = transaction.getMetadata(); - metadata.beginQuery(session); - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); - - // verify partitions were created - List partitionNames = transaction.getMetastore().getPartitionNames(tableName.getSchemaName(), tableName.getTableName()) - .orElseThrow(() -> new AssertionError("Table does not exist: " + tableName)); - assertEqualsIgnoreOrder(partitionNames, CREATE_TABLE_PARTITIONED_DATA.getMaterializedRows().stream() - .map(row -> "ds=" + row.getField(CREATE_TABLE_PARTITIONED_DATA.getTypes().size() - 1)) - .collect(toImmutableList())); - - // load the new table - List columnHandles = filterNonHiddenColumnHandles(metadata.getColumnHandles(session, tableHandle).values()); - - // verify the data - resultBuilder.rows(CREATE_TABLE_PARTITIONED_DATA.getMaterializedRows()); - MaterializedResult result = readTable(transaction, tableHandle, columnHandles, session, TupleDomain.all(), OptionalInt.empty(), Optional.of(storageFormat)); - assertEqualsIgnoreOrder(result.getMaterializedRows(), resultBuilder.build().getMaterializedRows()); - - // test statistics - for (String partitionName : partitionNames) { - HiveBasicStatistics statistics = getBasicStatisticsForPartition(transaction, tableName, COLUMN_NAMES_PARTITIONED, partitionName); - assertThat(statistics.getRowCount().getAsLong()).isEqualTo(i + 1L); - assertThat(statistics.getFileCount().getAsLong()).isEqualTo(i + 1L); - assertGreaterThan(statistics.getInMemoryDataSizeInBytes().getAsLong(), 0L); - assertGreaterThan(statistics.getOnDiskDataSizeInBytes().getAsLong(), 0L); - } - } - } - - // test rollback - Set existingFiles; - Location stagingPathRoot; - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorSession session = newSession(); - - existingFiles = listAllDataFiles(transaction, tableName.getSchemaName(), tableName.getTableName()); - assertThat(existingFiles.isEmpty()).isFalse(); - - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); - - // "stage" insert data - ConnectorInsertTableHandle insertTableHandle = metadata.beginInsert(session, tableHandle, ImmutableList.of(), NO_RETRIES); - stagingPathRoot = getStagingPathRoot(insertTableHandle); - ConnectorPageSink sink = pageSinkProvider.createPageSink(transaction.getTransactionHandle(), session, insertTableHandle, TESTING_PAGE_SINK_ID); - sink.appendPage(CREATE_TABLE_PARTITIONED_DATA.toPage()); - sink.appendPage(CREATE_TABLE_PARTITIONED_DATA.toPage()); - Collection fragments = getFutureValue(sink.finish()); - metadata.finishInsert(session, insertTableHandle, fragments, ImmutableList.of()); - - // verify all temp files start with the unique prefix - HdfsContext context = new HdfsContext(session); - Set tempFiles = listAllDataFiles(context, getStagingPathRoot(insertTableHandle)); - assertThat(!tempFiles.isEmpty()).isTrue(); - for (String filePath : tempFiles) { - assertThat(new Path(filePath).getName()).startsWith(session.getQueryId()); - } - - // verify statistics are visible from within of the current transaction - List partitionNames = transaction.getMetastore().getPartitionNames(tableName.getSchemaName(), tableName.getTableName()) - .orElseThrow(() -> new AssertionError("Table does not exist: " + tableName)); - for (String partitionName : partitionNames) { - HiveBasicStatistics partitionStatistics = getBasicStatisticsForPartition(transaction, tableName, COLUMN_NAMES_PARTITIONED, partitionName); - assertThat(partitionStatistics.getRowCount().getAsLong()).isEqualTo(5L); - } - - // rollback insert - transaction.rollback(); - } - - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorSession session = newSession(); - metadata.beginQuery(session); - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); - List columnHandles = filterNonHiddenColumnHandles(metadata.getColumnHandles(session, tableHandle).values()); - - // verify the data is unchanged - MaterializedResult result = readTable(transaction, tableHandle, columnHandles, session, TupleDomain.all(), OptionalInt.empty(), Optional.empty()); - assertEqualsIgnoreOrder(result.getMaterializedRows(), resultBuilder.build().getMaterializedRows()); - - // verify we did not modify the table directory - assertThat(listAllDataFiles(transaction, tableName.getSchemaName(), tableName.getTableName())).isEqualTo(existingFiles); - - // verify temp directory is empty - HdfsContext hdfsContext = new HdfsContext(session); - assertThat(listAllDataFiles(hdfsContext, stagingPathRoot).isEmpty()).isTrue(); - - // verify statistics have been rolled back - List partitionNames = transaction.getMetastore().getPartitionNames(tableName.getSchemaName(), tableName.getTableName()) - .orElseThrow(() -> new AssertionError("Table does not exist: " + tableName)); - for (String partitionName : partitionNames) { - HiveBasicStatistics partitionStatistics = getBasicStatisticsForPartition(transaction, tableName, COLUMN_NAMES_PARTITIONED, partitionName); - assertThat(partitionStatistics.getRowCount().getAsLong()).isEqualTo(3L); - } - } - } - - private void doInsertIntoExistingPartitionEmptyStatistics(HiveStorageFormat storageFormat, SchemaTableName tableName) - throws Exception - { - doCreateEmptyTable(tableName, storageFormat, CREATE_TABLE_COLUMNS_PARTITIONED); - insertData(tableName, CREATE_TABLE_PARTITIONED_DATA); - - eraseStatistics(tableName); - - insertData(tableName, CREATE_TABLE_PARTITIONED_DATA); - - try (Transaction transaction = newTransaction()) { - List partitionNames = transaction.getMetastore().getPartitionNames(tableName.getSchemaName(), tableName.getTableName()) - .orElseThrow(() -> new AssertionError("Table does not exist: " + tableName)); - - for (String partitionName : partitionNames) { - HiveBasicStatistics statistics = getBasicStatisticsForPartition(transaction, tableName, COLUMN_NAMES_PARTITIONED, partitionName); - assertThat(statistics.getRowCount()).isNotPresent(); - assertThat(statistics.getInMemoryDataSizeInBytes()).isNotPresent(); - // fileCount and rawSize statistics are computed on the fly by the metastore, thus cannot be erased - } - } - } - - private static HiveBasicStatistics getBasicStatisticsForTable(Transaction transaction, SchemaTableName table) - { - return transaction - .getMetastore() - .getTableStatistics(table.getSchemaName(), table.getTableName(), Optional.empty()) - .getBasicStatistics(); - } - - private static HiveBasicStatistics getBasicStatisticsForPartition(Transaction transaction, SchemaTableName table, Set columns, String partitionName) - { - return transaction - .getMetastore() - .getPartitionStatistics(table.getSchemaName(), table.getTableName(), columns, ImmutableSet.of(partitionName)) - .get(partitionName) - .getBasicStatistics(); - } - - private void eraseStatistics(SchemaTableName schemaTableName) - { - HiveMetastore metastoreClient = getMetastoreClient(); - metastoreClient.updateTableStatistics(schemaTableName.getSchemaName(), schemaTableName.getTableName(), NO_ACID_TRANSACTION, statistics -> new PartitionStatistics(createEmptyStatistics(), ImmutableMap.of())); - Table table = metastoreClient.getTable(schemaTableName.getSchemaName(), schemaTableName.getTableName()) - .orElseThrow(() -> new TableNotFoundException(schemaTableName)); - List partitionColumns = table.getPartitionColumns().stream() - .map(Column::getName) - .collect(toImmutableList()); - if (!table.getPartitionColumns().isEmpty()) { - List partitionNames = metastoreClient.getPartitionNamesByFilter(schemaTableName.getSchemaName(), schemaTableName.getTableName(), partitionColumns, TupleDomain.all()) - .orElse(ImmutableList.of()); - List partitions = metastoreClient - .getPartitionsByNames(table, partitionNames) - .values() - .stream() - .filter(Optional::isPresent) - .map(Optional::get) - .collect(toImmutableList()); - for (Partition partition : partitions) { - metastoreClient.updatePartitionsStatistics( - table, - makePartName(partitionColumns, partition.getValues()), - statistics -> new PartitionStatistics(createEmptyStatistics(), ImmutableMap.of())); - } - } - } - - /** - * @return query id - */ - private String insertData(SchemaTableName tableName, MaterializedResult data) - throws Exception - { - return insertData(tableName, data, ImmutableMap.of()); - } - - private String insertData(SchemaTableName tableName, MaterializedResult data, Map sessionProperties) - throws Exception - { - Location writePath; - Location targetPath; - String queryId; - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorSession session = newSession(sessionProperties); - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); - ConnectorInsertTableHandle insertTableHandle = metadata.beginInsert(session, tableHandle, ImmutableList.of(), NO_RETRIES); - queryId = session.getQueryId(); - writePath = getStagingPathRoot(insertTableHandle); - targetPath = getTargetPathRoot(insertTableHandle); - - ConnectorPageSink sink = pageSinkProvider.createPageSink(transaction.getTransactionHandle(), session, insertTableHandle, TESTING_PAGE_SINK_ID); - - // write data - sink.appendPage(data.toPage()); - Collection fragments = getFutureValue(sink.finish()); - - // commit the insert - metadata.finishInsert(session, insertTableHandle, fragments, ImmutableList.of()); - transaction.commit(); - } - - // check that temporary files are removed - if (!writePath.equals(targetPath)) { - HdfsContext context = new HdfsContext(newSession()); - FileSystem fileSystem = hdfsEnvironment.getFileSystem(context, new Path(writePath.toString())); - assertThat(fileSystem.exists(new Path(writePath.toString()))).isFalse(); - } - - return queryId; - } - - private void doTestMetadataDelete(HiveStorageFormat storageFormat, SchemaTableName tableName) - throws Exception - { - // creating the table - doCreateEmptyTable(tableName, storageFormat, CREATE_TABLE_COLUMNS_PARTITIONED); - - insertData(tableName, CREATE_TABLE_PARTITIONED_DATA); - - MaterializedResult.Builder expectedResultBuilder = MaterializedResult.resultBuilder(SESSION, CREATE_TABLE_PARTITIONED_DATA.getTypes()); - expectedResultBuilder.rows(CREATE_TABLE_PARTITIONED_DATA.getMaterializedRows()); - - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - ConnectorMetadata metadata = transaction.getMetadata(); - metadata.beginQuery(session); - - // verify partitions were created - List partitionNames = transaction.getMetastore().getPartitionNames(tableName.getSchemaName(), tableName.getTableName()) - .orElseThrow(() -> new AssertionError("Table does not exist: " + tableName)); - assertEqualsIgnoreOrder(partitionNames, CREATE_TABLE_PARTITIONED_DATA.getMaterializedRows().stream() - .map(row -> "ds=" + row.getField(CREATE_TABLE_PARTITIONED_DATA.getTypes().size() - 1)) - .collect(toImmutableList())); - - // verify table directory is not empty - Set filesAfterInsert = listAllDataFiles(transaction, tableName.getSchemaName(), tableName.getTableName()); - assertThat(filesAfterInsert.isEmpty()).isFalse(); - - // verify the data - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); - List columnHandles = filterNonHiddenColumnHandles(metadata.getColumnHandles(session, tableHandle).values()); - MaterializedResult result = readTable(transaction, tableHandle, columnHandles, session, TupleDomain.all(), OptionalInt.empty(), Optional.of(storageFormat)); - assertEqualsIgnoreOrder(result.getMaterializedRows(), expectedResultBuilder.build().getMaterializedRows()); - } - - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - ConnectorMetadata metadata = transaction.getMetadata(); - - // get ds column handle - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); - HiveColumnHandle dsColumnHandle = (HiveColumnHandle) metadata.getColumnHandles(session, tableHandle).get("ds"); - - // delete ds=2015-07-03 - session = newSession(); - TupleDomain tupleDomain = TupleDomain.fromFixedValues(ImmutableMap.of(dsColumnHandle, NullableValue.of(createUnboundedVarcharType(), utf8Slice("2015-07-03")))); - Constraint constraint = new Constraint(tupleDomain, tupleDomain.asPredicate(), tupleDomain.getDomains().orElseThrow().keySet()); - tableHandle = applyFilter(metadata, tableHandle, constraint); - tableHandle = metadata.applyDelete(session, tableHandle).get(); - metadata.executeDelete(session, tableHandle); - - transaction.commit(); - } - - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - ConnectorMetadata metadata = transaction.getMetadata(); - metadata.beginQuery(session); - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); - List columnHandles = filterNonHiddenColumnHandles(metadata.getColumnHandles(session, tableHandle).values()); - HiveColumnHandle dsColumnHandle = (HiveColumnHandle) metadata.getColumnHandles(session, tableHandle).get("ds"); - int dsColumnOrdinalPosition = columnHandles.indexOf(dsColumnHandle); - - // verify the data - ImmutableList expectedRows = expectedResultBuilder.build().getMaterializedRows().stream() - .filter(row -> !"2015-07-03".equals(row.getField(dsColumnOrdinalPosition))) - .collect(toImmutableList()); - MaterializedResult actualAfterDelete = readTable(transaction, tableHandle, columnHandles, session, TupleDomain.all(), OptionalInt.empty(), Optional.of(storageFormat)); - assertEqualsIgnoreOrder(actualAfterDelete.getMaterializedRows(), expectedRows); - } - - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); - HiveColumnHandle dsColumnHandle = (HiveColumnHandle) metadata.getColumnHandles(session, tableHandle).get("ds"); - - // delete ds=2015-07-01 and 2015-07-02 - session = newSession(); - TupleDomain tupleDomain2 = TupleDomain.withColumnDomains( - ImmutableMap.of(dsColumnHandle, Domain.create(ValueSet.ofRanges(Range.range(createUnboundedVarcharType(), utf8Slice("2015-07-01"), true, utf8Slice("2015-07-02"), true)), false))); - Constraint constraint2 = new Constraint(tupleDomain2, tupleDomain2.asPredicate(), tupleDomain2.getDomains().orElseThrow().keySet()); - tableHandle = applyFilter(metadata, tableHandle, constraint2); - tableHandle = metadata.applyDelete(session, tableHandle).get(); - metadata.executeDelete(session, tableHandle); - - transaction.commit(); - } - - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); - List columnHandles = ImmutableList.copyOf(metadata.getColumnHandles(session, tableHandle).values()); - - // verify the data - session = newSession(); - MaterializedResult actualAfterDelete2 = readTable(transaction, tableHandle, columnHandles, session, TupleDomain.all(), OptionalInt.empty(), Optional.of(storageFormat)); - assertEqualsIgnoreOrder(actualAfterDelete2.getMaterializedRows(), ImmutableList.of()); - - // verify table directory is empty - Set filesAfterDelete = listAllDataFiles(transaction, tableName.getSchemaName(), tableName.getTableName()); - assertThat(filesAfterDelete.isEmpty()).isTrue(); - } - } - - protected void assertGetRecords(String tableName, HiveStorageFormat hiveStorageFormat) - throws Exception - { - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - ConnectorMetadata metadata = transaction.getMetadata(); - metadata.beginQuery(session); - - ConnectorTableHandle tableHandle = getTableHandle(metadata, new SchemaTableName(database, tableName)); - ConnectorTableMetadata tableMetadata = metadata.getTableMetadata(session, tableHandle); - HiveSplit hiveSplit = getHiveSplit(tableHandle, transaction, session); - - List columnHandles = ImmutableList.copyOf(metadata.getColumnHandles(session, tableHandle).values()); - - ConnectorPageSource pageSource = pageSourceProvider.createPageSource(transaction.getTransactionHandle(), session, hiveSplit, tableHandle, columnHandles, DynamicFilter.EMPTY); - assertGetRecords(hiveStorageFormat, tableMetadata, hiveSplit, pageSource, columnHandles); - } - } - - protected HiveSplit getHiveSplit(ConnectorTableHandle tableHandle, Transaction transaction, ConnectorSession session) - { - List splits = getAllSplits(tableHandle, transaction, session); - assertThat(splits.size()).isEqualTo(1); - return (HiveSplit) getOnlyElement(splits); - } - - protected void assertGetRecords( - HiveStorageFormat hiveStorageFormat, - ConnectorTableMetadata tableMetadata, - HiveSplit hiveSplit, - ConnectorPageSource pageSource, - List columnHandles) - throws IOException - { - try { - MaterializedResult result = materializeSourceDataStream(newSession(), pageSource, getTypes(columnHandles)); - - assertPageSourceType(pageSource, hiveStorageFormat); - - ImmutableMap columnIndex = indexColumns(tableMetadata); - - long rowNumber = 0; - long completedBytes = 0; - for (MaterializedRow row : result) { - try { - assertValueTypes(row, tableMetadata.getColumns()); - } - catch (RuntimeException e) { - throw new RuntimeException("row " + rowNumber, e); - } - - rowNumber++; - Integer index; - Object value; - - // STRING - index = columnIndex.get("t_string"); - value = row.getField(index); - if (rowNumber % 19 == 0) { - assertThat(value).isNull(); - } - else if (rowNumber % 19 == 1) { - assertThat(value).isEqualTo(""); - } - else { - assertThat(value).isEqualTo("test"); - } - - // NUMBERS - assertThat(row.getField(columnIndex.get("t_tinyint"))).isEqualTo((byte) (1 + rowNumber)); - assertThat(row.getField(columnIndex.get("t_smallint"))).isEqualTo((short) (2 + rowNumber)); - assertThat(row.getField(columnIndex.get("t_int"))).isEqualTo((int) (3 + rowNumber)); - - index = columnIndex.get("t_bigint"); - if ((rowNumber % 13) == 0) { - assertThat(row.getField(index)).isNull(); - } - else { - assertThat(row.getField(index)).isEqualTo(4 + rowNumber); - } - - assertThat((Float) row.getField(columnIndex.get("t_float"))).isCloseTo(5.1f + rowNumber, offset(0.001f)); - assertThat(row.getField(columnIndex.get("t_double"))).isEqualTo(6.2 + rowNumber); - - // BOOLEAN - index = columnIndex.get("t_boolean"); - if ((rowNumber % 3) == 2) { - assertThat(row.getField(index)).isNull(); - } - else { - assertThat(row.getField(index)).isEqualTo((rowNumber % 3) != 0); - } - - // TIMESTAMP - index = columnIndex.get("t_timestamp"); - if (index != null) { - if ((rowNumber % 17) == 0) { - assertThat(row.getField(index)).isNull(); - } - else { - SqlTimestamp expected = sqlTimestampOf(3, 2011, 5, 6, 7, 8, 9, 123); - assertThat(row.getField(index)).isEqualTo(expected); - } - } - - // BINARY - index = columnIndex.get("t_binary"); - if (index != null) { - if ((rowNumber % 23) == 0) { - assertThat(row.getField(index)).isNull(); - } - else { - assertThat(row.getField(index)).isEqualTo(new SqlVarbinary("test binary".getBytes(UTF_8))); - } - } - - // DATE - index = columnIndex.get("t_date"); - if (index != null) { - if ((rowNumber % 37) == 0) { - assertThat(row.getField(index)).isNull(); - } - else { - SqlDate expected = new SqlDate(toIntExact(MILLISECONDS.toDays(new DateTime(2013, 8, 9, 0, 0, 0, UTC).getMillis()))); - assertThat(row.getField(index)).isEqualTo(expected); - } - } - - // VARCHAR(50) - index = columnIndex.get("t_varchar"); - if (index != null) { - value = row.getField(index); - if (rowNumber % 39 == 0) { - assertThat(value).isNull(); - } - else if (rowNumber % 39 == 1) { - // https://issues.apache.org/jira/browse/HIVE-13289 - // RCBINARY reads empty VARCHAR as null - if (hiveStorageFormat == RCBINARY) { - assertThat(value).isNull(); - } - else { - assertThat(value).isEqualTo(""); - } - } - else { - assertThat(value).isEqualTo("test varchar"); - } - } - - //CHAR(25) - index = columnIndex.get("t_char"); - if (index != null) { - value = row.getField(index); - if ((rowNumber % 41) == 0) { - assertThat(value).isNull(); - } - else { - assertThat(value).isEqualTo((rowNumber % 41) == 1 ? " " : "test char "); - } - } - - // MAP - index = columnIndex.get("t_map"); - if (index != null) { - if ((rowNumber % 27) == 0) { - assertThat(row.getField(index)).isNull(); - } - else { - assertThat(row.getField(index)).isEqualTo(ImmutableMap.of("test key", "test value")); - } - } - - // ARRAY - index = columnIndex.get("t_array_string"); - if (index != null) { - if ((rowNumber % 29) == 0) { - assertThat(row.getField(index)).isNull(); - } - else { - assertThat(row.getField(index)).isEqualTo(ImmutableList.of("abc", "xyz", "data")); - } - } - - // ARRAY - index = columnIndex.get("t_array_timestamp"); - if (index != null) { - if ((rowNumber % 43) == 0) { - assertThat(row.getField(index)).isNull(); - } - else { - SqlTimestamp expected = sqlTimestampOf(3, LocalDateTime.of(2011, 5, 6, 7, 8, 9, 123_000_000)); - assertThat(row.getField(index)).isEqualTo(ImmutableList.of(expected)); - } - } - - // ARRAY> - index = columnIndex.get("t_array_struct"); - if (index != null) { - if ((rowNumber % 31) == 0) { - assertThat(row.getField(index)).isNull(); - } - else { - List expected1 = ImmutableList.of("test abc", 0.1); - List expected2 = ImmutableList.of("test xyz", 0.2); - assertThat(row.getField(index)).isEqualTo(ImmutableList.of(expected1, expected2)); - } - } - - // STRUCT - index = columnIndex.get("t_struct"); - if (index != null) { - if ((rowNumber % 31) == 0) { - assertThat(row.getField(index)).isNull(); - } - else { - assertThat(row.getField(index) instanceof List).isTrue(); - List values = (List) row.getField(index); - assertThat(values.size()).isEqualTo(2); - assertThat(values.get(0)).isEqualTo("test abc"); - assertThat(values.get(1)).isEqualTo(0.1); - } - } - - // MAP>> - index = columnIndex.get("t_complex"); - if (index != null) { - if ((rowNumber % 33) == 0) { - assertThat(row.getField(index)).isNull(); - } - else { - List expected1 = ImmutableList.of("test abc", 0.1); - List expected2 = ImmutableList.of("test xyz", 0.2); - assertThat(row.getField(index)).isEqualTo(ImmutableMap.of(1, ImmutableList.of(expected1, expected2))); - } - } - - // NEW COLUMN - assertThat(row.getField(columnIndex.get("new_column"))).isNull(); - - long newCompletedBytes = pageSource.getCompletedBytes(); - assertThat(newCompletedBytes >= completedBytes).isTrue(); - // some formats (e.g., parquet) over read the data by a bit - assertLessThanOrEqual(newCompletedBytes, hiveSplit.getLength() + (100 * 1024)); - completedBytes = newCompletedBytes; - } - - assertLessThanOrEqual(completedBytes, hiveSplit.getLength() + (100 * 1024)); - assertThat(rowNumber).isEqualTo(100); - } - finally { - pageSource.close(); - } - } - - protected void dropTable(SchemaTableName table) - { - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorSession session = newSession(); - - ConnectorTableHandle handle = metadata.getTableHandle(session, table); - if (handle == null) { - return; - } - - metadata.dropTable(session, handle); - try { - // todo I have no idea why this is needed... maybe there is a propagation delay in the metastore? - metadata.dropTable(session, handle); - fail("expected NotFoundException"); - } - catch (TableNotFoundException expected) { - } - - transaction.commit(); - } - } - - protected ConnectorTableHandle getTableHandle(ConnectorMetadata metadata, SchemaTableName tableName) - { - ConnectorTableHandle handle = metadata.getTableHandle(newSession(), tableName); - checkArgument(handle != null, "table not found: %s", tableName); - return handle; - } - - private HiveTableHandle applyFilter(ConnectorMetadata metadata, ConnectorTableHandle tableHandle, Constraint constraint) - { - return metadata.applyFilter(newSession(), tableHandle, constraint) - .map(ConstraintApplicationResult::getHandle) - .map(HiveTableHandle.class::cast) - .orElseThrow(AssertionError::new); - } - - protected MaterializedResult readTable( - Transaction transaction, - ConnectorTableHandle tableHandle, - List columnHandles, - ConnectorSession session, - TupleDomain tupleDomain, - OptionalInt expectedSplitCount, - Optional expectedStorageFormat) - throws Exception - { - tableHandle = applyFilter(transaction.getMetadata(), tableHandle, new Constraint(tupleDomain)); - List splits = getAllSplits(getSplits(splitManager, transaction, session, tableHandle)); - if (expectedSplitCount.isPresent()) { - assertThat(splits.size()).isEqualTo(expectedSplitCount.getAsInt()); - } - - ImmutableList.Builder allRows = ImmutableList.builder(); - for (ConnectorSplit split : splits) { - try (ConnectorPageSource pageSource = pageSourceProvider.createPageSource(transaction.getTransactionHandle(), session, split, tableHandle, columnHandles, DynamicFilter.EMPTY)) { - expectedStorageFormat.ifPresent(format -> assertPageSourceType(pageSource, format)); - MaterializedResult result = materializeSourceDataStream(session, pageSource, getTypes(columnHandles)); - allRows.addAll(result.getMaterializedRows()); - } - } - return new MaterializedResult(allRows.build(), getTypes(columnHandles)); - } - - protected HiveMetastore getMetastoreClient() - { - return metastoreClient; - } - - protected LocationService getLocationService() - { - return locationService; - } - - protected static int getSplitCount(ConnectorSplitSource splitSource) - { - int splitCount = 0; - while (!splitSource.isFinished()) { - splitCount += getFutureValue(splitSource.getNextBatch(1000)).getSplits().size(); - } - return splitCount; - } - - private List getAllSplits(ConnectorTableHandle tableHandle, Transaction transaction, ConnectorSession session) - { - return getAllSplits(getSplits(splitManager, transaction, session, tableHandle)); - } - - protected static List getAllSplits(ConnectorSplitSource splitSource) - { - ImmutableList.Builder splits = ImmutableList.builder(); - while (!splitSource.isFinished()) { - splits.addAll(getFutureValue(splitSource.getNextBatch(1000)).getSplits()); - } - return splits.build(); - } - - protected static ConnectorSplitSource getSplits(ConnectorSplitManager splitManager, Transaction transaction, ConnectorSession session, ConnectorTableHandle tableHandle) - { - return splitManager.getSplits(transaction.getTransactionHandle(), session, tableHandle, DynamicFilter.EMPTY, Constraint.alwaysTrue()); - } - - protected String getPartitionId(Object partition) - { - return ((HivePartition) partition).getPartitionId(); - } - - protected static void assertPageSourceType(ConnectorPageSource pageSource, HiveStorageFormat hiveStorageFormat) - { - assertInstanceOf(((HivePageSource) pageSource).getPageSource(), pageSourceType(hiveStorageFormat), hiveStorageFormat.name()); - } - - private static Class pageSourceType(HiveStorageFormat hiveStorageFormat) - { - switch (hiveStorageFormat) { - case RCTEXT: - case RCBINARY: - return RcFilePageSource.class; - case ORC: - return OrcPageSource.class; - case PARQUET: - return ParquetPageSource.class; - case CSV: - case JSON: - case OPENX_JSON: - case TEXTFILE: - case SEQUENCEFILE: - return LinePageSource.class; - default: - throw new AssertionError("File type does not use a PageSource: " + hiveStorageFormat); - } - } - - private static void assertValueTypes(MaterializedRow row, List schema) - { - for (int columnIndex = 0; columnIndex < schema.size(); columnIndex++) { - ColumnMetadata column = schema.get(columnIndex); - Object value = row.getField(columnIndex); - if (value != null) { - if (BOOLEAN.equals(column.getType())) { - assertInstanceOf(value, Boolean.class); - } - else if (TINYINT.equals(column.getType())) { - assertInstanceOf(value, Byte.class); - } - else if (SMALLINT.equals(column.getType())) { - assertInstanceOf(value, Short.class); - } - else if (INTEGER.equals(column.getType())) { - assertInstanceOf(value, Integer.class); - } - else if (BIGINT.equals(column.getType())) { - assertInstanceOf(value, Long.class); - } - else if (DOUBLE.equals(column.getType())) { - assertInstanceOf(value, Double.class); - } - else if (REAL.equals(column.getType())) { - assertInstanceOf(value, Float.class); - } - else if (column.getType() instanceof VarcharType) { - assertInstanceOf(value, String.class); - } - else if (column.getType() instanceof CharType) { - assertInstanceOf(value, String.class); - } - else if (VARBINARY.equals(column.getType())) { - assertInstanceOf(value, SqlVarbinary.class); - } - else if (TIMESTAMP_MILLIS.equals(column.getType())) { - assertInstanceOf(value, SqlTimestamp.class); - } - else if (TIMESTAMP_TZ_MILLIS.equals(column.getType())) { - assertInstanceOf(value, SqlTimestampWithTimeZone.class); - } - else if (DATE.equals(column.getType())) { - assertInstanceOf(value, SqlDate.class); - } - else if (column.getType() instanceof ArrayType || column.getType() instanceof RowType) { - assertInstanceOf(value, List.class); - } - else if (column.getType() instanceof MapType) { - assertInstanceOf(value, Map.class); - } - else { - fail("Unknown primitive type " + columnIndex); - } - } - } - } - - private static void assertPrimitiveField(Map map, String name, Type type, boolean partitionKey) - { - assertThat(map).containsKey(name); - ColumnMetadata column = map.get(name); - assertThat(column.getType()) - .describedAs(name) - .isEqualTo(type); - assertThat(column.getExtraInfo()).isEqualTo(columnExtraInfo(partitionKey)); - } - - protected static ImmutableMap indexColumns(List columnHandles) - { - ImmutableMap.Builder index = ImmutableMap.builder(); - int i = 0; - for (ColumnHandle columnHandle : columnHandles) { - HiveColumnHandle hiveColumnHandle = (HiveColumnHandle) columnHandle; - index.put(hiveColumnHandle.getName(), i); - i++; - } - return index.buildOrThrow(); - } - - protected static ImmutableMap indexColumns(ConnectorTableMetadata tableMetadata) - { - ImmutableMap.Builder index = ImmutableMap.builder(); - int i = 0; - for (ColumnMetadata columnMetadata : tableMetadata.getColumns()) { - index.put(columnMetadata.getName(), i); - i++; - } - return index.buildOrThrow(); - } - - protected SchemaTableName temporaryTable(String tableName) - { - return temporaryTable(database, tableName); - } - - protected static SchemaTableName temporaryTable(String database, String tableName) - { - String randomName = UUID.randomUUID().toString().toLowerCase(ENGLISH).replace("-", ""); - return new SchemaTableName(database, TEMPORARY_TABLE_PREFIX + tableName + "_" + randomName); - } - - protected static Map createTableProperties(HiveStorageFormat storageFormat) - { - return createTableProperties(storageFormat, ImmutableList.of()); - } - - protected static Map createTableProperties(HiveStorageFormat storageFormat, Iterable partitionedBy) - { - return ImmutableMap.builder() - .put(STORAGE_FORMAT_PROPERTY, storageFormat) - .put(PARTITIONED_BY_PROPERTY, ImmutableList.copyOf(partitionedBy)) - .put(BUCKETED_BY_PROPERTY, ImmutableList.of()) - .put(BUCKET_COUNT_PROPERTY, 0) - .put(SORTED_BY_PROPERTY, ImmutableList.of()) - .buildOrThrow(); - } - - protected static List filterNonHiddenColumnHandles(Collection columnHandles) - { - return columnHandles.stream() - .filter(columnHandle -> !((HiveColumnHandle) columnHandle).isHidden()) - .collect(toList()); - } - - protected static List filterNonHiddenColumnMetadata(Collection columnMetadatas) - { - return columnMetadatas.stream() - .filter(columnMetadata -> !columnMetadata.isHidden()) - .collect(toList()); - } - - private void createEmptyTable(SchemaTableName schemaTableName, HiveStorageFormat hiveStorageFormat, List columns, List partitionColumns) - throws Exception - { - createEmptyTable(schemaTableName, hiveStorageFormat, columns, partitionColumns, Optional.empty(), false); - } - - private void createEmptyTable( - SchemaTableName schemaTableName, - HiveStorageFormat hiveStorageFormat, - List columns, - List partitionColumns, - Optional bucketProperty) - throws Exception - { - createEmptyTable(schemaTableName, hiveStorageFormat, columns, partitionColumns, bucketProperty, false); - } - - protected void createEmptyTable( - SchemaTableName schemaTableName, - HiveStorageFormat hiveStorageFormat, - List columns, - List partitionColumns, - Optional bucketProperty, - boolean isTransactional) - throws Exception - { - Path targetPath; - - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - - String tableOwner = session.getUser(); - String schemaName = schemaTableName.getSchemaName(); - String tableName = schemaTableName.getTableName(); - - LocationService locationService = getLocationService(); - targetPath = new Path(locationService.forNewTable(transaction.getMetastore(), session, schemaName, tableName).toString()); - - ImmutableMap.Builder tableParamBuilder = ImmutableMap.builder() - .put(TRINO_VERSION_NAME, TEST_SERVER_VERSION) - .put(TRINO_QUERY_ID_NAME, session.getQueryId()); - if (isTransactional) { - tableParamBuilder.put(TRANSACTIONAL, "true"); - } - Table.Builder tableBuilder = Table.builder() - .setDatabaseName(schemaName) - .setTableName(tableName) - .setOwner(Optional.of(tableOwner)) - .setTableType(MANAGED_TABLE.name()) - .setParameters(tableParamBuilder.buildOrThrow()) - .setDataColumns(columns) - .setPartitionColumns(partitionColumns); - - tableBuilder.getStorageBuilder() - .setLocation(targetPath.toString()) - .setStorageFormat(StorageFormat.create(hiveStorageFormat.getSerde(), hiveStorageFormat.getInputFormat(), hiveStorageFormat.getOutputFormat())) - .setBucketProperty(bucketProperty) - .setSerdeParameters(ImmutableMap.of()); - - PrincipalPrivileges principalPrivileges = testingPrincipalPrivilege(tableOwner, session.getUser()); - transaction.getMetastore().createTable(session, tableBuilder.build(), principalPrivileges, Optional.empty(), Optional.empty(), true, ZERO_TABLE_STATISTICS, false); - - transaction.commit(); - } - - HdfsContext context = new HdfsContext(newSession()); - List targetDirectoryList = listDirectory(context, targetPath); - assertThat(targetDirectoryList).isEqualTo(ImmutableList.of()); - } - - private void alterBucketProperty(SchemaTableName schemaTableName, Optional bucketProperty) - { - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - - String tableOwner = session.getUser(); - String schemaName = schemaTableName.getSchemaName(); - String tableName = schemaTableName.getTableName(); - - Optional
table = transaction.getMetastore().getTable(schemaName, tableName); - Table.Builder tableBuilder = Table.builder(table.get()); - tableBuilder.getStorageBuilder().setBucketProperty(bucketProperty); - PrincipalPrivileges principalPrivileges = testingPrincipalPrivilege(tableOwner, session.getUser()); - transaction.getMetastore().replaceTable(schemaName, tableName, tableBuilder.build(), principalPrivileges); - - transaction.commit(); - } - } - - protected PrincipalPrivileges testingPrincipalPrivilege(ConnectorSession session) - { - return testingPrincipalPrivilege(session.getUser(), session.getUser()); - } - - protected PrincipalPrivileges testingPrincipalPrivilege(String tableOwner, String grantor) - { - return new PrincipalPrivileges( - ImmutableMultimap.builder() - .put(tableOwner, new HivePrivilegeInfo(HivePrivilege.SELECT, true, new HivePrincipal(USER, grantor), new HivePrincipal(USER, grantor))) - .put(tableOwner, new HivePrivilegeInfo(HivePrivilege.INSERT, true, new HivePrincipal(USER, grantor), new HivePrincipal(USER, grantor))) - .put(tableOwner, new HivePrivilegeInfo(HivePrivilege.UPDATE, true, new HivePrincipal(USER, grantor), new HivePrincipal(USER, grantor))) - .put(tableOwner, new HivePrivilegeInfo(HivePrivilege.DELETE, true, new HivePrincipal(USER, grantor), new HivePrincipal(USER, grantor))) - .build(), - ImmutableMultimap.of()); - } - - private List listDirectory(HdfsContext context, Path path) - throws IOException - { - FileSystem fileSystem = hdfsEnvironment.getFileSystem(context, path); - return Arrays.stream(fileSystem.listStatus(path)) - .map(FileStatus::getPath) - .map(Path::getName) - .filter(name -> !name.startsWith(".trino")) - .collect(toList()); - } - - @Test - public void testTransactionDeleteInsert() - throws Exception - { - doTestTransactionDeleteInsert( - RCBINARY, - true, - ImmutableList.builder() - .add(new TransactionDeleteInsertTestCase(false, false, ROLLBACK_RIGHT_AWAY, Optional.empty())) - .add(new TransactionDeleteInsertTestCase(false, false, ROLLBACK_AFTER_DELETE, Optional.empty())) - .add(new TransactionDeleteInsertTestCase(false, false, ROLLBACK_AFTER_BEGIN_INSERT, Optional.empty())) - .add(new TransactionDeleteInsertTestCase(false, false, ROLLBACK_AFTER_APPEND_PAGE, Optional.empty())) - .add(new TransactionDeleteInsertTestCase(false, false, ROLLBACK_AFTER_SINK_FINISH, Optional.empty())) - .add(new TransactionDeleteInsertTestCase(false, false, ROLLBACK_AFTER_FINISH_INSERT, Optional.empty())) - .add(new TransactionDeleteInsertTestCase(false, false, COMMIT, Optional.of(new AddPartitionFailure()))) - .add(new TransactionDeleteInsertTestCase(false, false, COMMIT, Optional.of(new DirectoryRenameFailure()))) - .add(new TransactionDeleteInsertTestCase(false, false, COMMIT, Optional.of(new FileRenameFailure()))) - .add(new TransactionDeleteInsertTestCase(true, false, COMMIT, Optional.of(new DropPartitionFailure()))) - .add(new TransactionDeleteInsertTestCase(true, true, COMMIT, Optional.empty())) - .build()); - } - - @Test - public void testPreferredInsertLayout() - throws Exception - { - SchemaTableName tableName = temporaryTable("empty_partitioned_table"); - - try { - Column partitioningColumn = new Column("column2", HIVE_STRING, Optional.empty(), Map.of()); - List columns = ImmutableList.of( - new Column("column1", HIVE_STRING, Optional.empty(), Map.of()), - partitioningColumn); - createEmptyTable(tableName, ORC, columns, ImmutableList.of(partitioningColumn)); - - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorSession session = newSession(); - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); - Optional insertLayout = metadata.getInsertLayout(session, tableHandle); - assertThat(insertLayout.isPresent()).isTrue(); - assertThat(insertLayout.get().getPartitioning().isPresent()).isFalse(); - assertThat(insertLayout.get().getPartitionColumns()).isEqualTo(ImmutableList.of(partitioningColumn.getName())); - } - } - finally { - dropTable(tableName); - } - } - - @Test - public void testInsertBucketedTableLayout() - throws Exception - { - insertBucketedTableLayout(false); - } - - @Test - public void testInsertBucketedTransactionalTableLayout() - throws Exception - { - insertBucketedTableLayout(true); - } - - protected void insertBucketedTableLayout(boolean transactional) - throws Exception - { - SchemaTableName tableName = temporaryTable("empty_bucketed_table"); - try { - List columns = ImmutableList.of( - new Column("column1", HIVE_STRING, Optional.empty(), Map.of()), - new Column("column2", HIVE_LONG, Optional.empty(), Map.of())); - HiveBucketProperty bucketProperty = new HiveBucketProperty(ImmutableList.of("column1"), BUCKETING_V1, 4, ImmutableList.of()); - createEmptyTable(tableName, ORC, columns, ImmutableList.of(), Optional.of(bucketProperty), transactional); - - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorSession session = newSession(); - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); - Optional insertLayout = metadata.getInsertLayout(session, tableHandle); - assertThat(insertLayout.isPresent()).isTrue(); - ConnectorPartitioningHandle partitioningHandle = new HivePartitioningHandle( - bucketProperty.getBucketingVersion(), - bucketProperty.getBucketCount(), - ImmutableList.of(HIVE_STRING), - OptionalInt.empty(), - false); - assertThat(insertLayout.get().getPartitioning()).isEqualTo(Optional.of(partitioningHandle)); - assertThat(insertLayout.get().getPartitionColumns()).isEqualTo(ImmutableList.of("column1")); - ConnectorBucketNodeMap connectorBucketNodeMap = nodePartitioningProvider.getBucketNodeMapping(transaction.getTransactionHandle(), session, partitioningHandle).orElseThrow(); - assertThat(connectorBucketNodeMap.getBucketCount()).isEqualTo(4); - assertThat(connectorBucketNodeMap.hasFixedMapping()).isFalse(); - } - } - finally { - dropTable(tableName); - } - } - - @Test - public void testInsertPartitionedBucketedTableLayout() - throws Exception - { - insertPartitionedBucketedTableLayout(false); - } - - @Test - public void testInsertPartitionedBucketedTransactionalTableLayout() - throws Exception - { - insertPartitionedBucketedTableLayout(true); - } - - protected void insertPartitionedBucketedTableLayout(boolean transactional) - throws Exception - { - SchemaTableName tableName = temporaryTable("empty_partitioned_table"); - try { - Column partitioningColumn = new Column("column2", HIVE_LONG, Optional.empty(), Map.of()); - List columns = ImmutableList.of( - new Column("column1", HIVE_STRING, Optional.empty(), Map.of()), - partitioningColumn); - HiveBucketProperty bucketProperty = new HiveBucketProperty(ImmutableList.of("column1"), BUCKETING_V1, 4, ImmutableList.of()); - createEmptyTable(tableName, ORC, columns, ImmutableList.of(partitioningColumn), Optional.of(bucketProperty), transactional); - - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorSession session = newSession(); - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); - Optional insertLayout = metadata.getInsertLayout(session, tableHandle); - assertThat(insertLayout.isPresent()).isTrue(); - ConnectorPartitioningHandle partitioningHandle = new HivePartitioningHandle( - bucketProperty.getBucketingVersion(), - bucketProperty.getBucketCount(), - ImmutableList.of(HIVE_STRING), - OptionalInt.empty(), - true); - assertThat(insertLayout.get().getPartitioning()).isEqualTo(Optional.of(partitioningHandle)); - assertThat(insertLayout.get().getPartitionColumns()).isEqualTo(ImmutableList.of("column1", "column2")); - ConnectorBucketNodeMap connectorBucketNodeMap = nodePartitioningProvider.getBucketNodeMapping(transaction.getTransactionHandle(), session, partitioningHandle).orElseThrow(); - assertThat(connectorBucketNodeMap.getBucketCount()).isEqualTo(32); - assertThat(connectorBucketNodeMap.hasFixedMapping()).isFalse(); - } - } - finally { - dropTable(tableName); - } - } - - @Test - public void testPreferredCreateTableLayout() - { - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorSession session = newSession(); - Optional newTableLayout = metadata.getNewTableLayout( - session, - new ConnectorTableMetadata( - new SchemaTableName("schema", "table"), - ImmutableList.of( - new ColumnMetadata("column1", BIGINT), - new ColumnMetadata("column2", BIGINT)), - ImmutableMap.of( - PARTITIONED_BY_PROPERTY, ImmutableList.of("column2"), - BUCKETED_BY_PROPERTY, ImmutableList.of(), - BUCKET_COUNT_PROPERTY, 0, - SORTED_BY_PROPERTY, ImmutableList.of()))); - assertThat(newTableLayout.isPresent()).isTrue(); - assertThat(newTableLayout.get().getPartitioning().isPresent()).isFalse(); - assertThat(newTableLayout.get().getPartitionColumns()).isEqualTo(ImmutableList.of("column2")); - } - } - - @Test - public void testCreateBucketedTableLayout() - { - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorSession session = newSession(); - Optional newTableLayout = metadata.getNewTableLayout( - session, - new ConnectorTableMetadata( - new SchemaTableName("schema", "table"), - ImmutableList.of( - new ColumnMetadata("column1", BIGINT), - new ColumnMetadata("column2", BIGINT)), - ImmutableMap.of( - PARTITIONED_BY_PROPERTY, ImmutableList.of(), - BUCKETED_BY_PROPERTY, ImmutableList.of("column1"), - BUCKET_COUNT_PROPERTY, 10, - SORTED_BY_PROPERTY, ImmutableList.of()))); - assertThat(newTableLayout.isPresent()).isTrue(); - ConnectorPartitioningHandle partitioningHandle = new HivePartitioningHandle( - BUCKETING_V1, - 10, - ImmutableList.of(HIVE_LONG), - OptionalInt.empty(), - false); - assertThat(newTableLayout.get().getPartitioning()).isEqualTo(Optional.of(partitioningHandle)); - assertThat(newTableLayout.get().getPartitionColumns()).isEqualTo(ImmutableList.of("column1")); - ConnectorBucketNodeMap connectorBucketNodeMap = nodePartitioningProvider.getBucketNodeMapping(transaction.getTransactionHandle(), session, partitioningHandle).orElseThrow(); - assertThat(connectorBucketNodeMap.getBucketCount()).isEqualTo(10); - assertThat(connectorBucketNodeMap.hasFixedMapping()).isFalse(); - } - } - - @Test - public void testCreatePartitionedBucketedTableLayout() - { - try (Transaction transaction = newTransaction()) { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorSession session = newSession(); - Optional newTableLayout = metadata.getNewTableLayout( - session, - new ConnectorTableMetadata( - new SchemaTableName("schema", "table"), - ImmutableList.of( - new ColumnMetadata("column1", BIGINT), - new ColumnMetadata("column2", BIGINT)), - ImmutableMap.of( - PARTITIONED_BY_PROPERTY, ImmutableList.of("column2"), - BUCKETED_BY_PROPERTY, ImmutableList.of("column1"), - BUCKET_COUNT_PROPERTY, 10, - SORTED_BY_PROPERTY, ImmutableList.of()))); - assertThat(newTableLayout.isPresent()).isTrue(); - ConnectorPartitioningHandle partitioningHandle = new HivePartitioningHandle( - BUCKETING_V1, - 10, - ImmutableList.of(HIVE_LONG), - OptionalInt.empty(), - true); - assertThat(newTableLayout.get().getPartitioning()).isEqualTo(Optional.of(partitioningHandle)); - assertThat(newTableLayout.get().getPartitionColumns()).isEqualTo(ImmutableList.of("column1", "column2")); - ConnectorBucketNodeMap connectorBucketNodeMap = nodePartitioningProvider.getBucketNodeMapping(transaction.getTransactionHandle(), session, partitioningHandle).orElseThrow(); - assertThat(connectorBucketNodeMap.getBucketCount()).isEqualTo(32); - assertThat(connectorBucketNodeMap.hasFixedMapping()).isFalse(); - } - } - - protected void doTestTransactionDeleteInsert(HiveStorageFormat storageFormat, boolean allowInsertExisting, List testCases) - throws Exception - { - // There are 4 types of operations on a partition: add, drop, alter (drop then add), insert existing. - // There are 12 partitions in this test, 3 for each type. - // 3 is chosen to verify that cleanups, commit aborts, rollbacks are always as complete as possible regardless of failure. - MaterializedResult beforeData = - MaterializedResult.resultBuilder(SESSION, BIGINT, createUnboundedVarcharType(), createUnboundedVarcharType()) - .row(110L, "a", "alter1") - .row(120L, "a", "insert1") - .row(140L, "a", "drop1") - .row(210L, "b", "drop2") - .row(310L, "c", "alter2") - .row(320L, "c", "alter3") - .row(510L, "e", "drop3") - .row(610L, "f", "insert2") - .row(620L, "f", "insert3") - .build(); - Domain domainToDrop = Domain.create(ValueSet.of( - createUnboundedVarcharType(), - utf8Slice("alter1"), utf8Slice("alter2"), utf8Slice("alter3"), utf8Slice("drop1"), utf8Slice("drop2"), utf8Slice("drop3")), - false); - List extraRowsForInsertExisting = ImmutableList.of(); - if (allowInsertExisting) { - extraRowsForInsertExisting = MaterializedResult.resultBuilder(SESSION, BIGINT, createUnboundedVarcharType(), createUnboundedVarcharType()) - .row(121L, "a", "insert1") - .row(611L, "f", "insert2") - .row(621L, "f", "insert3") - .build() - .getMaterializedRows(); - } - MaterializedResult insertData = - MaterializedResult.resultBuilder(SESSION, BIGINT, createUnboundedVarcharType(), createUnboundedVarcharType()) - .row(111L, "a", "alter1") - .row(131L, "a", "add1") - .row(221L, "b", "add2") - .row(311L, "c", "alter2") - .row(321L, "c", "alter3") - .row(411L, "d", "add3") - .rows(extraRowsForInsertExisting) - .build(); - MaterializedResult afterData = - MaterializedResult.resultBuilder(SESSION, BIGINT, createUnboundedVarcharType(), createUnboundedVarcharType()) - .row(120L, "a", "insert1") - .row(610L, "f", "insert2") - .row(620L, "f", "insert3") - .rows(insertData.getMaterializedRows()) - .build(); - - for (TransactionDeleteInsertTestCase testCase : testCases) { - SchemaTableName temporaryDeleteInsert = temporaryTable("delete_insert"); - try { - createEmptyTable( - temporaryDeleteInsert, - storageFormat, - ImmutableList.of(new Column("col1", HIVE_LONG, Optional.empty(), Map.of())), - ImmutableList.of(new Column("pk1", HIVE_STRING, Optional.empty(), Map.of()), new Column("pk2", HIVE_STRING, Optional.empty(), Map.of()))); - insertData(temporaryDeleteInsert, beforeData); - try { - doTestTransactionDeleteInsert( - storageFormat, - temporaryDeleteInsert, - domainToDrop, - insertData, - testCase.isExpectCommittedData() ? afterData : beforeData, - testCase.getTag(), - testCase.isExpectQuerySucceed(), - testCase.getConflictTrigger()); - } - catch (AssertionError e) { - throw new AssertionError(format("Test case: %s", testCase), e); - } - } - finally { - dropTable(temporaryDeleteInsert); - } - } - } - - private void doTestTransactionDeleteInsert( - HiveStorageFormat storageFormat, - SchemaTableName tableName, - Domain domainToDrop, - MaterializedResult insertData, - MaterializedResult expectedData, - TransactionDeleteInsertTestTag tag, - boolean expectQuerySucceed, - Optional conflictTrigger) - throws Exception - { - Location writePath = null; - Location targetPath = null; - - try (Transaction transaction = newTransaction()) { - try { - ConnectorMetadata metadata = transaction.getMetadata(); - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); - ConnectorSession session; - rollbackIfEquals(tag, ROLLBACK_RIGHT_AWAY); - - // Query 1: delete - session = newSession(); - HiveColumnHandle dsColumnHandle = (HiveColumnHandle) metadata.getColumnHandles(session, tableHandle).get("pk2"); - TupleDomain tupleDomain = TupleDomain.withColumnDomains(ImmutableMap.of( - dsColumnHandle, domainToDrop)); - Constraint constraint = new Constraint(tupleDomain, tupleDomain.asPredicate(), tupleDomain.getDomains().orElseThrow().keySet()); - tableHandle = applyFilter(metadata, tableHandle, constraint); - tableHandle = metadata.applyDelete(session, tableHandle).get(); - metadata.executeDelete(session, tableHandle); - rollbackIfEquals(tag, ROLLBACK_AFTER_DELETE); - - // Query 2: insert - session = newSession(); - ConnectorInsertTableHandle insertTableHandle = metadata.beginInsert(session, tableHandle, ImmutableList.of(), NO_RETRIES); - rollbackIfEquals(tag, ROLLBACK_AFTER_BEGIN_INSERT); - writePath = getStagingPathRoot(insertTableHandle); - targetPath = getTargetPathRoot(insertTableHandle); - ConnectorPageSink sink = pageSinkProvider.createPageSink(transaction.getTransactionHandle(), session, insertTableHandle, TESTING_PAGE_SINK_ID); - sink.appendPage(insertData.toPage()); - rollbackIfEquals(tag, ROLLBACK_AFTER_APPEND_PAGE); - Collection fragments = getFutureValue(sink.finish()); - rollbackIfEquals(tag, ROLLBACK_AFTER_SINK_FINISH); - metadata.finishInsert(session, insertTableHandle, fragments, ImmutableList.of()); - rollbackIfEquals(tag, ROLLBACK_AFTER_FINISH_INSERT); - - assertThat(tag).isEqualTo(COMMIT); - - if (conflictTrigger.isPresent()) { - JsonCodec partitionUpdateCodec = JsonCodec.jsonCodec(PartitionUpdate.class); - List partitionUpdates = fragments.stream() - .map(Slice::getBytes) - .map(partitionUpdateCodec::fromJson) - .collect(toList()); - conflictTrigger.get().triggerConflict(session, tableName, insertTableHandle, partitionUpdates); - } - transaction.commit(); - if (conflictTrigger.isPresent()) { - assertThat(expectQuerySucceed).isTrue(); - conflictTrigger.get().verifyAndCleanup(session, tableName); - } - } - catch (TestingRollbackException e) { - transaction.rollback(); - } - catch (TrinoException e) { - assertThat(expectQuerySucceed).isFalse(); - if (conflictTrigger.isPresent()) { - conflictTrigger.get().verifyAndCleanup(newSession(), tableName); - } - } - } - - // check that temporary files are removed - if (writePath != null && !writePath.equals(targetPath)) { - HdfsContext context = new HdfsContext(newSession()); - FileSystem fileSystem = hdfsEnvironment.getFileSystem(context, new Path(writePath.toString())); - assertThat(fileSystem.exists(new Path(writePath.toString()))).isFalse(); - } - - try (Transaction transaction = newTransaction()) { - // verify partitions - List partitionNames = transaction.getMetastore() - .getPartitionNames(tableName.getSchemaName(), tableName.getTableName()) - .orElseThrow(() -> new AssertionError("Table does not exist: " + tableName)); - assertEqualsIgnoreOrder( - partitionNames, - expectedData.getMaterializedRows().stream() - .map(row -> format("pk1=%s/pk2=%s", row.getField(1), row.getField(2))) - .distinct() - .collect(toImmutableList())); - - // load the new table - ConnectorSession session = newSession(); - ConnectorMetadata metadata = transaction.getMetadata(); - metadata.beginQuery(session); - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); - List columnHandles = filterNonHiddenColumnHandles(metadata.getColumnHandles(session, tableHandle).values()); - - // verify the data - MaterializedResult result = readTable(transaction, tableHandle, columnHandles, session, TupleDomain.all(), OptionalInt.empty(), Optional.of(storageFormat)); - assertEqualsIgnoreOrder(result.getMaterializedRows(), expectedData.getMaterializedRows()); - } - } - - private static void rollbackIfEquals(TransactionDeleteInsertTestTag tag, TransactionDeleteInsertTestTag expectedTag) - { - if (expectedTag == tag) { - throw new TestingRollbackException(); - } - } - - private static class TestingRollbackException - extends RuntimeException - { - } - - protected static class TransactionDeleteInsertTestCase - { - private final boolean expectCommittedData; - private final boolean expectQuerySucceed; - private final TransactionDeleteInsertTestTag tag; - private final Optional conflictTrigger; - - public TransactionDeleteInsertTestCase(boolean expectCommittedData, boolean expectQuerySucceed, TransactionDeleteInsertTestTag tag, Optional conflictTrigger) - { - this.expectCommittedData = expectCommittedData; - this.expectQuerySucceed = expectQuerySucceed; - this.tag = tag; - this.conflictTrigger = conflictTrigger; - } - - public boolean isExpectCommittedData() - { - return expectCommittedData; - } - - public boolean isExpectQuerySucceed() - { - return expectQuerySucceed; - } - - public TransactionDeleteInsertTestTag getTag() - { - return tag; - } - - public Optional getConflictTrigger() - { - return conflictTrigger; - } - - @Override - public String toString() - { - return toStringHelper(this) - .add("tag", tag) - .add("conflictTrigger", conflictTrigger.map(conflictTrigger -> conflictTrigger.getClass().getName())) - .add("expectCommittedData", expectCommittedData) - .add("expectQuerySucceed", expectQuerySucceed) - .toString(); - } - } - - protected enum TransactionDeleteInsertTestTag - { - ROLLBACK_RIGHT_AWAY, - ROLLBACK_AFTER_DELETE, - ROLLBACK_AFTER_BEGIN_INSERT, - ROLLBACK_AFTER_APPEND_PAGE, - ROLLBACK_AFTER_SINK_FINISH, - ROLLBACK_AFTER_FINISH_INSERT, - COMMIT, - } - - protected interface ConflictTrigger - { - void triggerConflict(ConnectorSession session, SchemaTableName tableName, ConnectorInsertTableHandle insertTableHandle, List partitionUpdates) - throws IOException; - - void verifyAndCleanup(ConnectorSession session, SchemaTableName tableName) - throws IOException; - } - - protected class AddPartitionFailure - implements ConflictTrigger - { - private final ImmutableList copyPartitionFrom = ImmutableList.of("a", "insert1"); - private final String partitionNameToConflict = "pk1=b/pk2=add2"; - private Partition conflictPartition; - - @Override - public void triggerConflict(ConnectorSession session, SchemaTableName tableName, ConnectorInsertTableHandle insertTableHandle, List partitionUpdates) - { - // This method bypasses transaction interface because this method is inherently hacky and doesn't work well with the transaction abstraction. - // Additionally, this method is not part of a test. Its purpose is to set up an environment for another test. - HiveMetastore metastoreClient = getMetastoreClient(); - Table table = metastoreClient.getTable(tableName.getSchemaName(), tableName.getTableName()) - .orElseThrow(() -> new TableNotFoundException(tableName)); - Optional partition = metastoreClient.getPartition(table, copyPartitionFrom); - conflictPartition = Partition.builder(partition.get()) - .setValues(toPartitionValues(partitionNameToConflict)) - .build(); - metastoreClient.addPartitions( - tableName.getSchemaName(), - tableName.getTableName(), - ImmutableList.of(new PartitionWithStatistics(conflictPartition, partitionNameToConflict, PartitionStatistics.empty()))); - } - - @Override - public void verifyAndCleanup(ConnectorSession session, SchemaTableName tableName) - { - // This method bypasses transaction interface because this method is inherently hacky and doesn't work well with the transaction abstraction. - // Additionally, this method is not part of a test. Its purpose is to set up an environment for another test. - HiveMetastore metastoreClient = getMetastoreClient(); - Table table = metastoreClient.getTable(tableName.getSchemaName(), tableName.getTableName()) - .orElseThrow(() -> new TableNotFoundException(tableName)); - Optional actualPartition = metastoreClient.getPartition(table, toPartitionValues(partitionNameToConflict)); - // Make sure the partition inserted to trigger conflict was not overwritten - // Checking storage location is sufficient because implement never uses .../pk1=a/pk2=a2 as the directory for partition [b, b2]. - assertThat(actualPartition.get().getStorage().getLocation()).isEqualTo(conflictPartition.getStorage().getLocation()); - metastoreClient.dropPartition(tableName.getSchemaName(), tableName.getTableName(), conflictPartition.getValues(), false); - } - } - - protected class DropPartitionFailure - implements ConflictTrigger - { - private final ImmutableList partitionValueToConflict = ImmutableList.of("b", "drop2"); - - @Override - public void triggerConflict(ConnectorSession session, SchemaTableName tableName, ConnectorInsertTableHandle insertTableHandle, List partitionUpdates) - { - // This method bypasses transaction interface because this method is inherently hacky and doesn't work well with the transaction abstraction. - // Additionally, this method is not part of a test. Its purpose is to set up an environment for another test. - HiveMetastore metastoreClient = getMetastoreClient(); - metastoreClient.dropPartition(tableName.getSchemaName(), tableName.getTableName(), partitionValueToConflict, false); - } - - @Override - public void verifyAndCleanup(ConnectorSession session, SchemaTableName tableName) - { - // Do not add back the deleted partition because the implementation is expected to move forward instead of backward when delete fails - } - } - - protected class DirectoryRenameFailure - implements ConflictTrigger - { - private HdfsContext context; - private Path path; - - @Override - public void triggerConflict(ConnectorSession session, SchemaTableName tableName, ConnectorInsertTableHandle insertTableHandle, List partitionUpdates) - throws IOException - { - Location writePath = getStagingPathRoot(insertTableHandle); - Location targetPath = getTargetPathRoot(insertTableHandle); - if (writePath.equals(targetPath)) { - // This conflict does not apply. Trigger a rollback right away so that this test case passes. - throw new TestingRollbackException(); - } - path = new Path(targetPath.appendPath("pk1=b").appendPath("pk2=add2").toString()); - context = new HdfsContext(session); - if (!hdfsEnvironment.getFileSystem(context, path).mkdirs(path, hdfsEnvironment.getNewDirectoryPermissions().orElse(null))) { - throw new IOException("mkdirs returned false"); - } - } - - @Override - public void verifyAndCleanup(ConnectorSession session, SchemaTableName tableName) - throws IOException - { - assertThat(listDirectory(context, path)).isEqualTo(ImmutableList.of()); - hdfsEnvironment.getFileSystem(context, path).delete(path, false); - } - } - - protected class FileRenameFailure - implements ConflictTrigger - { - private HdfsContext context; - private Path path; - - @Override - public void triggerConflict(ConnectorSession session, SchemaTableName tableName, ConnectorInsertTableHandle insertTableHandle, List partitionUpdates) - throws IOException - { - for (PartitionUpdate partitionUpdate : partitionUpdates) { - if ("pk2=insert2".equals(partitionUpdate.getTargetPath().fileName())) { - path = new Path(partitionUpdate.getTargetPath().toString(), partitionUpdate.getFileNames().get(0)); - break; - } - } - assertThat(path).isNotNull(); - - context = new HdfsContext(session); - FileSystem fileSystem = hdfsEnvironment.getFileSystem(context, path); - fileSystem.createNewFile(path); - } - - @Override - public void verifyAndCleanup(ConnectorSession session, SchemaTableName tableName) - throws IOException - { - // The file we added to trigger a conflict was cleaned up because it matches the query prefix. - // Consider this the same as a network failure that caused the successful creation of file not reported to the caller. - assertThat(hdfsEnvironment.getFileSystem(context, path).exists(path)).isFalse(); - } - } - - private static class CountingDirectoryLister - implements DirectoryLister - { - private final AtomicInteger listCount = new AtomicInteger(); - - @Override - public RemoteIterator listFilesRecursively(TrinoFileSystem fs, Table table, Location location) - throws IOException - { - listCount.incrementAndGet(); - return new TrinoFileStatusRemoteIterator(fs.listFiles(location)); - } - - public int getListCount() - { - return listCount.get(); - } - - @Override - public void invalidate(Partition partition) - { - } - - @Override - public void invalidate(Table table) - { - } - } -} diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHiveLocal.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHiveLocal.java deleted file mode 100644 index 10d4b3130cdb8..0000000000000 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHiveLocal.java +++ /dev/null @@ -1,351 +0,0 @@ -/* - * 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.hive; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.io.RecursiveDeleteOption; -import com.google.common.reflect.ClassPath; -import io.airlift.log.Logger; -import io.trino.filesystem.Location; -import io.trino.plugin.hive.metastore.Column; -import io.trino.plugin.hive.metastore.Database; -import io.trino.plugin.hive.metastore.HiveMetastore; -import io.trino.plugin.hive.metastore.PrincipalPrivileges; -import io.trino.plugin.hive.metastore.SortingColumn; -import io.trino.plugin.hive.metastore.StorageFormat; -import io.trino.plugin.hive.metastore.Table; -import io.trino.spi.connector.ColumnHandle; -import io.trino.spi.connector.ConnectorMetadata; -import io.trino.spi.connector.ConnectorSession; -import io.trino.spi.connector.ConnectorTableHandle; -import io.trino.spi.connector.SchemaTableName; -import io.trino.spi.predicate.TupleDomain; -import io.trino.spi.security.PrincipalType; -import io.trino.testing.MaterializedResult; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; - -import java.io.File; -import java.io.IOException; -import java.io.InputStream; -import java.io.UncheckedIOException; -import java.nio.file.Files; -import java.nio.file.Path; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.OptionalInt; - -import static com.google.common.io.MoreFiles.deleteRecursively; -import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; -import static io.trino.plugin.hive.HiveMetadata.TABLE_COMMENT; -import static io.trino.plugin.hive.HiveMetadata.TRINO_QUERY_ID_NAME; -import static io.trino.plugin.hive.HiveMetadata.TRINO_VERSION_NAME; -import static io.trino.plugin.hive.HiveStorageFormat.ORC; -import static io.trino.plugin.hive.HiveStorageFormat.TEXTFILE; -import static io.trino.plugin.hive.HiveTestUtils.HDFS_ENVIRONMENT; -import static io.trino.plugin.hive.HiveType.HIVE_INT; -import static io.trino.plugin.hive.HiveType.HIVE_STRING; -import static io.trino.plugin.hive.TableType.EXTERNAL_TABLE; -import static io.trino.plugin.hive.TableType.MANAGED_TABLE; -import static io.trino.plugin.hive.metastore.PrincipalPrivileges.NO_PRIVILEGES; -import static io.trino.plugin.hive.metastore.StorageFormat.fromHiveStorageFormat; -import static io.trino.plugin.hive.util.HiveBucketing.BucketingVersion.BUCKETING_V1; -import static io.trino.plugin.hive.util.HiveUtil.SPARK_TABLE_PROVIDER_KEY; -import static java.nio.file.Files.copy; -import static java.util.Objects.requireNonNull; -import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.jupiter.api.Assumptions.abort; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; - -@TestInstance(PER_CLASS) -public abstract class AbstractTestHiveLocal - extends AbstractTestHive -{ - private static final Logger log = Logger.get(AbstractTestHiveLocal.class); - private static final String DEFAULT_TEST_DB_NAME = "test"; - - private File tempDir; - private final String testDbName; - - protected AbstractTestHiveLocal() - { - this(DEFAULT_TEST_DB_NAME); - } - - protected AbstractTestHiveLocal(String testDbName) - { - this.testDbName = requireNonNull(testDbName, "testDbName is null"); - } - - protected abstract HiveMetastore createMetastore(File tempDir); - - @BeforeAll - public void initialize() - throws Exception - { - tempDir = Files.createTempDirectory(null).toFile(); - - HiveMetastore metastore = createMetastore(tempDir); - - metastore.createDatabase( - Database.builder() - .setDatabaseName(testDbName) - .setOwnerName(Optional.of("public")) - .setOwnerType(Optional.of(PrincipalType.ROLE)) - .build()); - - HiveConfig hiveConfig = new HiveConfig() - .setParquetTimeZone("America/Los_Angeles") - .setRcfileTimeZone("America/Los_Angeles"); - - setup(testDbName, hiveConfig, metastore, HDFS_ENVIRONMENT); - - createTestTables(); - } - - protected void createTestTables() - throws Exception - { - Location location = Location.of(metastoreClient.getDatabase(database).orElseThrow() - .getLocation().orElseThrow()); - - createTestTable( - // Matches create-test.sql » trino_test_partition_format - Table.builder() - .setDatabaseName(database) - .setTableName(tablePartitionFormat.getTableName()) - .setTableType(MANAGED_TABLE.name()) - .setOwner(Optional.empty()) - .setDataColumns(List.of( - new Column("t_string", HiveType.HIVE_STRING, Optional.empty(), Map.of()), - new Column("t_tinyint", HiveType.HIVE_BYTE, Optional.empty(), Map.of()), - new Column("t_smallint", HiveType.HIVE_SHORT, Optional.empty(), Map.of()), - new Column("t_int", HiveType.HIVE_INT, Optional.empty(), Map.of()), - new Column("t_bigint", HiveType.HIVE_LONG, Optional.empty(), Map.of()), - new Column("t_float", HiveType.HIVE_FLOAT, Optional.empty(), Map.of()), - new Column("t_boolean", HiveType.HIVE_BOOLEAN, Optional.empty(), Map.of()))) - .setPartitionColumns(List.of( - new Column("ds", HiveType.HIVE_STRING, Optional.empty(), Map.of()), - new Column("file_format", HiveType.HIVE_STRING, Optional.empty(), Map.of()), - new Column("dummy", HiveType.HIVE_INT, Optional.empty(), Map.of()))) - .setParameter(TABLE_COMMENT, "Presto test data") - .withStorage(storage -> storage - .setStorageFormat(fromHiveStorageFormat(new HiveConfig().getHiveStorageFormat())) - .setLocation(Optional.of(location.appendPath(tablePartitionFormat.getTableName()).toString()))) - .build()); - - createTestTable( - // Matches create-test.sql » trino_test_partition_format - Table.builder() - .setDatabaseName(database) - .setTableName(tableUnpartitioned.getTableName()) - .setTableType(MANAGED_TABLE.name()) - .setOwner(Optional.empty()) - .setDataColumns(List.of( - new Column("t_string", HiveType.HIVE_STRING, Optional.empty(), Map.of()), - new Column("t_tinyint", HiveType.HIVE_BYTE, Optional.empty(), Map.of()))) - .setParameter(TABLE_COMMENT, "Presto test data") - .withStorage(storage -> storage - .setStorageFormat(fromHiveStorageFormat(TEXTFILE)) - .setLocation(Optional.of(location.appendPath(tableUnpartitioned.getTableName()).toString()))) - .build()); - } - - protected void createTestTable(Table table) - throws Exception - { - metastoreClient.createTable(table, NO_PRIVILEGES); - } - - @AfterAll - public void cleanup() - throws IOException - { - try { - for (String tableName : metastoreClient.getTables(database)) { - metastoreClient.dropTable(database, tableName, true); - } - metastoreClient.dropDatabase(testDbName, true); - } - finally { - deleteRecursively(tempDir.toPath(), ALLOW_INSECURE); - } - } - - @Override - protected ConnectorTableHandle getTableHandle(ConnectorMetadata metadata, SchemaTableName tableName) - { - if (tableName.getTableName().startsWith(TEMPORARY_TABLE_PREFIX)) { - return super.getTableHandle(metadata, tableName); - } - return abort("tests using existing tables are not supported"); - } - - @Test - @Override - public void testGetAllTableColumns() - { - abort("Test disabled for this subclass"); - } - - @Test - @Override - public void testGetAllTableColumnsInSchema() - { - abort("Test disabled for this subclass"); - } - - @Test - @Override - public void testGetTableNames() - { - abort("Test disabled for this subclass"); - } - - @Test - @Override - public void testGetTableSchemaOffline() - { - abort("Test disabled for this subclass"); - } - - @Test - public void testSparkBucketedTableValidation() - throws Exception - { - SchemaTableName table = temporaryTable("spark_bucket_validation"); - try { - doTestSparkBucketedTableValidation(table); - } - finally { - dropTable(table); - } - } - - private void doTestSparkBucketedTableValidation(SchemaTableName tableName) - throws Exception - { - Path externalLocation = copyResourceDirToTemporaryDirectory("spark_bucketed_nation"); - try { - createExternalTable( - tableName, - ORC, - ImmutableList.of( - new Column("nationkey", HIVE_INT, Optional.empty(), Map.of()), - new Column("name", HIVE_STRING, Optional.empty(), Map.of()), - new Column("regionkey", HIVE_INT, Optional.empty(), Map.of()), - new Column("comment", HIVE_STRING, Optional.empty(), Map.of())), - ImmutableList.of(), - Optional.of(new HiveBucketProperty( - ImmutableList.of("nationkey"), - BUCKETING_V1, - 3, - ImmutableList.of(new SortingColumn("name", SortingColumn.Order.ASCENDING)))), - Location.of(externalLocation.toUri().toString())); - - assertReadFailsWithMessageMatching(ORC, tableName, "Hive table is corrupt\\. File '.*/.*' is for bucket [0-2], but contains a row for bucket [0-2]."); - markTableAsCreatedBySpark(tableName, "orc"); - assertReadReturnsRowCount(ORC, tableName, 25); - } - finally { - deleteRecursively(externalLocation, RecursiveDeleteOption.ALLOW_INSECURE); - } - } - - private void assertReadReturnsRowCount(HiveStorageFormat storageFormat, SchemaTableName tableName, int rowCount) - throws Exception - { - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - ConnectorMetadata metadata = transaction.getMetadata(); - metadata.beginQuery(session); - ConnectorTableHandle tableHandle = getTableHandle(metadata, tableName); - List columnHandles = filterNonHiddenColumnHandles(metadata.getColumnHandles(session, tableHandle).values()); - MaterializedResult result = readTable(transaction, tableHandle, columnHandles, session, TupleDomain.all(), OptionalInt.empty(), Optional.of(storageFormat)); - assertThat(result.getRowCount()).isEqualTo(rowCount); - } - } - - private void markTableAsCreatedBySpark(SchemaTableName tableName, String provider) - { - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - PrincipalPrivileges principalPrivileges = testingPrincipalPrivilege(session); - Table oldTable = transaction.getMetastore().getTable(tableName.getSchemaName(), tableName.getTableName()).get(); - Table.Builder newTable = Table.builder(oldTable).setParameter(SPARK_TABLE_PROVIDER_KEY, provider); - transaction.getMetastore().replaceTable(tableName.getSchemaName(), tableName.getTableName(), newTable.build(), principalPrivileges); - transaction.commit(); - } - } - - private void createExternalTable(SchemaTableName schemaTableName, HiveStorageFormat hiveStorageFormat, List columns, List partitionColumns, Optional bucketProperty, Location externalLocation) - { - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - - String tableOwner = session.getUser(); - String schemaName = schemaTableName.getSchemaName(); - String tableName = schemaTableName.getTableName(); - - Table.Builder tableBuilder = Table.builder() - .setDatabaseName(schemaName) - .setTableName(tableName) - .setOwner(Optional.of(tableOwner)) - .setTableType(EXTERNAL_TABLE.name()) - .setParameters(ImmutableMap.of( - TRINO_VERSION_NAME, TEST_SERVER_VERSION, - TRINO_QUERY_ID_NAME, session.getQueryId())) - .setDataColumns(columns) - .setPartitionColumns(partitionColumns); - - tableBuilder.getStorageBuilder() - .setLocation(externalLocation.toString()) - .setStorageFormat(StorageFormat.create(hiveStorageFormat.getSerde(), hiveStorageFormat.getInputFormat(), hiveStorageFormat.getOutputFormat())) - .setBucketProperty(bucketProperty) - .setSerdeParameters(ImmutableMap.of()); - - PrincipalPrivileges principalPrivileges = testingPrincipalPrivilege(tableOwner, session.getUser()); - transaction.getMetastore().createTable(session, tableBuilder.build(), principalPrivileges, Optional.of(externalLocation), Optional.empty(), true, ZERO_TABLE_STATISTICS, false); - - transaction.commit(); - } - } - - private Path copyResourceDirToTemporaryDirectory(String resourceName) - throws IOException - { - Path tempDir = java.nio.file.Files.createTempDirectory(getClass().getSimpleName()).normalize(); - log.info("Copying resource dir '%s' to %s", resourceName, tempDir); - ClassPath.from(getClass().getClassLoader()) - .getResources().stream() - .filter(resourceInfo -> resourceInfo.getResourceName().startsWith(resourceName)) - .forEach(resourceInfo -> { - try { - Path target = tempDir.resolve(resourceInfo.getResourceName()); - java.nio.file.Files.createDirectories(target.getParent()); - try (InputStream inputStream = resourceInfo.asByteSource().openStream()) { - copy(inputStream, target); - } - } - catch (IOException e) { - throw new UncheckedIOException(e); - } - }); - return tempDir.resolve(resourceName).normalize(); - } -} diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileMetastore.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileMetastore.java deleted file mode 100644 index 2a7f1deb12524..0000000000000 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileMetastore.java +++ /dev/null @@ -1,84 +0,0 @@ -/* - * 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.hive; - -import io.trino.plugin.hive.metastore.HiveMetastore; -import io.trino.plugin.hive.metastore.file.FileHiveMetastore; -import io.trino.plugin.hive.metastore.file.FileHiveMetastoreConfig; -import org.junit.jupiter.api.Test; - -import java.io.File; - -import static io.trino.plugin.hive.HiveTestUtils.HDFS_FILE_SYSTEM_FACTORY; -import static org.junit.jupiter.api.Assumptions.abort; - -public class TestHiveFileMetastore - extends AbstractTestHiveLocal -{ - @Override - protected HiveMetastore createMetastore(File tempDir) - { - File baseDir = new File(tempDir, "metastore"); - return new FileHiveMetastore( - new NodeVersion("test_version"), - HDFS_FILE_SYSTEM_FACTORY, - true, - new FileHiveMetastoreConfig() - .setCatalogDirectory(baseDir.toURI().toString()) - .setMetastoreUser("test")); - } - - @Test - @Override - public void testMismatchSchemaTable() - { - // FileHiveMetastore only supports replaceTable() for views - } - - @Test - @Override - public void testPartitionSchemaMismatch() - { - // test expects an exception to be thrown - abort("FileHiveMetastore only supports replaceTable() for views"); - } - - @Test - @Override - public void testBucketedTableEvolution() - { - // FileHiveMetastore only supports replaceTable() for views - } - - @Test - @Override - public void testBucketedTableEvolutionWithDifferentReadBucketCount() - { - // FileHiveMetastore has various incompatibilities - } - - @Test - @Override - public void testTransactionDeleteInsert() - { - // FileHiveMetastore has various incompatibilities - } - - @Test - @Override - public void testInsertOverwriteUnpartitioned() - { - // FileHiveMetastore has various incompatibilities - } -} diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/glue/TestGlueHiveMetastore.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/glue/TestGlueHiveMetastore.java deleted file mode 100644 index 6d95ddbb5deab..0000000000000 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/glue/TestGlueHiveMetastore.java +++ /dev/null @@ -1,1605 +0,0 @@ -/* - * 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.hive.metastore.glue; - -import com.amazonaws.services.glue.AWSGlueAsync; -import com.amazonaws.services.glue.AWSGlueAsyncClientBuilder; -import com.amazonaws.services.glue.model.CreateTableRequest; -import com.amazonaws.services.glue.model.Database; -import com.amazonaws.services.glue.model.DeleteDatabaseRequest; -import com.amazonaws.services.glue.model.DeleteTableRequest; -import com.amazonaws.services.glue.model.EntityNotFoundException; -import com.amazonaws.services.glue.model.GetDatabasesRequest; -import com.amazonaws.services.glue.model.GetDatabasesResult; -import com.amazonaws.services.glue.model.TableInput; -import com.amazonaws.services.glue.model.UpdateTableRequest; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import io.airlift.concurrent.BoundedExecutor; -import io.airlift.log.Logger; -import io.airlift.slice.Slice; -import io.trino.plugin.hive.AbstractTestHiveLocal; -import io.trino.plugin.hive.HiveBasicStatistics; -import io.trino.plugin.hive.HiveMetastoreClosure; -import io.trino.plugin.hive.HiveType; -import io.trino.plugin.hive.PartitionStatistics; -import io.trino.plugin.hive.metastore.HiveColumnStatistics; -import io.trino.plugin.hive.metastore.HiveMetastore; -import io.trino.plugin.hive.metastore.PartitionWithStatistics; -import io.trino.plugin.hive.metastore.Table; -import io.trino.plugin.hive.metastore.glue.converter.GlueInputConverter; -import io.trino.spi.TrinoException; -import io.trino.spi.block.Block; -import io.trino.spi.block.BlockBuilder; -import io.trino.spi.connector.ColumnMetadata; -import io.trino.spi.connector.ConnectorMetadata; -import io.trino.spi.connector.ConnectorOutputTableHandle; -import io.trino.spi.connector.ConnectorPageSink; -import io.trino.spi.connector.ConnectorSession; -import io.trino.spi.connector.ConnectorTableMetadata; -import io.trino.spi.connector.SchemaTableName; -import io.trino.spi.connector.TableNotFoundException; -import io.trino.spi.predicate.Domain; -import io.trino.spi.predicate.Range; -import io.trino.spi.predicate.TupleDomain; -import io.trino.spi.predicate.ValueSet; -import io.trino.spi.statistics.ComputedStatistics; -import io.trino.spi.statistics.TableStatisticType; -import io.trino.spi.type.BigintType; -import io.trino.spi.type.DateType; -import io.trino.spi.type.IntegerType; -import io.trino.spi.type.SmallintType; -import io.trino.spi.type.TimestampType; -import io.trino.spi.type.TinyintType; -import io.trino.spi.type.VarcharType; -import io.trino.testing.MaterializedResult; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Test; - -import java.io.File; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.OptionalLong; -import java.util.Set; -import java.util.concurrent.Executor; -import java.util.function.Supplier; - -import static com.google.common.collect.ImmutableList.toImmutableList; -import static io.airlift.concurrent.MoreFutures.getFutureValue; -import static io.airlift.slice.Slices.utf8Slice; -import static io.trino.plugin.hive.HiveBasicStatistics.createEmptyStatistics; -import static io.trino.plugin.hive.HiveColumnStatisticType.MAX_VALUE; -import static io.trino.plugin.hive.HiveColumnStatisticType.MIN_VALUE; -import static io.trino.plugin.hive.HiveColumnStatisticType.NUMBER_OF_DISTINCT_VALUES; -import static io.trino.plugin.hive.HiveColumnStatisticType.NUMBER_OF_NON_NULL_VALUES; -import static io.trino.plugin.hive.HiveMetadata.TABLE_COMMENT; -import static io.trino.plugin.hive.HiveStorageFormat.ORC; -import static io.trino.plugin.hive.HiveStorageFormat.TEXTFILE; -import static io.trino.plugin.hive.HiveTestUtils.HDFS_FILE_SYSTEM_FACTORY; -import static io.trino.plugin.hive.TableType.EXTERNAL_TABLE; -import static io.trino.plugin.hive.TableType.VIRTUAL_VIEW; -import static io.trino.plugin.hive.ViewReaderUtil.ICEBERG_MATERIALIZED_VIEW_COMMENT; -import static io.trino.plugin.hive.ViewReaderUtil.PRESTO_VIEW_FLAG; -import static io.trino.plugin.hive.ViewReaderUtil.isTrinoMaterializedView; -import static io.trino.plugin.hive.acid.AcidTransaction.NO_ACID_TRANSACTION; -import static io.trino.plugin.hive.metastore.HiveColumnStatistics.createIntegerColumnStatistics; -import static io.trino.plugin.hive.metastore.glue.AwsSdkUtil.getPaginatedResults; -import static io.trino.plugin.hive.metastore.glue.PartitionFilterBuilder.DECIMAL_TYPE; -import static io.trino.plugin.hive.metastore.glue.PartitionFilterBuilder.decimalOf; -import static io.trino.plugin.hive.metastore.glue.TestingGlueHiveMetastore.createTestingAsyncGlueClient; -import static io.trino.plugin.hive.util.HiveUtil.DELTA_LAKE_PROVIDER; -import static io.trino.plugin.hive.util.HiveUtil.ICEBERG_TABLE_TYPE_NAME; -import static io.trino.plugin.hive.util.HiveUtil.ICEBERG_TABLE_TYPE_VALUE; -import static io.trino.plugin.hive.util.HiveUtil.SPARK_TABLE_PROVIDER_KEY; -import static io.trino.plugin.hive.util.HiveUtil.isDeltaLakeTable; -import static io.trino.plugin.hive.util.HiveUtil.isIcebergTable; -import static io.trino.spi.connector.RetryMode.NO_RETRIES; -import static io.trino.spi.type.BigintType.BIGINT; -import static io.trino.spi.type.VarcharType.VARCHAR; -import static io.trino.spi.type.VarcharType.createUnboundedVarcharType; -import static io.trino.testing.TestingPageSinkId.TESTING_PAGE_SINK_ID; -import static io.trino.type.InternalTypeManager.TESTING_TYPE_MANAGER; -import static java.lang.String.format; -import static java.lang.System.currentTimeMillis; -import static java.util.Collections.unmodifiableList; -import static java.util.Locale.ENGLISH; -import static java.util.Objects.requireNonNull; -import static java.util.UUID.randomUUID; -import static java.util.concurrent.TimeUnit.DAYS; -import static org.apache.hadoop.hive.common.FileUtils.makePartName; -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.junit.jupiter.api.Assumptions.abort; - -/* - * GlueHiveMetastore currently uses AWS Default Credential Provider Chain, - * See https://docs.aws.amazon.com/sdk-for-java/v1/developer-guide/credentials.html#credentials-default - * on ways to set your AWS credentials which will be needed to run this test. - */ -public class TestGlueHiveMetastore - extends AbstractTestHiveLocal -{ - private static final Logger log = Logger.get(TestGlueHiveMetastore.class); - - private static final String PARTITION_KEY = "part_key_1"; - private static final String PARTITION_KEY2 = "part_key_2"; - private static final String TEST_DATABASE_NAME_PREFIX = "test_glue"; - - private static final List CREATE_TABLE_COLUMNS = ImmutableList.of(new ColumnMetadata("id", BIGINT)); - private static final List CREATE_TABLE_COLUMNS_PARTITIONED_VARCHAR = ImmutableList.builder() - .addAll(CREATE_TABLE_COLUMNS) - .add(new ColumnMetadata(PARTITION_KEY, VarcharType.VARCHAR)) - .build(); - private static final List CREATE_TABLE_COLUMNS_PARTITIONED_TWO_KEYS = ImmutableList.builder() - .addAll(CREATE_TABLE_COLUMNS) - .add(new ColumnMetadata(PARTITION_KEY, VarcharType.VARCHAR)) - .add(new ColumnMetadata(PARTITION_KEY2, BigintType.BIGINT)) - .build(); - private static final List CREATE_TABLE_COLUMNS_PARTITIONED_TINYINT = ImmutableList.builder() - .addAll(CREATE_TABLE_COLUMNS) - .add(new ColumnMetadata(PARTITION_KEY, TinyintType.TINYINT)) - .build(); - private static final List CREATE_TABLE_COLUMNS_PARTITIONED_SMALLINT = ImmutableList.builder() - .addAll(CREATE_TABLE_COLUMNS) - .add(new ColumnMetadata(PARTITION_KEY, SmallintType.SMALLINT)) - .build(); - private static final List CREATE_TABLE_COLUMNS_PARTITIONED_INTEGER = ImmutableList.builder() - .addAll(CREATE_TABLE_COLUMNS) - .add(new ColumnMetadata(PARTITION_KEY, IntegerType.INTEGER)) - .build(); - private static final List CREATE_TABLE_COLUMNS_PARTITIONED_BIGINT = ImmutableList.builder() - .addAll(CREATE_TABLE_COLUMNS) - .add(new ColumnMetadata(PARTITION_KEY, BigintType.BIGINT)) - .build(); - private static final List CREATE_TABLE_COLUMNS_PARTITIONED_DECIMAL = ImmutableList.builder() - .addAll(CREATE_TABLE_COLUMNS) - .add(new ColumnMetadata(PARTITION_KEY, DECIMAL_TYPE)) - .build(); - private static final List CREATE_TABLE_COLUMNS_PARTITIONED_DATE = ImmutableList.builder() - .addAll(CREATE_TABLE_COLUMNS) - .add(new ColumnMetadata(PARTITION_KEY, DateType.DATE)) - .build(); - private static final List CREATE_TABLE_COLUMNS_PARTITIONED_TIMESTAMP = ImmutableList.builder() - .addAll(CREATE_TABLE_COLUMNS) - .add(new ColumnMetadata(PARTITION_KEY, TimestampType.TIMESTAMP_MILLIS)) - .build(); - private static final List VARCHAR_PARTITION_VALUES = ImmutableList.of("2020-01-01", "2020-02-01", "2020-03-01", "2020-04-01"); - - protected static final HiveBasicStatistics HIVE_BASIC_STATISTICS = new HiveBasicStatistics(1000, 5000, 3000, 4000); - protected static final HiveColumnStatistics INTEGER_COLUMN_STATISTICS = createIntegerColumnStatistics( - OptionalLong.of(-1000), - OptionalLong.of(1000), - OptionalLong.of(1), - OptionalLong.of(2)); - - private HiveMetastoreClosure metastore; - private AWSGlueAsync glueClient; - - public TestGlueHiveMetastore() - { - super(TEST_DATABASE_NAME_PREFIX + randomUUID().toString().toLowerCase(ENGLISH).replace("-", "")); - } - - protected AWSGlueAsync getGlueClient() - { - return glueClient; - } - - @BeforeAll - @Override - public void initialize() - throws Exception - { - super.initialize(); - // uncomment to get extra AWS debug information -// Logging logging = Logging.initialize(); -// logging.setLevel("com.amazonaws.request", Level.DEBUG); - - metastore = new HiveMetastoreClosure(metastoreClient, TESTING_TYPE_MANAGER, false); - glueClient = AWSGlueAsyncClientBuilder.defaultClient(); - } - - @Override - protected HiveMetastore createMetastore(File tempDir) - { - GlueHiveMetastoreConfig glueConfig = new GlueHiveMetastoreConfig(); - glueConfig.setDefaultWarehouseDir(tempDir.toURI().toString()); - glueConfig.setAssumeCanonicalPartitionKeys(true); - - Executor executor = new BoundedExecutor(this.executor, 10); - GlueMetastoreStats stats = new GlueMetastoreStats(); - return new GlueHiveMetastore( - HDFS_FILE_SYSTEM_FACTORY, - glueConfig, - executor, - new DefaultGlueColumnStatisticsProviderFactory(executor, executor), - createTestingAsyncGlueClient(glueConfig, stats), - stats, - new DefaultGlueMetastoreTableFilterProvider(true).get()); - } - - @Test - public void cleanupOrphanedDatabases() - { - long creationTimeMillisThreshold = currentTimeMillis() - DAYS.toMillis(1); - GlueHiveMetastore metastore = (GlueHiveMetastore) getMetastoreClient(); - GlueMetastoreStats stats = metastore.getStats(); - List orphanedDatabases = getPaginatedResults( - glueClient::getDatabases, - new GetDatabasesRequest(), - GetDatabasesRequest::setNextToken, - GetDatabasesResult::getNextToken, - stats.getGetDatabases()) - .map(GetDatabasesResult::getDatabaseList) - .flatMap(List::stream) - .filter(database -> database.getName().startsWith(TEST_DATABASE_NAME_PREFIX) && - database.getCreateTime().getTime() <= creationTimeMillisThreshold) - .map(Database::getName) - .collect(toImmutableList()); - - log.info("Found %s %s* databases that look orphaned, removing", orphanedDatabases.size(), TEST_DATABASE_NAME_PREFIX); - orphanedDatabases.forEach(database -> { - try { - glueClient.deleteDatabase(new DeleteDatabaseRequest() - .withName(database)); - } - catch (EntityNotFoundException e) { - log.info("Database [%s] not found, could be removed by other cleanup process", database); - } - catch (RuntimeException e) { - log.warn(e, "Failed to remove database [%s]", database); - } - }); - } - - @Test - @Override - public void testRenameTable() - { - // rename table is not yet supported by Glue - } - - @Test - @Override - public void testUpdateTableColumnStatisticsEmptyOptionalFields() - { - // this test expects consistency between written and read stats but this is not provided by glue at the moment - // when writing empty min/max statistics glue will return 0 to the readers - // in order to avoid incorrect data we skip writes for statistics with min/max = null - } - - @Test - @Override - public void testUpdatePartitionColumnStatisticsEmptyOptionalFields() - { - // this test expects consistency between written and read stats but this is not provided by glue at the moment - // when writing empty min/max statistics glue will return 0 to the readers - // in order to avoid incorrect data we skip writes for statistics with min/max = null - } - - @Test - @Override - public void testUpdateBasicPartitionStatistics() - throws Exception - { - SchemaTableName tableName = temporaryTable("update_basic_partition_statistics"); - try { - createDummyPartitionedTable(tableName, STATISTICS_PARTITIONED_TABLE_COLUMNS); - testUpdatePartitionStatistics( - tableName, - EMPTY_ROWCOUNT_STATISTICS, - ImmutableList.of(BASIC_STATISTICS_1, BASIC_STATISTICS_2), - ImmutableList.of(BASIC_STATISTICS_2, BASIC_STATISTICS_1)); - } - finally { - dropTable(tableName); - } - } - - @Test - @Override - public void testUpdatePartitionColumnStatistics() - throws Exception - { - SchemaTableName tableName = temporaryTable("update_partition_column_statistics"); - try { - createDummyPartitionedTable(tableName, STATISTICS_PARTITIONED_TABLE_COLUMNS); - // When the table has partitions, but row count statistics are set to zero, we treat this case as empty - // statistics to avoid underestimation in the CBO. This scenario may be caused when other engines are - // used to ingest data into partitioned hive tables. - testUpdatePartitionStatistics( - tableName, - EMPTY_ROWCOUNT_STATISTICS, - ImmutableList.of(STATISTICS_1_1, STATISTICS_1_2, STATISTICS_2), - ImmutableList.of(STATISTICS_1_2, STATISTICS_1_1, STATISTICS_2)); - } - finally { - dropTable(tableName); - } - } - - @Test - @Override - public void testStorePartitionWithStatistics() - throws Exception - { - // When the table has partitions, but row count statistics are set to zero, we treat this case as empty - // statistics to avoid underestimation in the CBO. This scenario may be caused when other engines are - // used to ingest data into partitioned hive tables. - testStorePartitionWithStatistics(STATISTICS_PARTITIONED_TABLE_COLUMNS, BASIC_STATISTICS_1, BASIC_STATISTICS_2, BASIC_STATISTICS_1, EMPTY_ROWCOUNT_STATISTICS); - } - - @Test - @Override - public void testGetPartitions() - throws Exception - { - try { - SchemaTableName tableName = temporaryTable("get_partitions"); - createDummyPartitionedTable(tableName, CREATE_TABLE_COLUMNS_PARTITIONED); - HiveMetastore metastoreClient = getMetastoreClient(); - Optional> partitionNames = metastoreClient.getPartitionNamesByFilter( - tableName.getSchemaName(), - tableName.getTableName(), - ImmutableList.of("ds"), TupleDomain.all()); - assertThat(partitionNames.isPresent()).isTrue(); - assertThat(partitionNames.get()).isEqualTo(ImmutableList.of("ds=2016-01-01", "ds=2016-01-02")); - } - finally { - dropTable(tablePartitionFormat); - } - } - - @Test - public void testGetPartitionsWithFilterUsingReservedKeywordsAsColumnName() - throws Exception - { - SchemaTableName tableName = temporaryTable("get_partitions_with_filter_using_reserved_keyword_column_name"); - try { - String reservedKeywordPartitionColumnName = "key"; - String regularColumnPartitionName = "int_partition"; - List columns = ImmutableList.builder() - .add(new ColumnMetadata("t_string", createUnboundedVarcharType())) - .add(new ColumnMetadata(reservedKeywordPartitionColumnName, createUnboundedVarcharType())) - .add(new ColumnMetadata(regularColumnPartitionName, BIGINT)) - .build(); - List partitionedBy = ImmutableList.of(reservedKeywordPartitionColumnName, regularColumnPartitionName); - - doCreateEmptyTable(tableName, ORC, columns, partitionedBy); - - HiveMetastoreClosure metastoreClient = new HiveMetastoreClosure(getMetastoreClient(), TESTING_TYPE_MANAGER, false); - Table table = metastoreClient.getTable(tableName.getSchemaName(), tableName.getTableName()) - .orElseThrow(() -> new TableNotFoundException(tableName)); - - String partitionName1 = makePartName(ImmutableList.of(reservedKeywordPartitionColumnName, regularColumnPartitionName), ImmutableList.of("value1", "1")); - String partitionName2 = makePartName(ImmutableList.of(reservedKeywordPartitionColumnName, regularColumnPartitionName), ImmutableList.of("value2", "2")); - - List partitions = ImmutableList.of(partitionName1, partitionName2) - .stream() - .map(partitionName -> new PartitionWithStatistics(createDummyPartition(table, partitionName), partitionName, PartitionStatistics.empty())) - .collect(toImmutableList()); - metastoreClient.addPartitions(tableName.getSchemaName(), tableName.getTableName(), partitions); - metastoreClient.updatePartitionsStatistics(tableName.getSchemaName(), tableName.getTableName(), partitionName1, currentStatistics -> ZERO_TABLE_STATISTICS); - metastoreClient.updatePartitionsStatistics(tableName.getSchemaName(), tableName.getTableName(), partitionName2, currentStatistics -> ZERO_TABLE_STATISTICS); - - Optional> partitionNames = metastoreClient.getPartitionNamesByFilter( - tableName.getSchemaName(), - tableName.getTableName(), - ImmutableList.of(reservedKeywordPartitionColumnName, regularColumnPartitionName), - TupleDomain.withColumnDomains(ImmutableMap.of(regularColumnPartitionName, Domain.singleValue(BIGINT, 2L)))); - assertThat(partitionNames.isPresent()).isTrue(); - assertThat(partitionNames.get()).isEqualTo(ImmutableList.of("key=value2/int_partition=2")); - - // KEY is a reserved keyword in the grammar of the SQL parser used internally by Glue API - // and therefore should not be used in the partition filter - partitionNames = metastoreClient.getPartitionNamesByFilter( - tableName.getSchemaName(), - tableName.getTableName(), - ImmutableList.of(reservedKeywordPartitionColumnName, regularColumnPartitionName), - TupleDomain.withColumnDomains(ImmutableMap.of(reservedKeywordPartitionColumnName, Domain.singleValue(VARCHAR, utf8Slice("value1"))))); - assertThat(partitionNames.isPresent()).isTrue(); - assertThat(partitionNames.get()).isEqualTo(ImmutableList.of("key=value1/int_partition=1", "key=value2/int_partition=2")); - } - finally { - dropTable(tableName); - } - } - - @Test - public void testGetDatabasesLogsStats() - { - GlueHiveMetastore metastore = (GlueHiveMetastore) getMetastoreClient(); - GlueMetastoreStats stats = metastore.getStats(); - double initialCallCount = stats.getGetDatabases().getTime().getAllTime().getCount(); - long initialFailureCount = stats.getGetDatabases().getTotalFailures().getTotalCount(); - getMetastoreClient().getAllDatabases(); - assertThat(stats.getGetDatabases().getTime().getAllTime().getCount()).isGreaterThan(initialCallCount); - assertThat(stats.getGetDatabases().getTime().getAllTime().getAvg()).isGreaterThan(0.0); - assertThat(stats.getGetDatabases().getTotalFailures().getTotalCount()).isEqualTo(initialFailureCount); - } - - @Test - public void testGetDatabaseFailureLogsStats() - { - GlueHiveMetastore metastore = (GlueHiveMetastore) getMetastoreClient(); - GlueMetastoreStats stats = metastore.getStats(); - long initialFailureCount = stats.getGetDatabase().getTotalFailures().getTotalCount(); - assertThatThrownBy(() -> getMetastoreClient().getDatabase(null)) - .isInstanceOf(TrinoException.class) - .hasMessageStartingWith("Database name cannot be equal to null or empty"); - assertThat(stats.getGetDatabase().getTotalFailures().getTotalCount()).isEqualTo(initialFailureCount + 1); - } - - @Test - public void testGetPartitionsFilterVarChar() - throws Exception - { - TupleDomain singleEquals = new PartitionFilterBuilder() - .addStringValues(PARTITION_KEY, "2020-01-01") - .build(); - TupleDomain greaterThan = new PartitionFilterBuilder() - .addRanges(PARTITION_KEY, Range.greaterThan(VarcharType.VARCHAR, utf8Slice("2020-02-01"))) - .build(); - TupleDomain betweenInclusive = new PartitionFilterBuilder() - .addRanges(PARTITION_KEY, Range.range(VarcharType.VARCHAR, utf8Slice("2020-02-01"), true, utf8Slice("2020-03-01"), true)) - .build(); - TupleDomain greaterThanOrEquals = new PartitionFilterBuilder() - .addRanges(PARTITION_KEY, Range.greaterThanOrEqual(VarcharType.VARCHAR, utf8Slice("2020-03-01"))) - .build(); - TupleDomain inClause = new PartitionFilterBuilder() - .addStringValues(PARTITION_KEY, "2020-01-01", "2020-02-01") - .build(); - TupleDomain lessThan = new PartitionFilterBuilder() - .addRanges(PARTITION_KEY, Range.lessThan(VarcharType.VARCHAR, utf8Slice("2020-03-01"))) - .build(); - doGetPartitionsFilterTest( - CREATE_TABLE_COLUMNS_PARTITIONED_VARCHAR, - PARTITION_KEY, - VARCHAR_PARTITION_VALUES, - ImmutableList.of(singleEquals, greaterThan, betweenInclusive, greaterThanOrEquals, inClause, lessThan, TupleDomain.all()), - ImmutableList.of( - ImmutableList.of("2020-01-01"), - ImmutableList.of("2020-03-01", "2020-04-01"), - ImmutableList.of("2020-02-01", "2020-03-01"), - ImmutableList.of("2020-03-01", "2020-04-01"), - ImmutableList.of("2020-01-01", "2020-02-01"), - ImmutableList.of("2020-01-01", "2020-02-01"), - ImmutableList.of("2020-01-01", "2020-02-01", "2020-03-01", "2020-04-01"))); - } - - @Test - public void testGetPartitionsFilterBigInt() - throws Exception - { - TupleDomain singleEquals = new PartitionFilterBuilder() - .addBigintValues(PARTITION_KEY, 1000L) - .build(); - TupleDomain greaterThan = new PartitionFilterBuilder() - .addRanges(PARTITION_KEY, Range.greaterThan(BigintType.BIGINT, 100L)) - .build(); - TupleDomain betweenInclusive = new PartitionFilterBuilder() - .addRanges(PARTITION_KEY, Range.range(BigintType.BIGINT, 100L, true, 1000L, true)) - .build(); - TupleDomain greaterThanOrEquals = new PartitionFilterBuilder() - .addRanges(PARTITION_KEY, Range.greaterThanOrEqual(BigintType.BIGINT, 100L)) - .build(); - TupleDomain inClause = new PartitionFilterBuilder() - .addBigintValues(PARTITION_KEY, 1L, 1000000L) - .build(); - TupleDomain lessThan = new PartitionFilterBuilder() - .addRanges(PARTITION_KEY, Range.lessThan(BigintType.BIGINT, 1000L)) - .build(); - doGetPartitionsFilterTest( - CREATE_TABLE_COLUMNS_PARTITIONED_BIGINT, - PARTITION_KEY, - ImmutableList.of("1", "100", "1000", "1000000"), - ImmutableList.of(singleEquals, greaterThan, betweenInclusive, greaterThanOrEquals, inClause, lessThan, TupleDomain.all()), - ImmutableList.of( - ImmutableList.of("1000"), - ImmutableList.of("1000", "1000000"), - ImmutableList.of("100", "1000"), - ImmutableList.of("100", "1000", "1000000"), - ImmutableList.of("1", "1000000"), - ImmutableList.of("1", "100"), - ImmutableList.of("1", "100", "1000", "1000000"))); - } - - @Test - public void testGetPartitionsFilterInteger() - throws Exception - { - TupleDomain singleEquals = new PartitionFilterBuilder() - .addIntegerValues(PARTITION_KEY, 1000L) - .build(); - TupleDomain greaterThan = new PartitionFilterBuilder() - .addRanges(PARTITION_KEY, Range.greaterThan(IntegerType.INTEGER, 100L)) - .build(); - TupleDomain betweenInclusive = new PartitionFilterBuilder() - .addRanges(PARTITION_KEY, Range.range(IntegerType.INTEGER, 100L, true, 1000L, true)) - .build(); - TupleDomain greaterThanOrEquals = new PartitionFilterBuilder() - .addRanges(PARTITION_KEY, Range.greaterThanOrEqual(IntegerType.INTEGER, 100L)) - .build(); - TupleDomain inClause = new PartitionFilterBuilder() - .addIntegerValues(PARTITION_KEY, 1L, 1000000L) - .build(); - TupleDomain lessThan = new PartitionFilterBuilder() - .addRanges(PARTITION_KEY, Range.lessThan(IntegerType.INTEGER, 1000L)) - .build(); - doGetPartitionsFilterTest( - CREATE_TABLE_COLUMNS_PARTITIONED_INTEGER, - PARTITION_KEY, - ImmutableList.of("1", "100", "1000", "1000000"), - ImmutableList.of(singleEquals, greaterThan, betweenInclusive, greaterThanOrEquals, inClause, lessThan, TupleDomain.all()), - ImmutableList.of( - ImmutableList.of("1000"), - ImmutableList.of("1000", "1000000"), - ImmutableList.of("100", "1000"), - ImmutableList.of("100", "1000", "1000000"), - ImmutableList.of("1", "1000000"), - ImmutableList.of("1", "100"), - ImmutableList.of("1", "100", "1000", "1000000"))); - } - - @Test - public void testGetPartitionsFilterSmallInt() - throws Exception - { - TupleDomain singleEquals = new PartitionFilterBuilder() - .addSmallintValues(PARTITION_KEY, 1000L) - .build(); - TupleDomain greaterThan = new PartitionFilterBuilder() - .addRanges(PARTITION_KEY, Range.greaterThan(SmallintType.SMALLINT, 100L)) - .build(); - TupleDomain betweenInclusive = new PartitionFilterBuilder() - .addRanges(PARTITION_KEY, Range.range(SmallintType.SMALLINT, 100L, true, 1000L, true)) - .build(); - TupleDomain greaterThanOrEquals = new PartitionFilterBuilder() - .addRanges(PARTITION_KEY, Range.greaterThanOrEqual(SmallintType.SMALLINT, 100L)) - .build(); - TupleDomain inClause = new PartitionFilterBuilder() - .addSmallintValues(PARTITION_KEY, 1L, 10000L) - .build(); - TupleDomain lessThan = new PartitionFilterBuilder() - .addRanges(PARTITION_KEY, Range.lessThan(SmallintType.SMALLINT, 1000L)) - .build(); - doGetPartitionsFilterTest( - CREATE_TABLE_COLUMNS_PARTITIONED_SMALLINT, - PARTITION_KEY, - ImmutableList.of("1", "100", "1000", "10000"), - ImmutableList.of(singleEquals, greaterThan, betweenInclusive, greaterThanOrEquals, inClause, lessThan, TupleDomain.all()), - ImmutableList.of( - ImmutableList.of("1000"), - ImmutableList.of("1000", "10000"), - ImmutableList.of("100", "1000"), - ImmutableList.of("100", "1000", "10000"), - ImmutableList.of("1", "10000"), - ImmutableList.of("1", "100"), - ImmutableList.of("1", "100", "1000", "10000"))); - } - - @Test - public void testGetPartitionsFilterTinyInt() - throws Exception - { - TupleDomain singleEquals = new PartitionFilterBuilder() - .addTinyintValues(PARTITION_KEY, 127L) - .build(); - TupleDomain greaterThan = new PartitionFilterBuilder() - .addRanges(PARTITION_KEY, Range.greaterThan(TinyintType.TINYINT, 10L)) - .build(); - TupleDomain betweenInclusive = new PartitionFilterBuilder() - .addRanges(PARTITION_KEY, Range.range(TinyintType.TINYINT, 10L, true, 100L, true)) - .build(); - TupleDomain greaterThanOrEquals = new PartitionFilterBuilder() - .addRanges(PARTITION_KEY, Range.greaterThanOrEqual(TinyintType.TINYINT, 10L)) - .build(); - TupleDomain inClause = new PartitionFilterBuilder() - .addTinyintValues(PARTITION_KEY, 1L, 127L) - .build(); - TupleDomain lessThan = new PartitionFilterBuilder() - .addRanges(PARTITION_KEY, Range.lessThan(TinyintType.TINYINT, 100L)) - .build(); - doGetPartitionsFilterTest( - CREATE_TABLE_COLUMNS_PARTITIONED_TINYINT, - PARTITION_KEY, - ImmutableList.of("1", "10", "100", "127"), - ImmutableList.of(singleEquals, greaterThan, betweenInclusive, greaterThanOrEquals, inClause, lessThan, TupleDomain.all()), - ImmutableList.of( - ImmutableList.of("127"), - ImmutableList.of("100", "127"), - ImmutableList.of("10", "100"), - ImmutableList.of("10", "100", "127"), - ImmutableList.of("1", "127"), - ImmutableList.of("1", "10"), - ImmutableList.of("1", "10", "100", "127"))); - } - - @Test - public void testGetPartitionsFilterTinyIntNegatives() - throws Exception - { - TupleDomain singleEquals = new PartitionFilterBuilder() - .addTinyintValues(PARTITION_KEY, -128L) - .build(); - TupleDomain greaterThan = new PartitionFilterBuilder() - .addRanges(PARTITION_KEY, Range.greaterThan(TinyintType.TINYINT, 0L)) - .build(); - TupleDomain betweenInclusive = new PartitionFilterBuilder() - .addRanges(PARTITION_KEY, Range.range(TinyintType.TINYINT, 0L, true, 50L, true)) - .build(); - TupleDomain greaterThanOrEquals = new PartitionFilterBuilder() - .addRanges(PARTITION_KEY, Range.greaterThanOrEqual(TinyintType.TINYINT, 0L)) - .build(); - TupleDomain inClause = new PartitionFilterBuilder() - .addTinyintValues(PARTITION_KEY, 0L, -128L) - .build(); - TupleDomain lessThan = new PartitionFilterBuilder() - .addRanges(PARTITION_KEY, Range.lessThan(TinyintType.TINYINT, 0L)) - .build(); - doGetPartitionsFilterTest( - CREATE_TABLE_COLUMNS_PARTITIONED_TINYINT, - PARTITION_KEY, - ImmutableList.of("-128", "0", "50", "100"), - ImmutableList.of(singleEquals, greaterThan, betweenInclusive, greaterThanOrEquals, inClause, lessThan, TupleDomain.all()), - ImmutableList.of( - ImmutableList.of("-128"), - ImmutableList.of("100", "50"), - ImmutableList.of("0", "50"), - ImmutableList.of("0", "100", "50"), - ImmutableList.of("-128", "0"), - ImmutableList.of("-128"), - ImmutableList.of("-128", "0", "100", "50"))); - } - - @Test - public void testGetPartitionsFilterDecimal() - throws Exception - { - String value1 = "1.000"; - String value2 = "10.134"; - String value3 = "25.111"; - String value4 = "30.333"; - - TupleDomain singleEquals = new PartitionFilterBuilder() - .addDecimalValues(PARTITION_KEY, value1) - .build(); - TupleDomain greaterThan = new PartitionFilterBuilder() - .addRanges(PARTITION_KEY, Range.greaterThan(DECIMAL_TYPE, decimalOf(value2))) - .build(); - TupleDomain betweenInclusive = new PartitionFilterBuilder() - .addRanges(PARTITION_KEY, Range.range(DECIMAL_TYPE, decimalOf(value2), true, decimalOf(value3), true)) - .build(); - TupleDomain greaterThanOrEquals = new PartitionFilterBuilder() - .addRanges(PARTITION_KEY, Range.greaterThanOrEqual(DECIMAL_TYPE, decimalOf(value3))) - .build(); - TupleDomain inClause = new PartitionFilterBuilder() - .addDecimalValues(PARTITION_KEY, value1, value4) - .build(); - TupleDomain lessThan = new PartitionFilterBuilder() - .addRanges(PARTITION_KEY, Range.lessThan(DECIMAL_TYPE, decimalOf("25.5"))) - .build(); - doGetPartitionsFilterTest( - CREATE_TABLE_COLUMNS_PARTITIONED_DECIMAL, - PARTITION_KEY, - ImmutableList.of(value1, value2, value3, value4), - ImmutableList.of(singleEquals, greaterThan, betweenInclusive, greaterThanOrEquals, inClause, lessThan, TupleDomain.all()), - ImmutableList.of( - ImmutableList.of(value1), - ImmutableList.of(value3, value4), - ImmutableList.of(value2, value3), - ImmutableList.of(value3, value4), - ImmutableList.of(value1, value4), - ImmutableList.of(value1, value2, value3), - ImmutableList.of(value1, value2, value3, value4))); - } - - // we don't presently know how to properly convert a Date type into a string that is compatible with Glue. - @Test - public void testGetPartitionsFilterDate() - throws Exception - { - TupleDomain singleEquals = new PartitionFilterBuilder() - .addDateValues(PARTITION_KEY, 18000L) - .build(); - TupleDomain greaterThan = new PartitionFilterBuilder() - .addRanges(PARTITION_KEY, Range.greaterThan(DateType.DATE, 19000L)) - .build(); - TupleDomain betweenInclusive = new PartitionFilterBuilder() - .addRanges(PARTITION_KEY, Range.range(DateType.DATE, 19000L, true, 20000L, true)) - .build(); - TupleDomain greaterThanOrEquals = new PartitionFilterBuilder() - .addRanges(PARTITION_KEY, Range.greaterThanOrEqual(DateType.DATE, 19000L)) - .build(); - TupleDomain inClause = new PartitionFilterBuilder() - .addDateValues(PARTITION_KEY, 18000L, 21000L) - .build(); - TupleDomain lessThan = new PartitionFilterBuilder() - .addRanges(PARTITION_KEY, Range.lessThan(DateType.DATE, 20000L)) - .build(); - // we are unable to convert Date to a string format that Glue will accept, so it should translate to the wildcard in all cases. Commented out results are - // what we expect if we are able to do a proper conversion - doGetPartitionsFilterTest( - CREATE_TABLE_COLUMNS_PARTITIONED_DATE, - PARTITION_KEY, - ImmutableList.of("18000", "19000", "20000", "21000"), - ImmutableList.of( - singleEquals, greaterThan, betweenInclusive, greaterThanOrEquals, inClause, lessThan, TupleDomain.all()), - ImmutableList.of( -// ImmutableList.of("18000"), -// ImmutableList.of("20000", "21000"), -// ImmutableList.of("19000", "20000"), -// ImmutableList.of("19000", "20000", "21000"), -// ImmutableList.of("18000", "21000"), -// ImmutableList.of("18000", "19000"), - ImmutableList.of("18000", "19000", "20000", "21000"), - ImmutableList.of("18000", "19000", "20000", "21000"), - ImmutableList.of("18000", "19000", "20000", "21000"), - ImmutableList.of("18000", "19000", "20000", "21000"), - ImmutableList.of("18000", "19000", "20000", "21000"), - ImmutableList.of("18000", "19000", "20000", "21000"), - ImmutableList.of("18000", "19000", "20000", "21000"))); - } - - @Test - public void testGetPartitionsFilterTwoPartitionKeys() - throws Exception - { - TupleDomain equalsFilter = new PartitionFilterBuilder() - .addStringValues(PARTITION_KEY, "2020-03-01") - .addBigintValues(PARTITION_KEY2, 300L) - .build(); - TupleDomain rangeFilter = new PartitionFilterBuilder() - .addRanges(PARTITION_KEY, Range.greaterThanOrEqual(VarcharType.VARCHAR, utf8Slice("2020-02-01"))) - .addRanges(PARTITION_KEY2, Range.greaterThan(BigintType.BIGINT, 200L)) - .build(); - - doGetPartitionsFilterTest( - CREATE_TABLE_COLUMNS_PARTITIONED_TWO_KEYS, - ImmutableList.of(PARTITION_KEY, PARTITION_KEY2), - ImmutableList.of( - PartitionValues.make("2020-01-01", "100"), - PartitionValues.make("2020-02-01", "200"), - PartitionValues.make("2020-03-01", "300"), - PartitionValues.make("2020-04-01", "400")), - ImmutableList.of(equalsFilter, rangeFilter, TupleDomain.all()), - ImmutableList.of( - ImmutableList.of(PartitionValues.make("2020-03-01", "300")), - ImmutableList.of( - PartitionValues.make("2020-03-01", "300"), - PartitionValues.make("2020-04-01", "400")), - ImmutableList.of( - PartitionValues.make("2020-01-01", "100"), - PartitionValues.make("2020-02-01", "200"), - PartitionValues.make("2020-03-01", "300"), - PartitionValues.make("2020-04-01", "400")))); - } - - @Test - public void testGetPartitionsFilterMaxLengthWildcard() - throws Exception - { - // this filter string will exceed the 2048 char limit set by glue, and we expect the filter to revert to the wildcard - TupleDomain filter = new PartitionFilterBuilder() - .addStringValues(PARTITION_KEY, "x".repeat(2048)) - .build(); - - doGetPartitionsFilterTest( - CREATE_TABLE_COLUMNS_PARTITIONED_VARCHAR, - PARTITION_KEY, - VARCHAR_PARTITION_VALUES, - ImmutableList.of(filter), - ImmutableList.of( - ImmutableList.of("2020-01-01", "2020-02-01", "2020-03-01", "2020-04-01"))); - } - - @Test - public void testGetPartitionsFilterTwoPartitionKeysPartialQuery() - throws Exception - { - // we expect the second constraint to still be present and provide filtering - TupleDomain equalsFilter = new PartitionFilterBuilder() - .addStringValues(PARTITION_KEY, "x".repeat(2048)) - .addBigintValues(PARTITION_KEY2, 300L) - .build(); - - doGetPartitionsFilterTest( - CREATE_TABLE_COLUMNS_PARTITIONED_TWO_KEYS, - ImmutableList.of(PARTITION_KEY, PARTITION_KEY2), - ImmutableList.of( - PartitionValues.make("2020-01-01", "100"), - PartitionValues.make("2020-02-01", "200"), - PartitionValues.make("2020-03-01", "300"), - PartitionValues.make("2020-04-01", "400")), - ImmutableList.of(equalsFilter), - ImmutableList.of(ImmutableList.of(PartitionValues.make("2020-03-01", "300")))); - } - - @Test - public void testGetPartitionsFilterNone() - throws Exception - { - // test both a global none and that with a single column none, and a valid domain with none() - TupleDomain noneFilter = new PartitionFilterBuilder() - .addDomain(PARTITION_KEY, Domain.none(VarcharType.VARCHAR)) - .build(); - doGetPartitionsFilterTest( - CREATE_TABLE_COLUMNS_PARTITIONED_VARCHAR, - PARTITION_KEY, - VARCHAR_PARTITION_VALUES, - ImmutableList.of(TupleDomain.none(), noneFilter), - ImmutableList.of(ImmutableList.of(), ImmutableList.of())); - } - - @Test - public void testGetPartitionsFilterNotNull() - throws Exception - { - TupleDomain notNullFilter = new PartitionFilterBuilder() - .addDomain(PARTITION_KEY, Domain.notNull(VarcharType.VARCHAR)) - .build(); - doGetPartitionsFilterTest( - CREATE_TABLE_COLUMNS_PARTITIONED_VARCHAR, - PARTITION_KEY, - VARCHAR_PARTITION_VALUES, - ImmutableList.of(notNullFilter), - ImmutableList.of(ImmutableList.of("2020-01-01", "2020-02-01", "2020-03-01", "2020-04-01"))); - } - - @Test - public void testGetPartitionsFilterIsNull() - throws Exception - { - TupleDomain isNullFilter = new PartitionFilterBuilder() - .addDomain(PARTITION_KEY, Domain.onlyNull(VarcharType.VARCHAR)) - .build(); - doGetPartitionsFilterTest( - CREATE_TABLE_COLUMNS_PARTITIONED_VARCHAR, - PARTITION_KEY, - VARCHAR_PARTITION_VALUES, - ImmutableList.of(isNullFilter), - ImmutableList.of(ImmutableList.of())); - } - - @Test - public void testGetPartitionsFilterIsNullWithValue() - throws Exception - { - List partitionList = new ArrayList<>(); - partitionList.add("100"); - partitionList.add(null); - - doGetPartitionsFilterTest( - CREATE_TABLE_COLUMNS_PARTITIONED_VARCHAR, - PARTITION_KEY, - partitionList, - ImmutableList.of(new PartitionFilterBuilder() - // IS NULL - .addDomain(PARTITION_KEY, Domain.onlyNull(VarcharType.VARCHAR)) - .build()), - ImmutableList.of(ImmutableList.of(GlueExpressionUtil.NULL_STRING))); - - doGetPartitionsFilterTest( - CREATE_TABLE_COLUMNS_PARTITIONED_VARCHAR, - PARTITION_KEY, - partitionList, - ImmutableList.of(new PartitionFilterBuilder() - // IS NULL or is a specific value - .addDomain(PARTITION_KEY, Domain.create(ValueSet.of(VARCHAR, utf8Slice("100")), true)) - .build()), - ImmutableList.of(ImmutableList.of("100", GlueExpressionUtil.NULL_STRING))); - } - - @Test - public void testGetPartitionsFilterEqualsOrIsNullWithValue() - throws Exception - { - TupleDomain equalsOrIsNullFilter = new PartitionFilterBuilder() - .addStringValues(PARTITION_KEY, "2020-03-01") - .addDomain(PARTITION_KEY, Domain.onlyNull(VarcharType.VARCHAR)) - .build(); - List partitionList = new ArrayList<>(); - partitionList.add("2020-01-01"); - partitionList.add("2020-02-01"); - partitionList.add("2020-03-01"); - partitionList.add(null); - - doGetPartitionsFilterTest( - CREATE_TABLE_COLUMNS_PARTITIONED_VARCHAR, - PARTITION_KEY, - partitionList, - ImmutableList.of(equalsOrIsNullFilter), - ImmutableList.of(ImmutableList.of("2020-03-01", GlueExpressionUtil.NULL_STRING))); - } - - @Test - public void testGetPartitionsFilterIsNotNull() - throws Exception - { - TupleDomain isNotNullFilter = new PartitionFilterBuilder() - .addDomain(PARTITION_KEY, Domain.notNull(VarcharType.VARCHAR)) - .build(); - List partitionList = new ArrayList<>(); - partitionList.add("100"); - partitionList.add(null); - - doGetPartitionsFilterTest( - CREATE_TABLE_COLUMNS_PARTITIONED_VARCHAR, - PARTITION_KEY, - partitionList, - ImmutableList.of(isNotNullFilter), - ImmutableList.of(ImmutableList.of("100"))); - } - - @Test - public void testGetPartitionsFilterUnsupported() - throws Exception - { - // Numeric types are unsupported for IS (NOT) NULL predicate pushdown - testGetPartitionsFilterUnsupported(CREATE_TABLE_COLUMNS_PARTITIONED_TINYINT, Domain.onlyNull(TinyintType.TINYINT), "127"); - testGetPartitionsFilterUnsupported(CREATE_TABLE_COLUMNS_PARTITIONED_SMALLINT, Domain.onlyNull(SmallintType.SMALLINT), "32767"); - testGetPartitionsFilterUnsupported(CREATE_TABLE_COLUMNS_PARTITIONED_INTEGER, Domain.onlyNull(IntegerType.INTEGER), "2147483647"); - testGetPartitionsFilterUnsupported(CREATE_TABLE_COLUMNS_PARTITIONED_BIGINT, Domain.onlyNull(BigintType.BIGINT), "9223372036854775807"); - testGetPartitionsFilterUnsupported(CREATE_TABLE_COLUMNS_PARTITIONED_DECIMAL, Domain.onlyNull(DECIMAL_TYPE), "12345.12345"); - - testGetPartitionsFilterUnsupported(CREATE_TABLE_COLUMNS_PARTITIONED_TINYINT, Domain.notNull(TinyintType.TINYINT), "127"); - testGetPartitionsFilterUnsupported(CREATE_TABLE_COLUMNS_PARTITIONED_SMALLINT, Domain.notNull(SmallintType.SMALLINT), "32767"); - testGetPartitionsFilterUnsupported(CREATE_TABLE_COLUMNS_PARTITIONED_INTEGER, Domain.notNull(IntegerType.INTEGER), "2147483647"); - testGetPartitionsFilterUnsupported(CREATE_TABLE_COLUMNS_PARTITIONED_BIGINT, Domain.notNull(BigintType.BIGINT), "9223372036854775807"); - testGetPartitionsFilterUnsupported(CREATE_TABLE_COLUMNS_PARTITIONED_DECIMAL, Domain.notNull(DECIMAL_TYPE), "12345.12345"); - - // Date and timestamp aren't numeric types, but the pushdown is unsupported because of GlueExpressionUtil.canConvertSqlTypeToStringForGlue - testGetPartitionsFilterUnsupported(CREATE_TABLE_COLUMNS_PARTITIONED_DATE, Domain.onlyNull(DateType.DATE), "2022-07-11"); - testGetPartitionsFilterUnsupported(CREATE_TABLE_COLUMNS_PARTITIONED_TIMESTAMP, Domain.onlyNull(TimestampType.TIMESTAMP_MILLIS), "2022-07-11 01:02:03.123"); - - testGetPartitionsFilterUnsupported(CREATE_TABLE_COLUMNS_PARTITIONED_DATE, Domain.notNull(DateType.DATE), "2022-07-11"); - testGetPartitionsFilterUnsupported(CREATE_TABLE_COLUMNS_PARTITIONED_TIMESTAMP, Domain.notNull(TimestampType.TIMESTAMP_MILLIS), "2022-07-11 01:02:03.123"); - } - - @Test - @Override - public void testPartitionSchemaMismatch() - { - abort("tests using existing tables are not supported"); - } - - private void testGetPartitionsFilterUnsupported(List columnMetadata, Domain domain, String partitionValue) - throws Exception - { - TupleDomain isNullFilter = new PartitionFilterBuilder() - .addDomain(PARTITION_KEY, domain) - .build(); - List partitionList = new ArrayList<>(); - partitionList.add(partitionValue); - partitionList.add(null); - - doGetPartitionsFilterTest( - columnMetadata, - PARTITION_KEY, - partitionList, - ImmutableList.of(isNullFilter), - // Currently, we get NULL partition from Glue and filter it in our side because - // (column '__HIVE_DEFAULT_PARTITION__') on numeric types causes exception on Glue. e.g. 'input string: "__HIVE_D" is not an integer' - ImmutableList.of(ImmutableList.of(partitionValue, GlueExpressionUtil.NULL_STRING))); - } - - @Test - public void testGetPartitionsFilterEqualsAndIsNotNull() - throws Exception - { - TupleDomain equalsAndIsNotNullFilter = new PartitionFilterBuilder() - .addDomain(PARTITION_KEY, Domain.notNull(VarcharType.VARCHAR)) - .addBigintValues(PARTITION_KEY2, 300L) - .build(); - - doGetPartitionsFilterTest( - CREATE_TABLE_COLUMNS_PARTITIONED_TWO_KEYS, - ImmutableList.of(PARTITION_KEY, PARTITION_KEY2), - ImmutableList.of( - PartitionValues.make("2020-01-01", "100"), - PartitionValues.make("2020-02-01", "200"), - PartitionValues.make("2020-03-01", "300"), - PartitionValues.make(null, "300")), - ImmutableList.of(equalsAndIsNotNullFilter), - ImmutableList.of(ImmutableList.of(PartitionValues.make("2020-03-01", "300")))); - } - - @Test - public void testUpdateStatisticsOnCreate() - { - SchemaTableName tableName = temporaryTable("update_statistics_create"); - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - ConnectorMetadata metadata = transaction.getMetadata(); - - List columns = ImmutableList.of(new ColumnMetadata("a_column", BigintType.BIGINT)); - ConnectorTableMetadata tableMetadata = new ConnectorTableMetadata(tableName, columns, createTableProperties(TEXTFILE)); - ConnectorOutputTableHandle createTableHandle = metadata.beginCreateTable(session, tableMetadata, Optional.empty(), NO_RETRIES); - - // write data - ConnectorPageSink sink = pageSinkProvider.createPageSink(transaction.getTransactionHandle(), session, createTableHandle, TESTING_PAGE_SINK_ID); - MaterializedResult data = MaterializedResult.resultBuilder(session, BigintType.BIGINT) - .row(1L) - .row(2L) - .row(3L) - .row(4L) - .row(5L) - .build(); - sink.appendPage(data.toPage()); - Collection fragments = getFutureValue(sink.finish()); - - // prepare statistics - ComputedStatistics statistics = ComputedStatistics.builder(ImmutableList.of(), ImmutableList.of()) - .addTableStatistic(TableStatisticType.ROW_COUNT, singleValueBlock(5)) - .addColumnStatistic(MIN_VALUE.createColumnStatisticMetadata("a_column"), singleValueBlock(1)) - .addColumnStatistic(MAX_VALUE.createColumnStatisticMetadata("a_column"), singleValueBlock(1)) - .addColumnStatistic(NUMBER_OF_DISTINCT_VALUES.createColumnStatisticMetadata("a_column"), singleValueBlock(1)) - .addColumnStatistic(NUMBER_OF_NON_NULL_VALUES.createColumnStatisticMetadata("a_column"), singleValueBlock(1)) - .build(); - - // finish CTAS - metadata.finishCreateTable(session, createTableHandle, fragments, ImmutableList.of(statistics)); - transaction.commit(); - } - finally { - dropTable(tableName); - } - } - - @Test - public void testUpdatePartitionedStatisticsOnCreate() - { - SchemaTableName tableName = temporaryTable("update_partitioned_statistics_create"); - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - ConnectorMetadata metadata = transaction.getMetadata(); - - List columns = ImmutableList.of( - new ColumnMetadata("a_column", BigintType.BIGINT), - new ColumnMetadata("part_column", BigintType.BIGINT)); - - ConnectorTableMetadata tableMetadata = new ConnectorTableMetadata(tableName, columns, createTableProperties(TEXTFILE, ImmutableList.of("part_column"))); - ConnectorOutputTableHandle createTableHandle = metadata.beginCreateTable(session, tableMetadata, Optional.empty(), NO_RETRIES); - - // write data - ConnectorPageSink sink = pageSinkProvider.createPageSink(transaction.getTransactionHandle(), session, createTableHandle, TESTING_PAGE_SINK_ID); - MaterializedResult data = MaterializedResult.resultBuilder(session, BigintType.BIGINT, BigintType.BIGINT) - .row(1L, 1L) - .row(2L, 1L) - .row(3L, 1L) - .row(4L, 2L) - .row(5L, 2L) - .build(); - sink.appendPage(data.toPage()); - Collection fragments = getFutureValue(sink.finish()); - - // prepare statistics - ComputedStatistics statistics1 = ComputedStatistics.builder(ImmutableList.of("part_column"), ImmutableList.of(singleValueBlock(1))) - .addTableStatistic(TableStatisticType.ROW_COUNT, singleValueBlock(3)) - .addColumnStatistic(MIN_VALUE.createColumnStatisticMetadata("a_column"), singleValueBlock(1)) - .addColumnStatistic(MAX_VALUE.createColumnStatisticMetadata("a_column"), singleValueBlock(1)) - .addColumnStatistic(NUMBER_OF_DISTINCT_VALUES.createColumnStatisticMetadata("a_column"), singleValueBlock(1)) - .addColumnStatistic(NUMBER_OF_NON_NULL_VALUES.createColumnStatisticMetadata("a_column"), singleValueBlock(1)) - .build(); - ComputedStatistics statistics2 = ComputedStatistics.builder(ImmutableList.of("part_column"), ImmutableList.of(singleValueBlock(2))) - .addTableStatistic(TableStatisticType.ROW_COUNT, singleValueBlock(2)) - .addColumnStatistic(MIN_VALUE.createColumnStatisticMetadata("a_column"), singleValueBlock(4)) - .addColumnStatistic(MAX_VALUE.createColumnStatisticMetadata("a_column"), singleValueBlock(4)) - .addColumnStatistic(NUMBER_OF_DISTINCT_VALUES.createColumnStatisticMetadata("a_column"), singleValueBlock(4)) - .addColumnStatistic(NUMBER_OF_NON_NULL_VALUES.createColumnStatisticMetadata("a_column"), singleValueBlock(4)) - .build(); - - // finish CTAS - metadata.finishCreateTable(session, createTableHandle, fragments, ImmutableList.of(statistics1, statistics2)); - transaction.commit(); - } - finally { - dropTable(tableName); - } - } - - @Test - public void testStatisticsLargeNumberOfColumns() - throws Exception - { - SchemaTableName tableName = temporaryTable("test_statistics_large_number_of_columns"); - try { - ImmutableList.Builder columns = ImmutableList.builder(); - ImmutableMap.Builder columnStatistics = ImmutableMap.builder(); - for (int i = 1; i < 1500; ++i) { - String columnName = "t_bigint " + i + "_" + String.join("", Collections.nCopies(240, "x")); - columns.add(new ColumnMetadata(columnName, BIGINT)); - columnStatistics.put( - columnName, - createIntegerColumnStatistics( - OptionalLong.of(-1000 - i), - OptionalLong.of(1000 + i), - OptionalLong.of(i), - OptionalLong.of(2L * i))); - } - - PartitionStatistics partitionStatistics = PartitionStatistics.builder() - .setBasicStatistics(HIVE_BASIC_STATISTICS) - .setColumnStatistics(columnStatistics.buildOrThrow()).build(); - - doCreateEmptyTable(tableName, ORC, columns.build()); - testUpdateTableStatistics(tableName, ZERO_TABLE_STATISTICS, partitionStatistics); - } - finally { - dropTable(tableName); - } - } - - @Test - public void testStatisticsLongColumnNames() - throws Exception - { - SchemaTableName tableName = temporaryTable("test_statistics_long_column_name"); - try { - String columnName1 = String.join("", Collections.nCopies(255, "x")); - String columnName2 = String.join("", Collections.nCopies(255, "ӆ")); - String columnName3 = String.join("", Collections.nCopies(255, "ö")); - - List columns = List.of( - new ColumnMetadata(columnName1, BIGINT), - new ColumnMetadata(columnName2, BIGINT), - new ColumnMetadata(columnName3, BIGINT)); - - Map columnStatistics = Map.of( - columnName1, INTEGER_COLUMN_STATISTICS, - columnName2, INTEGER_COLUMN_STATISTICS, - columnName3, INTEGER_COLUMN_STATISTICS); - PartitionStatistics partitionStatistics = PartitionStatistics.builder() - .setBasicStatistics(HIVE_BASIC_STATISTICS) - .setColumnStatistics(columnStatistics).build(); - - doCreateEmptyTable(tableName, ORC, columns); - - assertThat(metastore.getTableStatistics(tableName.getSchemaName(), tableName.getTableName(), Optional.empty())) - .isEqualTo(ZERO_TABLE_STATISTICS); - testUpdateTableStatistics(tableName, ZERO_TABLE_STATISTICS, partitionStatistics); - } - finally { - dropTable(tableName); - } - } - - @Test - public void testStatisticsColumnModification() - throws Exception - { - SchemaTableName tableName = temporaryTable("test_statistics_column_modification"); - try { - List columns = List.of( - new ColumnMetadata("column1", BIGINT), - new ColumnMetadata("column2", BIGINT), - new ColumnMetadata("column3", BIGINT)); - - doCreateEmptyTable(tableName, ORC, columns); - - Map columnStatistics = Map.of( - "column1", INTEGER_COLUMN_STATISTICS, - "column2", INTEGER_COLUMN_STATISTICS); - PartitionStatistics partitionStatistics = PartitionStatistics.builder() - .setBasicStatistics(HIVE_BASIC_STATISTICS) - .setColumnStatistics(columnStatistics).build(); - - // set table statistics for column1 - metastore.updateTableStatistics( - tableName.getSchemaName(), - tableName.getTableName(), - NO_ACID_TRANSACTION, - actualStatistics -> { - assertThat(actualStatistics).isEqualTo(ZERO_TABLE_STATISTICS); - return partitionStatistics; - }); - - assertThat(metastore.getTableStatistics(tableName.getSchemaName(), tableName.getTableName(), Optional.empty())) - .isEqualTo(partitionStatistics); - - metastore.renameColumn(tableName.getSchemaName(), tableName.getTableName(), "column1", "column4"); - assertThat(metastore.getTableStatistics(tableName.getSchemaName(), tableName.getTableName(), Optional.empty())) - .isEqualTo(new PartitionStatistics( - HIVE_BASIC_STATISTICS, - Map.of("column2", INTEGER_COLUMN_STATISTICS))); - - metastore.dropColumn(tableName.getSchemaName(), tableName.getTableName(), "column2"); - assertThat(metastore.getTableStatistics(tableName.getSchemaName(), tableName.getTableName(), Optional.empty())) - .isEqualTo(new PartitionStatistics(HIVE_BASIC_STATISTICS, Map.of())); - - metastore.addColumn(tableName.getSchemaName(), tableName.getTableName(), "column5", HiveType.HIVE_INT, "comment"); - assertThat(metastore.getTableStatistics(tableName.getSchemaName(), tableName.getTableName(), Optional.empty())) - .isEqualTo(new PartitionStatistics(HIVE_BASIC_STATISTICS, Map.of())); - - // TODO: column1 stats should be removed on column delete. However this is tricky since stats can be stored in multiple partitions. - metastore.renameColumn(tableName.getSchemaName(), tableName.getTableName(), "column4", "column1"); - assertThat(metastore.getTableStatistics(tableName.getSchemaName(), tableName.getTableName(), Optional.empty())) - .isEqualTo(new PartitionStatistics( - HIVE_BASIC_STATISTICS, - Map.of("column1", INTEGER_COLUMN_STATISTICS))); - } - finally { - dropTable(tableName); - } - } - - @Test - public void testStatisticsPartitionedTableColumnModification() - throws Exception - { - SchemaTableName tableName = temporaryTable("test_partitioned_table_statistics_column_modification"); - try { - List columns = List.of( - new ColumnMetadata("column1", BIGINT), - new ColumnMetadata("column2", BIGINT), - new ColumnMetadata("ds", VARCHAR)); - - Map columnStatistics = Map.of( - "column1", INTEGER_COLUMN_STATISTICS, - "column2", INTEGER_COLUMN_STATISTICS); - PartitionStatistics partitionStatistics = PartitionStatistics.builder() - .setBasicStatistics(HIVE_BASIC_STATISTICS) - .setColumnStatistics(columnStatistics).build(); - - createDummyPartitionedTable(tableName, columns); - GlueHiveMetastore metastoreClient = (GlueHiveMetastore) getMetastoreClient(); - double countBefore = metastoreClient.getStats().getBatchUpdatePartition().getTime().getAllTime().getCount(); - - metastore.updatePartitionsStatistics(tableName.getSchemaName(), tableName.getTableName(), "ds=2016-01-01", actualStatistics -> partitionStatistics); - - assertThat(metastoreClient.getStats().getBatchUpdatePartition().getTime().getAllTime().getCount()).isEqualTo(countBefore + 1); - PartitionStatistics tableStatistics = new PartitionStatistics(createEmptyStatistics(), Map.of()); - assertThat(metastore.getTableStatistics(tableName.getSchemaName(), tableName.getTableName(), Optional.empty())) - .isEqualTo(tableStatistics); - assertThat(metastore.getPartitionStatistics(tableName.getSchemaName(), tableName.getTableName(), Set.of("ds=2016-01-01"))) - .isEqualTo(Map.of("ds=2016-01-01", partitionStatistics)); - - // renaming table column does not rename partition columns - metastore.renameColumn(tableName.getSchemaName(), tableName.getTableName(), "column1", "column4"); - assertThat(metastore.getTableStatistics(tableName.getSchemaName(), tableName.getTableName(), Optional.empty())) - .isEqualTo(tableStatistics); - assertThat(metastore.getPartitionStatistics(tableName.getSchemaName(), tableName.getTableName(), Set.of("ds=2016-01-01"))) - .isEqualTo(Map.of("ds=2016-01-01", partitionStatistics)); - - // dropping table column does not drop partition columns - metastore.dropColumn(tableName.getSchemaName(), tableName.getTableName(), "column2"); - assertThat(metastore.getTableStatistics(tableName.getSchemaName(), tableName.getTableName(), Optional.empty())) - .isEqualTo(tableStatistics); - assertThat(metastore.getPartitionStatistics(tableName.getSchemaName(), tableName.getTableName(), Set.of("ds=2016-01-01"))) - .isEqualTo(Map.of("ds=2016-01-01", partitionStatistics)); - } - finally { - dropTable(tableName); - } - } - - @Test - public void testInvalidColumnStatisticsMetadata() - throws Exception - { - SchemaTableName tableName = temporaryTable("test_statistics_invalid_column_metadata"); - try { - List columns = List.of( - new ColumnMetadata("column1", BIGINT)); - - Map columnStatistics = Map.of( - "column1", INTEGER_COLUMN_STATISTICS); - PartitionStatistics partitionStatistics = PartitionStatistics.builder() - .setBasicStatistics(HIVE_BASIC_STATISTICS) - .setColumnStatistics(columnStatistics).build(); - - doCreateEmptyTable(tableName, ORC, columns); - - // set table statistics for column1 - metastore.updateTableStatistics( - tableName.getSchemaName(), - tableName.getTableName(), - NO_ACID_TRANSACTION, - actualStatistics -> { - assertThat(actualStatistics).isEqualTo(ZERO_TABLE_STATISTICS); - return partitionStatistics; - }); - - Table table = metastore.getTable(tableName.getSchemaName(), tableName.getTableName()).get(); - TableInput tableInput = GlueInputConverter.convertTable(table); - tableInput.setParameters(ImmutableMap.builder() - .putAll(tableInput.getParameters()) - .put("column_stats_bad_data", "bad data") - .buildOrThrow()); - getGlueClient().updateTable(new UpdateTableRequest() - .withDatabaseName(tableName.getSchemaName()) - .withTableInput(tableInput)); - - assertThat(metastore.getTableStatistics(tableName.getSchemaName(), tableName.getTableName(), Optional.empty())) - .isEqualTo(partitionStatistics); - } - finally { - dropTable(tableName); - } - } - - @Test - @Override - public void testPartitionColumnProperties() - { - // Glue currently does not support parameters on the partitioning columns - assertThatThrownBy(super::testPartitionColumnProperties) - .isInstanceOf(TrinoException.class) - .hasMessageStartingWith("Parameters not supported for partition columns (Service: AWSGlue; Status Code: 400; Error Code: InvalidInputException;"); - } - - @Test - public void testGlueObjectsWithoutStorageDescriptor() - { - // StorageDescriptor is an Optional field for Glue tables. - SchemaTableName table = temporaryTable("test_missing_storage_descriptor"); - DeleteTableRequest deleteTableRequest = new DeleteTableRequest() - .withDatabaseName(table.getSchemaName()) - .withName(table.getTableName()); - - try { - Supplier resetTableInput = () -> new TableInput() - .withStorageDescriptor(null) - .withName(table.getTableName()) - .withTableType(EXTERNAL_TABLE.name()); - - TableInput tableInput = resetTableInput.get(); - glueClient.createTable(new CreateTableRequest() - .withDatabaseName(database) - .withTableInput(tableInput)); - - assertThatThrownBy(() -> metastore.getTable(table.getSchemaName(), table.getTableName())) - .hasMessageStartingWith("Table StorageDescriptor is null for table"); - glueClient.deleteTable(deleteTableRequest); - - // Iceberg table - tableInput = resetTableInput.get().withParameters(ImmutableMap.of(ICEBERG_TABLE_TYPE_NAME, ICEBERG_TABLE_TYPE_VALUE)); - glueClient.createTable(new CreateTableRequest() - .withDatabaseName(database) - .withTableInput(tableInput)); - assertThat(isIcebergTable(metastore.getTable(table.getSchemaName(), table.getTableName()).orElseThrow())).isTrue(); - glueClient.deleteTable(deleteTableRequest); - - // Delta Lake table - tableInput = resetTableInput.get().withParameters(ImmutableMap.of(SPARK_TABLE_PROVIDER_KEY, DELTA_LAKE_PROVIDER)); - glueClient.createTable(new CreateTableRequest() - .withDatabaseName(database) - .withTableInput(tableInput)); - assertThat(isDeltaLakeTable(metastore.getTable(table.getSchemaName(), table.getTableName()).orElseThrow())).isTrue(); - glueClient.deleteTable(deleteTableRequest); - - // Iceberg materialized view - tableInput = resetTableInput.get().withTableType(VIRTUAL_VIEW.name()) - .withViewOriginalText("/* Presto Materialized View: eyJvcmlnaW5hbFNxbCI6IlNFTEVDVCAxIiwiY29sdW1ucyI6W3sibmFtZSI6ImEiLCJ0eXBlIjoiaW50ZWdlciJ9XX0= */") - .withViewExpandedText(ICEBERG_MATERIALIZED_VIEW_COMMENT) - .withParameters(ImmutableMap.of( - PRESTO_VIEW_FLAG, "true", - TABLE_COMMENT, ICEBERG_MATERIALIZED_VIEW_COMMENT)); - glueClient.createTable(new CreateTableRequest() - .withDatabaseName(database) - .withTableInput(tableInput)); - assertThat(isTrinoMaterializedView(metastore.getTable(table.getSchemaName(), table.getTableName()).orElseThrow())).isTrue(); - materializedViews.add(table); - try (Transaction transaction = newTransaction()) { - ConnectorSession session = newSession(); - ConnectorMetadata metadata = transaction.getMetadata(); - // Not a view - assertThat(metadata.listViews(session, Optional.empty())) - .doesNotContain(table); - assertThat(metadata.listViews(session, Optional.of(table.getSchemaName()))) - .doesNotContain(table); - assertThat(metadata.getView(session, table)).isEmpty(); - } - finally { - materializedViews.remove(table); - } - } - finally { - // Table cannot be dropped through HiveMetastore since a TableHandle cannot be created - glueClient.deleteTable(new DeleteTableRequest() - .withDatabaseName(table.getSchemaName()) - .withName(table.getTableName())); - } - } - - @Test - public void testAlterTableComment() - throws Exception - { - SchemaTableName tableName = temporaryTable("test_alter_table_comment"); - doCreateEmptyTable(tableName, ORC, ImmutableList.of(new ColumnMetadata("name", BIGINT)), ImmutableList.of()); - try { - assertThat(metastore.getTable(tableName.getSchemaName(), tableName.getTableName()).orElseThrow().getParameters()).doesNotContainKey(TABLE_COMMENT); - metastore.commentTable(tableName.getSchemaName(), tableName.getTableName(), Optional.of("a table comment")); - Map tableParameters = metastore.getTable(tableName.getSchemaName(), tableName.getTableName()).orElseThrow().getParameters(); - assertThat(tableParameters).containsEntry(TABLE_COMMENT, "a table comment"); - - metastore.commentTable(tableName.getSchemaName(), tableName.getTableName(), Optional.empty()); - tableParameters = metastore.getTable(tableName.getSchemaName(), tableName.getTableName()).orElseThrow().getParameters(); - assertThat(tableParameters.get(TABLE_COMMENT)).isNull(); - } - finally { - glueClient.deleteTable(new DeleteTableRequest() - .withDatabaseName(tableName.getSchemaName()) - .withName(tableName.getTableName())); - } - } - - @Test - public void testAlterColumnComment() - throws Exception - { - SchemaTableName tableName = temporaryTable("test_alter_column_comment"); - List columns = ImmutableList.of( - new ColumnMetadata("first_column", BIGINT), - new ColumnMetadata("second_column", VARCHAR), - new ColumnMetadata("partition_column", BIGINT)); - createDummyPartitionedTable(tableName, columns, ImmutableList.of("partition_column"), ImmutableList.of()); - try { - metastore.commentColumn(tableName.getSchemaName(), tableName.getTableName(), "second_column", Optional.of("second column comment")); - metastore.commentColumn(tableName.getSchemaName(), tableName.getTableName(), "partition_column", Optional.of("partition column comment")); - - Table withComment = metastore.getTable(tableName.getSchemaName(), tableName.getTableName()).orElseThrow(); - assertThat(withComment.getColumn("first_column").orElseThrow().getComment()).isEmpty(); - assertThat(withComment.getColumn("second_column").orElseThrow().getComment()).isEqualTo(Optional.of("second column comment")); - assertThat(withComment.getColumn("partition_column").orElseThrow().getComment()).isEqualTo(Optional.of("partition column comment")); - - metastore.commentColumn(tableName.getSchemaName(), tableName.getTableName(), "second_column", Optional.empty()); - withComment = metastore.getTable(tableName.getSchemaName(), tableName.getTableName()).orElseThrow(); - assertThat(withComment.getColumn("first_column").orElseThrow().getComment()).isEmpty(); - assertThat(withComment.getColumn("second_column").orElseThrow().getComment()).isEmpty(); - assertThat(withComment.getColumn("partition_column").orElseThrow().getComment()).isEqualTo(Optional.of("partition column comment")); - } - finally { - glueClient.deleteTable(new DeleteTableRequest() - .withDatabaseName(tableName.getSchemaName()) - .withName(tableName.getTableName())); - } - } - - private Block singleValueBlock(long value) - { - BlockBuilder blockBuilder = BIGINT.createBlockBuilder(null, 1); - BIGINT.writeLong(blockBuilder, value); - return blockBuilder.build(); - } - - private void doGetPartitionsFilterTest( - List columnMetadata, - String partitionColumnName, - List partitionStringValues, - List> filterList, - List> expectedSingleValueList) - throws Exception - { - List partitionValuesList = partitionStringValues.stream() - .map(PartitionValues::make) - .collect(toImmutableList()); - List> expectedPartitionValuesList = expectedSingleValueList.stream() - .map(expectedValue -> expectedValue.stream() - .map(PartitionValues::make) - .collect(toImmutableList())) - .collect(toImmutableList()); - doGetPartitionsFilterTest(columnMetadata, ImmutableList.of(partitionColumnName), partitionValuesList, filterList, expectedPartitionValuesList); - } - - /** - * @param filterList should be same sized list as expectedValuesList - */ - private void doGetPartitionsFilterTest( - List columnMetadata, - List partitionColumnNames, - List partitionValues, - List> filterList, - List> expectedValuesList) - throws Exception - { - try (CloseableSchamaTableName closeableTableName = new CloseableSchamaTableName(temporaryTable("get_partitions"))) { - SchemaTableName tableName = closeableTableName.getSchemaTableName(); - createDummyPartitionedTable(tableName, columnMetadata, partitionColumnNames, partitionValues); - HiveMetastore metastoreClient = getMetastoreClient(); - - for (int i = 0; i < filterList.size(); i++) { - TupleDomain filter = filterList.get(i); - List expectedValues = expectedValuesList.get(i); - List expectedResults = expectedValues.stream() - .map(expectedPartitionValues -> makePartName(partitionColumnNames, expectedPartitionValues.getValues())) - .collect(toImmutableList()); - - Optional> partitionNames = metastoreClient.getPartitionNamesByFilter( - tableName.getSchemaName(), - tableName.getTableName(), - partitionColumnNames, - filter); - assertThat(partitionNames.isPresent()).isTrue(); - assertThat(partitionNames.get()) - .describedAs(format("lists \nactual: %s\nexpected: %s\nmismatch for filter %s (input index %d)\n", partitionNames.get(), expectedResults, filter, i)) - .isEqualTo(expectedResults); - } - } - } - - private void createDummyPartitionedTable(SchemaTableName tableName, List columns, List partitionColumnNames, List partitionValues) - throws Exception - { - doCreateEmptyTable(tableName, ORC, columns, partitionColumnNames); - - HiveMetastoreClosure metastoreClient = new HiveMetastoreClosure(getMetastoreClient(), TESTING_TYPE_MANAGER, false); - Table table = metastoreClient.getTable(tableName.getSchemaName(), tableName.getTableName()) - .orElseThrow(() -> new TableNotFoundException(tableName)); - List partitions = new ArrayList<>(); - List partitionNames = new ArrayList<>(); - partitionValues.stream() - .map(partitionValue -> makePartName(partitionColumnNames, partitionValue.values)) - .forEach( - partitionName -> { - partitions.add(new PartitionWithStatistics(createDummyPartition(table, partitionName), partitionName, PartitionStatistics.empty())); - partitionNames.add(partitionName); - }); - metastoreClient.addPartitions(tableName.getSchemaName(), tableName.getTableName(), partitions); - partitionNames.forEach( - partitionName -> metastoreClient.updatePartitionsStatistics( - tableName.getSchemaName(), tableName.getTableName(), partitionName, currentStatistics -> ZERO_TABLE_STATISTICS)); - } - - private class CloseableSchamaTableName - implements AutoCloseable - { - private final SchemaTableName schemaTableName; - - private CloseableSchamaTableName(SchemaTableName schemaTableName) - { - this.schemaTableName = schemaTableName; - } - - public SchemaTableName getSchemaTableName() - { - return schemaTableName; - } - - @Override - public void close() - { - dropTable(schemaTableName); - } - } - - // container class for readability. Each value is one for a partitionKey, in order they appear in the schema - private static class PartitionValues - { - private final List values; - - private static PartitionValues make(String... values) - { - return new PartitionValues(Arrays.asList(values)); - } - - private PartitionValues(List values) - { - // Elements are nullable - //noinspection Java9CollectionFactory - this.values = unmodifiableList(new ArrayList<>(requireNonNull(values, "values is null"))); - } - - public List getValues() - { - return values; - } - } -} diff --git a/plugin/trino-hive/src/test/resources/spark_bucketed_nation/._SUCCESS.crc b/plugin/trino-hive/src/test/resources/spark_bucketed_nation/._SUCCESS.crc deleted file mode 100644 index 3b7b044936a890cd8d651d349a752d819d71d22c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8 PcmYc;N@ieSU}69O2$TUk diff --git a/plugin/trino-hive/src/test/resources/spark_bucketed_nation/.part-00000-0a72da44-98a5-4b85-a513-2b2e82ec274f_00000.c000.snappy.orc.crc b/plugin/trino-hive/src/test/resources/spark_bucketed_nation/.part-00000-0a72da44-98a5-4b85-a513-2b2e82ec274f_00000.c000.snappy.orc.crc deleted file mode 100644 index df1434b32cf94902c2713136841b24337975248c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 24 fcmYc;N@ieSU}9i7$=K0-_}0Q-eqZzI%j-A*VvY%c diff --git a/plugin/trino-hive/src/test/resources/spark_bucketed_nation/.part-00000-0a72da44-98a5-4b85-a513-2b2e82ec274f_00001.c000.snappy.orc.crc b/plugin/trino-hive/src/test/resources/spark_bucketed_nation/.part-00000-0a72da44-98a5-4b85-a513-2b2e82ec274f_00001.c000.snappy.orc.crc deleted file mode 100644 index 5eda6fd9ac791375fbc3ae4c025b82ffe1936774..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 20 bcmYc;N@ieSU}D(gcVp83g7%Q6vt=QUT3{{6R)w+IwUX+DWud{IcXxbhLmhmw=pEP>)6zx;$yuYJKL;x z-2F)73Q)v>s3%&1I3ZOC4jd{0<$$O-07a-$PkbD>AOSbThjQovf%te$9Vk6ih)5h} zc4pq_?c3k`zqf_bjEs;NiUozfA{vEdTnr$DVv^7o>k&{itz71cXdVp1A`b>I-6`}9 zCBz4gAVl8s%%bFkZyL*1MIiM$2ml zwVBymUdtX{ES9t7*<7KR(*V%CmS4=}59TzO3i<2;EJ|Abk!*fZD`rcTQe^>-D*0SF zdyp)YwQ_c$i!*`Hq23i*^a8sTbpt=iT*iGr)fmD}s zVopvg4sH6D?j#hxKGEtI|)JI?{GPK>!bQC8fu*itct=ZIAIk z%BL$d_k4IPLC7+kG_-MtA5Qr@K;M7xy?))m*Q2TL9 z+;v4E4n5&hSKAkXGP3bA`paX#@OsdudI+bHL(d02mR&f~7<&=3v%!5|4{X)eq9g$n zIV_(|`tj@aGj<7g%=?b~fsgNaXvfDR1ZvDQKuT6;^2y|QiUQD7x0E+)XwNb|X55|8 zobktx7i{%1LiT_d5<*d=+{Oz6R+2Dt$}V1*3n+@%1D2q25yKT6#Y&tPMmVJoJ1)v8 z5+6+k6bUY2xO9Q%lhvRSk8uKm!QD8nV)Q%SSQmfA;%18z`~K0`s&R4f7VWHhLhQY< zja4P_!vEE{v~4!7>L>gw9QuOWcvuKTzX|t8;Bk&qz6_D(NZ-(?5Qzmb7YO!;ZQV1O z4La-(+IoxjhoO4A2R#OBwIH!WoDhJ~Gb9Nc>*>DCCd$msC^NT4nYjsNW_u_*(!c)+ k9NIq1=`;CRBz$^JeQRscAr3`Ge#7a|b5{d_LTM)XC+n&b-v9sr diff --git a/plugin/trino-hive/src/test/resources/spark_bucketed_nation/part-00000-0a72da44-98a5-4b85-a513-2b2e82ec274f_00001.c000.snappy.orc b/plugin/trino-hive/src/test/resources/spark_bucketed_nation/part-00000-0a72da44-98a5-4b85-a513-2b2e82ec274f_00001.c000.snappy.orc deleted file mode 100644 index eb4387fc03ac4a665aaf04bc9567675e021c1539..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1480 zcmchW&rcIU6vt<#yJfl*T`D947&8_^h#)o^l>i4Z`@YG{P$K|F zG@)UA3UsU214aY@kceYF5yi;tBJ)$=s`4K(bmdn&hq0b)Gh$Ze+(>$QJZ&Z7m#r(q z8Pj?bwtDxyM} zZA&lbxCI(W8q9L)@j_WJPjAVU-N5gqGg9P~^Jba*flrx`ZbHV|$NWmM6v#luV;-k$ zMwSD8_*uT}IkN22Vp#-ENjTh75%@yMu@~jIyMuc z(omlY6HUE4EA?k_9 zF&$4t2r=r^jz-=>bp)D_#B|(+NVobzCL&ljdm;p@Lr@)hiu9JOMvMf)KpEO@0|*$8 zkP%LXv2Xi4WMfAAAEzLDV)Ig{{wy|D_bB)c;q4&#+DuTk)wE63hx~S>``Wb3^YGr5 z^w$(*Z)6e5Kf8=RAn+Ac&*O;sINBOl%Q7NA5G{1+$)h-)&>)IvtudDcj&$dErB!oT zk+;Uw5U)GxY*{QSK_n55D6300;ObyX?+=Oh|3bX~Z-_UatS+p>?*H$w;c)fZKZAW{ eDLn@8mzshX7tJIB@vbK*6}|H<5}C{lwfqEV3Zwb} diff --git a/plugin/trino-hive/src/test/resources/spark_bucketed_nation/part-00000-0a72da44-98a5-4b85-a513-2b2e82ec274f_00002.c000.snappy.orc b/plugin/trino-hive/src/test/resources/spark_bucketed_nation/part-00000-0a72da44-98a5-4b85-a513-2b2e82ec274f_00002.c000.snappy.orc deleted file mode 100644 index c2aa001763ea72ca2822c2ef6a14e5f954ec3bf5..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1200 zcmbtSO-~a+7@nQoEz_ZuvH}?zHIu55K-*yA!88&m1*K>|sA3x8!DX4&wcFigKLYhc zl!Ni49z7V(dhti7G0~%m9=&+uz`-Bj3y6Up)WBi(W8P<;_kEsuDz!pCLNuZ&M6IF> zd=5|IN;P&z}@6r)P9mS50LES8J6W@q^IV)@!krNpXL zGzQp{dn7R+i4c?^OsgU+p-0jH-H?DslD4mCYrwNgCxmYMeA#jQfE&6e8lh!%xbF(X z)Gh9b)lm3>pXIt?IH4W*957klKl)kAz^>^zwrQk!Xor3Xc-`-`-M|Ti&)4*p;LDa7 z+!H|Pu`fy$#S!|T4B?CJ0zy0U&*!W8a;Z|Q6bhA6Whq~pE-oz29@eybM!{#NdJ(#+ zY@)jqqu2N&wB;+pb$ks1Mn*f2aXORMCD%oy!~io!M}fJh06MODHv8-qWb!0|*{W&|J-KptatxUMj& zVS-@cgX@`u=zEFo29@?5if27a`QJlXzeG Date: Thu, 7 Dec 2023 10:53:05 +0800 Subject: [PATCH 088/350] Move Rubix initialization tests to HDFS module --- .../java/io/trino/hdfs/TestCachingSetup.java | 140 ++++++++++++++++ plugin/trino-hive-hadoop2/pom.xml | 6 - .../io/trino/plugin/hive/TestHivePlugin.java | 153 ------------------ 3 files changed, 140 insertions(+), 159 deletions(-) create mode 100644 lib/trino-hdfs/src/test/java/io/trino/hdfs/TestCachingSetup.java diff --git a/lib/trino-hdfs/src/test/java/io/trino/hdfs/TestCachingSetup.java b/lib/trino-hdfs/src/test/java/io/trino/hdfs/TestCachingSetup.java new file mode 100644 index 0000000000000..2b1492130cc88 --- /dev/null +++ b/lib/trino-hdfs/src/test/java/io/trino/hdfs/TestCachingSetup.java @@ -0,0 +1,140 @@ +/* + * 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.hdfs; + +import com.google.common.collect.ImmutableMap; +import com.qubole.rubix.core.CachingFileSystem; +import io.airlift.testing.TempFile; +import io.opentelemetry.api.OpenTelemetry; +import io.trino.filesystem.hdfs.HdfsFileSystemManager; +import io.trino.testing.TestingNodeManager; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.api.parallel.Execution; + +import java.io.IOException; +import java.nio.file.Path; +import java.util.Map; + +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; + +@TestInstance(PER_CLASS) +@Execution(SAME_THREAD) +class TestCachingSetup +{ + @BeforeEach + @AfterEach + public void deinitializeRubix() + { + // revert static Rubix initialization done by other tests + CachingFileSystem.deinitialize(); + } + + @Test + public void testS3SecurityMappingAndHiveCachingMutuallyExclusive(@TempDir Path tempDirectory) + throws IOException + { + try (TempFile mappingConfig = new TempFile()) { + assertThatThrownBy(() -> createFileSystemManager( + ImmutableMap.builder() + .put("hive.s3.security-mapping.config-file", mappingConfig.path().toString()) + .put("hive.cache.enabled", "true") + .put("hive.cache.location", tempDirectory.toString()) + .buildOrThrow())) + .hasMessageContaining("S3 security mapping is not compatible with Hive caching"); + } + } + + @Test + public void testGcsAccessTokenAndHiveCachingMutuallyExclusive(@TempDir Path tempDirectory) + { + assertThatThrownBy(() -> createFileSystemManager( + ImmutableMap.builder() + .put("hive.gcs.use-access-token", "true") + .put("hive.cache.enabled", "true") + .put("hive.cache.location", tempDirectory.toString()) + .buildOrThrow())) + .hasMessageContaining("Use of GCS access token is not compatible with Hive caching"); + } + + @Test + public void testHdfsImpersonationAndHiveCachingMutuallyExclusive(@TempDir Path tempDirectory) + { + assertThatThrownBy(() -> createFileSystemManager( + ImmutableMap.builder() + .put("hive.hdfs.impersonation.enabled", "true") + .put("hive.cache.enabled", "true") + .put("hive.cache.location", tempDirectory.toString()) + .buildOrThrow())) + .hasMessageContaining("HDFS impersonation is not compatible with Hive caching"); + } + + @Test + public void testRubixCache(@TempDir Path tempDirectory) + { + createFileSystemManager( + ImmutableMap.builder() + .put("hive.cache.enabled", "true") + .put("hive.cache.location", tempDirectory.toString()) + .buildOrThrow()); + } + + @Test + public void testRubixCacheWithNonExistingCacheDirectory() + { + assertThatThrownBy(() -> createFileSystemManager( + ImmutableMap.builder() + .put("hive.cache.enabled", "true") + .put("hive.cache.start-server-on-coordinator", "true") + .put("hive.cache.location", "/tmp/non/existing/directory") + .buildOrThrow())) + .hasMessageContaining("None of the cache parent directories exists"); + + assertThatThrownBy(() -> createFileSystemManager( + ImmutableMap.builder() + .put("hive.cache.enabled", "true") + .put("hive.cache.start-server-on-coordinator", "true") + .buildOrThrow())) + .hasMessageContaining("caching directories were not provided"); + + // cache directories should not be required when cache is not explicitly started on coordinator + createFileSystemManager( + ImmutableMap.builder() + .put("hive.cache.enabled", "true") + .buildOrThrow()); + } + + private static void createFileSystemManager(Map config) + { + HdfsFileSystemManager manager = new HdfsFileSystemManager( + ImmutableMap.builder() + .putAll(config) + .put("boostrap.quiet", "true") + .buildOrThrow(), + true, + true, + true, + "test", + new TestingNodeManager(), + OpenTelemetry.noop()); + manager.configure(); + manager.create(); + manager.stop(); + } +} diff --git a/plugin/trino-hive-hadoop2/pom.xml b/plugin/trino-hive-hadoop2/pom.xml index 54baeb2832c83..c9e0f36e549c4 100644 --- a/plugin/trino-hive-hadoop2/pom.xml +++ b/plugin/trino-hive-hadoop2/pom.xml @@ -76,12 +76,6 @@ runtime - - com.qubole.rubix - rubix-presto-shaded - runtime - - io.airlift concurrent diff --git a/plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/TestHivePlugin.java b/plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/TestHivePlugin.java index b3bb9c96a669a..4aa14051dd5d1 100644 --- a/plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/TestHivePlugin.java +++ b/plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/TestHivePlugin.java @@ -14,66 +14,26 @@ package io.trino.plugin.hive; import com.google.common.collect.ImmutableMap; -import com.qubole.rubix.core.CachingFileSystem; import io.trino.spi.Plugin; import io.trino.spi.connector.Connector; import io.trino.spi.connector.ConnectorFactory; import io.trino.testing.TestingConnectorContext; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; -import org.junit.jupiter.api.parallel.Execution; import java.io.File; -import java.io.IOException; import java.nio.file.Files; -import java.nio.file.Path; import static com.google.common.collect.MoreCollectors.onlyElement; import static com.google.common.collect.MoreCollectors.toOptional; import static com.google.common.collect.Streams.stream; -import static com.google.common.io.MoreFiles.deleteRecursively; -import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; import static io.trino.plugin.hive.HiveSessionProperties.InsertExistingPartitionsBehavior.APPEND; import static io.trino.plugin.hive.HiveSessionProperties.InsertExistingPartitionsBehavior.ERROR; import static java.nio.charset.StandardCharsets.UTF_8; -import static java.nio.file.Files.createTempDirectory; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; -@TestInstance(PER_CLASS) -@Execution(SAME_THREAD) // see @BeforeEach public class TestHivePlugin { - private Path tempDirectory; - - @BeforeAll - public void setup() - throws IOException - { - tempDirectory = createTempDirectory(getClass().getSimpleName()); - } - - @AfterAll - public void tearDown() - throws IOException - { - deleteRecursively(tempDirectory, ALLOW_INSECURE); - } - - @AfterEach - @BeforeEach - public void deinitializeRubix() - { - // revert static rubix initialization done by other tests - CachingFileSystem.deinitialize(); - } - @Test public void testCreateConnector() { @@ -141,44 +101,6 @@ public void testGlueMetastore() .hasMessageContaining("Error: Configuration property 'hive.metastore.uri' was not used"); } - @Test - public void testS3SecurityMappingAndHiveCachingMutuallyExclusive() - throws IOException - { - Path mappingConfig = Files.createTempFile(null, null); - ConnectorFactory connectorFactory = getHiveConnectorFactory(); - - assertThatThrownBy(() -> connectorFactory.create( - "test", - ImmutableMap.builder() - .put("hive.s3.security-mapping.config-file", mappingConfig.toString()) - .put("hive.cache.enabled", "true") - .put("hive.metastore.uri", "thrift://foo:1234") - .put("hive.cache.location", tempDirectory.toString()) - .put("bootstrap.quiet", "true") - .buildOrThrow(), - new TestingConnectorContext())) - .hasMessageContaining("S3 security mapping is not compatible with Hive caching"); - } - - @Test - public void testGcsAccessTokenAndHiveCachingMutuallyExclusive() - { - ConnectorFactory connectorFactory = getHiveConnectorFactory(); - - assertThatThrownBy(() -> connectorFactory.create( - "test", - ImmutableMap.builder() - .put("hive.gcs.use-access-token", "true") - .put("hive.cache.enabled", "true") - .put("hive.metastore.uri", "thrift://foo:1234") - .put("hive.cache.location", tempDirectory.toString()) - .put("bootstrap.quiet", "true") - .buildOrThrow(), - new TestingConnectorContext())) - .hasMessageContaining("Use of GCS access token is not compatible with Hive caching"); - } - @Test public void testImmutablePartitionsAndInsertOverwriteMutuallyExclusive() { @@ -236,81 +158,6 @@ private Object getDefaultValueInsertExistingPartitionsBehavior(Connector connect .getDefaultValue(); } - @Test - public void testHdfsImpersonationAndHiveCachingMutuallyExclusive() - { - ConnectorFactory connectorFactory = getHiveConnectorFactory(); - - assertThatThrownBy(() -> connectorFactory.create( - "test", - ImmutableMap.builder() - .put("hive.hdfs.impersonation.enabled", "true") - .put("hive.cache.enabled", "true") - .put("hive.metastore.uri", "thrift://foo:1234") - .put("hive.cache.location", tempDirectory.toString()) - .put("bootstrap.quiet", "true") - .buildOrThrow(), - new TestingConnectorContext())) - .hasMessageContaining("HDFS impersonation is not compatible with Hive caching"); - } - - @Test - public void testRubixCache() - { - ConnectorFactory connectorFactory = getHiveConnectorFactory(); - - connectorFactory.create( - "test", - ImmutableMap.builder() - .put("hive.cache.enabled", "true") - .put("hive.metastore.uri", "thrift://foo:1234") - .put("hive.cache.location", tempDirectory.toString()) - .put("bootstrap.quiet", "true") - .buildOrThrow(), - new TestingConnectorContext()) - .shutdown(); - } - - @Test - public void testRubixCacheWithNonExistingCacheDirectory() - { - ConnectorFactory connectorFactory = getHiveConnectorFactory(); - - assertThatThrownBy(() -> connectorFactory.create( - "test", - ImmutableMap.builder() - .put("hive.cache.enabled", "true") - .put("hive.cache.start-server-on-coordinator", "true") - .put("hive.metastore.uri", "thrift://foo:1234") - .put("hive.cache.location", "/tmp/non/existing/directory") - .put("bootstrap.quiet", "true") - .buildOrThrow(), - new TestingConnectorContext())) - .hasMessageContaining("None of the cache parent directories exists"); - - assertThatThrownBy(() -> connectorFactory.create( - "test", - ImmutableMap.builder() - .put("hive.cache.enabled", "true") - .put("hive.cache.start-server-on-coordinator", "true") - .put("hive.metastore.uri", "thrift://foo:1234") - .put("bootstrap.quiet", "true") - .buildOrThrow(), - new TestingConnectorContext())) - .hasMessageContaining("caching directories were not provided"); - - // cache directories should not be required when cache is not explicitly started on coordinator - connectorFactory.create( - "test", - ImmutableMap.builder() - .put("hive.cache.enabled", "true") - .put("hive.metastore.uri", "thrift://foo:1234") - .put("bootstrap.quiet", "true") - .buildOrThrow(), - new TestingConnectorContext()) - .shutdown(); - } - @Test public void testAllowAllAccessControl() { From 15cc6e58656d4c77b7f8460024d42c9f02e9a8d3 Mon Sep 17 00:00:00 2001 From: David Phillips Date: Wed, 6 Dec 2023 22:51:12 +0800 Subject: [PATCH 089/350] Move Hive plugin to Hive module --- .github/config/labeler-config.yml | 2 - .github/workflows/ci.yml | 2 +- client/trino-jdbc/pom.xml | 2 +- core/trino-server/src/main/provisio/trino.xml | 2 +- plugin/trino-hive-hadoop2/pom.xml | 213 ------------------ plugin/trino-hive/pom.xml | 31 +-- .../hive/CanonicalizeHiveTimezoneId.java | 0 .../java/io/trino/plugin/hive/HivePlugin.java | 0 .../io/trino/plugin/hive/TestHivePlugin.java | 0 pom.xml | 7 - .../trino-server-dev/etc/config.properties | 2 +- 11 files changed, 21 insertions(+), 240 deletions(-) delete mode 100644 plugin/trino-hive-hadoop2/pom.xml rename plugin/{trino-hive-hadoop2 => trino-hive}/src/main/java/io/trino/plugin/hive/CanonicalizeHiveTimezoneId.java (100%) rename plugin/{trino-hive-hadoop2 => trino-hive}/src/main/java/io/trino/plugin/hive/HivePlugin.java (100%) rename plugin/{trino-hive-hadoop2 => trino-hive}/src/test/java/io/trino/plugin/hive/TestHivePlugin.java (100%) diff --git a/.github/config/labeler-config.yml b/.github/config/labeler-config.yml index ae685830c4a43..e7ff2840486e0 100644 --- a/.github/config/labeler-config.yml +++ b/.github/config/labeler-config.yml @@ -3,7 +3,6 @@ - lib/trino-orc/** - lib/trino-parquet/** - lib/trino-hive-formats/** - - plugin/trino-hive-hadoop2/** - plugin/trino-hive/** - testing/trino-product-tests/** - lib/trino-filesystem/** @@ -20,7 +19,6 @@ delta-lake: hive: - plugin/trino-hive/** - - plugin/trino-hive-hadoop2/** hudi: - plugin/trino-hudi/** diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 42c188ebb8c73..258ae6d0630f8 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -301,7 +301,7 @@ jobs: - name: Install Hive Module run: | export MAVEN_OPTS="${MAVEN_INSTALL_OPTS}" - $MAVEN clean install ${MAVEN_FAST_INSTALL} ${MAVEN_GIB} -Dgib.logImpactedTo=gib-impacted.log -am -pl :trino-hive-hadoop2 + $MAVEN clean install ${MAVEN_FAST_INSTALL} ${MAVEN_GIB} -Dgib.logImpactedTo=gib-impacted.log -am -pl :trino-hive - name: Run Hive AWS Tests env: AWS_ACCESS_KEY_ID: ${{ secrets.TRINO_AWS_ACCESS_KEY_ID }} diff --git a/client/trino-jdbc/pom.xml b/client/trino-jdbc/pom.xml index 9312577b7979f..b3115a4b63c8a 100644 --- a/client/trino-jdbc/pom.xml +++ b/client/trino-jdbc/pom.xml @@ -193,7 +193,7 @@ io.trino - trino-hive-hadoop2 + trino-hive test diff --git a/core/trino-server/src/main/provisio/trino.xml b/core/trino-server/src/main/provisio/trino.xml index 519af7a4e5304..a11e4ade027e5 100644 --- a/core/trino-server/src/main/provisio/trino.xml +++ b/core/trino-server/src/main/provisio/trino.xml @@ -114,7 +114,7 @@ - + diff --git a/plugin/trino-hive-hadoop2/pom.xml b/plugin/trino-hive-hadoop2/pom.xml deleted file mode 100644 index c9e0f36e549c4..0000000000000 --- a/plugin/trino-hive-hadoop2/pom.xml +++ /dev/null @@ -1,213 +0,0 @@ - - - 4.0.0 - - - io.trino - trino-root - 435-SNAPSHOT - ../../pom.xml - - - trino-hive-hadoop2 - trino-plugin - Trino - Hive Connector - Apache Hadoop 2.x - - - ${project.parent.basedir} - - - - - com.google.guava - guava - - - - io.trino - trino-hive - - - - com.fasterxml.jackson.core - jackson-annotations - provided - - - - io.airlift - slice - provided - - - - io.opentelemetry - opentelemetry-api - provided - - - - io.opentelemetry - opentelemetry-context - provided - - - - io.trino - trino-spi - provided - - - - org.openjdk.jol - jol-core - provided - - - - com.amazonaws - aws-java-sdk-core - runtime - - - - com.amazonaws - aws-java-sdk-s3 - runtime - - - - io.airlift - concurrent - runtime - - - - io.airlift - json - runtime - - - - io.airlift - stats - runtime - - - - io.airlift - units - runtime - - - - io.trino - trino-filesystem - runtime - - - - io.trino - trino-plugin-toolkit - runtime - - - - org.alluxio - alluxio-shaded-client - runtime - - - - io.airlift - junit-extensions - test - - - - io.airlift - testing - test - - - - io.trino - trino-hdfs - test - - - - io.trino - trino-hive - test-jar - test - - - - io.trino - trino-main - test - - - - io.trino - trino-spi - test-jar - test - - - - io.trino - trino-testing - test - - - - io.trino - trino-testing-containers - test - - - - io.trino - trino-testing-services - test - - - - io.trino.hadoop - hadoop-apache - test - - - - io.trino.hive - hive-apache - test - - - - org.assertj - assertj-core - test - - - - org.junit.jupiter - junit-jupiter-api - test - - - - org.junit.jupiter - junit-jupiter-engine - test - - - - org.testng - testng - test - - - diff --git a/plugin/trino-hive/pom.xml b/plugin/trino-hive/pom.xml index 76f7d9f3fbdd3..653808044db12 100644 --- a/plugin/trino-hive/pom.xml +++ b/plugin/trino-hive/pom.xml @@ -10,6 +10,7 @@ trino-hive + trino-plugin Trino - Hive Connector @@ -121,16 +122,6 @@ units - - io.opentelemetry - opentelemetry-api - - - - io.opentelemetry - opentelemetry-context - - io.opentelemetry.instrumentation opentelemetry-aws-sdk-1.11 @@ -259,14 +250,20 @@ - io.trino - trino-spi + io.opentelemetry + opentelemetry-api provided - org.jetbrains - annotations + io.opentelemetry + opentelemetry-context + provided + + + + io.trino + trino-spi provided @@ -318,6 +315,12 @@ runtime + + org.jetbrains + annotations + runtime + + io.airlift junit-extensions diff --git a/plugin/trino-hive-hadoop2/src/main/java/io/trino/plugin/hive/CanonicalizeHiveTimezoneId.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/CanonicalizeHiveTimezoneId.java similarity index 100% rename from plugin/trino-hive-hadoop2/src/main/java/io/trino/plugin/hive/CanonicalizeHiveTimezoneId.java rename to plugin/trino-hive/src/main/java/io/trino/plugin/hive/CanonicalizeHiveTimezoneId.java diff --git a/plugin/trino-hive-hadoop2/src/main/java/io/trino/plugin/hive/HivePlugin.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePlugin.java similarity index 100% rename from plugin/trino-hive-hadoop2/src/main/java/io/trino/plugin/hive/HivePlugin.java rename to plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePlugin.java diff --git a/plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/TestHivePlugin.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHivePlugin.java similarity index 100% rename from plugin/trino-hive-hadoop2/src/test/java/io/trino/plugin/hive/TestHivePlugin.java rename to plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHivePlugin.java diff --git a/pom.xml b/pom.xml index 7faff11c08d13..81777afc45f89 100644 --- a/pom.xml +++ b/pom.xml @@ -77,7 +77,6 @@ plugin/trino-geospatial plugin/trino-google-sheets plugin/trino-hive - plugin/trino-hive-hadoop2 plugin/trino-http-event-listener plugin/trino-hudi plugin/trino-iceberg @@ -1139,12 +1138,6 @@ ${project.version} - - io.trino - trino-hive-hadoop2 - ${project.version} - - io.trino trino-hudi diff --git a/testing/trino-server-dev/etc/config.properties b/testing/trino-server-dev/etc/config.properties index b786657e8d2b7..67c8b8bcce95a 100644 --- a/testing/trino-server-dev/etc/config.properties +++ b/testing/trino-server-dev/etc/config.properties @@ -37,7 +37,7 @@ plugin.bundles=\ ../../plugin/trino-memory/pom.xml,\ ../../plugin/trino-jmx/pom.xml,\ ../../plugin/trino-raptor-legacy/pom.xml,\ - ../../plugin/trino-hive-hadoop2/pom.xml,\ + ../../plugin/trino-hive/pom.xml,\ ../../plugin/trino-hudi/pom.xml,\ ../../plugin/trino-example-http/pom.xml,\ ../../plugin/trino-kafka/pom.xml, \ From 938afa6ae2b990fc284f1f0859243dfacedad383 Mon Sep 17 00:00:00 2001 From: Martin Traverso Date: Fri, 8 Dec 2023 15:59:05 -0800 Subject: [PATCH 090/350] Remove unnecessary data providers --- .../TestTupleDomainParquetPredicate.java | 101 +++++------ .../parquet/reader/flat/TestNullsDecoder.java | 114 ++++++------ .../trino/parquet/writer/NullsProvider.java | 12 -- .../writer/TestDefinitionLevelWriter.java | 168 +++++++++--------- .../writer/TestRepetitionLevelWriter.java | 134 +++++++------- .../plugin/hive/TestHiveQlTranslation.java | 82 +++++---- .../TestDoubleToVarcharCoercions.java | 58 +++--- 7 files changed, 336 insertions(+), 333 deletions(-) diff --git a/lib/trino-parquet/src/test/java/io/trino/parquet/TestTupleDomainParquetPredicate.java b/lib/trino-parquet/src/test/java/io/trino/parquet/TestTupleDomainParquetPredicate.java index 206c6161249f0..66b58f7701146 100644 --- a/lib/trino-parquet/src/test/java/io/trino/parquet/TestTupleDomainParquetPredicate.java +++ b/lib/trino-parquet/src/test/java/io/trino/parquet/TestTupleDomainParquetPredicate.java @@ -46,7 +46,6 @@ import org.apache.parquet.schema.LogicalTypeAnnotation.TimeUnit; import org.apache.parquet.schema.PrimitiveType; import org.apache.parquet.schema.Types; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import java.io.ByteArrayOutputStream; @@ -402,20 +401,19 @@ public void testDate() .withMessage("Malformed Parquet file. Corrupted statistics for column \"[] required int32 DateColumn\": [min: 200, max: 100, num_nulls: 0] [testFile]"); } - @DataProvider - public Object[][] timestampPrecision() + @Test + public void testTimestampInt96() + throws ParquetCorruptionException { LocalDateTime baseTime = LocalDateTime.of(1970, 1, 19, 10, 28, 52, 123456789); - return new Object[][] { - {3, baseTime, baseTime.atZone(ZoneOffset.UTC).toInstant().toEpochMilli() * MICROSECONDS_PER_MILLISECOND}, - // note the rounding of micros - {6, baseTime, baseTime.atZone(ZoneOffset.UTC).toInstant().getEpochSecond() * MICROSECONDS_PER_SECOND + 123457}, - {9, baseTime, longTimestamp(9, baseTime)} - }; + + testTimestampInt96(3, baseTime, baseTime.atZone(ZoneOffset.UTC).toInstant().toEpochMilli() * MICROSECONDS_PER_MILLISECOND); + // note the rounding of micros + testTimestampInt96(6, baseTime, baseTime.atZone(ZoneOffset.UTC).toInstant().getEpochSecond() * MICROSECONDS_PER_SECOND + 123457); + testTimestampInt96(9, baseTime, longTimestamp(9, baseTime)); } - @Test(dataProvider = "timestampPrecision") - public void testTimestampInt96(int precision, LocalDateTime baseTime, Object baseDomainValue) + private void testTimestampInt96(int precision, LocalDateTime baseTime, Object baseDomainValue) throws ParquetCorruptionException { ColumnDescriptor columnDescriptor = createColumnDescriptor(INT96, "TimestampColumn"); @@ -426,8 +424,35 @@ public void testTimestampInt96(int precision, LocalDateTime baseTime, Object bas assertThat(getDomain(columnDescriptor, timestampType, 10, timestampColumnStats(baseTime.minusSeconds(10), baseTime), ID, UTC)).isEqualTo(create(ValueSet.all(timestampType), false)); } - @Test(dataProvider = "testTimestampInt64DataProvider") - public void testTimestampInt64(TimeUnit timeUnit, int precision, LocalDateTime baseTime, Object baseDomainValue) + @Test + public void testTimestampInt64() + throws ParquetCorruptionException + { + LocalDateTime baseTime = LocalDateTime.of(1970, 1, 19, 10, 28, 52, 123456789); + Object millisExpectedValue = baseTime.atZone(ZoneOffset.UTC).toInstant().toEpochMilli() * MICROSECONDS_PER_MILLISECOND; + // note the rounding of micros + Object microsExpectedValue = baseTime.atZone(ZoneOffset.UTC).toInstant().getEpochSecond() * MICROSECONDS_PER_SECOND + 123457; + Object nanosExpectedValue = longTimestamp(9, baseTime); + + Object nanosTruncatedToMillisExpectedValue = longTimestamp( + 9, + LocalDateTime.of(1970, 1, 19, 10, 28, 52, 123000000)); + Object nanosTruncatedToMicrosExpectedValue = longTimestamp( + 9, + LocalDateTime.of(1970, 1, 19, 10, 28, 52, 123457000)); + + testTimestampInt64(TimeUnit.MILLIS, 3, baseTime, millisExpectedValue); + testTimestampInt64(TimeUnit.MICROS, 3, baseTime, millisExpectedValue); + testTimestampInt64(TimeUnit.NANOS, 3, baseTime, millisExpectedValue); + testTimestampInt64(TimeUnit.MILLIS, 6, baseTime, millisExpectedValue); + testTimestampInt64(TimeUnit.MICROS, 6, baseTime, microsExpectedValue); + testTimestampInt64(TimeUnit.NANOS, 6, baseTime, microsExpectedValue); + testTimestampInt64(TimeUnit.MILLIS, 9, baseTime, nanosTruncatedToMillisExpectedValue); + testTimestampInt64(TimeUnit.MICROS, 9, baseTime, nanosTruncatedToMicrosExpectedValue); + testTimestampInt64(TimeUnit.NANOS, 9, baseTime, nanosExpectedValue); + } + + private void testTimestampInt64(TimeUnit timeUnit, int precision, LocalDateTime baseTime, Object baseDomainValue) throws ParquetCorruptionException { int parquetPrecision; @@ -471,34 +496,6 @@ else if (baseDomainValue instanceof LongTimestamp longTimestamp) { assertThat(getDomain(columnDescriptor, timestampType, 10, longColumnStats(minValue, maxValue), ID, UTC)).isEqualTo(create(ValueSet.ofRanges(range(timestampType, baseDomainValue, true, maxDomainValue, true)), false)); } - @DataProvider - public Object[][] testTimestampInt64DataProvider() - { - LocalDateTime baseTime = LocalDateTime.of(1970, 1, 19, 10, 28, 52, 123456789); - Object millisExpectedValue = baseTime.atZone(ZoneOffset.UTC).toInstant().toEpochMilli() * MICROSECONDS_PER_MILLISECOND; - // note the rounding of micros - Object microsExpectedValue = baseTime.atZone(ZoneOffset.UTC).toInstant().getEpochSecond() * MICROSECONDS_PER_SECOND + 123457; - Object nanosExpectedValue = longTimestamp(9, baseTime); - - Object nanosTruncatedToMillisExpectedValue = longTimestamp( - 9, - LocalDateTime.of(1970, 1, 19, 10, 28, 52, 123000000)); - Object nanosTruncatedToMicrosExpectedValue = longTimestamp( - 9, - LocalDateTime.of(1970, 1, 19, 10, 28, 52, 123457000)); - return new Object[][] { - {TimeUnit.MILLIS, 3, baseTime, millisExpectedValue}, - {TimeUnit.MICROS, 3, baseTime, millisExpectedValue}, - {TimeUnit.NANOS, 3, baseTime, millisExpectedValue}, - {TimeUnit.MILLIS, 6, baseTime, millisExpectedValue}, - {TimeUnit.MICROS, 6, baseTime, microsExpectedValue}, - {TimeUnit.NANOS, 6, baseTime, microsExpectedValue}, - {TimeUnit.MILLIS, 9, baseTime, nanosTruncatedToMillisExpectedValue}, - {TimeUnit.MICROS, 9, baseTime, nanosTruncatedToMicrosExpectedValue}, - {TimeUnit.NANOS, 9, baseTime, nanosExpectedValue}, - }; - } - private static long toEpochWithPrecision(LocalDateTime time, int precision) { long scaledEpochSeconds = time.toEpochSecond(ZoneOffset.UTC) * (long) Math.pow(10, precision); @@ -549,8 +546,16 @@ public void testVarcharMatchesWithStatistics() .isEqualTo(Optional.of(ImmutableList.of(column))); } - @Test(dataProvider = "typeForParquetInt32") - public void testIntegerMatchesWithStatistics(Type typeForParquetInt32) + @Test + public void testIntegerMatchesWithStatistics() + throws ParquetCorruptionException + { + testIntegerMatchesWithStatistics(INTEGER); + testIntegerMatchesWithStatistics(SMALLINT); + testIntegerMatchesWithStatistics(TINYINT); + } + + private void testIntegerMatchesWithStatistics(Type typeForParquetInt32) throws ParquetCorruptionException { ColumnDescriptor column = createColumnDescriptor(INT32, "Test column"); @@ -566,16 +571,6 @@ public void testIntegerMatchesWithStatistics(Type typeForParquetInt32) .isEqualTo(typeForParquetInt32 != INTEGER); // stats invalid for smallint/tinyint } - @DataProvider - public Object[][] typeForParquetInt32() - { - return new Object[][] { - {INTEGER}, - {SMALLINT}, - {TINYINT}, - }; - } - @Test public void testBigintMatchesWithStatistics() throws ParquetCorruptionException diff --git a/lib/trino-parquet/src/test/java/io/trino/parquet/reader/flat/TestNullsDecoder.java b/lib/trino-parquet/src/test/java/io/trino/parquet/reader/flat/TestNullsDecoder.java index 8003acbe7bdaf..6a33970b9f34a 100644 --- a/lib/trino-parquet/src/test/java/io/trino/parquet/reader/flat/TestNullsDecoder.java +++ b/lib/trino-parquet/src/test/java/io/trino/parquet/reader/flat/TestNullsDecoder.java @@ -16,17 +16,13 @@ import io.airlift.slice.Slices; import org.apache.parquet.bytes.HeapByteBufferAllocator; import org.apache.parquet.column.values.rle.RunLengthBitPackingHybridEncoder; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import java.io.IOException; import java.util.Arrays; import java.util.Random; -import java.util.stream.Stream; import static io.trino.parquet.reader.TestData.generateMixedData; -import static io.trino.testing.DataProviders.cartesianProduct; -import static io.trino.testing.DataProviders.toDataProvider; import static java.lang.Math.min; import static org.assertj.core.api.Assertions.assertThat; @@ -48,69 +44,69 @@ public class TestNullsDecoder MIXED_RANDOM_AND_GROUPED_ARRAY = generateMixedData(r, N, MAX_MIXED_GROUP_SIZE); } - @Test(dataProvider = "dataSets") - public void testDecoding(NullValuesProvider nullValuesProvider, int batchSize) + @Test + public void testDecoding() throws IOException { - boolean[] values = nullValuesProvider.getPositions(); - byte[] encoded = encode(values); - NullsDecoder decoder = new NullsDecoder(); - decoder.init(Slices.wrappedBuffer(encoded)); - boolean[] result = new boolean[N]; - int nonNullCount = 0; - for (int i = 0; i < N; i += batchSize) { - nonNullCount += decoder.readNext(result, i, min(batchSize, N - i)); + for (NullValuesProvider nullValuesProvider : NullValuesProvider.values()) { + for (int batchSize : Arrays.asList(1, 3, 16, 100, 1000)) { + boolean[] values = nullValuesProvider.getPositions(); + byte[] encoded = encode(values); + NullsDecoder decoder = new NullsDecoder(); + decoder.init(Slices.wrappedBuffer(encoded)); + boolean[] result = new boolean[N]; + int nonNullCount = 0; + for (int i = 0; i < N; i += batchSize) { + nonNullCount += decoder.readNext(result, i, min(batchSize, N - i)); + } + // Parquet encodes whether value exists, Trino whether value is null + boolean[] byteResult = flip(result); + assertThat(byteResult).containsExactly(values); + + int expectedNonNull = nonNullCount(values); + assertThat(nonNullCount).isEqualTo(expectedNonNull); + } } - // Parquet encodes whether value exists, Trino whether value is null - boolean[] byteResult = flip(result); - assertThat(byteResult).containsExactly(values); - - int expectedNonNull = nonNullCount(values); - assertThat(nonNullCount).isEqualTo(expectedNonNull); } - @Test(dataProvider = "dataSets") - public void testSkippedDecoding(NullValuesProvider nullValuesProvider, int batchSize) + @Test + public void testSkippedDecoding() throws IOException { - boolean[] values = nullValuesProvider.getPositions(); - byte[] encoded = encode(values); - NullsDecoder decoder = new NullsDecoder(); - decoder.init(Slices.wrappedBuffer(encoded)); - int nonNullCount = 0; - int numberOfBatches = (N + batchSize - 1) / batchSize; - Random random = new Random(batchSize * 0xFFFFFFFFL * N); - int skippedBatches = random.nextInt(numberOfBatches); - int alreadyRead = 0; - for (int i = 0; i < skippedBatches; i++) { - int chunkSize = min(batchSize, N - alreadyRead); - nonNullCount += decoder.skip(chunkSize); - alreadyRead += chunkSize; - } - assertThat(nonNullCount).isEqualTo(nonNullCount(values, alreadyRead)); - - boolean[] result = new boolean[N - alreadyRead]; - boolean[] expected = Arrays.copyOfRange(values, alreadyRead, values.length); - int offset = 0; - while (alreadyRead < N) { - int chunkSize = min(batchSize, N - alreadyRead); - nonNullCount += decoder.readNext(result, offset, chunkSize); - alreadyRead += chunkSize; - offset += chunkSize; + for (NullValuesProvider nullValuesProvider : NullValuesProvider.values()) { + for (int batchSize : Arrays.asList(1, 3, 16, 100, 1000)) { + boolean[] values = nullValuesProvider.getPositions(); + byte[] encoded = encode(values); + NullsDecoder decoder = new NullsDecoder(); + decoder.init(Slices.wrappedBuffer(encoded)); + int nonNullCount = 0; + int numberOfBatches = (N + batchSize - 1) / batchSize; + Random random = new Random(batchSize * 0xFFFFFFFFL * N); + int skippedBatches = random.nextInt(numberOfBatches); + int alreadyRead = 0; + for (int i = 0; i < skippedBatches; i++) { + int chunkSize = min(batchSize, N - alreadyRead); + nonNullCount += decoder.skip(chunkSize); + alreadyRead += chunkSize; + } + assertThat(nonNullCount).isEqualTo(nonNullCount(values, alreadyRead)); + + boolean[] result = new boolean[N - alreadyRead]; + boolean[] expected = Arrays.copyOfRange(values, alreadyRead, values.length); + int offset = 0; + while (alreadyRead < N) { + int chunkSize = min(batchSize, N - alreadyRead); + nonNullCount += decoder.readNext(result, offset, chunkSize); + alreadyRead += chunkSize; + offset += chunkSize; + } + // Parquet encodes whether value exists, Trino whether value is null + boolean[] byteResult = flip(result); + assertThat(byteResult).containsExactly(expected); + + assertThat(nonNullCount).isEqualTo(nonNullCount(values)); + } } - // Parquet encodes whether value exists, Trino whether value is null - boolean[] byteResult = flip(result); - assertThat(byteResult).containsExactly(expected); - - assertThat(nonNullCount).isEqualTo(nonNullCount(values)); - } - - @DataProvider(name = "dataSets") - public static Object[][] dataSets() - { - return cartesianProduct( - Arrays.stream(NullValuesProvider.values()).collect(toDataProvider()), - Stream.of(1, 3, 16, 100, 1000).collect(toDataProvider())); } private enum NullValuesProvider diff --git a/lib/trino-parquet/src/test/java/io/trino/parquet/writer/NullsProvider.java b/lib/trino-parquet/src/test/java/io/trino/parquet/writer/NullsProvider.java index f0b3836264689..304ae28eafcba 100644 --- a/lib/trino-parquet/src/test/java/io/trino/parquet/writer/NullsProvider.java +++ b/lib/trino-parquet/src/test/java/io/trino/parquet/writer/NullsProvider.java @@ -13,14 +13,9 @@ */ package io.trino.parquet.writer; -import org.testng.annotations.DataProvider; - import java.util.Arrays; import java.util.Optional; import java.util.Random; -import java.util.stream.Stream; - -import static io.trino.testing.DataProviders.toDataProvider; enum NullsProvider { @@ -98,11 +93,4 @@ Optional getNulls(int positionCount, Optional forcedNulls) } return Optional.of(nullPositions); } - - @DataProvider - public static Object[][] nullsProviders() - { - return Stream.of(NullsProvider.values()) - .collect(toDataProvider()); - } } diff --git a/lib/trino-parquet/src/test/java/io/trino/parquet/writer/TestDefinitionLevelWriter.java b/lib/trino-parquet/src/test/java/io/trino/parquet/writer/TestDefinitionLevelWriter.java index 3cab34ede5f5f..1bd3473a6cb89 100644 --- a/lib/trino-parquet/src/test/java/io/trino/parquet/writer/TestDefinitionLevelWriter.java +++ b/lib/trino-parquet/src/test/java/io/trino/parquet/writer/TestDefinitionLevelWriter.java @@ -42,105 +42,113 @@ public class TestDefinitionLevelWriter { private static final int POSITIONS = 8096; - @Test(dataProviderClass = NullsProvider.class, dataProvider = "nullsProviders") - public void testWritePrimitiveDefinitionLevels(NullsProvider nullsProvider) + @Test + public void testWritePrimitiveDefinitionLevels() { - Block block = new LongArrayBlock(POSITIONS, nullsProvider.getNulls(POSITIONS), new long[POSITIONS]); - int maxDefinitionLevel = 3; - // Write definition levels for all positions - assertDefinitionLevels(block, ImmutableList.of(), maxDefinitionLevel); + for (NullsProvider nullsProvider : NullsProvider.values()) { + Block block = new LongArrayBlock(POSITIONS, nullsProvider.getNulls(POSITIONS), new long[POSITIONS]); + int maxDefinitionLevel = 3; + // Write definition levels for all positions + assertDefinitionLevels(block, ImmutableList.of(), maxDefinitionLevel); - // Write definition levels for all positions one-at-a-time - assertDefinitionLevels(block, nCopies(block.getPositionCount(), 1), maxDefinitionLevel); + // Write definition levels for all positions one-at-a-time + assertDefinitionLevels(block, nCopies(block.getPositionCount(), 1), maxDefinitionLevel); - // Write definition levels for all positions with different group sizes - assertDefinitionLevels(block, generateGroupSizes(block.getPositionCount()), maxDefinitionLevel); + // Write definition levels for all positions with different group sizes + assertDefinitionLevels(block, generateGroupSizes(block.getPositionCount()), maxDefinitionLevel); + } } - @Test(dataProviderClass = NullsProvider.class, dataProvider = "nullsProviders") - public void testWriteRowDefinitionLevels(NullsProvider nullsProvider) + @Test + public void testWriteRowDefinitionLevels() { - RowBlock rowBlock = createRowBlock(nullsProvider.getNulls(POSITIONS), POSITIONS); - List fields = getNullSuppressedRowFieldsFromBlock(rowBlock); - int fieldMaxDefinitionLevel = 2; - // Write definition levels for all positions - for (int field = 0; field < fields.size(); field++) { - assertDefinitionLevels(rowBlock, fields, ImmutableList.of(), field, fieldMaxDefinitionLevel); - } + for (NullsProvider nullsProvider : NullsProvider.values()) { + RowBlock rowBlock = createRowBlock(nullsProvider.getNulls(POSITIONS), POSITIONS); + List fields = getNullSuppressedRowFieldsFromBlock(rowBlock); + int fieldMaxDefinitionLevel = 2; + // Write definition levels for all positions + for (int field = 0; field < fields.size(); field++) { + assertDefinitionLevels(rowBlock, fields, ImmutableList.of(), field, fieldMaxDefinitionLevel); + } - // Write definition levels for all positions one-at-a-time - for (int field = 0; field < fields.size(); field++) { - assertDefinitionLevels( - rowBlock, - fields, - nCopies(rowBlock.getPositionCount(), 1), - field, - fieldMaxDefinitionLevel); - } + // Write definition levels for all positions one-at-a-time + for (int field = 0; field < fields.size(); field++) { + assertDefinitionLevels( + rowBlock, + fields, + nCopies(rowBlock.getPositionCount(), 1), + field, + fieldMaxDefinitionLevel); + } - // Write definition levels for all positions with different group sizes - for (int field = 0; field < fields.size(); field++) { - assertDefinitionLevels( - rowBlock, - fields, - generateGroupSizes(rowBlock.getPositionCount()), - field, - fieldMaxDefinitionLevel); + // Write definition levels for all positions with different group sizes + for (int field = 0; field < fields.size(); field++) { + assertDefinitionLevels( + rowBlock, + fields, + generateGroupSizes(rowBlock.getPositionCount()), + field, + fieldMaxDefinitionLevel); + } } } - @Test(dataProviderClass = NullsProvider.class, dataProvider = "nullsProviders") - public void testWriteArrayDefinitionLevels(NullsProvider nullsProvider) + @Test + public void testWriteArrayDefinitionLevels() { - Block arrayBlock = createArrayBlock(nullsProvider.getNulls(POSITIONS), POSITIONS); - ColumnarArray columnarArray = toColumnarArray(arrayBlock); - int maxDefinitionLevel = 3; - // Write definition levels for all positions - assertDefinitionLevels( - columnarArray, - ImmutableList.of(), - maxDefinitionLevel); + for (NullsProvider nullsProvider : NullsProvider.values()) { + Block arrayBlock = createArrayBlock(nullsProvider.getNulls(POSITIONS), POSITIONS); + ColumnarArray columnarArray = toColumnarArray(arrayBlock); + int maxDefinitionLevel = 3; + // Write definition levels for all positions + assertDefinitionLevels( + columnarArray, + ImmutableList.of(), + maxDefinitionLevel); - // Write definition levels for all positions one-at-a-time - assertDefinitionLevels( - columnarArray, - nCopies(columnarArray.getPositionCount(), 1), - maxDefinitionLevel); + // Write definition levels for all positions one-at-a-time + assertDefinitionLevels( + columnarArray, + nCopies(columnarArray.getPositionCount(), 1), + maxDefinitionLevel); - // Write definition levels for all positions with different group sizes - assertDefinitionLevels( - columnarArray, - generateGroupSizes(columnarArray.getPositionCount()), - maxDefinitionLevel); + // Write definition levels for all positions with different group sizes + assertDefinitionLevels( + columnarArray, + generateGroupSizes(columnarArray.getPositionCount()), + maxDefinitionLevel); + } } - @Test(dataProviderClass = NullsProvider.class, dataProvider = "nullsProviders") - public void testWriteMapDefinitionLevels(NullsProvider nullsProvider) + @Test + public void testWriteMapDefinitionLevels() { - Block mapBlock = createMapBlock(nullsProvider.getNulls(POSITIONS), POSITIONS); - ColumnarMap columnarMap = toColumnarMap(mapBlock); - int keysMaxDefinitionLevel = 2; - int valuesMaxDefinitionLevel = 3; - // Write definition levels for all positions - assertDefinitionLevels( - columnarMap, - ImmutableList.of(), - keysMaxDefinitionLevel, - valuesMaxDefinitionLevel); + for (NullsProvider nullsProvider : NullsProvider.values()) { + Block mapBlock = createMapBlock(nullsProvider.getNulls(POSITIONS), POSITIONS); + ColumnarMap columnarMap = toColumnarMap(mapBlock); + int keysMaxDefinitionLevel = 2; + int valuesMaxDefinitionLevel = 3; + // Write definition levels for all positions + assertDefinitionLevels( + columnarMap, + ImmutableList.of(), + keysMaxDefinitionLevel, + valuesMaxDefinitionLevel); - // Write definition levels for all positions one-at-a-time - assertDefinitionLevels( - columnarMap, - nCopies(columnarMap.getPositionCount(), 1), - keysMaxDefinitionLevel, - valuesMaxDefinitionLevel); + // Write definition levels for all positions one-at-a-time + assertDefinitionLevels( + columnarMap, + nCopies(columnarMap.getPositionCount(), 1), + keysMaxDefinitionLevel, + valuesMaxDefinitionLevel); - // Write definition levels for all positions with different group sizes - assertDefinitionLevels( - columnarMap, - generateGroupSizes(columnarMap.getPositionCount()), - keysMaxDefinitionLevel, - valuesMaxDefinitionLevel); + // Write definition levels for all positions with different group sizes + assertDefinitionLevels( + columnarMap, + generateGroupSizes(columnarMap.getPositionCount()), + keysMaxDefinitionLevel, + valuesMaxDefinitionLevel); + } } private static void assertDefinitionLevels(Block block, List writePositionCounts, int maxDefinitionLevel) diff --git a/lib/trino-parquet/src/test/java/io/trino/parquet/writer/TestRepetitionLevelWriter.java b/lib/trino-parquet/src/test/java/io/trino/parquet/writer/TestRepetitionLevelWriter.java index 4f38866c003a0..e4e4808bdfbef 100644 --- a/lib/trino-parquet/src/test/java/io/trino/parquet/writer/TestRepetitionLevelWriter.java +++ b/lib/trino-parquet/src/test/java/io/trino/parquet/writer/TestRepetitionLevelWriter.java @@ -49,91 +49,99 @@ public class TestRepetitionLevelWriter { private static final int POSITIONS = 1024; - @Test(dataProviderClass = NullsProvider.class, dataProvider = "nullsProviders") - public void testWriteRowRepetitionLevels(NullsProvider nullsProvider) + @Test + public void testWriteRowRepetitionLevels() { - // Using an array of row blocks for testing as Structs don't have a repetition level by themselves - Optional valueIsNull = RANDOM_NULLS.getNulls(POSITIONS); - int[] arrayOffsets = generateOffsets(valueIsNull, POSITIONS); - int rowBlockPositions = arrayOffsets[POSITIONS]; - RowBlock rowBlock = createRowBlock(nullsProvider.getNulls(rowBlockPositions), rowBlockPositions); - ArrayBlock arrayBlock = fromElementBlock(POSITIONS, valueIsNull, arrayOffsets, rowBlock); + for (NullsProvider nullsProvider : NullsProvider.values()) { + // Using an array of row blocks for testing as Structs don't have a repetition level by themselves + Optional valueIsNull = RANDOM_NULLS.getNulls(POSITIONS); + int[] arrayOffsets = generateOffsets(valueIsNull, POSITIONS); + int rowBlockPositions = arrayOffsets[POSITIONS]; + RowBlock rowBlock = createRowBlock(nullsProvider.getNulls(rowBlockPositions), rowBlockPositions); + ArrayBlock arrayBlock = fromElementBlock(POSITIONS, valueIsNull, arrayOffsets, rowBlock); - ColumnarArray columnarArray = toColumnarArray(arrayBlock); - Block row = columnarArray.getElementsBlock(); - List nullSuppressedFields = getNullSuppressedRowFieldsFromBlock(row); - // Write Repetition levels for all positions - for (int fieldIndex = 0; fieldIndex < nullSuppressedFields.size(); fieldIndex++) { - Block field = nullSuppressedFields.get(fieldIndex); - assertRepetitionLevels(columnarArray, row, field, ImmutableList.of()); - assertRepetitionLevels(columnarArray, row, field, ImmutableList.of()); + ColumnarArray columnarArray = toColumnarArray(arrayBlock); + Block row = columnarArray.getElementsBlock(); + List nullSuppressedFields = getNullSuppressedRowFieldsFromBlock(row); + // Write Repetition levels for all positions + for (int fieldIndex = 0; fieldIndex < nullSuppressedFields.size(); fieldIndex++) { + Block field = nullSuppressedFields.get(fieldIndex); + assertRepetitionLevels(columnarArray, row, field, ImmutableList.of()); + assertRepetitionLevels(columnarArray, row, field, ImmutableList.of()); - // Write Repetition levels for all positions one-at-a-time - assertRepetitionLevels( - columnarArray, - row, - field, - nCopies(columnarArray.getPositionCount(), 1)); + // Write Repetition levels for all positions one-at-a-time + assertRepetitionLevels( + columnarArray, + row, + field, + nCopies(columnarArray.getPositionCount(), 1)); - // Write Repetition levels for all positions with different group sizes - assertRepetitionLevels( - columnarArray, - row, - field, - generateGroupSizes(columnarArray.getPositionCount())); + // Write Repetition levels for all positions with different group sizes + assertRepetitionLevels( + columnarArray, + row, + field, + generateGroupSizes(columnarArray.getPositionCount())); + } } } - @Test(dataProviderClass = NullsProvider.class, dataProvider = "nullsProviders") - public void testWriteArrayRepetitionLevels(NullsProvider nullsProvider) + @Test + public void testWriteArrayRepetitionLevels() { - Block arrayBlock = createArrayBlock(nullsProvider.getNulls(POSITIONS), POSITIONS); - ColumnarArray columnarArray = toColumnarArray(arrayBlock); - // Write Repetition levels for all positions - assertRepetitionLevels(columnarArray, ImmutableList.of()); + for (NullsProvider nullsProvider : NullsProvider.values()) { + Block arrayBlock = createArrayBlock(nullsProvider.getNulls(POSITIONS), POSITIONS); + ColumnarArray columnarArray = toColumnarArray(arrayBlock); + // Write Repetition levels for all positions + assertRepetitionLevels(columnarArray, ImmutableList.of()); - // Write Repetition levels for all positions one-at-a-time - assertRepetitionLevels(columnarArray, nCopies(columnarArray.getPositionCount(), 1)); + // Write Repetition levels for all positions one-at-a-time + assertRepetitionLevels(columnarArray, nCopies(columnarArray.getPositionCount(), 1)); - // Write Repetition levels for all positions with different group sizes - assertRepetitionLevels(columnarArray, generateGroupSizes(columnarArray.getPositionCount())); + // Write Repetition levels for all positions with different group sizes + assertRepetitionLevels(columnarArray, generateGroupSizes(columnarArray.getPositionCount())); + } } - @Test(dataProviderClass = NullsProvider.class, dataProvider = "nullsProviders") - public void testWriteMapRepetitionLevels(NullsProvider nullsProvider) + @Test + public void testWriteMapRepetitionLevels() { - Block mapBlock = createMapBlock(nullsProvider.getNulls(POSITIONS), POSITIONS); - ColumnarMap columnarMap = toColumnarMap(mapBlock); - // Write Repetition levels for all positions - assertRepetitionLevels(columnarMap, ImmutableList.of()); + for (NullsProvider nullsProvider : NullsProvider.values()) { + Block mapBlock = createMapBlock(nullsProvider.getNulls(POSITIONS), POSITIONS); + ColumnarMap columnarMap = toColumnarMap(mapBlock); + // Write Repetition levels for all positions + assertRepetitionLevels(columnarMap, ImmutableList.of()); - // Write Repetition levels for all positions one-at-a-time - assertRepetitionLevels(columnarMap, nCopies(columnarMap.getPositionCount(), 1)); + // Write Repetition levels for all positions one-at-a-time + assertRepetitionLevels(columnarMap, nCopies(columnarMap.getPositionCount(), 1)); - // Write Repetition levels for all positions with different group sizes - assertRepetitionLevels(columnarMap, generateGroupSizes(columnarMap.getPositionCount())); + // Write Repetition levels for all positions with different group sizes + assertRepetitionLevels(columnarMap, generateGroupSizes(columnarMap.getPositionCount())); + } } - @Test(dataProviderClass = NullsProvider.class, dataProvider = "nullsProviders") - public void testNestedStructRepetitionLevels(NullsProvider nullsProvider) + @Test + public void testNestedStructRepetitionLevels() { - RowBlock rowBlock = createNestedRowBlock(nullsProvider.getNulls(POSITIONS), POSITIONS); - List fieldBlocks = getNullSuppressedRowFieldsFromBlock(rowBlock); + for (NullsProvider nullsProvider : NullsProvider.values()) { + RowBlock rowBlock = createNestedRowBlock(nullsProvider.getNulls(POSITIONS), POSITIONS); + List fieldBlocks = getNullSuppressedRowFieldsFromBlock(rowBlock); - for (int field = 0; field < fieldBlocks.size(); field++) { - Block fieldBlock = fieldBlocks.get(field); - ColumnarMap columnarMap = toColumnarMap(fieldBlock); - for (Block mapElements : ImmutableList.of(columnarMap.getKeysBlock(), columnarMap.getValuesBlock())) { - ColumnarArray columnarArray = toColumnarArray(mapElements); + for (int field = 0; field < fieldBlocks.size(); field++) { + Block fieldBlock = fieldBlocks.get(field); + ColumnarMap columnarMap = toColumnarMap(fieldBlock); + for (Block mapElements : ImmutableList.of(columnarMap.getKeysBlock(), columnarMap.getValuesBlock())) { + ColumnarArray columnarArray = toColumnarArray(mapElements); - // Write Repetition levels for all positions - assertRepetitionLevels(rowBlock, columnarMap, columnarArray, ImmutableList.of()); + // Write Repetition levels for all positions + assertRepetitionLevels(rowBlock, columnarMap, columnarArray, ImmutableList.of()); - // Write Repetition levels for all positions one-at-a-time - assertRepetitionLevels(rowBlock, columnarMap, columnarArray, nCopies(rowBlock.getPositionCount(), 1)); + // Write Repetition levels for all positions one-at-a-time + assertRepetitionLevels(rowBlock, columnarMap, columnarArray, nCopies(rowBlock.getPositionCount(), 1)); - // Write Repetition levels for all positions with different group sizes - assertRepetitionLevels(rowBlock, columnarMap, columnarArray, generateGroupSizes(rowBlock.getPositionCount())); + // Write Repetition levels for all positions with different group sizes + assertRepetitionLevels(rowBlock, columnarMap, columnarArray, generateGroupSizes(rowBlock.getPositionCount())); + } } } } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveQlTranslation.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveQlTranslation.java index 79a129cdcf4f3..fccfa97c17e9c 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveQlTranslation.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveQlTranslation.java @@ -13,19 +13,15 @@ */ package io.trino.plugin.hive; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Iterators; import com.google.common.collect.Lists; -import com.google.common.collect.Sets; -import com.google.common.collect.Streams; import io.trino.sql.parser.SqlParser; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; -import java.util.Iterator; +import java.util.Collection; import java.util.List; import java.util.Map; -import java.util.stream.Stream; import static io.trino.plugin.hive.HiveErrorCode.HIVE_VIEW_TRANSLATION_ERROR; import static io.trino.plugin.hive.HiveToTrinoTranslator.translateHiveViewToTrino; @@ -104,33 +100,11 @@ public class TestHiveQlTranslation /** * Prepare all combinations of {@code n} of the given columns. */ - private static Iterator getNColumns(int n, Map columns) + private static List getNColumns(int n, Collection columns) { - Stream hiveNames = - Sets.cartesianProduct(nCopies(n, columns.keySet())).stream() - .map(names -> join(", ", names)); - - Stream trinoNames = - Lists.cartesianProduct(nCopies(n, List.copyOf(columns.values()))).stream() - .map(names -> join(", ", names)); - - return Streams.zip(hiveNames, trinoNames, (h, p) -> new Object[] {h, p}).iterator(); - } - - @DataProvider(name = "simple_hive_translation_columns") - public Iterator getSimpleColumns() - { - return Iterators.concat( - getNColumns(1, simpleColumnNames), - getNColumns(3, simpleColumnNames)); - } - - @DataProvider(name = "extended_hive_translation_columns") - public Iterator getExtendedColumns() - { - return Iterators.concat( - getNColumns(1, extendedColumnNames), - getNColumns(2, extendedColumnNames)); + return Lists.cartesianProduct(nCopies(n, List.copyOf(columns))).stream() + .map(names -> join(", ", names)) + .toList(); } @Test @@ -212,20 +186,44 @@ public void testPredicates() "SELECT '''' = '''' OR false"); } - @Test(dataProvider = "simple_hive_translation_columns") - public void testSimpleColumns(String hiveColumn, String trinoColumn) + @Test + public void testSimpleColumns() { - assertTranslation( - format("SELECT %s FROM sometable", hiveColumn), - format("SELECT %s FROM sometable", trinoColumn)); + List hiveColumns = ImmutableList.builder() + .addAll(getNColumns(1, simpleColumnNames.keySet())) + .addAll(getNColumns(3, simpleColumnNames.keySet())) + .build(); + + List trinoColumns = ImmutableList.builder() + .addAll(getNColumns(1, simpleColumnNames.values())) + .addAll(getNColumns(3, simpleColumnNames.values())) + .build(); + + for (int i = 0; i < hiveColumns.size(); i++) { + assertTranslation( + format("SELECT %s FROM sometable", hiveColumns.get(i)), + format("SELECT %s FROM sometable", trinoColumns.get(i))); + } } - @Test(dataProvider = "extended_hive_translation_columns") - public void testExtendedColumns(String hiveColumn, String trinoColumn) + @Test + public void testExtendedColumns() { - assertTranslation( - format("SELECT %s FROM sometable", hiveColumn), - format("SELECT %s FROM sometable", trinoColumn)); + List hiveColumns = ImmutableList.builder() + .addAll(getNColumns(1, extendedColumnNames.keySet())) + .addAll(getNColumns(3, extendedColumnNames.keySet())) + .build(); + + List trinoColumns = ImmutableList.builder() + .addAll(getNColumns(1, extendedColumnNames.values())) + .addAll(getNColumns(3, extendedColumnNames.values())) + .build(); + + for (int i = 0; i < hiveColumns.size(); i++) { + assertTranslation( + format("SELECT %s FROM sometable", hiveColumns.get(i)), + format("SELECT %s FROM sometable", trinoColumns.get(i))); + } } @Test diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/coercions/TestDoubleToVarcharCoercions.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/coercions/TestDoubleToVarcharCoercions.java index 526c085e463e2..3101c3d9e0c68 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/coercions/TestDoubleToVarcharCoercions.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/coercions/TestDoubleToVarcharCoercions.java @@ -18,11 +18,8 @@ import io.trino.spi.TrinoException; import io.trino.spi.block.Block; import io.trino.spi.type.Type; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; -import java.util.stream.Stream; - import static io.trino.plugin.hive.HiveTimestampPrecision.DEFAULT_PRECISION; import static io.trino.plugin.hive.HiveType.toHiveType; import static io.trino.plugin.hive.coercions.CoercionUtils.createCoercer; @@ -31,23 +28,50 @@ import static io.trino.spi.type.DoubleType.DOUBLE; import static io.trino.spi.type.VarcharType.createUnboundedVarcharType; import static io.trino.spi.type.VarcharType.createVarcharType; -import static io.trino.testing.DataProviders.cartesianProduct; -import static io.trino.testing.DataProviders.toDataProvider; -import static io.trino.testing.DataProviders.trueFalse; import static io.trino.type.InternalTypeManager.TESTING_TYPE_MANAGER; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; public class TestDoubleToVarcharCoercions { - @Test(dataProvider = "doubleValues") - public void testDoubleToVarcharCoercions(Double doubleValue, boolean treatNaNAsNull) + @Test + public void testDoubleToVarcharCoercions() + { + testDoubleToVarcharCoercions(Double.NEGATIVE_INFINITY, true); + testDoubleToVarcharCoercions(Double.MIN_VALUE, true); + testDoubleToVarcharCoercions(Double.MAX_VALUE, true); + testDoubleToVarcharCoercions(Double.POSITIVE_INFINITY, true); + testDoubleToVarcharCoercions(Double.parseDouble("123456789.12345678"), true); + + testDoubleToVarcharCoercions(Double.NEGATIVE_INFINITY, false); + testDoubleToVarcharCoercions(Double.MIN_VALUE, false); + testDoubleToVarcharCoercions(Double.MAX_VALUE, false); + testDoubleToVarcharCoercions(Double.POSITIVE_INFINITY, false); + testDoubleToVarcharCoercions(Double.parseDouble("123456789.12345678"), false); + } + + private void testDoubleToVarcharCoercions(Double doubleValue, boolean treatNaNAsNull) { assertCoercions(DOUBLE, doubleValue, createUnboundedVarcharType(), Slices.utf8Slice(doubleValue.toString()), treatNaNAsNull); } - @Test(dataProvider = "doubleValues") - public void testDoubleSmallerVarcharCoercions(Double doubleValue, boolean treatNaNAsNull) + @Test + public void testDoubleSmallerVarcharCoercions() + { + testDoubleSmallerVarcharCoercions(Double.NEGATIVE_INFINITY, true); + testDoubleSmallerVarcharCoercions(Double.MIN_VALUE, true); + testDoubleSmallerVarcharCoercions(Double.MAX_VALUE, true); + testDoubleSmallerVarcharCoercions(Double.POSITIVE_INFINITY, true); + testDoubleSmallerVarcharCoercions(Double.parseDouble("123456789.12345678"), true); + + testDoubleSmallerVarcharCoercions(Double.NEGATIVE_INFINITY, false); + testDoubleSmallerVarcharCoercions(Double.MIN_VALUE, false); + testDoubleSmallerVarcharCoercions(Double.MAX_VALUE, false); + testDoubleSmallerVarcharCoercions(Double.POSITIVE_INFINITY, false); + testDoubleSmallerVarcharCoercions(Double.parseDouble("123456789.12345678"), false); + } + + private void testDoubleSmallerVarcharCoercions(Double doubleValue, boolean treatNaNAsNull) { assertThatThrownBy(() -> assertCoercions(DOUBLE, doubleValue, createVarcharType(1), doubleValue.toString(), treatNaNAsNull)) .isInstanceOf(TrinoException.class) @@ -65,20 +89,6 @@ public void testNaNToVarcharCoercions() .hasMessageContaining("Varchar representation of NaN exceeds varchar(1) bounds"); } - @DataProvider - public Object[][] doubleValues() - { - return cartesianProduct( - Stream.of( - Double.NEGATIVE_INFINITY, - Double.MIN_VALUE, - Double.MAX_VALUE, - Double.POSITIVE_INFINITY, - Double.parseDouble("123456789.12345678")) - .collect(toDataProvider()), - trueFalse()); - } - public static void assertCoercions(Type fromType, Object valueToBeCoerced, Type toType, Object expectedValue, boolean treatNaNAsNull) { Block coercedValue = createCoercer(TESTING_TYPE_MANAGER, toHiveType(fromType), toHiveType(toType), new CoercionContext(DEFAULT_PRECISION, treatNaNAsNull)).orElseThrow() From 6b55f5ecfd8fa10a8b14c7ed586667ceb97cfd8b Mon Sep 17 00:00:00 2001 From: Martin Traverso Date: Fri, 8 Dec 2023 16:01:04 -0800 Subject: [PATCH 091/350] Migrate tests to JUnit --- .../java/io/trino/parquet/TestTupleDomainParquetPredicate.java | 2 +- .../java/io/trino/parquet/reader/flat/TestNullsDecoder.java | 2 +- .../java/io/trino/parquet/writer/TestDefinitionLevelWriter.java | 2 +- .../java/io/trino/parquet/writer/TestRepetitionLevelWriter.java | 2 +- .../test/java/io/trino/plugin/hive/TestHiveQlTranslation.java | 2 +- .../plugin/hive/coercions/TestDoubleToVarcharCoercions.java | 2 +- 6 files changed, 6 insertions(+), 6 deletions(-) diff --git a/lib/trino-parquet/src/test/java/io/trino/parquet/TestTupleDomainParquetPredicate.java b/lib/trino-parquet/src/test/java/io/trino/parquet/TestTupleDomainParquetPredicate.java index 66b58f7701146..04cefc95771c9 100644 --- a/lib/trino-parquet/src/test/java/io/trino/parquet/TestTupleDomainParquetPredicate.java +++ b/lib/trino-parquet/src/test/java/io/trino/parquet/TestTupleDomainParquetPredicate.java @@ -46,7 +46,7 @@ import org.apache.parquet.schema.LogicalTypeAnnotation.TimeUnit; import org.apache.parquet.schema.PrimitiveType; import org.apache.parquet.schema.Types; -import org.testng.annotations.Test; +import org.junit.jupiter.api.Test; import java.io.ByteArrayOutputStream; import java.math.BigInteger; diff --git a/lib/trino-parquet/src/test/java/io/trino/parquet/reader/flat/TestNullsDecoder.java b/lib/trino-parquet/src/test/java/io/trino/parquet/reader/flat/TestNullsDecoder.java index 6a33970b9f34a..ef420d9e8050e 100644 --- a/lib/trino-parquet/src/test/java/io/trino/parquet/reader/flat/TestNullsDecoder.java +++ b/lib/trino-parquet/src/test/java/io/trino/parquet/reader/flat/TestNullsDecoder.java @@ -16,7 +16,7 @@ import io.airlift.slice.Slices; import org.apache.parquet.bytes.HeapByteBufferAllocator; import org.apache.parquet.column.values.rle.RunLengthBitPackingHybridEncoder; -import org.testng.annotations.Test; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.util.Arrays; diff --git a/lib/trino-parquet/src/test/java/io/trino/parquet/writer/TestDefinitionLevelWriter.java b/lib/trino-parquet/src/test/java/io/trino/parquet/writer/TestDefinitionLevelWriter.java index 1bd3473a6cb89..5eda372804ef4 100644 --- a/lib/trino-parquet/src/test/java/io/trino/parquet/writer/TestDefinitionLevelWriter.java +++ b/lib/trino-parquet/src/test/java/io/trino/parquet/writer/TestDefinitionLevelWriter.java @@ -20,7 +20,7 @@ import io.trino.spi.block.ColumnarMap; import io.trino.spi.block.LongArrayBlock; import io.trino.spi.block.RowBlock; -import org.testng.annotations.Test; +import org.junit.jupiter.api.Test; import java.util.List; import java.util.Optional; diff --git a/lib/trino-parquet/src/test/java/io/trino/parquet/writer/TestRepetitionLevelWriter.java b/lib/trino-parquet/src/test/java/io/trino/parquet/writer/TestRepetitionLevelWriter.java index e4e4808bdfbef..40780a38a29ea 100644 --- a/lib/trino-parquet/src/test/java/io/trino/parquet/writer/TestRepetitionLevelWriter.java +++ b/lib/trino-parquet/src/test/java/io/trino/parquet/writer/TestRepetitionLevelWriter.java @@ -22,7 +22,7 @@ import io.trino.spi.block.RowBlock; import io.trino.spi.type.MapType; import io.trino.spi.type.TypeOperators; -import org.testng.annotations.Test; +import org.junit.jupiter.api.Test; import java.util.Iterator; import java.util.List; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveQlTranslation.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveQlTranslation.java index fccfa97c17e9c..51c8421522df5 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveQlTranslation.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveQlTranslation.java @@ -17,7 +17,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import io.trino.sql.parser.SqlParser; -import org.testng.annotations.Test; +import org.junit.jupiter.api.Test; import java.util.Collection; import java.util.List; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/coercions/TestDoubleToVarcharCoercions.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/coercions/TestDoubleToVarcharCoercions.java index 3101c3d9e0c68..61d94034222dd 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/coercions/TestDoubleToVarcharCoercions.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/coercions/TestDoubleToVarcharCoercions.java @@ -18,7 +18,7 @@ import io.trino.spi.TrinoException; import io.trino.spi.block.Block; import io.trino.spi.type.Type; -import org.testng.annotations.Test; +import org.junit.jupiter.api.Test; import static io.trino.plugin.hive.HiveTimestampPrecision.DEFAULT_PRECISION; import static io.trino.plugin.hive.HiveType.toHiveType; From 89b43deb536b5fac3a6888c77d7da6205e410938 Mon Sep 17 00:00:00 2001 From: Karol Sobczak Date: Fri, 8 Dec 2023 18:30:08 +0100 Subject: [PATCH 092/350] Support for 0 length copyRegion of dictionary last position checkValidRegion already validates that region is correct. Therefore copyRegion for position 42 and length 0 shoudn't fail for dictionary with 42 positions. --- .../src/main/java/io/trino/spi/block/DictionaryBlock.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/core/trino-spi/src/main/java/io/trino/spi/block/DictionaryBlock.java b/core/trino-spi/src/main/java/io/trino/spi/block/DictionaryBlock.java index 69c3902fd5e48..611eb63d5f894 100644 --- a/core/trino-spi/src/main/java/io/trino/spi/block/DictionaryBlock.java +++ b/core/trino-spi/src/main/java/io/trino/spi/block/DictionaryBlock.java @@ -420,6 +420,9 @@ public Block getRegion(int positionOffset, int length) public Block copyRegion(int position, int length) { checkValidRegion(positionCount, position, length); + if (length == 0) { + return dictionary.copyRegion(0, 0); + } // Avoid repeated volatile reads to the uniqueIds field int uniqueIds = this.uniqueIds; if (length <= 1 || (uniqueIds == dictionary.getPositionCount() && isSequentialIds)) { From 165f7d496ebda5e39168e59e9f9d85d8547dc1de Mon Sep 17 00:00:00 2001 From: Karol Sobczak Date: Sat, 9 Dec 2023 22:19:14 +0100 Subject: [PATCH 093/350] Add comment --- .../src/main/java/io/trino/spi/block/DictionaryBlock.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/trino-spi/src/main/java/io/trino/spi/block/DictionaryBlock.java b/core/trino-spi/src/main/java/io/trino/spi/block/DictionaryBlock.java index 611eb63d5f894..132aaca0761b1 100644 --- a/core/trino-spi/src/main/java/io/trino/spi/block/DictionaryBlock.java +++ b/core/trino-spi/src/main/java/io/trino/spi/block/DictionaryBlock.java @@ -421,6 +421,8 @@ public Block copyRegion(int position, int length) { checkValidRegion(positionCount, position, length); if (length == 0) { + // explicit support for case when length == 0 which might otherwise fail + // on getId(position) if position == positionCount return dictionary.copyRegion(0, 0); } // Avoid repeated volatile reads to the uniqueIds field From 976f82849125d32b0628dd595348ebbf11a97897 Mon Sep 17 00:00:00 2001 From: Marius Grama Date: Sun, 10 Dec 2023 13:08:17 +0100 Subject: [PATCH 094/350] Rename methods to correspond to the configuration setting --- .../main/java/io/trino/plugin/deltalake/DeltaLakeConfig.java | 4 ++-- .../io/trino/plugin/deltalake/DeltaLakeSessionProperties.java | 2 +- .../java/io/trino/plugin/deltalake/TestDeltaLakeConfig.java | 4 ++-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeConfig.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeConfig.java index 87ff4190d60fb..acef9be6c5228 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeConfig.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeConfig.java @@ -271,13 +271,13 @@ public long getDefaultCheckpointWritingInterval() return defaultCheckpointWritingInterval; } - public boolean isCheckpointPartitionFilterEnabled() + public boolean isCheckpointFilteringEnabled() { return checkpointFilteringEnabled; } @Config("delta.checkpoint-filtering.enabled") - public DeltaLakeConfig setCheckpointPartitionFilterEnabled(boolean checkpointFilteringEnabled) + public DeltaLakeConfig setCheckpointFilteringEnabled(boolean checkpointFilteringEnabled) { this.checkpointFilteringEnabled = checkpointFilteringEnabled; return this; diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSessionProperties.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSessionProperties.java index 8cfb02a3988ba..7b779e053a175 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSessionProperties.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSessionProperties.java @@ -201,7 +201,7 @@ public DeltaLakeSessionProperties( booleanProperty( CHECKPOINT_FILTERING_ENABLED, "Use filter in checkpoint reader", - deltaLakeConfig.isCheckpointPartitionFilterEnabled(), + deltaLakeConfig.isCheckpointFilteringEnabled(), false)); } diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeConfig.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeConfig.java index 4a6306dc524f8..dc42d2aba812c 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeConfig.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeConfig.java @@ -53,7 +53,7 @@ public void testDefaults() .setMaxPartitionsPerWriter(100) .setUnsafeWritesEnabled(false) .setDefaultCheckpointWritingInterval(10) - .setCheckpointPartitionFilterEnabled(false) + .setCheckpointFilteringEnabled(false) .setCheckpointRowStatisticsWritingEnabled(true) .setVacuumMinRetention(new Duration(7, DAYS)) .setHiveCatalogName(null) @@ -125,7 +125,7 @@ public void testExplicitPropertyMappings() .setUnsafeWritesEnabled(true) .setDefaultCheckpointWritingInterval(15) .setCheckpointRowStatisticsWritingEnabled(false) - .setCheckpointPartitionFilterEnabled(true) + .setCheckpointFilteringEnabled(true) .setVacuumMinRetention(new Duration(13, HOURS)) .setHiveCatalogName("hive") .setDynamicFilteringWaitTimeout(new Duration(30, MINUTES)) From 87d2cfc2f6b9fc3e6335d08c9cd8379ff2e07e5b Mon Sep 17 00:00:00 2001 From: praveenkrishna Date: Fri, 1 Dec 2023 16:37:16 +0530 Subject: [PATCH 095/350] Remove usage of MongoSession from Query table function --- .../io/trino/plugin/mongodb/ptf/Query.java | 23 ++++++++++--------- 1 file changed, 12 insertions(+), 11 deletions(-) diff --git a/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/ptf/Query.java b/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/ptf/Query.java index 413faba8d0653..23345ce3b297c 100644 --- a/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/ptf/Query.java +++ b/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/ptf/Query.java @@ -22,7 +22,6 @@ import io.trino.plugin.mongodb.MongoColumnHandle; import io.trino.plugin.mongodb.MongoMetadata; import io.trino.plugin.mongodb.MongoMetadataFactory; -import io.trino.plugin.mongodb.MongoSession; import io.trino.plugin.mongodb.MongoTableHandle; import io.trino.plugin.mongodb.RemoteTableName; import io.trino.spi.TrinoException; @@ -34,6 +33,7 @@ import io.trino.spi.connector.ConnectorTableSchema; import io.trino.spi.connector.ConnectorTransactionHandle; import io.trino.spi.connector.SchemaTableName; +import io.trino.spi.connector.TableNotFoundException; import io.trino.spi.function.table.AbstractConnectorTableFunction; import io.trino.spi.function.table.Argument; import io.trino.spi.function.table.ConnectorTableFunction; @@ -53,6 +53,7 @@ import static io.trino.spi.StandardErrorCode.INVALID_FUNCTION_ARGUMENT; import static io.trino.spi.function.table.ReturnTypeSpecification.GenericTable.GENERIC_TABLE; import static io.trino.spi.type.VarcharType.VARCHAR; +import static java.lang.String.format; import static java.util.Locale.ENGLISH; import static java.util.Objects.requireNonNull; @@ -63,29 +64,25 @@ public class Query public static final String NAME = "query"; private final MongoMetadata metadata; - private final MongoSession session; @Inject - public Query(MongoMetadataFactory mongoMetadataFactory, MongoSession session) + public Query(MongoMetadataFactory mongoMetadataFactory) { - requireNonNull(session, "session is null"); this.metadata = mongoMetadataFactory.create(); - this.session = session; } @Override public ConnectorTableFunction get() { - return new QueryFunction(metadata, session); + return new QueryFunction(metadata); } public static class QueryFunction extends AbstractConnectorTableFunction { private final MongoMetadata metadata; - private final MongoSession mongoSession; - public QueryFunction(MongoMetadata metadata, MongoSession mongoSession) + public QueryFunction(MongoMetadata metadata) { super( SCHEMA_NAME, @@ -105,7 +102,6 @@ public QueryFunction(MongoMetadata metadata, MongoSession mongoSession) .build()), GENERIC_TABLE); this.metadata = requireNonNull(metadata, "metadata is null"); - this.mongoSession = requireNonNull(mongoSession, "mongoSession is null"); } @Override @@ -124,11 +120,16 @@ public TableFunctionAnalysis analyze( if (!collection.equals(collection.toLowerCase(ENGLISH))) { throw new TrinoException(INVALID_FUNCTION_ARGUMENT, "Only lowercase collection name is supported"); } - RemoteTableName remoteTableName = mongoSession.toRemoteSchemaTableName(new SchemaTableName(database, collection)); + SchemaTableName schemaTableName = new SchemaTableName(database, collection); + MongoTableHandle tableHandle = metadata.getTableHandle(session, schemaTableName); + if (tableHandle == null) { + throw new TableNotFoundException(schemaTableName, format("Table '%s.%s' not found", database, collection), null); + } + RemoteTableName remoteTableName = metadata.getTableHandle(session, new SchemaTableName(database, collection)).getRemoteTableName(); // Don't store Document object to MongoTableHandle for avoiding serialization issue parseFilter(filter); - MongoTableHandle tableHandle = new MongoTableHandle(new SchemaTableName(database, collection), remoteTableName, Optional.of(filter)); + tableHandle = new MongoTableHandle(new SchemaTableName(database, collection), remoteTableName, Optional.of(filter)); ConnectorTableSchema tableSchema = metadata.getTableSchema(session, tableHandle); Map columnsByName = metadata.getColumnHandles(session, tableHandle); List columns = tableSchema.getColumns().stream() From 01a76bfb249048ee95a4803263a484ec26fd3d89 Mon Sep 17 00:00:00 2001 From: praveenkrishna Date: Fri, 1 Dec 2023 16:59:39 +0530 Subject: [PATCH 096/350] Introduce MongoTransactionManager --- .../plugin/mongodb/MongoClientModule.java | 1 + .../trino/plugin/mongodb/MongoConnector.java | 28 ++------ .../mongodb/MongoTransactionManager.java | 65 +++++++++++++++++++ 3 files changed, 73 insertions(+), 21 deletions(-) create mode 100644 plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/MongoTransactionManager.java diff --git a/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/MongoClientModule.java b/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/MongoClientModule.java index f856c7d21f1ae..7251965bcb825 100644 --- a/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/MongoClientModule.java +++ b/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/MongoClientModule.java @@ -45,6 +45,7 @@ public class MongoClientModule public void setup(Binder binder) { binder.bind(MongoConnector.class).in(Scopes.SINGLETON); + binder.bind(MongoTransactionManager.class).in(Scopes.SINGLETON); binder.bind(MongoSplitManager.class).in(Scopes.SINGLETON); binder.bind(MongoPageSourceProvider.class).in(Scopes.SINGLETON); binder.bind(MongoPageSinkProvider.class).in(Scopes.SINGLETON); diff --git a/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/MongoConnector.java b/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/MongoConnector.java index 03e16a0554e40..b90901e3d1147 100644 --- a/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/MongoConnector.java +++ b/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/MongoConnector.java @@ -29,43 +29,36 @@ import java.util.List; import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.collect.ImmutableList.toImmutableList; -import static io.trino.spi.transaction.IsolationLevel.READ_UNCOMMITTED; -import static io.trino.spi.transaction.IsolationLevel.checkConnectorSupports; import static java.util.Objects.requireNonNull; public class MongoConnector implements Connector { private final MongoSession mongoSession; + private final MongoTransactionManager transactionManager; private final MongoSplitManager splitManager; private final MongoPageSourceProvider pageSourceProvider; private final MongoPageSinkProvider pageSinkProvider; - private final MongoMetadataFactory mongoMetadataFactory; private final Set connectorTableFunctions; private final List> sessionProperties; - private final ConcurrentMap transactions = new ConcurrentHashMap<>(); - @Inject public MongoConnector( MongoSession mongoSession, + MongoTransactionManager transactionManager, MongoSplitManager splitManager, MongoPageSourceProvider pageSourceProvider, MongoPageSinkProvider pageSinkProvider, - MongoMetadataFactory mongoMetadataFactory, Set connectorTableFunctions, Set sessionPropertiesProviders) { this.mongoSession = mongoSession; + this.transactionManager = requireNonNull(transactionManager, "transactionManager is null"); this.splitManager = requireNonNull(splitManager, "splitManager is null"); this.pageSourceProvider = requireNonNull(pageSourceProvider, "pageSourceProvider is null"); this.pageSinkProvider = requireNonNull(pageSinkProvider, "pageSinkProvider is null"); - this.mongoMetadataFactory = requireNonNull(mongoMetadataFactory, "mongoMetadataFactory is null"); this.connectorTableFunctions = ImmutableSet.copyOf(requireNonNull(connectorTableFunctions, "connectorTableFunctions is null")); this.sessionProperties = sessionPropertiesProviders.stream() .flatMap(sessionPropertiesProvider -> sessionPropertiesProvider.getSessionProperties().stream()) @@ -75,32 +68,25 @@ public MongoConnector( @Override public ConnectorTransactionHandle beginTransaction(IsolationLevel isolationLevel, boolean readOnly, boolean autoCommit) { - checkConnectorSupports(READ_UNCOMMITTED, isolationLevel); - MongoTransactionHandle transaction = new MongoTransactionHandle(); - transactions.put(transaction, mongoMetadataFactory.create()); - return transaction; + return transactionManager.beginTransaction(isolationLevel); } @Override public ConnectorMetadata getMetadata(ConnectorSession session, ConnectorTransactionHandle transaction) { - MongoMetadata metadata = transactions.get(transaction); - checkArgument(metadata != null, "no such transaction: %s", transaction); - return metadata; + return transactionManager.getMetadata(transaction); } @Override public void commit(ConnectorTransactionHandle transaction) { - checkArgument(transactions.remove(transaction) != null, "no such transaction: %s", transaction); + transactionManager.commit(transaction); } @Override public void rollback(ConnectorTransactionHandle transaction) { - MongoMetadata metadata = transactions.remove(transaction); - checkArgument(metadata != null, "no such transaction: %s", transaction); - metadata.rollback(); + transactionManager.rollback(transaction); } @Override diff --git a/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/MongoTransactionManager.java b/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/MongoTransactionManager.java new file mode 100644 index 0000000000000..b2f074f19e1a6 --- /dev/null +++ b/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/MongoTransactionManager.java @@ -0,0 +1,65 @@ +/* + * 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.mongodb; + +import com.google.inject.Inject; +import io.trino.spi.connector.ConnectorTransactionHandle; +import io.trino.spi.transaction.IsolationLevel; + +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +import static com.google.common.base.Preconditions.checkArgument; +import static io.trino.spi.transaction.IsolationLevel.READ_COMMITTED; +import static io.trino.spi.transaction.IsolationLevel.checkConnectorSupports; +import static java.util.Objects.requireNonNull; + +public class MongoTransactionManager +{ + private final ConcurrentMap transactions = new ConcurrentHashMap<>(); + private final MongoMetadataFactory metadataFactory; + + @Inject + public MongoTransactionManager(MongoMetadataFactory metadataFactory) + { + this.metadataFactory = requireNonNull(metadataFactory, "metadataFactory is null"); + } + + public ConnectorTransactionHandle beginTransaction(IsolationLevel isolationLevel) + { + checkConnectorSupports(READ_COMMITTED, isolationLevel); + MongoTransactionHandle transaction = new MongoTransactionHandle(); + transactions.put(transaction, metadataFactory.create()); + return transaction; + } + + public MongoMetadata getMetadata(ConnectorTransactionHandle transaction) + { + MongoMetadata metadata = transactions.get(transaction); + checkArgument(metadata != null, "no such transaction: %s", transaction); + return metadata; + } + + public void commit(ConnectorTransactionHandle transaction) + { + checkArgument(transactions.remove(transaction) != null, "no such transaction: %s", transaction); + } + + public void rollback(ConnectorTransactionHandle transaction) + { + MongoMetadata metadata = transactions.remove(transaction); + checkArgument(metadata != null, "no such transaction: %s", transaction); + metadata.rollback(); + } +} From 0aa1f1b3550ec247da9b9b8e45bb8434302e1f8f Mon Sep 17 00:00:00 2001 From: praveenkrishna Date: Fri, 1 Dec 2023 17:00:10 +0530 Subject: [PATCH 097/350] Use MongoTransactionManager in Query table function --- .../io/trino/plugin/mongodb/ptf/Query.java | 25 ++++++++++--------- 1 file changed, 13 insertions(+), 12 deletions(-) diff --git a/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/ptf/Query.java b/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/ptf/Query.java index 23345ce3b297c..1634cd0b8e97a 100644 --- a/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/ptf/Query.java +++ b/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/ptf/Query.java @@ -21,8 +21,8 @@ import io.airlift.slice.Slice; import io.trino.plugin.mongodb.MongoColumnHandle; import io.trino.plugin.mongodb.MongoMetadata; -import io.trino.plugin.mongodb.MongoMetadataFactory; import io.trino.plugin.mongodb.MongoTableHandle; +import io.trino.plugin.mongodb.MongoTransactionManager; import io.trino.plugin.mongodb.RemoteTableName; import io.trino.spi.TrinoException; import io.trino.spi.connector.ColumnHandle; @@ -53,7 +53,6 @@ import static io.trino.spi.StandardErrorCode.INVALID_FUNCTION_ARGUMENT; import static io.trino.spi.function.table.ReturnTypeSpecification.GenericTable.GENERIC_TABLE; import static io.trino.spi.type.VarcharType.VARCHAR; -import static java.lang.String.format; import static java.util.Locale.ENGLISH; import static java.util.Objects.requireNonNull; @@ -63,26 +62,26 @@ public class Query public static final String SCHEMA_NAME = "system"; public static final String NAME = "query"; - private final MongoMetadata metadata; + private final MongoTransactionManager transactionManager; @Inject - public Query(MongoMetadataFactory mongoMetadataFactory) + public Query(MongoTransactionManager transactionManager) { - this.metadata = mongoMetadataFactory.create(); + this.transactionManager = requireNonNull(transactionManager, "transactionManager is null"); } @Override public ConnectorTableFunction get() { - return new QueryFunction(metadata); + return new QueryFunction(transactionManager); } public static class QueryFunction extends AbstractConnectorTableFunction { - private final MongoMetadata metadata; + private final MongoTransactionManager transactionManager; - public QueryFunction(MongoMetadata metadata) + public QueryFunction(MongoTransactionManager transactionManager) { super( SCHEMA_NAME, @@ -101,7 +100,7 @@ public QueryFunction(MongoMetadata metadata) .type(VARCHAR) .build()), GENERIC_TABLE); - this.metadata = requireNonNull(metadata, "metadata is null"); + this.transactionManager = requireNonNull(transactionManager, "transactionManager is null"); } @Override @@ -111,6 +110,7 @@ public TableFunctionAnalysis analyze( Map arguments, ConnectorAccessControl accessControl) { + MongoMetadata metadata = transactionManager.getMetadata(transaction); String database = ((Slice) ((ScalarArgument) arguments.get("DATABASE")).getValue()).toStringUtf8(); String collection = ((Slice) ((ScalarArgument) arguments.get("COLLECTION")).getValue()).toStringUtf8(); String filter = ((Slice) ((ScalarArgument) arguments.get("FILTER")).getValue()).toStringUtf8(); @@ -123,13 +123,14 @@ public TableFunctionAnalysis analyze( SchemaTableName schemaTableName = new SchemaTableName(database, collection); MongoTableHandle tableHandle = metadata.getTableHandle(session, schemaTableName); if (tableHandle == null) { - throw new TableNotFoundException(schemaTableName, format("Table '%s.%s' not found", database, collection), null); + throw new TableNotFoundException(schemaTableName); } - RemoteTableName remoteTableName = metadata.getTableHandle(session, new SchemaTableName(database, collection)).getRemoteTableName(); + + RemoteTableName remoteTableName = tableHandle.getRemoteTableName(); // Don't store Document object to MongoTableHandle for avoiding serialization issue parseFilter(filter); - tableHandle = new MongoTableHandle(new SchemaTableName(database, collection), remoteTableName, Optional.of(filter)); + tableHandle = new MongoTableHandle(schemaTableName, remoteTableName, Optional.of(filter)); ConnectorTableSchema tableSchema = metadata.getTableSchema(session, tableHandle); Map columnsByName = metadata.getColumnHandles(session, tableHandle); List columns = tableSchema.getColumns().stream() From f0d6610be77d62ec344fea5df5b9321f7bc69a10 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Mon, 11 Dec 2023 17:45:16 +0100 Subject: [PATCH 098/350] Respect Glue partition column type with CSV When using CSV serde, data columns are known to be Hive string, irrespective of what metastore says about that. However, the partitioning columns are whatever type they are declared with. --- .../glue/converter/GlueToTrinoConverter.java | 24 ++++++++++++------- 1 file changed, 15 insertions(+), 9 deletions(-) diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/converter/GlueToTrinoConverter.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/converter/GlueToTrinoConverter.java index 0e0f9b8707d1f..68d3a6a145902 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/converter/GlueToTrinoConverter.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/converter/GlueToTrinoConverter.java @@ -162,9 +162,9 @@ public static Table convertTable(com.amazonaws.services.glue.model.Table glueTab throw new TrinoException(HIVE_UNSUPPORTED_FORMAT, "Table StorageDescriptor is null for table '%s' %s".formatted(table, glueTable)); } boolean isCsv = sd.getSerdeInfo() != null && HiveStorageFormat.CSV.getSerde().equals(sd.getSerdeInfo().getSerializationLibrary()); - tableBuilder.setDataColumns(convertColumns(table, sd.getColumns(), isCsv)); + tableBuilder.setDataColumns(convertColumns(table, sd.getColumns(), ColumnType.DATA, isCsv)); if (glueTable.getPartitionKeys() != null) { - tableBuilder.setPartitionColumns(convertColumns(table, glueTable.getPartitionKeys(), isCsv)); + tableBuilder.setPartitionColumns(convertColumns(table, glueTable.getPartitionKeys(), ColumnType.PARTITION, isCsv)); } else { tableBuilder.setPartitionColumns(ImmutableList.of()); @@ -176,11 +176,11 @@ public static Table convertTable(com.amazonaws.services.glue.model.Table glueTab return tableBuilder.build(); } - private static Column convertColumn(SchemaTableName table, com.amazonaws.services.glue.model.Column glueColumn, boolean isCsv) + private static Column convertColumn(SchemaTableName table, com.amazonaws.services.glue.model.Column glueColumn, ColumnType columnType, boolean isCsv) { // OpenCSVSerde deserializes columns from csv file into strings, so we set the column type from the metastore // to string to avoid cast exceptions. - if (isCsv) { + if (columnType == ColumnType.DATA && isCsv) { //TODO(https://github.com/trinodb/trino/issues/7240) Add tests return new Column(glueColumn.getName(), HiveType.HIVE_STRING, Optional.ofNullable(glueColumn.getComment()), getColumnParameters(glueColumn)); } @@ -197,9 +197,9 @@ private static HiveType convertType(SchemaTableName table, com.amazonaws.service } } - private static List convertColumns(SchemaTableName table, List glueColumns, boolean isCsv) + private static List convertColumns(SchemaTableName table, List glueColumns, ColumnType columnType, boolean isCsv) { - return mappedCopy(glueColumns, glueColumn -> convertColumn(table, glueColumn, isCsv)); + return mappedCopy(glueColumns, glueColumn -> convertColumn(table, glueColumn, columnType, isCsv)); } private static Function, Map> parametersConverter() @@ -215,7 +215,7 @@ private static boolean isNullOrEmpty(List list) public static final class GluePartitionConverter implements Function { - private final BiFunction, Boolean, List> columnsConverter; + private final BiFunction, Boolean, List> dataColumnsConverter; private final Function, Map> parametersConverter = parametersConverter(); private final StorageConverter storageConverter = new StorageConverter(); private final String databaseName; @@ -228,7 +228,7 @@ public GluePartitionConverter(Table table) this.databaseName = requireNonNull(table.getDatabaseName(), "databaseName is null"); this.tableName = requireNonNull(table.getTableName(), "tableName is null"); this.tableParameters = table.getParameters(); - this.columnsConverter = memoizeLast((glueColumns, isCsv) -> convertColumns(table.getSchemaTableName(), glueColumns, isCsv)); + this.dataColumnsConverter = memoizeLast((glueColumns, isCsv) -> convertColumns(table.getSchemaTableName(), glueColumns, ColumnType.DATA, isCsv)); } @Override @@ -248,7 +248,7 @@ public Partition apply(com.amazonaws.services.glue.model.Partition gluePartition .setDatabaseName(databaseName) .setTableName(tableName) .setValues(gluePartition.getValues()) // No memoization benefit - .setColumns(columnsConverter.apply(sd.getColumns(), isCsv)) + .setColumns(dataColumnsConverter.apply(sd.getColumns(), isCsv)) .setParameters(parametersConverter.apply(getPartitionParameters(gluePartition))); storageConverter.setStorageBuilder(sd, partitionBuilder.getStorageBuilder(), tableParameters); @@ -347,4 +347,10 @@ public static List mappedCopy(List list, Function mapper) } return builder.build(); } + + private enum ColumnType + { + DATA, + PARTITION, + } } From 8e46a22189e5128e94c33eb6dc928b8468057c22 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jan=20Wa=C5=9B?= Date: Fri, 8 Dec 2023 17:51:51 +0100 Subject: [PATCH 099/350] Add example SQL routine for truncating strings --- docs/src/main/sphinx/routines/examples.md | 93 ++++++++++++++++++++++- 1 file changed, 91 insertions(+), 2 deletions(-) diff --git a/docs/src/main/sphinx/routines/examples.md b/docs/src/main/sphinx/routines/examples.md index 4e5c9d70dac2c..9d7fdd14680a3 100644 --- a/docs/src/main/sphinx/routines/examples.md +++ b/docs/src/main/sphinx/routines/examples.md @@ -377,11 +377,11 @@ TIME ZONE`. Date strings are commonly represented by ISO 8601 standard, such as `2023-12-01`, `2023-12-01T23`. Date strings are also often represented in the `YYYYmmdd` and `YYYYmmddHH` format, such as `20230101` and `2023010123`. Hive tables can use this format to represent day and hourly partitions, for example -`/day=20230101`, `/hour=2023010123`. +`/day=20230101`, `/hour=2023010123`. This routine parses date strings in a best-effort fashion and can be used as a replacement for date string manipulation functions such as `date`, `date_parse`, -`from_iso8601_date`, and `from_iso8601_timestamp`. +`from_iso8601_date`, and `from_iso8601_timestamp`. Note that the routine defaults the time value to `00:00:00.000` and the time zone to the session time zone. @@ -413,3 +413,92 @@ SELECT from_date_string('2023010123'); -- 2023-01-01 23:00:00.000 UTC (using the SELECT from_date_string(NULL); -- NULL (handles NULL string) SELECT from_date_string('abc'); -- NULL (not matched to any format) ``` + +## Truncating long strings + +This example routine `strtrunc` truncates strings longer than 60 characters, +leaving the first 30 and the last 25 characters, and cutting out extra +characters in the middle. + +```sql +FUNCTION strtrunc(input VARCHAR) +RETURNS VARCHAR +RETURN + CASE WHEN length(input) > 60 + THEN substr(input, 1, 30) || ' ... ' || substr(input, length(input) - 25) + ELSE input + END; +``` + +The preceding declaration is very compact and consists of only one complex +statement with a [`CASE` expression](case-expression) and multiple function +calls. It can therefore define the complete logic in the `RETURN` clause. + +The following statement shows the same capability within the routine itself. +Note the duplicate `RETURN` inside and outside the `CASE` statement and the +required `END CASE;`. The second `RETURN` statement is required, because a +routine must end with a `RETURN` statement. As a result the `ELSE` clause can be +omitted. + +```sql +FUNCTION strtrunc(input VARCHAR) +RETURNS VARCHAR +BEGIN + CASE WHEN length(input) > 60 + THEN + RETURN substr(input, 1, 30) || ' ... ' || substr(input, length(input) - 25); + ELSE + RETURN input; + END CASE; + RETURN input; +END; +``` + +The next example changes over from a `CASE` to an `IF` statement, and avoids the +duplicate `RETURN`: + +```sql +FUNCTION strtrunc(input VARCHAR) +RETURNS VARCHAR +BEGIN + IF length(input) > 60 THEN + RETURN substr(input, 1, 30) || ' ... ' || substr(input, length(input) - 25); + END IF; + RETURN input; +END; +``` + +All the preceding examples create the same output. Following is an example query +which generates long strings to truncate: + +```sql +WITH +data AS ( + SELECT substring('strtrunc truncates strings longer than 60 characters, leaving the prefix and suffix visible', 1, s.num) AS value + FROM table(sequence(start=>40, stop=>80, step=>5)) AS s(num) +) +SELECT + data.value + , strtrunc(data.value) AS truncated +FROM data +ORDER BY data.value; +``` + +The preceding query produces the following output with all variants of the +routine: + +``` + value | truncated +----------------------------------------------------------------------------------+--------------------------------------------------------------- + strtrunc truncates strings longer than 6 | strtrunc truncates strings longer than 6 + strtrunc truncates strings longer than 60 cha | strtrunc truncates strings longer than 60 cha + strtrunc truncates strings longer than 60 characte | strtrunc truncates strings longer than 60 characte + strtrunc truncates strings longer than 60 characters, l | strtrunc truncates strings longer than 60 characters, l + strtrunc truncates strings longer than 60 characters, leavin | strtrunc truncates strings longer than 60 characters, leavin + strtrunc truncates strings longer than 60 characters, leaving the | strtrunc truncates strings lon ... 60 characters, leaving the + strtrunc truncates strings longer than 60 characters, leaving the pref | strtrunc truncates strings lon ... aracters, leaving the pref + strtrunc truncates strings longer than 60 characters, leaving the prefix an | strtrunc truncates strings lon ... ers, leaving the prefix an + strtrunc truncates strings longer than 60 characters, leaving the prefix and suf | strtrunc truncates strings lon ... leaving the prefix and suf +``` + +A possible improvement is to introduce parameters for the total length. From df48f12ca892fb4274eed8d7e9adb9d73d13ed81 Mon Sep 17 00:00:00 2001 From: Manfred Moser Date: Mon, 11 Dec 2023 16:15:12 -0800 Subject: [PATCH 100/350] Add links for expressions and statements - To compare and contrast - IF - CASE - Add missing link for case in list page --- docs/src/main/sphinx/functions/conditional.md | 10 ++++++++-- docs/src/main/sphinx/functions/list-by-topic.md | 9 +++++---- docs/src/main/sphinx/functions/list.md | 2 +- docs/src/main/sphinx/routines/case.md | 1 + docs/src/main/sphinx/routines/if.md | 1 + 5 files changed, 16 insertions(+), 7 deletions(-) diff --git a/docs/src/main/sphinx/functions/conditional.md b/docs/src/main/sphinx/functions/conditional.md index 3e07ed29347c3..4ed0716aa6645 100644 --- a/docs/src/main/sphinx/functions/conditional.md +++ b/docs/src/main/sphinx/functions/conditional.md @@ -1,7 +1,6 @@ # Conditional expressions (case-expression)= - ## CASE The standard SQL `CASE` expression has two forms. @@ -52,8 +51,11 @@ SELECT a, b, END ``` -(if-function)= +SQL routines can use [`CASE` statements](/routines/case) that use a slightly +different syntax from the CASE expressions. Specifically note the requirements +for terminating each clause with a semicolon `;` and the usage of `END CASE`. +(if-expression)= ## IF The `IF` expression has two forms, one supplying only a @@ -93,6 +95,10 @@ SELECT FROM tpch.sf1.orders; ``` +SQL routines can use [`IF` statements](/routines/if) that use a slightly +different syntax from `IF` expressions. Specifically note the requirement +for terminating each clause with a semicolon `;` and the usage of `END IF`. + (coalesce-function)= ## COALESCE diff --git a/docs/src/main/sphinx/functions/list-by-topic.md b/docs/src/main/sphinx/functions/list-by-topic.md index d55aafbcba6bc..20e4b1d15e491 100644 --- a/docs/src/main/sphinx/functions/list-by-topic.md +++ b/docs/src/main/sphinx/functions/list-by-topic.md @@ -158,10 +158,11 @@ For more details, see {doc}`comparison` For more details, see {doc}`conditional` -- {ref}`coalesce ` -- {ref}`if ` -- {ref}`nullif ` -- {ref}`try ` +- [case](case-expression) +- [coalesce](coalesce-function) +- [if](if-expression) +- [nullif](nullif-function) +- [try](try-function) ## Conversion diff --git a/docs/src/main/sphinx/functions/list.md b/docs/src/main/sphinx/functions/list.md index e69699448ed77..823c252821fdd 100644 --- a/docs/src/main/sphinx/functions/list.md +++ b/docs/src/main/sphinx/functions/list.md @@ -200,7 +200,7 @@ ## I -- [if](if-function) +- [if](if-expression) - {func}`index` - {func}`infinity` - {func}`intersection_cardinality` diff --git a/docs/src/main/sphinx/routines/case.md b/docs/src/main/sphinx/routines/case.md index 926c1df925f3d..f7264d08b0962 100644 --- a/docs/src/main/sphinx/routines/case.md +++ b/docs/src/main/sphinx/routines/case.md @@ -60,3 +60,4 @@ documentation](/routines/examples). ## See also * [](/routines/introduction) +* [Conditional expressions using `CASE`](case-expression) diff --git a/docs/src/main/sphinx/routines/if.md b/docs/src/main/sphinx/routines/if.md index a02c9e659dc25..264beec66b522 100644 --- a/docs/src/main/sphinx/routines/if.md +++ b/docs/src/main/sphinx/routines/if.md @@ -45,3 +45,4 @@ documentation](/routines/examples). ## See also * [](/routines/introduction) +* [Conditional expressions using `IF`](if-expression) From 33dc08f957c98c2ce7489bfaaf4bd38b0f1823a3 Mon Sep 17 00:00:00 2001 From: Marius Grama Date: Mon, 23 Oct 2023 15:26:21 +0200 Subject: [PATCH 101/350] Add date to varchar coercion for hive --- docs/src/main/sphinx/connector/hive.md | 2 + .../plugin/hive/coercions/CoercionUtils.java | 4 ++ .../plugin/hive/coercions/DateCoercer.java | 35 +++++++++++++++++ .../plugin/hive/orc/OrcTypeTranslator.java | 5 +++ .../plugin/hive/util/HiveCoercionPolicy.java | 1 + .../hive/coercions/TestDateCoercer.java | 39 +++++++++++++++++-- .../product/hive/BaseTestHiveCoercion.java | 18 +++++++++ .../TestHiveCoercionOnPartitionedTable.java | 2 + .../TestHiveCoercionOnUnpartitionedTable.java | 2 + 9 files changed, 105 insertions(+), 3 deletions(-) diff --git a/docs/src/main/sphinx/connector/hive.md b/docs/src/main/sphinx/connector/hive.md index 2f051615a43e3..9982e5d5c3148 100644 --- a/docs/src/main/sphinx/connector/hive.md +++ b/docs/src/main/sphinx/connector/hive.md @@ -658,6 +658,8 @@ type conversions. * - `DECIMAL` - `DOUBLE`, `REAL`, `VARCHAR`, `TINYINT`, `SMALLINT`, `INTEGER`, `BIGINT`, as well as narrowing and widening conversions for `DECIMAL` +* - `DATE` + - `VARCHAR` * - `TIMESTAMP` - `VARCHAR`, `DATE` ::: diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/coercions/CoercionUtils.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/coercions/CoercionUtils.java index aa530a32e890c..fcbe2aa6fc257 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/coercions/CoercionUtils.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/coercions/CoercionUtils.java @@ -17,6 +17,7 @@ import io.trino.plugin.hive.HiveTimestampPrecision; import io.trino.plugin.hive.HiveType; import io.trino.plugin.hive.coercions.BooleanCoercer.BooleanToVarcharCoercer; +import io.trino.plugin.hive.coercions.DateCoercer.DateToVarcharCoercer; import io.trino.plugin.hive.coercions.DateCoercer.VarcharToDateCoercer; import io.trino.plugin.hive.coercions.TimestampCoercer.LongTimestampToDateCoercer; import io.trino.plugin.hive.coercions.TimestampCoercer.LongTimestampToVarcharCoercer; @@ -214,6 +215,9 @@ public static Type createTypeFromCoercer(TypeManager typeManager, HiveType fromH } return Optional.empty(); } + if (fromType instanceof DateType && toType instanceof VarcharType toVarcharType) { + return Optional.of(new DateToVarcharCoercer(toVarcharType)); + } if (fromType == DOUBLE && toType instanceof VarcharType toVarcharType) { return Optional.of(new DoubleToVarcharCoercer(toVarcharType, coercionContext.treatNaNAsNull())); } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/coercions/DateCoercer.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/coercions/DateCoercer.java index 3b9398a9e9253..752f72ac7ceb4 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/coercions/DateCoercer.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/coercions/DateCoercer.java @@ -13,16 +13,23 @@ */ package io.trino.plugin.hive.coercions; +import io.airlift.slice.Slice; import io.trino.spi.TrinoException; import io.trino.spi.block.Block; import io.trino.spi.block.BlockBuilder; import io.trino.spi.type.DateType; import io.trino.spi.type.VarcharType; +import java.time.DateTimeException; import java.time.LocalDate; import java.time.format.DateTimeParseException; +import static io.airlift.slice.SliceUtf8.countCodePoints; +import static io.airlift.slice.Slices.utf8Slice; import static io.trino.plugin.hive.HiveErrorCode.HIVE_INVALID_TIMESTAMP_COERCION; +import static io.trino.spi.StandardErrorCode.INVALID_ARGUMENTS; +import static io.trino.spi.type.DateType.DATE; +import static java.lang.String.format; import static java.time.format.DateTimeFormatter.ISO_LOCAL_DATE; public final class DateCoercer @@ -55,4 +62,32 @@ protected void applyCoercedValue(BlockBuilder blockBuilder, Block block, int pos } } } + + public static class DateToVarcharCoercer + extends TypeCoercer + { + public DateToVarcharCoercer(VarcharType toType) + { + super(DATE, toType); + } + + @Override + protected void applyCoercedValue(BlockBuilder blockBuilder, Block block, int position) + { + int value = fromType.getInt(block, position); + try { + if (value < START_OF_MODERN_ERA_DAYS) { + throw new TrinoException(HIVE_INVALID_TIMESTAMP_COERCION, "Coercion on historical dates is not supported"); + } + Slice converted = utf8Slice(ISO_LOCAL_DATE.format(LocalDate.ofEpochDay(value))); + if (!toType.isUnbounded() && countCodePoints(converted) > toType.getBoundedLength()) { + throw new TrinoException(INVALID_ARGUMENTS, format("Varchar representation of '%s' exceeds %s bounds", converted.toStringUtf8(), toType)); + } + toType.writeSlice(blockBuilder, converted); + } + catch (DateTimeException ignored) { + throw new IllegalArgumentException("Invalid date value: " + value + " is exceeding supported date range"); + } + } + } } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/orc/OrcTypeTranslator.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/orc/OrcTypeTranslator.java index d0f698012e0a0..97b117b7645e6 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/orc/OrcTypeTranslator.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/orc/OrcTypeTranslator.java @@ -15,6 +15,7 @@ import io.trino.orc.metadata.OrcType.OrcTypeKind; import io.trino.plugin.hive.coercions.BooleanCoercer.BooleanToVarcharCoercer; +import io.trino.plugin.hive.coercions.DateCoercer.DateToVarcharCoercer; import io.trino.plugin.hive.coercions.DateCoercer.VarcharToDateCoercer; import io.trino.plugin.hive.coercions.DoubleToVarcharCoercer; import io.trino.plugin.hive.coercions.IntegerNumberToDoubleCoercer; @@ -35,6 +36,7 @@ import static io.trino.orc.metadata.OrcType.OrcTypeKind.BOOLEAN; import static io.trino.orc.metadata.OrcType.OrcTypeKind.BYTE; +import static io.trino.orc.metadata.OrcType.OrcTypeKind.DATE; import static io.trino.orc.metadata.OrcType.OrcTypeKind.DOUBLE; import static io.trino.orc.metadata.OrcType.OrcTypeKind.INT; import static io.trino.orc.metadata.OrcType.OrcTypeKind.LONG; @@ -65,6 +67,9 @@ private OrcTypeTranslator() {} } return Optional.empty(); } + if (fromOrcType == DATE && toTrinoType instanceof VarcharType varcharType) { + return Optional.of(new DateToVarcharCoercer(varcharType)); + } if (isVarcharType(fromOrcType)) { if (toTrinoType instanceof TimestampType timestampType) { if (timestampType.isShort()) { diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/HiveCoercionPolicy.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/HiveCoercionPolicy.java index 28f22757c2df5..0af514ae19a7f 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/HiveCoercionPolicy.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/HiveCoercionPolicy.java @@ -81,6 +81,7 @@ private boolean canCoerce(HiveType fromHiveType, HiveType toHiveType, HiveTimest fromHiveType.equals(HIVE_LONG) || fromHiveType.equals(HIVE_TIMESTAMP) || fromHiveType.equals(HIVE_DOUBLE) || + fromHiveType.equals(HIVE_DATE) || fromType instanceof DecimalType; } if (toHiveType.equals(HIVE_DATE)) { diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/coercions/TestDateCoercer.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/coercions/TestDateCoercer.java index 581fb2dbb2596..37a86dad7803f 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/coercions/TestDateCoercer.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/coercions/TestDateCoercer.java @@ -14,8 +14,8 @@ package io.trino.plugin.hive.coercions; import io.trino.plugin.hive.coercions.CoercionUtils.CoercionContext; +import io.trino.spi.TrinoException; import io.trino.spi.block.Block; -import io.trino.spi.type.DateType; import io.trino.spi.type.Type; import org.junit.jupiter.api.Test; @@ -27,7 +27,10 @@ import static io.trino.plugin.hive.coercions.CoercionUtils.createCoercer; import static io.trino.spi.predicate.Utils.blockToNativeValue; import static io.trino.spi.predicate.Utils.nativeValueToBlock; +import static io.trino.spi.type.DateType.DATE; +import static io.trino.spi.type.VarcharType.VARCHAR; import static io.trino.spi.type.VarcharType.createUnboundedVarcharType; +import static io.trino.spi.type.VarcharType.createVarcharType; import static io.trino.type.InternalTypeManager.TESTING_TYPE_MANAGER; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -66,6 +69,28 @@ public void testThrowsExceptionWhenDateIsTooOld() .hasMessageMatching(".*Coercion on historical dates is not supported.*"); } + @Test + public void testDateToVarchar() + { + assertDateToVarcharCoercion(createUnboundedVarcharType(), LocalDate.parse("2023-01-10"), "2023-01-10"); + assertDateToVarcharCoercion(createUnboundedVarcharType(), LocalDate.parse("+10000-04-25"), "+10000-04-25"); + } + + @Test + public void testDateToLowerBoundedVarchar() + { + assertThatThrownBy(() -> assertDateToVarcharCoercion(createVarcharType(8), LocalDate.parse("2023-10-23"), "2023-10-23")) + .isInstanceOf(TrinoException.class) + .hasMessageContaining("Varchar representation of '2023-10-23' exceeds varchar(8) bounds"); + } + + @Test + public void testHistoricalDateToVarchar() + { + assertThatThrownBy(() -> assertDateToVarcharCoercion(createUnboundedVarcharType(), LocalDate.parse("1899-12-31"), null)) + .hasMessageMatching(".*Coercion on historical dates is not supported.*"); + } + private void assertVarcharToDateCoercion(Type fromType, String date) { assertVarcharToDateCoercion(fromType, date, fromDateToEpochDate(date)); @@ -73,12 +98,20 @@ private void assertVarcharToDateCoercion(Type fromType, String date) private void assertVarcharToDateCoercion(Type fromType, String date, Long expected) { - Block coercedValue = createCoercer(TESTING_TYPE_MANAGER, toHiveType(fromType), toHiveType(DateType.DATE), new CoercionContext(NANOSECONDS, false)).orElseThrow() + Block coercedValue = createCoercer(TESTING_TYPE_MANAGER, toHiveType(fromType), toHiveType(DATE), new CoercionContext(NANOSECONDS, false)).orElseThrow() .apply(nativeValueToBlock(fromType, utf8Slice(date))); - assertThat(blockToNativeValue(DateType.DATE, coercedValue)) + assertThat(blockToNativeValue(DATE, coercedValue)) .isEqualTo(expected); } + private void assertDateToVarcharCoercion(Type toType, LocalDate date, String expected) + { + Block coercedValue = createCoercer(TESTING_TYPE_MANAGER, toHiveType(DATE), toHiveType(toType), new CoercionContext(NANOSECONDS, false)).orElseThrow() + .apply(nativeValueToBlock(DATE, date.toEpochDay())); + assertThat(blockToNativeValue(VARCHAR, coercedValue)) + .isEqualTo(utf8Slice(expected)); + } + private long fromDateToEpochDate(String dateString) { LocalDate date = LocalDate.parse(dateString); diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/BaseTestHiveCoercion.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/BaseTestHiveCoercion.java index fea12c95e4bd4..3b9ccadbc2f81 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/BaseTestHiveCoercion.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/BaseTestHiveCoercion.java @@ -155,6 +155,8 @@ protected void doTestHiveCoercion(HiveTableDefinition tableDefinition) "string_to_double", "varchar_to_double_infinity", "varchar_to_special_double", + "date_to_string", + "date_to_bounded_varchar", "char_to_bigger_char", "char_to_smaller_char", "timestamp_millis_to_date", @@ -253,6 +255,8 @@ protected void insertTableRows(String tableName, String floatToDoubleType) " '1234.01234', " + " 'Infinity'," + " 'NaN'," + + " DATE '2023-09-28', " + + " DATE '2000-04-13', " + " 'abc', " + " 'abc', " + " TIMESTAMP '2022-12-31 23:59:59.999', " + @@ -323,6 +327,8 @@ protected void insertTableRows(String tableName, String floatToDoubleType) " '0', " + " '-Infinity'," + " 'Invalid Double'," + + " DATE '2123-09-27', " + + " DATE '1900-01-01', " + " '\uD83D\uDCB0\uD83D\uDCB0\uD83D\uDCB0', " + " '\uD83D\uDCB0\uD83D\uDCB0\uD83D\uDCB0', " + " TIMESTAMP '1970-01-01 00:00:00.123', " + @@ -567,6 +573,12 @@ else if (getHiveVersionMajor() == 3 && isFormat.test("orc")) { .put("varchar_to_special_double", Arrays.asList( coercedNaN == null ? null : Double.NaN, null)) + .put("date_to_string", ImmutableList.of( + "2023-09-28", + "2123-09-27")) + .put("date_to_bounded_varchar", ImmutableList.of( + "2000-04-13", + "1900-01-01")) .put("char_to_bigger_char", ImmutableList.of( "abc ", "\uD83D\uDCB0\uD83D\uDCB0\uD83D\uDCB0 ")) @@ -1057,6 +1069,8 @@ private void assertProperAlteredTableSchema(String tableName) row("string_to_double", "double"), row("varchar_to_double_infinity", "double"), row("varchar_to_special_double", "double"), + row("date_to_string", "varchar"), + row("date_to_bounded_varchar", "varchar(12)"), row("char_to_bigger_char", "char(4)"), row("char_to_smaller_char", "char(2)"), row("timestamp_millis_to_date", "date"), @@ -1143,6 +1157,8 @@ private void assertColumnTypes( .put("string_to_double", DOUBLE) .put("varchar_to_double_infinity", DOUBLE) .put("varchar_to_special_double", DOUBLE) + .put("date_to_string", VARCHAR) + .put("date_to_bounded_varchar", VARCHAR) .put("char_to_bigger_char", CHAR) .put("char_to_smaller_char", CHAR) .put("id", BIGINT) @@ -1224,6 +1240,8 @@ private static void alterTableColumnTypes(String tableName) onHive().executeQuery(format("ALTER TABLE %s CHANGE COLUMN varchar_to_bigger_varchar varchar_to_bigger_varchar varchar(4)", tableName)); onHive().executeQuery(format("ALTER TABLE %s CHANGE COLUMN varchar_to_smaller_varchar varchar_to_smaller_varchar varchar(2)", tableName)); onHive().executeQuery(format("ALTER TABLE %s CHANGE COLUMN varchar_to_date varchar_to_date date", tableName)); + onHive().executeQuery(format("ALTER TABLE %s CHANGE COLUMN date_to_string date_to_string string", tableName)); + onHive().executeQuery(format("ALTER TABLE %s CHANGE COLUMN date_to_bounded_varchar date_to_bounded_varchar varchar(12)", tableName)); onHive().executeQuery(format("ALTER TABLE %s CHANGE COLUMN varchar_to_distant_date varchar_to_distant_date date", tableName)); onHive().executeQuery(format("ALTER TABLE %s CHANGE COLUMN varchar_to_double varchar_to_double double", tableName)); onHive().executeQuery(format("ALTER TABLE %s CHANGE COLUMN string_to_double string_to_double double", tableName)); diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveCoercionOnPartitionedTable.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveCoercionOnPartitionedTable.java index b19900000b183..cd5d41a22338a 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveCoercionOnPartitionedTable.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveCoercionOnPartitionedTable.java @@ -158,6 +158,8 @@ private static HiveTableDefinition.HiveTableDefinitionBuilder tableDefinitionBui " string_to_double STRING," + " varchar_to_double_infinity VARCHAR(40)," + " varchar_to_special_double VARCHAR(40)," + + " date_to_string DATE," + + " date_to_bounded_varchar DATE," + " char_to_bigger_char CHAR(3)," + " char_to_smaller_char CHAR(3)," + " timestamp_millis_to_date TIMESTAMP," + diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveCoercionOnUnpartitionedTable.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveCoercionOnUnpartitionedTable.java index 128545f391fa3..8820004902c22 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveCoercionOnUnpartitionedTable.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveCoercionOnUnpartitionedTable.java @@ -107,6 +107,8 @@ varchar_to_double VARCHAR(40), string_to_double STRING, varchar_to_double_infinity VARCHAR(40), varchar_to_special_double VARCHAR(40), + date_to_string DATE, + date_to_bounded_varchar DATE, char_to_bigger_char CHAR(3), char_to_smaller_char CHAR(3), timestamp_millis_to_date TIMESTAMP, From de329bdb4369590f03801829da8f37b8a589ea98 Mon Sep 17 00:00:00 2001 From: praveenkrishna Date: Mon, 11 Dec 2023 12:26:49 +0530 Subject: [PATCH 102/350] Use correct HostnameVerifier when verify-hostnames set to true Previously we were using `NoopHostnameVerifier` which turns off hostname verification, if the HostnameVerifier is not set it uses `DefaultHostnameVerifier` which performs hostname verification --- .../trino/plugin/elasticsearch/client/ElasticsearchClient.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/plugin/trino-elasticsearch/src/main/java/io/trino/plugin/elasticsearch/client/ElasticsearchClient.java b/plugin/trino-elasticsearch/src/main/java/io/trino/plugin/elasticsearch/client/ElasticsearchClient.java index f23bb90862aef..3fcea05054f1a 100644 --- a/plugin/trino-elasticsearch/src/main/java/io/trino/plugin/elasticsearch/client/ElasticsearchClient.java +++ b/plugin/trino-elasticsearch/src/main/java/io/trino/plugin/elasticsearch/client/ElasticsearchClient.java @@ -225,7 +225,7 @@ private static BackpressureRestHighLevelClient createClient( buildSslContext(config.getKeystorePath(), config.getKeystorePassword(), config.getTrustStorePath(), config.getTruststorePassword()) .ifPresent(clientBuilder::setSSLContext); - if (config.isVerifyHostnames()) { + if (!config.isVerifyHostnames()) { clientBuilder.setSSLHostnameVerifier(NoopHostnameVerifier.INSTANCE); } } From 25b6488d7d857d1b2479e2e70ad3689b2b972cbb Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Tue, 12 Dec 2023 10:29:22 +0100 Subject: [PATCH 103/350] Update AWS SDK v1 to 1.12.610 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 81777afc45f89..42add6d84d7dd 100644 --- a/pom.xml +++ b/pom.xml @@ -176,7 +176,7 @@ 4.13.1 14.0.1 1.11.3 - 1.12.605 + 1.12.610 4.17.0 7.5.1 87 From 003330ced6c175d86628c863a093bef111b2bae6 Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Tue, 12 Dec 2023 10:29:54 +0100 Subject: [PATCH 104/350] Update Azure SDK to 1.2.19 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 42add6d84d7dd..37e47e90db144 100644 --- a/pom.xml +++ b/pom.xml @@ -207,7 +207,7 @@ com.azure azure-sdk-bom - 1.2.18 + 1.2.19 pom import From fbd677a805c90efdc2ed3c9fd892c256a5fa9c17 Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Tue, 12 Dec 2023 10:33:53 +0100 Subject: [PATCH 105/350] Update grpc to 1.60.0 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 37e47e90db144..19e1df1cb4386 100644 --- a/pom.xml +++ b/pom.xml @@ -247,7 +247,7 @@ io.grpc grpc-bom - 1.59.1 + 1.60.0 pom import From eef9d9378d412b8c0cb6329ed6d7069f8befe234 Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Tue, 12 Dec 2023 10:35:08 +0100 Subject: [PATCH 106/350] Update reactor-core to 3.4.34 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 19e1df1cb4386..ad1bffd450e35 100644 --- a/pom.xml +++ b/pom.xml @@ -930,7 +930,7 @@ io.projectreactor reactor-core - 3.4.33 + 3.4.34 From 30374dc0189d2cde145883a2510f53f24e3d11c6 Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Tue, 12 Dec 2023 10:35:34 +0100 Subject: [PATCH 107/350] Update JNA to 5.14.0 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index ad1bffd450e35..376709ab1d458 100644 --- a/pom.xml +++ b/pom.xml @@ -186,7 +186,7 @@ 10.1.0 1.43.3 1.4.2 - 5.13.0 + 5.14.0 0.12.3 3.6.0 1.9.20 From 638d3865877aa1a95bd47086a84fc77ca12684c2 Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Tue, 12 Dec 2023 10:37:57 +0100 Subject: [PATCH 108/350] Update kafka-clients to 3.6.1 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 376709ab1d458..614d2230097cd 100644 --- a/pom.xml +++ b/pom.xml @@ -188,7 +188,7 @@ 1.4.2 5.14.0 0.12.3 - 3.6.0 + 3.6.1 1.9.20 3.6.0 ${dep.airlift.version} From 491be08e56174f9de3c4afc2ffce7f4d10a085d8 Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Tue, 12 Dec 2023 10:38:45 +0100 Subject: [PATCH 109/350] Update postgresql to 42.7.1 --- pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 614d2230097cd..38949ea6c5432 100644 --- a/pom.xml +++ b/pom.xml @@ -1939,7 +1939,7 @@ org.checkerframework checker-qual - 3.40.0 + 3.41.0 @@ -2030,7 +2030,7 @@ org.postgresql postgresql - 42.7.0 + 42.7.1 From 8367575dcc1afe653bed5740b161d8b68403f511 Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Tue, 12 Dec 2023 10:40:14 +0100 Subject: [PATCH 110/350] Update AWS SDK v2 to 2.21.43 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 38949ea6c5432..9bb5dba1be965 100644 --- a/pom.xml +++ b/pom.xml @@ -295,7 +295,7 @@ software.amazon.awssdk bom - 2.21.38 + 2.21.43 pom import From 794c8dd8bde24e0c4e9b8c33d4f30d58b14866bf Mon Sep 17 00:00:00 2001 From: Manfred Moser Date: Mon, 11 Dec 2023 16:56:49 -0800 Subject: [PATCH 111/350] Add example SQL routine for formatting bytes MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Jan Waś --- docs/src/main/sphinx/routines/examples.md | 120 ++++++++++++++++++++++ 1 file changed, 120 insertions(+) diff --git a/docs/src/main/sphinx/routines/examples.md b/docs/src/main/sphinx/routines/examples.md index 9d7fdd14680a3..801307b40ff2a 100644 --- a/docs/src/main/sphinx/routines/examples.md +++ b/docs/src/main/sphinx/routines/examples.md @@ -502,3 +502,123 @@ routine: ``` A possible improvement is to introduce parameters for the total length. + +## Formatting bytes + +Trino includes a built-in `format_number()` function. However it is using units +that don't work well with bytes. The following `format_data_size` routine can +format large values of bytes into a human readable string. + +```sql +FUNCTION format_data_size(input BIGINT) +RETURNS VARCHAR + BEGIN + DECLARE value DOUBLE DEFAULT CAST(input AS DOUBLE); + DECLARE result BIGINT; + DECLARE base INT DEFAULT 1024; + DECLARE unit VARCHAR DEFAULT 'B'; + DECLARE format VARCHAR; + IF abs(value) >= base THEN + SET value = value / base; + SET unit = 'kB'; + END IF; + IF abs(value) >= base THEN + SET value = value / base; + SET unit = 'MB'; + END IF; + IF abs(value) >= base THEN + SET value = value / base; + SET unit = 'GB'; + END IF; + IF abs(value) >= base THEN + SET value = value / base; + SET unit = 'TB'; + END IF; + IF abs(value) >= base THEN + SET value = value / base; + SET unit = 'PB'; + END IF; + IF abs(value) >= base THEN + SET value = value / base; + SET unit = 'EB'; + END IF; + IF abs(value) >= base THEN + SET value = value / base; + SET unit = 'ZB'; + END IF; + IF abs(value) >= base THEN + SET value = value / base; + SET unit = 'YB'; + END IF; + IF abs(value) < 10 THEN + SET format = '%.2f'; + ELSEIF abs(value) < 100 THEN + SET format = '%.1f'; + ELSE + SET format = '%.0f'; + END IF; + RETURN format(format, value) || unit; + END; +``` + +Below is a query to show how it formats a wide range of values. + +```sql +WITH +data AS ( + SELECT CAST(pow(10, s.p) AS BIGINT) AS num + FROM table(sequence(start=>1, stop=>18)) AS s(p) + UNION ALL + SELECT -CAST(pow(10, s.p) AS BIGINT) AS num + FROM table(sequence(start=>1, stop=>18)) AS s(p) +) +SELECT + data.num + , format_data_size(data.num) AS formatted +FROM data +ORDER BY data.num; +``` + +The preceding query produces the following output: + +``` + num | formatted +----------------------+----------- + -1000000000000000000 | -888PB + -100000000000000000 | -88.8PB + -10000000000000000 | -8.88PB + -1000000000000000 | -909TB + -100000000000000 | -90.9TB + -10000000000000 | -9.09TB + -1000000000000 | -931GB + -100000000000 | -93.1GB + -10000000000 | -9.31GB + -1000000000 | -954MB + -100000000 | -95.4MB + -10000000 | -9.54MB + -1000000 | -977kB + -100000 | -97.7kB + -10000 | -9.77kB + -1000 | -1000B + -100 | -100B + -10 | -10.0B + 0 | 0.00B + 10 | 10.0B + 100 | 100B + 1000 | 1000B + 10000 | 9.77kB + 100000 | 97.7kB + 1000000 | 977kB + 10000000 | 9.54MB + 100000000 | 95.4MB + 1000000000 | 954MB + 10000000000 | 9.31GB + 100000000000 | 93.1GB + 1000000000000 | 931GB + 10000000000000 | 9.09TB + 100000000000000 | 90.9TB + 1000000000000000 | 909TB + 10000000000000000 | 8.88PB + 100000000000000000 | 88.8PB + 1000000000000000000 | 888PB +``` From 79bc8fa09b337ac163582a708b725eb34cf36b0c Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Tue, 12 Dec 2023 20:45:35 -0800 Subject: [PATCH 112/350] Upgrade Hudi test version to 0.14.0 --- plugin/trino-hudi/pom.xml | 36 ++++++++++++++++++- .../testing/TpchHudiTablesInitializer.java | 13 +++---- 2 files changed, 40 insertions(+), 9 deletions(-) diff --git a/plugin/trino-hudi/pom.xml b/plugin/trino-hudi/pom.xml index 5e6c486cf3e17..d609063eb6fb8 100644 --- a/plugin/trino-hudi/pom.xml +++ b/plugin/trino-hudi/pom.xml @@ -15,7 +15,8 @@ ${project.parent.basedir} - 0.12.3 + 0.14.0 + 2.4.9 @@ -188,6 +189,13 @@ runtime + + com.esotericsoftware + kryo-shaded + 4.0.2 + test + + io.airlift junit-extensions @@ -275,6 +283,32 @@ test + + org.apache.hbase + hbase-client + ${dep.hbase.version} + test + + + * + * + + + + + + org.apache.hbase + hbase-server + ${dep.hbase.version} + test + + + * + * + + + + org.apache.hudi hudi-client-common 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 64a44e0a8eefd..243501f8459de 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 @@ -47,6 +47,7 @@ import org.apache.hudi.client.HoodieJavaWriteClient; import org.apache.hudi.client.common.HoodieJavaEngineContext; import org.apache.hudi.common.bootstrap.index.NoOpBootstrapIndex; +import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.model.HoodieAvroPayload; import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieKey; @@ -239,6 +240,9 @@ private static HoodieJavaWriteClient createWriteClient(TpchTa .withArchivalConfig(archivalConfig) .withEmbeddedTimelineServerEnabled(false) .withMarkersType(MarkerType.DIRECT.name()) + // Disabling Hudi metadata table (MDT) in tests as the support of + // reading MDT is broken after removal of Hudi dependencies from compile time + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()) .build(); return new HoodieJavaWriteClient<>(new HoodieJavaEngineContext(conf), cfg); } @@ -272,14 +276,7 @@ private static RecordConverter createRecordConverter(TpchTable table) // wrap to a HoodieRecord HoodieKey key = new HoodieKey(uuid, PARTITION_PATH); HoodieAvroPayload data = new HoodieAvroPayload(Option.of(record)); - return new HoodieRecord<>(key, data) - { - @Override - public HoodieRecord newInstance() - { - return new HoodieAvroRecord<>(key, data, null); - } - }; + return new HoodieAvroRecord<>(key, data, null); }; } From 2260b231e235bcbffb9923d18d602200a16c093c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jan=20Wa=C5=9B?= Date: Wed, 6 Dec 2023 12:12:28 +0100 Subject: [PATCH 113/350] Remove product tests targeting Hive 2 or older --- .../product/hive/AbstractTestHiveViews.java | 13 +- .../product/hive/BaseTestHiveCoercion.java | 13 +- .../tests/product/hive/TestAvroSchemaUrl.java | 21 - .../product/hive/TestHiveBucketedTables.java | 12 +- .../hive/TestHiveMaterializedView.java | 18 - .../tests/product/hive/TestHiveMerge.java | 89 ++- .../hive/TestHiveTransactionalTable.java | 234 +++----- .../TestHiveTransactionalTableInsert.java | 5 - .../tests/product/hive/TestHiveViews.java | 140 ++--- .../product/hive/TestHiveViewsLegacy.java | 4 +- .../tests/product/hive/TestReadUniontype.java | 519 ------------------ .../trino/tests/product/hive/TestRoles.java | 30 - 12 files changed, 162 insertions(+), 936 deletions(-) delete mode 100644 testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestReadUniontype.java diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/AbstractTestHiveViews.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/AbstractTestHiveViews.java index cbf872c0d3613..a81c2dd5df042 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/AbstractTestHiveViews.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/AbstractTestHiveViews.java @@ -23,7 +23,6 @@ import io.trino.testng.services.Flaky; import io.trino.tests.product.utils.QueryExecutors; import org.intellij.lang.annotations.Language; -import org.testng.SkipException; import org.testng.annotations.Test; import java.math.BigDecimal; @@ -233,10 +232,6 @@ public void testViewWithUnsupportedCoercion() @Flaky(issue = RETRYABLE_FAILURES_ISSUES, match = RETRYABLE_FAILURES_MATCH) public void testOuterParentheses() { - if (getHiveVersionMajor() <= 1) { - throw new SkipException("The old Hive doesn't allow outer parentheses in a view definition"); - } - onHive().executeQuery("CREATE OR REPLACE VIEW view_outer_parentheses AS (SELECT 'parentheses' AS col FROM nation LIMIT 1)"); assertViewQuery("SELECT * FROM view_outer_parentheses", @@ -391,12 +386,10 @@ public void testHiveViewInInformationSchema() onTrino().executeQuery("CREATE TABLE test_schema.trino_table(a int)"); onTrino().executeQuery("CREATE VIEW test_schema.trino_test_view AS SELECT * FROM nation"); - boolean hiveWithTableNamesByType = getHiveVersionMajor() >= 3 || - (getHiveVersionMajor() == 2 && getHiveVersionMinor() >= 3); assertThat(onTrino().executeQuery("SELECT * FROM information_schema.tables WHERE table_schema = 'test_schema'")).containsOnly( row("hive", "test_schema", "trino_table", "BASE TABLE"), row("hive", "test_schema", "hive_table", "BASE TABLE"), - row("hive", "test_schema", "hive_test_view", hiveWithTableNamesByType ? "VIEW" : "BASE TABLE"), + row("hive", "test_schema", "hive_test_view", "VIEW"), row("hive", "test_schema", "trino_test_view", "VIEW")); assertThat(onTrino().executeQuery("SELECT view_definition FROM information_schema.views WHERE table_schema = 'test_schema' and table_name = 'hive_test_view'")).containsOnly( @@ -588,10 +581,6 @@ public void testUnionAllViews() @Flaky(issue = RETRYABLE_FAILURES_ISSUES, match = RETRYABLE_FAILURES_MATCH) public void testUnionDistinctViews() { - if (getHiveVersionMajor() < 1 || (getHiveVersionMajor() == 1 && getHiveVersionMinor() < 2)) { - throw new SkipException("UNION DISTINCT and plain UNION are not supported before Hive 1.2.0"); - } - onHive().executeQuery("DROP TABLE IF EXISTS union_helper"); onHive().executeQuery("CREATE TABLE union_helper (\n" + "r_regionkey BIGINT,\n" diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/BaseTestHiveCoercion.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/BaseTestHiveCoercion.java index 3b9ccadbc2f81..4c28ec22fe7ac 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/BaseTestHiveCoercion.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/BaseTestHiveCoercion.java @@ -353,7 +353,7 @@ protected Map> expectedValuesForEngineProvider(Engine engin if (Stream.of("rctext", "textfile", "sequencefile").anyMatch(isFormat)) { hiveValueForCaseChangeField = "\"lower2uppercase\":2"; } - else if (getHiveVersionMajor() == 3 && isFormat.test("orc")) { + else if (isFormat.test("orc")) { hiveValueForCaseChangeField = "\"LOWER2UPPERCASE\":null"; } else { @@ -813,7 +813,7 @@ private void assertNestedSubFields(String tableName) Map> expectedNestedFieldTrino = ImmutableMap.of("nested_field", ImmutableList.of(2L, 2L)); Map> expectedNestedFieldHive; - if (getHiveVersionMajor() == 3 && isFormat.test("orc")) { + if (isFormat.test("orc")) { expectedNestedFieldHive = ImmutableMap.of("nested_field", Arrays.asList(null, null)); } else { @@ -836,14 +836,7 @@ private void assertNestedSubFields(String tableName) } else if (isFormat.test("parquet")) { assertQueryResults(Engine.HIVE, subfieldQueryUpperCase, expectedNestedFieldHive, expectedColumns, 2, tableName); - - if (getHiveVersionMajor() == 1) { - assertThatThrownBy(() -> assertQueryResults(Engine.HIVE, subfieldQueryLowerCase, expectedNestedFieldHive, expectedColumns, 2, tableName)) - .hasMessageContaining("java.sql.SQLException"); - } - else { - assertQueryResults(Engine.HIVE, subfieldQueryLowerCase, expectedNestedFieldHive, expectedColumns, 2, tableName); - } + assertQueryResults(Engine.HIVE, subfieldQueryLowerCase, expectedNestedFieldHive, expectedColumns, 2, tableName); } else { assertQueryResults(Engine.HIVE, subfieldQueryUpperCase, expectedNestedFieldHive, expectedColumns, 2, tableName); diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestAvroSchemaUrl.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestAvroSchemaUrl.java index 83d9520ebc9cb..8530f9de494e1 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestAvroSchemaUrl.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestAvroSchemaUrl.java @@ -17,9 +17,7 @@ import io.trino.tempto.AfterMethodWithContext; import io.trino.tempto.BeforeMethodWithContext; import io.trino.tempto.hadoop.hdfs.HdfsClient; -import io.trino.tempto.query.QueryExecutionException; import io.trino.testng.services.Flaky; -import org.testng.SkipException; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @@ -27,7 +25,6 @@ import java.io.InputStream; import java.nio.file.Paths; -import static com.google.common.base.Strings.isNullOrEmpty; import static io.trino.tempto.assertions.QueryAssert.Row.row; import static io.trino.tempto.assertions.QueryAssert.assertQueryFailure; import static io.trino.tests.product.TestGroups.AVRO; @@ -199,13 +196,6 @@ public void testTableWithLongColumnType() @Flaky(issue = RETRYABLE_FAILURES_ISSUES, match = RETRYABLE_FAILURES_MATCH) public void testPartitionedTableWithLongColumnType() { - if (isOnHdp() && getHiveVersionMajor() < 3) { - // HDP 2.6 won't allow to define a partitioned table with schema having a column with type definition over 2000 characters. - // It is possible to create table with simpler schema and then alter the schema, but that results in different end state. - // To retain proper test coverage, this test needs to be disabled on HDP 2. - throw new SkipException("Skipping on HDP 2"); - } - onHive().executeQuery("DROP TABLE IF EXISTS test_avro_schema_url_partitioned_long_column"); onHive().executeQuery("" + "CREATE TABLE test_avro_schema_url_partitioned_long_column " + @@ -263,15 +253,4 @@ public void testHiveCreatedCamelCaseColumnTable() onHive().executeQuery("DROP TABLE IF EXISTS test_camelCase_avro_schema_url_hive"); } - - private boolean isOnHdp() - { - try { - String hdpVersion = (String) onHive().executeQuery("SET system:hdp.version").getOnlyValue(); - return !isNullOrEmpty(hdpVersion); - } - catch (QueryExecutionException e) { - return false; - } - } } diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveBucketedTables.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveBucketedTables.java index 96631d662f01c..1a7680ae983de 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveBucketedTables.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveBucketedTables.java @@ -302,14 +302,12 @@ public void testBucketingVersion() List bucketV1NameOptions = ImmutableList.of(bucketV1); List bucketV2NameOptions = ImmutableList.of(bucketV2Standard, bucketV2DirectInsert); - testBucketingVersion(BUCKETED_DEFAULT, value, false, (getHiveVersionMajor() < 3) ? bucketV1NameOptions : bucketV2NameOptions); - testBucketingVersion(BUCKETED_DEFAULT, value, true, (getHiveVersionMajor() < 3) ? bucketV1NameOptions : bucketV2NameOptions); + testBucketingVersion(BUCKETED_DEFAULT, value, false, bucketV2NameOptions); + testBucketingVersion(BUCKETED_DEFAULT, value, true, bucketV2NameOptions); testBucketingVersion(BUCKETED_V1, value, false, bucketV1NameOptions); testBucketingVersion(BUCKETED_V1, value, true, bucketV1NameOptions); - if (getHiveVersionMajor() >= 3) { - testBucketingVersion(BUCKETED_V2, value, false, bucketV2NameOptions); - testBucketingVersion(BUCKETED_V2, value, true, bucketV2NameOptions); - } + testBucketingVersion(BUCKETED_V2, value, false, bucketV2NameOptions); + testBucketingVersion(BUCKETED_V2, value, true, bucketV2NameOptions); } @Test(dataProvider = "testBucketingWithUnsupportedDataTypesDataProvider") @@ -459,7 +457,7 @@ private String getExpectedBucketVersion(BucketingType bucketingType) { switch (bucketingType) { case BUCKETED_DEFAULT: - return getHiveVersionMajor() < 3 ? "1" : "2"; + return "2"; case BUCKETED_V1: return "1"; case BUCKETED_V2: diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveMaterializedView.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveMaterializedView.java index accad931f3dad..0c0bacdb41fff 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveMaterializedView.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveMaterializedView.java @@ -27,19 +27,9 @@ public class TestHiveMaterializedView extends HiveProductTest { - private boolean isTestEnabled() - { - // MATERIALIZED VIEW is supported since Hive 3 - return getHiveVersionMajor() >= 3; - } - @BeforeMethodWithContext public void setUp() { - if (!isTestEnabled()) { - return; - } - onHive().executeQuery("" + "CREATE TABLE test_materialized_view_table(x string) " + "STORED AS ORC " + @@ -50,10 +40,6 @@ public void setUp() @AfterMethodWithContext public void tearDown() { - if (!isTestEnabled()) { - return; - } - onHive().executeQuery("DROP TABLE IF EXISTS test_materialized_view_table"); } @@ -71,10 +57,6 @@ public void testPartitionedMaterializedView() private void testMaterializedView(boolean partitioned) { - if (!isTestEnabled()) { - return; - } - onHive().executeQuery("DROP MATERIALIZED VIEW test_materialized_view_view"); onHive().executeQuery("" + "CREATE MATERIALIZED VIEW test_materialized_view_view " + diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveMerge.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveMerge.java index 974d2a29f8572..52046c538a91d 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveMerge.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveMerge.java @@ -16,7 +16,6 @@ import io.trino.tempto.assertions.QueryAssert; import io.trino.tempto.query.QueryResult; import io.trino.tests.product.hive.util.TemporaryHiveTable; -import org.testng.SkipException; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @@ -46,12 +45,12 @@ public class TestHiveMerge @Test(groups = HIVE_TRANSACTIONAL, timeOut = 60 * 60 * 1000) public void testMergeSimpleSelect() { - withTemporaryTable("merge_simple_select_target", true, false, NONE, targetTable -> { + withTemporaryTable("merge_simple_select_target", false, NONE, targetTable -> { onTrino().executeQuery(format("CREATE TABLE %s (customer VARCHAR, purchases INT, address VARCHAR) WITH (transactional = true)", targetTable)); onTrino().executeQuery(format("INSERT INTO %s (customer, purchases, address) VALUES ('Aaron', 5, 'Antioch'), ('Bill', 7, 'Buena'), ('Carol', 3, 'Cambridge'), ('Dave', 11, 'Devon')", targetTable)); - withTemporaryTable("merge_simple_select_source", true, false, NONE, sourceTable -> { + withTemporaryTable("merge_simple_select_source", false, NONE, sourceTable -> { onTrino().executeQuery(format("CREATE TABLE %s (customer VARCHAR, purchases INT, address VARCHAR) WITH (transactional = true)", sourceTable)); onTrino().executeQuery(format("INSERT INTO %s (customer, purchases, address) VALUES ('Aaron', 6, 'Arches'), ('Ed', 7, 'Etherville'), ('Carol', 9, 'Centreville'), ('Dave', 11, 'Darbyshire')", sourceTable)); @@ -71,12 +70,12 @@ public void testMergeSimpleSelect() @Test(groups = HIVE_TRANSACTIONAL, timeOut = 60 * 60 * 1000) public void testMergeSimpleSelectPartitioned() { - withTemporaryTable("merge_simple_select_partitioned_target", true, true, NONE, targetTable -> { + withTemporaryTable("merge_simple_select_partitioned_target", true, NONE, targetTable -> { onTrino().executeQuery(format("CREATE TABLE %s (customer VARCHAR, purchases INT, address VARCHAR) WITH (transactional = true, partitioned_by = ARRAY['address'])", targetTable)); onTrino().executeQuery(format("INSERT INTO %s (customer, purchases, address) VALUES ('Aaron', 5, 'Antioch'), ('Bill', 7, 'Buena'), ('Carol', 3, 'Cambridge'), ('Dave', 11, 'Devon')", targetTable)); - withTemporaryTable("merge_simple_select_partitioned_source", true, false, NONE, sourceTable -> { + withTemporaryTable("merge_simple_select_partitioned_source", false, NONE, sourceTable -> { onTrino().executeQuery(format("CREATE TABLE %s (customer VARCHAR, purchases INT, address VARCHAR) WITH (transactional = true)", sourceTable)); onTrino().executeQuery(format("INSERT INTO %s (customer, purchases, address) VALUES ('Aaron', 6, 'Arches'), ('Ed', 7, 'Etherville'), ('Carol', 9, 'Centreville'), ('Dave', 11, 'Darbyshire')", sourceTable)); @@ -97,7 +96,7 @@ public void testMergeSimpleSelectPartitioned() public void testMergeUpdateWithVariousLayouts(boolean partitioned, String bucketing) { BucketingType bucketingType = bucketing.isEmpty() ? NONE : BUCKETED_V2; - withTemporaryTable("merge_update_with_various_formats", true, partitioned, bucketingType, targetTable -> { + withTemporaryTable("merge_update_with_various_formats", partitioned, bucketingType, targetTable -> { StringBuilder builder = new StringBuilder(); builder.append("CREATE TABLE ") .append(targetTable) @@ -113,7 +112,7 @@ public void testMergeUpdateWithVariousLayouts(boolean partitioned, String bucket onTrino().executeQuery(format("INSERT INTO %s (customer, purchase) VALUES ('Dave', 'dates'), ('Lou', 'limes'), ('Carol', 'candles')", targetTable)); verifySelectForTrinoAndHive("SELECT * FROM " + targetTable, row("Dave", "dates"), row("Lou", "limes"), row("Carol", "candles")); - withTemporaryTable("merge_update_with_various_formats_source", true, false, NONE, sourceTable -> { + withTemporaryTable("merge_update_with_various_formats_source", false, NONE, sourceTable -> { onTrino().executeQuery(format("CREATE TABLE %s (customer VARCHAR, purchase VARCHAR) WITH (transactional = true)", sourceTable)); onTrino().executeQuery(format("INSERT INTO %s (customer, purchase) VALUES ('Craig', 'candles'), ('Len', 'limes'), ('Joe', 'jellybeans')", sourceTable)); @@ -133,13 +132,13 @@ public void testMergeUpdateWithVariousLayouts(boolean partitioned, String bucket @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) public void testMergeUnBucketedUnPartitionedFailure() { - withTemporaryTable("merge_with_various_formats_failure", true, false, NONE, targetTable -> { + withTemporaryTable("merge_with_various_formats_failure", false, NONE, targetTable -> { onTrino().executeQuery(format("CREATE TABLE %s (customer VARCHAR, purchase VARCHAR) WITH (transactional = true)", targetTable)); onTrino().executeQuery(format("INSERT INTO %s (customer, purchase) VALUES ('Dave', 'dates'), ('Lou', 'limes'), ('Carol', 'candles')", targetTable)); verifySelectForTrinoAndHive("SELECT * FROM " + targetTable, row("Dave", "dates"), row("Lou", "limes"), row("Carol", "candles")); - withTemporaryTable("merge_with_various_formats_failure_source", true, false, NONE, sourceTable -> { + withTemporaryTable("merge_with_various_formats_failure_source", false, NONE, sourceTable -> { onTrino().executeQuery(format("CREATE TABLE %s (customer VARCHAR, purchase VARCHAR) WITH (transactional = true)", sourceTable)); onTrino().executeQuery(format("INSERT INTO %s (customer, purchase) VALUES ('Craig', 'candles'), ('Len', 'limes'), ('Joe', 'jellybeans')", sourceTable)); @@ -170,7 +169,7 @@ public Object[][] partitionedAndBucketedProvider() @Test(groups = HIVE_TRANSACTIONAL, timeOut = 60 * 60 * 1000) public void testMergeMultipleOperationsUnbucketedUnpartitioned() { - withTemporaryTable("merge_multiple", true, false, NONE, targetTable -> { + withTemporaryTable("merge_multiple", false, NONE, targetTable -> { onTrino().executeQuery(format("CREATE TABLE %s (customer VARCHAR, purchases INT, zipcode INT, spouse VARCHAR, address VARCHAR) WITH (transactional = true)", targetTable)); testMergeMultipleOperationsInternal(targetTable, 32); }); @@ -179,7 +178,7 @@ public void testMergeMultipleOperationsUnbucketedUnpartitioned() @Test(groups = HIVE_TRANSACTIONAL, timeOut = 60 * 60 * 1000) public void testMergeMultipleOperationsUnbucketedPartitioned() { - withTemporaryTable("merge_multiple", true, true, NONE, targetTable -> { + withTemporaryTable("merge_multiple", true, NONE, targetTable -> { onTrino().executeQuery(format("CREATE TABLE %s (purchases INT, zipcode INT, spouse VARCHAR, address VARCHAR, customer VARCHAR) WITH (transactional = true, partitioned_by = ARRAY['address', 'customer'])", targetTable)); testMergeMultipleOperationsInternal(targetTable, 32); }); @@ -188,7 +187,7 @@ public void testMergeMultipleOperationsUnbucketedPartitioned() @Test(groups = HIVE_TRANSACTIONAL, timeOut = 60 * 60 * 1000) public void testMergeMultipleOperationsBucketedUnpartitioned() { - withTemporaryTable("merge_multiple", true, false, BUCKETED_V2, targetTable -> { + withTemporaryTable("merge_multiple", false, BUCKETED_V2, targetTable -> { onHive().executeQuery(format("CREATE TABLE %s (customer STRING, purchases INT, zipcode INT, spouse STRING, address STRING)" + " CLUSTERED BY(customer, zipcode, address) INTO 4 BUCKETS STORED AS ORC TBLPROPERTIES ('transactional'='true')", targetTable)); testMergeMultipleOperationsInternal(targetTable, 32); @@ -263,7 +262,7 @@ private List getRowsFromQueryResult(QueryResult result) @Test(groups = HIVE_TRANSACTIONAL, timeOut = 60 * 60 * 1000) public void testMergeSimpleQuery() { - withTemporaryTable("merge_simple_query_target", true, false, NONE, targetTable -> { + withTemporaryTable("merge_simple_query_target", false, NONE, targetTable -> { onTrino().executeQuery(format("CREATE TABLE %s (customer VARCHAR, purchases INT, address VARCHAR) WITH (transactional = true)", targetTable)); onTrino().executeQuery(format("INSERT INTO %s (customer, purchases, address) VALUES ('Aaron', 5, 'Antioch'), ('Bill', 7, 'Buena'), ('Carol', 3, 'Cambridge'), ('Dave', 11, 'Devon')", targetTable)); @@ -283,7 +282,7 @@ public void testMergeSimpleQuery() @Test(groups = HIVE_TRANSACTIONAL, timeOut = 60 * 60 * 1000) public void testMergeAllInserts() { - withTemporaryTable("merge_all_inserts", true, false, NONE, targetTable -> { + withTemporaryTable("merge_all_inserts", false, NONE, targetTable -> { onTrino().executeQuery(format("CREATE TABLE %s (customer VARCHAR, purchases INT, address VARCHAR) WITH (transactional = true)", targetTable)); onTrino().executeQuery(format("INSERT INTO %s (customer, purchases, address) VALUES ('Aaron', 11, 'Antioch'), ('Bill', 7, 'Buena')", targetTable)); @@ -301,7 +300,7 @@ public void testMergeAllInserts() @Test(groups = HIVE_TRANSACTIONAL, timeOut = 60 * 60 * 1000) public void testMergeSimpleQueryPartitioned() { - withTemporaryTable("merge_simple_query_partitioned_target", true, true, NONE, targetTable -> { + withTemporaryTable("merge_simple_query_partitioned_target", true, NONE, targetTable -> { onTrino().executeQuery(format("CREATE TABLE %s (customer VARCHAR, purchases INT, address VARCHAR) WITH (transactional = true, partitioned_by = ARRAY['address'])", targetTable)); onTrino().executeQuery(format("INSERT INTO %s (customer, purchases, address) VALUES ('Aaron', 5, 'Antioch'), ('Bill', 7, 'Buena'), ('Carol', 3, 'Cambridge'), ('Dave', 11, 'Devon')", targetTable)); @@ -322,12 +321,12 @@ public void testMergeSimpleQueryPartitioned() @Test(groups = HIVE_TRANSACTIONAL, timeOut = 60 * 60 * 1000) public void testMergeAllColumnsUpdated() { - withTemporaryTable("merge_all_columns_updated_target", true, false, NONE, targetTable -> { + withTemporaryTable("merge_all_columns_updated_target", false, NONE, targetTable -> { onTrino().executeQuery(format("CREATE TABLE %s (customer VARCHAR, purchases INT, address VARCHAR) WITH (transactional = true)", targetTable)); onTrino().executeQuery(format("INSERT INTO %s (customer, purchases, address) VALUES ('Dave', 11, 'Devon'), ('Aaron', 5, 'Antioch'), ('Bill', 7, 'Buena'), ('Carol', 3, 'Cambridge')", targetTable)); - withTemporaryTable("merge_all_columns_updated_source", true, false, NONE, sourceTable -> { + withTemporaryTable("merge_all_columns_updated_source", false, NONE, sourceTable -> { onTrino().executeQuery(format("CREATE TABLE %s (customer VARCHAR, purchases INT, address VARCHAR) WITH (transactional = true)", sourceTable)); onTrino().executeQuery(format("INSERT INTO %s (customer, purchases, address) VALUES ('Dave', 11, 'Darbyshire'), ('Aaron', 6, 'Arches'), ('Carol', 9, 'Centreville'), ('Ed', 7, 'Etherville')", sourceTable)); @@ -343,12 +342,12 @@ public void testMergeAllColumnsUpdated() @Test(groups = HIVE_TRANSACTIONAL, timeOut = 60 * 60 * 1000) public void testMergeAllMatchesDeleted() { - withTemporaryTable("merge_all_matches_deleted_target", true, false, NONE, targetTable -> { + withTemporaryTable("merge_all_matches_deleted_target", false, NONE, targetTable -> { onTrino().executeQuery(format("CREATE TABLE %s (customer VARCHAR, purchases INT, address VARCHAR) WITH (transactional = true)", targetTable)); onTrino().executeQuery(format("INSERT INTO %s (customer, purchases, address) VALUES ('Aaron', 5, 'Antioch'), ('Bill', 7, 'Buena'), ('Carol', 3, 'Cambridge'), ('Dave', 11, 'Devon')", targetTable)); - withTemporaryTable("merge_all_matches_deleted_source", true, false, NONE, sourceTable -> { + withTemporaryTable("merge_all_matches_deleted_source", false, NONE, sourceTable -> { onTrino().executeQuery(format("CREATE TABLE %s (customer VARCHAR, purchases INT, address VARCHAR) WITH (transactional = true)", sourceTable)); onTrino().executeQuery(format("INSERT INTO %s (customer, purchases, address) VALUES ('Aaron', 6, 'Arches'), ('Carol', 9, 'Centreville'), ('Dave', 11, 'Darbyshire'), ('Ed', 7, 'Etherville')", sourceTable)); @@ -364,12 +363,12 @@ public void testMergeAllMatchesDeleted() @Test(groups = HIVE_TRANSACTIONAL, timeOut = 60 * 60 * 1000, dataProvider = "partitionedBucketedFailure") public void testMergeMultipleRowsMatchFails(String createTableSql) { - withTemporaryTable("merge_all_matches_deleted_target", true, true, NONE, targetTable -> { + withTemporaryTable("merge_all_matches_deleted_target", true, NONE, targetTable -> { onHive().executeQuery(format(createTableSql, targetTable)); onTrino().executeQuery(format("INSERT INTO %s (customer, purchases, address) VALUES ('Aaron', 5, 'Antioch'), ('Bill', 7, 'Antioch')", targetTable)); - withTemporaryTable("merge_all_matches_deleted_source", true, false, NONE, sourceTable -> { + withTemporaryTable("merge_all_matches_deleted_source", false, NONE, sourceTable -> { onTrino().executeQuery(format("CREATE TABLE %s (customer VARCHAR, purchases INT, address VARCHAR) WITH (transactional = true)", sourceTable)); onTrino().executeQuery(format("INSERT INTO %s (customer, purchases, address) VALUES ('Aaron', 6, 'Adelphi'), ('Aaron', 8, 'Ashland')", sourceTable)); @@ -401,12 +400,12 @@ public Object[][] partitionedBucketedFailure() public void testMergeFailingPartitioning() { String testDescription = "failing_merge"; - withTemporaryTable(format("%s_target", testDescription), true, true, NONE, targetTable -> { + withTemporaryTable(format("%s_target", testDescription), true, NONE, targetTable -> { onHive().executeQuery(format("CREATE TABLE %s (customer STRING, purchases INT, address STRING) STORED AS ORC TBLPROPERTIES ('transactional'='true')", targetTable)); onTrino().executeQuery(format("INSERT INTO %s (customer, purchases, address) VALUES ('Aaron', 5, 'Antioch'), ('Bill', 7, 'Buena'), ('Carol', 3, 'Cambridge'), ('Dave', 11, 'Devon')", targetTable)); - withTemporaryTable(format("%s_source", testDescription), true, true, NONE, sourceTable -> { + withTemporaryTable(format("%s_source", testDescription), true, NONE, sourceTable -> { onHive().executeQuery(format("CREATE TABLE %s (purchases INT, address STRING) PARTITIONED BY (customer STRING) STORED AS ORC TBLPROPERTIES ('transactional'='true')", sourceTable)); onTrino().executeQuery(format("INSERT INTO %s (customer, purchases, address) VALUES ('Aaron', 6, 'Arches'), ('Ed', 7, 'Etherville'), ('Carol', 9, 'Centreville'), ('Dave', 11, 'Darbyshire')", sourceTable)); @@ -440,12 +439,12 @@ public void testMergeWithDifferentPartitioning(String testDescription, String cr private void testMergeWithDifferentPartitioningInternal(String testDescription, String createTargetTableSql, String createSourceTableSql) { - withTemporaryTable(format("%s_target", testDescription), true, true, NONE, targetTable -> { + withTemporaryTable(format("%s_target", testDescription), true, NONE, targetTable -> { onHive().executeQuery(format(createTargetTableSql, targetTable)); onTrino().executeQuery(format("INSERT INTO %s (customer, purchases, address) VALUES ('Aaron', 5, 'Antioch'), ('Bill', 7, 'Buena'), ('Carol', 3, 'Cambridge'), ('Dave', 11, 'Devon')", targetTable)); - withTemporaryTable(format("%s_source", testDescription), true, true, NONE, sourceTable -> { + withTemporaryTable(format("%s_source", testDescription), true, NONE, sourceTable -> { onHive().executeQuery(format(createSourceTableSql, sourceTable)); onTrino().executeQuery(format("INSERT INTO %s (customer, purchases, address) VALUES ('Aaron', 6, 'Arches'), ('Ed', 7, 'Etherville'), ('Carol', 9, 'Centreville'), ('Dave', 11, 'Darbyshire')", sourceTable)); @@ -502,7 +501,7 @@ public Object[][] targetAndSourceWithDifferentPartitioning() @Test(groups = HIVE_TRANSACTIONAL, timeOut = 60 * 60 * 1000) public void testMergeQueryWithStrangeCapitalization() { - withTemporaryTable("test_without_aliases_target", true, false, NONE, targetTable -> { + withTemporaryTable("test_without_aliases_target", false, NONE, targetTable -> { onTrino().executeQuery(format("CREATE TABLE %s (customer VARCHAR, purchases INT, address VARCHAR) WITH (transactional = true)", targetTable)); onTrino().executeQuery(format("INSERT INTO %s (customer, purchases, address) VALUES ('Aaron', 5, 'Antioch'), ('Bill', 7, 'Buena'), ('Carol', 3, 'Cambridge'), ('Dave', 11, 'Devon')", targetTable)); @@ -521,12 +520,12 @@ public void testMergeQueryWithStrangeCapitalization() @Test(groups = HIVE_TRANSACTIONAL, timeOut = 60 * 60 * 1000) public void testMergeWithoutTablesAliases() { - withTemporaryTable("test_without_aliases_target", true, false, NONE, targetTable -> { + withTemporaryTable("test_without_aliases_target", false, NONE, targetTable -> { onTrino().executeQuery(format("CREATE TABLE %s (cusTomer VARCHAR, purchases INT, address VARCHAR) WITH (transactional = true)", targetTable)); onTrino().executeQuery(format("INSERT INTO %s (customer, purchases, address) VALUES ('Aaron', 5, 'Antioch'), ('Bill', 7, 'Buena'), ('Carol', 3, 'Cambridge'), ('Dave', 11, 'Devon')", targetTable)); - withTemporaryTable("test_without_aliases_source", true, false, NONE, sourceTable -> { + withTemporaryTable("test_without_aliases_source", false, NONE, sourceTable -> { onTrino().executeQuery(format("CREATE TABLE %s (customer VARCHAR, purchases INT, address VARCHAR) WITH (transactional = true)", sourceTable)); onTrino().executeQuery(format("INSERT INTO %s (customer, purchases, address) VALUES ('Aaron', 6, 'Arches'), ('Ed', 7, 'Etherville'), ('Carol', 9, 'Centreville'), ('Dave', 11, 'Darbyshire')", sourceTable)); @@ -545,12 +544,12 @@ public void testMergeWithoutTablesAliases() @Test(groups = HIVE_TRANSACTIONAL, timeOut = 60 * 60 * 1000) public void testMergeWithUnpredictablePredicates() { - withTemporaryTable("test_without_aliases_target", true, false, NONE, targetTable -> { + withTemporaryTable("test_without_aliases_target", false, NONE, targetTable -> { onTrino().executeQuery(format("CREATE TABLE %s (cusTomer VARCHAR, purchases INT, address VARCHAR) WITH (transactional = true)", targetTable)); onTrino().executeQuery(format("INSERT INTO %s (customer, purchases, address) VALUES ('Aaron', 5, 'Antioch'), ('Bill', 7, 'Buena'), ('Carol', 3, 'Cambridge'), ('Dave', 11, 'Devon')", targetTable)); - withTemporaryTable("test_without_aliases_source", true, false, NONE, sourceTable -> { + withTemporaryTable("test_without_aliases_source", false, NONE, sourceTable -> { onTrino().executeQuery(format("CREATE TABLE %s (customer VARCHAR, purchases INT, address VARCHAR) WITH (transactional = true)", sourceTable)); onTrino().executeQuery(format("INSERT INTO %s (customer, purchases, address) VALUES ('Aaron', 6, 'Arches'), ('Carol', 9, 'Centreville'), ('Dave', 11, 'Darbyshire'), ('Ed', 7, 'Etherville')", sourceTable)); @@ -586,13 +585,13 @@ public void testMergeWithUnpredictablePredicates() @Test(groups = HIVE_TRANSACTIONAL, timeOut = 60 * 60 * 1000) public void testMergeWithSimplifiedUnpredictablePredicates() { - withTemporaryTable("test_without_aliases_target", true, false, NONE, targetTable -> { + withTemporaryTable("test_without_aliases_target", false, NONE, targetTable -> { onTrino().executeQuery(format("CREATE TABLE %s (customer VARCHAR, purchases INT, address VARCHAR) WITH (transactional = true)", targetTable)); onTrino().executeQuery(format("INSERT INTO %s (customer, purchases, address)" + " VALUES ('Dave', 11, 'Devon'), ('Dave', 11, 'Darbyshire')", targetTable)); - withTemporaryTable("test_without_aliases_source", true, false, NONE, sourceTable -> { + withTemporaryTable("test_without_aliases_source", false, NONE, sourceTable -> { onTrino().executeQuery(format("CREATE TABLE %s (customer VARCHAR, purchases INT, address VARCHAR) WITH (transactional = true)", sourceTable)); onTrino().executeQuery(format("INSERT INTO %s (customer, purchases, address) VALUES ('Dave', 11, 'Darbyshire')", sourceTable)); @@ -611,12 +610,12 @@ public void testMergeWithSimplifiedUnpredictablePredicates() @Test(groups = HIVE_TRANSACTIONAL, timeOut = 60 * 60 * 1000) public void testMergeCasts() { - withTemporaryTable("merge_cast_target", true, false, NONE, targetTable -> { + withTemporaryTable("merge_cast_target", false, NONE, targetTable -> { onTrino().executeQuery(format("CREATE TABLE %s (col1 TINYINT, col2 SMALLINT, col3 INT, col4 BIGINT, col5 REAL, col6 DOUBLE) WITH (transactional = true)", targetTable)); onTrino().executeQuery(format("INSERT INTO %s VALUES (1, 2, 3, 4, 5, 6)", targetTable)); - withTemporaryTable("test_without_aliases_source", true, false, NONE, sourceTable -> { + withTemporaryTable("test_without_aliases_source", false, NONE, sourceTable -> { onTrino().executeQuery(format("CREATE TABLE %s (col1 DOUBLE, col2 REAL, col3 BIGINT, col4 INT, col5 SMALLINT, col6 TINYINT) WITH (transactional = true)", sourceTable)); onTrino().executeQuery(format("INSERT INTO %s VALUES (2, 3, 4, 5, 6, 7)", sourceTable)); @@ -633,12 +632,12 @@ public void testMergeCasts() @Test(groups = HIVE_TRANSACTIONAL, timeOut = 60 * 60 * 1000) public void testMergeSubqueries() { - withTemporaryTable("merge_nation_target", true, false, NONE, targetTable -> { + withTemporaryTable("merge_nation_target", false, NONE, targetTable -> { onTrino().executeQuery(format("CREATE TABLE %s (nation_name VARCHAR, region_name VARCHAR) WITH (transactional = true)", targetTable)); onTrino().executeQuery(format("INSERT INTO %s (nation_name, region_name) VALUES ('FRANCE', 'EUROPE'), ('ALGERIA', 'AFRICA'), ('GERMANY', 'EUROPE')", targetTable)); - withTemporaryTable("merge_nation_source", true, false, NONE, sourceTable -> { + withTemporaryTable("merge_nation_source", false, NONE, sourceTable -> { onTrino().executeQuery(format("CREATE TABLE %s (nation_name VARCHAR, region_name VARCHAR) WITH (transactional = true)", sourceTable)); onTrino().executeQuery(format("INSERT INTO %s VALUES ('ALGERIA', 'AFRICA'), ('FRANCE', 'EUROPE'), ('EGYPT', 'MIDDLE EAST'), ('RUSSIA', 'EUROPE')", sourceTable)); @@ -658,7 +657,7 @@ public void testMergeSubqueries() @Test(groups = HIVE_TRANSACTIONAL, timeOut = 60 * 60 * 1000) public void testMergeOriginalFilesTarget() { - withTemporaryTable("region", true, false, NONE, targetTable -> { + withTemporaryTable("region", false, NONE, targetTable -> { onTrino().executeQuery(format("CREATE TABLE %s WITH (transactional=true) AS TABLE tpch.tiny.region", targetTable)); // This merge is illegal, because many nations have the same region @@ -680,7 +679,7 @@ public void testMergeOriginalFilesTarget() @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) public void testMergeOverManySplits() { - withTemporaryTable("delete_select", true, false, NONE, targetTable -> { + withTemporaryTable("delete_select", false, NONE, targetTable -> { onTrino().executeQuery(format("CREATE TABLE %s (orderkey bigint, custkey bigint, orderstatus varchar(1), totalprice double, orderdate date, orderpriority varchar(15), clerk varchar(15), shippriority integer, comment varchar(79)) WITH (transactional = true)", targetTable)); onTrino().executeQuery(format("INSERT INTO %s SELECT * FROM tpch.\"sf0.1\".orders", targetTable)); @@ -698,7 +697,7 @@ public void testMergeOverManySplits() @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) public void testMergeFalseJoinCondition() { - withTemporaryTable("join_false", true, false, NONE, targetTable -> { + withTemporaryTable("join_false", false, NONE, targetTable -> { onTrino().executeQuery(format("CREATE TABLE %s (customer VARCHAR, purchases INT, address VARCHAR) WITH (transactional = true)", targetTable)); onTrino().executeQuery(format("INSERT INTO %s (customer, purchases, address) VALUES ('Aaron', 11, 'Antioch'), ('Bill', 7, 'Buena')", targetTable)); @@ -750,20 +749,10 @@ public Object[][] inserterAndDeleterProvider() }; } - void withTemporaryTable(String rootName, boolean transactional, boolean isPartitioned, BucketingType bucketingType, Consumer testRunner) + void withTemporaryTable(String rootName, boolean isPartitioned, BucketingType bucketingType, Consumer testRunner) { - if (transactional) { - ensureTransactionalHive(); - } try (TemporaryHiveTable table = TemporaryHiveTable.temporaryHiveTable(tableName(rootName, isPartitioned, bucketingType))) { testRunner.accept(table.getName()); } } - - private void ensureTransactionalHive() - { - if (getHiveVersionMajor() < 3) { - throw new SkipException("Hive transactional tables are supported with Hive version 3 or above"); - } - } } diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveTransactionalTable.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveTransactionalTable.java index 1d38d71c96653..a4b569aa55558 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveTransactionalTable.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveTransactionalTable.java @@ -27,7 +27,6 @@ import io.trino.tempto.query.QueryResult; import io.trino.testng.services.Flaky; import io.trino.tests.product.hive.util.TemporaryHiveTable; -import org.testng.SkipException; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @@ -143,10 +142,6 @@ public void testReadFullAcidBucketedV2() private void doTestReadFullAcid(boolean isPartitioned, BucketingType bucketingType) { - if (getHiveVersionMajor() < 3) { - throw new SkipException("Hive transactional tables are supported with Hive version 3 or above"); - } - try (TemporaryHiveTable table = TemporaryHiveTable.temporaryHiveTable(tableName("read_full_acid", isPartitioned, bucketingType))) { String tableName = table.getName(); onHive().executeQuery("CREATE TABLE " + tableName + " (col INT, fcol INT) " + @@ -236,10 +231,6 @@ public void testReadInsertOnlyTextWithCustomFormatProperties() private void testReadInsertOnly(boolean isPartitioned, BucketingType bucketingType, String hiveTableFormatDefinition) { - if (getHiveVersionMajor() < 3) { - throw new SkipException("Hive transactional tables are supported with Hive version 3 or above"); - } - try (TemporaryHiveTable table = TemporaryHiveTable.temporaryHiveTable(tableName("insert_only", isPartitioned, bucketingType))) { String tableName = table.getName(); @@ -285,10 +276,6 @@ private void testReadInsertOnly(boolean isPartitioned, BucketingType bucketingTy @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testReadFullAcidWithOriginalFiles(boolean isPartitioned, BucketingType bucketingType) { - if (getHiveVersionMajor() < 3) { - throw new SkipException("Trino Hive transactional tables are supported with Hive version 3 or above"); - } - String tableName = "test_full_acid_acid_converted_table_read"; onHive().executeQuery("DROP TABLE IF EXISTS " + tableName); verify(bucketingType.getHiveTableProperties().isEmpty()); // otherwise we would need to include that in the CREATE TABLE's TBLPROPERTIES @@ -333,7 +320,7 @@ public void testReadFullAcidWithOriginalFiles(boolean isPartitioned, BucketingTy @Flaky(issue = RETRYABLE_FAILURES_ISSUES, match = RETRYABLE_FAILURES_MATCH) public void testUpdateFullAcidWithOriginalFilesTrinoInserting(boolean isPartitioned, BucketingType bucketingType) { - withTemporaryTable("trino_update_full_acid_acid_converted_table_read", true, isPartitioned, bucketingType, tableName -> { + withTemporaryTable("trino_update_full_acid_acid_converted_table_read", isPartitioned, bucketingType, tableName -> { onHive().executeQuery("DROP TABLE IF EXISTS " + tableName); verify(bucketingType.getHiveTableProperties().isEmpty()); // otherwise we would need to include that in the CREATE TABLE's TBLPROPERTIES onHive().executeQuery("CREATE TABLE " + tableName + " (col INT, fcol INT) " + @@ -383,7 +370,7 @@ public void testUpdateFullAcidWithOriginalFilesTrinoInserting(boolean isPartitio @Flaky(issue = RETRYABLE_FAILURES_ISSUES, match = RETRYABLE_FAILURES_MATCH) public void testUpdateFullAcidWithOriginalFilesTrinoInsertingAndDeleting(boolean isPartitioned, BucketingType bucketingType) { - withTemporaryTable("trino_update_full_acid_acid_converted_table_read", true, isPartitioned, bucketingType, tableName -> { + withTemporaryTable("trino_update_full_acid_acid_converted_table_read", isPartitioned, bucketingType, tableName -> { onHive().executeQuery("DROP TABLE IF EXISTS " + tableName); verify(bucketingType.getHiveTableProperties().isEmpty()); // otherwise we would need to include that in the CREATE TABLE's TBLPROPERTIES onHive().executeQuery("CREATE TABLE " + tableName + " (col INT, fcol INT) " + @@ -435,10 +422,6 @@ String makeValues(int colStart, int colCount, int fcol, boolean isPartitioned, i @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testReadInsertOnlyWithOriginalFiles(boolean isPartitioned, BucketingType bucketingType) { - if (getHiveVersionMajor() < 3) { - throw new SkipException("Trino Hive transactional tables are supported with Hive version 3 or above"); - } - String tableName = "test_insert_only_acid_converted_table_read"; onHive().executeQuery("DROP TABLE IF EXISTS " + tableName); verify(bucketingType.getHiveTableProperties().isEmpty()); // otherwise we would need to include that in the CREATE TABLE's TBLPROPERTIES @@ -470,27 +453,6 @@ public void testReadInsertOnlyWithOriginalFiles(boolean isPartitioned, Bucketing } } - @Test(groups = HIVE_TRANSACTIONAL) - @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) - public void testFailAcidBeforeHive3() - { - if (getHiveVersionMajor() >= 3) { - throw new SkipException("This tests behavior of ACID table before Hive 3 "); - } - - try (TemporaryHiveTable table = TemporaryHiveTable.temporaryHiveTable("test_fail_acid_before_hive3_" + randomNameSuffix())) { - String tableName = table.getName(); - onHive().executeQuery("" + - "CREATE TABLE " + tableName + "(a bigint) " + - "CLUSTERED BY(a) INTO 4 BUCKETS " + - "STORED AS ORC " + - "TBLPROPERTIES ('transactional'='true')"); - - assertQueryFailure(() -> onTrino().executeQuery("SELECT * FROM " + tableName)) - .hasMessageContaining("Failed to open transaction. Transactional tables support requires Hive metastore version at least 3.0"); - } - } - @DataProvider public Object[][] partitioningAndBucketingTypeDataProvider() { @@ -515,10 +477,6 @@ public Object[][] partitioningAndBucketingTypeSmokeDataProvider() @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testCtasAcidTable(boolean isPartitioned, BucketingType bucketingType) { - if (getHiveVersionMajor() < 3) { - throw new SkipException("Hive transactional tables are supported with Hive version 3 or above"); - } - try (TemporaryHiveTable table = TemporaryHiveTable.temporaryHiveTable(format("ctas_transactional_%s", randomNameSuffix()))) { String tableName = table.getName(); onTrino().executeQuery("CREATE TABLE " + tableName + " " + @@ -539,7 +497,7 @@ public void testCtasAcidTable(boolean isPartitioned, BucketingType bucketingType @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testCreateAcidTable(boolean isPartitioned, BucketingType bucketingType) { - withTemporaryTable("create_transactional", true, isPartitioned, bucketingType, tableName -> { + withTemporaryTable("create_transactional", isPartitioned, bucketingType, tableName -> { onTrino().executeQuery("CREATE TABLE " + tableName + " (col INTEGER, fcol INTEGER, partcol INTEGER)" + trinoTableProperties(ACID, isPartitioned, bucketingType)); @@ -552,7 +510,7 @@ public void testCreateAcidTable(boolean isPartitioned, BucketingType bucketingTy @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidTableColumnNameConflict(String columnName) { - withTemporaryTable("acid_column_name_conflict", true, true, NONE, tableName -> { + withTemporaryTable("acid_column_name_conflict", true, NONE, tableName -> { onHive().executeQuery("CREATE TABLE " + tableName + " (`" + columnName + "` INTEGER, fcol INTEGER, partcol INTEGER) STORED AS ORC " + hiveTableProperties(ACID, NONE)); onTrino().executeQuery("INSERT INTO " + tableName + " VALUES (1, 2, 3)"); assertThat(onTrino().executeQuery("SELECT * FROM " + tableName)).containsOnly(row(1, 2, 3)); @@ -576,7 +534,7 @@ public Object[][] acidFormatColumnNames() @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testSimpleUnpartitionedTransactionalInsert() { - withTemporaryTable("unpartitioned_transactional_insert", true, false, NONE, tableName -> { + withTemporaryTable("unpartitioned_transactional_insert", false, NONE, tableName -> { onTrino().executeQuery(format("CREATE TABLE %s (column1 INT, column2 BIGINT) WITH (transactional = true)", tableName)); String insertQuery = format("INSERT INTO %s VALUES (11, 100), (12, 200), (13, 300)", tableName); @@ -599,7 +557,7 @@ public void testSimpleUnpartitionedTransactionalInsert() @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testTransactionalPartitionInsert() { - withTemporaryTable("transactional_partition_insert", true, true, NONE, tableName -> { + withTemporaryTable("transactional_partition_insert", true, NONE, tableName -> { onTrino().executeQuery(format("CREATE TABLE %s (column1 INT, column2 BIGINT) WITH (transactional = true, partitioned_by = ARRAY['column2'])", tableName)); onTrino().executeQuery(format("INSERT INTO %s (column2, column1) VALUES %s, %s", @@ -645,7 +603,7 @@ public void testTransactionalBucketedPartitionedInsertOnly() private void testTransactionalBucketedPartitioned(boolean insertOnly) { - withTemporaryTable("bucketed_partitioned_insert_only", true, true, BUCKETED_V2, tableName -> { + withTemporaryTable("bucketed_partitioned_insert_only", true, BUCKETED_V2, tableName -> { String insertOnlyProperty = insertOnly ? ", 'transactional_properties'='insert_only'" : ""; onHive().executeQuery(format("CREATE TABLE %s (purchase STRING) PARTITIONED BY (customer STRING) CLUSTERED BY (purchase) INTO 3 BUCKETS" + " STORED AS ORC TBLPROPERTIES ('transactional' = 'true'%s)", @@ -673,7 +631,7 @@ private void testTransactionalBucketedPartitioned(boolean insertOnly) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testTransactionalUnpartitionedDelete(Engine inserter, Engine deleter) { - withTemporaryTable("unpartitioned_delete", true, false, NONE, tableName -> { + withTemporaryTable("unpartitioned_delete", false, NONE, tableName -> { onTrino().executeQuery(format("CREATE TABLE %s (column1 INTEGER, column2 BIGINT) WITH (format = 'ORC', transactional = true)", tableName)); execute(inserter, format("INSERT INTO %s (column1, column2) VALUES (1, 100), (2, 200), (3, 300), (4, 400), (5, 500)", tableName)); execute(deleter, format("DELETE FROM %s WHERE column2 = 100", tableName)); @@ -692,7 +650,7 @@ public void testTransactionalUnpartitionedDelete(Engine inserter, Engine deleter @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testMultiDelete(Engine inserter, Engine deleter) { - withTemporaryTable("unpartitioned_multi_delete", true, false, NONE, tableName -> { + withTemporaryTable("unpartitioned_multi_delete", false, NONE, tableName -> { onTrino().executeQuery(format("CREATE TABLE %s (column1 INT, column2 BIGINT) WITH (transactional = true)", tableName)); execute(inserter, format("INSERT INTO %s VALUES (1, 100), (2, 200), (3, 300), (4, 400), (5, 500)", tableName)); execute(inserter, format("INSERT INTO %s VALUES (6, 600), (7, 700), (8, 800), (9, 900), (10, 1000)", tableName)); @@ -714,8 +672,8 @@ public void testReadAfterMultiInsertAndDelete() // statement id, when filtering out deleted rows. // // For more context see https://issues.apache.org/jira/browse/HIVE-16832 - withTemporaryTable("partitioned_multi_insert", true, true, BucketingType.BUCKETED_V1, tableName -> { - withTemporaryTable("tmp_data_table", false, false, NONE, dataTableName -> { + withTemporaryTable("partitioned_multi_insert", true, BucketingType.BUCKETED_V1, tableName -> { + withTemporaryTable("tmp_data_table", false, NONE, dataTableName -> { onTrino().executeQuery(format("CREATE TABLE %s (a int, b int, c varchar(5)) WITH " + "(transactional = true, partitioned_by = ARRAY['c'], bucketed_by = ARRAY['a'], bucket_count = 2)", tableName)); onTrino().executeQuery(format("CREATE TABLE %s (x int)", dataTableName)); @@ -737,7 +695,7 @@ public void testReadAfterMultiInsertAndDelete() @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testTransactionalMetadataDelete(Engine inserter, Engine deleter) { - withTemporaryTable("metadata_delete", true, true, NONE, tableName -> { + withTemporaryTable("metadata_delete", true, NONE, tableName -> { onTrino().executeQuery(format("CREATE TABLE %s (column1 INT, column2 BIGINT) WITH (transactional = true, partitioned_by = ARRAY['column2'])", tableName)); execute(inserter, format("INSERT INTO %s (column2, column1) VALUES %s, %s", tableName, @@ -753,7 +711,7 @@ public void testTransactionalMetadataDelete(Engine inserter, Engine deleter) @Flaky(issue = RETRYABLE_FAILURES_ISSUES, match = RETRYABLE_FAILURES_MATCH) public void testNonTransactionalMetadataDelete() { - withTemporaryTable("non_transactional_metadata_delete", false, true, NONE, tableName -> { + withTemporaryTable("non_transactional_metadata_delete", true, NONE, tableName -> { onTrino().executeQuery(format("CREATE TABLE %s (column2 BIGINT, column1 INT) WITH (partitioned_by = ARRAY['column1'])", tableName)); execute(Engine.TRINO, format("INSERT INTO %s (column1, column2) VALUES %s, %s", @@ -775,7 +733,7 @@ public void testNonTransactionalMetadataDelete() @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testUnpartitionedDeleteAll(Engine inserter, Engine deleter) { - withTemporaryTable("unpartitioned_delete_all", true, false, NONE, tableName -> { + withTemporaryTable("unpartitioned_delete_all", false, NONE, tableName -> { onTrino().executeQuery(format("CREATE TABLE %s (column1 INT, column2 BIGINT) WITH (transactional = true)", tableName)); execute(inserter, format("INSERT INTO %s VALUES (1, 100), (2, 200), (3, 300), (4, 400), (5, 500)", tableName)); execute(deleter, "DELETE FROM " + tableName); @@ -787,7 +745,7 @@ public void testUnpartitionedDeleteAll(Engine inserter, Engine deleter) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testMultiColumnDelete(Engine inserter, Engine deleter) { - withTemporaryTable("multi_column_delete", true, false, NONE, tableName -> { + withTemporaryTable("multi_column_delete", false, NONE, tableName -> { onTrino().executeQuery(format("CREATE TABLE %s (column1 INT, column2 BIGINT) WITH (transactional = true)", tableName)); execute(inserter, format("INSERT INTO %s VALUES (1, 100), (2, 200), (3, 300), (4, 400), (5, 500)", tableName)); String where = " WHERE column1 >= 2 AND column2 <= 400"; @@ -800,7 +758,7 @@ public void testMultiColumnDelete(Engine inserter, Engine deleter) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testPartitionAndRowsDelete(Engine inserter, Engine deleter) { - withTemporaryTable("partition_and_rows_delete", true, true, NONE, tableName -> { + withTemporaryTable("partition_and_rows_delete", true, NONE, tableName -> { onTrino().executeQuery("CREATE TABLE " + tableName + " (column2 BIGINT, column1 INT) WITH (transactional = true, partitioned_by = ARRAY['column1'])"); execute(inserter, format("INSERT INTO %s (column1, column2) VALUES (1, 100), (1, 200), (2, 300), (2, 400), (2, 500)", tableName)); @@ -814,7 +772,7 @@ public void testPartitionAndRowsDelete(Engine inserter, Engine deleter) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testPartitionedInsertAndRowLevelDelete(Engine inserter, Engine deleter) { - withTemporaryTable("partitioned_row_level_delete", true, true, NONE, tableName -> { + withTemporaryTable("partitioned_row_level_delete", true, NONE, tableName -> { onTrino().executeQuery(format("CREATE TABLE %s (column2 INT, column1 BIGINT) WITH (transactional = true, partitioned_by = ARRAY['column1'])", tableName)); execute(inserter, format("INSERT INTO %s (column1, column2) VALUES %s, %s", @@ -838,7 +796,7 @@ public void testPartitionedInsertAndRowLevelDelete(Engine inserter, Engine delet @Flaky(issue = RETRYABLE_FAILURES_ISSUES, match = RETRYABLE_FAILURES_MATCH) public void testBucketedPartitionedDelete(Engine inserter, Engine deleter) { - withTemporaryTable("bucketed_partitioned_delete", true, true, NONE, tableName -> { + withTemporaryTable("bucketed_partitioned_delete", true, NONE, tableName -> { onHive().executeQuery(format("CREATE TABLE %s (purchase STRING) PARTITIONED BY (customer STRING) CLUSTERED BY (purchase) INTO 3 BUCKETS STORED AS ORC TBLPROPERTIES ('transactional' = 'true')", tableName)); execute(inserter, format("INSERT INTO %s (customer, purchase) VALUES", tableName) + @@ -869,7 +827,7 @@ public void testBucketedPartitionedDelete(Engine inserter, Engine deleter) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testDeleteAllRowsInPartition() { - withTemporaryTable("bucketed_partitioned_delete", true, true, NONE, tableName -> { + withTemporaryTable("bucketed_partitioned_delete", true, NONE, tableName -> { onHive().executeQuery(format("CREATE TABLE %s (purchase STRING) PARTITIONED BY (customer STRING) STORED AS ORC TBLPROPERTIES ('transactional' = 'true')", tableName)); log.info("About to insert"); @@ -887,7 +845,7 @@ public void testDeleteAllRowsInPartition() @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testDeleteAfterDelete() { - withTemporaryTable("delete_after_delete", true, false, NONE, tableName -> { + withTemporaryTable("delete_after_delete", false, NONE, tableName -> { onTrino().executeQuery(format("CREATE TABLE %s (id INT) WITH (transactional = true)", tableName)); onTrino().executeQuery(format("INSERT INTO %s VALUES (1), (2), (3)", tableName)); @@ -906,7 +864,7 @@ public void testDeleteAfterDelete() @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testDeleteAfterDeleteWithPredicate() { - withTemporaryTable("delete_after_delete_predicate", true, false, NONE, tableName -> { + withTemporaryTable("delete_after_delete_predicate", false, NONE, tableName -> { onTrino().executeQuery(format("CREATE TABLE %s (id INT) WITH (transactional = true)", tableName)); onTrino().executeQuery(format("INSERT INTO %s VALUES (1), (2), (3)", tableName)); @@ -926,7 +884,7 @@ public void testDeleteAfterDeleteWithPredicate() @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testBucketedUnpartitionedDelete(Engine inserter, Engine deleter) { - withTemporaryTable("bucketed_unpartitioned_delete", true, true, NONE, tableName -> { + withTemporaryTable("bucketed_unpartitioned_delete", true, NONE, tableName -> { onHive().executeQuery(format("CREATE TABLE %s (customer STRING, purchase STRING) CLUSTERED BY (purchase) INTO 3 BUCKETS STORED AS ORC TBLPROPERTIES ('transactional' = 'true')", tableName)); execute(inserter, format("INSERT INTO %s (customer, purchase) VALUES", tableName) + @@ -957,7 +915,7 @@ public void testBucketedUnpartitionedDelete(Engine inserter, Engine deleter) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testDeleteOverManySplits() { - withTemporaryTable("delete_select", true, false, NONE, tableName -> { + withTemporaryTable("delete_select", false, NONE, tableName -> { onTrino().executeQuery(format("CREATE TABLE %s WITH (transactional = true) AS SELECT * FROM tpch.sf10.orders", tableName)); log.info("About to delete selected rows"); @@ -971,7 +929,7 @@ public void testDeleteOverManySplits() @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testCorrectSelectCountStar(Engine inserter, Engine deleter) { - withTemporaryTable("select_count_star_delete", true, true, NONE, tableName -> { + withTemporaryTable("select_count_star_delete", true, NONE, tableName -> { onHive().executeQuery(format("CREATE TABLE %s (col1 INT, col2 BIGINT) PARTITIONED BY (col3 STRING) STORED AS ORC TBLPROPERTIES ('transactional'='true')", tableName)); execute(inserter, format("INSERT INTO %s VALUES (1, 100, 'a'), (2, 200, 'b'), (3, 300, 'c'), (4, 400, 'a'), (5, 500, 'b'), (6, 600, 'c')", tableName)); @@ -985,7 +943,7 @@ public void testCorrectSelectCountStar(Engine inserter, Engine deleter) public void testInsertOnlyMultipleWriters(boolean bucketed, Engine inserter1, Engine inserter2) { log.info("testInsertOnlyMultipleWriters bucketed %s, inserter1 %s, inserter2 %s", bucketed, inserter1, inserter2); - withTemporaryTable("insert_only_partitioned", true, true, NONE, tableName -> { + withTemporaryTable("insert_only_partitioned", true, NONE, tableName -> { onHive().executeQuery(format("CREATE TABLE %s (col1 INT, col2 BIGINT) PARTITIONED BY (col3 STRING) %s STORED AS ORC TBLPROPERTIES ('transactional'='true', 'transactional_properties'='insert_only')", tableName, bucketed ? "CLUSTERED BY (col2) INTO 3 BUCKETS" : "")); @@ -1009,7 +967,7 @@ public void testInsertOnlyMultipleWriters(boolean bucketed, Engine inserter1, En @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testInsertFailsInExplicitTrinoTransaction() { - withTemporaryTable("insert_fail_explicit_transaction", true, false, NONE, tableName -> { + withTemporaryTable("insert_fail_explicit_transaction", false, NONE, tableName -> { onTrino().executeQuery(format("CREATE TABLE %s (a_string varchar) WITH (format = 'ORC', transactional = true)", tableName)); onTrino().executeQuery("START TRANSACTION"); assertQueryFailure(() -> onTrino().executeQuery(format("INSERT INTO %s (a_string) VALUES ('Commander Bun Bun')", tableName))) @@ -1021,7 +979,7 @@ public void testInsertFailsInExplicitTrinoTransaction() @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testUpdateFailsInExplicitTrinoTransaction() { - withTemporaryTable("update_fail_explicit_transaction", true, false, NONE, tableName -> { + withTemporaryTable("update_fail_explicit_transaction", false, NONE, tableName -> { onTrino().executeQuery(format("CREATE TABLE %s (a_string varchar) WITH (format = 'ORC', transactional = true)", tableName)); onTrino().executeQuery("START TRANSACTION"); assertQueryFailure(() -> onTrino().executeQuery(format("UPDATE %s SET a_string = 'Commander Bun Bun'", tableName))) @@ -1033,7 +991,7 @@ public void testUpdateFailsInExplicitTrinoTransaction() @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testDeleteFailsInExplicitTrinoTransaction() { - withTemporaryTable("delete_fail_explicit_transaction", true, false, NONE, tableName -> { + withTemporaryTable("delete_fail_explicit_transaction", false, NONE, tableName -> { onTrino().executeQuery(format("CREATE TABLE %s (a_string varchar) WITH (format = 'ORC', transactional = true)", tableName)); onTrino().executeQuery("START TRANSACTION"); assertQueryFailure(() -> onTrino().executeQuery(format("DELETE FROM %s WHERE a_string = 'Commander Bun Bun'", tableName))) @@ -1045,8 +1003,7 @@ public void testDeleteFailsInExplicitTrinoTransaction() @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testColumnRenamesOrcPartitioned(boolean transactional) { - ensureSchemaEvolutionSupported(); - withTemporaryTable("test_column_renames_partitioned", transactional, false, NONE, tableName -> { + withTemporaryTable("test_column_renames_partitioned", false, NONE, tableName -> { onTrino().executeQuery(format("CREATE TABLE %s (id BIGINT, old_name VARCHAR, age INT, old_state VARCHAR)" + " WITH (format = 'ORC', transactional = %s, partitioned_by = ARRAY['old_state'])", tableName, transactional)); testOrcColumnRenames(tableName); @@ -1061,8 +1018,7 @@ public void testColumnRenamesOrcPartitioned(boolean transactional) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testColumnRenamesOrcNotPartitioned(boolean transactional) { - ensureSchemaEvolutionSupported(); - withTemporaryTable("test_orc_column_renames_not_partitioned", transactional, false, NONE, tableName -> { + withTemporaryTable("test_orc_column_renames_not_partitioned", false, NONE, tableName -> { onTrino().executeQuery(format("CREATE TABLE %s (id BIGINT, old_name VARCHAR, age INT, old_state VARCHAR)" + " WITH (format = 'ORC', transactional = %s)", tableName, transactional)); testOrcColumnRenames(tableName); @@ -1095,8 +1051,7 @@ private void testOrcColumnRenames(String tableName) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testOrcColumnSwap(boolean transactional) { - ensureSchemaEvolutionSupported(); - withTemporaryTable("test_orc_column_renames", transactional, false, NONE, tableName -> { + withTemporaryTable("test_orc_column_renames", false, NONE, tableName -> { onTrino().executeQuery(format("CREATE TABLE %s (name VARCHAR, state VARCHAR) WITH (format = 'ORC', transactional = %s)", tableName, transactional)); onTrino().executeQuery(format("INSERT INTO %s VALUES ('Katy', 'CA'), ('Joe', 'WA')", tableName)); verifySelectForTrinoAndHive("SELECT * FROM " + tableName, row("Katy", "CA"), row("Joe", "WA")); @@ -1113,8 +1068,7 @@ public void testOrcColumnSwap(boolean transactional) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testBehaviorOnParquetColumnRenames() { - ensureSchemaEvolutionSupported(); - withTemporaryTable("test_parquet_column_renames", false, false, NONE, tableName -> { + withTemporaryTable("test_parquet_column_renames", false, NONE, tableName -> { onTrino().executeQuery(format("CREATE TABLE %s (id BIGINT, old_name VARCHAR, age INT, old_state VARCHAR) WITH (format = 'PARQUET', transactional = false)", tableName)); onTrino().executeQuery(format("INSERT INTO %s VALUES (111, 'Katy', 57, 'CA'), (222, 'Joe', 72, 'WA')", tableName)); verifySelectForTrinoAndHive("SELECT * FROM " + tableName, row(111, "Katy", 57, "CA"), row(222, "Joe", 72, "WA")); @@ -1137,8 +1091,7 @@ public void testBehaviorOnParquetColumnRenames() @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testOrcColumnDropAdd(boolean transactional) { - ensureSchemaEvolutionSupported(); - withTemporaryTable("test_orc_add_drop", transactional, false, NONE, tableName -> { + withTemporaryTable("test_orc_add_drop", false, NONE, tableName -> { onTrino().executeQuery(format("CREATE TABLE %s (id BIGINT, old_name VARCHAR, age INT, old_state VARCHAR) WITH (transactional = %s)", tableName, transactional)); onTrino().executeQuery(format("INSERT INTO %s VALUES (111, 'Katy', 57, 'CA'), (222, 'Joe', 72, 'WA')", tableName)); verifySelectForTrinoAndHive("SELECT * FROM " + tableName, row(111, "Katy", 57, "CA"), row(222, "Joe", 72, "WA")); @@ -1160,8 +1113,7 @@ public void testOrcColumnDropAdd(boolean transactional) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testOrcColumnTypeChange(boolean transactional) { - ensureSchemaEvolutionSupported(); - withTemporaryTable("test_orc_column_type_change", transactional, false, NONE, tableName -> { + withTemporaryTable("test_orc_column_type_change", false, NONE, tableName -> { onTrino().executeQuery(format("CREATE TABLE %s (id INT, old_name VARCHAR, age TINYINT, old_state VARCHAR) WITH (transactional = %s)", tableName, transactional)); onTrino().executeQuery(format("INSERT INTO %s VALUES (111, 'Katy', 57, 'CA'), (222, 'Joe', 72, 'WA')", tableName)); verifySelectForTrinoAndHive("SELECT * FROM " + tableName, row(111, "Katy", 57, "CA"), row(222, "Joe", 72, "WA")); @@ -1180,8 +1132,7 @@ public void testOrcColumnTypeChange(boolean transactional) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testParquetColumnDropAdd() { - ensureSchemaEvolutionSupported(); - withTemporaryTable("test_parquet_add_drop", false, false, NONE, tableName -> { + withTemporaryTable("test_parquet_add_drop", false, NONE, tableName -> { onTrino().executeQuery(format("CREATE TABLE %s (id BIGINT, old_name VARCHAR, age INT, state VARCHAR) WITH (format = 'PARQUET')", tableName)); onTrino().executeQuery(format("INSERT INTO %s VALUES (111, 'Katy', 57, 'CA'), (222, 'Joe', 72, 'WA')", tableName)); verifySelectForTrinoAndHive("SELECT * FROM " + tableName, row(111, "Katy", 57, "CA"), row(222, "Joe", 72, "WA")); @@ -1217,7 +1168,7 @@ public Object[][] transactionModeProvider() @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidUpdateFailNonTransactional() { - withTemporaryTable("update_fail_nontransactional", true, true, NONE, tableName -> { + withTemporaryTable("update_fail_nontransactional", true, NONE, tableName -> { onTrino().executeQuery(format("CREATE TABLE %s (customer VARCHAR, purchase VARCHAR)", tableName)); log.info("About to insert"); @@ -1233,7 +1184,7 @@ public void testAcidUpdateFailNonTransactional() @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidUpdateFailInsertOnlyTable() { - withTemporaryTable("update_fail_insert_only", true, false, NONE, tableName -> { + withTemporaryTable("update_fail_insert_only", false, NONE, tableName -> { onHive().executeQuery("CREATE TABLE " + tableName + " (customer STRING, purchase STRING) " + "STORED AS ORC " + hiveTableProperties(INSERT_ONLY, NONE)); @@ -1251,7 +1202,7 @@ public void testAcidUpdateFailInsertOnlyTable() @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidDeleteFailNonTransactional() { - withTemporaryTable("delete_fail_nontransactional", true, true, NONE, tableName -> { + withTemporaryTable("delete_fail_nontransactional", true, NONE, tableName -> { onTrino().executeQuery(format("CREATE TABLE %s (customer VARCHAR, purchase VARCHAR)", tableName)); log.info("About to insert"); @@ -1267,7 +1218,7 @@ public void testAcidDeleteFailNonTransactional() @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidDeleteFailInsertOnlyTable() { - withTemporaryTable("delete_fail_insert_only", true, false, NONE, tableName -> { + withTemporaryTable("delete_fail_insert_only", false, NONE, tableName -> { onHive().executeQuery("CREATE TABLE " + tableName + " (customer STRING, purchase STRING) " + "STORED AS ORC " + hiveTableProperties(INSERT_ONLY, NONE)); @@ -1285,7 +1236,7 @@ public void testAcidDeleteFailInsertOnlyTable() @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidUpdateSucceedUpdatingPartitionKey() { - withTemporaryTable("fail_update_partition_key", true, true, NONE, tableName -> { + withTemporaryTable("fail_update_partition_key", true, NONE, tableName -> { onTrino().executeQuery(format("CREATE TABLE %s (col1 INT, col2 VARCHAR, col3 BIGINT) WITH (transactional = true, partitioned_by = ARRAY['col3'])", tableName)); log.info("About to insert"); @@ -1303,7 +1254,7 @@ public void testAcidUpdateSucceedUpdatingPartitionKey() @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidUpdateSucceedUpdatingBucketColumn() { - withTemporaryTable("fail_update_bucket_column", true, true, NONE, tableName -> { + withTemporaryTable("fail_update_bucket_column", true, NONE, tableName -> { onHive().executeQuery(format("CREATE TABLE %s (customer STRING, purchase STRING) CLUSTERED BY (purchase) INTO 3 BUCKETS STORED AS ORC TBLPROPERTIES ('transactional' = 'true')", tableName)); log.info("About to insert"); @@ -1321,7 +1272,7 @@ public void testAcidUpdateSucceedUpdatingBucketColumn() @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidUpdateFailOnIllegalCast() { - withTemporaryTable("fail_update_on_illegal_cast", true, true, NONE, tableName -> { + withTemporaryTable("fail_update_on_illegal_cast", true, NONE, tableName -> { onTrino().executeQuery(format("CREATE TABLE %s (col1 INT, col2 VARCHAR, col3 BIGINT) WITH (transactional = true)", tableName)); log.info("About to insert"); @@ -1337,7 +1288,7 @@ public void testAcidUpdateFailOnIllegalCast() @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidUpdateSimple() { - withTemporaryTable("acid_update_simple", true, true, NONE, tableName -> { + withTemporaryTable("acid_update_simple", true, NONE, tableName -> { onTrino().executeQuery(format("CREATE TABLE %s (col1 TINYINT, col2 VARCHAR, col3 BIGINT, col4 BOOLEAN, col5 INT) WITH (transactional = true)", tableName)); log.info("About to insert"); onTrino().executeQuery(format("INSERT INTO %s (col1, col2, col3, col4, col5) VALUES (7, 'ONE', 1000, true, 101), (13, 'TWO', 2000, false, 202)", tableName)); @@ -1352,7 +1303,7 @@ public void testAcidUpdateSimple() @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidUpdateSelectedValues() { - withTemporaryTable("acid_update_simple_selected", true, true, NONE, tableName -> { + withTemporaryTable("acid_update_simple_selected", true, NONE, tableName -> { onTrino().executeQuery(format("CREATE TABLE %s (col1 TINYINT, col2 VARCHAR, col3 BIGINT, col4 BOOLEAN, col5 INT) WITH (transactional = true)", tableName)); log.info("About to insert"); onTrino().executeQuery(format("INSERT INTO %s (col1, col2, col3, col4, col5) VALUES (7, 'ONE', 1000, true, 101), (13, 'TWO', 2000, false, 202)", tableName)); @@ -1367,7 +1318,7 @@ public void testAcidUpdateSelectedValues() @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidUpdateCopyColumn() { - withTemporaryTable("acid_update_copy_column", true, true, NONE, tableName -> { + withTemporaryTable("acid_update_copy_column", true, NONE, tableName -> { onTrino().executeQuery(format("CREATE TABLE %s (col1 int, col2 int, col3 VARCHAR) WITH (transactional = true)", tableName)); log.info("About to insert"); onTrino().executeQuery(format("INSERT INTO %s (col1, col2, col3) VALUES (7, 15, 'ONE'), (13, 17, 'DEUX')", tableName)); @@ -1382,7 +1333,7 @@ public void testAcidUpdateCopyColumn() @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidUpdateSomeLiteralNullColumnValues() { - withTemporaryTable("update_some_literal_null_columns", true, true, NONE, tableName -> { + withTemporaryTable("update_some_literal_null_columns", true, NONE, tableName -> { onTrino().executeQuery(format("CREATE TABLE %s (col1 TINYINT, col2 VARCHAR, col3 BIGINT, col4 BOOLEAN, col5 INT) WITH (transactional = true)", tableName)); log.info("About to insert"); onTrino().executeQuery(format("INSERT INTO %s (col1, col2, col3, col4, col5) VALUES (1, 'ONE', 1000, true, 101), (2, 'TWO', 2000, false, 202)", tableName)); @@ -1401,7 +1352,7 @@ public void testAcidUpdateSomeLiteralNullColumnValues() @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidUpdateSomeComputedNullColumnValues() { - withTemporaryTable("update_some_computed_null_columns", true, true, NONE, tableName -> { + withTemporaryTable("update_some_computed_null_columns", true, NONE, tableName -> { onTrino().executeQuery(format("CREATE TABLE %s (col1 TINYINT, col2 VARCHAR, col3 BIGINT, col4 BOOLEAN, col5 INT) WITH (transactional = true)", tableName)); log.info("About to insert"); onTrino().executeQuery(format("INSERT INTO %s (col1, col2, col3, col4, col5) VALUES (1, 'ONE', 1000, true, 101), (2, 'TWO', 2000, false, 202)", tableName)); @@ -1421,7 +1372,7 @@ public void testAcidUpdateSomeComputedNullColumnValues() @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidUpdateAllLiteralNullColumnValues() { - withTemporaryTable("update_all_literal_null_columns", true, true, NONE, tableName -> { + withTemporaryTable("update_all_literal_null_columns", true, NONE, tableName -> { onTrino().executeQuery(format("CREATE TABLE %s (col1 TINYINT, col2 VARCHAR, col3 BIGINT, col4 BOOLEAN, col5 INT) WITH (transactional = true)", tableName)); log.info("About to insert"); onTrino().executeQuery(format("INSERT INTO %s (col1, col2, col3, col4, col5) VALUES (1, 'ONE', 1000, true, 101), (2, 'TWO', 2000, false, 202)", tableName)); @@ -1436,7 +1387,7 @@ public void testAcidUpdateAllLiteralNullColumnValues() @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidUpdateAllComputedNullColumnValues() { - withTemporaryTable("update_all_computed_null_columns", true, true, NONE, tableName -> { + withTemporaryTable("update_all_computed_null_columns", true, NONE, tableName -> { onTrino().executeQuery(format("CREATE TABLE %s (col1 TINYINT, col2 VARCHAR, col3 BIGINT, col4 BOOLEAN, col5 INT) WITH (transactional = true)", tableName)); log.info("About to insert"); onTrino().executeQuery(format("INSERT INTO %s (col1, col2, col3, col4, col5) VALUES (1, 'ONE', 1000, true, 101), (2, 'TWO', 2000, false, 202)", tableName)); @@ -1452,7 +1403,7 @@ public void testAcidUpdateAllComputedNullColumnValues() @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidUpdateReversed() { - withTemporaryTable("update_reversed", true, true, NONE, tableName -> { + withTemporaryTable("update_reversed", true, NONE, tableName -> { onTrino().executeQuery(format("CREATE TABLE %s (col1 TINYINT, col2 VARCHAR, col3 BIGINT, col4 BOOLEAN, col5 INT) WITH (transactional = true)", tableName)); log.info("About to insert"); onTrino().executeQuery(format("INSERT INTO %s (col1, col2, col3, col4, col5) VALUES (1, 'ONE', 1000, true, 101), (2, 'TWO', 2000, false, 202)", tableName)); @@ -1467,7 +1418,7 @@ public void testAcidUpdateReversed() @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidUpdatePermuted() { - withTemporaryTable("update_permuted", true, true, NONE, tableName -> { + withTemporaryTable("update_permuted", true, NONE, tableName -> { onTrino().executeQuery(format("CREATE TABLE %s (col1 TINYINT, col2 VARCHAR, col3 BIGINT, col4 BOOLEAN, col5 INT) WITH (transactional = true)", tableName)); log.info("About to insert"); onTrino().executeQuery(format("INSERT INTO %s (col1, col2, col3, col4, col5) VALUES (1, 'ONE', 1000, true, 101), (2, 'TWO', 2000, false, 202)", tableName)); @@ -1482,7 +1433,7 @@ public void testAcidUpdatePermuted() @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidUpdateAllColumnsSetAndDependencies() { - withTemporaryTable("update_all_columns_set", true, true, NONE, tableName -> { + withTemporaryTable("update_all_columns_set", true, NONE, tableName -> { onTrino().executeQuery(format("CREATE TABLE %s (col1 TINYINT, col2 INT, col3 BIGINT, col4 INT, col5 TINYINT) WITH (transactional = true)", tableName)); log.info("About to insert"); onTrino().executeQuery(format("INSERT INTO %s (col1, col2, col3, col4, col5) VALUES (1, 2, 3, 4, 5), (21, 22, 23, 24, 25)", tableName)); @@ -1497,7 +1448,7 @@ public void testAcidUpdateAllColumnsSetAndDependencies() @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidUpdatePartitioned() { - withTemporaryTable("update_partitioned", true, true, NONE, tableName -> { + withTemporaryTable("update_partitioned", true, NONE, tableName -> { onTrino().executeQuery(format("CREATE TABLE %s (col1 INT, col2 VARCHAR, col3 BIGINT) WITH (transactional = true, partitioned_by = ARRAY['col3'])", tableName)); log.info("About to insert"); @@ -1514,7 +1465,7 @@ public void testAcidUpdatePartitioned() @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidUpdateBucketed() { - withTemporaryTable("update_bucketed", true, true, NONE, tableName -> { + withTemporaryTable("update_bucketed", true, NONE, tableName -> { onHive().executeQuery(format("CREATE TABLE %s (customer STRING, purchase STRING) CLUSTERED BY (customer) INTO 3 BUCKETS STORED AS ORC TBLPROPERTIES ('transactional' = 'true')", tableName)); log.info("About to insert"); @@ -1531,7 +1482,7 @@ public void testAcidUpdateBucketed() @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidUpdateMajorCompaction() { - withTemporaryTable("schema_evolution_column_addition", true, false, NONE, tableName -> { + withTemporaryTable("schema_evolution_column_addition", false, NONE, tableName -> { onTrino().executeQuery(format("CREATE TABLE %s (column1 INT, column2 BIGINT) WITH (transactional = true)", tableName)); onTrino().executeQuery(format("INSERT INTO %s VALUES (11, 100)", tableName)); onTrino().executeQuery(format("INSERT INTO %s VALUES (22, 200)", tableName)); @@ -1553,7 +1504,7 @@ public void testAcidUpdateMajorCompaction() @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidUpdateWithSubqueryPredicate() { - withTemporaryTable("test_update_subquery", true, false, NONE, tableName -> { + withTemporaryTable("test_update_subquery", false, NONE, tableName -> { onTrino().executeQuery(format("CREATE TABLE %s (column1 INT, column2 varchar) WITH (transactional = true)", tableName)); onTrino().executeQuery(format("INSERT INTO %s VALUES (1, 'x')", tableName)); onTrino().executeQuery(format("INSERT INTO %s VALUES (2, 'y')", tableName)); @@ -1562,7 +1513,7 @@ public void testAcidUpdateWithSubqueryPredicate() onTrino().executeQuery(format("UPDATE %s SET column2 = 'row updated' WHERE column1 = (SELECT min(regionkey) + 1 FROM tpch.tiny.region)", tableName)); verifySelectForTrinoAndHive("SELECT * FROM " + tableName, row(1, "row updated"), row(2, "y")); - withTemporaryTable("second_table", true, false, NONE, secondTable -> { + withTemporaryTable("second_table", false, NONE, secondTable -> { onTrino().executeQuery(format("CREATE TABLE %s (regionkey bigint, name varchar(25), comment varchar(152)) WITH (transactional = true)", secondTable)); onTrino().executeQuery(format("INSERT INTO %s SELECT * FROM tpch.tiny.region", secondTable)); @@ -1584,7 +1535,7 @@ public void testAcidUpdateWithSubqueryPredicate() @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidUpdateWithSubqueryAssignment() { - withTemporaryTable("test_update_subquery", true, false, NONE, tableName -> { + withTemporaryTable("test_update_subquery", false, NONE, tableName -> { onTrino().executeQuery(format("CREATE TABLE %s (column1 INT, column2 varchar) WITH (transactional = true)", tableName)); onTrino().executeQuery(format("INSERT INTO %s VALUES (1, 'x')", tableName)); onTrino().executeQuery(format("INSERT INTO %s VALUES (2, 'y')", tableName)); @@ -1593,7 +1544,7 @@ public void testAcidUpdateWithSubqueryAssignment() onTrino().executeQuery(format("UPDATE %s SET column2 = (SELECT max(name) FROM tpch.tiny.region)", tableName)); verifySelectForTrinoAndHive("SELECT * FROM " + tableName, row(1, "MIDDLE EAST"), row(2, "MIDDLE EAST")); - withTemporaryTable("second_table", true, false, NONE, secondTable -> { + withTemporaryTable("second_table", false, NONE, secondTable -> { onTrino().executeQuery(format("CREATE TABLE %s (regionkey bigint, name varchar(25), comment varchar(152)) WITH (transactional = true)", secondTable)); onTrino().executeQuery(format("INSERT INTO %s SELECT * FROM tpch.tiny.region", secondTable)); @@ -1615,7 +1566,7 @@ public void testAcidUpdateWithSubqueryAssignment() @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidUpdateDuplicateUpdateValue() { - withTemporaryTable("test_update_bug", true, false, NONE, tableName -> { + withTemporaryTable("test_update_bug", false, NONE, tableName -> { onTrino().executeQuery( format("CREATE TABLE %s (", tableName) + " yyyy integer," + @@ -1651,7 +1602,7 @@ public void testAcidUpdateDuplicateUpdateValue() @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidUpdateMultipleDuplicateValues() { - withTemporaryTable("test_update_multiple", true, false, NONE, tableName -> { + withTemporaryTable("test_update_multiple", false, NONE, tableName -> { onTrino().executeQuery( format("CREATE TABLE %s (c1 int, c2 int, c3 int, c4 int, c5 int, c6 int) WITH (transactional = true)", tableName)); @@ -1678,7 +1629,7 @@ public void testAcidUpdateMultipleDuplicateValues() @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) public void testInsertDeleteUpdateWithTrinoAndHive() { - withTemporaryTable("update_insert_delete_trino_hive", true, true, NONE, tableName -> { + withTemporaryTable("update_insert_delete_trino_hive", true, NONE, tableName -> { onTrino().executeQuery(format("CREATE TABLE %s (col1 TINYINT, col2 INT, col3 BIGINT, col4 INT, col5 TINYINT) WITH (transactional = true)", tableName)); log.info("Performing first insert on Trino"); @@ -1715,7 +1666,7 @@ public void testInsertDeleteUpdateWithTrinoAndHive() @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testDeleteFromOriginalFiles() { - withTemporaryTable("delete_original_files", true, true, NONE, tableName -> { + withTemporaryTable("delete_original_files", true, NONE, tableName -> { onTrino().executeQuery(format("CREATE TABLE %s WITH (transactional = true, partitioned_by = ARRAY['regionkey'])" + " AS SELECT nationkey, name, regionkey FROM tpch.tiny.nation", tableName)); verifyOriginalFiles(tableName, "WHERE regionkey = 4"); @@ -1743,7 +1694,7 @@ public void testDeleteWholePartitionWithOriginalFiles() private void testDeleteWholePartition(boolean withOriginalFiles) { - withTemporaryTable("delete_partitioned", true, true, NONE, tableName -> { + withTemporaryTable("delete_partitioned", true, NONE, tableName -> { if (withOriginalFiles) { onTrino().executeQuery(format("CREATE TABLE %s WITH (transactional = true, partitioned_by = ARRAY['regionkey'])" + " AS SELECT nationkey, name, regionkey FROM tpch.tiny.nation", tableName)); @@ -1778,7 +1729,7 @@ private void testDeleteWholePartition(boolean withOriginalFiles) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testUpdateOriginalFilesPartitioned() { - withTemporaryTable("update_original_files", true, true, NONE, tableName -> { + withTemporaryTable("update_original_files", true, NONE, tableName -> { onTrino().executeQuery(format("CREATE TABLE %s WITH (transactional = true, partitioned_by = ARRAY['regionkey'])" + " AS SELECT nationkey, name, regionkey FROM tpch.tiny.nation", tableName)); verifyOriginalFiles(tableName, "WHERE regionkey = 4"); @@ -1792,7 +1743,7 @@ public void testUpdateOriginalFilesPartitioned() @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testUpdateOriginalFilesUnpartitioned() { - withTemporaryTable("update_original_files", true, true, NONE, tableName -> { + withTemporaryTable("update_original_files", true, NONE, tableName -> { onTrino().executeQuery(format("CREATE TABLE %s WITH (transactional = true)" + " AS SELECT nationkey, name, regionkey FROM tpch.tiny.nation", tableName)); verifyOriginalFiles(tableName, "WHERE regionkey = 4"); @@ -1806,7 +1757,7 @@ public void testUpdateOriginalFilesUnpartitioned() @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testInsertRowIdCorrectness() { - withTemporaryTable("test_insert_row_id_correctness", true, false, NONE, tableName -> { + withTemporaryTable("test_insert_row_id_correctness", false, NONE, tableName -> { // We use tpch.tiny.supplier because it is the smallest table that // is written as multiple pages by the ORC writer. If it stops // being split into pages, this test won't detect issues arising @@ -1875,11 +1826,8 @@ public Object[][] inserterAndDeleterProvider() }; } - void withTemporaryTable(String rootName, boolean transactional, boolean isPartitioned, BucketingType bucketingType, Consumer testRunner) + void withTemporaryTable(String rootName, boolean isPartitioned, BucketingType bucketingType, Consumer testRunner) { - if (transactional) { - ensureTransactionalHive(); - } try (TemporaryHiveTable table = TemporaryHiveTable.temporaryHiveTable(tableName(rootName, isPartitioned, bucketingType) + randomNameSuffix())) { testRunner.accept(table.getName()); } @@ -1890,10 +1838,6 @@ void withTemporaryTable(String rootName, boolean transactional, boolean isPartit public void testFilesForAbortedTransactionsIgnored() throws Exception { - if (getHiveVersionMajor() < 3) { - throw new SkipException("Hive transactional tables are supported with Hive version 3 or above"); - } - String tableName = "test_aborted_transaction_table"; onHive().executeQuery("" + "CREATE TABLE " + tableName + " (col INT) " + @@ -1949,7 +1893,7 @@ public void testFilesForAbortedTransactionsIgnored() @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testDoubleUpdateAndThenReadFromHive() { - withTemporaryTable("test_double_update", true, false, NONE, tableName -> { + withTemporaryTable("test_double_update", false, NONE, tableName -> { onTrino().executeQuery( "CREATE TABLE test_double_update ( " + "column1 INT, " + @@ -1970,7 +1914,7 @@ public void testDoubleUpdateAndThenReadFromHive() @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testDeleteWithOriginalFiles() { - withTemporaryTable("test_delete_with_original_files", true, false, NONE, tableName -> { + withTemporaryTable("test_delete_with_original_files", false, NONE, tableName -> { // these 3 properties are necessary to make sure there is more than 1 original file created onTrino().executeQuery("SET SESSION scale_writers = true"); onTrino().executeQuery("SET SESSION writer_scaling_min_data_processed = '4kB'"); @@ -1992,7 +1936,7 @@ public void testDeleteWithOriginalFiles() @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testDeleteWithOriginalFilesWithWhereClause() { - withTemporaryTable("test_delete_with_original_files_with_where_clause", true, false, NONE, tableName -> { + withTemporaryTable("test_delete_with_original_files_with_where_clause", false, NONE, tableName -> { // these 3 properties are necessary to make sure there is more than 1 original file created onTrino().executeQuery("SET SESSION scale_writers = true"); onTrino().executeQuery("SET SESSION writer_scaling_min_data_processed = '4kB'"); @@ -2030,7 +1974,7 @@ private void validateFileIsDirectlyUnderTableLocation(String tableName) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testDeleteAfterMajorCompaction() { - withTemporaryTable("test_delete_after_major_compaction", true, false, NONE, tableName -> { + withTemporaryTable("test_delete_after_major_compaction", false, NONE, tableName -> { onTrino().executeQuery(format("CREATE TABLE %s WITH (transactional = true) AS SELECT * FROM tpch.tiny.nation", tableName)); compactTableAndWait(MAJOR, tableName, "", new Duration(3, MINUTES)); onTrino().executeQuery(format("DELETE FROM %s", tableName)); @@ -2054,7 +1998,7 @@ public void testUnbucketedTransactionalTableWithTaskWriterCountGreaterThanOne() private void unbucketedTransactionalTableWithTaskWriterCountGreaterThanOne(boolean isPartitioned) { - withTemporaryTable(format("test_unbucketed%s_transactional_table_with_task_writer_count_greater_than_one", isPartitioned ? "_partitioned" : ""), true, isPartitioned, NONE, tableName -> { + withTemporaryTable(format("test_unbucketed%s_transactional_table_with_task_writer_count_greater_than_one", isPartitioned ? "_partitioned" : ""), isPartitioned, NONE, tableName -> { onTrino().executeQuery(format( "CREATE TABLE %s " + "WITH (" + @@ -2098,14 +2042,11 @@ private void unbucketedTransactionalTableWithTaskWriterCountGreaterThanOne(boole @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testLargePartitionedDelete() { - if (getHiveVersionMajor() < 3) { - throw new SkipException("Hive transactional tables are supported with Hive version 3 or above"); - } - withTemporaryTable("large_delete_" + "stage1", false, false, NONE, tableStage1 -> { + withTemporaryTable("large_delete_" + "stage1", false, NONE, tableStage1 -> { onTrino().executeQuery("CREATE TABLE %s AS SELECT a, b, 20220101 AS d FROM UNNEST(SEQUENCE(1, 9001), SEQUENCE(1, 9001)) AS t(a, b)".formatted(tableStage1)); - withTemporaryTable("large_delete_" + "stage2", false, false, NONE, tableStage2 -> { + withTemporaryTable("large_delete_" + "stage2", false, NONE, tableStage2 -> { onTrino().executeQuery("CREATE TABLE %s AS SELECT a, b, 20220101 AS d FROM UNNEST(SEQUENCE(1, 100), SEQUENCE(1, 100)) AS t(a, b)".formatted(tableStage2)); - withTemporaryTable("large_delete_" + "new", true, true, NONE, tableNew -> { + withTemporaryTable("large_delete_" + "new", true, NONE, tableNew -> { onTrino().executeQuery(""" CREATE TABLE %s WITH (transactional=true, partitioned_by=ARRAY['d']) AS (SELECT stage1.a as a, stage1.b as b, stage1.d AS d FROM %s stage1, %s stage2 WHERE stage1.d = stage2.d) @@ -2132,14 +2073,11 @@ public void testLargePartitionedDelete() @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testLargePartitionedUpdate() { - if (getHiveVersionMajor() < 3) { - throw new SkipException("Hive transactional tables are supported with Hive version 3 or above"); - } - withTemporaryTable("large_update_" + "stage1", false, false, NONE, tableStage1 -> { + withTemporaryTable("large_update_" + "stage1", false, NONE, tableStage1 -> { onTrino().executeQuery("CREATE TABLE %s AS SELECT a, b, 20220101 AS d FROM UNNEST(SEQUENCE(1, 9001), SEQUENCE(1, 9001)) AS t(a, b)".formatted(tableStage1)); - withTemporaryTable("large_update_" + "stage2", false, false, NONE, tableStage2 -> { + withTemporaryTable("large_update_" + "stage2", false, NONE, tableStage2 -> { onTrino().executeQuery("CREATE TABLE %s AS SELECT a, b, 20220101 AS d FROM UNNEST(SEQUENCE(1, 100), SEQUENCE(1, 100)) AS t(a, b)".formatted(tableStage2)); - withTemporaryTable("large_update_" + "new", true, true, NONE, tableNew -> { + withTemporaryTable("large_update_" + "new", true, NONE, tableNew -> { onTrino().executeQuery(""" CREATE TABLE %s WITH (transactional=true, partitioned_by=ARRAY['d']) AS (SELECT stage1.a as a, stage1.b as b, stage1.d AS d FROM %s stage1, %s stage2 WHERE stage1.d = stage2.d) @@ -2350,20 +2288,6 @@ private String makeInsertValues(int col1Value, int col2First, int col2Last) return IntStream.rangeClosed(col2First, col2Last).mapToObj(i -> format("(%s, %s)", col1Value, i)).collect(Collectors.joining(", ")); } - private void ensureTransactionalHive() - { - if (getHiveVersionMajor() < 3) { - throw new SkipException("Hive transactional tables are supported with Hive version 3 or above"); - } - } - - private void ensureSchemaEvolutionSupported() - { - if (getHiveVersionMajor() < 3) { - throw new SkipException("Hive schema evolution requires Hive version 3 or above"); - } - } - public static void verifySelectForTrinoAndHive(String select, Row... rows) { verifySelect("onTrino", onTrino(), select, rows); diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveTransactionalTableInsert.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveTransactionalTableInsert.java index 07cfa1f9dc4b6..3d81a206265c5 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveTransactionalTableInsert.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveTransactionalTableInsert.java @@ -13,7 +13,6 @@ */ package io.trino.tests.product.hive; -import org.testng.SkipException; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @@ -31,10 +30,6 @@ public class TestHiveTransactionalTableInsert @Test(dataProvider = "transactionalTableType", groups = HIVE_TRANSACTIONAL) public void testInsertIntoTransactionalTable(TransactionalTableType type) { - if (getHiveVersionMajor() < 3) { - throw new SkipException("Hive transactional tables are supported with Hive version 3 or above"); - } - String tableName = "test_insert_into_transactional_table_" + type.name().toLowerCase(ENGLISH); onHive().executeQuery("" + "CREATE TABLE " + tableName + "(a bigint)" + diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveViews.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveViews.java index 9a6815e8b5d1f..68f6c8ebcbbef 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveViews.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveViews.java @@ -72,14 +72,8 @@ private void testFailingHiveViewsWithInformationSchema() // is used, so Trino's information_schema.views table does not include translated Hive views. String withSchemaFilter = "SELECT table_name FROM information_schema.views WHERE table_schema = 'test_list_failing_views'"; String withNoFilter = "SELECT table_name FROM information_schema.views"; - if (getHiveVersionMajor() == 3) { - assertThat(onTrino().executeQuery(withSchemaFilter)).containsOnly(row("correct_view")); - assertThat(onTrino().executeQuery(withNoFilter)).contains(row("correct_view")); - } - else { - assertThat(onTrino().executeQuery(withSchemaFilter)).hasNoRows(); - assertThat(onTrino().executeQuery(withNoFilter).rows()).doesNotContain(ImmutableList.of("correct_view")); - } + assertThat(onTrino().executeQuery(withSchemaFilter)).containsOnly(row("correct_view")); + assertThat(onTrino().executeQuery(withNoFilter)).contains(row("correct_view")); // Queries with filters on table_schema and table_name are optimized to only fetch the specified table and uses // a different API. so the Hive version does not matter here. @@ -114,14 +108,8 @@ private void testFailingHiveViewsWithSystemJdbc() "table_schem = 'test_list_failing_views' AND " + "table_type = 'VIEW'"; String withNoFilter = "SELECT table_name FROM system.jdbc.tables WHERE table_cat = 'hive' AND table_type = 'VIEW'"; - if (getHiveVersionMajor() == 3) { - assertThat(onTrino().executeQuery(withSchemaFilter)).containsOnly(row("correct_view"), row("failing_view")); - assertThat(onTrino().executeQuery(withNoFilter)).contains(row("correct_view"), row("failing_view")); - } - else { - assertThat(onTrino().executeQuery(withSchemaFilter)).hasNoRows(); - assertThat(onTrino().executeQuery(withNoFilter).rows()).doesNotContain(ImmutableList.of("correct_view")); - } + assertThat(onTrino().executeQuery(withSchemaFilter)).containsOnly(row("correct_view"), row("failing_view")); + assertThat(onTrino().executeQuery(withNoFilter)).contains(row("correct_view"), row("failing_view")); // Queries with filters on table_schema and table_name are optimized to only fetch the specified table and uses // a different API. so the Hive version does not matter here. @@ -341,57 +329,28 @@ public void testFromUtcTimestamp() "1970-01-29 16:00:00.000")); // check result on Hive - if (isObsoleteFromUtcTimestampSemantics()) { - // For older hive version we expect different results on Hive side; as from_utc_timestamp semantics changed over time. - // Currently view transformation logic always follows new semantics. - // Leaving Hive assertions as documentation. - assertThat(onHive().executeQuery("SELECT * FROM test_from_utc_timestamp_view")) - .containsOnly(row( - "1969-12-31 21:30:00.123", - "1969-12-31 21:30:00.123", - "1969-12-31 21:30:10.123", - "1969-12-31 21:30:10.123", - "1970-01-03 21:30:00.123", - "1970-01-03 21:30:00.123", - "1970-01-30 21:30:00.123", - "1970-01-30 21:30:00.123", - "1970-01-30 21:30:00", - "1970-01-30 21:30:00", - "1970-01-30 21:30:00.123", - "1970-01-30 21:30:00.123", - "1970-01-30 21:30:00.123", - "1970-01-30 21:30:00.123", - "1970-01-30 21:30:00", - "1970-01-30 21:30:00", - "1970-01-30 08:00:00", - "1970-01-30 08:00:00", - "1970-01-29 16:00:00", - "1970-01-29 16:00:00")); - } - else { - assertThat(onHive().executeQuery("SELECT * FROM test_from_utc_timestamp_view")) - .containsOnly(row( - "1969-12-31 16:00:00.123", - "1969-12-31 16:00:00.123", - "1969-12-31 16:00:10.123", - "1969-12-31 16:00:10.123", - "1970-01-03 16:00:00.123", - "1970-01-03 16:00:00.123", - "1970-01-30 16:00:00.123", - "1970-01-30 16:00:00.123", - "1970-01-30 16:00:00", - "1970-01-30 16:00:00", - "1970-01-30 16:00:00.123", - "1970-01-30 16:00:00.123", - "1970-01-30 16:00:00.123", - "1970-01-30 16:00:00.123", - "1970-01-30 16:00:00", - "1970-01-30 16:00:00", - "1970-01-30 08:00:00", - "1970-01-30 08:00:00", - "1970-01-29 16:00:00", - "1970-01-29 16:00:00")); - } + assertThat(onHive().executeQuery("SELECT * FROM test_from_utc_timestamp_view")) + .containsOnly(row( + "1969-12-31 16:00:00.123", + "1969-12-31 16:00:00.123", + "1969-12-31 16:00:10.123", + "1969-12-31 16:00:10.123", + "1970-01-03 16:00:00.123", + "1970-01-03 16:00:00.123", + "1970-01-30 16:00:00.123", + "1970-01-30 16:00:00.123", + "1970-01-30 16:00:00", + "1970-01-30 16:00:00", + "1970-01-30 16:00:00.123", + "1970-01-30 16:00:00.123", + "1970-01-30 16:00:00.123", + "1970-01-30 16:00:00.123", + "1970-01-30 16:00:00", + "1970-01-30 16:00:00", + "1970-01-30 08:00:00", + "1970-01-30 08:00:00", + "1970-01-29 16:00:00", + "1970-01-29 16:00:00")); } @Test(groups = HIVE_VIEWS) @@ -485,34 +444,13 @@ public void testFromUtcTimestampCornerCases() row("2128-06-11 01:53:20.001")); // check result on Hive - if (isObsoleteFromUtcTimestampSemantics()) { - // For older hive version we expect different results on Hive side; as from_utc_timestamp semantics changed over time. - // Currently view transformation logic always follows new semantics. - // Leaving Hive assertions as documentation. - assertThat(onHive().executeQuery("SELECT * FROM test_from_utc_timestamp_corner_cases_view")) - .containsOnly( - row("1811-07-23 12:51:39.999"), // ??? - row("1938-04-24 19:43:19.999"), - row("1969-12-31 21:29:59.999"), - row("1969-12-31 21:30:00.001"), - row("2128-06-11 07:38:20.001")); - } - else { - assertThat(onHive().executeQuery("SELECT * FROM test_from_utc_timestamp_corner_cases_view")) - .containsOnly( - row("1811-07-23 07:13:41.999"), - row("1938-04-24 14:13:19.999"), - row("1969-12-31 15:59:59.999"), - row("1969-12-31 16:00:00.001"), - row("2128-06-11 01:53:20.001")); - } - } - - private boolean isObsoleteFromUtcTimestampSemantics() - { - // It appears from_utc_timestamp semantics in Hive changes some time on the way. The guess is that it happened - // together with change of timestamp semantics at version 3.1. - return getHiveVersionMajor() < 3 || (getHiveVersionMajor() == 3 && getHiveVersionMinor() < 1); + assertThat(onHive().executeQuery("SELECT * FROM test_from_utc_timestamp_corner_cases_view")) + .containsOnly( + row("1811-07-23 07:13:41.999"), + row("1938-04-24 14:13:19.999"), + row("1969-12-31 15:59:59.999"), + row("1969-12-31 16:00:00.001"), + row("2128-06-11 01:53:20.001")); } @Test(groups = HIVE_VIEWS) @@ -526,17 +464,7 @@ public void testCastTimestampAsDecimal() onHive().executeQuery("CREATE VIEW cast_timestamp_as_decimal_view AS SELECT CAST(a_timestamp as DECIMAL(10,0)) a_cast_timestamp FROM cast_timestamp_as_decimal"); String testQuery = "SELECT * FROM cast_timestamp_as_decimal_view"; - if (getHiveVersionMajor() > 3 || (getHiveVersionMajor() == 3 && getHiveVersionMinor() >= 1)) { - assertViewQuery( - testQuery, - queryAssert -> queryAssert.containsOnly(row(new BigDecimal("631282394")))); - } - else { - // For Hive versions older than 3.1 semantics of cast timestamp to decimal is different and it takes into account timezone Hive VM uses. - // We cannot replicate the behaviour in Trino, hence test only documents different expected results. - assertThat(onTrino().executeQuery(testQuery)).containsOnly(row(new BigDecimal("631282394"))); - assertThat(onHive().executeQuery(testQuery)).containsOnly(row(new BigDecimal("631261694"))); - } + assertViewQuery(testQuery, queryAssert -> queryAssert.containsOnly(row(new BigDecimal("631282394")))); onHive().executeQuery("DROP VIEW cast_timestamp_as_decimal_view"); onHive().executeQuery("DROP TABLE cast_timestamp_as_decimal"); diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveViewsLegacy.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveViewsLegacy.java index 5aa21f74550b4..4ca69c9e76ed9 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveViewsLegacy.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveViewsLegacy.java @@ -72,12 +72,10 @@ public void testHiveViewInInformationSchema() onTrino().executeQuery("CREATE TABLE test_schema.trino_table(a int)"); onTrino().executeQuery("CREATE VIEW test_schema.trino_test_view AS SELECT * FROM nation"); - boolean hiveWithTableNamesByType = getHiveVersionMajor() >= 3 || - (getHiveVersionMajor() == 2 && getHiveVersionMinor() >= 3); assertThat(onTrino().executeQuery("SELECT * FROM information_schema.tables WHERE table_schema = 'test_schema'")).containsOnly( row("hive", "test_schema", "trino_table", "BASE TABLE"), row("hive", "test_schema", "hive_table", "BASE TABLE"), - row("hive", "test_schema", "hive_test_view", hiveWithTableNamesByType ? "VIEW" : "BASE TABLE"), + row("hive", "test_schema", "hive_test_view", "VIEW"), row("hive", "test_schema", "trino_test_view", "VIEW")); assertThat(onTrino().executeQuery("SELECT view_definition FROM information_schema.views WHERE table_schema = 'test_schema' and table_name = 'hive_test_view'")).containsOnly( diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestReadUniontype.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestReadUniontype.java deleted file mode 100644 index c061c57ca798e..0000000000000 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestReadUniontype.java +++ /dev/null @@ -1,519 +0,0 @@ -/* - * 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.tests.product.hive; - -import io.trino.jdbc.Row; -import io.trino.tempto.AfterMethodWithContext; -import io.trino.tempto.BeforeMethodWithContext; -import io.trino.tempto.query.QueryResult; -import io.trino.testng.services.Flaky; -import org.testng.SkipException; -import org.testng.annotations.DataProvider; -import org.testng.annotations.Test; - -import java.util.Arrays; -import java.util.List; - -import static io.trino.testing.TestingNames.randomNameSuffix; -import static io.trino.tests.product.TestGroups.AVRO; -import static io.trino.tests.product.TestGroups.SMOKE; -import static io.trino.tests.product.utils.HadoopTestUtils.RETRYABLE_FAILURES_ISSUES; -import static io.trino.tests.product.utils.HadoopTestUtils.RETRYABLE_FAILURES_MATCH; -import static io.trino.tests.product.utils.QueryExecutors.onHive; -import static io.trino.tests.product.utils.QueryExecutors.onTrino; -import static java.lang.String.format; -import static org.assertj.core.api.Assertions.assertThat; -import static org.testng.Assert.assertEquals; - -public class TestReadUniontype - extends HiveProductTest -{ - private static final String TABLE_NAME = "test_read_uniontype"; - private static final String TABLE_NAME_SCHEMA_EVOLUTION = "test_read_uniontype_with_schema_evolution"; - - @BeforeMethodWithContext - @AfterMethodWithContext - public void cleanup() - { - onHive().executeQuery(format("DROP TABLE IF EXISTS %s", TABLE_NAME)); - onHive().executeQuery(format("DROP TABLE IF EXISTS %s", TABLE_NAME_SCHEMA_EVOLUTION)); - } - - @DataProvider(name = "storage_formats") - public static Object[][] storageFormats() - { - return new String[][] {{"ORC"}, {"AVRO"}}; - } - - @DataProvider(name = "union_dereference_test_cases") - public static Object[][] unionDereferenceTestCases() - { - String tableUnionDereference = "test_union_dereference" + randomNameSuffix(); - // Hive insertion for union type in AVRO format has bugs, so we test on different table schemas for AVRO than ORC. - return new Object[][] {{ - format( - "CREATE TABLE %s (unionLevel0 UNIONTYPE<" + - "INT, STRING>)" + - "STORED AS %s", - tableUnionDereference, - "AVRO"), - format( - "INSERT INTO TABLE %s " + - "SELECT create_union(0, 321, 'row1') " + - "UNION ALL " + - "SELECT create_union(1, 55, 'row2') ", - tableUnionDereference), - format("SELECT unionLevel0.field0 FROM %s WHERE unionLevel0.field0 IS NOT NULL", tableUnionDereference), - Arrays.asList(321), - format("SELECT unionLevel0.tag FROM %s", tableUnionDereference), - Arrays.asList((byte) 0, (byte) 1), - "DROP TABLE IF EXISTS " + tableUnionDereference}, - // there is an internal issue in Hive 1.2: - // unionLevel1 is declared as unionType, but has to be inserted by create_union(tagId, Int, String) - { - format( - "CREATE TABLE %s (unionLevel0 UNIONTYPE>>, intLevel0 INT )" + - "STORED AS %s", - tableUnionDereference, - "AVRO"), - format( - "INSERT INTO TABLE %s " + - "SELECT create_union(2, 321, 'row1', named_struct('intLevel1', 1, 'stringLevel1', 'structval', 'unionLevel1', create_union(0, 5, 'testString'))), 8 " + - "UNION ALL " + - "SELECT create_union(2, 321, 'row1', named_struct('intLevel1', 1, 'stringLevel1', 'structval', 'unionLevel1', create_union(1, 5, 'testString'))), 8 ", - tableUnionDereference), - format("SELECT unionLevel0.field2.unionLevel1.field1 FROM %s WHERE unionLevel0.field2.unionLevel1.field1 IS NOT NULL", tableUnionDereference), - Arrays.asList(5), - format("SELECT unionLevel0.field2.unionLevel1.tag FROM %s", tableUnionDereference), - Arrays.asList((byte) 0, (byte) 1), - "DROP TABLE IF EXISTS " + tableUnionDereference}, - { - format( - "CREATE TABLE %s (unionLevel0 UNIONTYPE<" + - "STRUCT>>)" + - "STORED AS %s", - tableUnionDereference, - "ORC"), - format( - "INSERT INTO TABLE %s " + - "SELECT create_union(0, named_struct('unionLevel1', create_union(0, 'testString1', 23))) " + - "UNION ALL " + - "SELECT create_union(0, named_struct('unionLevel1', create_union(1, 'testString2', 45))) ", - tableUnionDereference), - format("SELECT unionLevel0.field0.unionLevel1.field0 FROM %s WHERE unionLevel0.field0.unionLevel1.field0 IS NOT NULL", tableUnionDereference), - Arrays.asList("testString1"), - format("SELECT unionLevel0.field0.unionLevel1.tag FROM %s", tableUnionDereference), - Arrays.asList((byte) 0, (byte) 1), - "DROP TABLE IF EXISTS " + tableUnionDereference}, - { - format( - "CREATE TABLE %s (unionLevel0 UNIONTYPE>>, intLevel0 INT )" + - "STORED AS %s", - tableUnionDereference, - "ORC"), - format( - "INSERT INTO TABLE %s " + - "SELECT create_union(2, 321, 'row1', named_struct('intLevel1', 1, 'stringLevel1', 'structval', 'unionLevel1', create_union(0, 'testString', 5))), 8 " + - "UNION ALL " + - "SELECT create_union(2, 321, 'row1', named_struct('intLevel1', 1, 'stringLevel1', 'structval', 'unionLevel1', create_union(1, 'testString', 5))), 8 ", - tableUnionDereference), - format("SELECT unionLevel0.field2.unionLevel1.field0 FROM %s WHERE unionLevel0.field2.unionLevel1.field0 IS NOT NULL", tableUnionDereference), - Arrays.asList("testString"), - format("SELECT unionLevel0.field2.unionLevel1.tag FROM %s", tableUnionDereference), - Arrays.asList((byte) 0, (byte) 1), - "DROP TABLE IF EXISTS " + tableUnionDereference}}; - } - - @Test(dataProvider = "storage_formats", groups = {SMOKE, AVRO}) - @Flaky(issue = RETRYABLE_FAILURES_ISSUES, match = RETRYABLE_FAILURES_MATCH) - public void testReadUniontype(String storageFormat) - { - // According to testing results, the Hive INSERT queries here only work in Hive 1.2 - if (getHiveVersionMajor() != 1 || getHiveVersionMinor() != 2) { - throw new SkipException("This test can only be run with Hive 1.2 (default config)"); - } - - onHive().executeQuery(format( - "CREATE TABLE %s (id INT,foo UNIONTYPE<" + - "INT," + - "DOUBLE," + - "ARRAY>)" + - "STORED AS %s", - TABLE_NAME, - storageFormat)); - - // Generate a file with rows: - // 0, {0: 36} - // 1, {1: 7.2} - // 2, {2: ['foo', 'bar']} - // 3, {1: 10.8} - // 4, {0: 144} - // 5, {2: ['hello'] - onHive().executeQuery(format( - "INSERT INTO TABLE %s " + - "SELECT 0, create_union(0, CAST(36 AS INT), CAST(NULL AS DOUBLE), ARRAY('foo','bar')) " + - "UNION ALL " + - "SELECT 1, create_union(1, CAST(NULL AS INT), CAST(7.2 AS DOUBLE), ARRAY('foo','bar')) " + - "UNION ALL " + - "SELECT 2, create_union(2, CAST(NULL AS INT), CAST(NULL AS DOUBLE), ARRAY('foo','bar')) " + - "UNION ALL " + - "SELECT 3, create_union(1, CAST(NULL AS INT), CAST(10.8 AS DOUBLE), ARRAY('foo','bar')) " + - "UNION ALL " + - "SELECT 4, create_union(0, CAST(144 AS INT), CAST(NULL AS DOUBLE), ARRAY('foo','bar')) " + - "UNION ALL " + - "SELECT 5, create_union(2, CAST(NULL AS INT), CAST(NULL AS DOUBLE), ARRAY('hello', 'world'))", - TABLE_NAME)); - // Generate a file with rows: - // 6, {0: 180} - // 7, {1: 21.6} - // 8, {0: 252} - onHive().executeQuery(format( - "INSERT INTO TABLE %s " + - "SELECT 6, create_union(0, CAST(180 AS INT), CAST(NULL AS DOUBLE), ARRAY('foo','bar')) " + - "UNION ALL " + - "SELECT 7, create_union(1, CAST(NULL AS INT), CAST(21.6 AS DOUBLE), ARRAY('foo','bar')) " + - "UNION ALL " + - "SELECT 8, create_union(0, CAST(252 AS INT), CAST(NULL AS DOUBLE), ARRAY('foo','bar'))", - TABLE_NAME)); - QueryResult selectAllResult = onTrino().executeQuery(format("SELECT * FROM %s", TABLE_NAME)); - assertEquals(selectAllResult.rows().size(), 9); - for (List row : selectAllResult.rows()) { - int id = (Integer) row.get(0); - switch (id) { - case 0: - assertStructEquals(row.get(1), new Object[] {(byte) 0, 36, null, null}); - break; - case 1: - assertStructEquals(row.get(1), new Object[] {(byte) 1, null, 7.2D, null}); - break; - case 2: - assertStructEquals(row.get(1), new Object[] {(byte) 2, null, null, Arrays.asList("foo", "bar")}); - break; - case 3: - assertStructEquals(row.get(1), new Object[] {(byte) 1, null, 10.8D, null}); - break; - case 4: - assertStructEquals(row.get(1), new Object[] {(byte) 0, 144, null, null}); - break; - case 5: - assertStructEquals(row.get(1), new Object[] {(byte) 2, null, null, Arrays.asList("hello", "world")}); - break; - case 6: - assertStructEquals(row.get(1), new Object[] {(byte) 0, 180, null, null}); - break; - case 7: - assertStructEquals(row.get(1), new Object[] {(byte) 1, null, 21.6, null}); - break; - case 8: - assertStructEquals(row.get(1), new Object[] {(byte) 0, 252, null, null}); - break; - } - } - } - - @Test(dataProvider = "union_dereference_test_cases", groups = {SMOKE, AVRO}) - @Flaky(issue = RETRYABLE_FAILURES_ISSUES, match = RETRYABLE_FAILURES_MATCH) - public void testReadUniontypeWithDereference(String createTableSql, String insertSql, String selectSql, List expectedResult, String selectTagSql, List expectedTagResult, String dropTableSql) - { - // According to testing results, the Hive INSERT queries here only work in Hive 1.2 - if (getHiveVersionMajor() != 1 || getHiveVersionMinor() != 2) { - throw new SkipException("This test can only be run with Hive 1.2 (default config)"); - } - - onHive().executeQuery(createTableSql); - onHive().executeQuery(insertSql); - - QueryResult result = onTrino().executeQuery(selectSql); - assertThat(result.column(1)).containsExactlyInAnyOrderElementsOf(expectedResult); - result = onTrino().executeQuery(selectTagSql); - assertThat(result.column(1)).containsExactlyInAnyOrderElementsOf(expectedTagResult); - - onTrino().executeQuery(dropTableSql); - } - - @Test(dataProvider = "storage_formats", groups = {SMOKE, AVRO}) - @Flaky(issue = RETRYABLE_FAILURES_ISSUES, match = RETRYABLE_FAILURES_MATCH) - public void testUnionTypeSchemaEvolution(String storageFormat) - { - // According to testing results, the Hive INSERT queries here only work in Hive 1.2 - if (getHiveVersionMajor() != 1 || getHiveVersionMinor() != 2) { - throw new SkipException("This test can only be run with Hive 1.2 (default config)"); - } - - onHive().executeQuery(format( - "CREATE TABLE %s (" - + "c0 INT," - + "c1 UNIONTYPE<" - + " STRUCT, " - + " STRUCT>) " - + "PARTITIONED BY (c2 INT) " - + "STORED AS %s", - TABLE_NAME_SCHEMA_EVOLUTION, - storageFormat)); - switch (storageFormat) { - case "AVRO": - testAvroSchemaEvolution(); - break; - case "ORC": - testORCSchemaEvolution(); - break; - default: - throw new UnsupportedOperationException("Unsupported table format."); - } - } - - /** - * When reading AVRO file, Trino needs the schema information from Hive metastore to deserialize Avro files. - * Therefore, when an ALTER table was issued in which the hive metastore changed the schema into an incompatible format, - * from Union to Struct or from Struct to Union in this case, Trino could not read those Avro files using the modified Hive metastore schema. - * However, when reading ORC files, Trino does not need schema information from Hive metastore to deserialize ORC files. - * Therefore, it can read ORC files even after changing the schema. - */ - @Test(groups = SMOKE) - @Flaky(issue = RETRYABLE_FAILURES_ISSUES, match = RETRYABLE_FAILURES_MATCH) - public void testORCUnionToStructSchemaEvolution() - { - // According to testing results, the Hive INSERT queries here only work in Hive 1.2 - if (getHiveVersionMajor() != 1 || getHiveVersionMinor() != 2) { - throw new SkipException("This test can only be run with Hive 1.2 (default config)"); - } - String tableReadUnionAsStruct = "test_read_union_as_struct_" + randomNameSuffix(); - - onHive().executeQuery("SET hive.exec.dynamic.partition.mode = nonstrict"); - onHive().executeQuery("SET hive.exec.dynamic.partition=true"); - - onHive().executeQuery(format( - "CREATE TABLE %s(" + - "c1 UNIONTYPE, STRUCT>) " + - "PARTITIONED BY (p INT) STORED AS %s", - tableReadUnionAsStruct, - "ORC")); - - onHive().executeQuery(format("INSERT INTO TABLE %s PARTITION(p) " + - "SELECT CREATE_UNION(1, NAMED_STRUCT('a', 'a1', 'b', 'b1'), NAMED_STRUCT('c', 'ignores', 'd', 'ignore')), 999 FROM (SELECT 1) t", - tableReadUnionAsStruct)); - - onHive().executeQuery(format("ALTER TABLE %s CHANGE COLUMN c1 c1 " + - " STRUCT, field1:STRUCT>", - tableReadUnionAsStruct)); - - onHive().executeQuery(format("INSERT INTO TABLE %s PARTITION(p) " + - "SELECT NAMED_STRUCT('tag', 0, 'field0', NAMED_STRUCT('a', 'a11', 'b', 'b1b'), 'field1', NAMED_STRUCT('c', 'ignores', 'd', 'ignores')), 100 FROM (SELECT 1) t", - tableReadUnionAsStruct)); - // using dereference - QueryResult selectAllResult = onTrino().executeQuery(format("SELECT c1.field0 FROM hive.default.%s", tableReadUnionAsStruct)); - // the first insert didn't add value to field0, since the tag is 1 during inserting - assertThat(selectAllResult.column(1)).containsExactlyInAnyOrder(null, Row.builder().addField("a", "a11").addField("b", "b1b").build()); - } - - /** - * When reading AVRO file, Trino needs the schema information from Hive metastore to deserialize Avro files. - * Therefore, when an ALTER table was issued in which the hive metastore changed the schema into an incompatible format, - * from Union to Struct or from Struct to Union in this case, Trino could not read those Avro files using the modified Hive metastore schema. - * However, when reading ORC files, Trino does not need schema information from Hive metastore to deserialize ORC files. - * Therefore, it can read ORC files even after changing the schema. - */ - @Test(groups = SMOKE) - @Flaky(issue = RETRYABLE_FAILURES_ISSUES, match = RETRYABLE_FAILURES_MATCH) - public void testORCStructToUnionSchemaEvolution() - { - // According to testing results, the Hive INSERT queries here only work in Hive 1.2 - if (getHiveVersionMajor() != 1 || getHiveVersionMinor() != 2) { - throw new SkipException("This test can only be run with Hive 1.2 (default config)"); - } - String tableReadStructAsUnion = "test_read_struct_as_union_" + randomNameSuffix(); - - onHive().executeQuery("SET hive.exec.dynamic.partition.mode = nonstrict"); - onHive().executeQuery("SET hive.exec.dynamic.partition=true"); - - onHive().executeQuery(format( - "CREATE TABLE %s(" + - "c1 STRUCT, field1:STRUCT>) " + - "PARTITIONED BY (p INT) STORED AS %s", - tableReadStructAsUnion, - "ORC")); - - onHive().executeQuery(format("INSERT INTO TABLE %s PARTITION(p) " + - "SELECT NAMED_STRUCT('tag', 0Y, 'field0', NAMED_STRUCT('a', 'a11', 'b', 'b1b'), 'field1', NAMED_STRUCT('c', 'ignores', 'd', 'ignores')), 100 FROM (SELECT 1) t", - tableReadStructAsUnion)); - - onHive().executeQuery(format("ALTER TABLE %s CHANGE COLUMN c1 c1 " + - " UNIONTYPE, STRUCT>", - tableReadStructAsUnion)); - - onHive().executeQuery(format("INSERT INTO TABLE %s PARTITION(p) " + - "SELECT CREATE_UNION(1, NAMED_STRUCT('a', 'a1', 'b', 'b1'), NAMED_STRUCT('c', 'ignores', 'd', 'ignore')), 999 from (SELECT 1) t", - tableReadStructAsUnion)); - - // using dereference - QueryResult selectAllResult = onTrino().executeQuery(format("SELECT c1.field0 FROM hive.default.%s", tableReadStructAsUnion)); - // the second insert didn't add value to field0, since the tag is 1 during inserting - assertThat(selectAllResult.column(1)).containsExactlyInAnyOrder(null, Row.builder().addField("a", "a11").addField("b", "b1b").build()); - } - - @Test(groups = SMOKE) - @Flaky(issue = RETRYABLE_FAILURES_ISSUES, match = RETRYABLE_FAILURES_MATCH) - public void testReadOrcUniontypeWithCheckpoint() - { - // According to testing results, the Hive INSERT queries here only work in Hive 1.2 - if (getHiveVersionMajor() != 1 || getHiveVersionMinor() != 2) { - throw new SkipException("This test can only be run with Hive 1.2 (default config)"); - } - - // Set the row group size to 1000 (the minimum value). - onHive().executeQuery(format( - "CREATE TABLE %s (id INT,foo UNIONTYPE<" + - "INT," + - "DOUBLE," + - "ARRAY>)" + - "STORED AS ORC TBLPROPERTIES (\"orc.row.index.stride\"=\"1000\")", - TABLE_NAME)); - - // Generate a file with 1100 rows, as the default row group size is 1000, reading 1100 rows will involve - // streaming checkpoint. - StringBuilder builder = new StringBuilder(); - for (int i = 0; i < 1100; i++) { - builder.append("SELECT 0, create_union(0, CAST(36 AS INT), CAST(NULL AS DOUBLE), ARRAY('foo','bar')) "); - if (i < 1099) { - builder.append("UNION ALL "); - } - } - onHive().executeQuery(format( - "INSERT INTO TABLE %s " + builder.toString(), TABLE_NAME)); - - QueryResult selectAllResult = onTrino().executeQuery(format("SELECT * FROM %s", TABLE_NAME)); - assertEquals(selectAllResult.rows().size(), 1100); - } - - private void testORCSchemaEvolution() - { - // Generate a file with rows: - // 0, {0: } - // 1, {1: } - onHive().executeQuery(format("INSERT INTO TABLE %s PARTITION (c2 = 5) " - + "SELECT 0, create_union(0, named_struct('a', 'a1', 'b', 'b1'), named_struct('c', 'ignore')) " - + "UNION ALL " - + "SELECT 1, create_union(1, named_struct('a', 'ignore', 'b', 'ignore'), named_struct('c', 'c1'))", - TABLE_NAME_SCHEMA_EVOLUTION)); - - // Add a coercible change inside union type column. - onHive().executeQuery(format("ALTER TABLE %S CHANGE COLUMN c1 c1 UNIONTYPE, STRUCT>", - TABLE_NAME_SCHEMA_EVOLUTION)); - - QueryResult selectAllResult = onTrino().executeQuery(format("SELECT c0, c1 FROM %s", TABLE_NAME_SCHEMA_EVOLUTION)); - assertEquals(selectAllResult.rows().size(), 2); - for (List row : selectAllResult.rows()) { - int id = (Integer) row.get(0); - switch (id) { - case 0: - Row rowValueFirst = rowBuilder().addField("a", "a1").addField("b", "b1").build(); - assertStructEquals(row.get(1), new Object[]{(byte) 0, rowValueFirst, null}); - break; - case 1: - Row rowValueSecond = rowBuilder().addField("c", "c1").addField("d", null).build(); - assertStructEquals(row.get(1), new Object[]{(byte) 1, null, rowValueSecond}); - break; - } - } - } - - private void testAvroSchemaEvolution() - { - /** - * The following insertion fails on avro. - * - * hive (default)> INSERT INTO TABLE u_username.test_ut_avro partition (c2 = 5) - * > SELECT 1, create_union(1, named_struct('a', 'ignore', 'b', 'ignore'), named_struct('c', 'c1')); - * - * Error: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing writable (null) - * at org.apache.hadoop.hive.ql.exec.mr.ExecMapper.map(ExecMapper.java:179) - * at org.apache.hadoop.mapred.MapRunner.run(MapRunner.java:54) - * at org.apache.hadoop.mapred.MapTask.runOldMapper(MapTask.java:459) - * at org.apache.hadoop.mapred.MapTask.run(MapTask.java:343) - * at org.apache.hadoop.mapred.YarnChild$2.run(YarnChild.java:177) - * at java.security.AccessController.doPrivileged(Native Method) - * at javax.security.auth.Subject.doAs(Subject.java:422) - * at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1893) - * at org.apache.hadoop.mapred.YarnChild.main(YarnChild.java:171) - * Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing writable (null) - * at org.apache.hadoop.hive.ql.exec.MapOperator.process(MapOperator.java:505) - * at org.apache.hadoop.hive.ql.exec.mr.ExecMapper.map(ExecMapper.java:170) - * ... 8 more - * Caused by: java.lang.ArrayIndexOutOfBoundsException: 1 - * at org.apache.avro.generic.GenericData$Record.get(GenericData.java:135) - * at org.apache.avro.generic.GenericData.getField(GenericData.java:580) - * at org.apache.avro.generic.GenericData.validate(GenericData.java:373) - * at org.apache.avro.generic.GenericData.validate(GenericData.java:395) - * at org.apache.avro.generic.GenericData.validate(GenericData.java:373) - * at org.apache.hadoop.hive.serde2.avro.AvroSerializer.serialize(AvroSerializer.java:96) - * - * So we try coercion logic on the first struct field inside the union (i.e. only for struct) only. - * - */ - // Generate a file with rows: - // 0, {0: } - // 1, {0: } - onHive().executeQuery(format( - "INSERT INTO TABLE %s PARTITION (c2 = 5) " - + "SELECT 0, create_union(0, named_struct('a', 'a1', 'b', 'b1'), named_struct('c', 'ignore')) " - + "UNION ALL " - + "SELECT 1, create_union(0, named_struct('a', 'a2', 'b', 'b2'), named_struct('c', 'ignore'))", - TABLE_NAME_SCHEMA_EVOLUTION)); - - // Add a coercible change inside union type column. - onHive().executeQuery(format("ALTER TABLE %S CHANGE COLUMN c1 c1 UNIONTYPE, STRUCT>", TABLE_NAME_SCHEMA_EVOLUTION)); - - QueryResult selectAllResult = onTrino().executeQuery(format("SELECT c0, c1 FROM %s", TABLE_NAME_SCHEMA_EVOLUTION)); - assertEquals(selectAllResult.rows().size(), 2); - for (List row : selectAllResult.rows()) { - int id = (Integer) row.get(0); - switch (id) { - case 0: - Row rowValueFirst = rowBuilder() - .addField("a", "a1") - .addField("b", "b1") - .addField("d", null) - .build(); - assertStructEquals(row.get(1), new Object[] {(byte) 0, rowValueFirst, null}); - break; - case 1: - Row rowValueSecond = rowBuilder() - .addField("a", "a2") - .addField("b", "b2") - .addField("d", null) - .build(); - assertStructEquals(row.get(1), new Object[] {(byte) 0, rowValueSecond, null}); - break; - } - } - } - - // TODO use Row as expected too, and use tempto QueryAssert - private static void assertStructEquals(Object actual, Object[] expected) - { - assertThat(actual).isInstanceOf(Row.class); - Row actualRow = (Row) actual; - assertEquals(actualRow.getFields().size(), expected.length); - for (int i = 0; i < actualRow.getFields().size(); i++) { - assertEquals(actualRow.getFields().get(i).getValue(), expected[i]); - } - } - - private static io.trino.jdbc.Row.Builder rowBuilder() - { - return io.trino.jdbc.Row.builder(); - } -} diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestRoles.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestRoles.java index 439bd97f61a1f..b092cb6481f85 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestRoles.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestRoles.java @@ -22,7 +22,6 @@ import io.trino.tempto.BeforeMethodWithContext; import io.trino.tempto.query.QueryExecutor; import io.trino.tempto.query.QueryResult; -import org.testng.SkipException; import org.testng.annotations.Test; import java.util.List; @@ -110,35 +109,6 @@ public void testListRoles() assertThat(actual.rows()).containsOnly(expected.rows().toArray(new List[] {})); } - @Test(groups = {ROLES, AUTHORIZATION, PROFILE_SPECIFIC_TESTS}) - public void testListGrants() - { - if (getHiveVersionMajor() >= 3) { - throw new SkipException(""); // TODO (https://github.com/trinodb/trino/issues/1218) this currently fails on HDP 3 - } - - onTrino().executeQuery("SHOW GRANTS"); // must not fail - onTrino().executeQuery("SELECT * FROM information_schema.table_privileges"); // must not fail - - onTrino().executeQuery("CREATE TABLE test_list_grants(c int)"); - - assertThat(onTrino().executeQuery("SHOW GRANTS")) - .contains( - row(userName, "USER", userName, "USER", "hive", "default", "test_list_grants", "SELECT", "YES", null), - row(userName, "USER", userName, "USER", "hive", "default", "test_list_grants", "INSERT", "YES", null), - row(userName, "USER", userName, "USER", "hive", "default", "test_list_grants", "UPDATE", "YES", null), - row(userName, "USER", userName, "USER", "hive", "default", "test_list_grants", "DELETE", "YES", null)); - - assertThat(onTrino().executeQuery("SELECT * FROM information_schema.table_privileges")) - .contains( - row(userName, "USER", userName, "USER", "hive", "default", "test_list_grants", "SELECT", "YES", null), - row(userName, "USER", userName, "USER", "hive", "default", "test_list_grants", "INSERT", "YES", null), - row(userName, "USER", userName, "USER", "hive", "default", "test_list_grants", "UPDATE", "YES", null), - row(userName, "USER", userName, "USER", "hive", "default", "test_list_grants", "DELETE", "YES", null)); - - onTrino().executeQuery("DROP TABLE test_list_grants"); - } - @Test(groups = {ROLES, AUTHORIZATION, PROFILE_SPECIFIC_TESTS}) public void testCreateDuplicateRole() { From 3cbf4986524425d2272b44e2d2287a6ad0beab14 Mon Sep 17 00:00:00 2001 From: praveenkrishna Date: Mon, 11 Dec 2023 17:02:33 +0530 Subject: [PATCH 114/350] Add a config to toggle local scheduling of Mongo splits Additionally disable local scheduling of Mongo splits --- .../plugin/mongodb/MongoClientConfig.java | 14 +++++++ .../plugin/mongodb/MongoClientModule.java | 7 ++++ .../mongodb/MongoServerDetailsProvider.java | 23 +++++++++++ .../plugin/mongodb/MongoSplitManager.java | 11 +++--- ...essionBasedMongoServerDetailsProvider.java | 39 +++++++++++++++++++ .../plugin/mongodb/TestMongoClientConfig.java | 7 +++- 6 files changed, 93 insertions(+), 8 deletions(-) create mode 100644 plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/MongoServerDetailsProvider.java create mode 100644 plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/SessionBasedMongoServerDetailsProvider.java diff --git a/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/MongoClientConfig.java b/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/MongoClientConfig.java index fa1e82640a52c..eeca4862120b7 100644 --- a/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/MongoClientConfig.java +++ b/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/MongoClientConfig.java @@ -45,6 +45,7 @@ public class MongoClientConfig private String requiredReplicaSetName; private String implicitRowFieldPrefix = "_pos"; private boolean projectionPushDownEnabled = true; + private boolean allowLocalScheduling; @NotNull public String getSchemaCollection() @@ -251,4 +252,17 @@ public MongoClientConfig setProjectionPushdownEnabled(boolean projectionPushDown this.projectionPushDownEnabled = projectionPushDownEnabled; return this; } + + public boolean isAllowLocalScheduling() + { + return allowLocalScheduling; + } + + @Config("mongodb.allow-local-scheduling") + @ConfigDescription("Assign mongo splits to host if worker and mongo share the same cluster") + public MongoClientConfig setAllowLocalScheduling(boolean allowLocalScheduling) + { + this.allowLocalScheduling = allowLocalScheduling; + return this; + } } diff --git a/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/MongoClientModule.java b/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/MongoClientModule.java index 7251965bcb825..f08d728027556 100644 --- a/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/MongoClientModule.java +++ b/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/MongoClientModule.java @@ -13,6 +13,7 @@ */ package io.trino.plugin.mongodb; +import com.google.common.collect.ImmutableList; import com.google.inject.Binder; import com.google.inject.Provides; import com.google.inject.Scopes; @@ -60,6 +61,12 @@ public void setup(Binder binder) MongoClientConfig::getTlsEnabled, new MongoSslModule())); + install(conditionalModule( + MongoClientConfig.class, + MongoClientConfig::isAllowLocalScheduling, + internalBinder -> internalBinder.bind(MongoServerDetailsProvider.class).toInstance(ImmutableList::of), + internalBinder -> internalBinder.bind(MongoServerDetailsProvider.class).to(SessionBasedMongoServerDetailsProvider.class).in(Scopes.SINGLETON))); + newSetBinder(binder, ConnectorTableFunction.class).addBinding().toProvider(Query.class).in(Scopes.SINGLETON); } diff --git a/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/MongoServerDetailsProvider.java b/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/MongoServerDetailsProvider.java new file mode 100644 index 0000000000000..d072a47b699b7 --- /dev/null +++ b/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/MongoServerDetailsProvider.java @@ -0,0 +1,23 @@ +/* + * 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.mongodb; + +import io.trino.spi.HostAddress; + +import java.util.List; + +public interface MongoServerDetailsProvider +{ + List getServerAddress(); +} diff --git a/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/MongoSplitManager.java b/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/MongoSplitManager.java index 2695498cde0db..979422b75bc25 100644 --- a/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/MongoSplitManager.java +++ b/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/MongoSplitManager.java @@ -14,7 +14,6 @@ package io.trino.plugin.mongodb; import com.google.inject.Inject; -import io.trino.spi.HostAddress; import io.trino.spi.connector.ConnectorSession; import io.trino.spi.connector.ConnectorSplitManager; import io.trino.spi.connector.ConnectorSplitSource; @@ -24,17 +23,17 @@ import io.trino.spi.connector.DynamicFilter; import io.trino.spi.connector.FixedSplitSource; -import java.util.List; +import static java.util.Objects.requireNonNull; public class MongoSplitManager implements ConnectorSplitManager { - private final List addresses; + private final MongoServerDetailsProvider serverDetailsProvider; @Inject - public MongoSplitManager(MongoSession session) + public MongoSplitManager(MongoServerDetailsProvider serverDetailsProvider) { - this.addresses = session.getAddresses(); + this.serverDetailsProvider = requireNonNull(serverDetailsProvider, "serverDetailsProvider is null"); } @Override @@ -45,7 +44,7 @@ public ConnectorSplitSource getSplits( DynamicFilter dynamicFilter, Constraint constraint) { - MongoSplit split = new MongoSplit(addresses); + MongoSplit split = new MongoSplit(serverDetailsProvider.getServerAddress()); return new FixedSplitSource(split); } diff --git a/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/SessionBasedMongoServerDetailsProvider.java b/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/SessionBasedMongoServerDetailsProvider.java new file mode 100644 index 0000000000000..0a0020274b01d --- /dev/null +++ b/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/SessionBasedMongoServerDetailsProvider.java @@ -0,0 +1,39 @@ +/* + * 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.mongodb; + +import com.google.inject.Inject; +import io.trino.spi.HostAddress; + +import java.util.List; + +import static java.util.Objects.requireNonNull; + +public class SessionBasedMongoServerDetailsProvider + implements MongoServerDetailsProvider +{ + private final MongoSession mongoSession; + + @Inject + public SessionBasedMongoServerDetailsProvider(MongoSession mongoSession) + { + this.mongoSession = requireNonNull(mongoSession, "mongoSession is null"); + } + + @Override + public List getServerAddress() + { + return mongoSession.getAddresses(); + } +} diff --git a/plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/TestMongoClientConfig.java b/plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/TestMongoClientConfig.java index 13bd693dfaf61..abbbc4424a1c3 100644 --- a/plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/TestMongoClientConfig.java +++ b/plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/TestMongoClientConfig.java @@ -43,7 +43,8 @@ public void testDefaults() .setWriteConcern(WriteConcernType.ACKNOWLEDGED) .setRequiredReplicaSetName(null) .setImplicitRowFieldPrefix("_pos") - .setProjectionPushdownEnabled(true)); + .setProjectionPushdownEnabled(true) + .setAllowLocalScheduling(false)); } @Test @@ -67,6 +68,7 @@ public void testExplicitPropertyMappings() .put("mongodb.required-replica-set", "replica_set") .put("mongodb.implicit-row-field-prefix", "_prefix") .put("mongodb.projection-pushdown-enabled", "false") + .put("mongodb.allow-local-scheduling", "true") .buildOrThrow(); MongoClientConfig expected = new MongoClientConfig() @@ -85,7 +87,8 @@ public void testExplicitPropertyMappings() .setWriteConcern(WriteConcernType.UNACKNOWLEDGED) .setRequiredReplicaSetName("replica_set") .setImplicitRowFieldPrefix("_prefix") - .setProjectionPushdownEnabled(false); + .setProjectionPushdownEnabled(false) + .setAllowLocalScheduling(true); assertFullMapping(properties, expected); } From 3acbf2f92bc3581740114f8938eca243f1e26da9 Mon Sep 17 00:00:00 2001 From: Alex Jo Date: Tue, 28 Nov 2023 10:51:46 -0500 Subject: [PATCH 115/350] Test Iceberg metadata file access for materialized view column queries --- .../io/trino/plugin/iceberg/TestIcebergFileOperations.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergFileOperations.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergFileOperations.java index 9e1235f2a3d38..4aa73b73b659f 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergFileOperations.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergFileOperations.java @@ -785,6 +785,11 @@ public void testSystemMetadataMaterializedViews() .add(new FileOperation(METADATA_JSON, INPUT_FILE_NEW_STREAM)) .build()); + assertFileSystemAccesses(session, "SELECT * FROM iceberg.information_schema.columns WHERE table_schema = CURRENT_SCHEMA AND table_name = 'mv1'", + ImmutableMultiset.builder() + .add(new FileOperation(METADATA_JSON, INPUT_FILE_NEW_STREAM)) + .build()); + assertUpdate("DROP SCHEMA " + schemaName + " CASCADE"); } From ebc74b79acabcd4547bcd5ea411cab8572e1c5c5 Mon Sep 17 00:00:00 2001 From: Alex Jo Date: Tue, 28 Nov 2023 16:07:04 -0500 Subject: [PATCH 116/350] Separate properties from MaterializedViewDefinition Materialized view properties can be more expensive to retrieve and are unused except for SHOW CREATE queries. Splitting them from the main definition reduces file system calls for metadata queries against the Iceberg connector. --- .../execution/CreateMaterializedViewTask.java | 5 +- .../metadata/MaterializedViewDefinition.java | 16 +---- .../main/java/io/trino/metadata/Metadata.java | 10 ++- .../io/trino/metadata/MetadataManager.java | 30 ++++++++- .../trino/sql/rewrite/ShowQueriesRewrite.java | 2 +- .../io/trino/testing/TestingMetadata.java | 14 ++++- .../tracing/TracingConnectorMetadata.java | 19 +++++- .../io/trino/tracing/TracingMetadata.java | 19 +++++- .../io/trino/connector/MockConnector.java | 14 ++++- .../execution/BaseDataDefinitionTaskTest.java | 42 ++++++------- .../io/trino/execution/TestAddColumnTask.java | 2 +- .../io/trino/execution/TestCommentTask.java | 6 +- .../TestCreateMaterializedViewTask.java | 18 +++++- .../trino/execution/TestCreateViewTask.java | 2 +- .../trino/execution/TestDropColumnTask.java | 2 +- .../TestDropMaterializedViewTask.java | 2 +- .../io/trino/execution/TestDropTableTask.java | 4 +- .../io/trino/execution/TestDropViewTask.java | 4 +- .../trino/execution/TestRenameColumnTask.java | 4 +- .../TestRenameMaterializedViewTask.java | 6 +- .../trino/execution/TestRenameTableTask.java | 6 +- .../trino/execution/TestRenameViewTask.java | 4 +- .../execution/TestSetColumnTypeTask.java | 2 +- .../execution/TestSetPropertiesTask.java | 8 ++- .../trino/metadata/AbstractMockMetadata.java | 14 ++++- .../io/trino/sql/analyzer/TestAnalyzer.java | 31 ++++++---- .../sql/planner/TestMaterializedViews.java | 10 +-- .../io/trino/sql/query/TestColumnMask.java | 9 +-- .../io/trino/testing/TestTestingMetadata.java | 5 +- core/trino-spi/pom.xml | 14 +++++ .../ConnectorMaterializedViewDefinition.java | 19 ++---- .../spi/connector/ConnectorMetadata.java | 13 +++- .../ClassLoaderSafeConnectorMetadata.java | 18 +++++- .../plugin/bigquery/BigQueryMetadata.java | 10 ++- .../hive/HiveMaterializedViewMetadata.java | 8 ++- .../io/trino/plugin/hive/HiveMetadata.java | 10 ++- .../NoneHiveMaterializedViewMetadata.java | 8 ++- .../trino/plugin/iceberg/IcebergMetadata.java | 16 ++++- .../iceberg/catalog/AbstractTrinoCatalog.java | 61 +++++++++++++------ .../plugin/iceberg/catalog/TrinoCatalog.java | 3 + .../catalog/glue/TrinoGlueCatalog.java | 52 +++------------- .../iceberg/catalog/hms/TrinoHiveCatalog.java | 59 ++++-------------- .../catalog/jdbc/TrinoJdbcCatalog.java | 14 ++++- .../catalog/nessie/TrinoNessieCatalog.java | 7 +++ .../catalog/rest/TrinoRestCatalog.java | 14 ++++- .../iceberg/TestIcebergFileOperations.java | 24 ++++---- .../execution/TestEventListenerBasic.java | 3 +- .../TestRefreshMaterializedView.java | 3 +- .../io/trino/security/TestAccessControl.java | 3 +- .../io/trino/tests/TestMockConnector.java | 3 +- 50 files changed, 409 insertions(+), 263 deletions(-) diff --git a/core/trino-main/src/main/java/io/trino/execution/CreateMaterializedViewTask.java b/core/trino-main/src/main/java/io/trino/execution/CreateMaterializedViewTask.java index 021ed3a89760c..ff4b109d209b0 100644 --- a/core/trino-main/src/main/java/io/trino/execution/CreateMaterializedViewTask.java +++ b/core/trino-main/src/main/java/io/trino/execution/CreateMaterializedViewTask.java @@ -160,8 +160,7 @@ public ListenableFuture execute( // system path elements are not stored .filter(element -> !element.getCatalogName().equals(GlobalSystemConnector.NAME)) .collect(toImmutableList()), - Optional.empty(), - properties); + Optional.empty()); Set specifiedPropertyKeys = statement.getProperties().stream() // property names are case-insensitive and normalized to lower case @@ -172,7 +171,7 @@ public ListenableFuture execute( .filter(specifiedPropertyKeys::contains) .collect(toImmutableMap(Function.identity(), properties::get)); accessControl.checkCanCreateMaterializedView(session.toSecurityContext(), name, explicitlySetProperties); - plannerContext.getMetadata().createMaterializedView(session, name, definition, statement.isReplace(), statement.isNotExists()); + plannerContext.getMetadata().createMaterializedView(session, name, definition, properties, statement.isReplace(), statement.isNotExists()); stateMachine.setOutput(analysis.getTarget()); stateMachine.setReferencedTables(analysis.getReferencedTables()); diff --git a/core/trino-main/src/main/java/io/trino/metadata/MaterializedViewDefinition.java b/core/trino-main/src/main/java/io/trino/metadata/MaterializedViewDefinition.java index caff709a19b55..6dc49947b0308 100644 --- a/core/trino-main/src/main/java/io/trino/metadata/MaterializedViewDefinition.java +++ b/core/trino-main/src/main/java/io/trino/metadata/MaterializedViewDefinition.java @@ -13,7 +13,6 @@ */ package io.trino.metadata; -import com.google.common.collect.ImmutableMap; import io.trino.spi.connector.CatalogSchemaName; import io.trino.spi.connector.CatalogSchemaTableName; import io.trino.spi.connector.ConnectorMaterializedViewDefinition; @@ -21,7 +20,6 @@ import java.time.Duration; import java.util.List; -import java.util.Map; import java.util.Optional; import static com.google.common.base.MoreObjects.toStringHelper; @@ -34,7 +32,6 @@ public class MaterializedViewDefinition { private final Optional gracePeriod; private final Optional storageTable; - private final Map properties; public MaterializedViewDefinition( String originalSql, @@ -45,14 +42,12 @@ public MaterializedViewDefinition( Optional comment, Identity owner, List path, - Optional storageTable, - Map properties) + Optional storageTable) { super(originalSql, catalog, schema, columns, comment, Optional.of(owner), path); checkArgument(gracePeriod.isEmpty() || !gracePeriod.get().isNegative(), "gracePeriod cannot be negative: %s", gracePeriod); this.gracePeriod = gracePeriod; this.storageTable = requireNonNull(storageTable, "storageTable is null"); - this.properties = ImmutableMap.copyOf(requireNonNull(properties, "properties is null")); } public Optional getGracePeriod() @@ -65,11 +60,6 @@ public Optional getStorageTable() return storageTable; } - public Map getProperties() - { - return properties; - } - public ConnectorMaterializedViewDefinition toConnectorMaterializedViewDefinition() { return new ConnectorMaterializedViewDefinition( @@ -83,8 +73,7 @@ public ConnectorMaterializedViewDefinition toConnectorMaterializedViewDefinition getGracePeriod(), getComment(), getRunAsIdentity().map(Identity::getUser), - getPath(), - properties); + getPath()); } @Override @@ -100,7 +89,6 @@ public String toString() .add("runAsIdentity", getRunAsIdentity()) .add("path", getPath()) .add("storageTable", storageTable.orElse(null)) - .add("properties", properties) .toString(); } } diff --git a/core/trino-main/src/main/java/io/trino/metadata/Metadata.java b/core/trino-main/src/main/java/io/trino/metadata/Metadata.java index bb2cad83e056f..1b7ffa8e706c4 100644 --- a/core/trino-main/src/main/java/io/trino/metadata/Metadata.java +++ b/core/trino-main/src/main/java/io/trino/metadata/Metadata.java @@ -710,7 +710,13 @@ default ResolvedFunction getCoercion(Type fromType, Type toType) /** * Creates the specified materialized view with the specified view definition. */ - void createMaterializedView(Session session, QualifiedObjectName viewName, MaterializedViewDefinition definition, boolean replace, boolean ignoreExisting); + void createMaterializedView( + Session session, + QualifiedObjectName viewName, + MaterializedViewDefinition definition, + Map properties, + boolean replace, + boolean ignoreExisting); /** * Drops the specified materialized view. @@ -740,6 +746,8 @@ default boolean isMaterializedView(Session session, QualifiedObjectName viewName */ Optional getMaterializedView(Session session, QualifiedObjectName viewName); + Map getMaterializedViewProperties(Session session, QualifiedObjectName objectName, MaterializedViewDefinition materializedViewDefinition); + /** * Method to get difference between the states of table at two different points in time/or as of given token-ids. * The method is used by the engine to determine if a materialized view is current with respect to the tables it depends on. diff --git a/core/trino-main/src/main/java/io/trino/metadata/MetadataManager.java b/core/trino-main/src/main/java/io/trino/metadata/MetadataManager.java index 1adeb8d71a4a5..a96e0977e6d8d 100644 --- a/core/trino-main/src/main/java/io/trino/metadata/MetadataManager.java +++ b/core/trino-main/src/main/java/io/trino/metadata/MetadataManager.java @@ -1530,7 +1530,13 @@ public void dropView(Session session, QualifiedObjectName viewName) } @Override - public void createMaterializedView(Session session, QualifiedObjectName viewName, MaterializedViewDefinition definition, boolean replace, boolean ignoreExisting) + public void createMaterializedView( + Session session, + QualifiedObjectName viewName, + MaterializedViewDefinition definition, + Map properties, + boolean replace, + boolean ignoreExisting) { CatalogMetadata catalogMetadata = getCatalogMetadataForWrite(session, viewName.getCatalogName()); CatalogHandle catalogHandle = catalogMetadata.getCatalogHandle(); @@ -1540,6 +1546,7 @@ public void createMaterializedView(Session session, QualifiedObjectName viewName session.toConnectorSession(catalogHandle), viewName.asSchemaTableName(), definition.toConnectorMaterializedViewDefinition(), + properties, replace, ignoreExisting); if (catalogMetadata.getSecurityManagement() == SYSTEM) { @@ -1673,8 +1680,7 @@ private static MaterializedViewDefinition createMaterializedViewDefinition(Conne view.getComment(), runAsIdentity, view.getPath(), - view.getStorageTable(), - view.getProperties()); + view.getStorageTable()); } private Optional getMaterializedViewInternal(Session session, QualifiedObjectName viewName) @@ -1695,6 +1701,24 @@ private Optional getMaterializedViewInterna return Optional.empty(); } + @Override + public Map getMaterializedViewProperties(Session session, QualifiedObjectName viewName, MaterializedViewDefinition materializedViewDefinition) + { + Optional catalog = getOptionalCatalogMetadata(session, viewName.getCatalogName()); + if (catalog.isPresent()) { + CatalogMetadata catalogMetadata = catalog.get(); + CatalogHandle catalogHandle = catalogMetadata.getCatalogHandle(session, viewName); + ConnectorMetadata metadata = catalogMetadata.getMetadataFor(session, catalogHandle); + + ConnectorSession connectorSession = session.toConnectorSession(catalogHandle); + return ImmutableMap.copyOf(metadata.getMaterializedViewProperties( + connectorSession, + viewName.asSchemaTableName(), + materializedViewDefinition.toConnectorMaterializedViewDefinition())); + } + return ImmutableMap.of(); + } + @Override public MaterializedViewFreshness getMaterializedViewFreshness(Session session, QualifiedObjectName viewName) { diff --git a/core/trino-main/src/main/java/io/trino/sql/rewrite/ShowQueriesRewrite.java b/core/trino-main/src/main/java/io/trino/sql/rewrite/ShowQueriesRewrite.java index 54c5ddec92f2c..055bb28eb64bd 100644 --- a/core/trino-main/src/main/java/io/trino/sql/rewrite/ShowQueriesRewrite.java +++ b/core/trino-main/src/main/java/io/trino/sql/rewrite/ShowQueriesRewrite.java @@ -606,7 +606,7 @@ protected Node visitShowCreate(ShowCreate node, Void context) accessControl.checkCanShowCreateTable(session.toSecurityContext(), new QualifiedObjectName(catalogName.getValue(), schemaName.getValue(), tableName.getValue())); - Map properties = viewDefinition.get().getProperties(); + Map properties = metadata.getMaterializedViewProperties(session, objectName, viewDefinition.get()); CatalogHandle catalogHandle = getRequiredCatalogHandle(metadata, session, node, catalogName.getValue()); Collection> allMaterializedViewProperties = materializedViewPropertyManager.getAllProperties(catalogHandle); List propertyNodes = buildProperties(objectName, Optional.empty(), INVALID_MATERIALIZED_VIEW_PROPERTY, properties, allMaterializedViewProperties); diff --git a/core/trino-main/src/main/java/io/trino/testing/TestingMetadata.java b/core/trino-main/src/main/java/io/trino/testing/TestingMetadata.java index 27f174bc7da52..7cbd8aabfc387 100644 --- a/core/trino-main/src/main/java/io/trino/testing/TestingMetadata.java +++ b/core/trino-main/src/main/java/io/trino/testing/TestingMetadata.java @@ -234,7 +234,13 @@ public Optional getView(ConnectorSession session, Schem } @Override - public void createMaterializedView(ConnectorSession session, SchemaTableName viewName, ConnectorMaterializedViewDefinition definition, boolean replace, boolean ignoreExisting) + public void createMaterializedView( + ConnectorSession session, + SchemaTableName viewName, + ConnectorMaterializedViewDefinition definition, + Map properties, + boolean replace, + boolean ignoreExisting) { if (replace) { materializedViews.put(viewName, definition); @@ -264,6 +270,12 @@ public Optional getMaterializedView(Connect return Optional.ofNullable(materializedViews.get(viewName)); } + @Override + public Map getMaterializedViewProperties(ConnectorSession session, SchemaTableName viewName, ConnectorMaterializedViewDefinition materializedViewDefinition) + { + return ImmutableMap.of(); + } + @Override public void dropMaterializedView(ConnectorSession session, SchemaTableName viewName) { diff --git a/core/trino-main/src/main/java/io/trino/tracing/TracingConnectorMetadata.java b/core/trino-main/src/main/java/io/trino/tracing/TracingConnectorMetadata.java index 89de7ad8879e4..774bc5b4a03a8 100644 --- a/core/trino-main/src/main/java/io/trino/tracing/TracingConnectorMetadata.java +++ b/core/trino-main/src/main/java/io/trino/tracing/TracingConnectorMetadata.java @@ -1245,11 +1245,17 @@ public void validateScan(ConnectorSession session, ConnectorTableHandle handle) } @Override - public void createMaterializedView(ConnectorSession session, SchemaTableName viewName, ConnectorMaterializedViewDefinition definition, boolean replace, boolean ignoreExisting) + public void createMaterializedView( + ConnectorSession session, + SchemaTableName viewName, + ConnectorMaterializedViewDefinition definition, + Map properties, + boolean replace, + boolean ignoreExisting) { Span span = startSpan("createMaterializedView", viewName); try (var ignored = scopedSpan(span)) { - delegate.createMaterializedView(session, viewName, definition, replace, ignoreExisting); + delegate.createMaterializedView(session, viewName, definition, properties, replace, ignoreExisting); } } @@ -1289,6 +1295,15 @@ public Optional getMaterializedView(Connect } } + @Override + public Map getMaterializedViewProperties(ConnectorSession session, SchemaTableName viewName, ConnectorMaterializedViewDefinition materializedViewDefinition) + { + Span span = startSpan("getMaterializedViewProperties", viewName); + try (var ignored = scopedSpan(span)) { + return delegate.getMaterializedViewProperties(session, viewName, materializedViewDefinition); + } + } + @Override public MaterializedViewFreshness getMaterializedViewFreshness(ConnectorSession session, SchemaTableName name) { diff --git a/core/trino-main/src/main/java/io/trino/tracing/TracingMetadata.java b/core/trino-main/src/main/java/io/trino/tracing/TracingMetadata.java index 09e35c8ef453d..d6e0dfedf6a58 100644 --- a/core/trino-main/src/main/java/io/trino/tracing/TracingMetadata.java +++ b/core/trino-main/src/main/java/io/trino/tracing/TracingMetadata.java @@ -1302,11 +1302,17 @@ public void dropLanguageFunction(Session session, QualifiedObjectName name, Stri } @Override - public void createMaterializedView(Session session, QualifiedObjectName viewName, MaterializedViewDefinition definition, boolean replace, boolean ignoreExisting) + public void createMaterializedView( + Session session, + QualifiedObjectName viewName, + MaterializedViewDefinition definition, + Map properties, + boolean replace, + boolean ignoreExisting) { Span span = startSpan("createMaterializedView", viewName); try (var ignored = scopedSpan(span)) { - delegate.createMaterializedView(session, viewName, definition, replace, ignoreExisting); + delegate.createMaterializedView(session, viewName, definition, properties, replace, ignoreExisting); } } @@ -1355,6 +1361,15 @@ public Optional getMaterializedView(Session session, } } + @Override + public Map getMaterializedViewProperties(Session session, QualifiedObjectName objectName, MaterializedViewDefinition materializedViewDefinition) + { + Span span = startSpan("getMaterializedViewProperties", objectName); + try (var ignored = scopedSpan(span)) { + return delegate.getMaterializedViewProperties(session, objectName, materializedViewDefinition); + } + } + @Override public MaterializedViewFreshness getMaterializedViewFreshness(Session session, QualifiedObjectName name) { diff --git a/core/trino-main/src/test/java/io/trino/connector/MockConnector.java b/core/trino-main/src/test/java/io/trino/connector/MockConnector.java index 82bf03c0a12ac..3a3cb7a85ff14 100644 --- a/core/trino-main/src/test/java/io/trino/connector/MockConnector.java +++ b/core/trino-main/src/test/java/io/trino/connector/MockConnector.java @@ -665,7 +665,13 @@ public void setViewAuthorization(ConnectorSession session, SchemaTableName viewN public void dropView(ConnectorSession session, SchemaTableName viewName) {} @Override - public void createMaterializedView(ConnectorSession session, SchemaTableName viewName, ConnectorMaterializedViewDefinition definition, boolean replace, boolean ignoreExisting) {} + public void createMaterializedView( + ConnectorSession session, + SchemaTableName viewName, + ConnectorMaterializedViewDefinition definition, + Map properties, + boolean replace, + boolean ignoreExisting) {} @Override public List listMaterializedViews(ConnectorSession session, Optional schemaName) @@ -680,6 +686,12 @@ public Optional getMaterializedView(Connect return Optional.ofNullable(getMaterializedViews.apply(session, viewName.toSchemaTablePrefix()).get(viewName)); } + @Override + public Map getMaterializedViewProperties(ConnectorSession session, SchemaTableName viewName, ConnectorMaterializedViewDefinition materializedViewDefinition) + { + return ImmutableMap.of(); + } + @Override public MaterializedViewFreshness getMaterializedViewFreshness(ConnectorSession session, SchemaTableName viewName) { diff --git a/core/trino-main/src/test/java/io/trino/execution/BaseDataDefinitionTaskTest.java b/core/trino-main/src/test/java/io/trino/execution/BaseDataDefinitionTaskTest.java index 8357a4d93a304..6d3dd9b72dde5 100644 --- a/core/trino-main/src/test/java/io/trino/execution/BaseDataDefinitionTaskTest.java +++ b/core/trino-main/src/test/java/io/trino/execution/BaseDataDefinitionTaskTest.java @@ -42,7 +42,6 @@ import io.trino.spi.connector.ColumnHandle; import io.trino.spi.connector.ColumnMetadata; import io.trino.spi.connector.ConnectorTableMetadata; -import io.trino.spi.connector.MaterializedViewNotFoundException; import io.trino.spi.connector.SaveMode; import io.trino.spi.connector.SchemaTableName; import io.trino.spi.connector.TestingColumnHandle; @@ -108,6 +107,7 @@ public abstract class BaseDataDefinitionTaskTest protected static final String MATERIALIZED_VIEW_PROPERTY_2_NAME = "property2"; protected static final String MATERIALIZED_VIEW_PROPERTY_2_DEFAULT_VALUE = "defaultProperty2Value"; + protected static final Map MATERIALIZED_VIEW_PROPERTIES = ImmutableMap.of(MATERIALIZED_VIEW_PROPERTY_2_NAME, MATERIALIZED_VIEW_PROPERTY_2_DEFAULT_VALUE); private LocalQueryRunner queryRunner; protected Session testSession; @@ -196,8 +196,7 @@ protected MaterializedViewDefinition someMaterializedView(String sql, List tables = new ConcurrentHashMap<>(); private final Map views = new ConcurrentHashMap<>(); private final Map materializedViews = new ConcurrentHashMap<>(); + private final Map> materializedViewProperties = new ConcurrentHashMap<>(); public MockMetadata(String catalogName) { @@ -455,10 +455,23 @@ public Optional getMaterializedView(Session session, } @Override - public void createMaterializedView(Session session, QualifiedObjectName viewName, MaterializedViewDefinition definition, boolean replace, boolean ignoreExisting) + public Map getMaterializedViewProperties(Session session, QualifiedObjectName viewName, MaterializedViewDefinition materializedViewDefinition) + { + return materializedViewProperties.get(viewName.asSchemaTableName()); + } + + @Override + public void createMaterializedView( + Session session, + QualifiedObjectName viewName, + MaterializedViewDefinition definition, + Map properties, + boolean replace, + boolean ignoreExisting) { checkArgument(ignoreExisting || !materializedViews.containsKey(viewName.asSchemaTableName())); materializedViews.put(viewName.asSchemaTableName(), definition); + materializedViewProperties.put(viewName.asSchemaTableName(), properties); } @Override @@ -467,9 +480,7 @@ public synchronized void setMaterializedViewProperties( QualifiedObjectName viewName, Map> properties) { - MaterializedViewDefinition existingDefinition = getMaterializedView(session, viewName) - .orElseThrow(() -> new MaterializedViewNotFoundException(viewName.asSchemaTableName())); - Map newProperties = new HashMap<>(existingDefinition.getProperties()); + Map newProperties = new HashMap<>(materializedViewProperties.getOrDefault(viewName.asSchemaTableName(), ImmutableMap.of())); for (Entry> entry : properties.entrySet()) { if (entry.getValue().isPresent()) { newProperties.put(entry.getKey(), entry.getValue().orElseThrow()); @@ -478,19 +489,7 @@ public synchronized void setMaterializedViewProperties( newProperties.remove(entry.getKey()); } } - materializedViews.put( - viewName.asSchemaTableName(), - new MaterializedViewDefinition( - existingDefinition.getOriginalSql(), - existingDefinition.getCatalog(), - existingDefinition.getSchema(), - existingDefinition.getColumns(), - existingDefinition.getGracePeriod(), - existingDefinition.getComment(), - existingDefinition.getRunAsIdentity().get(), - existingDefinition.getPath(), - existingDefinition.getStorageTable(), - newProperties)); + materializedViewProperties.put(viewName.asSchemaTableName(), newProperties); } @Override @@ -510,8 +509,7 @@ public void setMaterializedViewColumnComment(Session session, QualifiedObjectNam view.getComment(), view.getRunAsIdentity().get(), view.getPath(), - view.getStorageTable(), - view.getProperties())); + view.getStorageTable())); } @Override diff --git a/core/trino-main/src/test/java/io/trino/execution/TestAddColumnTask.java b/core/trino-main/src/test/java/io/trino/execution/TestAddColumnTask.java index 5b497a1cc2f66..1d0e7699454d4 100644 --- a/core/trino-main/src/test/java/io/trino/execution/TestAddColumnTask.java +++ b/core/trino-main/src/test/java/io/trino/execution/TestAddColumnTask.java @@ -163,7 +163,7 @@ public void testAddColumnOnView() public void testAddColumnOnMaterializedView() { QualifiedObjectName materializedViewName = qualifiedObjectName("existing_materialized_view"); - metadata.createMaterializedView(testSession, QualifiedObjectName.valueOf(materializedViewName.toString()), someMaterializedView(), false, false); + metadata.createMaterializedView(testSession, QualifiedObjectName.valueOf(materializedViewName.toString()), someMaterializedView(), MATERIALIZED_VIEW_PROPERTIES, false, false); assertTrinoExceptionThrownBy(() -> getFutureValue(executeAddColumn(asQualifiedName(materializedViewName), QualifiedName.of("test"), INTEGER, Optional.empty(), false, false))) .hasErrorCode(TABLE_NOT_FOUND) diff --git a/core/trino-main/src/test/java/io/trino/execution/TestCommentTask.java b/core/trino-main/src/test/java/io/trino/execution/TestCommentTask.java index bca446767f66b..2068a2b916f7b 100644 --- a/core/trino-main/src/test/java/io/trino/execution/TestCommentTask.java +++ b/core/trino-main/src/test/java/io/trino/execution/TestCommentTask.java @@ -69,7 +69,7 @@ public void testCommentTableOnView() public void testCommentTableOnMaterializedView() { QualifiedObjectName materializedViewName = qualifiedObjectName("existing_materialized_view"); - metadata.createMaterializedView(testSession, QualifiedObjectName.valueOf(materializedViewName.toString()), someMaterializedView(), false, false); + metadata.createMaterializedView(testSession, QualifiedObjectName.valueOf(materializedViewName.toString()), someMaterializedView(), MATERIALIZED_VIEW_PROPERTIES, false, false); assertTrinoExceptionThrownBy(() -> getFutureValue(setComment(TABLE, asQualifiedName(materializedViewName), Optional.of("new comment")))) .hasErrorCode(TABLE_NOT_FOUND) @@ -102,7 +102,7 @@ public void testCommentViewOnTable() public void testCommentViewOnMaterializedView() { QualifiedObjectName materializedViewName = qualifiedObjectName("existing_materialized_view"); - metadata.createMaterializedView(testSession, QualifiedObjectName.valueOf(materializedViewName.toString()), someMaterializedView(), false, false); + metadata.createMaterializedView(testSession, QualifiedObjectName.valueOf(materializedViewName.toString()), someMaterializedView(), MATERIALIZED_VIEW_PROPERTIES, false, false); assertTrinoExceptionThrownBy(() -> getFutureValue(setComment(VIEW, asQualifiedName(materializedViewName), Optional.of("new comment")))) .hasErrorCode(TABLE_NOT_FOUND) @@ -145,7 +145,7 @@ public void testCommentViewColumn() public void testCommentMaterializedViewColumn() { QualifiedObjectName materializedViewName = qualifiedObjectName("existing_materialized_view"); - metadata.createMaterializedView(testSession, QualifiedObjectName.valueOf(materializedViewName.toString()), someMaterializedView(), false, false); + metadata.createMaterializedView(testSession, QualifiedObjectName.valueOf(materializedViewName.toString()), someMaterializedView(), MATERIALIZED_VIEW_PROPERTIES, false, false); assertThat(metadata.isMaterializedView(testSession, materializedViewName)).isTrue(); QualifiedName columnName = qualifiedColumnName("existing_materialized_view", "test"); diff --git a/core/trino-main/src/test/java/io/trino/execution/TestCreateMaterializedViewTask.java b/core/trino-main/src/test/java/io/trino/execution/TestCreateMaterializedViewTask.java index 7b103a902c60f..c1a04c050bb8d 100644 --- a/core/trino-main/src/test/java/io/trino/execution/TestCreateMaterializedViewTask.java +++ b/core/trino-main/src/test/java/io/trino/execution/TestCreateMaterializedViewTask.java @@ -252,7 +252,7 @@ public void testCreateMaterializedViewWithDefaultProperties() Optional definitionOptional = metadata.getMaterializedView(testSession, QualifiedObjectName.valueOf(materializedViewName.toString())); assertThat(definitionOptional).isPresent(); - Map properties = definitionOptional.get().getProperties(); + Map properties = metadata.getMaterializedViewProperties(testSession, new QualifiedObjectName(TEST_CATALOG_NAME, "schema", "mv"), definitionOptional.get()); assertThat(properties.get("foo")).isEqualTo(DEFAULT_MATERIALIZED_VIEW_FOO_PROPERTY_VALUE); assertThat(properties.get("bar")).isEqualTo(DEFAULT_MATERIALIZED_VIEW_BAR_PROPERTY_VALUE); } @@ -310,11 +310,19 @@ private class MockMetadata extends AbstractMockMetadata { private final Map materializedViews = new ConcurrentHashMap<>(); + private final Map> materializedViewProperties = new ConcurrentHashMap<>(); @Override - public void createMaterializedView(Session session, QualifiedObjectName viewName, MaterializedViewDefinition definition, boolean replace, boolean ignoreExisting) + public void createMaterializedView( + Session session, + QualifiedObjectName viewName, + MaterializedViewDefinition definition, + Map properties, + boolean replace, + boolean ignoreExisting) { materializedViews.put(viewName.asSchemaTableName(), definition); + materializedViewProperties.put(viewName.asSchemaTableName(), properties); if (!ignoreExisting) { throw new TrinoException(ALREADY_EXISTS, "Materialized view already exists"); } @@ -375,6 +383,12 @@ public Optional getMaterializedView(Session session, return Optional.ofNullable(materializedViews.get(viewName.asSchemaTableName())); } + @Override + public Map getMaterializedViewProperties(Session session, QualifiedObjectName viewName, MaterializedViewDefinition materializedViewDefinition) + { + return materializedViewProperties.get(viewName.asSchemaTableName()); + } + @Override public Optional getView(Session session, QualifiedObjectName viewName) { diff --git a/core/trino-main/src/test/java/io/trino/execution/TestCreateViewTask.java b/core/trino-main/src/test/java/io/trino/execution/TestCreateViewTask.java index 77c2c8240384a..f9fcd7a69e682 100644 --- a/core/trino-main/src/test/java/io/trino/execution/TestCreateViewTask.java +++ b/core/trino-main/src/test/java/io/trino/execution/TestCreateViewTask.java @@ -127,7 +127,7 @@ public void testReplaceViewOnTableIfExists() public void testCreateViewOnMaterializedView() { QualifiedObjectName viewName = qualifiedObjectName("existing_materialized_view"); - metadata.createMaterializedView(testSession, viewName, someMaterializedView(), false, false); + metadata.createMaterializedView(testSession, viewName, someMaterializedView(), MATERIALIZED_VIEW_PROPERTIES, false, false); assertTrinoExceptionThrownBy(() -> getFutureValue(executeCreateView(asQualifiedName(viewName), false))) .hasErrorCode(TABLE_ALREADY_EXISTS) diff --git a/core/trino-main/src/test/java/io/trino/execution/TestDropColumnTask.java b/core/trino-main/src/test/java/io/trino/execution/TestDropColumnTask.java index a443dc529e807..8b6b234a48d08 100644 --- a/core/trino-main/src/test/java/io/trino/execution/TestDropColumnTask.java +++ b/core/trino-main/src/test/java/io/trino/execution/TestDropColumnTask.java @@ -156,7 +156,7 @@ public void testDropColumnOnView() public void testDropColumnOnMaterializedView() { QualifiedObjectName materializedViewName = qualifiedObjectName("existing_materialized_view"); - metadata.createMaterializedView(testSession, QualifiedObjectName.valueOf(materializedViewName.toString()), someMaterializedView(), false, false); + metadata.createMaterializedView(testSession, QualifiedObjectName.valueOf(materializedViewName.toString()), someMaterializedView(), MATERIALIZED_VIEW_PROPERTIES, false, false); assertTrinoExceptionThrownBy(() -> getFutureValue(executeDropColumn(asQualifiedName(materializedViewName), QualifiedName.of("test"), false, false))) .hasErrorCode(TABLE_NOT_FOUND) diff --git a/core/trino-main/src/test/java/io/trino/execution/TestDropMaterializedViewTask.java b/core/trino-main/src/test/java/io/trino/execution/TestDropMaterializedViewTask.java index bfbd422399b90..2a7a2ba2fba6f 100644 --- a/core/trino-main/src/test/java/io/trino/execution/TestDropMaterializedViewTask.java +++ b/core/trino-main/src/test/java/io/trino/execution/TestDropMaterializedViewTask.java @@ -36,7 +36,7 @@ public class TestDropMaterializedViewTask public void testDropExistingMaterializedView() { QualifiedObjectName viewName = qualifiedObjectName("existing_materialized_view"); - metadata.createMaterializedView(testSession, viewName, someMaterializedView(), false, false); + metadata.createMaterializedView(testSession, viewName, someMaterializedView(), MATERIALIZED_VIEW_PROPERTIES, false, false); assertThat(metadata.isMaterializedView(testSession, viewName)).isTrue(); getFutureValue(executeDropMaterializedView(asQualifiedName(viewName), false)); diff --git a/core/trino-main/src/test/java/io/trino/execution/TestDropTableTask.java b/core/trino-main/src/test/java/io/trino/execution/TestDropTableTask.java index 79fcb4237689f..48b4ba0e9c0fe 100644 --- a/core/trino-main/src/test/java/io/trino/execution/TestDropTableTask.java +++ b/core/trino-main/src/test/java/io/trino/execution/TestDropTableTask.java @@ -107,7 +107,7 @@ public void testDropTableIfExistsOnView() public void testDropTableOnMaterializedView() { QualifiedName viewName = qualifiedName("existing_materialized_view"); - metadata.createMaterializedView(testSession, asQualifiedObjectName(viewName), someMaterializedView(), false, false); + metadata.createMaterializedView(testSession, asQualifiedObjectName(viewName), someMaterializedView(), MATERIALIZED_VIEW_PROPERTIES, false, false); assertTrinoExceptionThrownBy(() -> getFutureValue(executeDropTable(viewName, false))) .hasErrorCode(GENERIC_USER_ERROR) @@ -118,7 +118,7 @@ public void testDropTableOnMaterializedView() public void testDropTableIfExistsOnMaterializedView() { QualifiedName viewName = qualifiedName("existing_materialized_view"); - metadata.createMaterializedView(testSession, asQualifiedObjectName(viewName), someMaterializedView(), false, false); + metadata.createMaterializedView(testSession, asQualifiedObjectName(viewName), someMaterializedView(), MATERIALIZED_VIEW_PROPERTIES, false, false); assertTrinoExceptionThrownBy(() -> getFutureValue(executeDropTable(viewName, true))) .hasErrorCode(GENERIC_USER_ERROR) diff --git a/core/trino-main/src/test/java/io/trino/execution/TestDropViewTask.java b/core/trino-main/src/test/java/io/trino/execution/TestDropViewTask.java index 88af3f3802ee2..b4838b2d6c995 100644 --- a/core/trino-main/src/test/java/io/trino/execution/TestDropViewTask.java +++ b/core/trino-main/src/test/java/io/trino/execution/TestDropViewTask.java @@ -88,7 +88,7 @@ public void testDropViewOnTableIfExists() public void testDropViewOnMaterializedView() { QualifiedName viewName = qualifiedName("existing_materialized_view"); - metadata.createMaterializedView(testSession, QualifiedObjectName.valueOf(viewName.toString()), someMaterializedView(), false, false); + metadata.createMaterializedView(testSession, QualifiedObjectName.valueOf(viewName.toString()), someMaterializedView(), MATERIALIZED_VIEW_PROPERTIES, false, false); assertTrinoExceptionThrownBy(() -> getFutureValue(executeDropView(viewName, false))) .hasErrorCode(GENERIC_USER_ERROR) @@ -99,7 +99,7 @@ public void testDropViewOnMaterializedView() public void testDropViewOnMaterializedViewIfExists() { QualifiedName viewName = qualifiedName("existing_materialized_view"); - metadata.createMaterializedView(testSession, QualifiedObjectName.valueOf(viewName.toString()), someMaterializedView(), false, false); + metadata.createMaterializedView(testSession, QualifiedObjectName.valueOf(viewName.toString()), someMaterializedView(), MATERIALIZED_VIEW_PROPERTIES, false, false); assertTrinoExceptionThrownBy(() -> getFutureValue(executeDropView(viewName, true))) .hasErrorCode(GENERIC_USER_ERROR) diff --git a/core/trino-main/src/test/java/io/trino/execution/TestRenameColumnTask.java b/core/trino-main/src/test/java/io/trino/execution/TestRenameColumnTask.java index 7441dc9969807..ec0729f8b5979 100644 --- a/core/trino-main/src/test/java/io/trino/execution/TestRenameColumnTask.java +++ b/core/trino-main/src/test/java/io/trino/execution/TestRenameColumnTask.java @@ -117,7 +117,7 @@ public void testRenameColumnOnView() public void testRenameColumnOnMaterializedView() { QualifiedObjectName materializedViewName = qualifiedObjectName("existing_materialized_view"); - metadata.createMaterializedView(testSession, QualifiedObjectName.valueOf(materializedViewName.toString()), someMaterializedView(), false, false); + metadata.createMaterializedView(testSession, QualifiedObjectName.valueOf(materializedViewName.toString()), someMaterializedView(), MATERIALIZED_VIEW_PROPERTIES, false, false); assertTrinoExceptionThrownBy(() -> getFutureValue(executeRenameColumn(asQualifiedName(materializedViewName), QualifiedName.of("a"), identifier("a_renamed"), false, false))) .hasErrorCode(TABLE_NOT_FOUND) @@ -211,7 +211,7 @@ public void testRenameFieldOnView() public void testRenameFieldOnMaterializedView() { QualifiedObjectName materializedViewName = qualifiedObjectName("existing_materialized_view"); - metadata.createMaterializedView(testSession, QualifiedObjectName.valueOf(materializedViewName.toString()), someMaterializedView(), false, false); + metadata.createMaterializedView(testSession, QualifiedObjectName.valueOf(materializedViewName.toString()), someMaterializedView(), MATERIALIZED_VIEW_PROPERTIES, false, false); assertTrinoExceptionThrownBy(() -> getFutureValue(executeRenameColumn(asQualifiedName(materializedViewName), QualifiedName.of("test"), identifier("x"), false, false))) .hasErrorCode(TABLE_NOT_FOUND) diff --git a/core/trino-main/src/test/java/io/trino/execution/TestRenameMaterializedViewTask.java b/core/trino-main/src/test/java/io/trino/execution/TestRenameMaterializedViewTask.java index e1527c7c06cf0..c898c14ee83e0 100644 --- a/core/trino-main/src/test/java/io/trino/execution/TestRenameMaterializedViewTask.java +++ b/core/trino-main/src/test/java/io/trino/execution/TestRenameMaterializedViewTask.java @@ -38,7 +38,7 @@ public void testRenameExistingMaterializedView() { QualifiedObjectName materializedViewName = qualifiedObjectName("existing_materialized_view"); QualifiedObjectName newMaterializedViewName = qualifiedObjectName("existing_materialized_view_new"); - metadata.createMaterializedView(testSession, materializedViewName, someMaterializedView(), false, false); + metadata.createMaterializedView(testSession, materializedViewName, someMaterializedView(), MATERIALIZED_VIEW_PROPERTIES, false, false); getFutureValue(executeRenameMaterializedView(asQualifiedName(materializedViewName), asQualifiedName(newMaterializedViewName))); assertThat(metadata.isMaterializedView(testSession, materializedViewName)).isFalse(); @@ -90,7 +90,7 @@ public void testRenameMaterializedViewOnTableIfExists() public void testRenameMaterializedViewTargetTableExists() { QualifiedObjectName materializedViewName = qualifiedObjectName("existing_materialized_view"); - metadata.createMaterializedView(testSession, materializedViewName, someMaterializedView(), false, false); + metadata.createMaterializedView(testSession, materializedViewName, someMaterializedView(), MATERIALIZED_VIEW_PROPERTIES, false, false); QualifiedObjectName tableName = qualifiedObjectName("existing_table"); metadata.createTable(testSession, TEST_CATALOG_NAME, someTable(tableName), FAIL); @@ -125,7 +125,7 @@ public void testRenameMaterializedViewOnViewIfExists() public void testRenameMaterializedViewTargetViewExists() { QualifiedObjectName materializedViewName = qualifiedObjectName("existing_materialized_view"); - metadata.createMaterializedView(testSession, materializedViewName, someMaterializedView(), false, false); + metadata.createMaterializedView(testSession, materializedViewName, someMaterializedView(), MATERIALIZED_VIEW_PROPERTIES, false, false); QualifiedName viewName = qualifiedName("existing_view"); metadata.createView(testSession, QualifiedObjectName.valueOf(viewName.toString()), someView(), false); diff --git a/core/trino-main/src/test/java/io/trino/execution/TestRenameTableTask.java b/core/trino-main/src/test/java/io/trino/execution/TestRenameTableTask.java index b6d06d046c66c..791b28051b841 100644 --- a/core/trino-main/src/test/java/io/trino/execution/TestRenameTableTask.java +++ b/core/trino-main/src/test/java/io/trino/execution/TestRenameTableTask.java @@ -90,7 +90,7 @@ public void testRenameTableOnViewIfExists() public void testRenameTableOnMaterializedView() { QualifiedName viewName = qualifiedName("existing_materialized_view"); - metadata.createMaterializedView(testSession, QualifiedObjectName.valueOf(viewName.toString()), someMaterializedView(), false, false); + metadata.createMaterializedView(testSession, QualifiedObjectName.valueOf(viewName.toString()), someMaterializedView(), MATERIALIZED_VIEW_PROPERTIES, false, false); assertTrinoExceptionThrownBy(() -> getFutureValue(executeRenameTable(viewName, qualifiedName("existing_materialized_view_new"), false))) .hasErrorCode(GENERIC_USER_ERROR) @@ -101,7 +101,7 @@ public void testRenameTableOnMaterializedView() public void testRenameTableOnMaterializedViewIfExists() { QualifiedName viewName = qualifiedName("existing_materialized_view"); - metadata.createMaterializedView(testSession, QualifiedObjectName.valueOf(viewName.toString()), someMaterializedView(), false, false); + metadata.createMaterializedView(testSession, QualifiedObjectName.valueOf(viewName.toString()), someMaterializedView(), MATERIALIZED_VIEW_PROPERTIES, false, false); assertTrinoExceptionThrownBy(() -> getFutureValue(executeRenameTable(viewName, qualifiedName("existing_materialized_view_new"), true))) .hasErrorCode(GENERIC_USER_ERROR) @@ -127,7 +127,7 @@ public void testRenameTableTargetMaterializedViewExists() QualifiedObjectName tableName = qualifiedObjectName("existing_table"); metadata.createTable(testSession, TEST_CATALOG_NAME, someTable(tableName), FAIL); QualifiedObjectName materializedViewName = qualifiedObjectName("existing_materialized_view"); - metadata.createMaterializedView(testSession, materializedViewName, someMaterializedView(), false, false); + metadata.createMaterializedView(testSession, materializedViewName, someMaterializedView(), MATERIALIZED_VIEW_PROPERTIES, false, false); assertTrinoExceptionThrownBy(() -> getFutureValue(executeRenameTable(asQualifiedName(tableName), asQualifiedName(materializedViewName), false))) .hasErrorCode(GENERIC_USER_ERROR) diff --git a/core/trino-main/src/test/java/io/trino/execution/TestRenameViewTask.java b/core/trino-main/src/test/java/io/trino/execution/TestRenameViewTask.java index 00c55d18b0ace..2ff29c5525788 100644 --- a/core/trino-main/src/test/java/io/trino/execution/TestRenameViewTask.java +++ b/core/trino-main/src/test/java/io/trino/execution/TestRenameViewTask.java @@ -71,7 +71,7 @@ public void testRenameViewOnTable() public void testRenameViewOnMaterializedView() { QualifiedName viewName = qualifiedName("existing_materialized_view"); - metadata.createMaterializedView(testSession, QualifiedObjectName.valueOf(viewName.toString()), someMaterializedView(), false, false); + metadata.createMaterializedView(testSession, QualifiedObjectName.valueOf(viewName.toString()), someMaterializedView(), MATERIALIZED_VIEW_PROPERTIES, false, false); assertTrinoExceptionThrownBy(() -> getFutureValue(executeRenameView(viewName, qualifiedName("existing_materialized_view_new")))) .hasErrorCode(TABLE_NOT_FOUND) @@ -97,7 +97,7 @@ public void testRenameViewTargetMaterializedViewExists() QualifiedName viewName = qualifiedName("existing_view"); metadata.createView(testSession, QualifiedObjectName.valueOf(viewName.toString()), someView(), false); QualifiedObjectName materializedViewName = qualifiedObjectName("existing_materialized_view"); - metadata.createMaterializedView(testSession, materializedViewName, someMaterializedView(), false, false); + metadata.createMaterializedView(testSession, materializedViewName, someMaterializedView(), MATERIALIZED_VIEW_PROPERTIES, false, false); assertTrinoExceptionThrownBy(() -> getFutureValue(executeRenameView(viewName, asQualifiedName(materializedViewName)))) .hasErrorCode(GENERIC_USER_ERROR) diff --git a/core/trino-main/src/test/java/io/trino/execution/TestSetColumnTypeTask.java b/core/trino-main/src/test/java/io/trino/execution/TestSetColumnTypeTask.java index 8de522663eebe..c7a0cc5f02739 100644 --- a/core/trino-main/src/test/java/io/trino/execution/TestSetColumnTypeTask.java +++ b/core/trino-main/src/test/java/io/trino/execution/TestSetColumnTypeTask.java @@ -113,7 +113,7 @@ public void testSetDataTypeOnView() public void testSetDataTypeOnMaterializedView() { QualifiedObjectName materializedViewName = qualifiedObjectName("existing_materialized_view"); - metadata.createMaterializedView(testSession, QualifiedObjectName.valueOf(materializedViewName.toString()), someMaterializedView(), false, false); + metadata.createMaterializedView(testSession, QualifiedObjectName.valueOf(materializedViewName.toString()), someMaterializedView(), MATERIALIZED_VIEW_PROPERTIES, false, false); assertTrinoExceptionThrownBy(() -> getFutureValue(executeSetColumnType(asQualifiedName(materializedViewName), QualifiedName.of("test"), toSqlType(INTEGER), false))) .hasErrorCode(TABLE_NOT_FOUND) diff --git a/core/trino-main/src/test/java/io/trino/execution/TestSetPropertiesTask.java b/core/trino-main/src/test/java/io/trino/execution/TestSetPropertiesTask.java index e351361d8d694..89de2490d2828 100644 --- a/core/trino-main/src/test/java/io/trino/execution/TestSetPropertiesTask.java +++ b/core/trino-main/src/test/java/io/trino/execution/TestSetPropertiesTask.java @@ -17,6 +17,7 @@ import com.google.common.collect.ImmutableMap; import io.trino.connector.CatalogServiceProvider; import io.trino.execution.warnings.WarningCollector; +import io.trino.metadata.MaterializedViewDefinition; import io.trino.metadata.QualifiedObjectName; import io.trino.metadata.TablePropertyManager; import io.trino.security.AllowAllAccessControl; @@ -37,7 +38,7 @@ public class TestSetPropertiesTask public void testSetMaterializedViewProperties() { QualifiedObjectName materializedViewName = qualifiedObjectName("test_materialized_view"); - metadata.createMaterializedView(testSession, materializedViewName, someMaterializedView(), false, false); + metadata.createMaterializedView(testSession, materializedViewName, someMaterializedView(), MATERIALIZED_VIEW_PROPERTIES, false, false); // set all properties to non-DEFAULT values and check the results executeSetProperties( @@ -47,7 +48,8 @@ public void testSetMaterializedViewProperties() ImmutableList.of( new Property(new Identifier(MATERIALIZED_VIEW_PROPERTY_1_NAME), new LongLiteral("111")), new Property(new Identifier(MATERIALIZED_VIEW_PROPERTY_2_NAME), new StringLiteral("abc"))))); - assertThat(metadata.getMaterializedView(testSession, materializedViewName).get().getProperties()).isEqualTo( + MaterializedViewDefinition materializedViewDefinition = metadata.getMaterializedView(testSession, materializedViewName).orElseThrow(); + assertThat(metadata.getMaterializedViewProperties(testSession, materializedViewName, materializedViewDefinition)).isEqualTo( ImmutableMap.of( MATERIALIZED_VIEW_PROPERTY_1_NAME, 111L, MATERIALIZED_VIEW_PROPERTY_2_NAME, "abc")); @@ -62,7 +64,7 @@ public void testSetMaterializedViewProperties() new Property(new Identifier(MATERIALIZED_VIEW_PROPERTY_2_NAME))))); // since the default value of property 1 is null, property 1 should not appear in the result, whereas property 2 should appear in // the result with its (non-null) default value - assertThat(metadata.getMaterializedView(testSession, materializedViewName).get().getProperties()).isEqualTo( + assertThat(metadata.getMaterializedViewProperties(testSession, materializedViewName, materializedViewDefinition)).isEqualTo( ImmutableMap.of(MATERIALIZED_VIEW_PROPERTY_2_NAME, MATERIALIZED_VIEW_PROPERTY_2_DEFAULT_VALUE)); } diff --git a/core/trino-main/src/test/java/io/trino/metadata/AbstractMockMetadata.java b/core/trino-main/src/test/java/io/trino/metadata/AbstractMockMetadata.java index 2611b045a74e8..6e94d564b9d87 100644 --- a/core/trino-main/src/test/java/io/trino/metadata/AbstractMockMetadata.java +++ b/core/trino-main/src/test/java/io/trino/metadata/AbstractMockMetadata.java @@ -910,7 +910,13 @@ public Optional> applyTopN(Session session, T } @Override - public void createMaterializedView(Session session, QualifiedObjectName viewName, MaterializedViewDefinition definition, boolean replace, boolean ignoreExisting) + public void createMaterializedView( + Session session, + QualifiedObjectName viewName, + MaterializedViewDefinition definition, + Map properties, + boolean replace, + boolean ignoreExisting) { throw new UnsupportedOperationException(); } @@ -939,6 +945,12 @@ public Optional getMaterializedView(Session session, throw new UnsupportedOperationException(); } + @Override + public Map getMaterializedViewProperties(Session session, QualifiedObjectName viewName, MaterializedViewDefinition materializedViewDefinition) + { + throw new UnsupportedOperationException(); + } + @Override public MaterializedViewFreshness getMaterializedViewFreshness(Session session, QualifiedObjectName name) { diff --git a/core/trino-main/src/test/java/io/trino/sql/analyzer/TestAnalyzer.java b/core/trino-main/src/test/java/io/trino/sql/analyzer/TestAnalyzer.java index 503f5ba2dbec0..613eb7ebbdf37 100644 --- a/core/trino-main/src/test/java/io/trino/sql/analyzer/TestAnalyzer.java +++ b/core/trino-main/src/test/java/io/trino/sql/analyzer/TestAnalyzer.java @@ -6847,9 +6847,14 @@ public void setup() Optional.of("comment"), Identity.ofUser("user"), ImmutableList.of(), - Optional.empty(), - ImmutableMap.of()); - inSetupTransaction(session -> metadata.createMaterializedView(session, new QualifiedObjectName(TPCH_CATALOG, "s1", "mv1"), materializedViewData1, false, true)); + Optional.empty()); + inSetupTransaction(session -> metadata.createMaterializedView( + session, + new QualifiedObjectName(TPCH_CATALOG, "s1", "mv1"), + materializedViewData1, + ImmutableMap.of(), + false, + true)); // valid view referencing table in same schema ViewDefinition viewData1 = new ViewDefinition( @@ -6971,8 +6976,8 @@ public void setup() Optional.empty(), Identity.ofUser("some user"), ImmutableList.of(), - Optional.of(new CatalogSchemaTableName(TPCH_CATALOG, "s1", "t1")), - ImmutableMap.of()), + Optional.of(new CatalogSchemaTableName(TPCH_CATALOG, "s1", "t1"))), + ImmutableMap.of(), false, false)); ViewDefinition viewDefinition = new ViewDefinition( @@ -7024,8 +7029,8 @@ public void setup() Identity.ofUser("some user"), ImmutableList.of(), // t3 has a, b column and hidden column x - Optional.of(new CatalogSchemaTableName(TPCH_CATALOG, "s1", "t3")), - ImmutableMap.of()), + Optional.of(new CatalogSchemaTableName(TPCH_CATALOG, "s1", "t3"))), + ImmutableMap.of(), false, false)); testingConnectorMetadata.markMaterializedViewIsFresh(freshMaterializedView.asSchemaTableName()); @@ -7043,8 +7048,8 @@ public void setup() Optional.empty(), Identity.ofUser("some user"), ImmutableList.of(), - Optional.of(new CatalogSchemaTableName(TPCH_CATALOG, "s1", "t2")), - ImmutableMap.of()), + Optional.of(new CatalogSchemaTableName(TPCH_CATALOG, "s1", "t2"))), + ImmutableMap.of(), false, false)); testingConnectorMetadata.markMaterializedViewIsFresh(freshMaterializedViewMismatchedColumnCount.asSchemaTableName()); @@ -7062,8 +7067,8 @@ public void setup() Optional.empty(), Identity.ofUser("some user"), ImmutableList.of(), - Optional.of(new CatalogSchemaTableName(TPCH_CATALOG, "s1", "t2")), - ImmutableMap.of()), + Optional.of(new CatalogSchemaTableName(TPCH_CATALOG, "s1", "t2"))), + ImmutableMap.of(), false, false)); testingConnectorMetadata.markMaterializedViewIsFresh(freshMaterializedMismatchedColumnName.asSchemaTableName()); @@ -7081,8 +7086,8 @@ public void setup() Optional.empty(), Identity.ofUser("some user"), ImmutableList.of(), - Optional.of(new CatalogSchemaTableName(TPCH_CATALOG, "s1", "t2")), - ImmutableMap.of()), + Optional.of(new CatalogSchemaTableName(TPCH_CATALOG, "s1", "t2"))), + ImmutableMap.of(), false, false)); testingConnectorMetadata.markMaterializedViewIsFresh(freshMaterializedMismatchedColumnType.asSchemaTableName()); diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/TestMaterializedViews.java b/core/trino-main/src/test/java/io/trino/sql/planner/TestMaterializedViews.java index 253be3ac8a244..0b5e5f98059a6 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/TestMaterializedViews.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/TestMaterializedViews.java @@ -134,13 +134,13 @@ protected LocalQueryRunner createLocalQueryRunner() Optional.empty(), Identity.ofUser("some user"), ImmutableList.of(), - Optional.of(new CatalogSchemaTableName(TEST_CATALOG_NAME, SCHEMA, "storage_table")), - ImmutableMap.of()); + Optional.of(new CatalogSchemaTableName(TEST_CATALOG_NAME, SCHEMA, "storage_table"))); queryRunner.inTransaction(session -> { metadata.createMaterializedView( session, freshMaterializedView, materializedViewDefinition, + ImmutableMap.of(), false, false); return null; @@ -153,6 +153,7 @@ protected LocalQueryRunner createLocalQueryRunner() session, notFreshMaterializedView, materializedViewDefinition, + ImmutableMap.of(), false, false); return null; @@ -167,14 +168,14 @@ protected LocalQueryRunner createLocalQueryRunner() Optional.empty(), Identity.ofUser("some user"), ImmutableList.of(), - Optional.of(new CatalogSchemaTableName(TEST_CATALOG_NAME, SCHEMA, "storage_table_with_casts")), - ImmutableMap.of()); + Optional.of(new CatalogSchemaTableName(TEST_CATALOG_NAME, SCHEMA, "storage_table_with_casts"))); QualifiedObjectName materializedViewWithCasts = new QualifiedObjectName(TEST_CATALOG_NAME, SCHEMA, "materialized_view_with_casts"); queryRunner.inTransaction(session -> { metadata.createMaterializedView( session, materializedViewWithCasts, materializedViewDefinitionWithCasts, + ImmutableMap.of(), false, false); return null; @@ -186,6 +187,7 @@ protected LocalQueryRunner createLocalQueryRunner() session, new QualifiedObjectName(TEST_CATALOG_NAME, SCHEMA, "stale_materialized_view_with_casts"), materializedViewDefinitionWithCasts, + ImmutableMap.of(), false, false); return null; diff --git a/core/trino-main/src/test/java/io/trino/sql/query/TestColumnMask.java b/core/trino-main/src/test/java/io/trino/sql/query/TestColumnMask.java index e1d6cc3420354..bce26d1224945 100644 --- a/core/trino-main/src/test/java/io/trino/sql/query/TestColumnMask.java +++ b/core/trino-main/src/test/java/io/trino/sql/query/TestColumnMask.java @@ -122,8 +122,7 @@ public TestColumnMask() Optional.of(Duration.ZERO), Optional.empty(), Optional.of(VIEW_OWNER), - ImmutableList.of(), - ImmutableMap.of()); + ImmutableList.of()); ConnectorMaterializedViewDefinition freshMaterializedView = new ConnectorMaterializedViewDefinition( "SELECT * FROM local.tiny.nation", @@ -138,8 +137,7 @@ public TestColumnMask() Optional.of(Duration.ZERO), Optional.empty(), Optional.of(VIEW_OWNER), - ImmutableList.of(), - ImmutableMap.of()); + ImmutableList.of()); ConnectorMaterializedViewDefinition materializedViewWithCasts = new ConnectorMaterializedViewDefinition( "SELECT nationkey, cast(name as varchar(1)) as name, regionkey, comment FROM local.tiny.nation", @@ -154,8 +152,7 @@ public TestColumnMask() Optional.of(Duration.ZERO), Optional.empty(), Optional.of(VIEW_OWNER), - ImmutableList.of(), - ImmutableMap.of()); + ImmutableList.of()); MockConnectorFactory mock = MockConnectorFactory.builder() .withGetColumns(schemaTableName -> { diff --git a/core/trino-main/src/test/java/io/trino/testing/TestTestingMetadata.java b/core/trino-main/src/test/java/io/trino/testing/TestTestingMetadata.java index 8b5bad533ce56..d188741497da5 100644 --- a/core/trino-main/src/test/java/io/trino/testing/TestTestingMetadata.java +++ b/core/trino-main/src/test/java/io/trino/testing/TestTestingMetadata.java @@ -43,7 +43,7 @@ private void testRenameMaterializedView(String source, String target) SchemaTableName newName = schemaTableName("schema", target); TestingMetadata metadata = new TestingMetadata(); ConnectorMaterializedViewDefinition viewDefinition = someMaterializedView(); - metadata.createMaterializedView(SESSION, initialName, viewDefinition, false, false); + metadata.createMaterializedView(SESSION, initialName, viewDefinition, ImmutableMap.of(), false, false); metadata.renameMaterializedView(SESSION, initialName, newName); @@ -62,7 +62,6 @@ private static ConnectorMaterializedViewDefinition someMaterializedView() Optional.of(Duration.ZERO), Optional.empty(), Optional.of("owner"), - ImmutableList.of(), - ImmutableMap.of()); + ImmutableList.of()); } } diff --git a/core/trino-spi/pom.xml b/core/trino-spi/pom.xml index 1f76552d4fc17..4c0ffdc0393fc 100644 --- a/core/trino-spi/pom.xml +++ b/core/trino-spi/pom.xml @@ -212,6 +212,20 @@ + + java.method.numberOfParametersChanged + method void io.trino.spi.connector.ConnectorMaterializedViewDefinition::<init>(java.lang.String, java.util.Optional<io.trino.spi.connector.CatalogSchemaTableName>, java.util.Optional<java.lang.String>, java.util.Optional<java.lang.String>, java.util.List<io.trino.spi.connector.ConnectorMaterializedViewDefinition.Column>, java.util.Optional<java.time.Duration>, java.util.Optional<java.lang.String>, java.util.Optional<java.lang.String>, java.util.List<io.trino.spi.connector.CatalogSchemaName>, java.util.Map<java.lang.String, java.lang.Object>) + method void io.trino.spi.connector.ConnectorMaterializedViewDefinition::<init>(java.lang.String, java.util.Optional<io.trino.spi.connector.CatalogSchemaTableName>, java.util.Optional<java.lang.String>, java.util.Optional<java.lang.String>, java.util.List<io.trino.spi.connector.ConnectorMaterializedViewDefinition.Column>, java.util.Optional<java.time.Duration>, java.util.Optional<java.lang.String>, java.util.Optional<java.lang.String>, java.util.List<io.trino.spi.connector.CatalogSchemaName>) + + + java.method.removed + method java.util.Map<java.lang.String, java.lang.Object> io.trino.spi.connector.ConnectorMaterializedViewDefinition::getProperties() + + + java.method.numberOfParametersChanged + method void io.trino.spi.connector.ConnectorMetadata::createMaterializedView(io.trino.spi.connector.ConnectorSession, io.trino.spi.connector.SchemaTableName, io.trino.spi.connector.ConnectorMaterializedViewDefinition, boolean, boolean) + method void io.trino.spi.connector.ConnectorMetadata::createMaterializedView(io.trino.spi.connector.ConnectorSession, io.trino.spi.connector.SchemaTableName, io.trino.spi.connector.ConnectorMaterializedViewDefinition, java.util.Map<java.lang.String, java.lang.Object>, boolean, boolean) + diff --git a/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorMaterializedViewDefinition.java b/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorMaterializedViewDefinition.java index 98a429d3dceef..d7ebd0097f493 100644 --- a/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorMaterializedViewDefinition.java +++ b/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorMaterializedViewDefinition.java @@ -17,7 +17,6 @@ import java.time.Duration; import java.util.List; -import java.util.Map; import java.util.Objects; import java.util.Optional; import java.util.StringJoiner; @@ -37,7 +36,6 @@ public class ConnectorMaterializedViewDefinition private final Optional comment; private final Optional owner; private final List path; - private final Map properties; public ConnectorMaterializedViewDefinition( String originalSql, @@ -48,8 +46,7 @@ public ConnectorMaterializedViewDefinition( Optional gracePeriod, Optional comment, Optional owner, - List path, - Map properties) + List path) { this.originalSql = requireNonNull(originalSql, "originalSql is null"); this.storageTable = requireNonNull(storageTable, "storageTable is null"); @@ -61,7 +58,6 @@ public ConnectorMaterializedViewDefinition( this.comment = requireNonNull(comment, "comment is null"); this.owner = requireNonNull(owner, "owner is null"); this.path = List.copyOf(path); - this.properties = requireNonNull(properties, "properties are null"); if (catalog.isEmpty() && schema.isPresent()) { throw new IllegalArgumentException("catalog must be present if schema is present"); @@ -116,11 +112,6 @@ public List getPath() return path; } - public Map getProperties() - { - return properties; - } - @Override public String toString() { @@ -133,9 +124,8 @@ public String toString() gracePeriod.ifPresent(value -> joiner.add("gracePeriod=" + gracePeriod)); comment.ifPresent(value -> joiner.add("comment=" + value)); joiner.add("owner=" + owner); - joiner.add("properties=" + properties); joiner.add(path.stream().map(CatalogSchemaName::toString).collect(joining(", ", "path=(", ")"))); - return getClass().getSimpleName() + joiner.toString(); + return getClass().getSimpleName() + joiner; } @Override @@ -156,14 +146,13 @@ public boolean equals(Object o) Objects.equals(gracePeriod, that.gracePeriod) && Objects.equals(comment, that.comment) && Objects.equals(owner, that.owner) && - Objects.equals(path, that.path) && - Objects.equals(properties, that.properties); + Objects.equals(path, that.path); } @Override public int hashCode() { - return Objects.hash(originalSql, storageTable, catalog, schema, columns, gracePeriod, comment, owner, path, properties); + return Objects.hash(originalSql, storageTable, catalog, schema, columns, gracePeriod, comment, owner, path); } public static final class Column diff --git a/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorMetadata.java b/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorMetadata.java index 3a006f8a0a84e..5f0bdc7602b50 100644 --- a/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorMetadata.java +++ b/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorMetadata.java @@ -1661,7 +1661,13 @@ default void validateScan(ConnectorSession session, ConnectorTableHandle handle) * * @throws TrinoException with {@code ALREADY_EXISTS} if the object already exists and {@param ignoreExisting} is not set */ - default void createMaterializedView(ConnectorSession session, SchemaTableName viewName, ConnectorMaterializedViewDefinition definition, boolean replace, boolean ignoreExisting) + default void createMaterializedView( + ConnectorSession session, + SchemaTableName viewName, + ConnectorMaterializedViewDefinition definition, + Map properties, + boolean replace, + boolean ignoreExisting) { throw new TrinoException(NOT_SUPPORTED, "This connector does not support creating materialized views"); } @@ -1709,6 +1715,11 @@ default Optional getMaterializedView(Connec return Optional.empty(); } + default Map getMaterializedViewProperties(ConnectorSession session, SchemaTableName viewName, ConnectorMaterializedViewDefinition materializedViewDefinition) + { + throw new TrinoException(NOT_SUPPORTED, "This connector does not support materialized views"); + } + /** * The method is used by the engine to determine if a materialized view is current with respect to the tables it depends on. * diff --git a/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/classloader/ClassLoaderSafeConnectorMetadata.java b/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/classloader/ClassLoaderSafeConnectorMetadata.java index dfbed636f83b6..05547e0d26f5c 100644 --- a/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/classloader/ClassLoaderSafeConnectorMetadata.java +++ b/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/classloader/ClassLoaderSafeConnectorMetadata.java @@ -1089,10 +1089,16 @@ public void validateScan(ConnectorSession session, ConnectorTableHandle handle) } @Override - public void createMaterializedView(ConnectorSession session, SchemaTableName viewName, ConnectorMaterializedViewDefinition definition, boolean replace, boolean ignoreExisting) + public void createMaterializedView( + ConnectorSession session, + SchemaTableName viewName, + ConnectorMaterializedViewDefinition definition, + Map properties, + boolean replace, + boolean ignoreExisting) { try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { - delegate.createMaterializedView(session, viewName, definition, replace, ignoreExisting); + delegate.createMaterializedView(session, viewName, definition, properties, replace, ignoreExisting); } } @@ -1128,6 +1134,14 @@ public Optional getMaterializedView(Connect } } + @Override + public Map getMaterializedViewProperties(ConnectorSession session, SchemaTableName viewName, ConnectorMaterializedViewDefinition viewDefinition) + { + try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { + return delegate.getMaterializedViewProperties(session, viewName, viewDefinition); + } + } + @Override public MaterializedViewFreshness getMaterializedViewFreshness(ConnectorSession session, SchemaTableName name) { diff --git a/plugin/trino-bigquery/src/main/java/io/trino/plugin/bigquery/BigQueryMetadata.java b/plugin/trino-bigquery/src/main/java/io/trino/plugin/bigquery/BigQueryMetadata.java index fd5e88a633ee7..f0e6341005f60 100644 --- a/plugin/trino-bigquery/src/main/java/io/trino/plugin/bigquery/BigQueryMetadata.java +++ b/plugin/trino-bigquery/src/main/java/io/trino/plugin/bigquery/BigQueryMetadata.java @@ -723,10 +723,16 @@ public ConnectorMergeTableHandle beginMerge(ConnectorSession session, ConnectorT } @Override - public void createMaterializedView(ConnectorSession session, SchemaTableName viewName, ConnectorMaterializedViewDefinition definition, boolean replace, boolean ignoreExisting) + public void createMaterializedView( + ConnectorSession session, + SchemaTableName viewName, + ConnectorMaterializedViewDefinition definition, + Map properties, + boolean replace, + boolean ignoreExisting) { // TODO Fix BaseBigQueryFailureRecoveryTest when implementing this method - ConnectorMetadata.super.createMaterializedView(session, viewName, definition, replace, ignoreExisting); + ConnectorMetadata.super.createMaterializedView(session, viewName, definition, properties, replace, ignoreExisting); } @Override diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMaterializedViewMetadata.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMaterializedViewMetadata.java index f47ad0dba86cd..b2f405f242117 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMaterializedViewMetadata.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMaterializedViewMetadata.java @@ -25,7 +25,13 @@ public interface HiveMaterializedViewMetadata { - void createMaterializedView(ConnectorSession session, SchemaTableName viewName, ConnectorMaterializedViewDefinition definition, boolean replace, boolean ignoreExisting); + void createMaterializedView( + ConnectorSession session, + SchemaTableName viewName, + ConnectorMaterializedViewDefinition definition, + Map properties, + boolean replace, + boolean ignoreExisting); void dropMaterializedView(ConnectorSession session, SchemaTableName viewName); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadata.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadata.java index a6c53d5543a56..c95a23fde674f 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadata.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadata.java @@ -3857,9 +3857,15 @@ public void cleanupQuery(ConnectorSession session) } @Override - public void createMaterializedView(ConnectorSession session, SchemaTableName viewName, ConnectorMaterializedViewDefinition definition, boolean replace, boolean ignoreExisting) + public void createMaterializedView( + ConnectorSession session, + SchemaTableName viewName, + ConnectorMaterializedViewDefinition definition, + Map properties, + boolean replace, + boolean ignoreExisting) { - hiveMaterializedViewMetadata.createMaterializedView(session, viewName, definition, replace, ignoreExisting); + hiveMaterializedViewMetadata.createMaterializedView(session, viewName, definition, properties, replace, ignoreExisting); } @Override diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/NoneHiveMaterializedViewMetadata.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/NoneHiveMaterializedViewMetadata.java index a23d562b616de..0c99c96f39976 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/NoneHiveMaterializedViewMetadata.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/NoneHiveMaterializedViewMetadata.java @@ -33,7 +33,13 @@ public class NoneHiveMaterializedViewMetadata implements HiveMaterializedViewMetadata { @Override - public void createMaterializedView(ConnectorSession session, SchemaTableName viewName, ConnectorMaterializedViewDefinition definition, boolean replace, boolean ignoreExisting) + public void createMaterializedView( + ConnectorSession session, + SchemaTableName viewName, + ConnectorMaterializedViewDefinition definition, + Map properties, + boolean replace, + boolean ignoreExisting) { throw new TrinoException(NOT_SUPPORTED, "This connector does not support creating materialized views"); } diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java index 34c85ee6a89ae..9595bb202cde9 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java @@ -2749,9 +2749,15 @@ Table getIcebergTable(ConnectorSession session, SchemaTableName schemaTableName) } @Override - public void createMaterializedView(ConnectorSession session, SchemaTableName viewName, ConnectorMaterializedViewDefinition definition, boolean replace, boolean ignoreExisting) + public void createMaterializedView( + ConnectorSession session, + SchemaTableName viewName, + ConnectorMaterializedViewDefinition definition, + Map properties, + boolean replace, + boolean ignoreExisting) { - catalog.createMaterializedView(session, viewName, definition, replace, ignoreExisting); + catalog.createMaterializedView(session, viewName, definition, properties, replace, ignoreExisting); } @Override @@ -2881,6 +2887,12 @@ public Optional getMaterializedView(Connect return catalog.getMaterializedView(session, viewName); } + @Override + public Map getMaterializedViewProperties(ConnectorSession session, SchemaTableName viewName, ConnectorMaterializedViewDefinition definition) + { + return catalog.getMaterializedViewProperties(session, viewName, definition); + } + @Override public void renameMaterializedView(ConnectorSession session, SchemaTableName source, SchemaTableName target) { diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/AbstractTrinoCatalog.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/AbstractTrinoCatalog.java index f9e80d3eb42f6..9f3627088aa0d 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/AbstractTrinoCatalog.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/AbstractTrinoCatalog.java @@ -77,6 +77,7 @@ import static io.trino.plugin.hive.metastore.glue.converter.GlueToTrinoConverter.mappedCopy; import static io.trino.plugin.hive.util.HiveUtil.escapeTableName; import static io.trino.plugin.iceberg.IcebergErrorCode.ICEBERG_FILESYSTEM_ERROR; +import static io.trino.plugin.iceberg.IcebergErrorCode.ICEBERG_INVALID_METADATA; import static io.trino.plugin.iceberg.IcebergMaterializedViewAdditionalProperties.STORAGE_SCHEMA; import static io.trino.plugin.iceberg.IcebergMaterializedViewAdditionalProperties.getStorageSchema; import static io.trino.plugin.iceberg.IcebergMaterializedViewDefinition.decodeMaterializedViewData; @@ -201,6 +202,25 @@ public Optional getMaterializedView(Connect protected abstract Optional doGetMaterializedView(ConnectorSession session, SchemaTableName schemaViewName); + @Override + public Map getMaterializedViewProperties(ConnectorSession session, SchemaTableName viewName, ConnectorMaterializedViewDefinition definition) + { + SchemaTableName storageTableName = definition.getStorageTable() + .orElseThrow(() -> new TrinoException(ICEBERG_INVALID_METADATA, "Materialized view definition is missing a storage table")) + .getSchemaTableName(); + + try { + Table storageTable = loadTable(session, definition.getStorageTable().orElseThrow().getSchemaTableName()); + return ImmutableMap.builder() + .putAll(getIcebergTableProperties(storageTable)) + .put(STORAGE_SCHEMA, storageTableName.getSchemaName()) + .buildOrThrow(); + } + catch (RuntimeException e) { + throw new TrinoException(ICEBERG_FILESYSTEM_ERROR, "Unable to load storage table metadata for materialized view: " + viewName); + } + } + protected Transaction newCreateTableTransaction( ConnectorSession session, SchemaTableName schemaTableName, @@ -281,20 +301,24 @@ protected void deleteTableDirectory(TrinoFileSystem fileSystem, SchemaTableName } } - protected Location createMaterializedViewStorage(ConnectorSession session, SchemaTableName viewName, ConnectorMaterializedViewDefinition definition) + protected Location createMaterializedViewStorage( + ConnectorSession session, + SchemaTableName viewName, + ConnectorMaterializedViewDefinition definition, + Map materializedViewProperties) { - if (getStorageSchema(definition.getProperties()).isPresent()) { + if (getStorageSchema(materializedViewProperties).isPresent()) { throw new TrinoException(NOT_SUPPORTED, "Materialized view property '%s' is not supported when hiding materialized view storage tables is enabled".formatted(STORAGE_SCHEMA)); } SchemaTableName storageTableName = new SchemaTableName(viewName.getSchemaName(), tableNameWithType(viewName.getTableName(), MATERIALIZED_VIEW_STORAGE)); - String tableLocation = getTableLocation(definition.getProperties()) + String tableLocation = getTableLocation(materializedViewProperties) .orElseGet(() -> defaultTableLocation(session, viewName)); - List columns = columnsForMaterializedView(definition); + List columns = columnsForMaterializedView(definition, materializedViewProperties); Schema schema = schemaFromMetadata(columns); - PartitionSpec partitionSpec = parsePartitionFields(schema, getPartitioning(definition.getProperties())); - SortOrder sortOrder = parseSortFields(schema, getSortOrder(definition.getProperties())); - Map properties = createTableProperties(new ConnectorTableMetadata(storageTableName, columns, definition.getProperties(), Optional.empty())); + PartitionSpec partitionSpec = parsePartitionFields(schema, getPartitioning(materializedViewProperties)); + SortOrder sortOrder = parseSortFields(schema, getSortOrder(materializedViewProperties)); + Map properties = createTableProperties(new ConnectorTableMetadata(storageTableName, columns, materializedViewProperties, Optional.empty())); TableMetadata metadata = newTableMetadata(schema, partitionSpec, sortOrder, tableLocation, properties); @@ -307,17 +331,21 @@ protected Location createMaterializedViewStorage(ConnectorSession session, Schem return metadataFileLocation; } - protected SchemaTableName createMaterializedViewStorageTable(ConnectorSession session, SchemaTableName viewName, ConnectorMaterializedViewDefinition definition) + protected SchemaTableName createMaterializedViewStorageTable( + ConnectorSession session, + SchemaTableName viewName, + ConnectorMaterializedViewDefinition definition, + Map materializedViewProperties) { // Generate a storage table name and create a storage table. The properties in the definition are table properties for the // storage table as indicated in the materialized view definition. String storageTableName = "st_" + randomUUID().toString().replace("-", ""); - String storageSchema = getStorageSchema(definition.getProperties()).orElse(viewName.getSchemaName()); + String storageSchema = getStorageSchema(materializedViewProperties).orElse(viewName.getSchemaName()); SchemaTableName storageTable = new SchemaTableName(storageSchema, storageTableName); - List columns = columnsForMaterializedView(definition); + List columns = columnsForMaterializedView(definition, materializedViewProperties); - ConnectorTableMetadata tableMetadata = new ConnectorTableMetadata(storageTable, columns, definition.getProperties(), Optional.empty()); + ConnectorTableMetadata tableMetadata = new ConnectorTableMetadata(storageTable, columns, materializedViewProperties, Optional.empty()); Transaction transaction = IcebergUtil.newCreateTableTransaction(this, tableMetadata, session, false); AppendFiles appendFiles = transaction.newAppend(); commit(appendFiles, session); @@ -325,7 +353,7 @@ protected SchemaTableName createMaterializedViewStorageTable(ConnectorSession se return storageTable; } - private List columnsForMaterializedView(ConnectorMaterializedViewDefinition definition) + private List columnsForMaterializedView(ConnectorMaterializedViewDefinition definition, Map materializedViewProperties) { Schema schemaWithTimestampTzPreserved = schemaFromMetadata(mappedCopy( definition.getColumns(), @@ -340,7 +368,7 @@ private List columnsForMaterializedView(ConnectorMaterializedVie } return new ColumnMetadata(column.getName(), type); })); - PartitionSpec partitionSpec = parsePartitionFields(schemaWithTimestampTzPreserved, getPartitioning(definition.getProperties())); + PartitionSpec partitionSpec = parsePartitionFields(schemaWithTimestampTzPreserved, getPartitioning(materializedViewProperties)); Set temporalPartitioningSources = partitionSpec.fields().stream() .flatMap(partitionField -> { Types.NestedField sourceField = schemaWithTimestampTzPreserved.findField(partitionField.sourceId()); @@ -422,7 +450,6 @@ private Type typeForMaterializedViewStorageTable(Type type) } protected ConnectorMaterializedViewDefinition getMaterializedViewDefinition( - Table icebergTable, Optional owner, String viewOriginalText, SchemaTableName storageTableName) @@ -437,11 +464,7 @@ protected ConnectorMaterializedViewDefinition getMaterializedViewDefinition( definition.getGracePeriod(), definition.getComment(), owner, - definition.getPath(), - ImmutableMap.builder() - .putAll(getIcebergTableProperties(icebergTable)) - .put(STORAGE_SCHEMA, storageTableName.getSchemaName()) - .buildOrThrow()); + definition.getPath()); } protected List toSpiMaterializedViewColumns(List columns) diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/TrinoCatalog.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/TrinoCatalog.java index 8d44190ee3079..e2aa957148ae4 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/TrinoCatalog.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/TrinoCatalog.java @@ -163,6 +163,7 @@ void createMaterializedView( ConnectorSession session, SchemaTableName viewName, ConnectorMaterializedViewDefinition definition, + Map materializedViewProperties, boolean replace, boolean ignoreExisting); @@ -172,6 +173,8 @@ void createMaterializedView( Optional getMaterializedView(ConnectorSession session, SchemaTableName viewName); + Map getMaterializedViewProperties(ConnectorSession session, SchemaTableName viewName, ConnectorMaterializedViewDefinition definition); + Optional getMaterializedViewStorageTable(ConnectorSession session, SchemaTableName viewName); void renameMaterializedView(ConnectorSession session, SchemaTableName source, SchemaTableName target); diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/TrinoGlueCatalog.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/TrinoGlueCatalog.java index fd79f33587d37..b901158e8d0d8 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/TrinoGlueCatalog.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/TrinoGlueCatalog.java @@ -866,7 +866,7 @@ else if (isTrinoMaterializedView(tableType, parameters)) { try { // Note: this is racy from cache invalidation perspective, but it should not matter here uncheckedCacheGet(materializedViewCache, schemaTableName, () -> { - ConnectorMaterializedViewDefinition materializedView = createMaterializedViewDefinition(session, schemaTableName, table); + ConnectorMaterializedViewDefinition materializedView = createMaterializedViewDefinition(schemaTableName, table); return new MaterializedViewData( materializedView, Optional.ofNullable(parameters.get(METADATA_LOCATION_PROP))); @@ -1128,6 +1128,7 @@ public void createMaterializedView( ConnectorSession session, SchemaTableName viewName, ConnectorMaterializedViewDefinition definition, + Map materializedViewProperties, boolean replace, boolean ignoreExisting) { @@ -1146,7 +1147,7 @@ public void createMaterializedView( } if (hideMaterializedViewStorageTable) { - Location storageMetadataLocation = createMaterializedViewStorage(session, viewName, definition); + Location storageMetadataLocation = createMaterializedViewStorage(session, viewName, definition, materializedViewProperties); TableInput materializedViewTableInput = getMaterializedViewTableInput( viewName.getTableName(), encodeMaterializedViewData(fromConnectorMaterializedViewDefinition(definition)), @@ -1160,7 +1161,7 @@ public void createMaterializedView( } } else { - createMaterializedViewWithStorageTable(session, viewName, definition, existing); + createMaterializedViewWithStorageTable(session, viewName, definition, materializedViewProperties, existing); } } @@ -1168,10 +1169,11 @@ private void createMaterializedViewWithStorageTable( ConnectorSession session, SchemaTableName viewName, ConnectorMaterializedViewDefinition definition, + Map materializedViewProperties, Optional existing) { // Create the storage table - SchemaTableName storageTable = createMaterializedViewStorageTable(session, viewName, definition); + SchemaTableName storageTable = createMaterializedViewStorageTable(session, viewName, definition, materializedViewProperties); // Create a view indicating the storage table TableInput materializedViewTableInput = getMaterializedViewTableInput( viewName.getTableName(), @@ -1218,8 +1220,7 @@ public void updateMaterializedViewColumnComment(ConnectorSession session, Schema definition.getGracePeriod(), definition.getComment(), definition.getOwner(), - definition.getPath(), - definition.getProperties()); + definition.getPath()); updateMaterializedView(viewName, newDefinition); } @@ -1293,11 +1294,10 @@ protected Optional doGetMaterializedView(Co return Optional.empty(); } - return Optional.of(createMaterializedViewDefinition(session, viewName, table)); + return Optional.of(createMaterializedViewDefinition(viewName, table)); } private ConnectorMaterializedViewDefinition createMaterializedViewDefinition( - ConnectorSession session, SchemaTableName viewName, com.amazonaws.services.glue.model.Table table) { @@ -1315,54 +1315,18 @@ private ConnectorMaterializedViewDefinition createMaterializedViewDefinition( .orElse(viewName.getSchemaName()); SchemaTableName storageTableName = new SchemaTableName(storageSchema, storageTable); - Table icebergTable; - try { - icebergTable = loadTable(session, storageTableName); - } - catch (RuntimeException e) { - // The materialized view could be removed concurrently. This may manifest in a number of ways, e.g. - // - io.trino.spi.connector.TableNotFoundException - // - org.apache.iceberg.exceptions.NotFoundException when accessing manifest file - // - other failures when reading storage table's metadata files - // Retry, as we're catching broadly. - throw new MaterializedViewMayBeBeingRemovedException(e); - } - String viewOriginalText = table.getViewOriginalText(); if (viewOriginalText == null) { throw new TrinoException(ICEBERG_BAD_DATA, "Materialized view did not have original text " + viewName); } return getMaterializedViewDefinition( - icebergTable, Optional.ofNullable(table.getOwner()), viewOriginalText, storageTableName); } SchemaTableName storageTableName = new SchemaTableName(viewName.getSchemaName(), tableNameWithType(viewName.getTableName(), MATERIALIZED_VIEW_STORAGE)); - Table icebergTable; - try { - TableMetadata metadata = getMaterializedViewTableMetadata(session, storageTableName, storageMetadataLocation); - IcebergTableOperations operations = tableOperationsProvider.createTableOperations( - this, - session, - storageTableName.getSchemaName(), - storageTableName.getTableName(), - Optional.empty(), - Optional.empty()); - operations.initializeFromMetadata(metadata); - icebergTable = new BaseTable(operations, quotedTableName(storageTableName), TRINO_METRICS_REPORTER); - } - catch (RuntimeException e) { - // The materialized view could be removed concurrently. This may manifest in a number of ways, e.g. - // - org.apache.iceberg.exceptions.NotFoundException when accessing manifest file - // - other failures when reading storage table's metadata files - // Retry, as we're catching broadly. - throw new MaterializedViewMayBeBeingRemovedException(e); - } - return getMaterializedViewDefinition( - icebergTable, Optional.ofNullable(table.getOwner()), table.getViewOriginalText(), storageTableName); diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/TrinoHiveCatalog.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/TrinoHiveCatalog.java index 6d0027e1c26df..c2de70ba78e10 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/TrinoHiveCatalog.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/TrinoHiveCatalog.java @@ -568,6 +568,7 @@ public void createMaterializedView( ConnectorSession session, SchemaTableName viewName, ConnectorMaterializedViewDefinition definition, + Map materializedViewProperties, boolean replace, boolean ignoreExisting) { @@ -586,7 +587,7 @@ public void createMaterializedView( } if (hideMaterializedViewStorageTable) { - Location storageMetadataLocation = createMaterializedViewStorage(session, viewName, definition); + Location storageMetadataLocation = createMaterializedViewStorage(session, viewName, definition, materializedViewProperties); Map viewProperties = createMaterializedViewProperties(session, storageMetadataLocation); Column dummyColumn = new Column("dummy", HIVE_STRING, Optional.empty(), ImmutableMap.of()); @@ -614,7 +615,7 @@ public void createMaterializedView( } } else { - createMaterializedViewWithStorageTable(session, viewName, definition, existing); + createMaterializedViewWithStorageTable(session, viewName, definition, materializedViewProperties, existing); } } @@ -622,9 +623,10 @@ private void createMaterializedViewWithStorageTable( ConnectorSession session, SchemaTableName viewName, ConnectorMaterializedViewDefinition definition, + Map materializedViewProperties, Optional existing) { - SchemaTableName storageTable = createMaterializedViewStorageTable(session, viewName, definition); + SchemaTableName storageTable = createMaterializedViewStorageTable(session, viewName, definition, materializedViewProperties); // Create a view indicating the storage table Map viewProperties = createMaterializedViewProperties(session, storageTable); @@ -686,8 +688,7 @@ public void updateMaterializedViewColumnComment(ConnectorSession session, Schema definition.getGracePeriod(), definition.getComment(), definition.getOwner(), - definition.getPath(), - definition.getProperties()); + definition.getPath()); replaceMaterializedView(session, viewName, existing, newDefinition); } @@ -766,23 +767,7 @@ protected Optional doGetMaterializedView(Co String storageSchema = Optional.ofNullable(materializedView.getParameters().get(STORAGE_SCHEMA)) .orElse(viewName.getSchemaName()); SchemaTableName storageTableName = new SchemaTableName(storageSchema, storageTable); - - Table icebergTable; - try { - icebergTable = loadTable(session, storageTableName); - } - catch (RuntimeException e) { - // The materialized view could be removed concurrently. This may manifest in a number of ways, e.g. - // - io.trino.spi.connector.TableNotFoundException - // - org.apache.iceberg.exceptions.NotFoundException when accessing manifest file - // - other failures when reading storage table's metadata files - // Retry, as we're catching broadly. - metastore.invalidateTable(viewName.getSchemaName(), viewName.getTableName()); - metastore.invalidateTable(storageSchema, storageTable); - throw new MaterializedViewMayBeBeingRemovedException(e); - } return Optional.of(getMaterializedViewDefinition( - icebergTable, materializedView.getOwner(), materializedView.getViewOriginalText() .orElseThrow(() -> new TrinoException(HIVE_INVALID_METADATA, "No view original text: " + viewName)), @@ -790,33 +775,11 @@ protected Optional doGetMaterializedView(Co } SchemaTableName storageTableName = new SchemaTableName(viewName.getSchemaName(), IcebergTableName.tableNameWithType(viewName.getTableName(), MATERIALIZED_VIEW_STORAGE)); - IcebergTableOperations operations = tableOperationsProvider.createTableOperations( - this, - session, - storageTableName.getSchemaName(), - storageTableName.getTableName(), - Optional.empty(), - Optional.empty()); - try { - TableMetadata metadata = getMaterializedViewTableMetadata(session, storageTableName, materializedView); - operations.initializeFromMetadata(metadata); - Table icebergTable = new BaseTable(operations, quotedTableName(storageTableName), TRINO_METRICS_REPORTER); - - return Optional.of(getMaterializedViewDefinition( - icebergTable, - materializedView.getOwner(), - materializedView.getViewOriginalText() - .orElseThrow(() -> new TrinoException(HIVE_INVALID_METADATA, "No view original text: " + viewName)), - storageTableName)); - } - catch (RuntimeException e) { - // The materialized view could be removed concurrently. This may manifest in a number of ways, e.g. - // - org.apache.iceberg.exceptions.NotFoundException when accessing manifest file - // - other failures when reading storage table's metadata files - // Retry, as we're catching broadly. - metastore.invalidateTable(viewName.getSchemaName(), viewName.getTableName()); - throw new MaterializedViewMayBeBeingRemovedException(e); - } + return Optional.of(getMaterializedViewDefinition( + materializedView.getOwner(), + materializedView.getViewOriginalText() + .orElseThrow(() -> new TrinoException(HIVE_INVALID_METADATA, "No view original text: " + viewName)), + storageTableName)); } @Override diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/jdbc/TrinoJdbcCatalog.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/jdbc/TrinoJdbcCatalog.java index 86a8d3191e9a6..1bde8682c8644 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/jdbc/TrinoJdbcCatalog.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/jdbc/TrinoJdbcCatalog.java @@ -445,7 +445,13 @@ public Optional getMaterializedViewStorageTable(ConnectorSession sess } @Override - public void createMaterializedView(ConnectorSession session, SchemaTableName schemaViewName, ConnectorMaterializedViewDefinition definition, boolean replace, boolean ignoreExisting) + public void createMaterializedView( + ConnectorSession session, + SchemaTableName schemaViewName, + ConnectorMaterializedViewDefinition definition, + Map materializedViewProperties, + boolean replace, + boolean ignoreExisting) { throw new TrinoException(NOT_SUPPORTED, "createMaterializedView is not supported for Iceberg JDBC catalogs"); } @@ -468,6 +474,12 @@ public Optional getMaterializedView(Connect return Optional.empty(); } + @Override + public Map getMaterializedViewProperties(ConnectorSession session, SchemaTableName viewName, ConnectorMaterializedViewDefinition definition) + { + throw new TrinoException(NOT_SUPPORTED, "getMaterializedViewProperties is not supported for Iceberg JDBC catalogs"); + } + @Override public void renameMaterializedView(ConnectorSession session, SchemaTableName source, SchemaTableName target) { diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/nessie/TrinoNessieCatalog.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/nessie/TrinoNessieCatalog.java index 361c2cfd6986a..1a8c3544920eb 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/nessie/TrinoNessieCatalog.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/nessie/TrinoNessieCatalog.java @@ -405,6 +405,7 @@ public void createMaterializedView( ConnectorSession session, SchemaTableName schemaViewName, ConnectorMaterializedViewDefinition definition, + Map materializedViewProperties, boolean replace, boolean ignoreExisting) { @@ -429,6 +430,12 @@ public Optional getMaterializedView(Connect return Optional.empty(); } + @Override + public Map getMaterializedViewProperties(ConnectorSession session, SchemaTableName viewName, ConnectorMaterializedViewDefinition definition) + { + throw new TrinoException(NOT_SUPPORTED, "The Iceberg Nessie catalog does not support materialized views"); + } + @Override public Optional getMaterializedViewStorageTable(ConnectorSession session, SchemaTableName viewName) { diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/rest/TrinoRestCatalog.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/rest/TrinoRestCatalog.java index 74d296d574edd..29f6ef8e24431 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/rest/TrinoRestCatalog.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/rest/TrinoRestCatalog.java @@ -438,7 +438,13 @@ public List listMaterializedViews(ConnectorSession session, Opt } @Override - public void createMaterializedView(ConnectorSession session, SchemaTableName viewName, ConnectorMaterializedViewDefinition definition, boolean replace, boolean ignoreExisting) + public void createMaterializedView( + ConnectorSession session, + SchemaTableName viewName, + ConnectorMaterializedViewDefinition definition, + Map materializedViewProperties, + boolean replace, + boolean ignoreExisting) { throw new TrinoException(NOT_SUPPORTED, "createMaterializedView is not supported for Iceberg REST catalog"); } @@ -461,6 +467,12 @@ public Optional getMaterializedView(Connect return Optional.empty(); } + @Override + public Map getMaterializedViewProperties(ConnectorSession session, SchemaTableName viewName, ConnectorMaterializedViewDefinition definition) + { + throw new TrinoException(NOT_SUPPORTED, "The Iceberg REST catalog does not support materialized views"); + } + @Override public Optional getMaterializedViewStorageTable(ConnectorSession session, SchemaTableName viewName) { diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergFileOperations.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergFileOperations.java index 4aa73b73b659f..236ca6d515a0e 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergFileOperations.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergFileOperations.java @@ -762,10 +762,10 @@ public void testSystemMetadataMaterializedViews() .build()); // Bulk retrieval without selecting freshness - assertFileSystemAccesses(session, "SELECT schema_name, name FROM system.metadata.materialized_views WHERE schema_name = CURRENT_SCHEMA", - ImmutableMultiset.builder() - .addCopies(new FileOperation(METADATA_JSON, INPUT_FILE_NEW_STREAM), 2) - .build()); + assertFileSystemAccesses( + session, + "SELECT schema_name, name FROM system.metadata.materialized_views WHERE schema_name = CURRENT_SCHEMA", + ImmutableMultiset.of()); // Bulk retrieval for two schemas assertFileSystemAccesses(session, "SELECT * FROM system.metadata.materialized_views WHERE schema_name IN (CURRENT_SCHEMA, 'non_existent')", @@ -780,15 +780,15 @@ public void testSystemMetadataMaterializedViews() .build()); // Pointed lookup without selecting freshness - assertFileSystemAccesses(session, "SELECT schema_name, name FROM system.metadata.materialized_views WHERE schema_name = CURRENT_SCHEMA AND name = 'mv1'", - ImmutableMultiset.builder() - .add(new FileOperation(METADATA_JSON, INPUT_FILE_NEW_STREAM)) - .build()); + assertFileSystemAccesses( + session, + "SELECT schema_name, name FROM system.metadata.materialized_views WHERE schema_name = CURRENT_SCHEMA AND name = 'mv1'", + ImmutableMultiset.of()); - assertFileSystemAccesses(session, "SELECT * FROM iceberg.information_schema.columns WHERE table_schema = CURRENT_SCHEMA AND table_name = 'mv1'", - ImmutableMultiset.builder() - .add(new FileOperation(METADATA_JSON, INPUT_FILE_NEW_STREAM)) - .build()); + assertFileSystemAccesses( + session, + "SELECT * FROM iceberg.information_schema.columns WHERE table_schema = CURRENT_SCHEMA AND table_name = 'mv1'", + ImmutableMultiset.of()); assertUpdate("DROP SCHEMA " + schemaName + " CASCADE"); } diff --git a/testing/trino-tests/src/test/java/io/trino/execution/TestEventListenerBasic.java b/testing/trino-tests/src/test/java/io/trino/execution/TestEventListenerBasic.java index ae77214a462fd..a583cfc40ec05 100644 --- a/testing/trino-tests/src/test/java/io/trino/execution/TestEventListenerBasic.java +++ b/testing/trino-tests/src/test/java/io/trino/execution/TestEventListenerBasic.java @@ -180,8 +180,7 @@ public Iterable getConnectorFactories() Optional.of(Duration.ZERO), Optional.empty(), Optional.of("alice"), - ImmutableList.of(), - ImmutableMap.of()); + ImmutableList.of()); SchemaTableName materializedViewName = new SchemaTableName("default", "test_materialized_view"); return ImmutableMap.of(materializedViewName, definition); }) diff --git a/testing/trino-tests/src/test/java/io/trino/execution/TestRefreshMaterializedView.java b/testing/trino-tests/src/test/java/io/trino/execution/TestRefreshMaterializedView.java index 423f58b13101e..316f87b8b5134 100644 --- a/testing/trino-tests/src/test/java/io/trino/execution/TestRefreshMaterializedView.java +++ b/testing/trino-tests/src/test/java/io/trino/execution/TestRefreshMaterializedView.java @@ -109,8 +109,7 @@ protected QueryRunner createQueryRunner() Optional.of(Duration.ZERO), Optional.empty(), Optional.of("alice"), - ImmutableList.of(), - ImmutableMap.of()))) + ImmutableList.of()))) .withDelegateMaterializedViewRefreshToConnector((connectorSession, schemaTableName) -> true) .withRefreshMaterializedView((connectorSession, schemaTableName) -> { startRefreshMaterializedView.set(null); diff --git a/testing/trino-tests/src/test/java/io/trino/security/TestAccessControl.java b/testing/trino-tests/src/test/java/io/trino/security/TestAccessControl.java index 8daed713e7599..e86fe06d1d5d6 100644 --- a/testing/trino-tests/src/test/java/io/trino/security/TestAccessControl.java +++ b/testing/trino-tests/src/test/java/io/trino/security/TestAccessControl.java @@ -214,8 +214,7 @@ public Map apply(Connector Optional.of(Duration.ZERO), Optional.of("comment"), Optional.of("owner"), - ImmutableList.of(), - ImmutableMap.of()); + ImmutableList.of()); return ImmutableMap.of( new SchemaTableName("default", "test_materialized_view"), materializedViewDefinition); } diff --git a/testing/trino-tests/src/test/java/io/trino/tests/TestMockConnector.java b/testing/trino-tests/src/test/java/io/trino/tests/TestMockConnector.java index 62e927c58b656..bc7f700696013 100644 --- a/testing/trino-tests/src/test/java/io/trino/tests/TestMockConnector.java +++ b/testing/trino-tests/src/test/java/io/trino/tests/TestMockConnector.java @@ -106,8 +106,7 @@ protected QueryRunner createQueryRunner() Optional.of(Duration.ZERO), Optional.empty(), Optional.of("alice"), - ImmutableList.of(), - ImmutableMap.of()))) + ImmutableList.of()))) .withData(schemaTableName -> { if (schemaTableName.equals(new SchemaTableName("default", "nation"))) { return TPCH_NATION_DATA; From 58406b9155b91bc2c40b598507bb4f9fd928d244 Mon Sep 17 00:00:00 2001 From: Elon Azoulay Date: Sun, 10 Dec 2023 01:31:32 -0800 Subject: [PATCH 117/350] Move dependency to root pom Multiple modules depend on org.threeten.bp.Duration. Move to the root pom. --- plugin/trino-bigquery/pom.xml | 6 ------ pom.xml | 6 ++++++ 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/plugin/trino-bigquery/pom.xml b/plugin/trino-bigquery/pom.xml index 2e339e1349cea..446a8930c7a42 100644 --- a/plugin/trino-bigquery/pom.xml +++ b/plugin/trino-bigquery/pom.xml @@ -41,12 +41,6 @@ conscrypt-openjdk-uber 2.5.2 - - - org.threeten - threetenbp - 1.6.8 - diff --git a/pom.xml b/pom.xml index 9bb5dba1be965..db31fce963101 100644 --- a/pom.xml +++ b/pom.xml @@ -2051,6 +2051,12 @@ 1.13.1 + + org.threeten + threetenbp + 1.6.8 + + org.xerial.snappy snappy-java From f3a3eb6ad6d990b549d5b0683a765e5cab5e8034 Mon Sep 17 00:00:00 2001 From: Elon Azoulay Date: Sun, 10 Dec 2023 01:31:34 -0800 Subject: [PATCH 118/350] Fix GcsFileSystemConfig validation message Cleanup --- .../java/io/trino/filesystem/gcs/GcsFileSystemConfig.java | 8 ++++---- .../io/trino/filesystem/gcs/TestGcsFileSystemConfig.java | 6 +++--- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/lib/trino-filesystem-gcs/src/main/java/io/trino/filesystem/gcs/GcsFileSystemConfig.java b/lib/trino-filesystem-gcs/src/main/java/io/trino/filesystem/gcs/GcsFileSystemConfig.java index f71b4997f6956..766837c435b4c 100644 --- a/lib/trino-filesystem-gcs/src/main/java/io/trino/filesystem/gcs/GcsFileSystemConfig.java +++ b/lib/trino-filesystem-gcs/src/main/java/io/trino/filesystem/gcs/GcsFileSystemConfig.java @@ -150,12 +150,12 @@ public GcsFileSystemConfig setJsonKeyFilePath(String jsonKeyFilePath) public void validate() { - // This cannot be normal validation, as it would make it impossible to write TestHiveGcsConfig.testExplicitPropertyMappings + // This cannot be normal validation, as it would make it impossible to write TestGcsFileSystemConfig.testExplicitPropertyMappings if (useGcsAccessToken) { - checkState(jsonKey == null, "Cannot specify 'hive.gcs.json-key' when 'hive.gcs.use-access-token' is set"); - checkState(jsonKeyFilePath == null, "Cannot specify 'hive.gcs.json-key-file-path' when 'hive.gcs.use-access-token' is set"); + checkState(jsonKey == null, "Cannot specify 'gcs.json-key' when 'gcs.use-access-token' is set"); + checkState(jsonKeyFilePath == null, "Cannot specify 'gcs.json-key-file-path' when 'gcs.use-access-token' is set"); } - checkState(jsonKey == null || jsonKeyFilePath == null, "'hive.gcs.json-key' and 'hive.gcs.json-key-file-path' cannot be both set"); + checkState(jsonKey == null || jsonKeyFilePath == null, "'gcs.json-key' and 'gcs.json-key-file-path' cannot be both set"); } } diff --git a/lib/trino-filesystem-gcs/src/test/java/io/trino/filesystem/gcs/TestGcsFileSystemConfig.java b/lib/trino-filesystem-gcs/src/test/java/io/trino/filesystem/gcs/TestGcsFileSystemConfig.java index 2c6dd4a9992ba..4a6c9cc61508b 100644 --- a/lib/trino-filesystem-gcs/src/test/java/io/trino/filesystem/gcs/TestGcsFileSystemConfig.java +++ b/lib/trino-filesystem-gcs/src/test/java/io/trino/filesystem/gcs/TestGcsFileSystemConfig.java @@ -80,20 +80,20 @@ public void testValidation() .setUseGcsAccessToken(true) .setJsonKey("{}}")::validate) .isInstanceOf(IllegalStateException.class) - .hasMessage("Cannot specify 'hive.gcs.json-key' when 'hive.gcs.use-access-token' is set"); + .hasMessage("Cannot specify 'gcs.json-key' when 'gcs.use-access-token' is set"); assertThatThrownBy( new GcsFileSystemConfig() .setUseGcsAccessToken(true) .setJsonKeyFilePath("/dev/null")::validate) .isInstanceOf(IllegalStateException.class) - .hasMessage("Cannot specify 'hive.gcs.json-key-file-path' when 'hive.gcs.use-access-token' is set"); + .hasMessage("Cannot specify 'gcs.json-key-file-path' when 'gcs.use-access-token' is set"); assertThatThrownBy( new GcsFileSystemConfig() .setJsonKey("{}") .setJsonKeyFilePath("/dev/null")::validate) .isInstanceOf(IllegalStateException.class) - .hasMessage("'hive.gcs.json-key' and 'hive.gcs.json-key-file-path' cannot be both set"); + .hasMessage("'gcs.json-key' and 'gcs.json-key-file-path' cannot be both set"); } } From 3be8c2f5c4d6c3c6a2246f0081935c40bb7fb8bc Mon Sep 17 00:00:00 2001 From: Elon Azoulay Date: Sun, 10 Dec 2023 01:31:35 -0800 Subject: [PATCH 119/350] Configure retries for trino-filesystem-gcs --- lib/trino-filesystem-gcs/pom.xml | 11 +++ .../filesystem/gcs/GcsFileSystemConfig.java | 89 +++++++++++++++++++ .../filesystem/gcs/GcsStorageFactory.java | 29 +++++- .../gcs/TestGcsFileSystemConfig.java | 42 +++++++-- .../filesystem/gcs/TestGcsFileSystemGcs.java | 18 ++++ 5 files changed, 182 insertions(+), 7 deletions(-) diff --git a/lib/trino-filesystem-gcs/pom.xml b/lib/trino-filesystem-gcs/pom.xml index 33acd16414f3b..a958870f39267 100644 --- a/lib/trino-filesystem-gcs/pom.xml +++ b/lib/trino-filesystem-gcs/pom.xml @@ -123,6 +123,11 @@ jakarta.validation-api + + org.threeten + threetenbp + + io.trino trino-spi @@ -141,6 +146,12 @@ test + + io.airlift + testing + test + + io.trino trino-filesystem diff --git a/lib/trino-filesystem-gcs/src/main/java/io/trino/filesystem/gcs/GcsFileSystemConfig.java b/lib/trino-filesystem-gcs/src/main/java/io/trino/filesystem/gcs/GcsFileSystemConfig.java index 766837c435b4c..b5c8652ccdd3e 100644 --- a/lib/trino-filesystem-gcs/src/main/java/io/trino/filesystem/gcs/GcsFileSystemConfig.java +++ b/lib/trino-filesystem-gcs/src/main/java/io/trino/filesystem/gcs/GcsFileSystemConfig.java @@ -18,10 +18,15 @@ import io.airlift.configuration.ConfigSecuritySensitive; import io.airlift.configuration.validation.FileExists; import io.airlift.units.DataSize; +import io.airlift.units.Duration; +import io.airlift.units.MinDuration; import jakarta.annotation.Nullable; +import jakarta.validation.constraints.AssertTrue; import jakarta.validation.constraints.Min; import jakarta.validation.constraints.NotNull; +import java.util.concurrent.TimeUnit; + import static com.google.common.base.Preconditions.checkState; import static io.airlift.units.DataSize.Unit.MEGABYTE; @@ -37,6 +42,12 @@ public class GcsFileSystemConfig private boolean useGcsAccessToken; private String jsonKey; private String jsonKeyFilePath; + private int maxRetries = 20; + private double backoffScaleFactor = 2.0; + private Duration maxRetryTime = new Duration(20, TimeUnit.SECONDS); + private Duration minBackoffDelay = new Duration(10, TimeUnit.MILLISECONDS); + // Note: there is no benefit to setting this much higher as the rpc quota is 1x per second: https://cloud.google.com/storage/docs/retry-strategy#java + private Duration maxBackoffDelay = new Duration(1100, TimeUnit.MILLISECONDS); @NotNull public DataSize getReadBlockSize() @@ -148,6 +159,84 @@ public GcsFileSystemConfig setJsonKeyFilePath(String jsonKeyFilePath) return this; } + @Min(0) + public int getMaxRetries() + { + return maxRetries; + } + + @Config("gcs.client.max-retries") + @ConfigDescription("Maximum number of RPC attempts") + public GcsFileSystemConfig setMaxRetries(int maxRetries) + { + this.maxRetries = maxRetries; + return this; + } + + @Min(1) + public double getBackoffScaleFactor() + { + return backoffScaleFactor; + } + + @Config("gcs.client.backoff-scale-factor") + @ConfigDescription("Scale factor for RPC retry delay") + public GcsFileSystemConfig setBackoffScaleFactor(double backoffScaleFactor) + { + this.backoffScaleFactor = backoffScaleFactor; + return this; + } + + @NotNull + public Duration getMaxRetryTime() + { + return maxRetryTime; + } + + @Config("gcs.client.max-retry-time") + @ConfigDescription("Total time limit for an RPC to be retried") + public GcsFileSystemConfig setMaxRetryTime(Duration maxRetryTime) + { + this.maxRetryTime = maxRetryTime; + return this; + } + + @NotNull + @MinDuration("0ms") + public Duration getMinBackoffDelay() + { + return minBackoffDelay; + } + + @Config("gcs.client.min-backoff-delay") + @ConfigDescription("Minimum delay between RPC retries") + public GcsFileSystemConfig setMinBackoffDelay(Duration minBackoffDelay) + { + this.minBackoffDelay = minBackoffDelay; + return this; + } + + @NotNull + @MinDuration("0ms") + public Duration getMaxBackoffDelay() + { + return maxBackoffDelay; + } + + @Config("gcs.client.max-backoff-delay") + @ConfigDescription("Maximum delay between RPC retries.") + public GcsFileSystemConfig setMaxBackoffDelay(Duration maxBackoffDelay) + { + this.maxBackoffDelay = maxBackoffDelay; + return this; + } + + @AssertTrue(message = "gcs.client.min-backoff-delay must be less than or equal to gcs.client.max-backoff-delay") + public boolean isRetryDelayValid() + { + return minBackoffDelay.compareTo(maxBackoffDelay) <= 0; + } + public void validate() { // This cannot be normal validation, as it would make it impossible to write TestGcsFileSystemConfig.testExplicitPropertyMappings diff --git a/lib/trino-filesystem-gcs/src/main/java/io/trino/filesystem/gcs/GcsStorageFactory.java b/lib/trino-filesystem-gcs/src/main/java/io/trino/filesystem/gcs/GcsStorageFactory.java index 176d45061fdac..bdb85f82d7e27 100644 --- a/lib/trino-filesystem-gcs/src/main/java/io/trino/filesystem/gcs/GcsStorageFactory.java +++ b/lib/trino-filesystem-gcs/src/main/java/io/trino/filesystem/gcs/GcsStorageFactory.java @@ -13,6 +13,7 @@ */ package io.trino.filesystem.gcs; +import com.google.api.gax.retrying.RetrySettings; import com.google.auth.oauth2.GoogleCredentials; import com.google.cloud.storage.Storage; import com.google.cloud.storage.StorageOptions; @@ -20,6 +21,7 @@ import com.google.common.collect.ImmutableList; import com.google.inject.Inject; import io.trino.spi.security.ConnectorIdentity; +import org.threeten.bp.Duration; import java.io.ByteArrayInputStream; import java.io.FileInputStream; @@ -29,6 +31,7 @@ import java.util.List; import java.util.Optional; +import static com.google.cloud.storage.StorageRetryStrategy.getUniformStorageRetryStrategy; import static com.google.common.base.Strings.nullToEmpty; import static java.nio.charset.StandardCharsets.UTF_8; @@ -39,6 +42,11 @@ public class GcsStorageFactory private final String projectId; private final boolean useGcsAccessToken; private final Optional jsonGoogleCredential; + private final int maxRetries; + private final double backoffScaleFactor; + private final Duration maxRetryTime; + private final Duration minBackoffDelay; + private final Duration maxBackoffDelay; @Inject public GcsStorageFactory(GcsFileSystemConfig config) @@ -62,6 +70,12 @@ else if (jsonKeyFilePath != null) { else { jsonGoogleCredential = Optional.empty(); } + this.maxRetries = config.getMaxRetries(); + this.backoffScaleFactor = config.getBackoffScaleFactor(); + // To avoid name collision by importing io.airlift.Duration + this.maxRetryTime = Duration.ofMillis(config.getMaxRetryTime().toMillis()); + this.minBackoffDelay = Duration.ofMillis(config.getMinBackoffDelay().toMillis()); + this.maxBackoffDelay = Duration.ofMillis(config.getMaxBackoffDelay().toMillis()); } public Storage create(ConnectorIdentity identity) @@ -81,7 +95,20 @@ public Storage create(ConnectorIdentity identity) if (projectId != null) { storageOptionsBuilder.setProjectId(projectId); } - return storageOptionsBuilder.setCredentials(credentials).build().getService(); + // Note: without uniform strategy we cannot retry idempotent operations. + // The trino-filesystem api does not violate the conditions for idempotency, see https://cloud.google.com/storage/docs/retry-strategy#java for details. + return storageOptionsBuilder + .setCredentials(credentials) + .setStorageRetryStrategy(getUniformStorageRetryStrategy()) + .setRetrySettings(RetrySettings.newBuilder() + .setMaxAttempts(maxRetries + 1) + .setRetryDelayMultiplier(backoffScaleFactor) + .setTotalTimeout(maxRetryTime) + .setInitialRetryDelay(minBackoffDelay) + .setMaxRetryDelay(maxBackoffDelay) + .build()) + .build() + .getService(); } catch (IOException e) { throw new UncheckedIOException(e); diff --git a/lib/trino-filesystem-gcs/src/test/java/io/trino/filesystem/gcs/TestGcsFileSystemConfig.java b/lib/trino-filesystem-gcs/src/test/java/io/trino/filesystem/gcs/TestGcsFileSystemConfig.java index 4a6c9cc61508b..d5b640f8f738a 100644 --- a/lib/trino-filesystem-gcs/src/test/java/io/trino/filesystem/gcs/TestGcsFileSystemConfig.java +++ b/lib/trino-filesystem-gcs/src/test/java/io/trino/filesystem/gcs/TestGcsFileSystemConfig.java @@ -15,6 +15,8 @@ import com.google.common.collect.ImmutableMap; import io.airlift.units.DataSize; +import io.airlift.units.Duration; +import jakarta.validation.constraints.AssertTrue; import org.junit.jupiter.api.Test; import java.io.IOException; @@ -25,6 +27,10 @@ import static io.airlift.configuration.testing.ConfigAssertions.assertFullMapping; import static io.airlift.configuration.testing.ConfigAssertions.assertRecordedDefaults; import static io.airlift.configuration.testing.ConfigAssertions.recordDefaults; +import static io.airlift.testing.ValidationAssertions.assertFailsValidation; +import static io.airlift.units.DataSize.Unit.MEGABYTE; +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static java.util.concurrent.TimeUnit.SECONDS; import static org.assertj.core.api.Assertions.assertThatThrownBy; public class TestGcsFileSystemConfig @@ -33,14 +39,19 @@ public class TestGcsFileSystemConfig void testDefaults() { assertRecordedDefaults(recordDefaults(GcsFileSystemConfig.class) - .setReadBlockSize(DataSize.of(2, DataSize.Unit.MEGABYTE)) - .setWriteBlockSize(DataSize.of(16, DataSize.Unit.MEGABYTE)) + .setReadBlockSize(DataSize.of(2, MEGABYTE)) + .setWriteBlockSize(DataSize.of(16, MEGABYTE)) .setPageSize(100) .setBatchSize(100) .setProjectId(null) .setUseGcsAccessToken(false) .setJsonKey(null) - .setJsonKeyFilePath(null)); + .setJsonKeyFilePath(null) + .setMaxRetries(20) + .setBackoffScaleFactor(2.0) + .setMaxRetryTime(new Duration(20, SECONDS)) + .setMinBackoffDelay(new Duration(10, MILLISECONDS)) + .setMaxBackoffDelay(new Duration(1100, MILLISECONDS))); } @Test @@ -58,17 +69,27 @@ void testExplicitPropertyMappings() .put("gcs.use-access-token", "true") .put("gcs.json-key", "{}") .put("gcs.json-key-file-path", jsonKeyFile.toString()) + .put("gcs.client.max-retries", "10") + .put("gcs.client.backoff-scale-factor", "3.0") + .put("gcs.client.max-retry-time", "10s") + .put("gcs.client.min-backoff-delay", "20ms") + .put("gcs.client.max-backoff-delay", "20ms") .buildOrThrow(); GcsFileSystemConfig expected = new GcsFileSystemConfig() - .setReadBlockSize(DataSize.of(51, DataSize.Unit.MEGABYTE)) - .setWriteBlockSize(DataSize.of(52, DataSize.Unit.MEGABYTE)) + .setReadBlockSize(DataSize.of(51, MEGABYTE)) + .setWriteBlockSize(DataSize.of(52, MEGABYTE)) .setPageSize(10) .setBatchSize(11) .setProjectId("project") .setUseGcsAccessToken(true) .setJsonKey("{}") - .setJsonKeyFilePath(jsonKeyFile.toString()); + .setJsonKeyFilePath(jsonKeyFile.toString()) + .setMaxRetries(10) + .setBackoffScaleFactor(3.0) + .setMaxRetryTime(new Duration(10, SECONDS)) + .setMinBackoffDelay(new Duration(20, MILLISECONDS)) + .setMaxBackoffDelay(new Duration(20, MILLISECONDS)); assertFullMapping(properties, expected); } @@ -95,5 +116,14 @@ public void testValidation() .setJsonKeyFilePath("/dev/null")::validate) .isInstanceOf(IllegalStateException.class) .hasMessage("'gcs.json-key' and 'gcs.json-key-file-path' cannot be both set"); + + assertFailsValidation( + new GcsFileSystemConfig() + .setJsonKey("{}") + .setMinBackoffDelay(new Duration(20, MILLISECONDS)) + .setMaxBackoffDelay(new Duration(19, MILLISECONDS)), + "retryDelayValid", + "gcs.client.min-backoff-delay must be less than or equal to gcs.client.max-backoff-delay", + AssertTrue.class); } } diff --git a/lib/trino-filesystem-gcs/src/test/java/io/trino/filesystem/gcs/TestGcsFileSystemGcs.java b/lib/trino-filesystem-gcs/src/test/java/io/trino/filesystem/gcs/TestGcsFileSystemGcs.java index b9a056e487fc0..852b30da274b9 100644 --- a/lib/trino-filesystem-gcs/src/test/java/io/trino/filesystem/gcs/TestGcsFileSystemGcs.java +++ b/lib/trino-filesystem-gcs/src/test/java/io/trino/filesystem/gcs/TestGcsFileSystemGcs.java @@ -13,10 +13,16 @@ */ package io.trino.filesystem.gcs; +import io.trino.filesystem.TrinoOutputFile; import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; import org.junit.jupiter.api.TestInstance; import java.io.IOException; +import java.io.OutputStream; + +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.assertj.core.api.Assertions.assertThatNoException; @TestInstance(TestInstance.Lifecycle.PER_CLASS) public class TestGcsFileSystemGcs @@ -28,4 +34,16 @@ void setup() { initialize(getRequiredEnvironmentVariable("GCP_CREDENTIALS_KEY")); } + + @Test + void testCreateFileRetry() + { + assertThatNoException().isThrownBy(() -> { + for (int i = 1; i <= 100; i++) { + TrinoOutputFile outputFile = getFileSystem().newOutputFile(getRootLocation().appendPath("testFile")); + try (OutputStream out = outputFile.createOrOverwrite()) { + out.write("test".getBytes(UTF_8)); + } + }}); + } } From 049388a8f1b8a7c1cee2bea8a8789452520cd077 Mon Sep 17 00:00:00 2001 From: Bhargavi Sagi Date: Fri, 8 Dec 2023 11:18:14 -0800 Subject: [PATCH 120/350] Replace TableToPartitionMapping with hiveColumnCoercion map --- .../hive/BackgroundHiveSplitLoader.java | 4 +- .../plugin/hive/HivePageSourceProvider.java | 9 +- .../plugin/hive/HivePartitionMetadata.java | 12 +- .../java/io/trino/plugin/hive/HiveSplit.java | 19 +-- .../trino/plugin/hive/HiveSplitManager.java | 21 ++- .../io/trino/plugin/hive/HiveSplitSource.java | 2 +- .../trino/plugin/hive/InternalHiveSplit.java | 16 ++- .../plugin/hive/TableToPartitionMapping.java | 127 ------------------ .../hive/util/InternalHiveSplitFactory.java | 11 +- .../hive/TestBackgroundHiveSplitLoader.java | 12 +- .../plugin/hive/TestHiveFileFormats.java | 2 +- .../trino/plugin/hive/TestHivePageSink.java | 2 +- .../io/trino/plugin/hive/TestHiveSplit.java | 5 +- .../plugin/hive/TestHiveSplitSource.java | 2 +- .../TestNodeLocalDynamicSplitPruning.java | 2 +- .../hive/TestOrcPageSourceMemoryTracking.java | 2 +- .../hive/TestTableToPartitionMapping.java | 47 ------- .../plugin/hive/orc/TestOrcPredicates.java | 3 +- 18 files changed, 63 insertions(+), 235 deletions(-) delete mode 100644 plugin/trino-hive/src/main/java/io/trino/plugin/hive/TableToPartitionMapping.java delete mode 100644 plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestTableToPartitionMapping.java diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/BackgroundHiveSplitLoader.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/BackgroundHiveSplitLoader.java index a79c7a6b2adce..da6d10b9abc06 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/BackgroundHiveSplitLoader.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/BackgroundHiveSplitLoader.java @@ -419,7 +419,7 @@ private ListenableFuture loadPartition(HivePartitionMetadata partition) partitionKeys, effectivePredicate, partitionMatchSupplier, - partition.getTableToPartitionMapping(), + partition.getHiveColumnCoercions(), Optional.empty(), Optional.empty(), getMaxInitialSplitSize(session), @@ -470,7 +470,7 @@ private ListenableFuture loadPartition(HivePartitionMetadata partition) partitionKeys, effectivePredicate, partitionMatchSupplier, - partition.getTableToPartitionMapping(), + partition.getHiveColumnCoercions(), bucketConversionRequiresWorkerParticipation ? bucketConversion : Optional.empty(), bucketValidation, getMaxInitialSplitSize(session), diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePageSourceProvider.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePageSourceProvider.java index ec34a5814c5a6..4d03e4ea5596a 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePageSourceProvider.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePageSourceProvider.java @@ -121,7 +121,7 @@ public ConnectorPageSource createPageSource( hiveSplit.getPartitionKeys(), hiveColumns, hiveSplit.getBucketConversion().map(BucketConversion::bucketColumnHandles).orElse(ImmutableList.of()), - hiveSplit.getTableToPartitionMapping(), + hiveSplit.getHiveColumnCoercions(), hiveSplit.getPath(), hiveSplit.getTableBucketNumber(), hiveSplit.getEstimatedFileSize(), @@ -382,7 +382,7 @@ public static List buildColumnMappings( List partitionKeys, List columns, List requiredInterimColumns, - TableToPartitionMapping tableToPartitionMapping, + Map hiveColumnCoercions, String path, OptionalInt bucketNumber, long estimatedFileSize, @@ -398,7 +398,7 @@ public static List buildColumnMappings( int regularIndex = 0; for (HiveColumnHandle column : columns) { - Optional baseTypeCoercionFrom = tableToPartitionMapping.getCoercion(column.getBaseHiveColumnIndex()); + Optional baseTypeCoercionFrom = Optional.ofNullable(hiveColumnCoercions.get(column.getBaseHiveColumnIndex())).map(HiveTypeName::toHiveType); if (column.getColumnType() == REGULAR) { if (column.isBaseColumn()) { baseColumnHiveIndices.add(column.getBaseHiveColumnIndex()); @@ -449,7 +449,8 @@ else if (isRowIdColumnHandle(column)) { } if (projectionsForColumn.containsKey(column.getBaseHiveColumnIndex())) { - columnMappings.add(interim(column, regularIndex, tableToPartitionMapping.getCoercion(column.getBaseHiveColumnIndex()))); + Optional baseTypeCoercionFrom = Optional.ofNullable(hiveColumnCoercions.get(column.getBaseHiveColumnIndex())).map(HiveTypeName::toHiveType); + columnMappings.add(interim(column, regularIndex, baseTypeCoercionFrom)); } else { // If coercion does not affect bucket number calculation, coercion doesn't need to be applied here. diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePartitionMetadata.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePartitionMetadata.java index f484f42868c91..15efa2929ad96 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePartitionMetadata.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePartitionMetadata.java @@ -13,8 +13,10 @@ */ package io.trino.plugin.hive; +import com.google.common.collect.ImmutableMap; import io.trino.plugin.hive.metastore.Partition; +import java.util.Map; import java.util.Optional; import static java.util.Objects.requireNonNull; @@ -23,16 +25,16 @@ public class HivePartitionMetadata { private final Optional partition; private final HivePartition hivePartition; - private final TableToPartitionMapping tableToPartitionMapping; + private final Map hiveColumnCoercions; HivePartitionMetadata( HivePartition hivePartition, Optional partition, - TableToPartitionMapping tableToPartitionMapping) + Map hiveColumnCoercions) { this.partition = requireNonNull(partition, "partition is null"); this.hivePartition = requireNonNull(hivePartition, "hivePartition is null"); - this.tableToPartitionMapping = requireNonNull(tableToPartitionMapping, "tableToPartitionMapping is null"); + this.hiveColumnCoercions = ImmutableMap.copyOf(requireNonNull(hiveColumnCoercions, "hiveColumnCoercions is null")); } public HivePartition getHivePartition() @@ -48,8 +50,8 @@ public Optional getPartition() return partition; } - public TableToPartitionMapping getTableToPartitionMapping() + public Map getHiveColumnCoercions() { - return tableToPartitionMapping; + return hiveColumnCoercions; } } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplit.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplit.java index f93e8a3d6a2a0..fc4c585ce957c 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplit.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplit.java @@ -40,6 +40,7 @@ public class HiveSplit implements ConnectorSplit { private static final int INSTANCE_SIZE = instanceSize(HiveSplit.class); + private static final int INTEGER_INSTANCE_SIZE = instanceSize(Integer.class); private final String path; private final long start; @@ -53,7 +54,7 @@ public class HiveSplit private final OptionalInt readBucketNumber; private final OptionalInt tableBucketNumber; private final boolean forceLocalScheduling; - private final TableToPartitionMapping tableToPartitionMapping; + private final Map hiveColumnCoercions; private final Optional bucketConversion; private final Optional bucketValidation; private final Optional acidInfo; @@ -72,7 +73,7 @@ public HiveSplit( @JsonProperty("readBucketNumber") OptionalInt readBucketNumber, @JsonProperty("tableBucketNumber") OptionalInt tableBucketNumber, @JsonProperty("forceLocalScheduling") boolean forceLocalScheduling, - @JsonProperty("tableToPartitionMapping") TableToPartitionMapping tableToPartitionMapping, + @JsonProperty("hiveColumnCoercions") Map hiveColumnCoercions, @JsonProperty("bucketConversion") Optional bucketConversion, @JsonProperty("bucketValidation") Optional bucketValidation, @JsonProperty("acidInfo") Optional acidInfo, @@ -91,7 +92,7 @@ public HiveSplit( readBucketNumber, tableBucketNumber, forceLocalScheduling, - tableToPartitionMapping, + hiveColumnCoercions, bucketConversion, bucketValidation, acidInfo, @@ -111,7 +112,7 @@ public HiveSplit( OptionalInt readBucketNumber, OptionalInt tableBucketNumber, boolean forceLocalScheduling, - TableToPartitionMapping tableToPartitionMapping, + Map hiveColumnCoercions, Optional bucketConversion, Optional bucketValidation, Optional acidInfo, @@ -127,7 +128,7 @@ public HiveSplit( requireNonNull(addresses, "addresses is null"); requireNonNull(readBucketNumber, "readBucketNumber is null"); requireNonNull(tableBucketNumber, "tableBucketNumber is null"); - requireNonNull(tableToPartitionMapping, "tableToPartitionMapping is null"); + requireNonNull(hiveColumnCoercions, "hiveColumnCoercions is null"); requireNonNull(bucketConversion, "bucketConversion is null"); requireNonNull(bucketValidation, "bucketValidation is null"); requireNonNull(acidInfo, "acidInfo is null"); @@ -144,7 +145,7 @@ public HiveSplit( this.readBucketNumber = readBucketNumber; this.tableBucketNumber = tableBucketNumber; this.forceLocalScheduling = forceLocalScheduling; - this.tableToPartitionMapping = tableToPartitionMapping; + this.hiveColumnCoercions = ImmutableMap.copyOf(hiveColumnCoercions); this.bucketConversion = bucketConversion; this.bucketValidation = bucketValidation; this.acidInfo = acidInfo; @@ -226,9 +227,9 @@ public boolean isForceLocalScheduling() } @JsonProperty - public TableToPartitionMapping getTableToPartitionMapping() + public Map getHiveColumnCoercions() { - return tableToPartitionMapping; + return hiveColumnCoercions; } @JsonProperty @@ -273,7 +274,7 @@ public long getRetainedSizeInBytes() + estimatedSizeOf(partitionName) + sizeOf(readBucketNumber) + sizeOf(tableBucketNumber) - + tableToPartitionMapping.getEstimatedSizeInBytes() + + estimatedSizeOf(hiveColumnCoercions, (Integer key) -> INTEGER_INSTANCE_SIZE, HiveTypeName::getEstimatedSizeInBytes) + sizeOf(bucketConversion, BucketConversion::getRetainedSizeInBytes) + sizeOf(bucketValidation, BucketValidation::getRetainedSizeInBytes) + sizeOf(acidInfo, AcidInfo::getRetainedSizeInBytes) diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplitManager.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplitManager.java index e007baeaf52ee..630caa38534ba 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplitManager.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplitManager.java @@ -80,7 +80,6 @@ import static io.trino.plugin.hive.HiveSessionProperties.isUseOrcColumnNames; import static io.trino.plugin.hive.HiveSessionProperties.isUseParquetColumnNames; import static io.trino.plugin.hive.HiveStorageFormat.getHiveStorageFormat; -import static io.trino.plugin.hive.TableToPartitionMapping.mapColumnsByIndex; import static io.trino.plugin.hive.metastore.MetastoreUtil.getProtectMode; import static io.trino.plugin.hive.metastore.MetastoreUtil.makePartitionName; import static io.trino.plugin.hive.metastore.MetastoreUtil.verifyOnline; @@ -305,7 +304,7 @@ private Iterator getPartitionMetadata( if (firstPartition.getPartitionId().equals(UNPARTITIONED_ID)) { hivePartitions.next(); checkArgument(!hivePartitions.hasNext(), "single partition is expected for unpartitioned table"); - return singletonIterator(new HivePartitionMetadata(firstPartition, Optional.empty(), TableToPartitionMapping.empty())); + return singletonIterator(new HivePartitionMetadata(firstPartition, Optional.empty(), ImmutableMap.of())); } HiveTimestampPrecision hiveTimestampPrecision = getTimestampPrecision(session); @@ -385,7 +384,7 @@ private static HivePartitionMetadata toPartitionMetadata( throw new TrinoException(HIVE_INVALID_METADATA, format("Table '%s' or partition '%s' has null columns", tableName, partName)); } - TableToPartitionMapping tableToPartitionMapping = getTableToPartitionMapping(usePartitionColumnNames, typeManager, hiveTimestampPrecision, tableName, partName, tableColumns, partitionColumns, neededColumnNames); + Map hiveColumnCoercions = getHiveColumnCoercions(usePartitionColumnNames, typeManager, hiveTimestampPrecision, tableName, partName, tableColumns, partitionColumns, neededColumnNames); if (bucketProperty.isPresent()) { HiveBucketProperty partitionBucketProperty = partition.getStorage().getBucketProperty() @@ -420,10 +419,10 @@ private static HivePartitionMetadata toPartitionMetadata( } } } - return new HivePartitionMetadata(hivePartition, Optional.of(partition), tableToPartitionMapping); + return new HivePartitionMetadata(hivePartition, Optional.of(partition), hiveColumnCoercions); } - private static TableToPartitionMapping getTableToPartitionMapping( + private static Map getHiveColumnCoercions( boolean usePartitionColumnNames, TypeManager typeManager, HiveTimestampPrecision hiveTimestampPrecision, @@ -434,7 +433,7 @@ private static TableToPartitionMapping getTableToPartitionMapping( Set neededColumnNames) { if (usePartitionColumnNames) { - return getTableToPartitionMappingByColumnNames(typeManager, tableName, partName, tableColumns, partitionColumns, neededColumnNames, hiveTimestampPrecision); + return getHiveColumnCoercionsByColumnNames(typeManager, tableName, partName, tableColumns, partitionColumns, neededColumnNames, hiveTimestampPrecision); } ImmutableMap.Builder columnCoercions = ImmutableMap.builder(); for (int i = 0; i < min(partitionColumns.size(), tableColumns.size()); i++) { @@ -451,7 +450,7 @@ private static TableToPartitionMapping getTableToPartitionMapping( columnCoercions.put(i, partitionType.getHiveTypeName()); } } - return mapColumnsByIndex(columnCoercions.buildOrThrow()); + return columnCoercions.buildOrThrow(); } private static boolean isPartitionUsesColumnNames(ConnectorSession session, Optional storageFormat) @@ -467,7 +466,7 @@ private static boolean isPartitionUsesColumnNames(ConnectorSession session, Opti }; } - private static TableToPartitionMapping getTableToPartitionMappingByColumnNames( + private static Map getHiveColumnCoercionsByColumnNames( TypeManager typeManager, SchemaTableName tableName, String partName, @@ -488,7 +487,6 @@ private static TableToPartitionMapping getTableToPartitionMappingByColumnNames( Map partitionColumnsByIndex = partitionColumnIndexesBuilder.buildOrThrow(); ImmutableMap.Builder columnCoercions = ImmutableMap.builder(); - ImmutableMap.Builder tableToPartitionColumns = ImmutableMap.builder(); for (int tableColumnIndex = 0; tableColumnIndex < tableColumns.size(); tableColumnIndex++) { Column tableColumn = tableColumns.get(tableColumnIndex); HiveType tableType = tableColumn.getType(); @@ -496,18 +494,17 @@ private static TableToPartitionMapping getTableToPartitionMappingByColumnNames( if (partitionColumnIndex == null) { continue; } - tableToPartitionColumns.put(tableColumnIndex, partitionColumnIndex); Column partitionColumn = partitionColumns.get(partitionColumnIndex); HiveType partitionType = partitionColumn.getType(); if (!tableType.equals(partitionType)) { if (!canCoerce(typeManager, partitionType, tableType, hiveTimestampPrecision)) { throw tablePartitionColumnMismatchException(tableName, partName, tableColumn.getName(), tableType, partitionColumn.getName(), partitionType); } - columnCoercions.put(partitionColumnIndex, partitionType.getHiveTypeName()); + columnCoercions.put(tableColumnIndex, partitionType.getHiveTypeName()); } } - return new TableToPartitionMapping(Optional.of(tableToPartitionColumns.buildOrThrow()), columnCoercions.buildOrThrow()); + return columnCoercions.buildOrThrow(); } private static TrinoException tablePartitionColumnMismatchException(SchemaTableName tableName, String partName, String tableColumnName, HiveType tableType, String partitionColumnName, HiveType partitionType) diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplitSource.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplitSource.java index 63f36aec02698..2da1750229f50 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplitSource.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplitSource.java @@ -309,7 +309,7 @@ else if (maxSplitBytes * 2 >= remainingBlockBytes) { internalSplit.getReadBucketNumber(), internalSplit.getTableBucketNumber(), internalSplit.isForceLocalScheduling(), - internalSplit.getTableToPartitionMapping(), + internalSplit.getHiveColumnCoercions(), internalSplit.getBucketConversion(), internalSplit.getBucketValidation(), internalSplit.getAcidInfo(), diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/InternalHiveSplit.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/InternalHiveSplit.java index 28b11595a7bcf..6984b7e9d34ac 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/InternalHiveSplit.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/InternalHiveSplit.java @@ -14,6 +14,7 @@ package io.trino.plugin.hive; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import io.trino.annotation.NotThreadSafe; import io.trino.plugin.hive.HiveSplit.BucketConversion; import io.trino.plugin.hive.HiveSplit.BucketValidation; @@ -38,6 +39,7 @@ public class InternalHiveSplit { private static final int INSTANCE_SIZE = instanceSize(InternalHiveSplit.class) + instanceSize(OptionalInt.class); + private static final int INTEGER_INSTANCE_SIZE = instanceSize(Integer.class); private final String path; private final long end; @@ -51,7 +53,7 @@ public class InternalHiveSplit private final OptionalInt tableBucketNumber; private final boolean splittable; private final boolean forceLocalScheduling; - private final TableToPartitionMapping tableToPartitionMapping; + private final Map hiveColumnCoercions; private final Optional bucketConversion; private final Optional bucketValidation; private final Optional acidInfo; @@ -74,7 +76,7 @@ public InternalHiveSplit( OptionalInt tableBucketNumber, boolean splittable, boolean forceLocalScheduling, - TableToPartitionMapping tableToPartitionMapping, + Map hiveColumnCoercions, Optional bucketConversion, Optional bucketValidation, Optional acidInfo, @@ -90,7 +92,7 @@ public InternalHiveSplit( requireNonNull(blocks, "blocks is null"); requireNonNull(readBucketNumber, "readBucketNumber is null"); requireNonNull(tableBucketNumber, "tableBucketNumber is null"); - requireNonNull(tableToPartitionMapping, "tableToPartitionMapping is null"); + requireNonNull(hiveColumnCoercions, "hiveColumnCoercions is null"); requireNonNull(bucketConversion, "bucketConversion is null"); requireNonNull(bucketValidation, "bucketValidation is null"); requireNonNull(acidInfo, "acidInfo is null"); @@ -109,7 +111,7 @@ public InternalHiveSplit( this.tableBucketNumber = tableBucketNumber; this.splittable = splittable; this.forceLocalScheduling = forceLocalScheduling; - this.tableToPartitionMapping = tableToPartitionMapping; + this.hiveColumnCoercions = ImmutableMap.copyOf(hiveColumnCoercions); this.bucketConversion = bucketConversion; this.bucketValidation = bucketValidation; this.acidInfo = acidInfo; @@ -176,9 +178,9 @@ public boolean isForceLocalScheduling() return forceLocalScheduling; } - public TableToPartitionMapping getTableToPartitionMapping() + public Map getHiveColumnCoercions() { - return tableToPartitionMapping; + return hiveColumnCoercions; } public Optional getBucketConversion() @@ -221,7 +223,7 @@ public int getEstimatedSizeInBytes() estimatedSizeOf(partitionKeys, HivePartitionKey::getEstimatedSizeInBytes) + estimatedSizeOf(blocks, InternalHiveBlock::getEstimatedSizeInBytes) + estimatedSizeOf(partitionName) + - tableToPartitionMapping.getEstimatedSizeInBytes(); + estimatedSizeOf(hiveColumnCoercions, (Integer key) -> INTEGER_INSTANCE_SIZE, HiveTypeName::getEstimatedSizeInBytes); return toIntExact(result); } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/TableToPartitionMapping.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/TableToPartitionMapping.java deleted file mode 100644 index fcabc3ba27d73..0000000000000 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/TableToPartitionMapping.java +++ /dev/null @@ -1,127 +0,0 @@ -/* - * 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.hive; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.ImmutableMap; -import it.unimi.dsi.fastutil.ints.Int2IntArrayMap; -import it.unimi.dsi.fastutil.ints.Int2IntMaps; - -import java.util.Map; -import java.util.Objects; -import java.util.Optional; - -import static com.google.common.base.MoreObjects.toStringHelper; -import static io.airlift.slice.SizeOf.estimatedSizeOf; -import static io.airlift.slice.SizeOf.instanceSize; -import static io.airlift.slice.SizeOf.sizeOfIntArray; -import static java.lang.Math.toIntExact; -import static java.util.Objects.requireNonNull; - -public class TableToPartitionMapping -{ - public static TableToPartitionMapping empty() - { - return new TableToPartitionMapping(Optional.empty(), ImmutableMap.of()); - } - - public static TableToPartitionMapping mapColumnsByIndex(Map columnCoercions) - { - return new TableToPartitionMapping(Optional.empty(), columnCoercions); - } - - // Overhead of ImmutableMap is not accounted because of its complexity. - private static final int INSTANCE_SIZE = instanceSize(TableToPartitionMapping.class); - private static final int INTEGER_INSTANCE_SIZE = instanceSize(Integer.class); - private static final int OPTIONAL_INSTANCE_SIZE = instanceSize(Optional.class); - private static final int INT_2_INT_ARRAY_MAP_INSTANCE_SIZE = instanceSize(Int2IntArrayMap.class); - - private final Optional> tableToPartitionColumns; - private final Map partitionColumnCoercions; - - @JsonCreator - public TableToPartitionMapping( - @JsonProperty("tableToPartitionColumns") Optional> tableToPartitionColumns, - @JsonProperty("partitionColumnCoercions") Map partitionColumnCoercions) - { - if (tableToPartitionColumns.map(TableToPartitionMapping::isIdentityMapping).orElse(true)) { - this.tableToPartitionColumns = Optional.empty(); - } - else { - // we use Int2IntArrayMap due to much lower memory footprint than ImmutableMap - this.tableToPartitionColumns = tableToPartitionColumns.map(mapping -> Int2IntMaps.unmodifiable(new Int2IntArrayMap(mapping))); - } - this.partitionColumnCoercions = ImmutableMap.copyOf(requireNonNull(partitionColumnCoercions, "partitionColumnCoercions is null")); - } - - @VisibleForTesting - static boolean isIdentityMapping(Map map) - { - for (int i = 0; i < map.size(); i++) { - if (!Objects.equals(map.get(i), i)) { - return false; - } - } - return true; - } - - @JsonProperty - public Map getPartitionColumnCoercions() - { - return partitionColumnCoercions; - } - - @JsonProperty - public Optional> getTableToPartitionColumns() - { - return tableToPartitionColumns; - } - - public Optional getCoercion(int tableColumnIndex) - { - return getPartitionColumnIndex(tableColumnIndex) - .flatMap(partitionColumnIndex -> Optional.ofNullable(partitionColumnCoercions.get(partitionColumnIndex))) - .map(HiveTypeName::toHiveType); - } - - private Optional getPartitionColumnIndex(int tableColumnIndex) - { - if (tableToPartitionColumns.isEmpty()) { - return Optional.of(tableColumnIndex); - } - return Optional.ofNullable(tableToPartitionColumns.get().get(tableColumnIndex)); - } - - public int getEstimatedSizeInBytes() - { - long result = INSTANCE_SIZE + - estimatedSizeOf(partitionColumnCoercions, (Integer key) -> INTEGER_INSTANCE_SIZE, HiveTypeName::getEstimatedSizeInBytes) + - OPTIONAL_INSTANCE_SIZE + - tableToPartitionColumns - .map(tableToPartitionColumns -> INT_2_INT_ARRAY_MAP_INSTANCE_SIZE + 2 * sizeOfIntArray(tableToPartitionColumns.size())) - .orElse(0L); - return toIntExact(result); - } - - @Override - public String toString() - { - return toStringHelper(this) - .add("columnCoercions", partitionColumnCoercions) - .add("tableToPartitionColumns", tableToPartitionColumns) - .toString(); - } -} diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/InternalHiveSplitFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/InternalHiveSplitFactory.java index 2e6588d3d918d..545023c3809ef 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/InternalHiveSplitFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/InternalHiveSplitFactory.java @@ -14,6 +14,7 @@ package io.trino.plugin.hive.util; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import io.airlift.units.DataSize; import io.trino.plugin.hive.AcidInfo; import io.trino.plugin.hive.HiveColumnHandle; @@ -21,9 +22,9 @@ import io.trino.plugin.hive.HiveSplit; import io.trino.plugin.hive.HiveSplit.BucketConversion; import io.trino.plugin.hive.HiveStorageFormat; +import io.trino.plugin.hive.HiveTypeName; import io.trino.plugin.hive.InternalHiveSplit; import io.trino.plugin.hive.InternalHiveSplit.InternalHiveBlock; -import io.trino.plugin.hive.TableToPartitionMapping; import io.trino.plugin.hive.fs.BlockLocation; import io.trino.plugin.hive.fs.TrinoFileStatus; import io.trino.plugin.hive.orc.OrcPageSourceFactory; @@ -53,7 +54,7 @@ public class InternalHiveSplitFactory private final Map strippedSchema; private final List partitionKeys; private final Optional pathDomain; - private final TableToPartitionMapping tableToPartitionMapping; + private final Map hiveColumnCoercions; private final BooleanSupplier partitionMatchSupplier; private final Optional bucketConversion; private final Optional bucketValidation; @@ -68,7 +69,7 @@ public InternalHiveSplitFactory( List partitionKeys, TupleDomain effectivePredicate, BooleanSupplier partitionMatchSupplier, - TableToPartitionMapping tableToPartitionMapping, + Map hiveColumnCoercions, Optional bucketConversion, Optional bucketValidation, DataSize minimumTargetSplitSize, @@ -81,7 +82,7 @@ public InternalHiveSplitFactory( this.partitionKeys = requireNonNull(partitionKeys, "partitionKeys is null"); pathDomain = getPathDomain(requireNonNull(effectivePredicate, "effectivePredicate is null")); this.partitionMatchSupplier = requireNonNull(partitionMatchSupplier, "partitionMatchSupplier is null"); - this.tableToPartitionMapping = requireNonNull(tableToPartitionMapping, "tableToPartitionMapping is null"); + this.hiveColumnCoercions = ImmutableMap.copyOf(requireNonNull(hiveColumnCoercions, "hiveColumnCoercions is null")); this.bucketConversion = requireNonNull(bucketConversion, "bucketConversion is null"); this.bucketValidation = requireNonNull(bucketValidation, "bucketValidation is null"); this.forceLocalScheduling = forceLocalScheduling; @@ -191,7 +192,7 @@ private Optional createInternalHiveSplit( tableBucketNumber, splittable, forceLocalScheduling && allBlocksHaveAddress(blocks), - tableToPartitionMapping, + hiveColumnCoercions, bucketConversion, bucketValidation, acidInfo, diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestBackgroundHiveSplitLoader.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestBackgroundHiveSplitLoader.java index 555ce79943bfb..57d8c8221cf93 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestBackgroundHiveSplitLoader.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestBackgroundHiveSplitLoader.java @@ -793,7 +793,7 @@ public void testBuildManifestFileIterator() List.of(), TupleDomain.all(), () -> true, - TableToPartitionMapping.empty(), + ImmutableMap.of(), Optional.empty(), Optional.empty(), DataSize.of(512, MEGABYTE), @@ -834,7 +834,7 @@ public void testBuildManifestFileIteratorNestedDirectory() List.of(), TupleDomain.all(), () -> true, - TableToPartitionMapping.empty(), + ImmutableMap.of(), Optional.empty(), Optional.empty(), DataSize.of(512, MEGABYTE), @@ -940,7 +940,7 @@ private static HivePartitionMetadata createPartitionMetadata() return new HivePartitionMetadata( new HivePartition(SIMPLE_TABLE.getSchemaTableName()), Optional.empty(), - TableToPartitionMapping.empty()); + ImmutableMap.of()); } private static void createOrcAcidFile(TrinoFileSystem fileSystem, Location location) @@ -1102,7 +1102,7 @@ private BackgroundHiveSplitLoader backgroundHiveSplitLoader( new HivePartitionMetadata( new HivePartition(new SchemaTableName("testSchema", "table_name")), Optional.empty(), - TableToPartitionMapping.empty())); + ImmutableMap.of())); return new BackgroundHiveSplitLoader( table, @@ -1133,7 +1133,7 @@ private BackgroundHiveSplitLoader backgroundHiveSplitLoader( new HivePartitionMetadata( new HivePartition(new SchemaTableName("testSchema", "table_name")), Optional.empty(), - TableToPartitionMapping.empty())); + ImmutableMap.of())); return backgroundHiveSplitLoader(partitions, locations, directoryLister, 100); } @@ -1209,7 +1209,7 @@ protected HivePartitionMetadata computeNext() { position++; return switch (position) { - case 0 -> new HivePartitionMetadata(new HivePartition(new SchemaTableName("testSchema", "table_name")), Optional.empty(), TableToPartitionMapping.empty()); + case 0 -> new HivePartitionMetadata(new HivePartition(new SchemaTableName("testSchema", "table_name")), Optional.empty(), ImmutableMap.of()); case 1 -> throw new RuntimeException("OFFLINE"); default -> endOfData(); }; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileFormats.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileFormats.java index ac702ba3b9f18..77c855173bce9 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileFormats.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileFormats.java @@ -974,7 +974,7 @@ private static void testPageSourceFactory( partitionKeys, getColumnHandles(testReadColumns), ImmutableList.of(), - TableToPartitionMapping.empty(), + ImmutableMap.of(), location.toString(), OptionalInt.empty(), paddedFileSize, diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHivePageSink.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHivePageSink.java index d4e5bce1c78c6..8eb1f26cf1618 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHivePageSink.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHivePageSink.java @@ -261,7 +261,7 @@ private static ConnectorPageSource createPageSource(TrinoFileSystemFactory fileS OptionalInt.empty(), OptionalInt.empty(), false, - TableToPartitionMapping.empty(), + ImmutableMap.of(), Optional.empty(), Optional.empty(), Optional.empty(), diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveSplit.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveSplit.java index 2195a07d35cfe..947d52b31e53e 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveSplit.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveSplit.java @@ -73,7 +73,7 @@ public void testJsonRoundTrip() OptionalInt.empty(), OptionalInt.empty(), true, - TableToPartitionMapping.mapColumnsByIndex(ImmutableMap.of(1, new HiveTypeName("string"))), + ImmutableMap.of(1, new HiveTypeName("string")), Optional.of(new HiveSplit.BucketConversion( BUCKETING_V1, 32, @@ -93,8 +93,7 @@ public void testJsonRoundTrip() assertThat(actual.getEstimatedFileSize()).isEqualTo(expected.getEstimatedFileSize()); assertThat(actual.getSchema()).isEqualTo(expected.getSchema()); assertThat(actual.getPartitionKeys()).isEqualTo(expected.getPartitionKeys()); - assertThat(actual.getTableToPartitionMapping().getPartitionColumnCoercions()).isEqualTo(expected.getTableToPartitionMapping().getPartitionColumnCoercions()); - assertThat(actual.getTableToPartitionMapping().getTableToPartitionColumns()).isEqualTo(expected.getTableToPartitionMapping().getTableToPartitionColumns()); + assertThat(actual.getHiveColumnCoercions()).isEqualTo(expected.getHiveColumnCoercions()); assertThat(actual.getBucketConversion()).isEqualTo(expected.getBucketConversion()); assertThat(actual.isForceLocalScheduling()).isEqualTo(expected.isForceLocalScheduling()); assertThat(actual.getAcidInfo().get()).isEqualTo(expected.getAcidInfo().get()); diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveSplitSource.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveSplitSource.java index 159b977920161..acd4791ee7139 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveSplitSource.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveSplitSource.java @@ -331,7 +331,7 @@ private TestSplit(int id, OptionalInt bucketNumber, DataSize fileSize, BooleanSu bucketNumber, true, false, - TableToPartitionMapping.empty(), + ImmutableMap.of(), Optional.empty(), Optional.empty(), Optional.empty(), diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestNodeLocalDynamicSplitPruning.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestNodeLocalDynamicSplitPruning.java index d67a1fa1b0373..52c13e20554a7 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestNodeLocalDynamicSplitPruning.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestNodeLocalDynamicSplitPruning.java @@ -134,7 +134,7 @@ private static ConnectorPageSource createTestingPageSource(HiveTransactionHandle OptionalInt.of(1), OptionalInt.of(1), false, - TableToPartitionMapping.empty(), + ImmutableMap.of(), Optional.empty(), Optional.empty(), Optional.empty(), diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestOrcPageSourceMemoryTracking.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestOrcPageSourceMemoryTracking.java index 50cb6a652300f..5d386113f92d0 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestOrcPageSourceMemoryTracking.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestOrcPageSourceMemoryTracking.java @@ -563,7 +563,7 @@ public ConnectorPageSource newPageSource(FileFormatDataSourceStats stats, Connec partitionKeys, columns, ImmutableList.of(), - TableToPartitionMapping.empty(), + ImmutableMap.of(), fileSplit.getPath().toString(), OptionalInt.empty(), fileSplit.getLength(), diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestTableToPartitionMapping.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestTableToPartitionMapping.java deleted file mode 100644 index 5a6f169e73323..0000000000000 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestTableToPartitionMapping.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * 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.hive; - -import com.google.common.collect.ImmutableMap; -import org.junit.jupiter.api.Test; - -import static io.trino.plugin.hive.TableToPartitionMapping.isIdentityMapping; -import static org.assertj.core.api.Assertions.assertThat; - -public class TestTableToPartitionMapping -{ - @Test - public void testIsOneToOneMapping() - { - assertThat(isIdentityMapping(ImmutableMap.builder() - .put(0, 0) - .put(1, 1) - .put(2, 2) - .put(3, 3) - .buildOrThrow())).isTrue(); - assertThat(isIdentityMapping(ImmutableMap.builder() - .put(0, 0) - .put(1, 1) - .put(2, 2) - .put(3, 3) - .put(5, 5) - .buildOrThrow())).isFalse(); - assertThat(isIdentityMapping(ImmutableMap.builder() - .put(0, 0) - .put(1, 1) - .put(2, 2) - .put(4, 5) - .buildOrThrow())).isFalse(); - } -} diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/orc/TestOrcPredicates.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/orc/TestOrcPredicates.java index e8a657ecb4e2f..f5cf9b88a1931 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/orc/TestOrcPredicates.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/orc/TestOrcPredicates.java @@ -30,7 +30,6 @@ import io.trino.plugin.hive.HivePageSourceProvider; import io.trino.plugin.hive.HiveType; import io.trino.plugin.hive.NodeVersion; -import io.trino.plugin.hive.TableToPartitionMapping; import io.trino.plugin.hive.WriterKind; import io.trino.plugin.hive.metastore.StorageFormat; import io.trino.spi.Page; @@ -167,7 +166,7 @@ private static ConnectorPageSource createPageSource( ImmutableList.of(), columns, ImmutableList.of(), - TableToPartitionMapping.empty(), + ImmutableMap.of(), location.toString(), OptionalInt.empty(), length, From 259e2af70f4d1475cf7205dcc12bb45aab2b047c Mon Sep 17 00:00:00 2001 From: Gaurav Sehgal Date: Sat, 18 Nov 2023 20:40:30 -0500 Subject: [PATCH 121/350] Unblock driver based on configurable timeout --- .../io/trino/execution/SqlTaskManager.java | 3 + .../io/trino/execution/TaskManagerConfig.java | 15 +++ .../java/io/trino/memory/QueryContext.java | 6 ++ .../main/java/io/trino/operator/Driver.java | 10 ++ .../java/io/trino/operator/DriverContext.java | 20 ++++ .../io/trino/operator/PipelineContext.java | 5 +- .../java/io/trino/operator/TaskContext.java | 6 ++ .../io/trino/testing/TestingTaskContext.java | 25 ++--- .../execution/MockRemoteTaskFactory.java | 1 + .../TestMemoryRevokingScheduler.java | 1 + .../java/io/trino/execution/TestSqlTask.java | 5 +- .../trino/execution/TestSqlTaskExecution.java | 7 +- .../execution/TestTaskManagerConfig.java | 3 + .../java/io/trino/memory/TestMemoryPools.java | 1 + .../io/trino/memory/TestMemoryTracking.java | 4 + .../operator/GroupByHashYieldAssertion.java | 1 + .../java/io/trino/operator/TestDriver.java | 98 ++++++++++++++++++- .../operator/TestingOperatorContext.java | 2 + 18 files changed, 196 insertions(+), 17 deletions(-) diff --git a/core/trino-main/src/main/java/io/trino/execution/SqlTaskManager.java b/core/trino-main/src/main/java/io/trino/execution/SqlTaskManager.java index f4c8fe43ab19e..12d3a4a2a01e6 100644 --- a/core/trino-main/src/main/java/io/trino/execution/SqlTaskManager.java +++ b/core/trino-main/src/main/java/io/trino/execution/SqlTaskManager.java @@ -123,6 +123,7 @@ public class SqlTaskManager private final ScheduledExecutorService taskManagementExecutor; private final ScheduledExecutorService driverYieldExecutor; + private final ScheduledExecutorService driverTimeoutExecutor; private final Duration infoCacheTime; private final Duration clientTimeout; @@ -216,6 +217,7 @@ public SqlTaskManager( this.taskManagementExecutor = taskManagementExecutor.getExecutor(); this.driverYieldExecutor = newScheduledThreadPool(config.getTaskYieldThreads(), threadsNamed("task-yield-%s")); + this.driverTimeoutExecutor = newScheduledThreadPool(config.getDriverTimeoutThreads(), threadsNamed("task-driver-timeout-%s")); SqlTaskExecutionFactory sqlTaskExecutionFactory = new SqlTaskExecutionFactory(taskNotificationExecutor, taskExecutor, planner, splitMonitor, tracer, config); @@ -269,6 +271,7 @@ private QueryContext createQueryContext( gcMonitor, taskNotificationExecutor, driverYieldExecutor, + driverTimeoutExecutor, maxQuerySpillPerNode, localSpillManager.getSpillSpaceTracker()); } diff --git a/core/trino-main/src/main/java/io/trino/execution/TaskManagerConfig.java b/core/trino-main/src/main/java/io/trino/execution/TaskManagerConfig.java index 4d584c36034ab..709e2d03c861a 100644 --- a/core/trino-main/src/main/java/io/trino/execution/TaskManagerConfig.java +++ b/core/trino-main/src/main/java/io/trino/execution/TaskManagerConfig.java @@ -100,6 +100,7 @@ public class TaskManagerConfig private int taskNotificationThreads = 5; private int taskYieldThreads = 3; + private int driverTimeoutThreads = 5; private BigDecimal levelTimeMultiplier = new BigDecimal(2.0); @@ -569,6 +570,20 @@ public TaskManagerConfig setTaskYieldThreads(int taskYieldThreads) return this; } + @Min(1) + public int getDriverTimeoutThreads() + { + return driverTimeoutThreads; + } + + @Config("task.driver-timeout-threads") + @ConfigDescription("Number of threads used for timing out blocked drivers if the timeout is set") + public TaskManagerConfig setDriverTimeoutThreads(int driverTimeoutThreads) + { + this.driverTimeoutThreads = driverTimeoutThreads; + return this; + } + public boolean isInterruptStuckSplitTasksEnabled() { return interruptStuckSplitTasksEnabled; diff --git a/core/trino-main/src/main/java/io/trino/memory/QueryContext.java b/core/trino-main/src/main/java/io/trino/memory/QueryContext.java index 571771987181b..3331e60d539d1 100644 --- a/core/trino-main/src/main/java/io/trino/memory/QueryContext.java +++ b/core/trino-main/src/main/java/io/trino/memory/QueryContext.java @@ -63,6 +63,7 @@ public class QueryContext private final GcMonitor gcMonitor; private final Executor notificationExecutor; private final ScheduledExecutorService yieldExecutor; + private final ScheduledExecutorService timeoutExecutor; private final long maxSpill; private final SpillSpaceTracker spillSpaceTracker; private final Map taskContexts = new ConcurrentHashMap<>(); @@ -86,6 +87,7 @@ public QueryContext( GcMonitor gcMonitor, Executor notificationExecutor, ScheduledExecutorService yieldExecutor, + ScheduledExecutorService timeoutExecutor, DataSize maxSpill, SpillSpaceTracker spillSpaceTracker) { @@ -97,6 +99,7 @@ public QueryContext( gcMonitor, notificationExecutor, yieldExecutor, + timeoutExecutor, maxSpill, spillSpaceTracker); } @@ -109,6 +112,7 @@ public QueryContext( GcMonitor gcMonitor, Executor notificationExecutor, ScheduledExecutorService yieldExecutor, + ScheduledExecutorService timeoutExecutor, DataSize maxSpill, SpillSpaceTracker spillSpaceTracker) { @@ -118,6 +122,7 @@ public QueryContext( this.gcMonitor = requireNonNull(gcMonitor, "gcMonitor is null"); this.notificationExecutor = requireNonNull(notificationExecutor, "notificationExecutor is null"); this.yieldExecutor = requireNonNull(yieldExecutor, "yieldExecutor is null"); + this.timeoutExecutor = requireNonNull(timeoutExecutor, "timeoutExecutor is null"); this.maxSpill = maxSpill.toBytes(); this.spillSpaceTracker = requireNonNull(spillSpaceTracker, "spillSpaceTracker is null"); this.guaranteedMemory = guaranteedMemory; @@ -257,6 +262,7 @@ public TaskContext addTaskContext( gcMonitor, notificationExecutor, yieldExecutor, + timeoutExecutor, session, taskMemoryContext, notifyStatusChanged, diff --git a/core/trino-main/src/main/java/io/trino/operator/Driver.java b/core/trino-main/src/main/java/io/trino/operator/Driver.java index f6949c4bb204a..0033344d7fe57 100644 --- a/core/trino-main/src/main/java/io/trino/operator/Driver.java +++ b/core/trino-main/src/main/java/io/trino/operator/Driver.java @@ -46,12 +46,15 @@ import static com.google.common.base.Preconditions.checkState; import static com.google.common.base.Throwables.throwIfUnchecked; import static com.google.common.base.Verify.verify; +import static com.google.common.util.concurrent.Futures.nonCancellationPropagating; +import static com.google.common.util.concurrent.Futures.withTimeout; import static com.google.common.util.concurrent.MoreExecutors.directExecutor; import static io.airlift.concurrent.MoreFutures.getFutureValue; import static io.trino.operator.Operator.NOT_BLOCKED; import static io.trino.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR; import static java.lang.Boolean.TRUE; import static java.util.Objects.requireNonNull; +import static java.util.concurrent.TimeUnit.MILLISECONDS; import static java.util.concurrent.TimeUnit.NANOSECONDS; // @@ -455,6 +458,13 @@ private ListenableFuture processInternal(OperationTimer operationTimer) // unblock when the first future is complete ListenableFuture blocked = firstFinishedFuture(blockedFutures); + if (driverContext.getBlockedTimeout().isPresent()) { + blocked = withTimeout( + nonCancellationPropagating(blocked), + driverContext.getBlockedTimeout().get().toMillis(), + MILLISECONDS, + driverContext.getTimeoutExecutor()); + } // driver records serial blocked time driverContext.recordBlocked(blocked); // each blocked operator is responsible for blocking the execution diff --git a/core/trino-main/src/main/java/io/trino/operator/DriverContext.java b/core/trino-main/src/main/java/io/trino/operator/DriverContext.java index 0ea5d3a5d70e7..67998f0a5e45a 100644 --- a/core/trino-main/src/main/java/io/trino/operator/DriverContext.java +++ b/core/trino-main/src/main/java/io/trino/operator/DriverContext.java @@ -28,6 +28,7 @@ import org.joda.time.DateTime; import java.util.List; +import java.util.Optional; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.Executor; import java.util.concurrent.ScheduledExecutorService; @@ -54,6 +55,7 @@ public class DriverContext private final PipelineContext pipelineContext; private final Executor notificationExecutor; private final ScheduledExecutorService yieldExecutor; + private final ScheduledExecutorService timeoutExecutor; private final AtomicBoolean finished = new AtomicBoolean(); @@ -70,6 +72,7 @@ public class DriverContext private final AtomicReference executionStartTime = new AtomicReference<>(); private final AtomicReference executionEndTime = new AtomicReference<>(); + private final AtomicReference> blockedTimeout = new AtomicReference<>(Optional.empty()); private final MemoryTrackingContext driverMemoryContext; @@ -82,12 +85,14 @@ public DriverContext( PipelineContext pipelineContext, Executor notificationExecutor, ScheduledExecutorService yieldExecutor, + ScheduledExecutorService timeoutExecutor, MemoryTrackingContext driverMemoryContext, long splitWeight) { this.pipelineContext = requireNonNull(pipelineContext, "pipelineContext is null"); this.notificationExecutor = requireNonNull(notificationExecutor, "notificationExecutor is null"); this.yieldExecutor = requireNonNull(yieldExecutor, "yieldExecutor is null"); + this.timeoutExecutor = requireNonNull(timeoutExecutor, "timeoutExecutor is null"); this.driverMemoryContext = requireNonNull(driverMemoryContext, "driverMemoryContext is null"); this.yieldSignal = new DriverYieldSignal(); this.splitWeight = splitWeight; @@ -447,6 +452,21 @@ public ScheduledExecutorService getYieldExecutor() return yieldExecutor; } + public ScheduledExecutorService getTimeoutExecutor() + { + return timeoutExecutor; + } + + public void setBlockedTimeout(Duration duration) + { + this.blockedTimeout.set(Optional.of(duration)); + } + + public Optional getBlockedTimeout() + { + return blockedTimeout.get(); + } + private static long nanosBetween(long start, long end) { return max(0, end - start); diff --git a/core/trino-main/src/main/java/io/trino/operator/PipelineContext.java b/core/trino-main/src/main/java/io/trino/operator/PipelineContext.java index 605d970564e1f..1f64fd5fb76dc 100644 --- a/core/trino-main/src/main/java/io/trino/operator/PipelineContext.java +++ b/core/trino-main/src/main/java/io/trino/operator/PipelineContext.java @@ -55,6 +55,7 @@ public class PipelineContext private final TaskContext taskContext; private final Executor notificationExecutor; private final ScheduledExecutorService yieldExecutor; + private final ScheduledExecutorService timeoutExecutor; private final int pipelineId; private final boolean inputPipeline; @@ -105,7 +106,7 @@ public class PipelineContext private final MemoryTrackingContext pipelineMemoryContext; - public PipelineContext(int pipelineId, TaskContext taskContext, Executor notificationExecutor, ScheduledExecutorService yieldExecutor, MemoryTrackingContext pipelineMemoryContext, boolean inputPipeline, boolean outputPipeline, boolean partitioned) + public PipelineContext(int pipelineId, TaskContext taskContext, Executor notificationExecutor, ScheduledExecutorService yieldExecutor, ScheduledExecutorService timeoutExecutor, MemoryTrackingContext pipelineMemoryContext, boolean inputPipeline, boolean outputPipeline, boolean partitioned) { this.pipelineId = pipelineId; this.inputPipeline = inputPipeline; @@ -114,6 +115,7 @@ public PipelineContext(int pipelineId, TaskContext taskContext, Executor notific this.taskContext = requireNonNull(taskContext, "taskContext is null"); this.notificationExecutor = requireNonNull(notificationExecutor, "notificationExecutor is null"); this.yieldExecutor = requireNonNull(yieldExecutor, "yieldExecutor is null"); + this.timeoutExecutor = requireNonNull(timeoutExecutor, "timeoutExecutor is null"); this.pipelineMemoryContext = requireNonNull(pipelineMemoryContext, "pipelineMemoryContext is null"); // Initialize the local memory contexts with the ExchangeOperator tag as ExchangeOperator will do the local memory allocations pipelineMemoryContext.initializeLocalMemoryContexts(ExchangeOperator.class.getSimpleName()); @@ -156,6 +158,7 @@ public DriverContext addDriverContext(long splitWeight) this, notificationExecutor, yieldExecutor, + timeoutExecutor, pipelineMemoryContext.newMemoryTrackingContext(), splitWeight); drivers.add(driverContext); diff --git a/core/trino-main/src/main/java/io/trino/operator/TaskContext.java b/core/trino-main/src/main/java/io/trino/operator/TaskContext.java index 3cc4a93e0b2a7..e72670b2f4c69 100644 --- a/core/trino-main/src/main/java/io/trino/operator/TaskContext.java +++ b/core/trino-main/src/main/java/io/trino/operator/TaskContext.java @@ -69,6 +69,7 @@ public class TaskContext private final GcMonitor gcMonitor; private final Executor notificationExecutor; private final ScheduledExecutorService yieldExecutor; + private final ScheduledExecutorService timeoutExecutor; private final Session session; private final long createNanos = System.nanoTime(); @@ -117,6 +118,7 @@ public static TaskContext createTaskContext( GcMonitor gcMonitor, Executor notificationExecutor, ScheduledExecutorService yieldExecutor, + ScheduledExecutorService timeoutExecutor, Session session, MemoryTrackingContext taskMemoryContext, Runnable notifyStatusChanged, @@ -129,6 +131,7 @@ public static TaskContext createTaskContext( gcMonitor, notificationExecutor, yieldExecutor, + timeoutExecutor, session, taskMemoryContext, notifyStatusChanged, @@ -144,6 +147,7 @@ private TaskContext( GcMonitor gcMonitor, Executor notificationExecutor, ScheduledExecutorService yieldExecutor, + ScheduledExecutorService timeoutExecutor, Session session, MemoryTrackingContext taskMemoryContext, Runnable notifyStatusChanged, @@ -155,6 +159,7 @@ private TaskContext( this.queryContext = requireNonNull(queryContext, "queryContext is null"); this.notificationExecutor = requireNonNull(notificationExecutor, "notificationExecutor is null"); this.yieldExecutor = requireNonNull(yieldExecutor, "yieldExecutor is null"); + this.timeoutExecutor = requireNonNull(timeoutExecutor, "timeoutExecutor is null"); this.session = session; this.taskMemoryContext = requireNonNull(taskMemoryContext, "taskMemoryContext is null"); @@ -186,6 +191,7 @@ public PipelineContext addPipelineContext(int pipelineId, boolean inputPipeline, this, notificationExecutor, yieldExecutor, + timeoutExecutor, taskMemoryContext.newMemoryTrackingContext(), inputPipeline, outputPipeline, diff --git a/core/trino-main/src/main/java/io/trino/testing/TestingTaskContext.java b/core/trino-main/src/main/java/io/trino/testing/TestingTaskContext.java index 04d4fb8d00909..f5cb6160b2715 100644 --- a/core/trino-main/src/main/java/io/trino/testing/TestingTaskContext.java +++ b/core/trino-main/src/main/java/io/trino/testing/TestingTaskContext.java @@ -39,21 +39,21 @@ public final class TestingTaskContext private TestingTaskContext() {} - public static TaskContext createTaskContext(Executor notificationExecutor, ScheduledExecutorService yieldExecutor, Session session) + public static TaskContext createTaskContext(Executor notificationExecutor, ScheduledExecutorService scheduledExecutor, Session session) { - return builder(notificationExecutor, yieldExecutor, session).build(); + return builder(notificationExecutor, scheduledExecutor, session).build(); } - public static TaskContext createTaskContext(Executor notificationExecutor, ScheduledExecutorService yieldExecutor, Session session, DataSize maxMemory) + public static TaskContext createTaskContext(Executor notificationExecutor, ScheduledExecutorService scheduledExecutor, Session session, DataSize maxMemory) { - return builder(notificationExecutor, yieldExecutor, session) + return builder(notificationExecutor, scheduledExecutor, session) .setQueryMaxMemory(maxMemory) .build(); } - public static TaskContext createTaskContext(Executor notificationExecutor, ScheduledExecutorService yieldExecutor, Session session, TaskStateMachine taskStateMachine) + public static TaskContext createTaskContext(Executor notificationExecutor, ScheduledExecutorService scheduledExecutor, Session session, TaskStateMachine taskStateMachine) { - return builder(notificationExecutor, yieldExecutor, session) + return builder(notificationExecutor, scheduledExecutor, session) .setTaskStateMachine(taskStateMachine) .build(); } @@ -73,15 +73,15 @@ private static TaskContext createTaskContext(QueryContext queryContext, Session true); } - public static Builder builder(Executor notificationExecutor, ScheduledExecutorService yieldExecutor, Session session) + public static Builder builder(Executor notificationExecutor, ScheduledExecutorService scheduledExecutor, Session session) { - return new Builder(notificationExecutor, yieldExecutor, session); + return new Builder(notificationExecutor, scheduledExecutor, session); } public static class Builder { private final Executor notificationExecutor; - private final ScheduledExecutorService yieldExecutor; + private final ScheduledExecutorService scheduledExecutor; private final Session session; private QueryId queryId = new QueryId("test_query"); private TaskStateMachine taskStateMachine; @@ -90,10 +90,10 @@ public static class Builder private DataSize maxSpillSize = DataSize.of(1, GIGABYTE); private DataSize queryMaxSpillSize = DataSize.of(1, GIGABYTE); - private Builder(Executor notificationExecutor, ScheduledExecutorService yieldExecutor, Session session) + private Builder(Executor notificationExecutor, ScheduledExecutorService scheduledExecutor, Session session) { this.notificationExecutor = notificationExecutor; - this.yieldExecutor = yieldExecutor; + this.scheduledExecutor = scheduledExecutor; this.session = session; } @@ -148,7 +148,8 @@ public TaskContext build() 0L, GC_MONITOR, notificationExecutor, - yieldExecutor, + scheduledExecutor, + scheduledExecutor, queryMaxSpillSize, spillSpaceTracker); diff --git a/core/trino-main/src/test/java/io/trino/execution/MockRemoteTaskFactory.java b/core/trino-main/src/test/java/io/trino/execution/MockRemoteTaskFactory.java index 681afda39efe5..b0fde885e1b10 100644 --- a/core/trino-main/src/test/java/io/trino/execution/MockRemoteTaskFactory.java +++ b/core/trino-main/src/test/java/io/trino/execution/MockRemoteTaskFactory.java @@ -212,6 +212,7 @@ public MockRemoteTask( new TestingGcMonitor(), executor, scheduledExecutor, + scheduledExecutor, DataSize.of(1, MEGABYTE), spillSpaceTracker); this.taskContext = queryContext.addTaskContext(taskStateMachine, TEST_SESSION, () -> {}, true, true); diff --git a/core/trino-main/src/test/java/io/trino/execution/TestMemoryRevokingScheduler.java b/core/trino-main/src/test/java/io/trino/execution/TestMemoryRevokingScheduler.java index c5855316582ee..1c9d4bbf56802 100644 --- a/core/trino-main/src/test/java/io/trino/execution/TestMemoryRevokingScheduler.java +++ b/core/trino-main/src/test/java/io/trino/execution/TestMemoryRevokingScheduler.java @@ -292,6 +292,7 @@ private QueryContext getOrCreateQueryContext(QueryId queryId) new TestingGcMonitor(), executor, scheduledExecutor, + scheduledExecutor, DataSize.of(1, GIGABYTE), spillSpaceTracker)); } diff --git a/core/trino-main/src/test/java/io/trino/execution/TestSqlTask.java b/core/trino-main/src/test/java/io/trino/execution/TestSqlTask.java index d865468401e46..48e4ea9d60901 100644 --- a/core/trino-main/src/test/java/io/trino/execution/TestSqlTask.java +++ b/core/trino-main/src/test/java/io/trino/execution/TestSqlTask.java @@ -92,6 +92,7 @@ public class TestSqlTask private TaskExecutor taskExecutor; private ScheduledExecutorService taskNotificationExecutor; private ScheduledExecutorService driverYieldExecutor; + private ScheduledExecutorService driverTimeoutExecutor; private SqlTaskExecutionFactory sqlTaskExecutionFactory; private final AtomicInteger nextTaskId = new AtomicInteger(); @@ -104,7 +105,7 @@ public void setUp() taskNotificationExecutor = newScheduledThreadPool(10, threadsNamed("task-notification-%s")); driverYieldExecutor = newScheduledThreadPool(2, threadsNamed("driver-yield-%s")); - + driverTimeoutExecutor = newScheduledThreadPool(2, threadsNamed("driver-timeout-%s")); LocalExecutionPlanner planner = createTestingPlanner(); sqlTaskExecutionFactory = new SqlTaskExecutionFactory( @@ -123,6 +124,7 @@ public void destroy() taskExecutor = null; taskNotificationExecutor.shutdownNow(); driverYieldExecutor.shutdown(); + driverTimeoutExecutor.shutdown(); sqlTaskExecutionFactory = null; } @@ -435,6 +437,7 @@ private SqlTask createInitialTask() new TestingGcMonitor(), taskNotificationExecutor, driverYieldExecutor, + driverTimeoutExecutor, DataSize.of(1, MEGABYTE), new SpillSpaceTracker(DataSize.of(1, GIGABYTE))); diff --git a/core/trino-main/src/test/java/io/trino/execution/TestSqlTaskExecution.java b/core/trino-main/src/test/java/io/trino/execution/TestSqlTaskExecution.java index 282c65442b80e..aae9ad8661ac8 100644 --- a/core/trino-main/src/test/java/io/trino/execution/TestSqlTaskExecution.java +++ b/core/trino-main/src/test/java/io/trino/execution/TestSqlTaskExecution.java @@ -98,6 +98,7 @@ public void testSimple() { ScheduledExecutorService taskNotificationExecutor = newScheduledThreadPool(10, threadsNamed("task-notification-%s")); ScheduledExecutorService driverYieldExecutor = newScheduledThreadPool(2, threadsNamed("driver-yield-%s")); + ScheduledExecutorService driverTimeoutExecutor = newScheduledThreadPool(2, threadsNamed("driver-timeout-%s")); TaskExecutor taskExecutor = new TimeSharingTaskExecutor(5, 10, 3, 4, Ticker.systemTicker()); taskExecutor.start(); @@ -131,7 +132,7 @@ public void testSimple() ImmutableList.of(testingScanOperatorFactory, taskOutputOperatorFactory), OptionalInt.empty())), ImmutableList.of(TABLE_SCAN_NODE_ID)); - TaskContext taskContext = newTestingTaskContext(taskNotificationExecutor, driverYieldExecutor, taskStateMachine); + TaskContext taskContext = newTestingTaskContext(taskNotificationExecutor, driverYieldExecutor, driverTimeoutExecutor, taskStateMachine); SqlTaskExecution sqlTaskExecution = new SqlTaskExecution( taskStateMachine, taskContext, @@ -197,10 +198,11 @@ public void testSimple() taskExecutor.stop(); taskNotificationExecutor.shutdownNow(); driverYieldExecutor.shutdown(); + driverTimeoutExecutor.shutdown(); } } - private TaskContext newTestingTaskContext(ScheduledExecutorService taskNotificationExecutor, ScheduledExecutorService driverYieldExecutor, TaskStateMachine taskStateMachine) + private TaskContext newTestingTaskContext(ScheduledExecutorService taskNotificationExecutor, ScheduledExecutorService driverYieldExecutor, ScheduledExecutorService driverTimeoutExecutor, TaskStateMachine taskStateMachine) { QueryContext queryContext = new QueryContext( new QueryId("queryid"), @@ -209,6 +211,7 @@ private TaskContext newTestingTaskContext(ScheduledExecutorService taskNotificat new TestingGcMonitor(), taskNotificationExecutor, driverYieldExecutor, + driverTimeoutExecutor, DataSize.of(1, MEGABYTE), new SpillSpaceTracker(DataSize.of(1, GIGABYTE))); return queryContext.addTaskContext(taskStateMachine, TEST_SESSION, () -> {}, false, false); diff --git a/core/trino-main/src/test/java/io/trino/execution/TestTaskManagerConfig.java b/core/trino-main/src/test/java/io/trino/execution/TestTaskManagerConfig.java index 73685aa8f09b7..965ee6e277cee 100644 --- a/core/trino-main/src/test/java/io/trino/execution/TestTaskManagerConfig.java +++ b/core/trino-main/src/test/java/io/trino/execution/TestTaskManagerConfig.java @@ -71,6 +71,7 @@ public void testDefaults() .setHttpTimeoutThreads(3) .setTaskNotificationThreads(5) .setTaskYieldThreads(3) + .setDriverTimeoutThreads(5) .setLevelTimeMultiplier(new BigDecimal("2")) .setStatisticsCpuTimerEnabled(true) .setInterruptStuckSplitTasksEnabled(true) @@ -116,6 +117,7 @@ public void testExplicitPropertyMappings() .put("task.http-timeout-threads", "10") .put("task.task-notification-threads", "13") .put("task.task-yield-threads", "8") + .put("task.driver-timeout-threads", "10") .put("task.level-time-multiplier", "2.1") .put("task.statistics-cpu-timer-enabled", "false") .put("task.interrupt-stuck-split-tasks-enabled", "false") @@ -156,6 +158,7 @@ public void testExplicitPropertyMappings() .setHttpTimeoutThreads(10) .setTaskNotificationThreads(13) .setTaskYieldThreads(8) + .setDriverTimeoutThreads(10) .setLevelTimeMultiplier(new BigDecimal("2.1")) .setStatisticsCpuTimerEnabled(false) .setInterruptStuckSplitTasksEnabled(false) diff --git a/core/trino-main/src/test/java/io/trino/memory/TestMemoryPools.java b/core/trino-main/src/test/java/io/trino/memory/TestMemoryPools.java index fde67f51b1dd4..0b2b3cb1f0499 100644 --- a/core/trino-main/src/test/java/io/trino/memory/TestMemoryPools.java +++ b/core/trino-main/src/test/java/io/trino/memory/TestMemoryPools.java @@ -97,6 +97,7 @@ private void setUp(Supplier> driversSupplier) new TestingGcMonitor(), localQueryRunner.getExecutor(), localQueryRunner.getScheduler(), + localQueryRunner.getScheduler(), TEN_MEGABYTES, spillSpaceTracker); taskContext = createTaskContext(queryContext, localQueryRunner.getExecutor(), localQueryRunner.getDefaultSession()); diff --git a/core/trino-main/src/test/java/io/trino/memory/TestMemoryTracking.java b/core/trino-main/src/test/java/io/trino/memory/TestMemoryTracking.java index 6069671e6611e..4f39206b198b6 100644 --- a/core/trino-main/src/test/java/io/trino/memory/TestMemoryTracking.java +++ b/core/trino-main/src/test/java/io/trino/memory/TestMemoryTracking.java @@ -68,11 +68,13 @@ public class TestMemoryTracking private MemoryPool memoryPool; private ExecutorService notificationExecutor; private ScheduledExecutorService yieldExecutor; + private ScheduledExecutorService timeoutExecutor; @AfterEach public void tearDown() { notificationExecutor.shutdownNow(); + timeoutExecutor.shutdownNow(); yieldExecutor.shutdownNow(); queryContext = null; taskContext = null; @@ -87,6 +89,7 @@ public void setUpTest() { notificationExecutor = newCachedThreadPool(daemonThreadsNamed("local-query-runner-executor-%s")); yieldExecutor = newScheduledThreadPool(2, daemonThreadsNamed("local-query-runner-scheduler-%s")); + timeoutExecutor = newScheduledThreadPool(2, daemonThreadsNamed("local-query-runner-driver-timeout-%s")); memoryPool = new MemoryPool(memoryPoolSize); queryContext = new QueryContext( @@ -96,6 +99,7 @@ public void setUpTest() new TestingGcMonitor(), notificationExecutor, yieldExecutor, + timeoutExecutor, queryMaxSpillSize, spillSpaceTracker); taskContext = queryContext.addTaskContext( diff --git a/core/trino-main/src/test/java/io/trino/operator/GroupByHashYieldAssertion.java b/core/trino-main/src/test/java/io/trino/operator/GroupByHashYieldAssertion.java index 298ca2f30931e..a02624fb6a384 100644 --- a/core/trino-main/src/test/java/io/trino/operator/GroupByHashYieldAssertion.java +++ b/core/trino-main/src/test/java/io/trino/operator/GroupByHashYieldAssertion.java @@ -87,6 +87,7 @@ public static GroupByHashYieldResult finishOperatorWithYieldingGroupByHash(List< new TestingGcMonitor(), EXECUTOR, SCHEDULED_EXECUTOR, + SCHEDULED_EXECUTOR, DataSize.of(512, MEGABYTE), new SpillSpaceTracker(DataSize.of(512, MEGABYTE))); diff --git a/core/trino-main/src/test/java/io/trino/operator/TestDriver.java b/core/trino-main/src/test/java/io/trino/operator/TestDriver.java index d76f0ad55f5ab..8b3313b2c0d0b 100644 --- a/core/trino-main/src/test/java/io/trino/operator/TestDriver.java +++ b/core/trino-main/src/test/java/io/trino/operator/TestDriver.java @@ -62,8 +62,10 @@ import static io.trino.testing.TestingHandles.TEST_CATALOG_HANDLE; import static io.trino.testing.TestingHandles.TEST_TABLE_HANDLE; import static io.trino.testing.TestingTaskContext.createTaskContext; +import static java.lang.Thread.sleep; import static java.util.concurrent.Executors.newCachedThreadPool; import static java.util.concurrent.Executors.newScheduledThreadPool; +import static java.util.concurrent.TimeUnit.MILLISECONDS; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_METHOD; @@ -285,6 +287,53 @@ public void testUnblocksOnFinish() assertThat(blocked.isDone()).isTrue(); } + @Test + public void testUnblocksOnTimeout() + throws InterruptedException + { + List types = ImmutableList.of(VARCHAR, BIGINT, BIGINT); + driverContext.setBlockedTimeout(new Duration(70, MILLISECONDS)); + // Create driver with 3 operators, one of which is blocked such that it will not move any page and + // return a blocked timeout future. + Operator operator1 = createSinkOperator(types, 1, "test1"); + BlockedOperator operator2 = createBlockedOperator(types, 2, "test2"); + Operator operator3 = createSinkOperator(types, 3, "test3"); + Operator operator4 = createSinkOperator(types, 4, "test3"); + Driver driver = Driver.createDriver(driverContext, operator1, operator2, operator3, operator4); + + ListenableFuture blocked = driver.processForDuration(new Duration(200, MILLISECONDS)); + assertThat(blocked.isDone()).isFalse(); + // wait for the blocked future to be timed out + sleep(100); + assertThat(blocked.isDone()).isTrue(); + // verify that the blocked operator is not cancelled or done due to timeout + assertThat(operator2.isCancelled()).isFalse(); + assertThat(operator2.isDone()).isFalse(); + } + + @Test + public void testUnblocksWhenBlockedOperatorIsUnblockedAndTimeoutIsSet() + { + List types = ImmutableList.of(VARCHAR, BIGINT, BIGINT); + driverContext.setBlockedTimeout(new Duration(100, MILLISECONDS)); + // Create driver with 3 operators, one of which is blocked such that it will not move any page and + // return a blocked timeout future. + Operator operator1 = createSinkOperator(types, 1, "test1"); + BlockedOperator operator2 = createBlockedOperator(types, 2, "test2"); + Operator operator3 = createSinkOperator(types, 3, "test3"); + Operator operator4 = createSinkOperator(types, 4, "test3"); + Driver driver = Driver.createDriver(driverContext, operator1, operator2, operator3, operator4); + + ListenableFuture blocked = driver.processForDuration(new Duration(200, MILLISECONDS)); + assertThat(blocked.isDone()).isFalse(); + // unblock the blocked operator + operator2.setDone(); + // verify that the blocked future is done but is not cancelled + assertThat(operator2.isDone()).isTrue(); + assertThat(blocked.isDone()).isTrue(); + assertThat(operator2.isCancelled()).isFalse(); + } + @Test public void testBrokenOperatorAddSource() { @@ -336,10 +385,22 @@ private static Split newMockSplit() } private PageConsumerOperator createSinkOperator(List types) + { + return createSinkOperator(types, 1, "test"); + } + + private PageConsumerOperator createSinkOperator(List types, int operatorId, String planNodeId) + { + // materialize the output to catch some type errors + MaterializedResult.Builder resultBuilder = MaterializedResult.resultBuilder(driverContext.getSession(), types); + return new PageConsumerOperator(driverContext.addOperatorContext(operatorId, new PlanNodeId(planNodeId), "sink"), resultBuilder::page, Function.identity()); + } + + private BlockedOperator createBlockedOperator(List types, int operatorId, String planNodeId) { // materialize the output to catch some type errors MaterializedResult.Builder resultBuilder = MaterializedResult.resultBuilder(driverContext.getSession(), types); - return new PageConsumerOperator(driverContext.addOperatorContext(1, new PlanNodeId("test"), "sink"), resultBuilder::page, Function.identity()); + return new BlockedOperator(driverContext.addOperatorContext(operatorId, new PlanNodeId(planNodeId), "sink"), resultBuilder::page, Function.identity()); } private static class BrokenOperator @@ -477,6 +538,41 @@ void setFinished() } } + private static class BlockedOperator + extends PageConsumerOperator + { + private final SettableFuture blocked = SettableFuture.create(); + + public BlockedOperator( + OperatorContext operatorContext, + Consumer pageConsumer, + Function pagePreprocessor) + { + super(operatorContext, pageConsumer, pagePreprocessor); + } + + @Override + public ListenableFuture isBlocked() + { + return blocked; + } + + private void setDone() + { + blocked.set(null); + } + + private boolean isDone() + { + return blocked.isDone(); + } + + private boolean isCancelled() + { + return blocked.isCancelled(); + } + } + private static class AlwaysBlockedTableScanOperator extends TableScanOperator { diff --git a/core/trino-main/src/test/java/io/trino/operator/TestingOperatorContext.java b/core/trino-main/src/test/java/io/trino/operator/TestingOperatorContext.java index db29308522379..677e71e6540c7 100644 --- a/core/trino-main/src/test/java/io/trino/operator/TestingOperatorContext.java +++ b/core/trino-main/src/test/java/io/trino/operator/TestingOperatorContext.java @@ -42,6 +42,7 @@ public static OperatorContext create(ScheduledExecutorService scheduledExecutor) taskContext, executor, scheduledExecutor, + scheduledExecutor, pipelineMemoryContext, false, false, @@ -51,6 +52,7 @@ public static OperatorContext create(ScheduledExecutorService scheduledExecutor) pipelineContext, executor, scheduledExecutor, + scheduledExecutor, pipelineMemoryContext, 0L); From 7e708ba2e79046b13bd833ec054deb9545961c58 Mon Sep 17 00:00:00 2001 From: Gaurav Sehgal Date: Fri, 1 Dec 2023 23:40:14 -0500 Subject: [PATCH 122/350] Add support for closing idle writers This change will help in reducing memory footprint in long-running partitioned writes. Thus, we can scale more to increase the write performance. --- .../io/trino/SystemSessionProperties.java | 21 +++ .../trino/operator/TableWriterOperator.java | 55 +++++- .../spi/connector/ConnectorPageSink.java | 8 + .../ClassLoaderSafeConnectorPageSink.java | 8 + .../deltalake/AbstractDeltaLakePageSink.java | 26 ++- .../plugin/deltalake/DeltaLakeConfig.java | 15 ++ .../deltalake/DeltaLakeSessionProperties.java | 11 ++ .../deltalake/TestCloseIdleWriters.java | 103 +++++++++++ .../plugin/deltalake/TestDeltaLakeConfig.java | 4 + .../java/io/trino/plugin/hive/HiveConfig.java | 14 ++ .../io/trino/plugin/hive/HivePageSink.java | 37 +++- .../plugin/hive/HiveSessionProperties.java | 11 ++ .../plugin/hive/TestCloseIdleWriters.java | 91 ++++++++++ .../io/trino/plugin/hive/TestHiveConfig.java | 3 + .../trino/plugin/hive/TestHivePageSink.java | 165 +++++++++++++++--- .../trino/plugin/iceberg/IcebergConfig.java | 16 ++ .../trino/plugin/iceberg/IcebergPageSink.java | 31 +++- .../iceberg/IcebergSessionProperties.java | 11 ++ .../plugin/iceberg/TestCloseIdleWriters.java | 91 ++++++++++ .../plugin/iceberg/TestIcebergConfig.java | 3 + 20 files changed, 688 insertions(+), 36 deletions(-) create mode 100644 plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestCloseIdleWriters.java create mode 100644 plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestCloseIdleWriters.java create mode 100644 plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestCloseIdleWriters.java diff --git a/core/trino-main/src/main/java/io/trino/SystemSessionProperties.java b/core/trino-main/src/main/java/io/trino/SystemSessionProperties.java index 2f56a09680cc6..c08a13a91391e 100644 --- a/core/trino-main/src/main/java/io/trino/SystemSessionProperties.java +++ b/core/trino-main/src/main/java/io/trino/SystemSessionProperties.java @@ -53,6 +53,7 @@ import static io.trino.spi.type.TimeZoneKey.getTimeZoneKey; import static java.lang.Math.min; import static java.lang.String.format; +import static java.util.concurrent.TimeUnit.SECONDS; public final class SystemSessionProperties implements SystemSessionPropertiesProvider @@ -207,6 +208,8 @@ public final class SystemSessionProperties public static final String USE_COST_BASED_PARTITIONING = "use_cost_based_partitioning"; public static final String FORCE_SPILLING_JOIN = "force_spilling_join"; public static final String PAGE_PARTITIONING_BUFFER_POOL_SIZE = "page_partitioning_buffer_pool_size"; + public static final String IDLE_WRITER_MIN_DATA_SIZE_THRESHOLD = "idle_writer_min_data_size_threshold"; + public static final String CLOSE_IDLE_WRITERS_TRIGGER_DURATION = "close_idle_writers_trigger_duration"; private final List> sessionProperties; @@ -1058,6 +1061,14 @@ public SystemSessionProperties( integerProperty(PAGE_PARTITIONING_BUFFER_POOL_SIZE, "Maximum number of free buffers in the per task partitioned page buffer pool. Setting this to zero effectively disables the pool", taskManagerConfig.getPagePartitioningBufferPoolSize(), + true), + dataSizeProperty(IDLE_WRITER_MIN_DATA_SIZE_THRESHOLD, + "Minimum amount of data written by a writer operator on average before it tries to close the idle writers", + DataSize.of(256, MEGABYTE), + true), + durationProperty(CLOSE_IDLE_WRITERS_TRIGGER_DURATION, + "The duration after which the writer operator tries to close the idle writers", + new Duration(5, SECONDS), true)); } @@ -1896,4 +1907,14 @@ public static int getPagePartitioningBufferPoolSize(Session session) { return session.getSystemProperty(PAGE_PARTITIONING_BUFFER_POOL_SIZE, Integer.class); } + + public static DataSize getIdleWriterMinDataSizeThreshold(Session session) + { + return session.getSystemProperty(IDLE_WRITER_MIN_DATA_SIZE_THRESHOLD, DataSize.class); + } + + public static Duration getCloseIdleWritersTriggerDuration(Session session) + { + return session.getSystemProperty(CLOSE_IDLE_WRITERS_TRIGGER_DURATION, Duration.class); + } } diff --git a/core/trino-main/src/main/java/io/trino/operator/TableWriterOperator.java b/core/trino-main/src/main/java/io/trino/operator/TableWriterOperator.java index 37619858e1e45..ac4efe5b63074 100644 --- a/core/trino-main/src/main/java/io/trino/operator/TableWriterOperator.java +++ b/core/trino-main/src/main/java/io/trino/operator/TableWriterOperator.java @@ -20,7 +20,9 @@ import com.google.common.primitives.Ints; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; +import io.airlift.log.Logger; import io.airlift.slice.Slice; +import io.airlift.units.DataSize; import io.airlift.units.Duration; import io.trino.Session; import io.trino.memory.context.LocalMemoryContext; @@ -42,6 +44,7 @@ import java.util.Collection; import java.util.List; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Supplier; @@ -53,6 +56,8 @@ import static com.google.common.util.concurrent.MoreExecutors.directExecutor; import static io.airlift.concurrent.MoreFutures.getFutureValue; import static io.airlift.concurrent.MoreFutures.toListenableFuture; +import static io.trino.SystemSessionProperties.getCloseIdleWritersTriggerDuration; +import static io.trino.SystemSessionProperties.getIdleWriterMinDataSizeThreshold; import static io.trino.SystemSessionProperties.isStatisticsCpuTimerEnabled; import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.spi.type.VarbinaryType.VARBINARY; @@ -64,6 +69,7 @@ public class TableWriterOperator implements Operator { + private static final Logger LOG = Logger.get(TableWriterOperator.class); public static final int ROW_COUNT_CHANNEL = 0; public static final int FRAGMENT_CHANNEL = 1; public static final int STATS_START_CHANNEL = 2; @@ -111,10 +117,22 @@ public TableWriterOperatorFactory( public Operator createOperator(DriverContext driverContext) { checkState(!closed, "Factory is already closed"); + // Driver should call getOutput() periodically on TableWriterOperator to close idle writers which will essentially + // decrease the memory usage even if no pages were added to that writer thread. + if (getCloseIdleWritersTriggerDuration(session).toMillis() > 0) { + driverContext.setBlockedTimeout(getCloseIdleWritersTriggerDuration(session)); + } OperatorContext context = driverContext.addOperatorContext(operatorId, planNodeId, TableWriterOperator.class.getSimpleName()); Operator statisticsAggregationOperator = statisticsAggregationOperatorFactory.createOperator(driverContext); boolean statisticsCpuTimerEnabled = !(statisticsAggregationOperator instanceof DevNullOperator) && isStatisticsCpuTimerEnabled(session); - return new TableWriterOperator(context, createPageSink(driverContext), columnChannels, statisticsAggregationOperator, types, statisticsCpuTimerEnabled); + return new TableWriterOperator( + context, + createPageSink(driverContext), + columnChannels, + statisticsAggregationOperator, + types, + statisticsCpuTimerEnabled, + getIdleWriterMinDataSizeThreshold(session)); } private ConnectorPageSink createPageSink(DriverContext driverContext) @@ -159,6 +177,7 @@ private enum State private final AtomicLong pageSinkPeakMemoryUsage = new AtomicLong(); private final Operator statisticAggregationOperator; private final List types; + private final DataSize idleWriterMinDataSizeThreshold; private ListenableFuture blocked = NOT_BLOCKED; private CompletableFuture> finishFuture; @@ -170,8 +189,10 @@ private enum State private final OperationTiming statisticsTiming = new OperationTiming(); private final boolean statisticsCpuTimerEnabled; - private final Supplier tableWriterInfoSupplier; + // This records the last physical written data size when connector closeIdleWriters is triggered. + private long lastPhysicalWrittenDataSize; + private boolean newPagesAdded; public TableWriterOperator( OperatorContext operatorContext, @@ -179,7 +200,8 @@ public TableWriterOperator( List columnChannels, Operator statisticAggregationOperator, List types, - boolean statisticsCpuTimerEnabled) + boolean statisticsCpuTimerEnabled, + DataSize idleWriterMinDataSizeThreshold) { this.operatorContext = requireNonNull(operatorContext, "operatorContext is null"); this.pageSinkMemoryContext = operatorContext.newLocalUserMemoryContext(TableWriterOperator.class.getSimpleName()); @@ -188,6 +210,7 @@ public TableWriterOperator( this.statisticAggregationOperator = requireNonNull(statisticAggregationOperator, "statisticAggregationOperator is null"); this.types = ImmutableList.copyOf(requireNonNull(types, "types is null")); this.statisticsCpuTimerEnabled = statisticsCpuTimerEnabled; + this.idleWriterMinDataSizeThreshold = requireNonNull(idleWriterMinDataSizeThreshold, "idleWriterMinDataSizeThreshold is null"); this.tableWriterInfoSupplier = createTableWriterInfoSupplier(pageSinkPeakMemoryUsage, statisticsTiming, pageSink); this.operatorContext.setInfoSupplier(tableWriterInfoSupplier); } @@ -259,14 +282,20 @@ public void addInput(Page page) rowCount += page.getPositionCount(); updateWrittenBytes(); operatorContext.recordWriterInputDataSize(page.getSizeInBytes()); + newPagesAdded = true; } @Override public Page getOutput() { - if (!blocked.isDone()) { + tryClosingIdleWriters(); + // This method could be called even when new pages have not been added. In that case, we don't have to + // try to get the output from the aggregation operator. It could be expensive since getOutput() is + // called quite frequently. + if (!(blocked.isDone() && (newPagesAdded || state != State.RUNNING))) { return null; } + newPagesAdded = false; if (!statisticAggregationOperator.isFinished()) { OperationTimer timer = new OperationTimer(statisticsCpuTimerEnabled); @@ -365,6 +394,24 @@ private void updateWrittenBytes() writtenBytes = current; } + private void tryClosingIdleWriters() + { + long physicalWrittenDataSize = getTaskContext().getPhysicalWrittenDataSize(); + Optional writerCount = getTaskContext().getMaxWriterCount(); + if (writerCount.isEmpty() || physicalWrittenDataSize - lastPhysicalWrittenDataSize <= idleWriterMinDataSizeThreshold.toBytes() * writerCount.get()) { + return; + } + pageSink.closeIdleWriters(); + updateMemoryUsage(); + updateWrittenBytes(); + lastPhysicalWrittenDataSize = physicalWrittenDataSize; + } + + private TaskContext getTaskContext() + { + return operatorContext.getDriverContext().getPipelineContext().getTaskContext(); + } + private void updateMemoryUsage() { long pageSinkMemoryUsage = pageSink.getMemoryUsage(); diff --git a/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorPageSink.java b/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorPageSink.java index b7fc9e2897ad8..952eef2aece46 100644 --- a/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorPageSink.java +++ b/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorPageSink.java @@ -61,6 +61,14 @@ default long getValidationCpuNanos() */ CompletableFuture appendPage(Page page); + /** + * Closes the idle partition writers that have not received any data since the last time this + * method is called. This method is called periodically based on some + * data written threshold by the TableWriterOperator. It is needed to avoid high memory + * usage due to stale partitions kept in memory during partitioned writes. + */ + default void closeIdleWriters() {} + /** * Notifies the connector that no more pages will be appended and returns * connector-specific information that will be sent to the coordinator to diff --git a/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/classloader/ClassLoaderSafeConnectorPageSink.java b/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/classloader/ClassLoaderSafeConnectorPageSink.java index 6b399d2d40ded..a4f76d74b3d28 100644 --- a/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/classloader/ClassLoaderSafeConnectorPageSink.java +++ b/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/classloader/ClassLoaderSafeConnectorPageSink.java @@ -69,6 +69,14 @@ public CompletableFuture appendPage(Page page) } } + @Override + public void closeIdleWriters() + { + try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { + delegate.closeIdleWriters(); + } + } + @Override public CompletableFuture> finish() { diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/AbstractDeltaLakePageSink.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/AbstractDeltaLakePageSink.java index b4f6c3e86c4b6..e5b2e8aef0b01 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/AbstractDeltaLakePageSink.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/AbstractDeltaLakePageSink.java @@ -100,11 +100,12 @@ public abstract class AbstractDeltaLakePageSink private final DeltaLakeWriterStats stats; private final String trinoVersion; private final long targetMaxFileSize; - + private final long idleWriterMinFileSize; private long writtenBytes; private long memoryUsage; private final List closedWriterRollbackActions = new ArrayList<>(); + private final List activeWriters = new ArrayList<>(); protected final ImmutableList.Builder dataFileInfos = ImmutableList.builder(); private final DeltaLakeParquetSchemaMapping parquetSchemaMapping; @@ -190,6 +191,7 @@ public AbstractDeltaLakePageSink( this.trinoVersion = requireNonNull(trinoVersion, "trinoVersion is null"); this.targetMaxFileSize = DeltaLakeSessionProperties.getTargetMaxFileSize(session); + this.idleWriterMinFileSize = DeltaLakeSessionProperties.getIdleWriterMinFileSize(session); } protected abstract void processSynthesizedColumn(DeltaLakeColumnHandle column); @@ -312,6 +314,7 @@ private void writePage(Page page) } DeltaLakeWriter writer = writers.get(index); + verify(writer != null, "Expected writer at index %s", index); long currentWritten = writer.getWrittenBytes(); long currentMemory = writer.getMemoryUsage(); @@ -320,6 +323,22 @@ private void writePage(Page page) writtenBytes += writer.getWrittenBytes() - currentWritten; memoryUsage += writer.getMemoryUsage() - currentMemory; + // Mark this writer as active (i.e. not idle) + activeWriters.set(index, true); + } + } + + @Override + public void closeIdleWriters() + { + for (int writerIndex = 0; writerIndex < writers.size(); writerIndex++) { + DeltaLakeWriter writer = writers.get(writerIndex); + if (activeWriters.get(writerIndex) || writer == null || writer.getWrittenBytes() <= idleWriterMinFileSize) { + activeWriters.set(writerIndex, false); + continue; + } + LOG.debug("Closing writer %s with %s bytes written", writerIndex, writer.getWrittenBytes()); + closeWriter(writerIndex); } } @@ -334,6 +353,7 @@ private int[] getWriterIndexes(Page page) // expand writers list to new size while (writers.size() <= pageIndexer.getMaxIndex()) { writers.add(null); + activeWriters.add(false); } // create missing writers for (int position = 0; position < page.getPositionCount(); position++) { @@ -374,7 +394,6 @@ private int[] getWriterIndexes(Page page) memoryUsage += writer.getMemoryUsage(); } verify(writers.size() == pageIndexer.getMaxIndex() + 1); - verify(!writers.contains(null)); return writerIndexes; } @@ -387,6 +406,9 @@ private String getRelativeFilePath(Optional partitionName, String fileNa protected void closeWriter(int writerIndex) { DeltaLakeWriter writer = writers.get(writerIndex); + if (writer == null) { + return; + } long currentWritten = writer.getWrittenBytes(); long currentMemory = writer.getMemoryUsage(); diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeConfig.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeConfig.java index acef9be6c5228..99d1ff1c8d275 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeConfig.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeConfig.java @@ -76,6 +76,7 @@ public class DeltaLakeConfig private boolean deleteSchemaLocationsFallback; private String parquetTimeZone = TimeZone.getDefault().getID(); private DataSize targetMaxFileSize = DataSize.of(1, GIGABYTE); + private DataSize idleWriterMinFileSize = DataSize.of(16, MEGABYTE); private boolean uniqueTableLocation = true; private boolean registerTableProcedureEnabled; private boolean projectionPushdownEnabled = true; @@ -450,6 +451,20 @@ public DeltaLakeConfig setTargetMaxFileSize(DataSize targetMaxFileSize) return this; } + @NotNull + public DataSize getIdleWriterMinFileSize() + { + return idleWriterMinFileSize; + } + + @Config("delta.idle-writer-min-file-size") + @ConfigDescription("Minimum data written by a single partition writer before it can be consider as 'idle' and could be closed by the engine") + public DeltaLakeConfig setIdleWriterMinFileSize(DataSize idleWriterMinFileSize) + { + this.idleWriterMinFileSize = idleWriterMinFileSize; + return this; + } + public boolean isUniqueTableLocation() { return uniqueTableLocation; diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSessionProperties.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSessionProperties.java index 7b779e053a175..753ca07a80692 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSessionProperties.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSessionProperties.java @@ -59,6 +59,7 @@ public final class DeltaLakeSessionProperties private static final String PARQUET_WRITER_BLOCK_SIZE = "parquet_writer_block_size"; private static final String PARQUET_WRITER_PAGE_SIZE = "parquet_writer_page_size"; private static final String TARGET_MAX_FILE_SIZE = "target_max_file_size"; + private static final String IDLE_WRITER_MIN_FILE_SIZE = "idle_writer_min_file_size"; private static final String COMPRESSION_CODEC = "compression_codec"; // This property is not supported by Delta Lake and exists solely for technical reasons. @Deprecated @@ -150,6 +151,11 @@ public DeltaLakeSessionProperties( "Target maximum size of written files; the actual size may be larger", deltaLakeConfig.getTargetMaxFileSize(), false), + dataSizeProperty( + IDLE_WRITER_MIN_FILE_SIZE, + "Minimum data written by a single partition writer before it can be consider as 'idle' and could be closed by the engine", + deltaLakeConfig.getIdleWriterMinFileSize(), + false), enumProperty( TIMESTAMP_PRECISION, "Internal Delta Lake connector property", @@ -266,6 +272,11 @@ public static long getTargetMaxFileSize(ConnectorSession session) return session.getProperty(TARGET_MAX_FILE_SIZE, DataSize.class).toBytes(); } + public static long getIdleWriterMinFileSize(ConnectorSession session) + { + return session.getProperty(IDLE_WRITER_MIN_FILE_SIZE, DataSize.class).toBytes(); + } + public static Duration getDynamicFilteringWaitTimeout(ConnectorSession session) { return session.getProperty(DYNAMIC_FILTERING_WAIT_TIMEOUT, Duration.class); diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestCloseIdleWriters.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestCloseIdleWriters.java new file mode 100644 index 0000000000000..865bdf9d55eb2 --- /dev/null +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestCloseIdleWriters.java @@ -0,0 +1,103 @@ +/* + * 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.deltalake; + +import com.google.common.collect.ImmutableMap; +import io.trino.Session; +import io.trino.testing.AbstractTestQueryFramework; +import io.trino.testing.DistributedQueryRunner; +import io.trino.testing.QueryRunner; +import org.intellij.lang.annotations.Language; +import org.junit.jupiter.api.Test; + +import java.nio.file.Files; +import java.nio.file.Path; + +import static io.trino.SystemSessionProperties.IDLE_WRITER_MIN_DATA_SIZE_THRESHOLD; +import static io.trino.SystemSessionProperties.SCALE_WRITERS; +import static io.trino.SystemSessionProperties.TASK_MAX_WRITER_COUNT; +import static io.trino.SystemSessionProperties.TASK_MIN_WRITER_COUNT; +import static io.trino.SystemSessionProperties.TASK_SCALE_WRITERS_ENABLED; +import static io.trino.plugin.deltalake.DeltaLakeQueryRunner.DELTA_CATALOG; +import static io.trino.testing.TestingNames.randomNameSuffix; +import static org.assertj.core.api.Assertions.assertThat; + +public class TestCloseIdleWriters + extends AbstractTestQueryFramework +{ + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + Path metastoreDirectory = Files.createTempDirectory(DELTA_CATALOG); + metastoreDirectory.toFile().deleteOnExit(); + DistributedQueryRunner queryRunner = DeltaLakeQueryRunner.builder() + .setCatalogName(DELTA_CATALOG) + .setNodeCount(1) + // Set the target max file size to 100GB so that we don't close writers due to file size in append + // page. + .setDeltaProperties(ImmutableMap.of( + "hive.metastore", "file", + "hive.metastore.catalog.dir", metastoreDirectory.toUri().toString(), + "delta.target-max-file-size", "100GB", + "delta.idle-writer-min-file-size", "0.1MB")) + .build(); + queryRunner.execute("CREATE SCHEMA IF NOT EXISTS tpch"); + return queryRunner; + } + + @Test + public void testCloseIdleWriters() + { + String tableName = "task_close_idle_writers_" + randomNameSuffix(); + try { + // Create a table with two partitions (0 and 1). Using the order by trick we will write the partitions in + // this order 0, 1, and then again 0. This way we are sure that during partition 1 write there will + // be an idle writer for partition 0. Additionally, during second partition 0 write, there will be an idle + // writer for partition 1. + @Language("SQL") String createTableSql = """ + CREATE TABLE %s WITH (partitioned_by = ARRAY['shipmodeVal']) + AS SELECT orderkey, partkey, suppkey, linenumber, quantity, extendedprice, + discount, tax, returnflag, linestatus, commitdate, receiptdate, shipinstruct, + comment, shipdate, + CASE + WHEN shipmode IN ('AIR', 'FOB', 'SHIP', 'TRUCK') THEN 0 + WHEN shipmode IN ('MAIL', 'RAIL', 'REG AIR') THEN 1 + ELSE 2 + END AS shipmodeVal + FROM tpch.tiny.lineitem + ORDER BY shipmode + LIMIT 60174 + """.formatted(tableName); + + // Disable all kind of scaling and set idle writer threshold to 10MB + assertUpdate( + Session.builder(getSession()) + .setSystemProperty(SCALE_WRITERS, "false") + .setSystemProperty(TASK_SCALE_WRITERS_ENABLED, "false") + .setSystemProperty(TASK_MAX_WRITER_COUNT, "1") + .setSystemProperty(TASK_MIN_WRITER_COUNT, "1") + .setSystemProperty(IDLE_WRITER_MIN_DATA_SIZE_THRESHOLD, "0.1MB") + .build(), + createTableSql, + 60174); + long files = (long) computeScalar("SELECT count(DISTINCT \"$path\") FROM " + tableName); + // There should more than 2 files since we triggered close idle writers. + assertThat(files).isGreaterThan(2); + } + finally { + assertUpdate("DROP TABLE IF EXISTS " + tableName); + } + } +} diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeConfig.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeConfig.java index dc42d2aba812c..10213ebb2765a 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeConfig.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeConfig.java @@ -27,6 +27,7 @@ import static io.airlift.configuration.testing.ConfigAssertions.assertRecordedDefaults; import static io.airlift.configuration.testing.ConfigAssertions.recordDefaults; import static io.airlift.units.DataSize.Unit.GIGABYTE; +import static io.airlift.units.DataSize.Unit.MEGABYTE; import static io.trino.plugin.hive.util.TestHiveUtil.nonDefaultTimeZone; import static java.util.concurrent.TimeUnit.DAYS; import static java.util.concurrent.TimeUnit.HOURS; @@ -66,6 +67,7 @@ public void testDefaults() .setParquetTimeZone(TimeZone.getDefault().getID()) .setPerTransactionMetastoreCacheMaximumSize(1000) .setTargetMaxFileSize(DataSize.of(1, GIGABYTE)) + .setIdleWriterMinFileSize(DataSize.of(16, MEGABYTE)) .setUniqueTableLocation(true) .setRegisterTableProcedureEnabled(false) .setProjectionPushdownEnabled(true) @@ -103,6 +105,7 @@ public void testExplicitPropertyMappings() .put("delta.delete-schema-locations-fallback", "true") .put("delta.parquet.time-zone", nonDefaultTimeZone().getID()) .put("delta.target-max-file-size", "2 GB") + .put("delta.idle-writer-min-file-size", "1MB") .put("delta.unique-table-location", "false") .put("delta.register-table-procedure.enabled", "true") .put("delta.projection-pushdown-enabled", "false") @@ -137,6 +140,7 @@ public void testExplicitPropertyMappings() .setParquetTimeZone(nonDefaultTimeZone().getID()) .setPerTransactionMetastoreCacheMaximumSize(500) .setTargetMaxFileSize(DataSize.of(2, GIGABYTE)) + .setIdleWriterMinFileSize(DataSize.of(1, MEGABYTE)) .setUniqueTableLocation(false) .setRegisterTableProcedureEnabled(true) .setProjectionPushdownEnabled(false) diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveConfig.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveConfig.java index a67a4ca9d6ae3..6a7a1817f9a0f 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveConfig.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveConfig.java @@ -167,6 +167,7 @@ public class HiveConfig private Optional hudiCatalogName = Optional.empty(); private DataSize targetMaxFileSize = DataSize.of(1, GIGABYTE); + private DataSize idleWriterMinFileSize = DataSize.of(16, MEGABYTE); private boolean sizeBasedSplitWeightsEnabled = true; private double minimumAssignedSplitWeight = 0.05; @@ -269,6 +270,19 @@ public HiveConfig setTargetMaxFileSize(DataSize targetMaxFileSize) return this; } + public DataSize getIdleWriterMinFileSize() + { + return idleWriterMinFileSize; + } + + @Config("hive.idle-writer-min-file-size") + @ConfigDescription("Minimum data written by a single partition writer before it can be consider as 'idle' and could be closed by the engine") + public HiveConfig setIdleWriterMinFileSize(DataSize idleWriterMinFileSize) + { + this.idleWriterMinFileSize = idleWriterMinFileSize; + return this; + } + public boolean isForceLocalScheduling() { return forceLocalScheduling; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePageSink.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePageSink.java index c703ea951feb9..f1e688fb71430 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePageSink.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePageSink.java @@ -20,6 +20,7 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import io.airlift.json.JsonCodec; +import io.airlift.log.Logger; import io.airlift.slice.Slice; import io.trino.plugin.hive.util.HiveBucketing.BucketingVersion; import io.trino.spi.Page; @@ -64,6 +65,7 @@ public class HivePageSink implements ConnectorPageSink, ConnectorMergeSink { + private static final Logger LOG = Logger.get(HivePageSink.class); private static final int MAX_PAGE_POSITIONS = 4096; private final HiveWriterFactory writerFactory; @@ -85,9 +87,11 @@ public class HivePageSink private final List writers = new ArrayList<>(); private final long targetMaxFileSize; + private final long idleWriterMinFileSize; private final List closedWriterRollbackActions = new ArrayList<>(); private final List partitionUpdates = new ArrayList<>(); private final List> verificationTasks = new ArrayList<>(); + private final List activeWriters = new ArrayList<>(); private final boolean isMergeSink; private long writtenBytes; @@ -161,6 +165,7 @@ public HivePageSink( } this.targetMaxFileSize = HiveSessionProperties.getTargetMaxFileSize(session).toBytes(); + this.idleWriterMinFileSize = HiveSessionProperties.getIdleWriterMinFileSize(session).toBytes(); } @Override @@ -191,6 +196,9 @@ private ListenableFuture> doMergeSinkFinish() { ImmutableList.Builder resultSlices = ImmutableList.builder(); for (HiveWriter writer : writers) { + if (writer == null) { + continue; + } writer.commit(); MergeFileWriter mergeFileWriter = (MergeFileWriter) writer.getFileWriter(); PartitionUpdateAndMergeResults results = mergeFileWriter.getPartitionUpdateAndMergeResults(writer.getPartitionUpdate()); @@ -198,6 +206,7 @@ private ListenableFuture> doMergeSinkFinish() } List result = resultSlices.build(); writtenBytes = writers.stream() + .filter(Objects::nonNull) .mapToLong(HiveWriter::getWrittenBytes) .sum(); return Futures.immediateFuture(result); @@ -308,6 +317,7 @@ private void writePage(Page page) } HiveWriter writer = writers.get(index); + verify(writer != null, "Expected writer at index %s", index); long currentWritten = writer.getWrittenBytes(); long currentMemory = writer.getMemoryUsage(); @@ -316,12 +326,17 @@ private void writePage(Page page) writtenBytes += (writer.getWrittenBytes() - currentWritten); memoryUsage += (writer.getMemoryUsage() - currentMemory); + // Mark this writer as active (i.e. not idle) + activeWriters.set(index, true); } } private void closeWriter(int writerIndex) { HiveWriter writer = writers.get(writerIndex); + if (writer == null) { + return; + } long currentWritten = writer.getWrittenBytes(); long currentMemory = writer.getMemoryUsage(); @@ -338,6 +353,26 @@ private void closeWriter(int writerIndex) partitionUpdates.add(wrappedBuffer(partitionUpdateCodec.toJsonBytes(partitionUpdate))); } + @Override + public void closeIdleWriters() + { + // For transactional tables we don't want to split output files because there is an explicit or implicit bucketing + // and file names have no random component (e.g. bucket_00000) + if (bucketFunction != null || isTransactional) { + return; + } + + for (int writerIndex = 0; writerIndex < writers.size(); writerIndex++) { + HiveWriter writer = writers.get(writerIndex); + if (activeWriters.get(writerIndex) || writer == null || writer.getWrittenBytes() <= idleWriterMinFileSize) { + activeWriters.set(writerIndex, false); + continue; + } + LOG.debug("Closing writer %s with %s bytes written", writerIndex, writer.getWrittenBytes()); + closeWriter(writerIndex); + } + } + private int[] getWriterIndexes(Page page) { Page partitionColumns = extractColumns(page, partitionColumnsInputIndex); @@ -350,6 +385,7 @@ private int[] getWriterIndexes(Page page) // expand writers list to new size while (writers.size() <= pagePartitioner.getMaxIndex()) { writers.add(null); + activeWriters.add(false); } // create missing writers @@ -378,7 +414,6 @@ private int[] getWriterIndexes(Page page) memoryUsage += writer.getMemoryUsage(); } verify(writers.size() == pagePartitioner.getMaxIndex() + 1); - verify(!writers.contains(null)); return writerIndexes; } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSessionProperties.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSessionProperties.java index fbeb33a40a5c3..c0a71c1fe9957 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSessionProperties.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSessionProperties.java @@ -63,6 +63,7 @@ public final class HiveSessionProperties private static final String BUCKET_EXECUTION_ENABLED = "bucket_execution_enabled"; private static final String VALIDATE_BUCKETING = "validate_bucketing"; private static final String TARGET_MAX_FILE_SIZE = "target_max_file_size"; + private static final String IDLE_WRITER_MIN_FILE_SIZE = "idle_writer_min_file_size"; private static final String PARALLEL_PARTITIONED_BUCKETED_WRITES = "parallel_partitioned_bucketed_writes"; private static final String FORCE_LOCAL_SCHEDULING = "force_local_scheduling"; private static final String INSERT_EXISTING_PARTITIONS_BEHAVIOR = "insert_existing_partitions_behavior"; @@ -169,6 +170,11 @@ public HiveSessionProperties( "Target maximum size of written files; the actual size may be larger", hiveConfig.getTargetMaxFileSize(), false), + dataSizeProperty( + IDLE_WRITER_MIN_FILE_SIZE, + "Minimum data written by a single partition writer before it can be consider as 'idle' and could be closed by the engine", + hiveConfig.getIdleWriterMinFileSize(), + false), booleanProperty( PARALLEL_PARTITIONED_BUCKETED_WRITES, "Improve parallelism of partitioned and bucketed table writes", @@ -555,6 +561,11 @@ public static DataSize getTargetMaxFileSize(ConnectorSession session) return session.getProperty(TARGET_MAX_FILE_SIZE, DataSize.class); } + public static DataSize getIdleWriterMinFileSize(ConnectorSession session) + { + return session.getProperty(IDLE_WRITER_MIN_FILE_SIZE, DataSize.class); + } + public static boolean isParallelPartitionedBucketedWrites(ConnectorSession session) { return session.getProperty(PARALLEL_PARTITIONED_BUCKETED_WRITES, Boolean.class); diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestCloseIdleWriters.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestCloseIdleWriters.java new file mode 100644 index 0000000000000..60c65e468e87c --- /dev/null +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestCloseIdleWriters.java @@ -0,0 +1,91 @@ +/* + * 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.hive; + +import com.google.common.collect.ImmutableMap; +import io.trino.Session; +import io.trino.testing.AbstractTestQueryFramework; +import io.trino.testing.QueryRunner; +import org.intellij.lang.annotations.Language; +import org.junit.jupiter.api.Test; + +import static io.trino.SystemSessionProperties.IDLE_WRITER_MIN_DATA_SIZE_THRESHOLD; +import static io.trino.SystemSessionProperties.SCALE_WRITERS; +import static io.trino.SystemSessionProperties.TASK_MAX_WRITER_COUNT; +import static io.trino.SystemSessionProperties.TASK_MIN_WRITER_COUNT; +import static io.trino.SystemSessionProperties.TASK_SCALE_WRITERS_ENABLED; +import static io.trino.testing.TestingNames.randomNameSuffix; +import static org.assertj.core.api.Assertions.assertThat; + +public class TestCloseIdleWriters + extends AbstractTestQueryFramework +{ + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + return HiveQueryRunner.builder() + .setNodeCount(1) + // Set the target max file size to 100GB so that we don't close writers due to file size in append + // page. + .setHiveProperties(ImmutableMap.of( + "hive.target-max-file-size", "100GB", + "hive.idle-writer-min-file-size", "0.1MB")) + .build(); + } + + @Test + public void testCloseIdleWriters() + { + String tableName = "task_close_idle_writers_" + randomNameSuffix(); + try { + // Create a table with two partitions (0 and 1). Using the order by trick we will write the partitions in + // this order 0, 1, and then again 0. This way we are sure that during partition 1 write there will + // be an idle writer for partition 0. Additionally, during second partition 0 write, there will be an idle + // writer for partition 1. + @Language("SQL") String createTableSql = """ + CREATE TABLE %s WITH (format = 'ORC', partitioned_by = ARRAY['shipmodeVal']) + AS SELECT orderkey, partkey, suppkey, linenumber, quantity, extendedprice, + discount, tax, returnflag, linestatus, commitdate, receiptdate, shipinstruct, + comment, shipdate, + CASE + WHEN shipmode IN ('AIR', 'FOB', 'SHIP', 'TRUCK') THEN 0 + WHEN shipmode IN ('MAIL', 'RAIL', 'REG AIR') THEN 1 + ELSE 2 + END AS shipmodeVal + FROM tpch.tiny.lineitem + ORDER BY shipmode + LIMIT 60174 + """.formatted(tableName); + + // Disable all kind of scaling and set idle writer threshold to 10MB + assertUpdate( + Session.builder(getSession()) + .setSystemProperty(SCALE_WRITERS, "false") + .setSystemProperty(TASK_SCALE_WRITERS_ENABLED, "false") + .setSystemProperty(TASK_MAX_WRITER_COUNT, "1") + .setSystemProperty(TASK_MIN_WRITER_COUNT, "1") + .setSystemProperty(IDLE_WRITER_MIN_DATA_SIZE_THRESHOLD, "0.1MB") + .build(), + createTableSql, + 60174); + long files = (long) computeScalar("SELECT count(DISTINCT \"$path\") FROM " + tableName); + // There should more than 2 files since we triggered close idle writers. + assertThat(files).isGreaterThan(2); + } + finally { + assertUpdate("DROP TABLE IF EXISTS " + tableName); + } + } +} diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveConfig.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveConfig.java index 7abc47c55b61d..63cb594b7bae1 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveConfig.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveConfig.java @@ -55,6 +55,7 @@ public void testDefaults() .setMaxSplitsPerSecond(null) .setDomainCompactionThreshold(1000) .setTargetMaxFileSize(DataSize.of(1, GIGABYTE)) + .setIdleWriterMinFileSize(DataSize.of(16, MEGABYTE)) .setForceLocalScheduling(false) .setMaxConcurrentFileSystemOperations(20) .setMaxConcurrentMetastoreDrops(20) @@ -138,6 +139,7 @@ public void testExplicitPropertyMappings() .put("hive.max-splits-per-second", "1") .put("hive.domain-compaction-threshold", "42") .put("hive.target-max-file-size", "72MB") + .put("hive.idle-writer-min-file-size", "1MB") .put("hive.recursive-directories", "true") .put("hive.ignore-absent-partitions", "true") .put("hive.storage-format", "SEQUENCEFILE") @@ -218,6 +220,7 @@ public void testExplicitPropertyMappings() .setMaxSplitsPerSecond(1) .setDomainCompactionThreshold(42) .setTargetMaxFileSize(DataSize.of(72, Unit.MEGABYTE)) + .setIdleWriterMinFileSize(DataSize.of(1, MEGABYTE)) .setForceLocalScheduling(true) .setMaxConcurrentFileSystemOperations(100) .setMaxConcurrentMetastoreDrops(100) diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHivePageSink.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHivePageSink.java index 8eb1f26cf1618..a319cbfc4b301 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHivePageSink.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHivePageSink.java @@ -18,6 +18,7 @@ import com.google.common.collect.ImmutableMap; import io.airlift.json.JsonCodec; import io.airlift.slice.Slices; +import io.airlift.units.DataSize; import io.trino.filesystem.FileEntry; import io.trino.filesystem.FileIterator; import io.trino.filesystem.Location; @@ -56,16 +57,21 @@ import java.util.Map; import java.util.Optional; import java.util.OptionalInt; +import java.util.function.Function; import java.util.stream.Stream; import static com.google.common.collect.ImmutableList.toImmutableList; import static io.airlift.concurrent.MoreFutures.getFutureValue; import static io.airlift.testing.Assertions.assertGreaterThan; +import static io.airlift.units.DataSize.Unit.BYTE; +import static io.airlift.units.DataSize.Unit.MEGABYTE; import static io.trino.hive.thrift.metastore.hive_metastoreConstants.FILE_INPUT_FORMAT; +import static io.trino.plugin.hive.HiveColumnHandle.ColumnType.PARTITION_KEY; import static io.trino.plugin.hive.HiveColumnHandle.ColumnType.REGULAR; import static io.trino.plugin.hive.HiveColumnHandle.createBaseColumn; import static io.trino.plugin.hive.HiveCompressionOption.LZ4; import static io.trino.plugin.hive.HiveCompressionOption.NONE; +import static io.trino.plugin.hive.HiveStorageFormat.PARQUET; import static io.trino.plugin.hive.HiveTestUtils.HDFS_FILE_SYSTEM_FACTORY; import static io.trino.plugin.hive.HiveTestUtils.PAGE_SORTER; import static io.trino.plugin.hive.HiveTestUtils.getDefaultHiveFileWriterFactories; @@ -85,6 +91,7 @@ import static io.trino.spi.type.VarcharType.VARCHAR; import static io.trino.spi.type.VarcharType.createUnboundedVarcharType; import static io.trino.testing.TestingPageSinkId.TESTING_PAGE_SINK_ID; +import static io.trino.tpch.LineItemColumn.SHIP_MODE; import static io.trino.type.InternalTypeManager.TESTING_TYPE_MANAGER; import static java.lang.Math.round; import static java.lang.String.format; @@ -125,7 +132,7 @@ void testAllFormats() if (codec == NONE) { continue; } - if ((format == HiveStorageFormat.PARQUET) && (codec == LZ4)) { + if ((format == PARQUET) && (codec == LZ4)) { // TODO (https://github.com/trinodb/trino/issues/9142) LZ4 is not supported with native Parquet writer continue; } @@ -145,6 +152,72 @@ void testAllFormats() } } + @Test + public void testCloseIdleWritersWhenIdleWriterMinFileSizeLimitIsReached() + throws IOException + { + testCloseIdleWriters(DataSize.of(1, BYTE), 2, 1); + } + + @Test + public void testCloseIdleWritersWhenIdleWriterMinFileSizeLimitIsNotReached() + throws IOException + { + testCloseIdleWriters(DataSize.of(100, MEGABYTE), 1, 1); + } + + private void testCloseIdleWriters(DataSize idleWritersMinFileSize, int expectedTruckFiles, int expectedShipFiles) + throws IOException + { + HiveConfig config = new HiveConfig() + .setIdleWriterMinFileSize(idleWritersMinFileSize) + .setHiveStorageFormat(PARQUET) + .setHiveCompressionCodec(NONE); + SortingFileWriterConfig sortingFileWriterConfig = new SortingFileWriterConfig(); + + TrinoFileSystemFactory fileSystemFactory = new MemoryFileSystemFactory(); + HiveMetastore metastore = createTestingFileHiveMetastore(fileSystemFactory, Location.of("memory:///metastore")); + + HiveTransactionHandle transaction = new HiveTransactionHandle(false); + HiveWriterStats stats = new HiveWriterStats(); + List columnHandles = getPartitionedColumnHandles(SHIP_MODE.getColumnName()); + Location location = makeFileName(config); + ConnectorPageSink pageSink = createPageSink(fileSystemFactory, transaction, config, sortingFileWriterConfig, metastore, location, stats, columnHandles); + Page truckPage = createPage(lineItem -> lineItem.getShipMode().equals("TRUCK")); + Page shipPage = createPage(lineItem -> lineItem.getShipMode().equals("SHIP")); + + pageSink.appendPage(truckPage); + pageSink.appendPage(shipPage); + // This call will mark the truck and ship partition as idle. + pageSink.closeIdleWriters(); + + // This call will mark the ship partition as non-idle. + pageSink.appendPage(shipPage); + // This call will close the truck partition if idleWritersMinFileSize limit is reached since + // it is still idle. + pageSink.closeIdleWriters(); + + pageSink.appendPage(truckPage); + pageSink.appendPage(shipPage); + + getFutureValue(pageSink.finish()); + FileIterator fileIterator = fileSystemFactory.create(ConnectorIdentity.ofUser("test")).listFiles(location); + + int truckFileCount = 0; + int shipFileCount = 0; + while (fileIterator.hasNext()) { + FileEntry file = fileIterator.next(); + if (file.location().toString().contains("TRUCK")) { + truckFileCount++; + } + else if (file.location().toString().contains("SHIP")) { + shipFileCount++; + } + } + assertThat(truckFileCount).isEqualTo(expectedTruckFiles); + assertThat(shipFileCount).isEqualTo(expectedShipFiles); + } + private static boolean isSupportedCodec(HiveStorageFormat storageFormat, HiveCompressionOption compressionOption) { if (storageFormat == HiveStorageFormat.AVRO && compressionOption == LZ4) { @@ -163,17 +236,52 @@ private static long writeTestFile(TrinoFileSystemFactory fileSystemFactory, Hive { HiveTransactionHandle transaction = new HiveTransactionHandle(false); HiveWriterStats stats = new HiveWriterStats(); - ConnectorPageSink pageSink = createPageSink(fileSystemFactory, transaction, config, sortingFileWriterConfig, metastore, location, stats); + ConnectorPageSink pageSink = createPageSink(fileSystemFactory, transaction, config, sortingFileWriterConfig, metastore, location, stats, getColumnHandles()); List columns = getTestColumns(); List columnTypes = columns.stream() .map(LineItemColumn::getType) .map(TestHivePageSink::getType) .map(hiveType -> TESTING_TYPE_MANAGER.getType(hiveType.getTypeSignature())) .collect(toList()); + Page page = createPage(lineItem -> true); + pageSink.appendPage(page); + getFutureValue(pageSink.finish()); + + FileIterator fileIterator = fileSystemFactory.create(ConnectorIdentity.ofUser("test")).listFiles(location); + FileEntry fileEntry = fileIterator.next(); + assertThat(fileIterator.hasNext()).isFalse(); + List pages = new ArrayList<>(); + try (ConnectorPageSource pageSource = createPageSource(fileSystemFactory, transaction, config, fileEntry.location())) { + while (!pageSource.isFinished()) { + Page nextPage = pageSource.getNextPage(); + if (nextPage != null) { + pages.add(nextPage.getLoadedPage()); + } + } + } + + MaterializedResult expectedResults = toMaterializedResult(getHiveSession(config), columnTypes, ImmutableList.of(page)); + MaterializedResult results = toMaterializedResult(getHiveSession(config), columnTypes, pages); + assertThat(results).containsExactlyElementsOf(expectedResults); + assertThat(round(stats.getInputPageSizeInBytes().getAllTime().getMax())).isEqualTo(page.getRetainedSizeInBytes()); + return fileEntry.length(); + } + + private static Page createPage(Function filter) + { + List columns = getTestColumns(); + List columnTypes = columns.stream() + .map(LineItemColumn::getType) + .map(TestHivePageSink::getType) + .map(hiveType -> TESTING_TYPE_MANAGER.getType(hiveType.getTypeSignature())) + .collect(toList()); PageBuilder pageBuilder = new PageBuilder(columnTypes); int rows = 0; for (LineItem lineItem : new LineItemGenerator(0.01, 1, 1)) { + if (!filter.apply(lineItem)) { + continue; + } rows++; if (rows >= NUM_ROWS) { break; @@ -203,29 +311,7 @@ private static long writeTestFile(TrinoFileSystemFactory fileSystemFactory, Hive } } } - Page page = pageBuilder.build(); - pageSink.appendPage(page); - getFutureValue(pageSink.finish()); - - FileIterator fileIterator = fileSystemFactory.create(ConnectorIdentity.ofUser("test")).listFiles(location); - FileEntry fileEntry = fileIterator.next(); - assertThat(fileIterator.hasNext()).isFalse(); - - List pages = new ArrayList<>(); - try (ConnectorPageSource pageSource = createPageSource(fileSystemFactory, transaction, config, fileEntry.location())) { - while (!pageSource.isFinished()) { - Page nextPage = pageSource.getNextPage(); - if (nextPage != null) { - pages.add(nextPage.getLoadedPage()); - } - } - } - - MaterializedResult expectedResults = toMaterializedResult(getHiveSession(config), columnTypes, ImmutableList.of(page)); - MaterializedResult results = toMaterializedResult(getHiveSession(config), columnTypes, pages); - assertThat(results).containsExactlyElementsOf(expectedResults); - assertThat(round(stats.getInputPageSizeInBytes().getAllTime().getMax())).isEqualTo(page.getRetainedSizeInBytes()); - return fileEntry.length(); + return pageBuilder.build(); } static MaterializedResult toMaterializedResult(ConnectorSession session, List types, List pages) @@ -274,13 +360,21 @@ private static ConnectorPageSource createPageSource(TrinoFileSystemFactory fileS return provider.createPageSource(transaction, getHiveSession(config), split, table, ImmutableList.copyOf(getColumnHandles()), DynamicFilter.EMPTY); } - private static ConnectorPageSink createPageSink(TrinoFileSystemFactory fileSystemFactory, HiveTransactionHandle transaction, HiveConfig config, SortingFileWriterConfig sortingFileWriterConfig, HiveMetastore metastore, Location location, HiveWriterStats stats) + private static ConnectorPageSink createPageSink( + TrinoFileSystemFactory fileSystemFactory, + HiveTransactionHandle transaction, + HiveConfig config, + SortingFileWriterConfig sortingFileWriterConfig, + HiveMetastore metastore, + Location location, + HiveWriterStats stats, + List columnHandles) { LocationHandle locationHandle = new LocationHandle(location, location, DIRECT_TO_TARGET_NEW_DIRECTORY); HiveOutputTableHandle handle = new HiveOutputTableHandle( SCHEMA_NAME, TABLE_NAME, - getColumnHandles(), + columnHandles, new HivePageSinkMetadata(new SchemaTableName(SCHEMA_NAME, TABLE_NAME), metastore.getTable(SCHEMA_NAME, TABLE_NAME), ImmutableMap.of()), locationHandle, config.getHiveStorageFormat(), @@ -323,6 +417,23 @@ private static List getColumnHandles() return handles.build(); } + private static List getPartitionedColumnHandles(String partitionColumn) + { + ImmutableList.Builder handles = ImmutableList.builder(); + List columns = getTestColumns(); + for (int i = 0; i < columns.size(); i++) { + LineItemColumn column = columns.get(i); + Type type = getType(column.getType()); + if (column.getColumnName().equals(partitionColumn)) { + handles.add(createBaseColumn(column.getColumnName(), i, HiveType.toHiveType(type), type, PARTITION_KEY, Optional.empty())); + } + else { + handles.add(createBaseColumn(column.getColumnName(), i, HiveType.toHiveType(type), type, REGULAR, Optional.empty())); + } + } + return handles.build(); + } + private static List getTestColumns() { return Stream.of(LineItemColumn.values()) diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergConfig.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergConfig.java index 5b3e0d7f8df5c..67e4b9675c2a6 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergConfig.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergConfig.java @@ -30,6 +30,7 @@ import java.util.Optional; import static io.airlift.units.DataSize.Unit.GIGABYTE; +import static io.airlift.units.DataSize.Unit.MEGABYTE; import static io.trino.plugin.hive.HiveCompressionCodec.ZSTD; import static io.trino.plugin.iceberg.CatalogType.HIVE_METASTORE; import static io.trino.plugin.iceberg.IcebergFileFormat.PARQUET; @@ -67,6 +68,7 @@ public class IcebergConfig private Duration expireSnapshotsMinRetention = new Duration(7, DAYS); private Duration removeOrphanFilesMinRetention = new Duration(7, DAYS); private DataSize targetMaxFileSize = DataSize.of(1, GIGABYTE); + private DataSize idleWriterMinFileSize = DataSize.of(16, MEGABYTE); // This is meant to protect users who are misusing schema locations (by // putting schemas in locations with extraneous files), so default to false // to avoid deleting those files if Trino is unable to check. @@ -315,6 +317,20 @@ public IcebergConfig setTargetMaxFileSize(DataSize targetMaxFileSize) return this; } + @NotNull + public DataSize getIdleWriterMinFileSize() + { + return idleWriterMinFileSize; + } + + @Config("iceberg.idle-writer-min-file-size") + @ConfigDescription("Minimum data written by a single partition writer before it can be consider as 'idle' and could be closed by the engine") + public IcebergConfig setIdleWriterMinFileSize(DataSize idleWriterMinFileSize) + { + this.idleWriterMinFileSize = idleWriterMinFileSize; + return this; + } + public boolean isDeleteSchemaLocationsFallback() { return this.deleteSchemaLocationsFallback; diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSink.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSink.java index b54d8f25fa9bd..284385d0e49b4 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSink.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSink.java @@ -16,6 +16,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Streams; import io.airlift.json.JsonCodec; +import io.airlift.log.Logger; import io.airlift.slice.Slice; import io.airlift.units.DataSize; import io.trino.filesystem.Location; @@ -91,6 +92,8 @@ public class IcebergPageSink implements ConnectorPageSink { + private static final Logger LOG = Logger.get(IcebergPageSink.class); + private static final int MAX_PAGE_POSITIONS = 4096; private final int maxOpenWriters; @@ -105,6 +108,7 @@ public class IcebergPageSink private final MetricsConfig metricsConfig; private final PagePartitioner pagePartitioner; private final long targetMaxFileSize; + private final long idleWriterMinFileSize; private final Map storageProperties; private final List sortOrder; private final boolean sortedWritingEnabled; @@ -120,6 +124,7 @@ public class IcebergPageSink private final List writers = new ArrayList<>(); private final List closedWriterRollbackActions = new ArrayList<>(); private final Collection commitTasks = new ArrayList<>(); + private final List activeWriters = new ArrayList<>(); private long writtenBytes; private long memoryUsage; @@ -157,6 +162,7 @@ public IcebergPageSink( this.maxOpenWriters = maxOpenWriters; this.pagePartitioner = new PagePartitioner(pageIndexerFactory, toPartitionColumns(inputColumns, partitionSpec)); this.targetMaxFileSize = IcebergSessionProperties.getTargetMaxFileSize(session); + this.idleWriterMinFileSize = IcebergSessionProperties.getIdleWriterMinFileSize(session); this.storageProperties = requireNonNull(storageProperties, "storageProperties is null"); this.sortOrder = requireNonNull(sortOrder, "sortOrder is null"); this.sortedWritingEnabled = isSortedWritingEnabled(session); @@ -300,7 +306,9 @@ private void writePage(Page page) pageForWriter = pageForWriter.getPositions(positions, 0, positions.length); } - IcebergFileWriter writer = writers.get(index).getWriter(); + WriteContext writeContext = writers.get(index); + verify(writeContext != null, "Expected writer at index %s", index); + IcebergFileWriter writer = writeContext.getWriter(); long currentWritten = writer.getWrittenBytes(); long currentMemory = writer.getMemoryUsage(); @@ -309,6 +317,8 @@ private void writePage(Page page) writtenBytes += (writer.getWrittenBytes() - currentWritten); memoryUsage += (writer.getMemoryUsage() - currentMemory); + // Mark this writer as active (i.e. not idle) + activeWriters.set(index, true); } } @@ -323,6 +333,7 @@ private int[] getWriterIndexes(Page page) // expand writers list to new size while (writers.size() <= pagePartitioner.getMaxIndex()) { writers.add(null); + activeWriters.add(false); } // create missing writers @@ -369,14 +380,30 @@ private int[] getWriterIndexes(Page page) memoryUsage += writer.getWriter().getMemoryUsage(); } verify(writers.size() == pagePartitioner.getMaxIndex() + 1); - verify(!writers.contains(null)); return writerIndexes; } + @Override + public void closeIdleWriters() + { + for (int writerIndex = 0; writerIndex < writers.size(); writerIndex++) { + WriteContext writeContext = writers.get(writerIndex); + if (activeWriters.get(writerIndex) || writeContext == null || writeContext.getWriter().getWrittenBytes() <= idleWriterMinFileSize) { + activeWriters.set(writerIndex, false); + continue; + } + LOG.debug("Closing writer %s with %s bytes written", writerIndex, writeContext.getWriter().getWrittenBytes()); + closeWriter(writerIndex); + } + } + private void closeWriter(int writerIndex) { WriteContext writeContext = writers.get(writerIndex); + if (writeContext == null) { + return; + } IcebergFileWriter writer = writeContext.getWriter(); long currentWritten = writer.getWrittenBytes(); diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSessionProperties.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSessionProperties.java index 42868452dc522..613eea702275c 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSessionProperties.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSessionProperties.java @@ -84,6 +84,7 @@ public final class IcebergSessionProperties public static final String EXTENDED_STATISTICS_ENABLED = "extended_statistics_enabled"; private static final String PROJECTION_PUSHDOWN_ENABLED = "projection_pushdown_enabled"; private static final String TARGET_MAX_FILE_SIZE = "target_max_file_size"; + private static final String IDLE_WRITER_MIN_FILE_SIZE = "idle_writer_min_file_size"; public static final String COLLECT_EXTENDED_STATISTICS_ON_WRITE = "collect_extended_statistics_on_write"; private static final String HIVE_CATALOG_NAME = "hive_catalog_name"; private static final String MINIMUM_ASSIGNED_SPLIT_WEIGHT = "minimum_assigned_split_weight"; @@ -279,6 +280,11 @@ public IcebergSessionProperties( "Target maximum size of written files; the actual size may be larger", icebergConfig.getTargetMaxFileSize(), false)) + .add(dataSizeProperty( + IDLE_WRITER_MIN_FILE_SIZE, + "Minimum data written by a single partition writer before it can be consider as 'idle' and could be closed by the engine", + icebergConfig.getIdleWriterMinFileSize(), + false)) .add(booleanProperty( COLLECT_EXTENDED_STATISTICS_ON_WRITE, COLLECT_EXTENDED_STATISTICS_ON_WRITE_DESCRIPTION, @@ -492,6 +498,11 @@ public static long getTargetMaxFileSize(ConnectorSession session) return session.getProperty(TARGET_MAX_FILE_SIZE, DataSize.class).toBytes(); } + public static long getIdleWriterMinFileSize(ConnectorSession session) + { + return session.getProperty(IDLE_WRITER_MIN_FILE_SIZE, DataSize.class).toBytes(); + } + public static Optional getHiveCatalogName(ConnectorSession session) { return Optional.ofNullable(session.getProperty(HIVE_CATALOG_NAME, String.class)); diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestCloseIdleWriters.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestCloseIdleWriters.java new file mode 100644 index 0000000000000..869555a555094 --- /dev/null +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestCloseIdleWriters.java @@ -0,0 +1,91 @@ +/* + * 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.iceberg; + +import com.google.common.collect.ImmutableMap; +import io.trino.Session; +import io.trino.testing.AbstractTestQueryFramework; +import io.trino.testing.QueryRunner; +import org.intellij.lang.annotations.Language; +import org.junit.jupiter.api.Test; + +import static io.trino.SystemSessionProperties.IDLE_WRITER_MIN_DATA_SIZE_THRESHOLD; +import static io.trino.SystemSessionProperties.SCALE_WRITERS; +import static io.trino.SystemSessionProperties.TASK_MAX_WRITER_COUNT; +import static io.trino.SystemSessionProperties.TASK_MIN_WRITER_COUNT; +import static io.trino.SystemSessionProperties.TASK_SCALE_WRITERS_ENABLED; +import static io.trino.testing.TestingNames.randomNameSuffix; +import static org.assertj.core.api.Assertions.assertThat; + +public class TestCloseIdleWriters + extends AbstractTestQueryFramework +{ + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + return IcebergQueryRunner.builder() + .setNodeCount(1) + // Set the target max file size to 100GB so that we don't close writers due to file size in append + // page. + .setIcebergProperties(ImmutableMap.of( + "iceberg.target-max-file-size", "100GB", + "iceberg.idle-writer-min-file-size", "0.1MB")) + .build(); + } + + @Test + public void testCloseIdleWriters() + { + String tableName = "task_close_idle_writers_" + randomNameSuffix(); + try { + // Create a table with two partitions (0 and 1). Using the order by trick we will write the partitions in + // this order 0, 1, and then again 0. This way we are sure that during partition 1 write there will + // be an idle writer for partition 0. Additionally, during second partition 0 write, there will be an idle + // writer for partition 1. + @Language("SQL") String createTableSql = """ + CREATE TABLE %s WITH (format = 'ORC', partitioning = ARRAY['shipmodeVal']) + AS SELECT orderkey, partkey, suppkey, linenumber, quantity, extendedprice, + discount, tax, returnflag, linestatus, commitdate, receiptdate, shipinstruct, + comment, shipdate, + CASE + WHEN shipmode IN ('AIR', 'FOB', 'SHIP', 'TRUCK') THEN 0 + WHEN shipmode IN ('MAIL', 'RAIL', 'REG AIR') THEN 1 + ELSE 2 + END AS shipmodeVal + FROM tpch.tiny.lineitem + ORDER BY shipmode + LIMIT 60174 + """.formatted(tableName); + + // Disable all kind of scaling and set idle writer threshold to 5MB + assertUpdate( + Session.builder(getSession()) + .setSystemProperty(SCALE_WRITERS, "false") + .setSystemProperty(TASK_SCALE_WRITERS_ENABLED, "false") + .setSystemProperty(TASK_MAX_WRITER_COUNT, "1") + .setSystemProperty(TASK_MIN_WRITER_COUNT, "1") + .setSystemProperty(IDLE_WRITER_MIN_DATA_SIZE_THRESHOLD, "0.1MB") + .build(), + createTableSql, + 60174); + long files = (long) computeScalar("SELECT count(DISTINCT \"$path\") FROM " + tableName); + // There should more than 2 files since we triggered close idle writers. + assertThat(files).isGreaterThan(2); + } + finally { + assertUpdate("DROP TABLE IF EXISTS " + tableName); + } + } +} diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergConfig.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergConfig.java index 89a5d16e14f4b..4ca2417beba46 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergConfig.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergConfig.java @@ -60,6 +60,7 @@ public void testDefaults() .setRemoveOrphanFilesMinRetention(new Duration(7, DAYS)) .setDeleteSchemaLocationsFallback(false) .setTargetMaxFileSize(DataSize.of(1, GIGABYTE)) + .setIdleWriterMinFileSize(DataSize.of(16, MEGABYTE)) .setMinimumAssignedSplitWeight(0.05) .setHideMaterializedViewStorageTable(true) .setMaterializedViewsStorageSchema(null) @@ -89,6 +90,7 @@ public void testExplicitPropertyMappings() .put("iceberg.remove_orphan_files.min-retention", "14h") .put("iceberg.delete-schema-locations-fallback", "true") .put("iceberg.target-max-file-size", "1MB") + .put("iceberg.idle-writer-min-file-size", "1MB") .put("iceberg.minimum-assigned-split-weight", "0.01") .put("iceberg.materialized-views.hide-storage-table", "false") .put("iceberg.materialized-views.storage-schema", "mv_storage_schema") @@ -115,6 +117,7 @@ public void testExplicitPropertyMappings() .setRemoveOrphanFilesMinRetention(new Duration(14, HOURS)) .setDeleteSchemaLocationsFallback(true) .setTargetMaxFileSize(DataSize.of(1, MEGABYTE)) + .setIdleWriterMinFileSize(DataSize.of(1, MEGABYTE)) .setMinimumAssignedSplitWeight(0.01) .setHideMaterializedViewStorageTable(false) .setMaterializedViewsStorageSchema("mv_storage_schema") From f52ce042ea04a558936319c113242939a1d207b8 Mon Sep 17 00:00:00 2001 From: Martin Traverso Date: Wed, 13 Dec 2023 07:47:59 -0800 Subject: [PATCH 123/350] Fix incorrect LIKE for patterns with repeating substrings After the initial KMP mismatch, the next match was being skipped at position i == longest match. --- .../src/main/java/io/trino/likematcher/FjsMatcher.java | 2 +- .../src/test/java/io/trino/likematcher/TestLikeMatcher.java | 6 ++++++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/core/trino-main/src/main/java/io/trino/likematcher/FjsMatcher.java b/core/trino-main/src/main/java/io/trino/likematcher/FjsMatcher.java index 24fee50d8072b..365bff87b28a9 100644 --- a/core/trino-main/src/main/java/io/trino/likematcher/FjsMatcher.java +++ b/core/trino-main/src/main/java/io/trino/likematcher/FjsMatcher.java @@ -146,7 +146,7 @@ else if (matchEnd >= inputLimit - 1) { j = kmpShifts[j]; // Continue to match the whole pattern using KMP - while (j > 0) { + while (j >= 0) { int size = findLongestMatch(input, i, pattern, j, Math.min(inputLimit - i, pattern.length - j)); i += size; j += size; diff --git a/core/trino-main/src/test/java/io/trino/likematcher/TestLikeMatcher.java b/core/trino-main/src/test/java/io/trino/likematcher/TestLikeMatcher.java index fa0f9d6c38d29..398cf1daba604 100644 --- a/core/trino-main/src/test/java/io/trino/likematcher/TestLikeMatcher.java +++ b/core/trino-main/src/test/java/io/trino/likematcher/TestLikeMatcher.java @@ -96,6 +96,12 @@ public void test() assertFalse(match("%abaaa%", "ababaa")); + assertTrue(match("%paya%", "papaya")); + assertTrue(match("%paya%", "papapaya")); + assertTrue(match("%paya%", "papapapaya")); + assertTrue(match("%paya%", "papapapapaya")); + assertTrue(match("%paya%", "papapapapapaya")); + // utf-8 LikeMatcher singleOptimized = LikePattern.compile("_", Optional.empty(), true).getMatcher(); LikeMatcher multipleOptimized = LikePattern.compile("_a%b_", Optional.empty(), true).getMatcher(); // prefix and suffix with _a and b_ to avoid optimizations From dec4f6dc2868dc7c734c9f9c6b668149f283f246 Mon Sep 17 00:00:00 2001 From: Manfred Moser Date: Wed, 13 Dec 2023 16:18:18 -0800 Subject: [PATCH 124/350] Add docs for mongodb.allow-local-scheduling --- docs/src/main/sphinx/connector/mongodb.md | 10 ++++++++++ .../io/trino/plugin/mongodb/MongoClientConfig.java | 2 +- 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/docs/src/main/sphinx/connector/mongodb.md b/docs/src/main/sphinx/connector/mongodb.md index ff66ceaef6780..5edcf63b64512 100644 --- a/docs/src/main/sphinx/connector/mongodb.md +++ b/docs/src/main/sphinx/connector/mongodb.md @@ -59,6 +59,7 @@ The following configuration properties are available: | `mongodb.write-concern` | The write concern | | `mongodb.required-replica-set` | The required replica set name | | `mongodb.cursor-batch-size` | The number of elements to return in a batch | +| `mongodb.allow-local-scheduling` | Assign MongoDB splits to a specific worker | ### `mongodb.connection-url` @@ -203,6 +204,15 @@ Do not use a batch size of `1`. This property is optional; the default is `0`. +### `mongodb.allow-local-scheduling` + +Set the value of this property to `true` if Trino and MongoDB share the same +cluster, and specific MongoDB splits should be processed on the same worker and +MongoDB node. Note that a shared deployment is not recommended, and enabling +this property can lead to resource contention. + +This property is optional, and defaults to false. + (table-definition-label)= ## Table definition diff --git a/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/MongoClientConfig.java b/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/MongoClientConfig.java index eeca4862120b7..9d87b72ff47c2 100644 --- a/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/MongoClientConfig.java +++ b/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/MongoClientConfig.java @@ -259,7 +259,7 @@ public boolean isAllowLocalScheduling() } @Config("mongodb.allow-local-scheduling") - @ConfigDescription("Assign mongo splits to host if worker and mongo share the same cluster") + @ConfigDescription("Assign MongoDB splits to a specific host if worker and MongoDB share the same cluster") public MongoClientConfig setAllowLocalScheduling(boolean allowLocalScheduling) { this.allowLocalScheduling = allowLocalScheduling; From b4e65d08c509a98c8496bcaec39b636f8bc4b0a5 Mon Sep 17 00:00:00 2001 From: kasiafi <30203062+kasiafi@users.noreply.github.com> Date: Tue, 9 May 2023 20:50:49 +0200 Subject: [PATCH 125/350] Add support for JSON_TABLE --- .../io/trino/metadata/FunctionManager.java | 5 +- .../trino/metadata/GlobalFunctionCatalog.java | 21 + .../io/trino/metadata/MetadataManager.java | 5 +- .../trino/operator/TableFunctionOperator.java | 2 +- .../trino/operator/table/json/JsonTable.java | 215 +++++ .../operator/table/json/JsonTableColumn.java | 31 + .../table/json/JsonTableOrdinalityColumn.java | 19 + .../table/json/JsonTablePlanCross.java | 30 + .../table/json/JsonTablePlanLeaf.java | 31 + .../table/json/JsonTablePlanNode.java | 32 + .../table/json/JsonTablePlanSingle.java | 33 + .../table/json/JsonTablePlanUnion.java | 30 + .../table/json/JsonTableQueryColumn.java | 40 + .../table/json/JsonTableValueColumn.java | 36 + .../operator/table/json/execution/Column.java | 24 + .../json/execution/ExecutionPlanner.java | 159 ++++ .../table/json/execution/FragmentCross.java | 93 ++ .../table/json/execution/FragmentLeaf.java | 109 +++ .../table/json/execution/FragmentSingle.java | 156 ++++ .../table/json/execution/FragmentUnion.java | 96 ++ .../JsonTableProcessingFragment.java | 63 ++ .../json/execution/OrdinalityColumn.java | 40 + .../table/json/execution/QueryColumn.java | 63 ++ .../json/execution/SequenceEvaluator.java | 93 ++ .../table/json/execution/ValueColumn.java | 93 ++ .../java/io/trino/sql/analyzer/Analysis.java | 55 +- .../sql/analyzer/ExpressionAnalyzer.java | 224 ++++- .../trino/sql/analyzer/JsonPathAnalyzer.java | 10 +- .../trino/sql/analyzer/StatementAnalyzer.java | 294 +++++- .../io/trino/sql/planner/RelationPlanner.java | 461 ++++++++++ .../planner/ResolvedFunctionCallRewriter.java | 7 +- .../io/trino/sql/planner/TranslationMap.java | 70 +- .../io/trino/testing/LocalQueryRunner.java | 5 +- .../dispatcher/TestLocalDispatchQuery.java | 5 +- .../metadata/TestGlobalFunctionCatalog.java | 10 +- .../io/trino/sql/analyzer/TestAnalyzer.java | 539 ++++++++++- .../sql/planner/JsonTablePlanComparator.java | 125 +++ .../io/trino/sql/planner/TestJsonTable.java | 549 +++++++++++ .../sql/planner/TestingPlannerContext.java | 5 +- .../io/trino/sql/query/TestJsonTable.java | 867 ++++++++++++++++++ .../TestJsonPath2016TypeSerialization.java | 2 +- .../io/trino/sql/jsonpath/PathParser.java | 25 +- .../io/trino/sql/jsonpath/TestPathParser.java | 2 +- .../java/io/trino/spi/StandardErrorCode.java | 3 + 44 files changed, 4675 insertions(+), 102 deletions(-) create mode 100644 core/trino-main/src/main/java/io/trino/operator/table/json/JsonTable.java create mode 100644 core/trino-main/src/main/java/io/trino/operator/table/json/JsonTableColumn.java create mode 100644 core/trino-main/src/main/java/io/trino/operator/table/json/JsonTableOrdinalityColumn.java create mode 100644 core/trino-main/src/main/java/io/trino/operator/table/json/JsonTablePlanCross.java create mode 100644 core/trino-main/src/main/java/io/trino/operator/table/json/JsonTablePlanLeaf.java create mode 100644 core/trino-main/src/main/java/io/trino/operator/table/json/JsonTablePlanNode.java create mode 100644 core/trino-main/src/main/java/io/trino/operator/table/json/JsonTablePlanSingle.java create mode 100644 core/trino-main/src/main/java/io/trino/operator/table/json/JsonTablePlanUnion.java create mode 100644 core/trino-main/src/main/java/io/trino/operator/table/json/JsonTableQueryColumn.java create mode 100644 core/trino-main/src/main/java/io/trino/operator/table/json/JsonTableValueColumn.java create mode 100644 core/trino-main/src/main/java/io/trino/operator/table/json/execution/Column.java create mode 100644 core/trino-main/src/main/java/io/trino/operator/table/json/execution/ExecutionPlanner.java create mode 100644 core/trino-main/src/main/java/io/trino/operator/table/json/execution/FragmentCross.java create mode 100644 core/trino-main/src/main/java/io/trino/operator/table/json/execution/FragmentLeaf.java create mode 100644 core/trino-main/src/main/java/io/trino/operator/table/json/execution/FragmentSingle.java create mode 100644 core/trino-main/src/main/java/io/trino/operator/table/json/execution/FragmentUnion.java create mode 100644 core/trino-main/src/main/java/io/trino/operator/table/json/execution/JsonTableProcessingFragment.java create mode 100644 core/trino-main/src/main/java/io/trino/operator/table/json/execution/OrdinalityColumn.java create mode 100644 core/trino-main/src/main/java/io/trino/operator/table/json/execution/QueryColumn.java create mode 100644 core/trino-main/src/main/java/io/trino/operator/table/json/execution/SequenceEvaluator.java create mode 100644 core/trino-main/src/main/java/io/trino/operator/table/json/execution/ValueColumn.java create mode 100644 core/trino-main/src/test/java/io/trino/sql/planner/JsonTablePlanComparator.java create mode 100644 core/trino-main/src/test/java/io/trino/sql/planner/TestJsonTable.java create mode 100644 core/trino-main/src/test/java/io/trino/sql/query/TestJsonTable.java diff --git a/core/trino-main/src/main/java/io/trino/metadata/FunctionManager.java b/core/trino-main/src/main/java/io/trino/metadata/FunctionManager.java index 23e071d2ecc2c..9a7c208380ea8 100644 --- a/core/trino-main/src/main/java/io/trino/metadata/FunctionManager.java +++ b/core/trino-main/src/main/java/io/trino/metadata/FunctionManager.java @@ -324,7 +324,10 @@ private record FunctionKey(ResolvedFunction resolvedFunction, InvocationConventi public static FunctionManager createTestingFunctionManager() { TypeOperators typeOperators = new TypeOperators(); - GlobalFunctionCatalog functionCatalog = new GlobalFunctionCatalog(); + GlobalFunctionCatalog functionCatalog = new GlobalFunctionCatalog( + () -> { throw new UnsupportedOperationException(); }, + () -> { throw new UnsupportedOperationException(); }, + () -> { throw new UnsupportedOperationException(); }); functionCatalog.addFunctions(SystemFunctionBundle.create(new FeaturesConfig(), typeOperators, new BlockTypeOperators(typeOperators), UNKNOWN)); functionCatalog.addFunctions(new InternalFunctionBundle(new LiteralFunction(new InternalBlockEncodingSerde(new BlockEncodingManager(), TESTING_TYPE_MANAGER)))); return new FunctionManager(CatalogServiceProvider.fail(), functionCatalog, LanguageFunctionProvider.DISABLED); diff --git a/core/trino-main/src/main/java/io/trino/metadata/GlobalFunctionCatalog.java b/core/trino-main/src/main/java/io/trino/metadata/GlobalFunctionCatalog.java index 81056644011f9..71a8e17eb5b85 100644 --- a/core/trino-main/src/main/java/io/trino/metadata/GlobalFunctionCatalog.java +++ b/core/trino-main/src/main/java/io/trino/metadata/GlobalFunctionCatalog.java @@ -18,9 +18,12 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Multimap; import com.google.errorprone.annotations.ThreadSafe; +import com.google.inject.Inject; +import com.google.inject.Provider; import io.trino.connector.system.GlobalSystemConnector; import io.trino.operator.table.ExcludeColumns.ExcludeColumnsFunctionHandle; import io.trino.operator.table.Sequence.SequenceFunctionHandle; +import io.trino.operator.table.json.JsonTable.JsonTableFunctionHandle; import io.trino.spi.function.AggregationFunctionMetadata; import io.trino.spi.function.AggregationImplementation; import io.trino.spi.function.BoundSignature; @@ -37,6 +40,7 @@ import io.trino.spi.function.WindowFunctionSupplier; import io.trino.spi.function.table.ConnectorTableFunctionHandle; import io.trino.spi.function.table.TableFunctionProcessorProvider; +import io.trino.spi.type.TypeManager; import io.trino.spi.type.TypeSignature; import java.util.Collection; @@ -53,19 +57,33 @@ import static io.trino.metadata.OperatorNameUtil.unmangleOperator; import static io.trino.operator.table.ExcludeColumns.getExcludeColumnsFunctionProcessorProvider; import static io.trino.operator.table.Sequence.getSequenceFunctionProcessorProvider; +import static io.trino.operator.table.json.JsonTable.getJsonTableFunctionProcessorProvider; import static io.trino.spi.function.FunctionKind.AGGREGATE; import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.spi.type.BooleanType.BOOLEAN; import static io.trino.spi.type.IntegerType.INTEGER; import static java.util.Locale.ENGLISH; +import static java.util.Objects.requireNonNull; @ThreadSafe public class GlobalFunctionCatalog implements FunctionProvider { public static final String BUILTIN_SCHEMA = "builtin"; + + private final Provider metadata; + private final Provider typeManager; + private final Provider functionManager; private volatile FunctionMap functions = new FunctionMap(); + @Inject + public GlobalFunctionCatalog(Provider metadata, Provider typeManager, Provider functionManager) + { + this.metadata = requireNonNull(metadata, "metadata is null"); + this.typeManager = requireNonNull(typeManager, "typeManager is null"); + this.functionManager = requireNonNull(functionManager, "functionManager is null"); + } + public final synchronized void addFunctions(FunctionBundle functionBundle) { for (FunctionMetadata functionMetadata : functionBundle.getFunctions()) { @@ -187,6 +205,9 @@ public TableFunctionProcessorProvider getTableFunctionProcessorProvider(Connecto if (functionHandle instanceof SequenceFunctionHandle) { return getSequenceFunctionProcessorProvider(); } + if (functionHandle instanceof JsonTableFunctionHandle) { + return getJsonTableFunctionProcessorProvider(metadata.get(), typeManager.get(), functionManager.get()); + } return null; } diff --git a/core/trino-main/src/main/java/io/trino/metadata/MetadataManager.java b/core/trino-main/src/main/java/io/trino/metadata/MetadataManager.java index a96e0977e6d8d..9c9221e38ebf2 100644 --- a/core/trino-main/src/main/java/io/trino/metadata/MetadataManager.java +++ b/core/trino-main/src/main/java/io/trino/metadata/MetadataManager.java @@ -2761,7 +2761,10 @@ public MetadataManager build() GlobalFunctionCatalog globalFunctionCatalog = this.globalFunctionCatalog; if (globalFunctionCatalog == null) { - globalFunctionCatalog = new GlobalFunctionCatalog(); + globalFunctionCatalog = new GlobalFunctionCatalog( + () -> { throw new UnsupportedOperationException(); }, + () -> { throw new UnsupportedOperationException(); }, + () -> { throw new UnsupportedOperationException(); }); TypeOperators typeOperators = new TypeOperators(); globalFunctionCatalog.addFunctions(SystemFunctionBundle.create(new FeaturesConfig(), typeOperators, new BlockTypeOperators(typeOperators), UNKNOWN)); globalFunctionCatalog.addFunctions(new InternalFunctionBundle(new LiteralFunction(new InternalBlockEncodingSerde(new BlockEncodingManager(), typeManager)))); diff --git a/core/trino-main/src/main/java/io/trino/operator/TableFunctionOperator.java b/core/trino-main/src/main/java/io/trino/operator/TableFunctionOperator.java index f309a6d145c50..7b41bde101eda 100644 --- a/core/trino-main/src/main/java/io/trino/operator/TableFunctionOperator.java +++ b/core/trino-main/src/main/java/io/trino/operator/TableFunctionOperator.java @@ -128,7 +128,6 @@ public TableFunctionOperatorFactory( { requireNonNull(planNodeId, "planNodeId is null"); requireNonNull(tableFunctionProvider, "tableFunctionProvider is null"); - requireNonNull(catalogHandle, "catalogHandle is null"); requireNonNull(functionHandle, "functionHandle is null"); requireNonNull(requiredChannels, "requiredChannels is null"); requireNonNull(markerChannels, "markerChannels is null"); @@ -272,6 +271,7 @@ public TableFunctionOperator( this.operatorContext = operatorContext; this.session = operatorContext.getSession().toConnectorSession(catalogHandle); + this.processEmptyInput = !pruneWhenEmpty; PagesIndex pagesIndex = pagesIndexFactory.newPagesIndex(sourceTypes, expectedPositions); diff --git a/core/trino-main/src/main/java/io/trino/operator/table/json/JsonTable.java b/core/trino-main/src/main/java/io/trino/operator/table/json/JsonTable.java new file mode 100644 index 0000000000000..3b64b58b34f88 --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/operator/table/json/JsonTable.java @@ -0,0 +1,215 @@ +/* + * 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.operator.table.json; + +import com.fasterxml.jackson.databind.JsonNode; +import com.google.common.collect.ImmutableList; +import io.trino.metadata.FunctionManager; +import io.trino.metadata.Metadata; +import io.trino.operator.table.json.execution.JsonTableProcessingFragment; +import io.trino.spi.Page; +import io.trino.spi.PageBuilder; +import io.trino.spi.block.SqlRow; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.function.table.ConnectorTableFunctionHandle; +import io.trino.spi.function.table.TableFunctionDataProcessor; +import io.trino.spi.function.table.TableFunctionProcessorProvider; +import io.trino.spi.function.table.TableFunctionProcessorState; +import io.trino.spi.type.RowType; +import io.trino.spi.type.Type; +import io.trino.spi.type.TypeManager; + +import java.util.Arrays; +import java.util.List; +import java.util.Optional; + +import static com.google.common.collect.Iterables.getOnlyElement; +import static io.trino.operator.scalar.json.ParameterUtil.getParametersArray; +import static io.trino.operator.table.json.execution.ExecutionPlanner.getExecutionPlan; +import static io.trino.spi.function.table.TableFunctionProcessorState.Finished.FINISHED; +import static io.trino.spi.function.table.TableFunctionProcessorState.Processed.produced; +import static io.trino.spi.function.table.TableFunctionProcessorState.Processed.usedInput; +import static io.trino.spi.type.BigintType.BIGINT; +import static io.trino.spi.type.TypeUtils.readNativeValue; +import static io.trino.spi.type.TypeUtils.writeNativeValue; +import static io.trino.type.Json2016Type.JSON_2016; +import static java.util.Objects.requireNonNull; + +/** + * Implements feature ISO/IEC 9075-2:2023(E) 7.11 'JSON table' + * including features T824, T827, T838 + */ +public class JsonTable +{ + private JsonTable() {} + + /** + * This class comprises all information necessary to execute the json_table function: + * + * @param processingPlan the root of the processing plan tree + * @param outer the parent-child relationship between the input relation and the processingPlan result + * @param errorOnError the error behavior: true for ERROR ON ERROR, false for EMPTY ON ERROR + * @param parametersType type of the row containing JSON path parameters for the root JSON path. The function expects the parameters row in the channel 1. + * Other channels in the input page correspond to JSON context item (channel 0), and default values for the value columns. Each value column in the processingPlan + * knows the indexes of its default channels. + * @param outputTypes types of the proper columns produced by the function + */ + public record JsonTableFunctionHandle(JsonTablePlanNode processingPlan, boolean outer, boolean errorOnError, RowType parametersType, Type[] outputTypes) + implements ConnectorTableFunctionHandle + { + public JsonTableFunctionHandle + { + requireNonNull(processingPlan, "processingPlan is null"); + requireNonNull(parametersType, "parametersType is null"); + requireNonNull(outputTypes, "outputTypes is null"); + } + } + + public static TableFunctionProcessorProvider getJsonTableFunctionProcessorProvider(Metadata metadata, TypeManager typeManager, FunctionManager functionManager) + { + return new TableFunctionProcessorProvider() + { + @Override + public TableFunctionDataProcessor getDataProcessor(ConnectorSession session, ConnectorTableFunctionHandle handle) + { + JsonTableFunctionHandle jsonTableFunctionHandle = (JsonTableFunctionHandle) handle; + Object[] newRow = new Object[jsonTableFunctionHandle.outputTypes().length]; + JsonTableProcessingFragment executionPlan = getExecutionPlan( + jsonTableFunctionHandle.processingPlan(), + newRow, + jsonTableFunctionHandle.errorOnError(), + jsonTableFunctionHandle.outputTypes(), + session, + metadata, + typeManager, + functionManager); + return new JsonTableFunctionProcessor(executionPlan, newRow, jsonTableFunctionHandle.outputTypes(), jsonTableFunctionHandle.parametersType(), jsonTableFunctionHandle.outer()); + } + }; + } + + public static class JsonTableFunctionProcessor + implements TableFunctionDataProcessor + { + private final PageBuilder pageBuilder; + private final int properColumnsCount; + private final JsonTableProcessingFragment executionPlan; + private final Object[] newRow; + private final RowType parametersType; + private final boolean outer; + + private long totalPositionsProcessed; + private int currentPosition = -1; + private boolean currentPositionAlreadyProduced; + + public JsonTableFunctionProcessor(JsonTableProcessingFragment executionPlan, Object[] newRow, Type[] outputTypes, RowType parametersType, boolean outer) + { + this.pageBuilder = new PageBuilder(ImmutableList.builder() + .add(outputTypes) + .add(BIGINT) // add additional position for pass-through index + .build()); + this.properColumnsCount = outputTypes.length; + this.executionPlan = requireNonNull(executionPlan, "executionPlan is null"); + this.newRow = requireNonNull(newRow, "newRow is null"); + this.parametersType = requireNonNull(parametersType, "parametersType is null"); + this.outer = outer; + } + + @Override + public TableFunctionProcessorState process(List> input) + { + // no more input pages + if (input == null) { + if (pageBuilder.isEmpty()) { + return FINISHED; + } + return flushPageBuilder(); + } + + Page inputPage = getOnlyElement(input).orElseThrow(); + while (!pageBuilder.isFull()) { + // new input page + if (currentPosition == -1) { + if (inputPage.getPositionCount() == 0) { + return usedInput(); + } + else { + currentPosition = 0; + currentPositionAlreadyProduced = false; + totalPositionsProcessed++; + SqlRow parametersRow = (SqlRow) readNativeValue(parametersType, inputPage.getBlock(1), currentPosition); + executionPlan.resetRoot( + (JsonNode) readNativeValue(JSON_2016, inputPage.getBlock(0), currentPosition), + inputPage, + currentPosition, + getParametersArray(parametersType, parametersRow)); + } + } + + // try to get output row for the current position (one position can produce multiple rows) + boolean gotNewRow = executionPlan.getRow(); + if (gotNewRow) { + currentPositionAlreadyProduced = true; + addOutputRow(); + } + else { + if (outer && !currentPositionAlreadyProduced) { + addNullPaddedRow(); + } + // go to next position in the input page + currentPosition++; + if (currentPosition < inputPage.getPositionCount()) { + currentPositionAlreadyProduced = false; + totalPositionsProcessed++; + SqlRow parametersRow = (SqlRow) readNativeValue(parametersType, inputPage.getBlock(1), currentPosition); + executionPlan.resetRoot( + (JsonNode) readNativeValue(JSON_2016, inputPage.getBlock(0), currentPosition), + inputPage, + currentPosition, + getParametersArray(parametersType, parametersRow)); + } + else { + currentPosition = -1; + return usedInput(); + } + } + } + + return flushPageBuilder(); + } + + private TableFunctionProcessorState flushPageBuilder() + { + TableFunctionProcessorState result = produced(pageBuilder.build()); + pageBuilder.reset(); + return result; + } + + private void addOutputRow() + { + pageBuilder.declarePosition(); + for (int channel = 0; channel < properColumnsCount; channel++) { + writeNativeValue(pageBuilder.getType(channel), pageBuilder.getBlockBuilder(channel), newRow[channel]); + } + // pass-through index from partition start + BIGINT.writeLong(pageBuilder.getBlockBuilder(properColumnsCount), totalPositionsProcessed - 1); + } + + private void addNullPaddedRow() + { + Arrays.fill(newRow, null); + addOutputRow(); + } + } +} diff --git a/core/trino-main/src/main/java/io/trino/operator/table/json/JsonTableColumn.java b/core/trino-main/src/main/java/io/trino/operator/table/json/JsonTableColumn.java new file mode 100644 index 0000000000000..8727e4254c67f --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/operator/table/json/JsonTableColumn.java @@ -0,0 +1,31 @@ +/* + * 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.operator.table.json; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; + +@JsonTypeInfo( + use = JsonTypeInfo.Id.NAME, + property = "@type") +@JsonSubTypes({ + @JsonSubTypes.Type(value = JsonTableOrdinalityColumn.class, name = "ordinality"), + @JsonSubTypes.Type(value = JsonTableQueryColumn.class, name = "query"), + @JsonSubTypes.Type(value = JsonTableValueColumn.class, name = "value"), +}) + +public sealed interface JsonTableColumn + permits JsonTableOrdinalityColumn, JsonTableQueryColumn, JsonTableValueColumn +{ +} diff --git a/core/trino-main/src/main/java/io/trino/operator/table/json/JsonTableOrdinalityColumn.java b/core/trino-main/src/main/java/io/trino/operator/table/json/JsonTableOrdinalityColumn.java new file mode 100644 index 0000000000000..904bb385e4429 --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/operator/table/json/JsonTableOrdinalityColumn.java @@ -0,0 +1,19 @@ +/* + * 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.operator.table.json; + +public record JsonTableOrdinalityColumn(int outputIndex) + implements JsonTableColumn +{ +} diff --git a/core/trino-main/src/main/java/io/trino/operator/table/json/JsonTablePlanCross.java b/core/trino-main/src/main/java/io/trino/operator/table/json/JsonTablePlanCross.java new file mode 100644 index 0000000000000..f61c13f920c9b --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/operator/table/json/JsonTablePlanCross.java @@ -0,0 +1,30 @@ +/* + * 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.operator.table.json; + +import com.google.common.collect.ImmutableList; + +import java.util.List; + +import static com.google.common.base.Preconditions.checkArgument; + +public record JsonTablePlanCross(List siblings) + implements JsonTablePlanNode +{ + public JsonTablePlanCross(List siblings) + { + this.siblings = ImmutableList.copyOf(siblings); + checkArgument(siblings.size() >= 2, "less than 2 siblings in Cross node"); + } +} diff --git a/core/trino-main/src/main/java/io/trino/operator/table/json/JsonTablePlanLeaf.java b/core/trino-main/src/main/java/io/trino/operator/table/json/JsonTablePlanLeaf.java new file mode 100644 index 0000000000000..f1cbafbe86cef --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/operator/table/json/JsonTablePlanLeaf.java @@ -0,0 +1,31 @@ +/* + * 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.operator.table.json; + +import com.google.common.collect.ImmutableList; +import io.trino.json.ir.IrJsonPath; + +import java.util.List; + +import static java.util.Objects.requireNonNull; + +public record JsonTablePlanLeaf(IrJsonPath path, List columns) + implements JsonTablePlanNode +{ + public JsonTablePlanLeaf(IrJsonPath path, List columns) + { + this.path = requireNonNull(path, "path is null"); + this.columns = ImmutableList.copyOf(columns); + } +} diff --git a/core/trino-main/src/main/java/io/trino/operator/table/json/JsonTablePlanNode.java b/core/trino-main/src/main/java/io/trino/operator/table/json/JsonTablePlanNode.java new file mode 100644 index 0000000000000..73b56a75fb17f --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/operator/table/json/JsonTablePlanNode.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.operator.table.json; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; + +@JsonTypeInfo( + use = JsonTypeInfo.Id.NAME, + property = "@type") +@JsonSubTypes({ + @JsonSubTypes.Type(value = JsonTablePlanCross.class, name = "cross"), + @JsonSubTypes.Type(value = JsonTablePlanLeaf.class, name = "leaf"), + @JsonSubTypes.Type(value = JsonTablePlanSingle.class, name = "single"), + @JsonSubTypes.Type(value = JsonTablePlanUnion.class, name = "union"), +}) + +public sealed interface JsonTablePlanNode + permits JsonTablePlanCross, JsonTablePlanLeaf, JsonTablePlanSingle, JsonTablePlanUnion +{ +} diff --git a/core/trino-main/src/main/java/io/trino/operator/table/json/JsonTablePlanSingle.java b/core/trino-main/src/main/java/io/trino/operator/table/json/JsonTablePlanSingle.java new file mode 100644 index 0000000000000..49423e2c4bd2b --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/operator/table/json/JsonTablePlanSingle.java @@ -0,0 +1,33 @@ +/* + * 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.operator.table.json; + +import com.google.common.collect.ImmutableList; +import io.trino.json.ir.IrJsonPath; + +import java.util.List; + +import static java.util.Objects.requireNonNull; + +public record JsonTablePlanSingle(IrJsonPath path, List columns, boolean outer, JsonTablePlanNode child) + implements JsonTablePlanNode +{ + public JsonTablePlanSingle(IrJsonPath path, List columns, boolean outer, JsonTablePlanNode child) + { + this.path = requireNonNull(path, "path is null"); + this.columns = ImmutableList.copyOf(columns); + this.outer = outer; + this.child = requireNonNull(child, "child is null"); + } +} diff --git a/core/trino-main/src/main/java/io/trino/operator/table/json/JsonTablePlanUnion.java b/core/trino-main/src/main/java/io/trino/operator/table/json/JsonTablePlanUnion.java new file mode 100644 index 0000000000000..e8a1f1caeaf4a --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/operator/table/json/JsonTablePlanUnion.java @@ -0,0 +1,30 @@ +/* + * 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.operator.table.json; + +import com.google.common.collect.ImmutableList; + +import java.util.List; + +import static com.google.common.base.Preconditions.checkArgument; + +public record JsonTablePlanUnion(List siblings) + implements JsonTablePlanNode +{ + public JsonTablePlanUnion(List siblings) + { + this.siblings = ImmutableList.copyOf(siblings); + checkArgument(siblings.size() >= 2, "less than 2 siblings in Union node"); + } +} diff --git a/core/trino-main/src/main/java/io/trino/operator/table/json/JsonTableQueryColumn.java b/core/trino-main/src/main/java/io/trino/operator/table/json/JsonTableQueryColumn.java new file mode 100644 index 0000000000000..117df03c2c25f --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/operator/table/json/JsonTableQueryColumn.java @@ -0,0 +1,40 @@ +/* + * 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.operator.table.json; + +import io.trino.json.ir.IrJsonPath; +import io.trino.metadata.ResolvedFunction; + +import static java.util.Objects.requireNonNull; + +/** + * This representation does not contain all properties of the column as specified in json_table invocation. + * Certain properties are handled by the output function which is applied later. + * These are: output format and quotes behavior. + */ +public record JsonTableQueryColumn( + int outputIndex, + ResolvedFunction function, + IrJsonPath path, + long wrapperBehavior, + long emptyBehavior, + long errorBehavior) + implements JsonTableColumn +{ + public JsonTableQueryColumn + { + requireNonNull(function, "function is null"); + requireNonNull(path, "path is null"); + } +} diff --git a/core/trino-main/src/main/java/io/trino/operator/table/json/JsonTableValueColumn.java b/core/trino-main/src/main/java/io/trino/operator/table/json/JsonTableValueColumn.java new file mode 100644 index 0000000000000..6d87bc4a5ffd8 --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/operator/table/json/JsonTableValueColumn.java @@ -0,0 +1,36 @@ +/* + * 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.operator.table.json; + +import io.trino.json.ir.IrJsonPath; +import io.trino.metadata.ResolvedFunction; + +import static java.util.Objects.requireNonNull; + +public record JsonTableValueColumn( + int outputIndex, + ResolvedFunction function, + IrJsonPath path, + long emptyBehavior, + int emptyDefaultInput, // channel number or -1 when default not specified + long errorBehavior, + int errorDefaultInput) // channel number or -1 when default not specified + implements JsonTableColumn +{ + public JsonTableValueColumn + { + requireNonNull(function, "function is null"); + requireNonNull(path, "path is null"); + } +} diff --git a/core/trino-main/src/main/java/io/trino/operator/table/json/execution/Column.java b/core/trino-main/src/main/java/io/trino/operator/table/json/execution/Column.java new file mode 100644 index 0000000000000..15eab03d10d33 --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/operator/table/json/execution/Column.java @@ -0,0 +1,24 @@ +/* + * 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.operator.table.json.execution; + +import com.fasterxml.jackson.databind.JsonNode; +import io.trino.spi.Page; + +public interface Column +{ + Object evaluate(long sequentialNumber, JsonNode item, Page input, int position); + + int getOutputIndex(); +} diff --git a/core/trino-main/src/main/java/io/trino/operator/table/json/execution/ExecutionPlanner.java b/core/trino-main/src/main/java/io/trino/operator/table/json/execution/ExecutionPlanner.java new file mode 100644 index 0000000000000..e6c4879db057a --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/operator/table/json/execution/ExecutionPlanner.java @@ -0,0 +1,159 @@ +/* + * 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.operator.table.json.execution; + +import com.google.common.collect.ImmutableList; +import io.trino.json.JsonPathInvocationContext; +import io.trino.metadata.FunctionManager; +import io.trino.metadata.Metadata; +import io.trino.operator.table.json.JsonTableColumn; +import io.trino.operator.table.json.JsonTableOrdinalityColumn; +import io.trino.operator.table.json.JsonTablePlanCross; +import io.trino.operator.table.json.JsonTablePlanLeaf; +import io.trino.operator.table.json.JsonTablePlanNode; +import io.trino.operator.table.json.JsonTablePlanSingle; +import io.trino.operator.table.json.JsonTablePlanUnion; +import io.trino.operator.table.json.JsonTableQueryColumn; +import io.trino.operator.table.json.JsonTableValueColumn; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.function.InvocationConvention; +import io.trino.spi.function.ScalarFunctionImplementation; +import io.trino.spi.type.Type; +import io.trino.spi.type.TypeManager; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Throwables.throwIfUnchecked; +import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.trino.spi.function.InvocationConvention.InvocationArgumentConvention.BOXED_NULLABLE; +import static io.trino.spi.function.InvocationConvention.InvocationArgumentConvention.NEVER_NULL; +import static io.trino.spi.function.InvocationConvention.InvocationReturnConvention.NULLABLE_RETURN; + +public class ExecutionPlanner +{ + private ExecutionPlanner() + { + } + + public static JsonTableProcessingFragment getExecutionPlan( + JsonTablePlanNode plan, + Object[] newRow, + boolean errorOnError, + Type[] outputTypes, + ConnectorSession session, + Metadata metadata, + TypeManager typeManager, + FunctionManager functionManager) + { + if (plan instanceof JsonTablePlanLeaf planLeaf) { + return new FragmentLeaf( + planLeaf.path(), + planLeaf.columns().stream() + .map(column -> getColumn(column, outputTypes, session, functionManager)) + .collect(toImmutableList()), + errorOnError, + newRow, + session, + metadata, + typeManager, + functionManager); + } + if (plan instanceof JsonTablePlanSingle planSingle) { + return new FragmentSingle( + planSingle.path(), + planSingle.columns().stream() + .map(column -> getColumn(column, outputTypes, session, functionManager)) + .collect(toImmutableList()), + errorOnError, + planSingle.outer(), + getExecutionPlan(planSingle.child(), newRow, errorOnError, outputTypes, session, metadata, typeManager, functionManager), + newRow, + session, + metadata, + typeManager, + functionManager); + } + if (plan instanceof JsonTablePlanCross planCross) { + return new FragmentCross(planCross.siblings().stream() + .map(sibling -> getExecutionPlan(sibling, newRow, errorOnError, outputTypes, session, metadata, typeManager, functionManager)) + .collect(toImmutableList())); + } + JsonTablePlanUnion planUnion = (JsonTablePlanUnion) plan; + return new FragmentUnion( + planUnion.siblings().stream() + .map(sibling -> getExecutionPlan(sibling, newRow, errorOnError, outputTypes, session, metadata, typeManager, functionManager)) + .collect(toImmutableList()), + newRow); + } + + private static Column getColumn(JsonTableColumn column, Type[] outputTypes, ConnectorSession session, FunctionManager functionManager) + { + if (column instanceof JsonTableValueColumn valueColumn) { + ScalarFunctionImplementation implementation = functionManager.getScalarFunctionImplementation( + valueColumn.function(), + new InvocationConvention( + ImmutableList.of(BOXED_NULLABLE, BOXED_NULLABLE, BOXED_NULLABLE, NEVER_NULL, BOXED_NULLABLE, NEVER_NULL, BOXED_NULLABLE), + NULLABLE_RETURN, + true, + true)); + JsonPathInvocationContext context; + checkArgument(implementation.getInstanceFactory().isPresent(), "instance factory is missing"); + try { + context = (JsonPathInvocationContext) implementation.getInstanceFactory().get().invoke(); + } + catch (Throwable throwable) { + throwIfUnchecked(throwable); + throw new RuntimeException(throwable); + } + return new ValueColumn( + valueColumn.outputIndex(), + implementation.getMethodHandle() + .bindTo(context) + .bindTo(session), + valueColumn.path(), + valueColumn.emptyBehavior(), + valueColumn.emptyDefaultInput(), + valueColumn.errorBehavior(), + valueColumn.errorDefaultInput(), + outputTypes[valueColumn.outputIndex()]); + } + if (column instanceof JsonTableQueryColumn queryColumn) { + ScalarFunctionImplementation implementation = functionManager.getScalarFunctionImplementation( + queryColumn.function(), + new InvocationConvention( + ImmutableList.of(BOXED_NULLABLE, BOXED_NULLABLE, BOXED_NULLABLE, NEVER_NULL, NEVER_NULL, NEVER_NULL), + NULLABLE_RETURN, + true, + true)); + JsonPathInvocationContext context; + checkArgument(implementation.getInstanceFactory().isPresent(), "instance factory is missing"); + try { + context = (JsonPathInvocationContext) implementation.getInstanceFactory().get().invoke(); + } + catch (Throwable throwable) { + throwIfUnchecked(throwable); + throw new RuntimeException(throwable); + } + return new QueryColumn( + queryColumn.outputIndex(), + implementation.getMethodHandle() + .bindTo(context) + .bindTo(session), + queryColumn.path(), + queryColumn.wrapperBehavior(), + queryColumn.emptyBehavior(), + queryColumn.errorBehavior()); + } + return new OrdinalityColumn(((JsonTableOrdinalityColumn) column).outputIndex()); + } +} diff --git a/core/trino-main/src/main/java/io/trino/operator/table/json/execution/FragmentCross.java b/core/trino-main/src/main/java/io/trino/operator/table/json/execution/FragmentCross.java new file mode 100644 index 0000000000000..56cbdbe724be0 --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/operator/table/json/execution/FragmentCross.java @@ -0,0 +1,93 @@ +/* + * 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.operator.table.json.execution; + +import com.fasterxml.jackson.databind.JsonNode; +import com.google.common.collect.ImmutableList; +import io.trino.spi.Page; + +import java.util.Arrays; +import java.util.List; + +import static com.google.common.base.Preconditions.checkArgument; +import static java.util.Objects.requireNonNull; + +public class FragmentCross + implements JsonTableProcessingFragment +{ + private final List siblings; + private final int[] outputLayout; + + private Page input; + private int position; + private JsonNode currentItem; + private int currentSiblingIndex; + + public FragmentCross(List siblings) + { + this.siblings = ImmutableList.copyOf(siblings); + checkArgument(siblings.size() >= 2, "less than 2 siblings in Cross node"); + this.outputLayout = siblings.stream() + .map(JsonTableProcessingFragment::getOutputLayout) + .flatMapToInt(Arrays::stream) + .toArray(); + } + + @Override + public void reset(JsonNode item, Page input, int position) + { + this.currentItem = requireNonNull(item, "item is null"); + this.input = requireNonNull(input, "input is null"); + this.position = position; + siblings.get(0).reset(item, input, position); + this.currentSiblingIndex = 0; + } + + /** + * All values produced by the siblings are stored on corresponding positions in `newRow`. It is a temporary representation of the result row, and is shared by all Fragments. + * The values in `newRow` are not cleared between subsequent calls to getRow(), so that the parts which do not change are automatically reused. + */ + @Override + public boolean getRow() + { + while (currentSiblingIndex >= 0) { + boolean currentSiblingProducedRow = siblings.get(currentSiblingIndex).getRow(); + if (currentSiblingProducedRow) { + for (int i = currentSiblingIndex + 1; i < siblings.size(); i++) { + JsonTableProcessingFragment sibling = siblings.get(i); + sibling.reset(currentItem, input, position); + boolean siblingProducedRow = sibling.getRow(); + if (!siblingProducedRow) { + // if any sibling is empty, the whole CROSS fragment is empty + return false; + } + } + currentSiblingIndex = siblings.size() - 1; + return true; + } + + // current sibling is finished + currentSiblingIndex--; + } + + // fragment is finished + return false; + } + + @Override + public int[] getOutputLayout() + { + return outputLayout; + } +} diff --git a/core/trino-main/src/main/java/io/trino/operator/table/json/execution/FragmentLeaf.java b/core/trino-main/src/main/java/io/trino/operator/table/json/execution/FragmentLeaf.java new file mode 100644 index 0000000000000..9a11e63067d41 --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/operator/table/json/execution/FragmentLeaf.java @@ -0,0 +1,109 @@ +/* + * 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.operator.table.json.execution; + +import com.fasterxml.jackson.databind.JsonNode; +import com.google.common.collect.ImmutableList; +import io.trino.json.JsonPathEvaluator; +import io.trino.json.ir.IrJsonPath; +import io.trino.metadata.FunctionManager; +import io.trino.metadata.Metadata; +import io.trino.spi.Page; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.type.TypeManager; + +import java.util.List; + +import static io.trino.operator.table.json.execution.SequenceEvaluator.getSequence; +import static java.util.Objects.requireNonNull; + +public class FragmentLeaf + implements JsonTableProcessingFragment +{ + private static final Object[] NO_PARAMETERS = new Object[0]; + + private final JsonPathEvaluator pathEvaluator; + private final List columns; + private final boolean errorOnError; + private final int[] outputLayout; + + // the place where the computed values (or nulls) are stored while computing an output row + private final Object[] newRow; + + private Page input; + private int position; + private List sequence; + private int nextItemIndex; + + public FragmentLeaf( + IrJsonPath path, + List columns, + boolean errorOnError, + Object[] newRow, + ConnectorSession session, + Metadata metadata, + TypeManager typeManager, + FunctionManager functionManager) + { + requireNonNull(path, "path is null"); + this.pathEvaluator = new JsonPathEvaluator(path, session, metadata, typeManager, functionManager); + this.columns = ImmutableList.copyOf(columns); + this.errorOnError = errorOnError; + this.outputLayout = columns.stream() + .mapToInt(Column::getOutputIndex) + .toArray(); + this.newRow = requireNonNull(newRow, "newRow is null"); + } + + @Override + public void reset(JsonNode item, Page input, int position) + { + resetRoot(item, input, position, NO_PARAMETERS); + } + + /** + * FragmentLeaf can be the root Fragment. The root fragment is the only fragment that may have path parameters. + * Prepares the root Fragment to produce rows for the new JSON item and a set of path parameters. + */ + @Override + public void resetRoot(JsonNode item, Page input, int position, Object[] pathParameters) + { + requireNonNull(pathParameters, "pathParameters is null"); + this.input = requireNonNull(input, "input is null"); + this.position = position; + this.nextItemIndex = 0; + this.sequence = getSequence(item, pathParameters, pathEvaluator, errorOnError); + } + + @Override + public boolean getRow() + { + if (nextItemIndex >= sequence.size()) { + // fragment is finished + return false; + } + JsonNode currentItem = sequence.get(nextItemIndex); + nextItemIndex++; // it is correct to pass the updated value to `column.evaluate()` because ordinality numbers are 1-based according to ISO/IEC 9075-2:2016(E) 7.11 p.461 General rules. + for (Column column : columns) { + newRow[column.getOutputIndex()] = column.evaluate(nextItemIndex, currentItem, input, position); + } + return true; + } + + @Override + public int[] getOutputLayout() + { + return outputLayout; + } +} diff --git a/core/trino-main/src/main/java/io/trino/operator/table/json/execution/FragmentSingle.java b/core/trino-main/src/main/java/io/trino/operator/table/json/execution/FragmentSingle.java new file mode 100644 index 0000000000000..d3d285f0658e3 --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/operator/table/json/execution/FragmentSingle.java @@ -0,0 +1,156 @@ +/* + * 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.operator.table.json.execution; + +import com.fasterxml.jackson.databind.JsonNode; +import com.google.common.collect.ImmutableList; +import io.trino.json.JsonPathEvaluator; +import io.trino.json.ir.IrJsonPath; +import io.trino.metadata.FunctionManager; +import io.trino.metadata.Metadata; +import io.trino.spi.Page; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.type.TypeManager; + +import java.util.Arrays; +import java.util.List; +import java.util.stream.IntStream; + +import static io.trino.operator.table.json.execution.SequenceEvaluator.getSequence; +import static java.util.Objects.requireNonNull; + +public class FragmentSingle + implements JsonTableProcessingFragment +{ + private static final Object[] NO_PARAMETERS = new Object[] {}; + + private final JsonPathEvaluator pathEvaluator; + private final List columns; + private final boolean errorOnError; + private final boolean outer; + private final JsonTableProcessingFragment child; + private final int[] outputLayout; + + // the place where the computed values (or nulls) are stored while computing an output row + private final Object[] newRow; + + private Page input; + private int position; + private List sequence; + private int nextItemIndex; + + // start processing next item from the sequence + private boolean processNextItem; + + // indicates if we need to produce null-padded row for OUTER + private boolean childAlreadyProduced; + + public FragmentSingle( + IrJsonPath path, + List columns, + boolean errorOnError, + boolean outer, + JsonTableProcessingFragment child, + Object[] newRow, + ConnectorSession session, + Metadata metadata, + TypeManager typeManager, + FunctionManager functionManager) + { + requireNonNull(path, "path is null"); + this.pathEvaluator = new JsonPathEvaluator(path, session, metadata, typeManager, functionManager); + this.columns = ImmutableList.copyOf(columns); + this.errorOnError = errorOnError; + this.outer = outer; + this.child = requireNonNull(child, "child is null"); + this.outputLayout = IntStream.concat( + columns.stream() + .mapToInt(Column::getOutputIndex), + Arrays.stream(child.getOutputLayout())) + .toArray(); + this.newRow = requireNonNull(newRow, "newRow is null"); + } + + @Override + public void reset(JsonNode item, Page input, int position) + { + resetRoot(item, input, position, NO_PARAMETERS); + } + + /** + * FragmentSingle can be the root Fragment. The root fragment is the only fragment that may have path parameters. + * Prepares the root Fragment to produce rows for the new JSON item and a set of path parameters. + */ + @Override + public void resetRoot(JsonNode item, Page input, int position, Object[] pathParameters) + { + requireNonNull(pathParameters, "pathParameters is null"); + this.input = requireNonNull(input, "input is null"); + this.position = position; + this.nextItemIndex = 0; + this.processNextItem = true; + this.sequence = getSequence(item, pathParameters, pathEvaluator, errorOnError); + } + + /** + * All values produced by the columns are stored on corresponding positions in `newRow`. + * The values in `newRow` are not cleared between subsequent calls to `getRow()`, so the values for columns are automatically reused during iterating over child. + */ + @Override + public boolean getRow() + { + while (true) { + if (processNextItem) { + if (nextItemIndex >= sequence.size()) { + // fragment is finished + return false; + } + JsonNode currentItem = sequence.get(nextItemIndex); + nextItemIndex++; // it is correct to pass the updated value to `column.evaluate()` because ordinality numbers are 1-based according to ISO/IEC 9075-2:2016(E) 7.11 p.461 General rules. + for (Column column : columns) { + newRow[column.getOutputIndex()] = column.evaluate(nextItemIndex, currentItem, input, position); + } + child.reset(currentItem, input, position); + childAlreadyProduced = false; + processNextItem = false; + } + + boolean childProducedRow = child.getRow(); + if (childProducedRow) { + childAlreadyProduced = true; + return true; + } + + // child is finished + processNextItem = true; + if (outer && !childAlreadyProduced) { + appendNulls(child); + return true; + } + } + } + + private void appendNulls(JsonTableProcessingFragment fragment) + { + for (int column : fragment.getOutputLayout()) { + newRow[column] = null; + } + } + + @Override + public int[] getOutputLayout() + { + return outputLayout; + } +} diff --git a/core/trino-main/src/main/java/io/trino/operator/table/json/execution/FragmentUnion.java b/core/trino-main/src/main/java/io/trino/operator/table/json/execution/FragmentUnion.java new file mode 100644 index 0000000000000..30ae142f9dfad --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/operator/table/json/execution/FragmentUnion.java @@ -0,0 +1,96 @@ +/* + * 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.operator.table.json.execution; + +import com.fasterxml.jackson.databind.JsonNode; +import com.google.common.collect.ImmutableList; +import io.trino.spi.Page; + +import java.util.Arrays; +import java.util.List; + +import static com.google.common.base.Preconditions.checkArgument; +import static java.util.Objects.requireNonNull; + +public class FragmentUnion + implements JsonTableProcessingFragment +{ + private final List siblings; + private final int[] outputLayout; + + // the place where the computed values (or nulls) are stored while computing an output row + private final Object[] newRow; + + private int currentSiblingIndex; + + public FragmentUnion(List siblings, Object[] newRow) + { + this.siblings = ImmutableList.copyOf(siblings); + checkArgument(siblings.size() >= 2, "less than 2 siblings in Union node"); + this.outputLayout = siblings.stream() + .map(JsonTableProcessingFragment::getOutputLayout) + .flatMapToInt(Arrays::stream) + .toArray(); + this.newRow = requireNonNull(newRow, "newRow is null"); + } + + @Override + public void reset(JsonNode item, Page input, int position) + { + requireNonNull(item, "item is null"); + requireNonNull(input, "input is null"); + siblings.stream() + .forEach(sibling -> sibling.reset(item, input, position)); + this.currentSiblingIndex = 0; + appendNulls(this); + } + + /** + * The values produced by the current sibling are stored on corresponding positions in `newRow`, and for other siblings `newRow` is filled with nulls. + * The values in `newRow` are not cleared between subsequent calls to getRow(), so that the parts which do not change are automatically reused. + */ + @Override + public boolean getRow() + { + while (true) { + if (currentSiblingIndex >= siblings.size()) { + // fragment is finished + return false; + } + + JsonTableProcessingFragment currentSibling = siblings.get(currentSiblingIndex); + boolean currentSiblingProducedRow = currentSibling.getRow(); + if (currentSiblingProducedRow) { + return true; + } + + // current sibling is finished + appendNulls(currentSibling); + currentSiblingIndex++; + } + } + + private void appendNulls(JsonTableProcessingFragment fragment) + { + for (int column : fragment.getOutputLayout()) { + newRow[column] = null; + } + } + + @Override + public int[] getOutputLayout() + { + return outputLayout; + } +} diff --git a/core/trino-main/src/main/java/io/trino/operator/table/json/execution/JsonTableProcessingFragment.java b/core/trino-main/src/main/java/io/trino/operator/table/json/execution/JsonTableProcessingFragment.java new file mode 100644 index 0000000000000..bfe518b41036c --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/operator/table/json/execution/JsonTableProcessingFragment.java @@ -0,0 +1,63 @@ +/* + * 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.operator.table.json.execution; + +import com.fasterxml.jackson.databind.JsonNode; +import io.trino.spi.Page; + +public interface JsonTableProcessingFragment +{ + /** + * Prepares the Fragment to produce rows for the new JSON item. + * Note: This method must be called for each new JSON item. Due to nesting, there might be multiple JSON items to process for a single position in the input page. + * Therefore, input and position may not change for subsequent calls. + * + * @param item the new JSON item + * @param input the input Page currently processed by json_table function + * @param position the currently processed position in the input page + */ + void reset(JsonNode item, Page input, int position); + + /** + * Prepares the root Fragment to produce rows for the new JSON item and new set of path parameters. + * Note: at the root level, there is one JSON item and one set of path parameters to process for each position in the input page. + * + * @param item the new JSON item + * @param input the input Page currently processed by json_table function + * @param position the currently processed position in the input page + * @param pathParameters JSON path parameters for the top-level JSON path + */ + default void resetRoot(JsonNode item, Page input, int position, Object[] pathParameters) + { + throw new IllegalStateException("not the root fragment"); + } + + /** + * Tries to produce output values for all columns included in the Fragment, + * and stores them in corresponding positions in `newRow`. + * Note: According to OUTER or UNION semantics, some values might be null-padded instead of computed. + * Note: a single JSON item might result in multiple output rows. To fully process a JSON item, the caller must: + * - reset the Fragment with the JSON item + * - call getRow() and collect output rows as long as `true` is returned + * If `false` is returned, there is no output row available, and the JSON item is fully processed + * + * @return true if row was produced, false if row was not produced (Fragment is finished) + */ + boolean getRow(); + + /** + * Returns an array containing indexes of columns produced by the fragment within all columns produced by json_table. + */ + int[] getOutputLayout(); +} diff --git a/core/trino-main/src/main/java/io/trino/operator/table/json/execution/OrdinalityColumn.java b/core/trino-main/src/main/java/io/trino/operator/table/json/execution/OrdinalityColumn.java new file mode 100644 index 0000000000000..d26479ecf9e41 --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/operator/table/json/execution/OrdinalityColumn.java @@ -0,0 +1,40 @@ +/* + * 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.operator.table.json.execution; + +import com.fasterxml.jackson.databind.JsonNode; +import io.trino.spi.Page; + +public class OrdinalityColumn + implements Column +{ + private final int outputIndex; + + public OrdinalityColumn(int outputIndex) + { + this.outputIndex = outputIndex; + } + + @Override + public Object evaluate(long sequentialNumber, JsonNode item, Page input, int position) + { + return sequentialNumber; + } + + @Override + public int getOutputIndex() + { + return outputIndex; + } +} diff --git a/core/trino-main/src/main/java/io/trino/operator/table/json/execution/QueryColumn.java b/core/trino-main/src/main/java/io/trino/operator/table/json/execution/QueryColumn.java new file mode 100644 index 0000000000000..613ec5c41db39 --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/operator/table/json/execution/QueryColumn.java @@ -0,0 +1,63 @@ +/* + * 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.operator.table.json.execution; + +import com.fasterxml.jackson.databind.JsonNode; +import io.trino.json.ir.IrJsonPath; +import io.trino.spi.Page; + +import java.lang.invoke.MethodHandle; + +import static com.google.common.base.Throwables.throwIfUnchecked; +import static java.util.Objects.requireNonNull; + +public class QueryColumn + implements Column +{ + private final int outputIndex; + private final MethodHandle methodHandle; + private final IrJsonPath path; + private final long wrapperBehavior; + private final long emptyBehavior; + private final long errorBehavior; + + public QueryColumn(int outputIndex, MethodHandle methodHandle, IrJsonPath path, long wrapperBehavior, long emptyBehavior, long errorBehavior) + { + this.outputIndex = outputIndex; + this.methodHandle = requireNonNull(methodHandle, "methodHandle is null"); + this.path = requireNonNull(path, "path is null"); + this.wrapperBehavior = wrapperBehavior; + this.emptyBehavior = emptyBehavior; + this.errorBehavior = errorBehavior; + } + + @Override + public Object evaluate(long sequentialNumber, JsonNode item, Page input, int position) + { + try { + return methodHandle.invoke(item, path, null, wrapperBehavior, emptyBehavior, errorBehavior); + } + catch (Throwable throwable) { + // According to ISO/IEC 9075-2:2016(E) 7.11 p.462 General rules 1) e) ii) 3) D) any exception thrown by column evaluation should be propagated. + throwIfUnchecked(throwable); + throw new RuntimeException(throwable); + } + } + + @Override + public int getOutputIndex() + { + return outputIndex; + } +} diff --git a/core/trino-main/src/main/java/io/trino/operator/table/json/execution/SequenceEvaluator.java b/core/trino-main/src/main/java/io/trino/operator/table/json/execution/SequenceEvaluator.java new file mode 100644 index 0000000000000..32b4fe0b9389c --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/operator/table/json/execution/SequenceEvaluator.java @@ -0,0 +1,93 @@ +/* + * 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.operator.table.json.execution; + +import com.fasterxml.jackson.databind.JsonNode; +import com.google.common.collect.ImmutableList; +import io.trino.json.JsonPathEvaluator; +import io.trino.json.PathEvaluationException; +import io.trino.json.ir.TypedValue; +import io.trino.operator.scalar.json.JsonOutputConversionException; + +import java.util.List; +import java.util.Optional; + +import static com.google.common.base.Preconditions.checkState; +import static io.trino.json.JsonInputErrorNode.JSON_ERROR; +import static io.trino.json.ir.SqlJsonLiteralConverter.getJsonNode; +import static java.lang.String.format; + +public class SequenceEvaluator +{ + private SequenceEvaluator() + { + } + + // creates a sequence of JSON items, and applies error handling + public static List getSequence(JsonNode item, Object[] pathParameters, JsonPathEvaluator pathEvaluator, boolean errorOnError) + { + if (item == null) { + // According to ISO/IEC 9075-2:2016(E) 7.11 p.461 General rules 1) a) empty table should be returned for null input. Empty sequence will result in an empty table. + return ImmutableList.of(); + } + // According to ISO/IEC 9075-2:2016(E) 7.11 p.461 General rules 1) e) exception thrown by path evaluation should be handled accordingly to json_table error behavior (ERROR or EMPTY). + // handle input conversion error for the context item + if (item.equals(JSON_ERROR)) { + checkState(!errorOnError, "input conversion error should have been thrown in the input function"); + // the error behavior is EMPTY ON ERROR. Empty sequence will result in an empty table. + return ImmutableList.of(); + } + // handle input conversion error for the path parameters + for (Object parameter : pathParameters) { + if (parameter.equals(JSON_ERROR)) { + checkState(!errorOnError, "input conversion error should have been thrown in the input function"); + // the error behavior is EMPTY ON ERROR. Empty sequence will result in an empty table. + return ImmutableList.of(); + } + } + // evaluate path into a sequence + List pathResult; + try { + pathResult = pathEvaluator.evaluate(item, pathParameters); + } + catch (PathEvaluationException e) { + if (errorOnError) { + throw e; + } + // the error behavior is EMPTY ON ERROR. Empty sequence will result in an empty table. + return ImmutableList.of(); + } + // convert sequence to JSON items + ImmutableList.Builder builder = ImmutableList.builder(); + for (Object element : pathResult) { + if (element instanceof TypedValue typedValue) { + Optional jsonNode = getJsonNode(typedValue); + if (jsonNode.isEmpty()) { + if (errorOnError) { + throw new JsonOutputConversionException(format( + "JSON path returned a scalar SQL value of type %s that cannot be represented as JSON", + ((TypedValue) element).getType())); + } + // the error behavior is EMPTY ON ERROR. Empty sequence will result in an empty table. + return ImmutableList.of(); + } + builder.add(jsonNode.get()); + } + else { + builder.add((JsonNode) element); + } + } + return builder.build(); + } +} diff --git a/core/trino-main/src/main/java/io/trino/operator/table/json/execution/ValueColumn.java b/core/trino-main/src/main/java/io/trino/operator/table/json/execution/ValueColumn.java new file mode 100644 index 0000000000000..a8c29d8baff7d --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/operator/table/json/execution/ValueColumn.java @@ -0,0 +1,93 @@ +/* + * 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.operator.table.json.execution; + +import com.fasterxml.jackson.databind.JsonNode; +import io.trino.json.ir.IrJsonPath; +import io.trino.spi.Page; +import io.trino.spi.type.Type; + +import java.lang.invoke.MethodHandle; + +import static com.google.common.base.Throwables.throwIfUnchecked; +import static io.trino.spi.type.TypeUtils.readNativeValue; +import static java.util.Objects.requireNonNull; + +public class ValueColumn + implements Column +{ + private final int outputIndex; + private final MethodHandle methodHandle; + private final IrJsonPath path; + private final long emptyBehavior; + private final int emptyDefaultInput; + private final long errorBehavior; + private final int errorDefaultInput; + private final Type resultType; + + public ValueColumn( + int outputIndex, + MethodHandle methodHandle, + IrJsonPath path, + long emptyBehavior, + int emptyDefaultInput, + long errorBehavior, + int errorDefaultInput, + Type resultType) + { + this.outputIndex = outputIndex; + this.methodHandle = requireNonNull(methodHandle, "methodHandle is null"); + this.path = requireNonNull(path, "path is null"); + this.emptyBehavior = emptyBehavior; + this.emptyDefaultInput = emptyDefaultInput; + this.errorBehavior = errorBehavior; + this.errorDefaultInput = errorDefaultInput; + this.resultType = requireNonNull(resultType, "resultType is null"); + } + + @Override + public Object evaluate(long sequentialNumber, JsonNode item, Page input, int position) + { + Object emptyDefault; + if (emptyDefaultInput == -1) { + emptyDefault = null; + } + else { + emptyDefault = readNativeValue(resultType, input.getBlock(emptyDefaultInput), position); + } + + Object errorDefault; + if (errorDefaultInput == -1) { + errorDefault = null; + } + else { + errorDefault = readNativeValue(resultType, input.getBlock(errorDefaultInput), position); + } + + try { + return methodHandle.invoke(item, path, null, emptyBehavior, emptyDefault, errorBehavior, errorDefault); + } + catch (Throwable throwable) { + // According to ISO/IEC 9075-2:2016(E) 7.11 p.462 General rules 1) e) ii) 2) D) any exception thrown by column evaluation should be propagated. + throwIfUnchecked(throwable); + throw new RuntimeException(throwable); + } + } + + @Override + public int getOutputIndex() + { + return outputIndex; + } +} diff --git a/core/trino-main/src/main/java/io/trino/sql/analyzer/Analysis.java b/core/trino-main/src/main/java/io/trino/sql/analyzer/Analysis.java index 96b29a1b31cd8..c878a5b118584 100644 --- a/core/trino-main/src/main/java/io/trino/sql/analyzer/Analysis.java +++ b/core/trino-main/src/main/java/io/trino/sql/analyzer/Analysis.java @@ -62,6 +62,8 @@ import io.trino.sql.tree.Identifier; import io.trino.sql.tree.InPredicate; import io.trino.sql.tree.Join; +import io.trino.sql.tree.JsonTable; +import io.trino.sql.tree.JsonTableColumnDefinition; import io.trino.sql.tree.LambdaArgumentDeclaration; import io.trino.sql.tree.MeasureDefinition; import io.trino.sql.tree.Node; @@ -161,9 +163,10 @@ public class Analysis private final Set> patternAggregations = new LinkedHashSet<>(); // for JSON features - private final Map, JsonPathAnalysis> jsonPathAnalyses = new LinkedHashMap<>(); + private final Map, JsonPathAnalysis> jsonPathAnalyses = new LinkedHashMap<>(); private final Map, ResolvedFunction> jsonInputFunctions = new LinkedHashMap<>(); - private final Map, ResolvedFunction> jsonOutputFunctions = new LinkedHashMap<>(); + private final Map, ResolvedFunction> jsonOutputFunctions = new LinkedHashMap<>(); + private final Map, JsonTableAnalysis> jsonTableAnalyses = new LinkedHashMap<>(); private final Map, List> aggregates = new LinkedHashMap<>(); private final Map, List> orderByAggregates = new LinkedHashMap<>(); @@ -204,7 +207,7 @@ public class Analysis private final Map, Type> sortKeyCoercionsForFrameBoundComparison = new LinkedHashMap<>(); private final Map, ResolvedFunction> frameBoundCalculations = new LinkedHashMap<>(); private final Map, List> relationCoercions = new LinkedHashMap<>(); - private final Map, RoutineEntry> resolvedFunctions = new LinkedHashMap<>(); + private final Map, RoutineEntry> resolvedFunctions = new LinkedHashMap<>(); private final Map, LambdaArgumentDeclaration> lambdaArgumentReferences = new LinkedHashMap<>(); private final Map columns = new LinkedHashMap<>(); @@ -656,12 +659,12 @@ public Set getResolvedFunctions() .collect(toImmutableSet()); } - public ResolvedFunction getResolvedFunction(Expression node) + public ResolvedFunction getResolvedFunction(Node node) { return resolvedFunctions.get(NodeRef.of(node)).getFunction(); } - public void addResolvedFunction(Expression node, ResolvedFunction function, String authorization) + public void addResolvedFunction(Node node, ResolvedFunction function, String authorization) { resolvedFunctions.put(NodeRef.of(node), new RoutineEntry(function, authorization)); } @@ -1021,14 +1024,19 @@ public boolean isPatternAggregation(FunctionCall function) return patternAggregations.contains(NodeRef.of(function)); } - public void setJsonPathAnalyses(Map, JsonPathAnalysis> pathAnalyses) + public void setJsonPathAnalyses(Map, JsonPathAnalysis> pathAnalyses) { jsonPathAnalyses.putAll(pathAnalyses); } - public JsonPathAnalysis getJsonPathAnalysis(Expression expression) + public void setJsonPathAnalysis(Node node, JsonPathAnalysis pathAnalysis) { - return jsonPathAnalyses.get(NodeRef.of(expression)); + jsonPathAnalyses.put(NodeRef.of(node), pathAnalysis); + } + + public JsonPathAnalysis getJsonPathAnalysis(Node node) + { + return jsonPathAnalyses.get(NodeRef.of(node)); } public void setJsonInputFunctions(Map, ResolvedFunction> functions) @@ -1041,14 +1049,24 @@ public ResolvedFunction getJsonInputFunction(Expression expression) return jsonInputFunctions.get(NodeRef.of(expression)); } - public void setJsonOutputFunctions(Map, ResolvedFunction> functions) + public void setJsonOutputFunctions(Map, ResolvedFunction> functions) { jsonOutputFunctions.putAll(functions); } - public ResolvedFunction getJsonOutputFunction(Expression expression) + public ResolvedFunction getJsonOutputFunction(Node node) + { + return jsonOutputFunctions.get(NodeRef.of(node)); + } + + public void addJsonTableAnalysis(JsonTable jsonTable, JsonTableAnalysis analysis) + { + jsonTableAnalyses.put(NodeRef.of(jsonTable), analysis); + } + + public JsonTableAnalysis getJsonTableAnalysis(JsonTable jsonTable) { - return jsonOutputFunctions.get(NodeRef.of(expression)); + return jsonTableAnalyses.get(NodeRef.of(jsonTable)); } public Map>> getTableColumnReferences() @@ -2388,4 +2406,19 @@ public ConnectorTransactionHandle getTransactionHandle() return transactionHandle; } } + + public record JsonTableAnalysis( + CatalogHandle catalogHandle, + ConnectorTransactionHandle transactionHandle, + RowType parametersType, + List> orderedOutputColumns) + { + public JsonTableAnalysis + { + requireNonNull(catalogHandle, "catalogHandle is null"); + requireNonNull(transactionHandle, "transactionHandle is null"); + requireNonNull(parametersType, "parametersType is null"); + requireNonNull(orderedOutputColumns, "orderedOutputColumns is null"); + } + } } diff --git a/core/trino-main/src/main/java/io/trino/sql/analyzer/ExpressionAnalyzer.java b/core/trino-main/src/main/java/io/trino/sql/analyzer/ExpressionAnalyzer.java index 0ef7db9ef3843..c97fcd62bd4d1 100644 --- a/core/trino-main/src/main/java/io/trino/sql/analyzer/ExpressionAnalyzer.java +++ b/core/trino-main/src/main/java/io/trino/sql/analyzer/ExpressionAnalyzer.java @@ -78,6 +78,7 @@ import io.trino.sql.tree.CurrentSchema; import io.trino.sql.tree.CurrentTime; import io.trino.sql.tree.CurrentUser; +import io.trino.sql.tree.DataType; import io.trino.sql.tree.DecimalLiteral; import io.trino.sql.tree.DereferenceExpression; import io.trino.sql.tree.DoubleLiteral; @@ -106,6 +107,7 @@ import io.trino.sql.tree.JsonPathParameter; import io.trino.sql.tree.JsonPathParameter.JsonFormat; import io.trino.sql.tree.JsonQuery; +import io.trino.sql.tree.JsonTable; import io.trino.sql.tree.JsonValue; import io.trino.sql.tree.LambdaArgumentDeclaration; import io.trino.sql.tree.LambdaExpression; @@ -123,6 +125,7 @@ import io.trino.sql.tree.ProcessingMode; import io.trino.sql.tree.QualifiedName; import io.trino.sql.tree.QuantifiedComparisonExpression; +import io.trino.sql.tree.QueryColumn; import io.trino.sql.tree.RangeQuantifier; import io.trino.sql.tree.Row; import io.trino.sql.tree.RowPattern; @@ -139,6 +142,7 @@ import io.trino.sql.tree.TimestampLiteral; import io.trino.sql.tree.Trim; import io.trino.sql.tree.TryExpression; +import io.trino.sql.tree.ValueColumn; import io.trino.sql.tree.VariableDefinition; import io.trino.sql.tree.WhenClause; import io.trino.sql.tree.WindowFrame; @@ -297,7 +301,7 @@ public class ExpressionAnalyzer // Cache from SQL type name to Type; every Type in the cache has a CAST defined from VARCHAR private final Cache varcharCastableTypeCache = buildNonEvictableCache(CacheBuilder.newBuilder().maximumSize(1000)); - private final Map, ResolvedFunction> resolvedFunctions = new LinkedHashMap<>(); + private final Map, ResolvedFunction> resolvedFunctions = new LinkedHashMap<>(); private final Set> subqueries = new LinkedHashSet<>(); private final Set> existsSubqueries = new LinkedHashSet<>(); private final Map, Type> expressionCoercions = new LinkedHashMap<>(); @@ -336,9 +340,9 @@ public class ExpressionAnalyzer private final Set> patternAggregations = new LinkedHashSet<>(); // for JSON functions - private final Map, JsonPathAnalysis> jsonPathAnalyses = new LinkedHashMap<>(); + private final Map, JsonPathAnalysis> jsonPathAnalyses = new LinkedHashMap<>(); private final Map, ResolvedFunction> jsonInputFunctions = new LinkedHashMap<>(); - private final Map, ResolvedFunction> jsonOutputFunctions = new LinkedHashMap<>(); + private final Map, ResolvedFunction> jsonOutputFunctions = new LinkedHashMap<>(); private final Session session; private final Map, Expression> parameters; @@ -402,7 +406,7 @@ private ExpressionAnalyzer( this.functionResolver = plannerContext.getFunctionResolver(warningCollector); } - public Map, ResolvedFunction> getResolvedFunctions() + public Map, ResolvedFunction> getResolvedFunctions() { return unmodifiableMap(resolvedFunctions); } @@ -500,6 +504,42 @@ private Type analyze(Expression expression, Scope baseScope, Context context) return visitor.process(expression, new StackableAstVisitor.StackableAstVisitorContext<>(context)); } + private RowType analyzeJsonPathInvocation(JsonTable node, Scope scope, CorrelationSupport correlationSupport) + { + Visitor visitor = new Visitor(scope, warningCollector); + List inputTypes = visitor.analyzeJsonPathInvocation("JSON_TABLE", node, node.getJsonPathInvocation(), new StackableAstVisitor.StackableAstVisitorContext<>(Context.notInLambda(scope, correlationSupport))); + return (RowType) inputTypes.get(2); + } + + private Type analyzeJsonValueExpression(ValueColumn column, JsonPathAnalysis pathAnalysis, Scope scope, CorrelationSupport correlationSupport) + { + Visitor visitor = new Visitor(scope, warningCollector); + List pathInvocationArgumentTypes = ImmutableList.of(JSON_2016, plannerContext.getTypeManager().getType(TypeId.of(JsonPath2016Type.NAME)), JSON_NO_PARAMETERS_ROW_TYPE); + return visitor.analyzeJsonValueExpression( + column, + pathAnalysis, + Optional.of(column.getType()), + pathInvocationArgumentTypes, + column.getEmptyBehavior(), + column.getEmptyDefault(), + column.getErrorBehavior(), + column.getErrorDefault(), + new StackableAstVisitor.StackableAstVisitorContext<>(Context.notInLambda(scope, correlationSupport))); + } + + private Type analyzeJsonQueryExpression(QueryColumn column, Scope scope) + { + Visitor visitor = new Visitor(scope, warningCollector); + List pathInvocationArgumentTypes = ImmutableList.of(JSON_2016, plannerContext.getTypeManager().getType(TypeId.of(JsonPath2016Type.NAME)), JSON_NO_PARAMETERS_ROW_TYPE); + return visitor.analyzeJsonQueryExpression( + column, + column.getWrapperBehavior(), + column.getQuotesBehavior(), + pathInvocationArgumentTypes, + Optional.of(column.getType()), + Optional.of(column.getFormat())); + } + private void analyzeWindow(ResolvedWindow window, Scope scope, Node originalNode, CorrelationSupport correlationSupport) { Visitor visitor = new Visitor(scope, warningCollector); @@ -566,7 +606,7 @@ public Set> getPatternAggregations() return patternAggregations; } - public Map, JsonPathAnalysis> getJsonPathAnalyses() + public Map, JsonPathAnalysis> getJsonPathAnalyses() { return jsonPathAnalyses; } @@ -576,7 +616,7 @@ public Map, ResolvedFunction> getJsonInputFunctions() return jsonInputFunctions; } - public Map, ResolvedFunction> getJsonOutputFunctions() + public Map, ResolvedFunction> getJsonOutputFunctions() { return jsonOutputFunctions; } @@ -2532,15 +2572,38 @@ public Type visitJsonExists(JsonExists node, StackableAstVisitorContext public Type visitJsonValue(JsonValue node, StackableAstVisitorContext context) { List pathInvocationArgumentTypes = analyzeJsonPathInvocation("JSON_VALUE", node, node.getJsonPathInvocation(), context); + Type returnedType = analyzeJsonValueExpression( + node, + jsonPathAnalyses.get(NodeRef.of(node)), + node.getReturnedType(), + pathInvocationArgumentTypes, + node.getEmptyBehavior(), + node.getEmptyDefault(), + Optional.of(node.getErrorBehavior()), + node.getErrorDefault(), + context); + return setExpressionType(node, returnedType); + } + private Type analyzeJsonValueExpression( + Node node, + JsonPathAnalysis pathAnalysis, + Optional declaredReturnedType, + List pathInvocationArgumentTypes, + JsonValue.EmptyOrErrorBehavior emptyBehavior, + Optional declaredEmptyDefault, + Optional errorBehavior, + Optional declaredErrorDefault, + StackableAstVisitorContext context) + { // validate returned type Type returnedType = VARCHAR; // default - if (node.getReturnedType().isPresent()) { + if (declaredReturnedType.isPresent()) { try { - returnedType = plannerContext.getTypeManager().getType(toTypeSignature(node.getReturnedType().get())); + returnedType = plannerContext.getTypeManager().getType(toTypeSignature(declaredReturnedType.get())); } catch (TypeNotFoundException e) { - throw semanticException(TYPE_MISMATCH, node, "Unknown type: %s", node.getReturnedType().get()); + throw semanticException(TYPE_MISMATCH, node, "Unknown type: %s", declaredReturnedType.get()); } } @@ -2550,10 +2613,9 @@ public Type visitJsonValue(JsonValue node, StackableAstVisitorContext c !isDateTimeType(returnedType) || returnedType.equals(INTERVAL_DAY_TIME) || returnedType.equals(INTERVAL_YEAR_MONTH)) { - throw semanticException(TYPE_MISMATCH, node, "Invalid return type of function JSON_VALUE: %s", node.getReturnedType().get()); + throw semanticException(TYPE_MISMATCH, node, "Invalid return type of function JSON_VALUE: %s", declaredReturnedType.get()); } - JsonPathAnalysis pathAnalysis = jsonPathAnalyses.get(NodeRef.of(node)); Type resultType = pathAnalysis.getType(pathAnalysis.getPath()); if (resultType != null && !resultType.equals(returnedType)) { try { @@ -2565,20 +2627,23 @@ public Type visitJsonValue(JsonValue node, StackableAstVisitorContext c } // validate default values for empty and error behavior - if (node.getEmptyDefault().isPresent()) { - Expression emptyDefault = node.getEmptyDefault().get(); - if (node.getEmptyBehavior() != DEFAULT) { - throw semanticException(INVALID_FUNCTION_ARGUMENT, emptyDefault, "Default value specified for %s ON EMPTY behavior", node.getEmptyBehavior()); + if (declaredEmptyDefault.isPresent()) { + Expression emptyDefault = declaredEmptyDefault.get(); + if (emptyBehavior != DEFAULT) { + throw semanticException(INVALID_FUNCTION_ARGUMENT, emptyDefault, "Default value specified for %s ON EMPTY behavior", emptyBehavior); } Type type = process(emptyDefault, context); // this would normally be done after function resolution, but we know that the default expression is always coerced to the returnedType coerceType(emptyDefault, type, returnedType, "Function JSON_VALUE default ON EMPTY result"); } - if (node.getErrorDefault().isPresent()) { - Expression errorDefault = node.getErrorDefault().get(); - if (node.getErrorBehavior() != DEFAULT) { - throw semanticException(INVALID_FUNCTION_ARGUMENT, errorDefault, "Default value specified for %s ON ERROR behavior", node.getErrorBehavior()); + if (declaredErrorDefault.isPresent()) { + Expression errorDefault = declaredErrorDefault.get(); + if (errorBehavior.isEmpty()) { + throw new IllegalStateException("error default specified without error behavior specified"); + } + if (errorBehavior.orElseThrow() != DEFAULT) { + throw semanticException(INVALID_FUNCTION_ARGUMENT, errorDefault, "Default value specified for %s ON ERROR behavior", errorBehavior.orElseThrow()); } Type type = process(errorDefault, context); // this would normally be done after function resolution, but we know that the default expression is always coerced to the returnedType @@ -2606,21 +2671,32 @@ public Type visitJsonValue(JsonValue node, StackableAstVisitorContext c throw new TrinoException(e::getErrorCode, extractLocation(node), e.getMessage(), e); } resolvedFunctions.put(NodeRef.of(node), function); - Type type = function.getSignature().getReturnType(); - return setExpressionType(node, type); + return function.getSignature().getReturnType(); } @Override public Type visitJsonQuery(JsonQuery node, StackableAstVisitorContext context) { List pathInvocationArgumentTypes = analyzeJsonPathInvocation("JSON_QUERY", node, node.getJsonPathInvocation(), context); + Type returnedType = analyzeJsonQueryExpression( + node, + node.getWrapperBehavior(), + node.getQuotesBehavior(), + pathInvocationArgumentTypes, + node.getReturnedType(), + node.getOutputFormat()); + return setExpressionType(node, returnedType); + } - // validate wrapper and quotes behavior - if ((node.getWrapperBehavior() == CONDITIONAL || node.getWrapperBehavior() == UNCONDITIONAL) && node.getQuotesBehavior().isPresent()) { - throw semanticException(INVALID_FUNCTION_ARGUMENT, node, "%s QUOTES behavior specified with WITH %s ARRAY WRAPPER behavior", node.getQuotesBehavior().get(), node.getWrapperBehavior()); - } - + private Type analyzeJsonQueryExpression( + Node node, + JsonQuery.ArrayWrapperBehavior wrapperBehavior, + Optional quotesBehavior, + List pathInvocationArgumentTypes, + Optional declaredReturnedType, + Optional declaredOutputFormat) + { // wrapper behavior, empty behavior and error behavior will be passed as arguments to function // quotes behavior is handled by the corresponding output function List argumentTypes = ImmutableList.builder() @@ -2630,6 +2706,11 @@ public Type visitJsonQuery(JsonQuery node, StackableAstVisitorContext c .add(TINYINT) // error behavior: enum encoded as integer value .build(); + // validate wrapper and quotes behavior + if ((wrapperBehavior == CONDITIONAL || wrapperBehavior == UNCONDITIONAL) && quotesBehavior.isPresent()) { + throw semanticException(INVALID_FUNCTION_ARGUMENT, node, "%s QUOTES behavior specified with WITH %s ARRAY WRAPPER behavior", quotesBehavior.get(), wrapperBehavior); + } + // resolve function ResolvedFunction function; try { @@ -2645,15 +2726,15 @@ public Type visitJsonQuery(JsonQuery node, StackableAstVisitorContext c // analyze returned type and format Type returnedType = VARCHAR; // default - if (node.getReturnedType().isPresent()) { + if (declaredReturnedType.isPresent()) { try { - returnedType = plannerContext.getTypeManager().getType(toTypeSignature(node.getReturnedType().get())); + returnedType = plannerContext.getTypeManager().getType(toTypeSignature(declaredReturnedType.get())); } catch (TypeNotFoundException e) { - throw semanticException(TYPE_MISMATCH, node, "Unknown type: %s", node.getReturnedType().get()); + throw semanticException(TYPE_MISMATCH, node, "Unknown type: %s", declaredReturnedType.get()); } } - JsonFormat outputFormat = node.getOutputFormat().orElse(JsonFormat.JSON); // default + JsonFormat outputFormat = declaredOutputFormat.orElse(JsonFormat.JSON); // default // resolve function to format output ResolvedFunction outputFunction = getOutputFunction(returnedType, outputFormat, node); @@ -2670,13 +2751,15 @@ public Type visitJsonQuery(JsonQuery node, StackableAstVisitorContext c } } - return setExpressionType(node, returnedType); + return returnedType; } - private List analyzeJsonPathInvocation(String functionName, Expression node, JsonPathInvocation jsonPathInvocation, StackableAstVisitorContext context) + private List analyzeJsonPathInvocation(String functionName, Node node, JsonPathInvocation jsonPathInvocation, StackableAstVisitorContext context) { jsonPathInvocation.getPathName().ifPresent(pathName -> { - throw semanticException(INVALID_PATH, pathName, "JSON path name is not allowed in %s function", functionName); + if (!(node instanceof JsonTable)) { + throw semanticException(INVALID_PATH, pathName, "JSON path name is not allowed in %s function", functionName); + } }); // ANALYZE THE CONTEXT ITEM @@ -3444,6 +3527,79 @@ public static ExpressionAnalysis analyzeExpression( analyzer.getWindowFunctions()); } + public static ParametersTypeAndAnalysis analyzeJsonPathInvocation( + JsonTable node, + Session session, + PlannerContext plannerContext, + StatementAnalyzerFactory statementAnalyzerFactory, + AccessControl accessControl, + Scope scope, + Analysis analysis, + WarningCollector warningCollector, + CorrelationSupport correlationSupport) + { + ExpressionAnalyzer analyzer = new ExpressionAnalyzer(plannerContext, accessControl, statementAnalyzerFactory, analysis, session, TypeProvider.empty(), warningCollector); + RowType parametersRowType = analyzer.analyzeJsonPathInvocation(node, scope, correlationSupport); + updateAnalysis(analysis, analyzer, session, accessControl); + return new ParametersTypeAndAnalysis( + parametersRowType, + new ExpressionAnalysis( + analyzer.getExpressionTypes(), + analyzer.getExpressionCoercions(), + analyzer.getSubqueryInPredicates(), + analyzer.getSubqueries(), + analyzer.getExistsSubqueries(), + analyzer.getColumnReferences(), + analyzer.getTypeOnlyCoercions(), + analyzer.getQuantifiedComparisons(), + analyzer.getWindowFunctions())); + } + + public record ParametersTypeAndAnalysis(RowType parametersType, ExpressionAnalysis expressionAnalysis) {} + + public static TypeAndAnalysis analyzeJsonValueExpression( + ValueColumn column, + JsonPathAnalysis pathAnalysis, + Session session, + PlannerContext plannerContext, + StatementAnalyzerFactory statementAnalyzerFactory, + AccessControl accessControl, + Scope scope, + Analysis analysis, + WarningCollector warningCollector, + CorrelationSupport correlationSupport) + { + ExpressionAnalyzer analyzer = new ExpressionAnalyzer(plannerContext, accessControl, statementAnalyzerFactory, analysis, session, TypeProvider.empty(), warningCollector); + Type type = analyzer.analyzeJsonValueExpression(column, pathAnalysis, scope, correlationSupport); + updateAnalysis(analysis, analyzer, session, accessControl); + return new TypeAndAnalysis(type, new ExpressionAnalysis( + analyzer.getExpressionTypes(), + analyzer.getExpressionCoercions(), + analyzer.getSubqueryInPredicates(), + analyzer.getSubqueries(), + analyzer.getExistsSubqueries(), + analyzer.getColumnReferences(), + analyzer.getTypeOnlyCoercions(), + analyzer.getQuantifiedComparisons(), + analyzer.getWindowFunctions())); + } + + public static Type analyzeJsonQueryExpression( + QueryColumn column, + Session session, + PlannerContext plannerContext, + StatementAnalyzerFactory statementAnalyzerFactory, + AccessControl accessControl, + Scope scope, + Analysis analysis, + WarningCollector warningCollector) + { + ExpressionAnalyzer analyzer = new ExpressionAnalyzer(plannerContext, accessControl, statementAnalyzerFactory, analysis, session, TypeProvider.empty(), warningCollector); + Type type = analyzer.analyzeJsonQueryExpression(column, scope); + updateAnalysis(analysis, analyzer, session, accessControl); + return type; + } + public static void analyzeExpressionWithoutSubqueries( Session session, PlannerContext plannerContext, @@ -3715,4 +3871,6 @@ public Optional getLabel() return label; } } + + public record TypeAndAnalysis(Type type, ExpressionAnalysis analysis) {} } diff --git a/core/trino-main/src/main/java/io/trino/sql/analyzer/JsonPathAnalyzer.java b/core/trino-main/src/main/java/io/trino/sql/analyzer/JsonPathAnalyzer.java index 093d9f016cbef..ef949ccd2527d 100644 --- a/core/trino-main/src/main/java/io/trino/sql/analyzer/JsonPathAnalyzer.java +++ b/core/trino-main/src/main/java/io/trino/sql/analyzer/JsonPathAnalyzer.java @@ -59,6 +59,7 @@ import io.trino.sql.jsonpath.tree.StartsWithPredicate; import io.trino.sql.jsonpath.tree.TypeMethod; import io.trino.sql.tree.Node; +import io.trino.sql.tree.NodeLocation; import io.trino.sql.tree.StringLiteral; import java.util.LinkedHashMap; @@ -108,11 +109,18 @@ public JsonPathAnalysis analyzeJsonPath(StringLiteral path, Map pa Location pathStart = extractLocation(path) .map(location -> new Location(location.getLineNumber(), location.getColumnNumber())) .orElseThrow(() -> new IllegalStateException("missing NodeLocation in path")); - PathNode root = new PathParser(pathStart).parseJsonPath(path.getValue()); + PathNode root = PathParser.withRelativeErrorLocation(pathStart).parseJsonPath(path.getValue()); new Visitor(parameterTypes, path).process(root); return new JsonPathAnalysis((JsonPath) root, types, jsonParameters); } + public JsonPathAnalysis analyzeImplicitJsonPath(String path, NodeLocation location) + { + PathNode root = PathParser.withFixedErrorLocation(new Location(location.getLineNumber(), location.getColumnNumber())).parseJsonPath(path); + new Visitor(ImmutableMap.of(), new StringLiteral(path)).process(root); + return new JsonPathAnalysis((JsonPath) root, types, jsonParameters); + } + /** * This visitor determines and validates output types of PathNodes, whenever they can be deduced and represented as SQL types. * In some cases, the type of a PathNode can be determined without context. E.g., the `double()` method always returns DOUBLE. diff --git a/core/trino-main/src/main/java/io/trino/sql/analyzer/StatementAnalyzer.java b/core/trino-main/src/main/java/io/trino/sql/analyzer/StatementAnalyzer.java index 3641b35579ee2..4a935a59777c2 100644 --- a/core/trino-main/src/main/java/io/trino/sql/analyzer/StatementAnalyzer.java +++ b/core/trino-main/src/main/java/io/trino/sql/analyzer/StatementAnalyzer.java @@ -23,11 +23,13 @@ import com.google.common.collect.Iterables; import com.google.common.collect.ListMultimap; import com.google.common.collect.Multimap; +import com.google.common.collect.Sets; import com.google.common.collect.Streams; import com.google.common.math.IntMath; import io.airlift.slice.Slice; import io.trino.Session; import io.trino.SystemSessionProperties; +import io.trino.connector.system.GlobalSystemConnector; import io.trino.execution.Column; import io.trino.execution.warnings.WarningCollector; import io.trino.metadata.AnalyzePropertyManager; @@ -104,6 +106,7 @@ import io.trino.sql.InterpretedFunctionInvoker; import io.trino.sql.PlannerContext; import io.trino.sql.analyzer.Analysis.GroupingSetAnalysis; +import io.trino.sql.analyzer.Analysis.JsonTableAnalysis; import io.trino.sql.analyzer.Analysis.MergeAnalysis; import io.trino.sql.analyzer.Analysis.ResolvedWindow; import io.trino.sql.analyzer.Analysis.SelectExpression; @@ -111,6 +114,9 @@ import io.trino.sql.analyzer.Analysis.TableArgumentAnalysis; import io.trino.sql.analyzer.Analysis.TableFunctionInvocationAnalysis; import io.trino.sql.analyzer.Analysis.UnnestAnalysis; +import io.trino.sql.analyzer.ExpressionAnalyzer.ParametersTypeAndAnalysis; +import io.trino.sql.analyzer.ExpressionAnalyzer.TypeAndAnalysis; +import io.trino.sql.analyzer.JsonPathAnalyzer.JsonPathAnalysis; import io.trino.sql.analyzer.PatternRecognitionAnalyzer.PatternRecognitionAnalysis; import io.trino.sql.analyzer.Scope.AsteriskedIdentifierChainBasis; import io.trino.sql.parser.ParsingException; @@ -173,7 +179,11 @@ import io.trino.sql.tree.JoinCriteria; import io.trino.sql.tree.JoinOn; import io.trino.sql.tree.JoinUsing; +import io.trino.sql.tree.JsonPathInvocation; +import io.trino.sql.tree.JsonPathParameter; import io.trino.sql.tree.JsonTable; +import io.trino.sql.tree.JsonTableColumnDefinition; +import io.trino.sql.tree.JsonTableSpecificPlan; import io.trino.sql.tree.Lateral; import io.trino.sql.tree.Limit; import io.trino.sql.tree.LongLiteral; @@ -184,16 +194,23 @@ import io.trino.sql.tree.MergeInsert; import io.trino.sql.tree.MergeUpdate; import io.trino.sql.tree.NaturalJoin; +import io.trino.sql.tree.NestedColumns; import io.trino.sql.tree.Node; +import io.trino.sql.tree.NodeLocation; import io.trino.sql.tree.NodeRef; import io.trino.sql.tree.Offset; import io.trino.sql.tree.OrderBy; +import io.trino.sql.tree.OrdinalityColumn; import io.trino.sql.tree.Parameter; import io.trino.sql.tree.PatternRecognitionRelation; +import io.trino.sql.tree.PlanLeaf; +import io.trino.sql.tree.PlanParentChild; +import io.trino.sql.tree.PlanSiblings; import io.trino.sql.tree.Prepare; import io.trino.sql.tree.Property; import io.trino.sql.tree.QualifiedName; import io.trino.sql.tree.Query; +import io.trino.sql.tree.QueryColumn; import io.trino.sql.tree.QueryPeriod; import io.trino.sql.tree.QuerySpecification; import io.trino.sql.tree.RefreshMaterializedView; @@ -227,6 +244,7 @@ import io.trino.sql.tree.SortItem; import io.trino.sql.tree.StartTransaction; import io.trino.sql.tree.Statement; +import io.trino.sql.tree.StringLiteral; import io.trino.sql.tree.SubqueryExpression; import io.trino.sql.tree.SubscriptExpression; import io.trino.sql.tree.Table; @@ -242,6 +260,7 @@ import io.trino.sql.tree.Update; import io.trino.sql.tree.UpdateAssignment; import io.trino.sql.tree.Use; +import io.trino.sql.tree.ValueColumn; import io.trino.sql.tree.Values; import io.trino.sql.tree.VariableDefinition; import io.trino.sql.tree.Window; @@ -294,6 +313,7 @@ import static io.trino.spi.StandardErrorCode.COLUMN_NOT_FOUND; import static io.trino.spi.StandardErrorCode.COLUMN_TYPE_UNKNOWN; import static io.trino.spi.StandardErrorCode.DUPLICATE_COLUMN_NAME; +import static io.trino.spi.StandardErrorCode.DUPLICATE_COLUMN_OR_PATH_NAME; import static io.trino.spi.StandardErrorCode.DUPLICATE_NAMED_QUERY; import static io.trino.spi.StandardErrorCode.DUPLICATE_PROPERTY; import static io.trino.spi.StandardErrorCode.DUPLICATE_RANGE_VARIABLE; @@ -312,6 +332,7 @@ import static io.trino.spi.StandardErrorCode.INVALID_LIMIT_CLAUSE; import static io.trino.spi.StandardErrorCode.INVALID_ORDER_BY; import static io.trino.spi.StandardErrorCode.INVALID_PARTITION_BY; +import static io.trino.spi.StandardErrorCode.INVALID_PLAN; import static io.trino.spi.StandardErrorCode.INVALID_RECURSIVE_REFERENCE; import static io.trino.spi.StandardErrorCode.INVALID_ROW_FILTER; import static io.trino.spi.StandardErrorCode.INVALID_TABLE_FUNCTION_INVOCATION; @@ -324,6 +345,7 @@ import static io.trino.spi.StandardErrorCode.MISSING_COLUMN_NAME; import static io.trino.spi.StandardErrorCode.MISSING_GROUP_BY; import static io.trino.spi.StandardErrorCode.MISSING_ORDER_BY; +import static io.trino.spi.StandardErrorCode.MISSING_PATH_NAME; import static io.trino.spi.StandardErrorCode.MISSING_RETURN_TYPE; import static io.trino.spi.StandardErrorCode.NESTED_RECURSIVE; import static io.trino.spi.StandardErrorCode.NESTED_ROW_PATTERN_RECOGNITION; @@ -363,6 +385,8 @@ import static io.trino.sql.analyzer.AggregationAnalyzer.verifySourceAggregations; import static io.trino.sql.analyzer.Analyzer.verifyNoAggregateWindowOrGroupingFunctions; import static io.trino.sql.analyzer.CanonicalizationAware.canonicalizationAwareKey; +import static io.trino.sql.analyzer.ExpressionAnalyzer.analyzeJsonQueryExpression; +import static io.trino.sql.analyzer.ExpressionAnalyzer.analyzeJsonValueExpression; import static io.trino.sql.analyzer.ExpressionAnalyzer.createConstantAnalyzer; import static io.trino.sql.analyzer.ExpressionTreeUtils.asQualifiedName; import static io.trino.sql.analyzer.ExpressionTreeUtils.extractAggregateFunctions; @@ -3250,6 +3274,17 @@ protected Scope visitJoin(Join node, Optional scope) } } } + else if (isJsonTable(node.getRight())) { + if (criteria != null) { + if (!(criteria instanceof JoinOn) || !((JoinOn) criteria).getExpression().equals(TRUE_LITERAL)) { + throw semanticException( + NOT_SUPPORTED, + criteria instanceof JoinOn ? ((JoinOn) criteria).getExpression() : node, + "%s JOIN involving JSON_TABLE is only supported with condition ON TRUE", + node.getType().name()); + } + } + } else if (node.getType() == FULL) { if (!(criteria instanceof JoinOn) || !((JoinOn) criteria).getExpression().equals(TRUE_LITERAL)) { throw semanticException( @@ -3776,7 +3811,7 @@ private boolean isLateralRelation(Relation node) if (node instanceof AliasedRelation) { return isLateralRelation(((AliasedRelation) node).getRelation()); } - return node instanceof Unnest || node instanceof Lateral; + return node instanceof Unnest || node instanceof Lateral || node instanceof JsonTable; } private boolean isUnnestRelation(Relation node) @@ -3787,6 +3822,14 @@ private boolean isUnnestRelation(Relation node) return node instanceof Unnest; } + private boolean isJsonTable(Relation node) + { + if (node instanceof AliasedRelation) { + return isJsonTable(((AliasedRelation) node).getRelation()); + } + return node instanceof JsonTable; + } + @Override protected Scope visitValues(Values node, Optional scope) { @@ -3862,9 +3905,254 @@ else if (actualType instanceof RowType) { } @Override - protected Scope visitJsonTable(JsonTable node, Optional context) + protected Scope visitJsonTable(JsonTable node, Optional scope) + { + Scope enclosingScope = createScope(scope); + + // analyze the context item, the root JSON path, and the path parameters + RowType parametersType = analyzeJsonPathInvocation(node, enclosingScope); + + // json_table is implemented as a table function provided by the global catalog. + CatalogHandle catalogHandle = getRequiredCatalogHandle(metadata, session, node, GlobalSystemConnector.NAME); + ConnectorTransactionHandle transactionHandle = transactionManager.getConnectorTransaction(session.getRequiredTransactionId(), catalogHandle); + + // all column and path names must be unique + Set uniqueNames = new HashSet<>(); + JsonPathInvocation rootPath = node.getJsonPathInvocation(); + rootPath.getPathName().ifPresent(name -> uniqueNames.add(name.getCanonicalValue())); + + ImmutableList.Builder outputFields = ImmutableList.builder(); + ImmutableList.Builder> orderedOutputColumns = ImmutableList.builder(); + analyzeJsonTableColumns(node.getColumns(), uniqueNames, outputFields, orderedOutputColumns, enclosingScope, node); + + analysis.addJsonTableAnalysis(node, new JsonTableAnalysis(catalogHandle, transactionHandle, parametersType, orderedOutputColumns.build())); + + node.getPlan().ifPresent(plan -> { + if (plan instanceof JsonTableSpecificPlan specificPlan) { + validateJsonTableSpecificPlan(rootPath, specificPlan, node.getColumns()); + } + else { + // if PLAN DEFAULT is specified, all nested paths should be named + checkAllNestedPathsNamed(node.getColumns()); + } + }); + + return createAndAssignScope(node, scope, outputFields.build()); + } + + private RowType analyzeJsonPathInvocation(JsonTable node, Scope scope) + { + verifyNoAggregateWindowOrGroupingFunctions(session, functionResolver, accessControl, node.getJsonPathInvocation().getInputExpression(), "JSON_TABLE input expression"); + node.getJsonPathInvocation().getPathParameters().stream() + .map(JsonPathParameter::getParameter) + .forEach(parameter -> verifyNoAggregateWindowOrGroupingFunctions(session, functionResolver, accessControl, parameter, "JSON_TABLE path parameter")); + + ParametersTypeAndAnalysis parametersTypeAndAnalysis = ExpressionAnalyzer.analyzeJsonPathInvocation( + node, + session, + plannerContext, + statementAnalyzerFactory, + accessControl, + scope, + analysis, + WarningCollector.NOOP, + correlationSupport); + // context item and passed path parameters can contain subqueries - the subqueries are recorded under the enclosing JsonTable node + analysis.recordSubqueries(node, parametersTypeAndAnalysis.expressionAnalysis()); + return parametersTypeAndAnalysis.parametersType(); + } + + private void analyzeJsonTableColumns( + List columns, + Set uniqueNames, + ImmutableList.Builder outputFields, + ImmutableList.Builder> orderedOutputColumns, + Scope enclosingScope, + JsonTable jsonTable) + { + for (JsonTableColumnDefinition column : columns) { + if (column instanceof OrdinalityColumn ordinalityColumn) { + String name = ordinalityColumn.getName().getCanonicalValue(); + if (!uniqueNames.add(name)) { + throw semanticException(DUPLICATE_COLUMN_OR_PATH_NAME, ordinalityColumn.getName(), "All column and path names in JSON_TABLE invocation must be unique"); + } + outputFields.add(Field.newUnqualified(name, BIGINT)); + orderedOutputColumns.add(NodeRef.of(ordinalityColumn)); + } + else if (column instanceof ValueColumn valueColumn) { + String name = valueColumn.getName().getCanonicalValue(); + if (!uniqueNames.add(name)) { + throw semanticException(DUPLICATE_COLUMN_OR_PATH_NAME, valueColumn.getName(), "All column and path names in JSON_TABLE invocation must be unique"); + } + valueColumn.getEmptyDefault().ifPresent(expression -> verifyNoAggregateWindowOrGroupingFunctions(session, functionResolver, accessControl, expression, "default expression for JSON_TABLE column")); + valueColumn.getErrorDefault().ifPresent(expression -> verifyNoAggregateWindowOrGroupingFunctions(session, functionResolver, accessControl, expression, "default expression for JSON_TABLE column")); + JsonPathAnalysis pathAnalysis = valueColumn.getJsonPath() + .map(this::analyzeJsonPath) + .orElseGet(() -> analyzeImplicitJsonPath(getImplicitJsonPath(name), valueColumn.getLocation())); + analysis.setJsonPathAnalysis(valueColumn, pathAnalysis); + TypeAndAnalysis typeAndAnalysis = analyzeJsonValueExpression( + valueColumn, + pathAnalysis, + session, + plannerContext, + statementAnalyzerFactory, + accessControl, + enclosingScope, + analysis, + warningCollector, + correlationSupport); + // default values can contain subqueries - the subqueries are recorded under the enclosing JsonTable node + analysis.recordSubqueries(jsonTable, typeAndAnalysis.analysis()); + outputFields.add(Field.newUnqualified(name, typeAndAnalysis.type())); + orderedOutputColumns.add(NodeRef.of(valueColumn)); + } + else if (column instanceof QueryColumn queryColumn) { + String name = queryColumn.getName().getCanonicalValue(); + if (!uniqueNames.add(name)) { + throw semanticException(DUPLICATE_COLUMN_OR_PATH_NAME, queryColumn.getName(), "All column and path names in JSON_TABLE invocation must be unique"); + } + JsonPathAnalysis pathAnalysis = queryColumn.getJsonPath() + .map(this::analyzeJsonPath) + .orElseGet(() -> analyzeImplicitJsonPath(getImplicitJsonPath(name), queryColumn.getLocation())); + analysis.setJsonPathAnalysis(queryColumn, pathAnalysis); + Type type = analyzeJsonQueryExpression(queryColumn, session, plannerContext, statementAnalyzerFactory, accessControl, enclosingScope, analysis, warningCollector); + outputFields.add(Field.newUnqualified(name, type)); + orderedOutputColumns.add(NodeRef.of(queryColumn)); + } + else if (column instanceof NestedColumns nestedColumns) { + nestedColumns.getPathName().ifPresent(name -> { + if (!uniqueNames.add(name.getCanonicalValue())) { + throw semanticException(DUPLICATE_COLUMN_OR_PATH_NAME, name, "All column and path names in JSON_TABLE invocation must be unique"); + } + }); + JsonPathAnalysis pathAnalysis = analyzeJsonPath(nestedColumns.getJsonPath()); + analysis.setJsonPathAnalysis(nestedColumns, pathAnalysis); + analyzeJsonTableColumns(nestedColumns.getColumns(), uniqueNames, outputFields, orderedOutputColumns, enclosingScope, jsonTable); + } + else { + throw new IllegalArgumentException("unexpected type of JSON_TABLE column: " + column.getClass().getSimpleName()); + } + } + } + + private static String getImplicitJsonPath(String name) + { + // TODO the spec misses the path mode. I put 'lax', but it should be confirmed, as the path mode is meaningful for the semantics of the implicit path. + return "lax $.\"" + name.replace("\"", "\"\"") + '"'; + } + + private JsonPathAnalysis analyzeJsonPath(StringLiteral path) + { + return new JsonPathAnalyzer( + plannerContext.getMetadata(), + session, + createConstantAnalyzer(plannerContext, accessControl, session, analysis.getParameters(), WarningCollector.NOOP, analysis.isDescribe())) + .analyzeJsonPath(path, ImmutableMap.of()); + } + + private JsonPathAnalysis analyzeImplicitJsonPath(String path, Optional columnLocation) + { + return new JsonPathAnalyzer( + plannerContext.getMetadata(), + session, + createConstantAnalyzer(plannerContext, accessControl, session, analysis.getParameters(), WarningCollector.NOOP, analysis.isDescribe())) + .analyzeImplicitJsonPath(path, columnLocation.orElseThrow(() -> new IllegalStateException("missing NodeLocation for JSON_TABLE column"))); + } + + private void validateJsonTableSpecificPlan(JsonPathInvocation rootPath, JsonTableSpecificPlan rootPlan, List rootColumns) + { + String rootPathName = rootPath.getPathName() + .orElseThrow(() -> semanticException(MISSING_PATH_NAME, rootPath, "All JSON paths must be named when specific plan is given")) + .getCanonicalValue(); + String rootPlanName; + if (rootPlan instanceof PlanLeaf planLeaf) { + rootPlanName = planLeaf.getName().getCanonicalValue(); + } + else if (rootPlan instanceof PlanParentChild planParentChild) { + rootPlanName = planParentChild.getParent().getName().getCanonicalValue(); + } + else { + throw semanticException(INVALID_PLAN, rootPlan, "JSON_TABLE plan must either be a single path name or it must be rooted in parent-child relationship (OUTER or INNER)"); + } + validateJsonTablePlan(ImmutableMap.of(rootPathName, rootColumns), ImmutableMap.of(rootPlanName, rootPlan), rootPlan); + } + + private void validateJsonTablePlan(Map> actualNodes, Map planNodes, JsonTableSpecificPlan rootPlan) + { + Set unhandledActualNodes = Sets.difference(actualNodes.keySet(), planNodes.keySet()); + if (!unhandledActualNodes.isEmpty()) { + throw semanticException(INVALID_PLAN, rootPlan, "JSON_TABLE plan should contain all JSON paths available at each level of nesting. Paths not included: %s", String.join(", ", unhandledActualNodes)); + } + Set irrelevantPlanChildren = Sets.difference(planNodes.keySet(), actualNodes.keySet()); + if (!irrelevantPlanChildren.isEmpty()) { + throw semanticException(INVALID_PLAN, rootPlan, "JSON_TABLE plan includes unavailable JSON path names: %s", String.join(", ", irrelevantPlanChildren)); + } + + // recurse into child nodes + actualNodes.forEach((name, columns) -> { + JsonTableSpecificPlan plan = planNodes.get(name); + + Map> actualChildren = columns.stream() + .filter(NestedColumns.class::isInstance) + .map(NestedColumns.class::cast) + .collect(toImmutableMap( + child -> child.getPathName() + .orElseThrow(() -> semanticException(MISSING_PATH_NAME, child.getJsonPath(), "All JSON paths must be named when specific plan is given")) + .getCanonicalValue(), + NestedColumns::getColumns)); + + Map planChildren; + if (plan instanceof PlanLeaf) { + planChildren = ImmutableMap.of(); + } + else if (plan instanceof PlanParentChild planParentChild) { + planChildren = new HashMap<>(); + getPlanSiblings(planParentChild.getChild(), planChildren); + } + else { + throw new IllegalStateException("unexpected JSON_TABLE plan node: " + plan.getClass().getSimpleName()); + } + + validateJsonTablePlan(actualChildren, planChildren, rootPlan); + }); + } + + private void getPlanSiblings(JsonTableSpecificPlan plan, Map plansByName) + { + if (plan instanceof PlanLeaf planLeaf) { + if (plansByName.put(planLeaf.getName().getCanonicalValue(), planLeaf) != null) { + throw semanticException(INVALID_PLAN, planLeaf, "Duplicate reference to JSON path name in sibling plan: %s", planLeaf.getName().getCanonicalValue()); + } + } + else if (plan instanceof PlanParentChild planParentChild) { + if (plansByName.put(planParentChild.getParent().getName().getCanonicalValue(), planParentChild) != null) { + throw semanticException(INVALID_PLAN, planParentChild.getParent(), "Duplicate reference to JSON path name in sibling plan: %s", planParentChild.getParent().getName().getCanonicalValue()); + } + } + else if (plan instanceof PlanSiblings planSiblings) { + for (JsonTableSpecificPlan sibling : planSiblings.getSiblings()) { + getPlanSiblings(sibling, plansByName); + } + } + } + + // Per SQL standard ISO/IEC STANDARD 9075-2, p. 453, g), i), and p. 821, 2), b), when PLAN DEFAULT is specified, all nested paths must be named, but the root path does not have to be named. + private void checkAllNestedPathsNamed(List columns) { - throw semanticException(NOT_SUPPORTED, node, "JSON_TABLE is not yet supported"); + List nestedColumns = columns.stream() + .filter(NestedColumns.class::isInstance) + .map(NestedColumns.class::cast) + .collect(toImmutableList()); + + nestedColumns.stream() + .forEach(definition -> { + if (definition.getPathName().isEmpty()) { + throw semanticException(MISSING_PATH_NAME, definition.getJsonPath(), "All nested JSON paths must be named when default plan is given"); + } + }); + + nestedColumns.stream() + .forEach(definition -> checkAllNestedPathsNamed(definition.getColumns())); } private void analyzeWindowDefinitions(QuerySpecification node, Scope scope) diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/RelationPlanner.java b/core/trino-main/src/main/java/io/trino/sql/planner/RelationPlanner.java index a88fa26caf753..cdcc86b8eb3af 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/RelationPlanner.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/RelationPlanner.java @@ -19,14 +19,28 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.ListMultimap; import io.trino.Session; +import io.trino.json.ir.IrJsonPath; +import io.trino.metadata.ResolvedFunction; import io.trino.metadata.TableFunctionHandle; import io.trino.metadata.TableHandle; +import io.trino.operator.table.json.JsonTable.JsonTableFunctionHandle; +import io.trino.operator.table.json.JsonTableColumn; +import io.trino.operator.table.json.JsonTableOrdinalityColumn; +import io.trino.operator.table.json.JsonTablePlanCross; +import io.trino.operator.table.json.JsonTablePlanLeaf; +import io.trino.operator.table.json.JsonTablePlanNode; +import io.trino.operator.table.json.JsonTablePlanSingle; +import io.trino.operator.table.json.JsonTablePlanUnion; +import io.trino.operator.table.json.JsonTableQueryColumn; +import io.trino.operator.table.json.JsonTableValueColumn; import io.trino.spi.connector.ColumnHandle; +import io.trino.spi.function.table.TableArgument; import io.trino.spi.type.RowType; import io.trino.spi.type.Type; import io.trino.sql.ExpressionUtils; import io.trino.sql.PlannerContext; import io.trino.sql.analyzer.Analysis; +import io.trino.sql.analyzer.Analysis.JsonTableAnalysis; import io.trino.sql.analyzer.Analysis.TableArgumentAnalysis; import io.trino.sql.analyzer.Analysis.TableFunctionInvocationAnalysis; import io.trino.sql.analyzer.Analysis.UnnestAnalysis; @@ -34,6 +48,7 @@ import io.trino.sql.analyzer.RelationType; import io.trino.sql.analyzer.Scope; import io.trino.sql.planner.QueryPlanner.PlanAndMappings; +import io.trino.sql.planner.TranslationMap.ParametersRow; import io.trino.sql.planner.plan.Assignments; import io.trino.sql.planner.plan.CorrelatedJoinNode; import io.trino.sql.planner.plan.DataOrganizationSpecification; @@ -61,27 +76,45 @@ import io.trino.sql.planner.rowpattern.ir.IrRowPattern; import io.trino.sql.tree.AliasedRelation; import io.trino.sql.tree.AstVisitor; +import io.trino.sql.tree.BooleanLiteral; import io.trino.sql.tree.Cast; import io.trino.sql.tree.CoalesceExpression; import io.trino.sql.tree.ComparisonExpression; import io.trino.sql.tree.Except; import io.trino.sql.tree.Expression; +import io.trino.sql.tree.FunctionCall; +import io.trino.sql.tree.GenericLiteral; import io.trino.sql.tree.Identifier; import io.trino.sql.tree.IfExpression; import io.trino.sql.tree.Intersect; import io.trino.sql.tree.Join; import io.trino.sql.tree.JoinCriteria; import io.trino.sql.tree.JoinUsing; +import io.trino.sql.tree.JsonPathParameter; +import io.trino.sql.tree.JsonQuery; +import io.trino.sql.tree.JsonTable; +import io.trino.sql.tree.JsonTableColumnDefinition; +import io.trino.sql.tree.JsonTableDefaultPlan; +import io.trino.sql.tree.JsonTablePlan.ParentChildPlanType; +import io.trino.sql.tree.JsonTablePlan.SiblingsPlanType; +import io.trino.sql.tree.JsonTableSpecificPlan; +import io.trino.sql.tree.JsonValue; import io.trino.sql.tree.LambdaArgumentDeclaration; import io.trino.sql.tree.Lateral; import io.trino.sql.tree.MeasureDefinition; import io.trino.sql.tree.NaturalJoin; +import io.trino.sql.tree.NestedColumns; import io.trino.sql.tree.Node; import io.trino.sql.tree.NodeRef; +import io.trino.sql.tree.OrdinalityColumn; import io.trino.sql.tree.PatternRecognitionRelation; import io.trino.sql.tree.PatternSearchMode; +import io.trino.sql.tree.PlanLeaf; +import io.trino.sql.tree.PlanParentChild; +import io.trino.sql.tree.PlanSiblings; import io.trino.sql.tree.QualifiedName; import io.trino.sql.tree.Query; +import io.trino.sql.tree.QueryColumn; import io.trino.sql.tree.QuerySpecification; import io.trino.sql.tree.Relation; import io.trino.sql.tree.Row; @@ -97,6 +130,7 @@ import io.trino.sql.tree.TableSubquery; import io.trino.sql.tree.Union; import io.trino.sql.tree.Unnest; +import io.trino.sql.tree.ValueColumn; import io.trino.sql.tree.Values; import io.trino.sql.tree.VariableDefinition; import io.trino.type.TypeCoercion; @@ -118,6 +152,7 @@ import static io.trino.spi.StandardErrorCode.CONSTRAINT_VIOLATION; import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; import static io.trino.spi.type.BooleanType.BOOLEAN; +import static io.trino.spi.type.StandardTypes.TINYINT; import static io.trino.sql.NodeUtils.getSortItemsFromOrderBy; import static io.trino.sql.analyzer.SemanticExceptions.semanticException; import static io.trino.sql.analyzer.TypeSignatureTranslator.toSqlType; @@ -133,11 +168,18 @@ import static io.trino.sql.planner.plan.AggregationNode.singleGroupingSet; import static io.trino.sql.tree.BooleanLiteral.TRUE_LITERAL; import static io.trino.sql.tree.Join.Type.CROSS; +import static io.trino.sql.tree.Join.Type.FULL; import static io.trino.sql.tree.Join.Type.IMPLICIT; import static io.trino.sql.tree.Join.Type.INNER; +import static io.trino.sql.tree.Join.Type.LEFT; +import static io.trino.sql.tree.JsonQuery.QuotesBehavior.KEEP; +import static io.trino.sql.tree.JsonQuery.QuotesBehavior.OMIT; +import static io.trino.sql.tree.JsonTablePlan.ParentChildPlanType.OUTER; +import static io.trino.sql.tree.JsonTablePlan.SiblingsPlanType.UNION; import static io.trino.sql.tree.PatternRecognitionRelation.RowsPerMatch.ONE; import static io.trino.sql.tree.PatternSearchMode.Mode.INITIAL; import static io.trino.sql.tree.SkipTo.Position.PAST_LAST; +import static io.trino.type.Json2016Type.JSON_2016; import static java.lang.Boolean.TRUE; import static java.util.Locale.ENGLISH; import static java.util.Objects.requireNonNull; @@ -685,6 +727,16 @@ protected RelationPlan visitJoin(Join node, Void context) return planJoinUnnest(leftPlan, node, unnest.get()); } + Optional jsonTable = getJsonTable(node.getRight()); + if (jsonTable.isPresent()) { + return planJoinJsonTable( + newPlanBuilder(leftPlan, analysis, lambdaDeclarationToSymbolMap, session, plannerContext), + leftPlan.getFieldMappings(), + node.getType(), + jsonTable.get(), + analysis.getScope(node)); + } + Optional lateral = getLateral(node.getRight()); if (lateral.isPresent()) { return planCorrelatedJoin(node, leftPlan, lateral.get()); @@ -1002,6 +1054,17 @@ private static Optional getUnnest(Relation relation) return Optional.empty(); } + private static Optional getJsonTable(Relation relation) + { + if (relation instanceof AliasedRelation) { + return getJsonTable(((AliasedRelation) relation).getRelation()); + } + if (relation instanceof JsonTable) { + return Optional.of((JsonTable) relation); + } + return Optional.empty(); + } + private static Optional getLateral(Relation relation) { if (relation instanceof AliasedRelation) { @@ -1124,6 +1187,393 @@ private RelationPlan planUnnest(PlanBuilder subPlan, Unnest node, List r return new RelationPlan(unnestNode, outputScope, unnestNode.getOutputSymbols(), outerContext); } + private RelationPlan planJoinJsonTable(PlanBuilder leftPlan, List leftFieldMappings, Join.Type joinType, JsonTable jsonTable, Scope outputScope) + { + PlanBuilder planBuilder = leftPlan; + + // extract input expressions + ImmutableList.Builder builder = ImmutableList.builder(); + Expression inputExpression = jsonTable.getJsonPathInvocation().getInputExpression(); + builder.add(inputExpression); + List pathParameters = jsonTable.getJsonPathInvocation().getPathParameters(); + pathParameters.stream() + .map(JsonPathParameter::getParameter) + .forEach(builder::add); + List defaultExpressions = getDefaultExpressions(jsonTable.getColumns()); + builder.addAll(defaultExpressions); + List inputExpressions = builder.build(); + + planBuilder = subqueryPlanner.handleSubqueries(planBuilder, inputExpressions, analysis.getSubqueries(jsonTable)); + planBuilder = planBuilder.appendProjections(inputExpressions, symbolAllocator, idAllocator); + + // apply coercions + // coercions might be necessary for the context item and path parameters before the input functions are applied + // also, the default expressions in value columns (DEFAULT ... ON EMPTY / ON ERROR) might need a coercion to match the required output type + PlanAndMappings coerced = coerce(planBuilder, inputExpressions, analysis, idAllocator, symbolAllocator, typeCoercion); + planBuilder = coerced.getSubPlan(); + + // apply the input function to the input expression + BooleanLiteral failOnError = new BooleanLiteral(jsonTable.getErrorBehavior().orElse(JsonTable.ErrorBehavior.EMPTY) == JsonTable.ErrorBehavior.ERROR ? "true" : "false"); + ResolvedFunction inputToJson = analysis.getJsonInputFunction(inputExpression); + Expression inputJson = new FunctionCall(inputToJson.toQualifiedName(), ImmutableList.of(coerced.get(inputExpression).toSymbolReference(), failOnError)); + + // apply the input functions to the JSON path parameters having FORMAT, + // and collect all JSON path parameters in a Row + List coercedParameters = pathParameters.stream() + .map(parameter -> new JsonPathParameter( + parameter.getLocation(), + parameter.getName(), + coerced.get(parameter.getParameter()).toSymbolReference(), + parameter.getFormat())) + .collect(toImmutableList()); + JsonTableAnalysis jsonTableAnalysis = analysis.getJsonTableAnalysis(jsonTable); + RowType parametersType = jsonTableAnalysis.parametersType(); + ParametersRow orderedParameters = planBuilder.getTranslations().getParametersRow(pathParameters, coercedParameters, parametersType, failOnError); + Expression parametersRow = orderedParameters.getParametersRow(); + + // append projections for inputJson and parametersRow + // cannot use the 'appendProjections()' method because the projected expressions include resolved input functions, so they are not pure AST expressions + Symbol inputJsonSymbol = symbolAllocator.newSymbol("inputJson", JSON_2016); + Symbol parametersRowSymbol = symbolAllocator.newSymbol("parametersRow", parametersType); + ProjectNode appended = new ProjectNode( + idAllocator.getNextId(), + planBuilder.getRoot(), + Assignments.builder() + .putIdentities(planBuilder.getRoot().getOutputSymbols()) + .put(inputJsonSymbol, inputJson) + .put(parametersRowSymbol, parametersRow) + .build()); + planBuilder = planBuilder.withNewRoot(appended); + + // identify the required symbols + ImmutableList.Builder requiredSymbolsBuilder = ImmutableList.builder() + .add(inputJsonSymbol) + .add(parametersRowSymbol); + defaultExpressions.stream() + .map(coerced::get) + .distinct() + .forEach(requiredSymbolsBuilder::add); + List requiredSymbols = requiredSymbolsBuilder.build(); + + // map the default expressions of value columns to indexes in the required columns list + // use a HashMap because there might be duplicate expressions + Map defaultExpressionsMapping = new HashMap<>(); + for (Expression defaultExpression : defaultExpressions) { + defaultExpressionsMapping.put(defaultExpression, requiredSymbols.indexOf(coerced.get(defaultExpression))); + } + + // rewrite the root JSON path to IR using parameters + IrJsonPath rootPath = new JsonPathTranslator(session, plannerContext).rewriteToIr(analysis.getJsonPathAnalysis(jsonTable), orderedParameters.getParametersOrder()); + + // create json_table execution plan + List> orderedColumns = jsonTableAnalysis.orderedOutputColumns(); + Map, Integer> outputIndexMapping = IntStream.range(0, orderedColumns.size()) + .boxed() + .collect(toImmutableMap(orderedColumns::get, Function.identity())); + JsonTablePlanNode executionPlan; + boolean defaultErrorOnError = jsonTable.getErrorBehavior().map(errorBehavior -> errorBehavior == JsonTable.ErrorBehavior.ERROR).orElse(false); + if (jsonTable.getPlan().isEmpty()) { + executionPlan = getPlanFromDefaults(rootPath, jsonTable.getColumns(), OUTER, UNION, defaultErrorOnError, outputIndexMapping, defaultExpressionsMapping); + } + else if (jsonTable.getPlan().orElseThrow() instanceof JsonTableDefaultPlan defaultPlan) { + executionPlan = getPlanFromDefaults(rootPath, jsonTable.getColumns(), defaultPlan.getParentChild(), defaultPlan.getSiblings(), defaultErrorOnError, outputIndexMapping, defaultExpressionsMapping); + } + else { + executionPlan = getPlanFromSpecification(rootPath, jsonTable.getColumns(), (JsonTableSpecificPlan) jsonTable.getPlan().orElseThrow(), defaultErrorOnError, outputIndexMapping, defaultExpressionsMapping); + } + + // create new symbols for json_table function's proper columns + // These are the types produced by the table function. + // For ordinality and value columns, the types match the expected output type. + // Query columns return JSON_2016. Later we need to apply an output function, and potentially a coercion to match the declared output type. + RelationType jsonTableRelationType = analysis.getScope(jsonTable).getRelationType(); + List properOutputs = IntStream.range(0, orderedColumns.size()) + .mapToObj(index -> { + if (orderedColumns.get(index).getNode() instanceof QueryColumn queryColumn) { + return symbolAllocator.newSymbol(queryColumn.getName().getCanonicalValue(), JSON_2016); + } + return symbolAllocator.newSymbol(jsonTableRelationType.getFieldByIndex(index)); + }) + .collect(toImmutableList()); + + // pass through all columns from the left side of the join + List passThroughColumns = leftFieldMappings.stream() + .map(symbol -> new PassThroughColumn(symbol, false)) + .collect(toImmutableList()); + + // determine the join type between the input, and the json_table result + // this join type is not described in the plan, it depends on the enclosing join whose right source is the json_table + // since json_table is a lateral relation, and the join condition is 'true', effectively the join type is either LEFT OUTER or INNER + boolean outer = joinType == LEFT || joinType == FULL; + + // create the TableFunctionNode and TableFunctionHandle + JsonTableFunctionHandle functionHandle = new JsonTableFunctionHandle( + executionPlan, + outer, + defaultErrorOnError, + parametersType, + properOutputs.stream() + .map(symbolAllocator.getTypes()::get) + .toArray(Type[]::new)); + + TableFunctionNode tableFunctionNode = new TableFunctionNode( + idAllocator.getNextId(), + "$json_table", + jsonTableAnalysis.catalogHandle(), + ImmutableMap.of("$input", new TableArgument(getRowType(planBuilder.getRoot()), ImmutableList.of(), ImmutableList.of())), + properOutputs, + ImmutableList.of(planBuilder.getRoot()), + ImmutableList.of(new TableArgumentProperties( + "$input", + true, + true, + new PassThroughSpecification(true, passThroughColumns), + requiredSymbols, + Optional.empty())), + ImmutableList.of(), + new TableFunctionHandle( + jsonTableAnalysis.catalogHandle(), + functionHandle, + jsonTableAnalysis.transactionHandle())); + + // append output functions and coercions for query columns + // The table function returns JSON_2016 for query columns. We need to apply output functions and coercions to match the declared output type. + // create output layout: first the left side of the join, next the proper columns + ImmutableList.Builder outputLayout = ImmutableList.builder() + .addAll(leftFieldMappings); + Assignments.Builder assignments = Assignments.builder() + .putIdentities(leftFieldMappings); + for (int i = 0; i < properOutputs.size(); i++) { + Symbol properOutput = properOutputs.get(i); + if (orderedColumns.get(i).getNode() instanceof QueryColumn queryColumn) { + // apply output function + GenericLiteral errorBehavior = new GenericLiteral( + TINYINT, + String.valueOf(queryColumn.getErrorBehavior().orElse(defaultErrorOnError ? JsonQuery.EmptyOrErrorBehavior.ERROR : JsonQuery.EmptyOrErrorBehavior.NULL).ordinal())); + BooleanLiteral omitQuotes = new BooleanLiteral(queryColumn.getQuotesBehavior().orElse(KEEP) == OMIT ? "true" : "false"); + ResolvedFunction outputFunction = analysis.getJsonOutputFunction(queryColumn); + Expression result = new FunctionCall(outputFunction.toQualifiedName(), ImmutableList.of(properOutput.toSymbolReference(), errorBehavior, omitQuotes)); + + // cast to declared returned type + Type expectedType = jsonTableRelationType.getFieldByIndex(i).getType(); + Type resultType = outputFunction.getSignature().getReturnType(); + if (!resultType.equals(expectedType)) { + result = new Cast(result, toSqlType(expectedType)); + } + + Symbol output = symbolAllocator.newSymbol(result, expectedType); + outputLayout.add(output); + assignments.put(output, result); + } + else { + outputLayout.add(properOutput); + assignments.putIdentity(properOutput); + } + } + + ProjectNode projectNode = new ProjectNode( + idAllocator.getNextId(), + tableFunctionNode, + assignments.build()); + + return new RelationPlan(projectNode, outputScope, outputLayout.build(), outerContext); + } + + private static List getDefaultExpressions(List columns) + { + ImmutableList.Builder builder = ImmutableList.builder(); + for (JsonTableColumnDefinition column : columns) { + if (column instanceof ValueColumn valueColumn) { + valueColumn.getEmptyDefault().ifPresent(builder::add); + valueColumn.getErrorDefault().ifPresent(builder::add); + } + else if (column instanceof NestedColumns nestedColumns) { + builder.addAll(getDefaultExpressions(nestedColumns.getColumns())); + } + } + return builder.build(); + } + + private JsonTablePlanNode getPlanFromDefaults( + IrJsonPath path, + List columnDefinitions, + ParentChildPlanType parentChildPlanType, + SiblingsPlanType siblingsPlanType, + boolean defaultErrorOnError, + Map, Integer> outputIndexMapping, + Map defaultExpressionsMapping) + { + ImmutableList.Builder columns = ImmutableList.builder(); + ImmutableList.Builder childrenBuilder = ImmutableList.builder(); + + for (JsonTableColumnDefinition columnDefinition : columnDefinitions) { + if (columnDefinition instanceof NestedColumns nestedColumns) { + IrJsonPath nestedPath = new JsonPathTranslator(session, plannerContext).rewriteToIr(analysis.getJsonPathAnalysis(nestedColumns), ImmutableList.of()); + childrenBuilder.add(getPlanFromDefaults( + nestedPath, + nestedColumns.getColumns(), + parentChildPlanType, + siblingsPlanType, + defaultErrorOnError, + outputIndexMapping, + defaultExpressionsMapping)); + } + else { + columns.add(getColumn(columnDefinition, defaultErrorOnError, outputIndexMapping, defaultExpressionsMapping)); + } + } + + List children = childrenBuilder.build(); + if (children.isEmpty()) { + return new JsonTablePlanLeaf(path, columns.build()); + } + + JsonTablePlanNode child; + if (children.size() == 1) { + child = getOnlyElement(children); + } + else if (siblingsPlanType == UNION) { + child = new JsonTablePlanUnion(children); + } + else { + child = new JsonTablePlanCross(children); + } + + return new JsonTablePlanSingle(path, columns.build(), parentChildPlanType == OUTER, child); + } + + private JsonTablePlanNode getPlanFromSpecification( + IrJsonPath path, + List columnDefinitions, + JsonTableSpecificPlan specificPlan, + boolean defaultErrorOnError, + Map, Integer> outputIndexMapping, + Map defaultExpressionsMapping) + { + ImmutableList.Builder columns = ImmutableList.builder(); + ImmutableMap.Builder childrenBuilder = ImmutableMap.builder(); + Map planSiblings; + if (specificPlan instanceof PlanLeaf) { + planSiblings = ImmutableMap.of(); + } + else { + planSiblings = getSiblings(((PlanParentChild) specificPlan).getChild()); + } + + for (JsonTableColumnDefinition columnDefinition : columnDefinitions) { + if (columnDefinition instanceof NestedColumns nestedColumns) { + IrJsonPath nestedPath = new JsonPathTranslator(session, plannerContext).rewriteToIr(analysis.getJsonPathAnalysis(nestedColumns), ImmutableList.of()); + String nestedPathName = nestedColumns.getPathName().orElseThrow().getCanonicalValue(); + JsonTablePlanNode child = getPlanFromSpecification( + nestedPath, + nestedColumns.getColumns(), + planSiblings.get(nestedPathName), + defaultErrorOnError, + outputIndexMapping, + defaultExpressionsMapping); + childrenBuilder.put(nestedPathName, child); + } + else { + columns.add(getColumn(columnDefinition, defaultErrorOnError, outputIndexMapping, defaultExpressionsMapping)); + } + } + + Map children = childrenBuilder.buildOrThrow(); + if (children.isEmpty()) { + return new JsonTablePlanLeaf(path, columns.build()); + } + + PlanParentChild planParentChild = (PlanParentChild) specificPlan; + boolean outer = planParentChild.getType() == OUTER; + JsonTablePlanNode child = combineSiblings(children, planParentChild.getChild()); + return new JsonTablePlanSingle(path, columns.build(), outer, child); + } + + private Map getSiblings(JsonTableSpecificPlan plan) + { + if (plan instanceof PlanLeaf planLeaf) { + return ImmutableMap.of(planLeaf.getName().getCanonicalValue(), planLeaf); + } + if (plan instanceof PlanParentChild planParentChild) { + return ImmutableMap.of(planParentChild.getParent().getName().getCanonicalValue(), planParentChild); + } + PlanSiblings planSiblings = (PlanSiblings) plan; + ImmutableMap.Builder siblings = ImmutableMap.builder(); + for (JsonTableSpecificPlan sibling : planSiblings.getSiblings()) { + siblings.putAll(getSiblings(sibling)); + } + return siblings.buildOrThrow(); + } + + private JsonTableColumn getColumn( + JsonTableColumnDefinition columnDefinition, + boolean defaultErrorOnError, + Map, Integer> outputIndexMapping, + Map defaultExpressionsMapping) + { + int index = outputIndexMapping.get(NodeRef.of(columnDefinition)); + + if (columnDefinition instanceof OrdinalityColumn) { + return new JsonTableOrdinalityColumn(index); + } + ResolvedFunction columnFunction = analysis.getResolvedFunction(columnDefinition); + IrJsonPath columnPath = new JsonPathTranslator(session, plannerContext).rewriteToIr(analysis.getJsonPathAnalysis(columnDefinition), ImmutableList.of()); + if (columnDefinition instanceof QueryColumn queryColumn) { + return new JsonTableQueryColumn( + index, + columnFunction, + columnPath, + queryColumn.getWrapperBehavior().ordinal(), + queryColumn.getEmptyBehavior().ordinal(), + queryColumn.getErrorBehavior().orElse(defaultErrorOnError ? JsonQuery.EmptyOrErrorBehavior.ERROR : JsonQuery.EmptyOrErrorBehavior.NULL).ordinal()); + } + if (columnDefinition instanceof ValueColumn valueColumn) { + int emptyDefault = valueColumn.getEmptyDefault() + .map(defaultExpressionsMapping::get) + .orElse(-1); + int errorDefault = valueColumn.getErrorDefault() + .map(defaultExpressionsMapping::get) + .orElse(-1); + return new JsonTableValueColumn( + index, + columnFunction, + columnPath, + valueColumn.getEmptyBehavior().ordinal(), + emptyDefault, + valueColumn.getErrorBehavior().orElse(defaultErrorOnError ? JsonValue.EmptyOrErrorBehavior.ERROR : JsonValue.EmptyOrErrorBehavior.NULL).ordinal(), + errorDefault); + } + throw new IllegalStateException("unexpected column definition: " + columnDefinition.getClass().getSimpleName()); + } + + private JsonTablePlanNode combineSiblings(Map siblings, JsonTableSpecificPlan plan) + { + if (plan instanceof PlanLeaf planLeaf) { + return siblings.get(planLeaf.getName().getCanonicalValue()); + } + if (plan instanceof PlanParentChild planParentChild) { + return siblings.get(planParentChild.getParent().getName().getCanonicalValue()); + } + PlanSiblings planSiblings = (PlanSiblings) plan; + List siblingNodes = planSiblings.getSiblings().stream() + .map(sibling -> combineSiblings(siblings, sibling)) + .collect(toImmutableList()); + if (planSiblings.getType() == UNION) { + return new JsonTablePlanUnion(siblingNodes); + } + return new JsonTablePlanCross(siblingNodes); + } + + private RowType getRowType(PlanNode node) + { + // create a RowType based on output symbols of a node + // The node is an intermediate stage of planning json_table. There's no recorded relation type available for this node. + // The returned RowType is only used in plan printer + return RowType.from(node.getOutputSymbols().stream() + .map(symbol -> new RowType.Field(Optional.of(symbol.getName()), symbolAllocator.getTypes().get(symbol))) + .collect(toImmutableList())); + } + @Override protected RelationPlan visitTableSubquery(TableSubquery node, Void context) { @@ -1206,6 +1656,17 @@ private PlanBuilder planSingleEmptyRow(Optional parent) return new PlanBuilder(translations, values); } + @Override + protected RelationPlan visitJsonTable(JsonTable node, Void context) + { + return planJoinJsonTable( + planSingleEmptyRow(analysis.getScope(node).getOuterQueryParent()), + ImmutableList.of(), + INNER, + node, + analysis.getScope(node)); + } + @Override protected RelationPlan visitUnion(Union node, Void context) { diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/ResolvedFunctionCallRewriter.java b/core/trino-main/src/main/java/io/trino/sql/planner/ResolvedFunctionCallRewriter.java index 2a5e457e611ac..8b2c6c9851064 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/ResolvedFunctionCallRewriter.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/ResolvedFunctionCallRewriter.java @@ -18,6 +18,7 @@ import io.trino.sql.tree.ExpressionRewriter; import io.trino.sql.tree.ExpressionTreeRewriter; import io.trino.sql.tree.FunctionCall; +import io.trino.sql.tree.Node; import io.trino.sql.tree.NodeRef; import java.util.Map; @@ -29,7 +30,7 @@ public final class ResolvedFunctionCallRewriter { private ResolvedFunctionCallRewriter() {} - public static Expression rewriteResolvedFunctions(Expression expression, Map, ResolvedFunction> resolvedFunctions) + public static Expression rewriteResolvedFunctions(Expression expression, Map, ResolvedFunction> resolvedFunctions) { return ExpressionTreeRewriter.rewriteWith(new Visitor(resolvedFunctions), expression); } @@ -37,9 +38,9 @@ public static Expression rewriteResolvedFunctions(Expression expression, Map { - private final Map, ResolvedFunction> resolvedFunctions; + private final Map, ResolvedFunction> resolvedFunctions; - public Visitor(Map, ResolvedFunction> resolvedFunctions) + public Visitor(Map, ResolvedFunction> resolvedFunctions) { this.resolvedFunctions = requireNonNull(resolvedFunctions, "resolvedFunctions is null"); } diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/TranslationMap.java b/core/trino-main/src/main/java/io/trino/sql/planner/TranslationMap.java index 9e5ec42b8aa4e..29de734e88044 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/TranslationMap.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/TranslationMap.java @@ -913,40 +913,6 @@ public Expression rewriteJsonQuery(JsonQuery node, Void context, ExpressionTreeR return coerceIfNecessary(node, result); } - private ParametersRow getParametersRow( - List pathParameters, - List rewrittenPathParameters, - Type parameterRowType, - BooleanLiteral failOnError) - { - Expression parametersRow; - List parametersOrder; - if (!pathParameters.isEmpty()) { - ImmutableList.Builder parameters = ImmutableList.builder(); - for (int i = 0; i < pathParameters.size(); i++) { - ResolvedFunction parameterToJson = analysis.getJsonInputFunction(pathParameters.get(i).getParameter()); - Expression rewrittenParameter = rewrittenPathParameters.get(i).getParameter(); - if (parameterToJson != null) { - parameters.add(new FunctionCall(parameterToJson.toQualifiedName(), ImmutableList.of(rewrittenParameter, failOnError))); - } - else { - parameters.add(rewrittenParameter); - } - } - parametersRow = new Cast(new Row(parameters.build()), toSqlType(parameterRowType)); - parametersOrder = pathParameters.stream() - .map(parameter -> parameter.getName().getCanonicalValue()) - .collect(toImmutableList()); - } - else { - checkState(JSON_NO_PARAMETERS_ROW_TYPE.equals(parameterRowType), "invalid type of parameters row when no parameters are passed"); - parametersRow = new Cast(new NullLiteral(), toSqlType(JSON_NO_PARAMETERS_ROW_TYPE)); - parametersOrder = ImmutableList.of(); - } - - return new ParametersRow(parametersRow, parametersOrder); - } - @Override public Expression rewriteJsonObject(JsonObject node, Void context, ExpressionTreeRewriter treeRewriter) { @@ -1132,7 +1098,41 @@ public Scope getScope() return scope; } - private static class ParametersRow + public ParametersRow getParametersRow( + List pathParameters, + List rewrittenPathParameters, + Type parameterRowType, + BooleanLiteral failOnError) + { + Expression parametersRow; + List parametersOrder; + if (!pathParameters.isEmpty()) { + ImmutableList.Builder parameters = ImmutableList.builder(); + for (int i = 0; i < pathParameters.size(); i++) { + ResolvedFunction parameterToJson = analysis.getJsonInputFunction(pathParameters.get(i).getParameter()); + Expression rewrittenParameter = rewrittenPathParameters.get(i).getParameter(); + if (parameterToJson != null) { + parameters.add(new FunctionCall(parameterToJson.toQualifiedName(), ImmutableList.of(rewrittenParameter, failOnError))); + } + else { + parameters.add(rewrittenParameter); + } + } + parametersRow = new Cast(new Row(parameters.build()), toSqlType(parameterRowType)); + parametersOrder = pathParameters.stream() + .map(parameter -> parameter.getName().getCanonicalValue()) + .collect(toImmutableList()); + } + else { + checkState(JSON_NO_PARAMETERS_ROW_TYPE.equals(parameterRowType), "invalid type of parameters row when no parameters are passed"); + parametersRow = new Cast(new NullLiteral(), toSqlType(JSON_NO_PARAMETERS_ROW_TYPE)); + parametersOrder = ImmutableList.of(); + } + + return new ParametersRow(parametersRow, parametersOrder); + } + + public static class ParametersRow { private final Expression parametersRow; private final List parametersOrder; diff --git a/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java b/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java index e922227cca368..36c2514e6eb5c 100644 --- a/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java +++ b/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java @@ -378,7 +378,10 @@ private LocalQueryRunner( TypeManager typeManager = new InternalTypeManager(typeRegistry); InternalBlockEncodingSerde blockEncodingSerde = new InternalBlockEncodingSerde(blockEncodingManager, typeManager); - this.globalFunctionCatalog = new GlobalFunctionCatalog(); + this.globalFunctionCatalog = new GlobalFunctionCatalog( + this::getMetadata, + this::getTypeManager, + this::getFunctionManager); globalFunctionCatalog.addFunctions(new InternalFunctionBundle(new LiteralFunction(blockEncodingSerde))); globalFunctionCatalog.addFunctions(SystemFunctionBundle.create(featuresConfig, typeOperators, blockTypeOperators, nodeManager.getCurrentNode().getNodeVersion())); this.groupProvider = new TestingGroupProviderManager(); diff --git a/core/trino-main/src/test/java/io/trino/dispatcher/TestLocalDispatchQuery.java b/core/trino-main/src/test/java/io/trino/dispatcher/TestLocalDispatchQuery.java index d74a931f7b1e9..3a0b59b5b1955 100644 --- a/core/trino-main/src/test/java/io/trino/dispatcher/TestLocalDispatchQuery.java +++ b/core/trino-main/src/test/java/io/trino/dispatcher/TestLocalDispatchQuery.java @@ -133,7 +133,10 @@ public void testSubmittedForDispatchedQuery() metadata, new FunctionManager( new ConnectorCatalogServiceProvider<>("function provider", new NoConnectorServicesProvider(), ConnectorServices::getFunctionProvider), - new GlobalFunctionCatalog(), + new GlobalFunctionCatalog( + () -> { throw new UnsupportedOperationException(); }, + () -> { throw new UnsupportedOperationException(); }, + () -> { throw new UnsupportedOperationException(); }), LanguageFunctionProvider.DISABLED), new QueryMonitorConfig()); CreateTable createTable = new CreateTable(QualifiedName.of("table"), ImmutableList.of(), FAIL, ImmutableList.of(), Optional.empty()); diff --git a/core/trino-main/src/test/java/io/trino/metadata/TestGlobalFunctionCatalog.java b/core/trino-main/src/test/java/io/trino/metadata/TestGlobalFunctionCatalog.java index 8c756bd8da6d5..2c6a42d9e975c 100644 --- a/core/trino-main/src/test/java/io/trino/metadata/TestGlobalFunctionCatalog.java +++ b/core/trino-main/src/test/java/io/trino/metadata/TestGlobalFunctionCatalog.java @@ -100,7 +100,10 @@ public void testDuplicateFunctions() FunctionBundle functionBundle = extractFunctions(CustomAdd.class); TypeOperators typeOperators = new TypeOperators(); - GlobalFunctionCatalog globalFunctionCatalog = new GlobalFunctionCatalog(); + GlobalFunctionCatalog globalFunctionCatalog = new GlobalFunctionCatalog( + () -> { throw new UnsupportedOperationException(); }, + () -> { throw new UnsupportedOperationException(); }, + () -> { throw new UnsupportedOperationException(); }); globalFunctionCatalog.addFunctions(SystemFunctionBundle.create(new FeaturesConfig(), typeOperators, new BlockTypeOperators(typeOperators), NodeVersion.UNKNOWN)); globalFunctionCatalog.addFunctions(functionBundle); assertThatThrownBy(() -> globalFunctionCatalog.addFunctions(functionBundle)) @@ -114,7 +117,10 @@ public void testConflictingScalarAggregation() FunctionBundle functions = extractFunctions(ScalarSum.class); TypeOperators typeOperators = new TypeOperators(); - GlobalFunctionCatalog globalFunctionCatalog = new GlobalFunctionCatalog(); + GlobalFunctionCatalog globalFunctionCatalog = new GlobalFunctionCatalog( + () -> { throw new UnsupportedOperationException(); }, + () -> { throw new UnsupportedOperationException(); }, + () -> { throw new UnsupportedOperationException(); }); globalFunctionCatalog.addFunctions(SystemFunctionBundle.create(new FeaturesConfig(), typeOperators, new BlockTypeOperators(typeOperators), NodeVersion.UNKNOWN)); assertThatThrownBy(() -> globalFunctionCatalog.addFunctions(functions)) .isInstanceOf(IllegalStateException.class) diff --git a/core/trino-main/src/test/java/io/trino/sql/analyzer/TestAnalyzer.java b/core/trino-main/src/test/java/io/trino/sql/analyzer/TestAnalyzer.java index 613eb7ebbdf37..8478d856e9c2f 100644 --- a/core/trino-main/src/test/java/io/trino/sql/analyzer/TestAnalyzer.java +++ b/core/trino-main/src/test/java/io/trino/sql/analyzer/TestAnalyzer.java @@ -118,6 +118,7 @@ import static io.trino.spi.StandardErrorCode.COLUMN_NOT_FOUND; import static io.trino.spi.StandardErrorCode.COLUMN_TYPE_UNKNOWN; import static io.trino.spi.StandardErrorCode.DUPLICATE_COLUMN_NAME; +import static io.trino.spi.StandardErrorCode.DUPLICATE_COLUMN_OR_PATH_NAME; import static io.trino.spi.StandardErrorCode.DUPLICATE_NAMED_QUERY; import static io.trino.spi.StandardErrorCode.DUPLICATE_PARAMETER_NAME; import static io.trino.spi.StandardErrorCode.DUPLICATE_PROPERTY; @@ -143,6 +144,7 @@ import static io.trino.spi.StandardErrorCode.INVALID_PARTITION_BY; import static io.trino.spi.StandardErrorCode.INVALID_PATH; import static io.trino.spi.StandardErrorCode.INVALID_PATTERN_RECOGNITION_FUNCTION; +import static io.trino.spi.StandardErrorCode.INVALID_PLAN; import static io.trino.spi.StandardErrorCode.INVALID_PROCESSING_MODE; import static io.trino.spi.StandardErrorCode.INVALID_RANGE; import static io.trino.spi.StandardErrorCode.INVALID_RECURSIVE_REFERENCE; @@ -160,6 +162,7 @@ import static io.trino.spi.StandardErrorCode.MISSING_GROUP_BY; import static io.trino.spi.StandardErrorCode.MISSING_ORDER_BY; import static io.trino.spi.StandardErrorCode.MISSING_OVER; +import static io.trino.spi.StandardErrorCode.MISSING_PATH_NAME; import static io.trino.spi.StandardErrorCode.MISSING_ROW_PATTERN; import static io.trino.spi.StandardErrorCode.MISSING_SCHEMA_NAME; import static io.trino.spi.StandardErrorCode.MISSING_VARIABLE_DEFINITIONS; @@ -6738,11 +6741,541 @@ public void testTableFunctionRequiredColumns() } @Test - public void testJsonTable() + public void testJsonTableColumnTypes() { - assertFails("SELECT * FROM JSON_TABLE('[1, 2, 3]', 'lax $[2]' COLUMNS(o FOR ORDINALITY))") + // ordinality column + analyze(""" + SELECT * + FROM JSON_TABLE( + '[1, 2, 3]', + 'lax $[2]' + COLUMNS( + o FOR ORDINALITY)) + """); + + // regular column + analyze(""" + SELECT * + FROM JSON_TABLE( + '[1, 2, 3]', + 'lax $' + COLUMNS( + id BIGINT + PATH 'lax $[1]' + DEFAULT 0 ON EMPTY + ERROR ON ERROR)) + """); + + // formatted column + analyze(""" + SELECT * + FROM JSON_TABLE( + '[1, 2, 3]', + 'lax $' + COLUMNS( + id VARBINARY + FORMAT JSON ENCODING UTF16 + PATH 'lax $[1]' + WITHOUT WRAPPER + OMIT QUOTES + EMPTY ARRAY ON EMPTY + NULL ON ERROR)) + """); + + // nested columns + analyze(""" + SELECT * + FROM JSON_TABLE( + '[1, 2, 3]', + 'lax $' + COLUMNS( + NESTED PATH 'lax $[*]' AS nested_path COLUMNS ( + o FOR ORDINALITY, + id BIGINT PATH 'lax $[1]'))) + """); + } + + @Test + public void testJsonTableColumnAndPathNameUniqueness() + { + // root path is named + analyze(""" + SELECT * + FROM JSON_TABLE( + '[1, 2, 3]', + 'lax $[2]' AS root_path + COLUMNS( + o FOR ORDINALITY)) + """); + + // nested path is named + analyze(""" + SELECT * + FROM JSON_TABLE( + '[1, 2, 3]', + 'lax $' + COLUMNS( + NESTED PATH 'lax $[*]' AS nested_path COLUMNS ( + o FOR ORDINALITY))) + """); + + // root and nested paths are named + analyze(""" + SELECT * + FROM JSON_TABLE( + '[1, 2, 3]', + 'lax $' AS root_path + COLUMNS( + NESTED PATH 'lax $[*]' AS nested_path COLUMNS ( + o FOR ORDINALITY))) + """); + + // duplicate path name + assertFails(""" + SELECT * + FROM JSON_TABLE( + '[1, 2, 3]', + 'lax $' AS some_path + COLUMNS( + NESTED PATH 'lax $[*]' AS some_path COLUMNS ( + o FOR ORDINALITY))) + """) + .hasErrorCode(DUPLICATE_COLUMN_OR_PATH_NAME) + .hasMessage("line 6:35: All column and path names in JSON_TABLE invocation must be unique"); + + // duplicate column name + assertFails(""" + SELECT * + FROM JSON_TABLE( + '[1, 2, 3]', + 'lax $[2]' + COLUMNS( + id FOR ORDINALITY, + id BIGINT)) + """) + .hasErrorCode(DUPLICATE_COLUMN_OR_PATH_NAME) + .hasMessage("line 7:9: All column and path names in JSON_TABLE invocation must be unique"); + + // column and path names are the same + assertFails(""" + SELECT * + FROM JSON_TABLE( + '[1, 2, 3]', + 'lax $[2]' AS some_name + COLUMNS( + some_name FOR ORDINALITY)) + """) + .hasErrorCode(DUPLICATE_COLUMN_OR_PATH_NAME) + .hasMessage("line 6:9: All column and path names in JSON_TABLE invocation must be unique"); + + assertFails(""" + SELECT * + FROM JSON_TABLE( + '[1, 2, 3]', + 'lax $' + COLUMNS( + NESTED PATH 'lax $[*]' AS some_name COLUMNS ( + some_name FOR ORDINALITY))) + """) + .hasErrorCode(DUPLICATE_COLUMN_OR_PATH_NAME) + .hasMessage("line 7:13: All column and path names in JSON_TABLE invocation must be unique"); + + // duplicate name is deeply nested + assertFails(""" + SELECT * + FROM JSON_TABLE( + '[1, 2, 3]', + 'lax $[2]' + COLUMNS( + NESTED PATH 'lax $[*]' AS some_name COLUMNS ( + NESTED PATH 'lax $' AS another_name COLUMNS ( + NESTED PATH 'lax $' AS yet_another_name COLUMNS ( + some_name FOR ORDINALITY))))) + """) + .hasErrorCode(DUPLICATE_COLUMN_OR_PATH_NAME) + .hasMessage("line 9:21: All column and path names in JSON_TABLE invocation must be unique"); + } + + @Test + public void testJsonTableColumnAndPathNameIdentifierSemantics() + { + assertFails(""" + SELECT * + FROM JSON_TABLE( + '[1, 2, 3]', + 'lax $[2]' AS some_name + COLUMNS( + Some_Name FOR ORDINALITY)) + """) + .hasErrorCode(DUPLICATE_COLUMN_OR_PATH_NAME) + .hasMessage("line 6:9: All column and path names in JSON_TABLE invocation must be unique"); + + analyze(""" + SELECT * + FROM JSON_TABLE( + '[1, 2, 3]', + 'lax $[2]' AS some_name + COLUMNS( + "some_name" FOR ORDINALITY)) + """); + } + + @Test + public void testJsonTableOutputColumns() + { + analyze(""" + SELECT a, b, c, d, e + FROM JSON_TABLE( + '[1, 2, 3]', + 'lax $' + COLUMNS( + a FOR ORDINALITY, + b BIGINT, + c VARBINARY FORMAT JSON ENCODING UTF16, + NESTED PATH 'lax $[*]' COLUMNS ( + d FOR ORDINALITY, + e BIGINT))) + """); + } + + @Test + public void testImplicitJsonPath() + { + // column name: Ab + // canonical name: AB + // implicit path: lax $."AB" + // resolved member accessor: $.AB + analyze(""" + SELECT * + FROM JSON_TABLE( + '[1, 2, 3]', + 'lax $[2]' + COLUMNS(Ab BIGINT)) + """); + + // column name: Ab + // canonical name: Ab + // implicit path: lax $."Ab" + // resolved member accessor: $.Ab + analyze(""" + SELECT * + FROM JSON_TABLE( + '[1, 2, 3]', + 'lax $[2]' + COLUMNS("Ab" BIGINT)) + """); + + // column name: ? + // canonical name: ? + // implicit path: lax $."?" + // resolved member accessor: $.? + analyze(""" + SELECT * + FROM JSON_TABLE( + '[1, 2, 3]', + 'lax $[2]' + COLUMNS("?" BIGINT)) + """); + + // column name: " + // canonical name: " + // implicit path: lax $."""" + // resolved member accessor $." + analyze(""" + SELECT * + FROM JSON_TABLE( + '[1, 2, 3]', + 'lax $[2]' + COLUMNS("\"\"" BIGINT)) + """); + } + + @Test + public void testJsonTableSpecificPlan() + { + assertFails(""" + SELECT * + FROM JSON_TABLE( + '[1, 2, 3]', + 'lax $[2]' + COLUMNS(id BIGINT) + PLAN (root_path)) + """) + .hasErrorCode(MISSING_PATH_NAME) + .hasMessage("line 3:5: All JSON paths must be named when specific plan is given"); + + assertFails(""" + SELECT * + FROM JSON_TABLE( + '[1, 2, 3]', + 'lax $[2]' AS root_path + COLUMNS(id BIGINT) + PLAN (root_path UNION another_path)) + """) + .hasErrorCode(INVALID_PLAN) + .hasMessage("line 6:11: JSON_TABLE plan must either be a single path name or it must be rooted in parent-child relationship (OUTER or INNER)"); + + assertFails(""" + SELECT * + FROM JSON_TABLE( + '[1, 2, 3]', + 'lax $' AS root_path + COLUMNS(id BIGINT) + PLAN (another_path)) + """) + .hasErrorCode(INVALID_PLAN) + .hasMessage("line 6:11: JSON_TABLE plan should contain all JSON paths available at each level of nesting. Paths not included: ROOT_PATH"); + + assertFails(""" + SELECT * + FROM JSON_TABLE( + '[1, 2, 3]', + 'lax $' AS root_path + COLUMNS( + NESTED PATH 'lax $' COLUMNS(id BIGINT)) + PLAN (root_path OUTER another_path)) + """) + .hasErrorCode(MISSING_PATH_NAME) + .hasMessage("line 6:21: All JSON paths must be named when specific plan is given"); + + assertFails(""" + SELECT * + FROM JSON_TABLE( + '[1, 2, 3]', + 'lax $' AS root_path + COLUMNS( + NESTED PATH 'lax $' AS nested_path_1 COLUMNS(id_1 BIGINT), + NESTED PATH 'lax $' AS nested_path_2 COLUMNS(id_2 BIGINT)) + PLAN (root_path OUTER (nested_path_1 CROSS another_path))) + """) + .hasErrorCode(INVALID_PLAN) + .hasMessage("line 8:11: JSON_TABLE plan should contain all JSON paths available at each level of nesting. Paths not included: NESTED_PATH_2"); + + assertFails(""" + SELECT * + FROM JSON_TABLE( + '[1, 2, 3]', + 'lax $' AS root_path + COLUMNS( + NESTED PATH 'lax $' AS nested_path_1 COLUMNS(id_1 BIGINT), + NESTED PATH 'lax $' AS nested_path_2 COLUMNS(id_2 BIGINT)) + PLAN (root_path OUTER (nested_path_1 CROSS another_path CROSS nested_path_2))) + """) + .hasErrorCode(INVALID_PLAN) + .hasMessage("line 8:11: JSON_TABLE plan includes unavailable JSON path names: ANOTHER_PATH"); + + assertFails(""" + SELECT * + FROM JSON_TABLE( + '[1, 2, 3]', + 'lax $' AS root_path + COLUMNS( + NESTED PATH 'lax $' AS nested_path_1 COLUMNS(id_1 BIGINT), + NESTED PATH 'lax $' AS nested_path_2 COLUMNS( + id_2 BIGINT, + NESTED PATH 'lax $' AS nested_path_3 COLUMNS(id_3 BIGINT))) + PLAN (root_path OUTER (nested_path_1 CROSS (nested_path_2 UNION nested_path_3)))) + """) + .hasErrorCode(INVALID_PLAN) + .hasMessage("line 10:11: JSON_TABLE plan includes unavailable JSON path names: NESTED_PATH_3"); // nested_path_3 is on another nesting level + + assertFails(""" + SELECT * + FROM JSON_TABLE( + '[1, 2, 3]', + 'lax $' AS root_path + COLUMNS( + NESTED PATH 'lax $' AS nested_path_1 COLUMNS(id_1 BIGINT), + NESTED PATH 'lax $' AS nested_path_2 COLUMNS(id_2 BIGINT)) + PLAN (root_path OUTER (nested_path_1 CROSS (nested_path_2 UNION nested_path_1)))) + """) + .hasErrorCode(INVALID_PLAN) + .hasMessage("line 8:69: Duplicate reference to JSON path name in sibling plan: NESTED_PATH_1"); + + analyze(""" + SELECT * + FROM JSON_TABLE( + '[1, 2, 3]', + 'lax $' AS root_path + COLUMNS( + NESTED PATH 'lax $' AS nested_path_1 COLUMNS(id_1 BIGINT), + NESTED PATH 'lax $' AS nested_path_2 COLUMNS( + id_2 BIGINT, + NESTED PATH 'lax $' AS nested_path_3 COLUMNS(id_3 BIGINT))) + PLAN (root_path OUTER (nested_path_1 CROSS (nested_path_2 INNER nested_path_3)))) + """); + } + + @Test + public void testJsonTableDefaultPlan() + { + analyze(""" + SELECT * + FROM JSON_TABLE( + '[1, 2, 3]', + 'lax $[2]' + COLUMNS(id BIGINT) + PLAN DEFAULT(CROSS, INNER)) + """); + + assertFails(""" + SELECT * + FROM JSON_TABLE( + '[1, 2, 3]', + 'lax $' AS root_path + COLUMNS( + NESTED PATH 'lax $' COLUMNS(id BIGINT)) + PLAN DEFAULT(OUTER, UNION)) + """) + .hasErrorCode(MISSING_PATH_NAME) + .hasMessage("line 6:21: All nested JSON paths must be named when default plan is given"); + } + + @Test + public void tstJsonTableInJoin() + { + analyze(""" + SELECT * + FROM t1, t2, JSON_TABLE('[1, 2, 3]', 'lax $[2]' COLUMNS(o FOR ORDINALITY)) + """); + + // join condition + analyze(""" + SELECT * + FROM t1 + LEFT JOIN + JSON_TABLE('[1, 2, 3]', 'lax $[2]' COLUMNS(o FOR ORDINALITY)) + ON TRUE + """); + + assertFails(""" + SELECT * + FROM t1 + RIGHT JOIN + JSON_TABLE('[1, 2, 3]', 'lax $[2]' COLUMNS(o FOR ORDINALITY)) t + ON t.o > t1.a + """) .hasErrorCode(NOT_SUPPORTED) - .hasMessage("line 1:15: JSON_TABLE is not yet supported"); + .hasMessage("line 5:12: RIGHT JOIN involving JSON_TABLE is only supported with condition ON TRUE"); + + // correlation in context item + analyze(""" + SELECT * + FROM t6 + LEFT JOIN + JSON_TABLE(b, 'lax $[2]' COLUMNS(o FOR ORDINALITY)) + ON TRUE + """); + + // correlation in default value + analyze(""" + SELECT * + FROM t6 + LEFT JOIN + JSON_TABLE('[1, 2, 3]', 'lax $[2]' COLUMNS(x BIGINT DEFAULT a ON EMPTY)) + ON TRUE + """); + + // correlation in path parameter + analyze(""" + SELECT * + FROM t6 + LEFT JOIN + JSON_TABLE('[1, 2, 3]', 'lax $[2]' PASSING a AS parameter_name COLUMNS(o FOR ORDINALITY)) + ON TRUE + """); + + // invalid correlation in right join + assertFails(""" + SELECT * + FROM t6 + RIGHT JOIN + JSON_TABLE('[1, 2, 3]', 'lax $[2]' PASSING a AS parameter_name COLUMNS(o FOR ORDINALITY)) + ON TRUE + """) + .hasErrorCode(INVALID_COLUMN_REFERENCE) + .hasMessage("line 4:48: LATERAL reference not allowed in RIGHT JOIN"); + } + + @Test + public void testSubqueryInJsonTable() + { + analyze(""" + SELECT * + FROM JSON_TABLE( + (SELECT '[1, 2, 3]'), + 'lax $[2]' PASSING (SELECT 1) AS parameter_name + COLUMNS( + x BIGINT DEFAULT (SELECT 2) ON EMPTY)) + """); + } + + @Test + public void testAggregationInJsonTable() + { + assertFails(""" + SELECT * + FROM JSON_TABLE( + CAST(sum(1) AS varchar), + 'lax $' PASSING 2 AS parameter_name + COLUMNS( + x BIGINT DEFAULT 3 ON EMPTY DEFAULT 4 ON ERROR)) + """) + .hasErrorCode(EXPRESSION_NOT_SCALAR) + .hasMessage("line 3:5: JSON_TABLE input expression cannot contain aggregations, window functions or grouping operations: [sum(1)]"); + + assertFails(""" + SELECT * + FROM JSON_TABLE( + '1', + 'lax $' PASSING avg(2) AS parameter_name + COLUMNS( + x BIGINT DEFAULT 3 ON EMPTY DEFAULT 4 ON ERROR)) + """) + .hasErrorCode(EXPRESSION_NOT_SCALAR) + .hasMessage("line 4:21: JSON_TABLE path parameter cannot contain aggregations, window functions or grouping operations: [avg(2)]"); + + assertFails(""" + SELECT * + FROM JSON_TABLE( + '1', + 'lax $' PASSING 2 AS parameter_name + COLUMNS( + x BIGINT DEFAULT min(3) ON EMPTY DEFAULT 4 ON ERROR)) + """) + .hasErrorCode(EXPRESSION_NOT_SCALAR) + .hasMessage("line 6:26: default expression for JSON_TABLE column cannot contain aggregations, window functions or grouping operations: [min(3)]"); + + assertFails(""" + SELECT * + FROM JSON_TABLE( + '1', + 'lax $' PASSING 2 AS parameter_name + COLUMNS( + x BIGINT DEFAULT 3 ON EMPTY DEFAULT max(4) ON ERROR)) + """) + .hasErrorCode(EXPRESSION_NOT_SCALAR) + .hasMessage("line 6:45: default expression for JSON_TABLE column cannot contain aggregations, window functions or grouping operations: [max(4)]"); + } + + @Test + public void testAliasJsonTable() + { + analyze(""" + SELECT t.y + FROM JSON_TABLE( + '[1, 2, 3]', + 'lax $[2]' + COLUMNS(x BIGINT)) t(y) + """); + + analyze(""" + SELECT t.x + FROM JSON_TABLE( + '[1, 2, 3]', + 'lax $[2]' + COLUMNS(x BIGINT)) t + """); } @Test diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/JsonTablePlanComparator.java b/core/trino-main/src/test/java/io/trino/sql/planner/JsonTablePlanComparator.java new file mode 100644 index 0000000000000..8e34adfd2f2dd --- /dev/null +++ b/core/trino-main/src/test/java/io/trino/sql/planner/JsonTablePlanComparator.java @@ -0,0 +1,125 @@ +/* + * 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.sql.planner; + +import io.trino.operator.table.json.JsonTableColumn; +import io.trino.operator.table.json.JsonTableOrdinalityColumn; +import io.trino.operator.table.json.JsonTablePlanCross; +import io.trino.operator.table.json.JsonTablePlanLeaf; +import io.trino.operator.table.json.JsonTablePlanNode; +import io.trino.operator.table.json.JsonTablePlanSingle; +import io.trino.operator.table.json.JsonTablePlanUnion; +import io.trino.operator.table.json.JsonTableQueryColumn; +import io.trino.operator.table.json.JsonTableValueColumn; + +import java.util.Comparator; +import java.util.List; + +import static java.util.Objects.requireNonNull; + +public class JsonTablePlanComparator +{ + private JsonTablePlanComparator() {} + + public static Comparator planComparator() + { + return (actual, expected) -> { + requireNonNull(actual, "actual is null"); + requireNonNull(expected, "expected is null"); + return compare(actual, expected) ? 0 : -1; + }; + } + + private static boolean compare(JsonTablePlanNode left, JsonTablePlanNode right) + { + if (left == right) { + return true; + } + if (left.getClass() != right.getClass()) { + return false; + } + if (left instanceof JsonTablePlanLeaf leftPlan) { + JsonTablePlanLeaf rightPlan = (JsonTablePlanLeaf) right; + return leftPlan.path().equals(rightPlan.path()) && + compareColumns(leftPlan.columns(), rightPlan.columns()); + } + if (left instanceof JsonTablePlanSingle leftPlan) { + JsonTablePlanSingle rightPlan = (JsonTablePlanSingle) right; + return leftPlan.path().equals(rightPlan.path()) && + compareColumns(leftPlan.columns(), rightPlan.columns()) && + leftPlan.outer() == rightPlan.outer() && + compare(leftPlan.child(), rightPlan.child()); + } + List leftSiblings; + List rightSiblings; + if (left instanceof JsonTablePlanCross leftPlan) { + leftSiblings = leftPlan.siblings(); + rightSiblings = ((JsonTablePlanCross) right).siblings(); + } + else { + leftSiblings = ((JsonTablePlanUnion) left).siblings(); + rightSiblings = ((JsonTablePlanUnion) right).siblings(); + } + if (leftSiblings.size() != rightSiblings.size()) { + return false; + } + for (int i = 0; i < leftSiblings.size(); i++) { + if (!compare(leftSiblings.get(i), rightSiblings.get(i))) { + return false; + } + } + return true; + } + + private static boolean compareColumns(List leftColumns, List rightColumns) + { + if (leftColumns.size() != rightColumns.size()) { + return false; + } + for (int i = 0; i < leftColumns.size(); i++) { + if (!compareColumn(leftColumns.get(i), rightColumns.get(i))) { + return false; + } + } + return true; + } + + private static boolean compareColumn(JsonTableColumn left, JsonTableColumn right) + { + if (left.getClass() != right.getClass()) { + return false; + } + if (left instanceof JsonTableOrdinalityColumn leftColumn) { + return leftColumn.outputIndex() == ((JsonTableOrdinalityColumn) right).outputIndex(); + } + if (left instanceof JsonTableQueryColumn leftColumn) { + JsonTableQueryColumn rightColumn = (JsonTableQueryColumn) right; + return leftColumn.outputIndex() == rightColumn.outputIndex() && + leftColumn.function().equals(rightColumn.function()) && + leftColumn.path().equals(rightColumn.path()) && + leftColumn.wrapperBehavior() == rightColumn.wrapperBehavior() && + leftColumn.emptyBehavior() == rightColumn.emptyBehavior() && + leftColumn.errorBehavior() == rightColumn.errorBehavior(); + } + JsonTableValueColumn leftColumn = (JsonTableValueColumn) left; + JsonTableValueColumn rightColumn = (JsonTableValueColumn) right; + return leftColumn.outputIndex() == rightColumn.outputIndex() && + leftColumn.function().equals(rightColumn.function()) && + leftColumn.path().equals(rightColumn.path()) && + leftColumn.emptyBehavior() == rightColumn.emptyBehavior() && + leftColumn.emptyDefaultInput() == rightColumn.emptyDefaultInput() && + leftColumn.errorBehavior() == rightColumn.errorBehavior() && + leftColumn.errorDefaultInput() == rightColumn.errorDefaultInput(); + } +} diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/TestJsonTable.java b/core/trino-main/src/test/java/io/trino/sql/planner/TestJsonTable.java new file mode 100644 index 0000000000000..4631acea7921b --- /dev/null +++ b/core/trino-main/src/test/java/io/trino/sql/planner/TestJsonTable.java @@ -0,0 +1,549 @@ +/* + * 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.sql.planner; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import io.trino.execution.warnings.WarningCollector; +import io.trino.json.ir.IrJsonPath; +import io.trino.metadata.ResolvedFunction; +import io.trino.metadata.TestingFunctionResolution; +import io.trino.operator.table.json.JsonTable; +import io.trino.operator.table.json.JsonTablePlanCross; +import io.trino.operator.table.json.JsonTablePlanLeaf; +import io.trino.operator.table.json.JsonTablePlanNode; +import io.trino.operator.table.json.JsonTablePlanSingle; +import io.trino.operator.table.json.JsonTablePlanUnion; +import io.trino.operator.table.json.JsonTableQueryColumn; +import io.trino.operator.table.json.JsonTableValueColumn; +import io.trino.sql.planner.assertions.BasePlanTest; +import io.trino.sql.planner.optimizations.PlanNodeSearcher; +import io.trino.sql.planner.plan.TableFunctionNode; +import io.trino.sql.tree.JsonQuery; +import io.trino.sql.tree.JsonValue; +import org.intellij.lang.annotations.Language; +import org.junit.jupiter.api.Test; + +import static com.google.common.collect.Iterables.getOnlyElement; +import static io.trino.execution.querystats.PlanOptimizersStatsCollector.createPlanOptimizersStatsCollector; +import static io.trino.operator.scalar.json.JsonQueryFunction.JSON_QUERY_FUNCTION_NAME; +import static io.trino.operator.scalar.json.JsonValueFunction.JSON_VALUE_FUNCTION_NAME; +import static io.trino.spi.type.BigintType.BIGINT; +import static io.trino.spi.type.BooleanType.BOOLEAN; +import static io.trino.spi.type.TinyintType.TINYINT; +import static io.trino.sql.analyzer.ExpressionAnalyzer.JSON_NO_PARAMETERS_ROW_TYPE; +import static io.trino.sql.analyzer.TypeSignatureProvider.fromTypes; +import static io.trino.sql.planner.JsonTablePlanComparator.planComparator; +import static io.trino.sql.planner.LogicalPlanner.Stage.CREATED; +import static io.trino.sql.planner.PathNodes.contextVariable; +import static io.trino.sql.planner.PathNodes.literal; +import static io.trino.sql.planner.PathNodes.memberAccessor; +import static io.trino.sql.planner.assertions.PlanMatchPattern.anyTree; +import static io.trino.sql.planner.assertions.PlanMatchPattern.expression; +import static io.trino.sql.planner.assertions.PlanMatchPattern.project; +import static io.trino.sql.planner.assertions.PlanMatchPattern.strictOutput; +import static io.trino.sql.planner.assertions.PlanMatchPattern.tableFunction; +import static io.trino.sql.planner.assertions.PlanMatchPattern.values; +import static io.trino.sql.planner.assertions.TableFunctionMatcher.TableArgumentValue.Builder.tableArgument; +import static io.trino.type.Json2016Type.JSON_2016; +import static io.trino.type.TestJsonPath2016TypeSerialization.JSON_PATH_2016; +import static org.assertj.core.api.Assertions.assertThat; + +public class TestJsonTable + extends BasePlanTest +{ + private static final ResolvedFunction JSON_VALUE_FUNCTION = new TestingFunctionResolution().resolveFunction( + JSON_VALUE_FUNCTION_NAME, + fromTypes(JSON_2016, JSON_PATH_2016, JSON_NO_PARAMETERS_ROW_TYPE, TINYINT, BIGINT, TINYINT, BIGINT)); + + private static final ResolvedFunction JSON_QUERY_FUNCTION = new TestingFunctionResolution().resolveFunction( + JSON_QUERY_FUNCTION_NAME, + fromTypes(JSON_2016, JSON_PATH_2016, JSON_NO_PARAMETERS_ROW_TYPE, TINYINT, TINYINT, TINYINT)); + + @Test + public void testJsonTableInitialPlan() + { + assertPlan( + """ + SELECT * + FROM (SELECT '[1, 2, 3]', 4) t(json_col, int_col), JSON_TABLE( + json_col, + 'lax $' AS root_path PASSING int_col AS id, '[ala]' FORMAT JSON AS name + COLUMNS( + bigint_col BIGINT DEFAULT 5 ON EMPTY DEFAULT int_col ON ERROR, + varchar_col VARCHAR FORMAT JSON ERROR ON ERROR) + EMPTY ON ERROR) + """, + CREATED, + strictOutput(// left-side columns first, json_table columns next + ImmutableList.of("json_col", "int_col", "bigint_col", "formatted_varchar_col"), + anyTree( + project( + ImmutableMap.of("formatted_varchar_col", expression("\"$json_to_varchar\"(varchar_col, tinyint '1', false)")), + tableFunction(builder -> builder + .name("$json_table") + .addTableArgument( + "$input", + tableArgument(0) + .rowSemantics() + .passThroughColumns() + .passThroughSymbols(ImmutableSet.of("json_col", "int_col"))) + .properOutputs(ImmutableList.of("bigint_col", "varchar_col")), + project( + ImmutableMap.of( + "context_item", expression("\"$varchar_to_json\"(json_col_coerced, false)"), // apply input function to context item + "parameters_row", expression("CAST(ROW (int_col, \"$varchar_to_json\"(name_coerced, false)) AS ROW(ID integer, NAME json2016))")), // apply input function to formatted path parameter and gather path parameters in a row + project(// coerce context item, path parameters and default expressions + ImmutableMap.of( + "name_coerced", expression("CAST(name AS VARCHAR)"), // cast formatted path parameter to VARCHAR for the input function + "default_value_coerced", expression("CAST(default_value AS BIGINT)"), // cast default value to BIGINT to match declared return type for the column + "json_col_coerced", expression("CAST(json_col AS VARCHAR)"), // cast context item to VARCHAR for the input function + "int_col_coerced", expression("CAST(int_col AS BIGINT)")), // cast default value to BIGINT to match declared return type for the column + project(// pre-project context item, path parameters and default expressions + ImmutableMap.of( + "name", expression("'[ala]'"), + "default_value", expression("5")), + anyTree( + project( + ImmutableMap.of( + "json_col", expression("'[1, 2, 3]'"), + "int_col", expression("4")), + values(1))))))))))); + } + + @Test + public void testImplicitColumnPath() + { + assertJsonTablePlan( + """ + SELECT * + FROM (SELECT 1, 2, 3), JSON_TABLE( + '[1, 2, 3]', + 'lax $' AS root_path + COLUMNS( + first_col BIGINT, + "Second_Col" BIGINT, + "_""_'_?_" BIGINT)) + """, + new JsonTablePlanLeaf( + new IrJsonPath(true, contextVariable()), + ImmutableList.of( + valueColumn(0, new IrJsonPath(true, memberAccessor(contextVariable(), "FIRST_COL"))), + valueColumn(1, new IrJsonPath(true, memberAccessor(contextVariable(), "Second_Col"))), + valueColumn(2, new IrJsonPath(true, memberAccessor(contextVariable(), "_\"_'_?_")))))); + } + + @Test + public void testExplicitColumnPath() + { + assertJsonTablePlan( + """ + SELECT * + FROM (SELECT 1, 2, 3), JSON_TABLE( + '[1, 2, 3]', + 'lax $' AS root_path + COLUMNS( + first_col BIGINT PATH 'lax $.a', + "Second_Col" BIGINT PATH 'lax $.B', + "_""_'_?_" BIGINT PATH 'lax false')) + """, + new JsonTablePlanLeaf( + new IrJsonPath(true, contextVariable()), + ImmutableList.of( + valueColumn(0, new IrJsonPath(true, memberAccessor(contextVariable(), "a"))), + valueColumn(1, new IrJsonPath(true, memberAccessor(contextVariable(), "B"))), + valueColumn(2, new IrJsonPath(true, literal(BOOLEAN, false)))))); + } + + @Test + public void testColumnOutputIndex() + { + // output indexes follow the declaration order: [a, b, c, d] + assertJsonTablePlan( + """ + SELECT * + FROM (SELECT 1, 2, 3), JSON_TABLE( + '[1, 2, 3]', + 'lax $' AS root_path + COLUMNS( + a BIGINT, + NESTED PATH 'lax $.x' COLUMNS( + b BIGINT, + NESTED PATH 'lax $.y' COLUMNS( + c BIGINT)), + d BIGINT)) + """, + new JsonTablePlanSingle( + new IrJsonPath(true, contextVariable()), + ImmutableList.of( + valueColumn(0, new IrJsonPath(true, memberAccessor(contextVariable(), "A"))), + valueColumn(3, new IrJsonPath(true, memberAccessor(contextVariable(), "D")))), + true, + new JsonTablePlanSingle( + new IrJsonPath(true, memberAccessor(contextVariable(), "x")), + ImmutableList.of(valueColumn(1, new IrJsonPath(true, memberAccessor(contextVariable(), "B")))), + true, + new JsonTablePlanLeaf( + new IrJsonPath(true, memberAccessor(contextVariable(), "y")), + ImmutableList.of(valueColumn(2, new IrJsonPath(true, memberAccessor(contextVariable(), "C")))))))); + } + + @Test + public void testColumnBehavior() + { + assertJsonTablePlan( + """ + SELECT * + FROM (SELECT 1, 2, 3), JSON_TABLE( + '[1, 2, 3]', + 'lax $' AS root_path + COLUMNS( + a BIGINT, + b BIGINT NULL ON EMPTY ERROR ON ERROR, + c BIGINT DEFAULT 1 ON EMPTY DEFAULT 2 ON ERROR, + d VARCHAR FORMAT JSON, + e VARCHAR FORMAT JSON WITH CONDITIONAL ARRAY WRAPPER NULL ON EMPTY ERROR ON ERROR, + f VARCHAR FORMAT JSON OMIT QUOTES EMPTY ARRAY ON EMPTY EMPTY OBJECT ON ERROR)) + """, + new JsonTablePlanLeaf( + new IrJsonPath(true, contextVariable()), + ImmutableList.of( + valueColumn( + 0, + new IrJsonPath(true, memberAccessor(contextVariable(), "A")), + JsonValue.EmptyOrErrorBehavior.NULL, + -1, + JsonValue.EmptyOrErrorBehavior.NULL, + -1), + valueColumn( + 1, + new IrJsonPath(true, memberAccessor(contextVariable(), "B")), + JsonValue.EmptyOrErrorBehavior.NULL, + -1, + JsonValue.EmptyOrErrorBehavior.ERROR, + -1), + valueColumn( + 2, + new IrJsonPath(true, memberAccessor(contextVariable(), "C")), + JsonValue.EmptyOrErrorBehavior.DEFAULT, + 2, + JsonValue.EmptyOrErrorBehavior.DEFAULT, + 3), + queryColumn( + 3, + new IrJsonPath(true, memberAccessor(contextVariable(), "D")), + JsonQuery.ArrayWrapperBehavior.WITHOUT, + JsonQuery.EmptyOrErrorBehavior.NULL, + JsonQuery.EmptyOrErrorBehavior.NULL), + queryColumn( + 4, + new IrJsonPath(true, memberAccessor(contextVariable(), "E")), + JsonQuery.ArrayWrapperBehavior.CONDITIONAL, + JsonQuery.EmptyOrErrorBehavior.NULL, + JsonQuery.EmptyOrErrorBehavior.ERROR), + queryColumn( + 5, + new IrJsonPath(true, memberAccessor(contextVariable(), "F")), + JsonQuery.ArrayWrapperBehavior.WITHOUT, + JsonQuery.EmptyOrErrorBehavior.EMPTY_ARRAY, + JsonQuery.EmptyOrErrorBehavior.EMPTY_OBJECT)))); + } + + @Test + public void testInheritedErrorBehavior() + { + // the column has no explicit error behavior, and json_table has no explicit error behavior. The default behavior for column is NULL ON ERROR. + assertJsonTablePlan( + """ + SELECT * + FROM (SELECT 1, 2, 3), JSON_TABLE( + '[1, 2, 3]', + 'lax $' AS root_path + COLUMNS( + a BIGINT)) + """, + new JsonTablePlanLeaf( + new IrJsonPath(true, contextVariable()), + ImmutableList.of( + valueColumn( + 0, + new IrJsonPath(true, memberAccessor(contextVariable(), "A")), + JsonValue.EmptyOrErrorBehavior.NULL, + -1, + JsonValue.EmptyOrErrorBehavior.NULL, + -1)))); + + // the column has no explicit error behavior, and json_table has explicit ERROR ON ERROR. The default behavior for column is ERROR ON ERROR. + assertJsonTablePlan( + """ + SELECT * + FROM (SELECT 1, 2, 3), JSON_TABLE( + '[1, 2, 3]', + 'lax $' AS root_path + COLUMNS( + a BIGINT) + ERROR ON ERROR) + """, + new JsonTablePlanLeaf( + new IrJsonPath(true, contextVariable()), + ImmutableList.of( + valueColumn( + 0, + new IrJsonPath(true, memberAccessor(contextVariable(), "A")), + JsonValue.EmptyOrErrorBehavior.NULL, + -1, + JsonValue.EmptyOrErrorBehavior.ERROR, + -1)))); + + // the column has no explicit error behavior, and json_table has explicit EMPTY ON ERROR. The default behavior for column is NULL ON ERROR. + assertJsonTablePlan( + """ + SELECT * + FROM (SELECT 1, 2, 3), JSON_TABLE( + '[1, 2, 3]', + 'lax $' AS root_path + COLUMNS( + a BIGINT) + EMPTY ON ERROR) + """, + new JsonTablePlanLeaf( + new IrJsonPath(true, contextVariable()), + ImmutableList.of( + valueColumn( + 0, + new IrJsonPath(true, memberAccessor(contextVariable(), "A")), + JsonValue.EmptyOrErrorBehavior.NULL, + -1, + JsonValue.EmptyOrErrorBehavior.NULL, + -1)))); + + // the column has explicit NULL ON ERROR behavior, and json_table has no explicit ERROR ON ERROR. The behavior for column is the one explicitly specified. + assertJsonTablePlan( + """ + SELECT * + FROM (SELECT 1, 2, 3), JSON_TABLE( + '[1, 2, 3]', + 'lax $' AS root_path + COLUMNS( + a BIGINT NULL ON ERROR) + ERROR ON ERROR) + """, + new JsonTablePlanLeaf( + new IrJsonPath(true, contextVariable()), + ImmutableList.of( + valueColumn( + 0, + new IrJsonPath(true, memberAccessor(contextVariable(), "A")), + JsonValue.EmptyOrErrorBehavior.NULL, + -1, + JsonValue.EmptyOrErrorBehavior.NULL, + -1)))); + } + + @Test + public void testImplicitDefaultPlan() + { + // implicit plan settings are OUTER, UNION + assertJsonTablePlan( + """ + SELECT * + FROM (SELECT 1, 2, 3), JSON_TABLE( + '[1, 2, 3]', + 'lax $' AS root_path + COLUMNS( + NESTED PATH 'lax $.a' COLUMNS(col_1 BIGINT), + NESTED PATH 'lax $.b' COLUMNS( + NESTED PATH 'lax $.c' COLUMNS(col_2 BIGINT), + NESTED PATH 'lax $.d' COLUMNS(col_3 BIGINT)), + NESTED PATH 'lax $.e' COLUMNS(col_4 BIGINT))) + """, + new JsonTablePlanSingle( + new IrJsonPath(true, contextVariable()), + ImmutableList.of(), + true, + new JsonTablePlanUnion(ImmutableList.of( + new JsonTablePlanLeaf( + new IrJsonPath(true, memberAccessor(contextVariable(), "a")), + ImmutableList.of(valueColumn(0, new IrJsonPath(true, memberAccessor(contextVariable(), "COL_1"))))), + new JsonTablePlanSingle( + new IrJsonPath(true, memberAccessor(contextVariable(), "b")), + ImmutableList.of(), + true, + new JsonTablePlanUnion(ImmutableList.of( + new JsonTablePlanLeaf( + new IrJsonPath(true, memberAccessor(contextVariable(), "c")), + ImmutableList.of(valueColumn(1, new IrJsonPath(true, memberAccessor(contextVariable(), "COL_2"))))), + new JsonTablePlanLeaf( + new IrJsonPath(true, memberAccessor(contextVariable(), "d")), + ImmutableList.of(valueColumn(2, new IrJsonPath(true, memberAccessor(contextVariable(), "COL_3")))))))), + new JsonTablePlanLeaf( + new IrJsonPath(true, memberAccessor(contextVariable(), "e")), + ImmutableList.of(valueColumn(3, new IrJsonPath(true, memberAccessor(contextVariable(), "COL_4"))))))))); + } + + @Test + public void testExplicitDefaultPlan() + { + assertJsonTablePlan( + """ + SELECT * + FROM (SELECT 1, 2, 3), JSON_TABLE( + '[1, 2, 3]', + 'lax $' AS root_path + COLUMNS( + NESTED PATH 'lax $.a' AS a COLUMNS(col_1 BIGINT), + NESTED PATH 'lax $.b' AS b COLUMNS( + NESTED PATH 'lax $.c' AS c COLUMNS(col_2 BIGINT), + NESTED PATH 'lax $.d' AS d COLUMNS(col_3 BIGINT)), + NESTED PATH 'lax $.e' AS e COLUMNS(col_4 BIGINT)) + PLAN DEFAULT (INNER, CROSS)) + """, + new JsonTablePlanSingle( + new IrJsonPath(true, contextVariable()), + ImmutableList.of(), + false, + new JsonTablePlanCross(ImmutableList.of( + new JsonTablePlanLeaf( + new IrJsonPath(true, memberAccessor(contextVariable(), "a")), + ImmutableList.of(valueColumn(0, new IrJsonPath(true, memberAccessor(contextVariable(), "COL_1"))))), + new JsonTablePlanSingle( + new IrJsonPath(true, memberAccessor(contextVariable(), "b")), + ImmutableList.of(), + false, + new JsonTablePlanCross(ImmutableList.of( + new JsonTablePlanLeaf( + new IrJsonPath(true, memberAccessor(contextVariable(), "c")), + ImmutableList.of(valueColumn(1, new IrJsonPath(true, memberAccessor(contextVariable(), "COL_2"))))), + new JsonTablePlanLeaf( + new IrJsonPath(true, memberAccessor(contextVariable(), "d")), + ImmutableList.of(valueColumn(2, new IrJsonPath(true, memberAccessor(contextVariable(), "COL_3")))))))), + new JsonTablePlanLeaf( + new IrJsonPath(true, memberAccessor(contextVariable(), "e")), + ImmutableList.of(valueColumn(3, new IrJsonPath(true, memberAccessor(contextVariable(), "COL_4"))))))))); + + assertJsonTablePlan( + """ + SELECT * + FROM (SELECT 1, 2, 3), JSON_TABLE( + '[1, 2, 3]', + 'lax $' AS root_path + COLUMNS( + NESTED PATH 'lax $.a' AS a COLUMNS(col_1 BIGINT), + NESTED PATH 'lax $.b' AS b COLUMNS( + NESTED PATH 'lax $.c' AS c COLUMNS(col_2 BIGINT), + NESTED PATH 'lax $.d' AS d COLUMNS(col_3 BIGINT)), + NESTED PATH 'lax $.e' AS e COLUMNS(col_4 BIGINT)) + PLAN DEFAULT (CROSS)) + """, + new JsonTablePlanSingle( + new IrJsonPath(true, contextVariable()), + ImmutableList.of(), + true, + new JsonTablePlanCross(ImmutableList.of( + new JsonTablePlanLeaf( + new IrJsonPath(true, memberAccessor(contextVariable(), "a")), + ImmutableList.of(valueColumn(0, new IrJsonPath(true, memberAccessor(contextVariable(), "COL_1"))))), + new JsonTablePlanSingle( + new IrJsonPath(true, memberAccessor(contextVariable(), "b")), + ImmutableList.of(), + true, + new JsonTablePlanCross(ImmutableList.of( + new JsonTablePlanLeaf( + new IrJsonPath(true, memberAccessor(contextVariable(), "c")), + ImmutableList.of(valueColumn(1, new IrJsonPath(true, memberAccessor(contextVariable(), "COL_2"))))), + new JsonTablePlanLeaf( + new IrJsonPath(true, memberAccessor(contextVariable(), "d")), + ImmutableList.of(valueColumn(2, new IrJsonPath(true, memberAccessor(contextVariable(), "COL_3")))))))), + new JsonTablePlanLeaf( + new IrJsonPath(true, memberAccessor(contextVariable(), "e")), + ImmutableList.of(valueColumn(3, new IrJsonPath(true, memberAccessor(contextVariable(), "COL_4"))))))))); + } + + @Test + public void testSpecificPlan() + { + assertJsonTablePlan( + """ + SELECT * + FROM (SELECT 1, 2, 3), JSON_TABLE( + '[1, 2, 3]', + 'lax $' AS root_path + COLUMNS( + NESTED PATH 'lax $.a' AS a COLUMNS(col_1 BIGINT), + NESTED PATH 'lax $.b' AS b COLUMNS( + NESTED PATH 'lax $.c' AS c COLUMNS(col_2 BIGINT), + NESTED PATH 'lax $.d' AS d COLUMNS(col_3 BIGINT)), + NESTED PATH 'lax $.e' AS e COLUMNS(col_4 BIGINT)) + PLAN (ROOT_PATH INNER (((B OUTER (D CROSS C)) UNION E) CROSS A))) + """, + new JsonTablePlanSingle( + new IrJsonPath(true, contextVariable()), + ImmutableList.of(), + false, + new JsonTablePlanCross(ImmutableList.of( + new JsonTablePlanUnion(ImmutableList.of( + new JsonTablePlanSingle( + new IrJsonPath(true, memberAccessor(contextVariable(), "b")), + ImmutableList.of(), + true, + new JsonTablePlanCross(ImmutableList.of( + new JsonTablePlanLeaf( + new IrJsonPath(true, memberAccessor(contextVariable(), "d")), + ImmutableList.of(valueColumn(2, new IrJsonPath(true, memberAccessor(contextVariable(), "COL_3"))))), + new JsonTablePlanLeaf( + new IrJsonPath(true, memberAccessor(contextVariable(), "c")), + ImmutableList.of(valueColumn(1, new IrJsonPath(true, memberAccessor(contextVariable(), "COL_2")))))))), + new JsonTablePlanLeaf( + new IrJsonPath(true, memberAccessor(contextVariable(), "e")), + ImmutableList.of(valueColumn(3, new IrJsonPath(true, memberAccessor(contextVariable(), "COL_4"))))))), + new JsonTablePlanLeaf( + new IrJsonPath(true, memberAccessor(contextVariable(), "a")), + ImmutableList.of(valueColumn(0, new IrJsonPath(true, memberAccessor(contextVariable(), "COL_1"))))))))); + } + + private static JsonTableValueColumn valueColumn(int outputIndex, IrJsonPath path) + { + return valueColumn(outputIndex, path, JsonValue.EmptyOrErrorBehavior.NULL, -1, JsonValue.EmptyOrErrorBehavior.NULL, -1); + } + + private static JsonTableValueColumn valueColumn(int outputIndex, IrJsonPath path, JsonValue.EmptyOrErrorBehavior emptyBehavior, int emptyDefaultInput, JsonValue.EmptyOrErrorBehavior errorBehavior, int errorDefaultInput) + { + return new JsonTableValueColumn(outputIndex, JSON_VALUE_FUNCTION, path, emptyBehavior.ordinal(), emptyDefaultInput, errorBehavior.ordinal(), errorDefaultInput); + } + + private static JsonTableQueryColumn queryColumn(int outputIndex, IrJsonPath path, JsonQuery.ArrayWrapperBehavior wrapperBehavior, JsonQuery.EmptyOrErrorBehavior emptyBehavior, JsonQuery.EmptyOrErrorBehavior errorBehavior) + { + return new JsonTableQueryColumn(outputIndex, JSON_QUERY_FUNCTION, path, wrapperBehavior.ordinal(), emptyBehavior.ordinal(), errorBehavior.ordinal()); + } + + private void assertJsonTablePlan(@Language("SQL") String sql, JsonTablePlanNode expectedPlan) + { + try { + getQueryRunner().inTransaction(transactionSession -> { + Plan queryPlan = getQueryRunner().createPlan(transactionSession, sql, ImmutableList.of(), CREATED, WarningCollector.NOOP, createPlanOptimizersStatsCollector()); + TableFunctionNode tableFunctionNode = getOnlyElement(PlanNodeSearcher.searchFrom(queryPlan.getRoot()).where(TableFunctionNode.class::isInstance).findAll()); + JsonTablePlanNode actualPlan = ((JsonTable.JsonTableFunctionHandle) tableFunctionNode.getHandle().getFunctionHandle()).processingPlan(); + assertThat(actualPlan) + .usingComparator(planComparator()) + .isEqualTo(expectedPlan); + return null; + }); + } + catch (Throwable e) { + e.addSuppressed(new Exception("Query: " + sql)); + throw e; + } + } +} diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/TestingPlannerContext.java b/core/trino-main/src/test/java/io/trino/sql/planner/TestingPlannerContext.java index a7e8f9bc4bded..f2e8d9647efd6 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/TestingPlannerContext.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/TestingPlannerContext.java @@ -119,7 +119,10 @@ public PlannerContext build() types.forEach(typeRegistry::addType); parametricTypes.forEach(typeRegistry::addParametricType); - GlobalFunctionCatalog globalFunctionCatalog = new GlobalFunctionCatalog(); + GlobalFunctionCatalog globalFunctionCatalog = new GlobalFunctionCatalog( + () -> { throw new UnsupportedOperationException(); }, + () -> { throw new UnsupportedOperationException(); }, + () -> { throw new UnsupportedOperationException(); }); globalFunctionCatalog.addFunctions(SystemFunctionBundle.create(featuresConfig, typeOperators, new BlockTypeOperators(typeOperators), UNKNOWN)); functionBundles.forEach(globalFunctionCatalog::addFunctions); diff --git a/core/trino-main/src/test/java/io/trino/sql/query/TestJsonTable.java b/core/trino-main/src/test/java/io/trino/sql/query/TestJsonTable.java new file mode 100644 index 0000000000000..c5f15ed662057 --- /dev/null +++ b/core/trino-main/src/test/java/io/trino/sql/query/TestJsonTable.java @@ -0,0 +1,867 @@ +/* + * 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.sql.query; + +import io.trino.Session; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; + +import static com.google.common.io.BaseEncoding.base16; +import static io.trino.spi.StandardErrorCode.PATH_EVALUATION_ERROR; +import static io.trino.testing.assertions.TrinoExceptionAssert.assertTrinoExceptionThrownBy; +import static java.nio.charset.StandardCharsets.UTF_16LE; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; + +@TestInstance(PER_CLASS) +public class TestJsonTable +{ + private QueryAssertions assertions; + + @BeforeAll + public void init() + { + assertions = new QueryAssertions(); + } + + @AfterAll + public void teardown() + { + assertions.close(); + assertions = null; + } + + @Test + public void testSimple() + { + assertThat(assertions.query(""" + SELECT first, last + FROM (SELECT '{"a" : [1, 2, 3], "b" : [4, 5, 6]}') t(json_col), JSON_TABLE( + json_col, + 'lax $.a' + COLUMNS( + first bigint PATH 'lax $[0]', + last bigint PATH 'lax $[last]')) + """)) + .matches("VALUES (BIGINT '1', BIGINT '3')"); + + assertThat(assertions.query(""" + SELECT * + FROM + (SELECT '{"a" : {"b" : [1, 2, 3], "c" : [[4, 5, 6], [7, 8, 9]]}}') t(json_col), + JSON_TABLE( + json_col, + 'lax $.a' AS "path_a" + COLUMNS( + NESTED PATH 'lax $.b[*]' AS "path_b" + COLUMNS (c1 integer PATH 'lax $ * 10'), + NESTED PATH 'lax $.c' AS "path_c" + COLUMNS ( + NESTED PATH 'lax $[0][*]' AS "path_d" COLUMNS (c2 integer PATH 'lax $ * 100'), + NESTED PATH 'lax $[last][*]' AS "path_e" COLUMNS (c3 integer PATH 'lax $ * 1000'))) + PLAN ("path_a" OUTER ("path_b" UNION ("path_c" INNER ("path_d" CROSS "path_e"))))) + """)) + .matches(""" + VALUES + ('{"a" : {"b" : [1, 2, 3], "c" : [[4, 5, 6], [7, 8, 9]]}}', 10, CAST(null AS integer), CAST(null AS integer)), + ('{"a" : {"b" : [1, 2, 3], "c" : [[4, 5, 6], [7, 8, 9]]}}', 20, null, null), + ('{"a" : {"b" : [1, 2, 3], "c" : [[4, 5, 6], [7, 8, 9]]}}', 30, null, null), + ('{"a" : {"b" : [1, 2, 3], "c" : [[4, 5, 6], [7, 8, 9]]}}', null, 400, 7000), + ('{"a" : {"b" : [1, 2, 3], "c" : [[4, 5, 6], [7, 8, 9]]}}', null, 400, 8000), + ('{"a" : {"b" : [1, 2, 3], "c" : [[4, 5, 6], [7, 8, 9]]}}', null, 400, 9000), + ('{"a" : {"b" : [1, 2, 3], "c" : [[4, 5, 6], [7, 8, 9]]}}', null, 500, 7000), + ('{"a" : {"b" : [1, 2, 3], "c" : [[4, 5, 6], [7, 8, 9]]}}', null, 500, 8000), + ('{"a" : {"b" : [1, 2, 3], "c" : [[4, 5, 6], [7, 8, 9]]}}', null, 500, 9000), + ('{"a" : {"b" : [1, 2, 3], "c" : [[4, 5, 6], [7, 8, 9]]}}', null, 600, 7000), + ('{"a" : {"b" : [1, 2, 3], "c" : [[4, 5, 6], [7, 8, 9]]}}', null, 600, 8000), + ('{"a" : {"b" : [1, 2, 3], "c" : [[4, 5, 6], [7, 8, 9]]}}', null, 600, 9000) + """); + } + + @Test + public void testSubqueries() + { + // test subqueries in: context item, value of path parameter "index", empty default, error default + assertThat(assertions.query(""" + SELECT empty_default, error_default + FROM (SELECT '[[1, 2, 3], [4, 5, 6]]') t(json_col), JSON_TABLE( + (SELECT json_col), + 'lax $[$index]' PASSING (SELECT 0) AS "index" + COLUMNS( + empty_default bigint PATH 'lax $[-42]' DEFAULT (SELECT -42) ON EMPTY, + error_default bigint PATH 'strict $[42]' DEFAULT (SELECT 42) ON ERROR)) + """)) + .matches("VALUES (BIGINT '-42', BIGINT '42')"); + } + + @Test + public void testCorrelation() + { + // test correlation in: context item, value of path parameter "index", empty default, error default + assertThat(assertions.query(""" + SELECT empty_default, error_default + FROM (SELECT '[[1, 2, 3], [4, 5, 6]]', 0, -42, 42) t(json_col, index_col, empty_default_col, error_default_col), + JSON_TABLE( + json_col, + 'lax $[$index]' PASSING index_col AS "index" + COLUMNS( + empty_default bigint PATH 'lax $[-42]' DEFAULT empty_default_col ON EMPTY, + error_default bigint PATH 'strict $[42]' DEFAULT error_default_col ON ERROR)) + """)) + .matches("VALUES (BIGINT '-42', BIGINT '42')"); + } + + @Test + public void testParameters() + { + // test parameters in: context item, value of path parameter "index", empty default, error default + Session session = Session.builder(assertions.getDefaultSession()) + .addPreparedStatement( + "my_query", + """ + SELECT empty_default, error_default + FROM JSON_TABLE( + ?, + 'lax $[$index]' PASSING ? AS "index" + COLUMNS( + empty_default bigint PATH 'lax $[-42]' DEFAULT ? ON EMPTY, + error_default bigint PATH 'strict $[42]' DEFAULT ? ON ERROR)) + """) + .build(); + assertThat(assertions.query(session, "EXECUTE my_query USING '[[1, 2, 3], [4, 5, 6]]', 0, -42, 42")) + .matches("VALUES (BIGINT '-42', BIGINT '42')"); + } + + @Test + public void testOutputLayout() + { + // first the columns from the left side of the join (json_col, index_col, empty_default_col, error_default_col), next the json_table columns (empty_default, error_default) + assertThat(assertions.query(""" + SELECT * + FROM (SELECT '[[1, 2, 3], [4, 5, 6]]', 0, -42, 42) t(json_col, index_col, empty_default_col, error_default_col), + JSON_TABLE( + json_col, + 'lax $[$index]' PASSING index_col AS "index" + COLUMNS( + empty_default bigint PATH 'lax $[-42]' DEFAULT empty_default_col * 2 ON EMPTY, + error_default bigint PATH 'strict $[42]' DEFAULT error_default_col * 2 ON ERROR)) + """)) + .matches("VALUES ('[[1, 2, 3], [4, 5, 6]]', 0, -42, 42, BIGINT '-84', BIGINT '84')"); + + // json_table columns in order of declaration + assertThat(assertions.query(""" + SELECT * + FROM JSON_TABLE( + '[]', + 'lax $' AS "p" + COLUMNS( + a varchar(1) PATH 'lax "A"', + NESTED PATH 'lax $' AS "p1" + COLUMNS ( + b varchar(1) PATH 'lax "B"', + NESTED PATH 'lax $' AS "p2 "COLUMNS ( + c varchar(1) PATH 'lax "C"', + d varchar(1) PATH 'lax "D"'), + e varchar(1) PATH 'lax "E"'), + f varchar(1) PATH 'lax "F"', + NESTED PATH 'lax $' AS "p3" + COLUMNS (g varchar(1) PATH 'lax "G"'), + h varchar(1) PATH 'lax "H"') + PLAN DEFAULT (CROSS)) + """)) + .matches("VALUES ('A', 'B', 'C', 'D', 'E', 'F', 'G', 'H')"); + } + + @Test + public void testJoinTypes() + { + // implicit CROSS join + assertThat(assertions.query(""" + SELECT * + FROM (VALUES ('[1, 2, 3]'), ('[4, 5, 6, 7, 8]')) t(json_col), + JSON_TABLE( + json_col, + 'lax $[4]' + COLUMNS(a integer PATH 'lax $')) + """)) + .matches("VALUES ('[4, 5, 6, 7, 8]', 8)"); + + // INNER join + assertThat(assertions.query(""" + SELECT * + FROM (VALUES ('[1, 2, 3]'), ('[4, 5, 6, 7, 8]')) t(json_col) + INNER JOIN + JSON_TABLE( + json_col, + 'lax $[4]' + COLUMNS(a integer PATH 'lax $')) + ON TRUE + """)) + .matches("VALUES ('[4, 5, 6, 7, 8]', 8)"); + + // LEFT join + assertThat(assertions.query(""" + SELECT * + FROM (VALUES ('[1, 2, 3]'), ('[4, 5, 6, 7, 8]')) t(json_col) + LEFT JOIN + JSON_TABLE( + json_col, + 'lax $[4]' + COLUMNS(a integer PATH 'lax $')) + ON TRUE + """)) + .matches(""" + VALUES + ('[1, 2, 3]', CAST(null AS integer)), + ('[4, 5, 6, 7, 8]', 8) + """); + + // RIGHT join is effectively INNER. Correlation is not allowed in RIGHT join + assertThat(assertions.query(""" + SELECT * + FROM (VALUES 1) t(x) + RIGHT JOIN + JSON_TABLE( + '[1, 2, 3]', + 'lax $[4]' + COLUMNS(a integer PATH 'lax $')) + ON TRUE + """)) + .returnsEmptyResult(); + + // FULL join. Correlation is not allowed in FULL join + assertThat(assertions.query(""" + SELECT * + FROM (VALUES 1) t(x) + FULL JOIN + JSON_TABLE( + '[1, 2, 3]', + 'lax $[4]' + COLUMNS(a integer PATH 'lax $')) + ON TRUE + """)) + .matches("VALUES (1, CAST(null AS integer))"); + } + + @Test + public void testParentChildRelationship() + { + assertThat(assertions.query(""" + SELECT * + FROM JSON_TABLE( + '[]', + 'lax $' AS "root_path" + COLUMNS( + a varchar(1) PATH 'lax "A"', + NESTED PATH 'lax $[*]' AS "nested_path" + COLUMNS (b varchar(1) PATH 'lax "B"')) + PLAN ("root_path" OUTER "nested_path")) + """)) + .matches("VALUES ('A', CAST(null AS varchar(1)))"); + + assertThat(assertions.query(""" + SELECT * + FROM JSON_TABLE( + '[]', + 'lax $' AS "root_path" + COLUMNS( + a varchar(1) PATH 'lax "A"', + NESTED PATH 'lax $[*]' AS "nested_path" + COLUMNS (b varchar(1) PATH 'lax "B"')) + PLAN ("root_path" INNER "nested_path")) + """)) + .returnsEmptyResult(); + + assertThat(assertions.query(""" + SELECT * + FROM JSON_TABLE( + '[[], [1]]', + 'lax $' AS "root_path" + COLUMNS( + a varchar(1) PATH 'lax "A"', + NESTED PATH 'lax $[*]' AS "nested_path_1" + COLUMNS ( + b varchar(1) PATH 'lax "B"', + NESTED PATH 'lax $[*]' AS "nested_path_2" + COLUMNS( + c varchar(1) PATH 'lax "C"'))) + PLAN ("root_path" OUTER ("nested_path_1" OUTER "nested_path_2"))) + """)) + .matches(""" + VALUES + ('A', 'B', CAST(null AS varchar(1))), + ('A', 'B', 'C') + """); + + assertThat(assertions.query(""" + SELECT * + FROM JSON_TABLE( + '[[], [1]]', + 'lax $' AS "root_path" + COLUMNS( + a varchar(1) PATH 'lax "A"', + NESTED PATH 'lax $[*]' AS "nested_path_1" + COLUMNS ( + b varchar(1) PATH 'lax "B"', + NESTED PATH 'lax $[*]' AS "nested_path_2" + COLUMNS( + c varchar(1) PATH 'lax "C"'))) + PLAN ("root_path" OUTER ("nested_path_1" INNER "nested_path_2"))) + """)) + .matches("VALUES ('A', 'B', 'C')"); + + // intermediately nested path returns empty sequence + assertThat(assertions.query(""" + SELECT * + FROM JSON_TABLE( + '[]', + 'lax $' AS "root_path" + COLUMNS( + a varchar(1) PATH 'lax "A"', + NESTED PATH 'lax $[*]' AS "nested_path_1" + COLUMNS ( + b varchar(1) PATH 'lax "B"', + NESTED PATH 'lax $' AS "nested_path_2" + COLUMNS( + c varchar(1) PATH 'lax "C"'))) + PLAN ("root_path" OUTER ("nested_path_1" INNER "nested_path_2"))) + """)) + .matches("VALUES ('A', CAST(null AS varchar(1)), CAST(null AS varchar(1)))"); + } + + @Test + public void testSiblingsRelationship() + { + // each sibling produces 1 row + assertThat(assertions.query(""" + SELECT * + FROM JSON_TABLE( + '[]', + 'lax $' AS "root_path" + COLUMNS( + a varchar(1) PATH 'lax "A"', + NESTED PATH 'lax $' AS "nested_path_b" + COLUMNS (b varchar(1) PATH 'lax "B"'), + NESTED PATH 'lax $' AS "nested_path_c" + COLUMNS (c varchar(1) PATH 'lax "C"'), + NESTED PATH 'lax $' AS "nested_path_d" + COLUMNS (d varchar(1) PATH 'lax "D"')) + PLAN ("root_path" INNER ("nested_path_c" UNION ("nested_path_d" CROSS "nested_path_b")))) + """)) + .matches(""" + VALUES + ('A', CAST(null AS varchar(1)), 'C', CAST(null AS varchar(1))), + ('A', 'B', CAST(null AS varchar(1)), 'D') + """); + + // each sibling produces 2 rows + assertThat(assertions.query(""" + SELECT * + FROM JSON_TABLE( + '[10, 1000]', + 'lax $' AS "root_path" + COLUMNS( + a varchar(1) PATH 'lax "A"', + NESTED PATH 'lax $[*]' AS "nested_path_1" + COLUMNS (b integer PATH 'lax $ * 1'), + NESTED PATH 'lax $[*]' AS "nested_path_2" + COLUMNS (c integer PATH 'lax $ * 2'), + NESTED PATH 'lax $[*]' AS "nested_path_3" + COLUMNS (d integer PATH 'lax $ * 3')) + PLAN ("root_path" INNER ("nested_path_2" UNION ("nested_path_3" CROSS "nested_path_1")))) + """)) + .matches(""" + VALUES + ('A', CAST(null AS integer), 20, CAST(null AS integer)), + ('A', null, 2000, null), + ('A', 10, null, 30), + ('A', 10, null, 3000), + ('A', 1000, null, 30), + ('A', 1000, null, 3000) + """); + + // one sibling produces empty result -- CROSS result is empty + assertThat(assertions.query(""" + SELECT * + FROM JSON_TABLE( + '[10, 1000]', + 'lax $' AS "root_path" + COLUMNS( + a varchar(1) PATH 'lax "A"', + NESTED PATH 'lax $[*]' AS "nested_path_1" + COLUMNS (b integer PATH 'lax $ * 1'), + NESTED PATH 'lax $[42]' AS "nested_path_2" + COLUMNS (c integer PATH 'lax $ * 2')) + PLAN ("root_path" INNER ("nested_path_1" CROSS "nested_path_2"))) + """)) + .returnsEmptyResult(); + + // one sibling produces empty result -- UNION result contains the other sibling's result + assertThat(assertions.query(""" + SELECT * + FROM JSON_TABLE( + '[10, 1000]', + 'lax $' AS "root_path" + COLUMNS( + a varchar(1) PATH 'lax "A"', + NESTED PATH 'lax $[*]' AS "nested_path_1" + COLUMNS (b integer PATH 'lax $ * 1'), + NESTED PATH 'lax $[42]' AS "nested_path_2" + COLUMNS (c integer PATH 'lax $ * 2')) + PLAN ("root_path" INNER ("nested_path_1" UNION "nested_path_2"))) + """)) + .matches(""" + VALUES + ('A', 10, CAST(null AS integer)), + ('A', 1000, null) + """); + } + + @Test + public void testImplicitColumnPath() + { + assertThat(assertions.query(""" + SELECT * + FROM JSON_TABLE( + '{"A" : 42, "b" : true}', + 'lax $' + COLUMNS( + a integer, + "b" boolean)) + """)) + .matches("VALUES (42, true)"); + + // the implicit column path is 'lax $.C'. It produces empty sequence, so the ON EMPTY clause determines the result + assertThat(assertions.query(""" + SELECT * + FROM JSON_TABLE( + '{"A" : 42, "b" : true}', + 'lax $' + COLUMNS(c varchar (5) DEFAULT 'empty' ON EMPTY DEFAULT 'error' ON ERROR)) + """)) + .matches("VALUES 'empty'"); + } + + @Test + public void testRootPathErrorHandling() + { + // error during root path evaluation handled according to top level EMPTY ON ERROR clause + assertThat(assertions.query(""" + SELECT * + FROM JSON_TABLE( + '[]', + 'strict $[42]' + COLUMNS(a integer PATH 'lax 1') + EMPTY ON ERROR) + """)) + .returnsEmptyResult(); + + // error during root path evaluation handled according to top level ON ERROR clause which defaults to EMPTY ON ERROR + assertThat(assertions.query(""" + SELECT * + FROM JSON_TABLE( + '[]', + 'strict $[42]' + COLUMNS(a integer PATH 'lax 1')) + """)) + .returnsEmptyResult(); + + // error during root path evaluation handled according to top level ERROR ON ERROR clause + assertTrinoExceptionThrownBy(() -> assertions.query(""" + SELECT * + FROM JSON_TABLE( + '[]', + 'strict $[42]' + COLUMNS(a integer PATH 'lax 1') + ERROR ON ERROR) + """)) + .hasErrorCode(PATH_EVALUATION_ERROR) + .hasMessage("path evaluation failed: structural error: invalid array subscript for empty array"); + } + + @Test + public void testNestedPathErrorHandling() + { + // error during nested path evaluation handled according to top level EMPTY ON ERROR clause + assertThat(assertions.query(""" + SELECT * + FROM JSON_TABLE( + '[]', + 'lax $' AS "root_path" + COLUMNS( + a integer PATH 'lax 1', + NESTED PATH 'strict $[42]' AS "nested_path" + COLUMNS(b integer PATH 'lax 2')) + PLAN DEFAULT(INNER) + EMPTY ON ERROR) + """)) + .returnsEmptyResult(); + + // error during nested path evaluation handled according to top level ON ERROR clause which defaults to EMPTY ON ERROR + assertThat(assertions.query(""" + SELECT * + FROM JSON_TABLE( + '[]', + 'lax $' AS "root_path" + COLUMNS( + a integer PATH 'lax 1', + NESTED PATH 'strict $[42]' AS "nested_path" + COLUMNS(b integer PATH 'lax 2')) + PLAN DEFAULT(INNER)) + """)) + .returnsEmptyResult(); + + // error during nested path evaluation handled according to top level ERROR ON ERROR clause + assertTrinoExceptionThrownBy(() -> assertions.query(""" + SELECT * + FROM JSON_TABLE( + '[]', + 'lax $' AS "root_path" + COLUMNS( + a integer PATH 'lax 1', + NESTED PATH 'strict $[42]' AS "nested_path" + COLUMNS(b integer PATH 'lax 2')) + PLAN DEFAULT(INNER) + ERROR ON ERROR) + """)) + .hasErrorCode(PATH_EVALUATION_ERROR) + .hasMessage("path evaluation failed: structural error: invalid array subscript for empty array"); + } + + @Test + public void testColumnPathErrorHandling() + { + // error during column path evaluation handled according to column's ERROR ON ERROR clause + assertTrinoExceptionThrownBy(() -> assertions.query(""" + SELECT * + FROM JSON_TABLE( + '[]', + 'lax $' + COLUMNS(a integer PATH 'strict $[42]' ERROR ON ERROR) + EMPTY ON ERROR) + """)) + .hasErrorCode(PATH_EVALUATION_ERROR) + .hasMessage("path evaluation failed: structural error: invalid array subscript for empty array"); + + // error during column path evaluation handled according to column's ON ERROR clause which defaults to NULL ON ERROR + assertThat(assertions.query(""" + SELECT * + FROM JSON_TABLE( + '[]', + 'lax $' + COLUMNS(a integer PATH 'strict $[42]') + EMPTY ON ERROR) + """)) + .matches("VALUES CAST(null as integer)"); + + // error during column path evaluation handled according to column's ON ERROR clause which defaults to ERROR ON ERROR because the top level error behavior is ERROR ON ERROR + assertTrinoExceptionThrownBy(() -> assertions.query(""" + SELECT * + FROM JSON_TABLE( + '[]', + 'lax $' + COLUMNS(a integer PATH 'strict $[42]') + ERROR ON ERROR) + """)) + .hasErrorCode(PATH_EVALUATION_ERROR) + .hasMessage("path evaluation failed: structural error: invalid array subscript for empty array"); + } + + @Test + public void testEmptyInput() + { + assertThat(assertions.query(""" + SELECT * + FROM (SELECT '[]' WHERE rand() > 1) t(json_col), + JSON_TABLE( + json_col, + 'lax $' + COLUMNS(a integer PATH 'lax 1')) + """)) + .returnsEmptyResult(); + } + + @Test + public void testNullInput() + { + // if input is null, json_table returns empty result + assertThat(assertions.query(""" + SELECT * + FROM JSON_TABLE( + CAST (null AS varchar), + 'lax $' + COLUMNS(a integer PATH 'lax 1')) + """)) + .returnsEmptyResult(); + + assertThat(assertions.query(""" + SELECT * + FROM (VALUES (CAST(null AS varchar)), (CAST(null AS varchar)), (CAST(null AS varchar))) t(json_col), + JSON_TABLE( + json_col, + 'lax $' + COLUMNS(a integer PATH 'lax 1')) + """)) + .returnsEmptyResult(); + + assertThat(assertions.query(""" + SELECT * + FROM (VALUES (CAST(null AS varchar)), (CAST(null AS varchar)), (CAST(null AS varchar))) t(json_col), + JSON_TABLE( + json_col, + 'lax $' + COLUMNS( + NESTED PATH 'lax $' + COLUMNS(a integer PATH 'lax 1'))) + """)) + .returnsEmptyResult(); + + // null as formatted input evaluates to empty sequence. json_table returns empty result + assertThat(assertions.query(""" + SELECT * + FROM JSON_TABLE( + CAST (null AS varchar) FORMAT JSON, + 'lax $' + COLUMNS(a varchar FORMAT JSON PATH 'lax $')) + """)) + .returnsEmptyResult(); + } + + @Test + public void testNullPathParameter() + { + // null as SQL-value parameter "index" is evaluated to a JSON null, and causes type mismatch + assertTrinoExceptionThrownBy(() -> assertions.query(""" + SELECT * + FROM (SELECT '[1, 2, 3]', CAST(null AS integer)) t(json_col, index_col), + JSON_TABLE( + json_col, + 'lax $[$index]' PASSING index_col AS "index" + COLUMNS(a integer PATH 'lax 1') + ERROR ON ERROR) + """)) + .hasErrorCode(PATH_EVALUATION_ERROR) + .hasMessage("path evaluation failed: invalid item type. Expected: NUMBER, actual: NULL"); + + // null as JSON (formatted) parameter "index" evaluates to empty sequence, and causes type mismatch + assertTrinoExceptionThrownBy(() -> assertions.query(""" + SELECT * + FROM (SELECT '[1, 2, 3]', CAST(null AS varchar)) t(json_col, index_col), + JSON_TABLE( + json_col, + 'lax $[$index]' PASSING index_col FORMAT JSON AS "index" + COLUMNS(a integer PATH 'lax 1') + ERROR ON ERROR) + """)) + .hasErrorCode(PATH_EVALUATION_ERROR) + .hasMessage("path evaluation failed: array subscript 'from' value must be singleton numeric"); + } + + @Test + public void testNullDefaultValue() + { + assertThat(assertions.query(""" + SELECT a + FROM (SELECT null) t(empty_default), + JSON_TABLE( + '[1, 2, 3]', + 'lax $' + COLUMNS(a integer PATH 'lax $[42]' DEFAULT empty_default ON EMPTY DEFAULT -1 ON ERROR)) + """)) + .matches("VALUES CAST(null AS integer)"); + + assertThat(assertions.query(""" + SELECT a + FROM (SELECT null) t(error_default), + JSON_TABLE( + '[1, 2, 3]', + 'lax $' + COLUMNS(a integer PATH 'strict $[42]' DEFAULT -1 ON EMPTY DEFAULT error_default ON ERROR)) + """)) + .matches("VALUES CAST(null AS integer)"); + } + + @Test + public void testValueColumnCoercion() + { + // returned value cast to declared type + assertThat(assertions.query(""" + SELECT * + FROM JSON_TABLE( + '[1, 2, 3]', + 'lax $' + COLUMNS(a real PATH 'lax $[last]')) + """)) + .matches("VALUES REAL '3'"); + + // default value cast to declared type + assertThat(assertions.query(""" + SELECT * + FROM JSON_TABLE( + '[1, 2, 3]', + 'lax $' + COLUMNS(a real PATH 'lax $[42]' DEFAULT 42 ON EMPTY)) + """)) + .matches("VALUES REAL '42'"); + + // default ON EMPTY value is null. It is cast to declared type + assertThat(assertions.query(""" + SELECT * + FROM JSON_TABLE( + '[1, 2, 3]', + 'lax $' + COLUMNS(a real PATH 'lax $[42]')) + """)) + .matches("VALUES CAST(null AS REAL)"); + + // default value cast to declared type + assertThat(assertions.query(""" + SELECT * + FROM JSON_TABLE( + '[1, 2, 3]', + 'lax $' + COLUMNS(a real PATH 'strict $[42]' DEFAULT 42 ON ERROR)) + """)) + .matches("VALUES REAL '42'"); + + // default ON ERROR value is null. It is cast to declared type + assertThat(assertions.query(""" + SELECT * + FROM JSON_TABLE( + '[1, 2, 3]', + 'lax $' + COLUMNS(a real PATH 'strict $[42]')) + """)) + .matches("VALUES CAST(null AS REAL)"); + } + + @Test + public void testQueryColumnFormat() + { + assertThat(assertions.query(""" + SELECT * + FROM JSON_TABLE( + '[{"a" : true}]', + 'lax $' + COLUMNS(a varchar(50) FORMAT JSON PATH 'lax $[0]')) + """)) + .matches("VALUES CAST('{\"a\":true}' AS VARCHAR(50))"); + + String varbinaryLiteral = "X'" + base16().encode("{\"a\":true}".getBytes(UTF_16LE)) + "'"; + assertThat(assertions.query(""" + SELECT * + FROM JSON_TABLE( + '[{"a" : true}]', + 'lax $' + COLUMNS(a varbinary FORMAT JSON ENCODING UTF16 PATH 'lax $[0]')) + """)) + .matches("VALUES " + varbinaryLiteral); + + assertThat(assertions.query(""" + SELECT * + FROM JSON_TABLE( + '[{"a" : true}]', + 'lax $' + COLUMNS(a char(50) FORMAT JSON PATH 'lax $[42]' EMPTY OBJECT ON EMPTY)) + """)) + .matches("VALUES CAST('{}' AS CHAR(50))"); + + varbinaryLiteral = "X'" + base16().encode("[]".getBytes(UTF_16LE)) + "'"; + assertThat(assertions.query(""" + SELECT * + FROM JSON_TABLE( + '[{"a" : true}]', + 'lax $' + COLUMNS(a varbinary FORMAT JSON ENCODING UTF16 PATH 'strict $[42]' EMPTY ARRAY ON ERROR)) + """)) + .matches("VALUES " + varbinaryLiteral); + + assertThat(assertions.query(""" + SELECT * + FROM JSON_TABLE( + '[{"a" : true}]', + 'lax $' + COLUMNS(a varbinary FORMAT JSON ENCODING UTF16 PATH 'lax $[42]' NULL ON EMPTY)) + """)) + .matches("VALUES CAST(null AS VARBINARY)"); + } + + @Test + public void testOrdinalityColumn() + { + assertThat(assertions.query(""" + SELECT * + FROM JSON_TABLE( + '["a", "b", "c", "d", "e", "f", "g", "h"]', + 'lax $[*]' AS "root_path" + COLUMNS( + o FOR ORDINALITY, + x varchar(1) PATH 'lax $')) + """)) + .matches(""" + VALUES + (BIGINT '1', 'a'), + (2, 'b'), + (3, 'c'), + (4, 'd'), + (5, 'e'), + (6, 'f'), + (7, 'g'), + (8, 'h') + """); + + assertThat(assertions.query(""" + SELECT * + FROM (VALUES + ('[["a", "b"], ["c", "d"], ["e", "f"]]'), + ('[["g", "h"], ["i", "j"], ["k", "l"]]')) t(json_col), + JSON_TABLE( + json_col, + 'lax $' AS "root_path" + COLUMNS( + o FOR ORDINALITY, + NESTED PATH 'lax $[0][*]' AS "nested_path_1" + COLUMNS ( + x1 varchar PATH 'lax $', + o1 FOR ORDINALITY), + NESTED PATH 'lax $[1][*]' AS "nested_path_2" + COLUMNS ( + x2 varchar PATH 'lax $', + o2 FOR ORDINALITY), + NESTED PATH 'lax $[2][*]' AS "nested_path_3" + COLUMNS ( + x3 varchar PATH 'lax $', + o3 FOR ORDINALITY)) + PLAN ("root_path" INNER ("nested_path_2" UNION ("nested_path_3" CROSS "nested_path_1")))) + """)) + .matches(""" + VALUES + ('[["a", "b"], ["c", "d"], ["e", "f"]]', BIGINT '1', VARCHAR 'a', BIGINT '1', CAST(null AS varchar), CAST(null AS bigint), VARCHAR 'e', BIGINT '1'), + ('[["a", "b"], ["c", "d"], ["e", "f"]]', 1, 'a', 1, null, null, 'f', 2), + ('[["a", "b"], ["c", "d"], ["e", "f"]]', 1, 'b', 2, null, null, 'e', 1), + ('[["a", "b"], ["c", "d"], ["e", "f"]]', 1, 'b', 2, null, null, 'f', 2), + ('[["a", "b"], ["c", "d"], ["e", "f"]]', 1, null, null, 'c', 1, null, null), + ('[["a", "b"], ["c", "d"], ["e", "f"]]', 1, null, null, 'd', 2, null, null), + + ('[["g", "h"], ["i", "j"], ["k", "l"]]', 1, VARCHAR 'g', BIGINT '1', CAST(null AS varchar), CAST(null AS bigint), VARCHAR 'k', BIGINT '1'), + ('[["g", "h"], ["i", "j"], ["k", "l"]]', 1, 'g', 1, null, null, 'l', 2), + ('[["g", "h"], ["i", "j"], ["k", "l"]]', 1, 'h', 2, null, null, 'k', 1), + ('[["g", "h"], ["i", "j"], ["k", "l"]]', 1, 'h', 2, null, null, 'l', 2), + ('[["g", "h"], ["i", "j"], ["k", "l"]]', 1, null, null, 'i', 1, null, null), + ('[["g", "h"], ["i", "j"], ["k", "l"]]', 1, null, null, 'j', 2, null, null) + """); + } +} diff --git a/core/trino-main/src/test/java/io/trino/type/TestJsonPath2016TypeSerialization.java b/core/trino-main/src/test/java/io/trino/type/TestJsonPath2016TypeSerialization.java index 7e28a1efc9ebf..70cbf37a7d6c1 100644 --- a/core/trino-main/src/test/java/io/trino/type/TestJsonPath2016TypeSerialization.java +++ b/core/trino-main/src/test/java/io/trino/type/TestJsonPath2016TypeSerialization.java @@ -71,7 +71,7 @@ public class TestJsonPath2016TypeSerialization { - private static final Type JSON_PATH_2016 = new JsonPath2016Type(new TypeDeserializer(TESTING_TYPE_MANAGER), new TestingBlockEncodingSerde()); + public static final Type JSON_PATH_2016 = new JsonPath2016Type(new TypeDeserializer(TESTING_TYPE_MANAGER), new TestingBlockEncodingSerde()); private static final RecursiveComparisonConfiguration COMPARISON_CONFIGURATION = RecursiveComparisonConfiguration.builder().withStrictTypeChecking(true).build(); @Test diff --git a/core/trino-parser/src/main/java/io/trino/sql/jsonpath/PathParser.java b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/PathParser.java index e3be63349577b..b15687e98550e 100644 --- a/core/trino-parser/src/main/java/io/trino/sql/jsonpath/PathParser.java +++ b/core/trino-parser/src/main/java/io/trino/sql/jsonpath/PathParser.java @@ -40,13 +40,13 @@ public final class PathParser { private final BaseErrorListener errorListener; - public PathParser(Location startLocation) + public static PathParser withRelativeErrorLocation(Location startLocation) { requireNonNull(startLocation, "startLocation is null"); int pathStartLine = startLocation.line(); int pathStartColumn = startLocation.column(); - this.errorListener = new BaseErrorListener() + return new PathParser(new BaseErrorListener() { @Override public void syntaxError(Recognizer recognizer, Object offendingSymbol, int line, int charPositionInLine, String message, RecognitionException e) @@ -58,7 +58,26 @@ public void syntaxError(Recognizer recognizer, Object offendingSymbol, int int columnInQuery = line == 1 ? pathStartColumn + 1 + charPositionInLine : charPositionInLine + 1; throw new ParsingException(message, e, lineInQuery, columnInQuery); } - }; + }); + } + + public static PathParser withFixedErrorLocation(Location location) + { + requireNonNull(location, "location is null"); + + return new PathParser(new BaseErrorListener() + { + @Override + public void syntaxError(Recognizer recognizer, Object offendingSymbol, int line, int charPositionInLine, String message, RecognitionException e) + { + throw new ParsingException(message, e, location.line, location.column); + } + }); + } + + private PathParser(BaseErrorListener errorListener) + { + this.errorListener = requireNonNull(errorListener, "errorListener is null"); } public PathNode parseJsonPath(String path) diff --git a/core/trino-parser/src/test/java/io/trino/sql/jsonpath/TestPathParser.java b/core/trino-parser/src/test/java/io/trino/sql/jsonpath/TestPathParser.java index f294b2299d69c..b645c75c0b3e6 100644 --- a/core/trino-parser/src/test/java/io/trino/sql/jsonpath/TestPathParser.java +++ b/core/trino-parser/src/test/java/io/trino/sql/jsonpath/TestPathParser.java @@ -75,7 +75,7 @@ public class TestPathParser { - private static final PathParser PATH_PARSER = new PathParser(new Location(1, 0)); + private static final PathParser PATH_PARSER = PathParser.withRelativeErrorLocation(new Location(1, 0)); private static final RecursiveComparisonConfiguration COMPARISON_CONFIGURATION = RecursiveComparisonConfiguration.builder().withStrictTypeChecking(true).build(); @Test diff --git a/core/trino-spi/src/main/java/io/trino/spi/StandardErrorCode.java b/core/trino-spi/src/main/java/io/trino/spi/StandardErrorCode.java index 9500f3e4a66b0..4751394a433dd 100644 --- a/core/trino-spi/src/main/java/io/trino/spi/StandardErrorCode.java +++ b/core/trino-spi/src/main/java/io/trino/spi/StandardErrorCode.java @@ -148,6 +148,9 @@ public enum StandardErrorCode INVALID_CATALOG_PROPERTY(124, USER_ERROR), CATALOG_UNAVAILABLE(125, USER_ERROR), MISSING_RETURN(126, USER_ERROR), + DUPLICATE_COLUMN_OR_PATH_NAME(127, USER_ERROR), + MISSING_PATH_NAME(128, USER_ERROR), + INVALID_PLAN(129, USER_ERROR), GENERIC_INTERNAL_ERROR(65536, INTERNAL_ERROR), TOO_MANY_REQUESTS_FAILED(65537, INTERNAL_ERROR), From 1fe285f1574b1965151a5d3c5db1c7640d6f0ee6 Mon Sep 17 00:00:00 2001 From: Manfred Moser Date: Wed, 13 Dec 2023 17:25:14 -0800 Subject: [PATCH 126/350] Add docs for elasticsearch.tls.verify-hostnames --- docs/src/main/sphinx/connector/elasticsearch.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/src/main/sphinx/connector/elasticsearch.md b/docs/src/main/sphinx/connector/elasticsearch.md index 337c638b42efd..b3ae9fd0e032f 100644 --- a/docs/src/main/sphinx/connector/elasticsearch.md +++ b/docs/src/main/sphinx/connector/elasticsearch.md @@ -112,6 +112,9 @@ The allowed configuration values are: * - `elasticsearch.tls.truststore-password` - The key password for the trust store specified by `elasticsearch.tls.truststore-path`. +* - `elasticsearch.tls.verify-hostnames` + - Flag to determine if the hostnames in the certificates must be verified. Defaults + to `true`. ::: (elasticesearch-type-mapping)= From 7ab1e590bc9961f89771f05f6923f56e1bc62e71 Mon Sep 17 00:00:00 2001 From: Martin Traverso Date: Wed, 13 Dec 2023 16:34:43 -0800 Subject: [PATCH 127/350] Revert "Allow differentiating materialized views from tables efficiently" This needs more though and a different approach that does not involve modifying the information_schema tables. This reverts commit 1c5e5eb0d3949ff5d6f6bfd42a72428b4a2d5668. --- .../InformationSchemaPageSource.java | 31 +++++++------------ .../InformationSchemaTable.java | 1 - .../io/trino/testing/BaseConnectorTest.java | 8 ++--- 3 files changed, 15 insertions(+), 25 deletions(-) diff --git a/core/trino-main/src/main/java/io/trino/connector/informationschema/InformationSchemaPageSource.java b/core/trino-main/src/main/java/io/trino/connector/informationschema/InformationSchemaPageSource.java index b7a7f6ee41862..143cc6493c63d 100644 --- a/core/trino-main/src/main/java/io/trino/connector/informationschema/InformationSchemaPageSource.java +++ b/core/trino-main/src/main/java/io/trino/connector/informationschema/InformationSchemaPageSource.java @@ -277,43 +277,34 @@ private void addColumnsRecords(QualifiedTablePrefix prefix) private void addTablesRecords(QualifiedTablePrefix prefix) { - boolean needsTableType = requiredColumns.contains("table_type") || requiredColumns.contains("trino_relation_type"); + boolean needsTableType = requiredColumns.contains("table_type"); Set relations; - Map relationTypes; + Set views; if (needsTableType) { - relationTypes = getRelationTypes(session, metadata, accessControl, prefix); + Map relationTypes = getRelationTypes(session, metadata, accessControl, prefix); relations = relationTypes.keySet(); + views = relationTypes.entrySet().stream() + .filter(entry -> entry.getValue() == RelationType.VIEW) + .map(Entry::getKey) + .collect(toImmutableSet()); } else { relations = listTables(session, metadata, accessControl, prefix); - relationTypes = null; + views = Set.of(); } + // TODO (https://github.com/trinodb/trino/issues/8207) define a type for materialized views for (SchemaTableName name : relations) { String type = null; - String trinoRelationType = null; if (needsTableType) { - switch (relationTypes.get(name)) { - case TABLE -> { - type = "BASE TABLE"; - trinoRelationType = type; - } - case VIEW -> { - type = "VIEW"; - trinoRelationType = type; - } - case MATERIALIZED_VIEW -> { - type = "BASE TABLE"; - trinoRelationType = "MATERIALIZED VIEW"; - } - } + // if table and view names overlap, the view wins + type = views.contains(name) ? "VIEW" : "BASE TABLE"; } addRecord( prefix.getCatalogName(), name.getSchemaName(), name.getTableName(), type, - trinoRelationType, null); if (isLimitExhausted()) { return; diff --git a/core/trino-main/src/main/java/io/trino/connector/informationschema/InformationSchemaTable.java b/core/trino-main/src/main/java/io/trino/connector/informationschema/InformationSchemaTable.java index e45bb8594d742..605076aa38419 100644 --- a/core/trino-main/src/main/java/io/trino/connector/informationschema/InformationSchemaTable.java +++ b/core/trino-main/src/main/java/io/trino/connector/informationschema/InformationSchemaTable.java @@ -47,7 +47,6 @@ public enum InformationSchemaTable .column("table_schema", createUnboundedVarcharType()) .column("table_name", createUnboundedVarcharType()) .column("table_type", createUnboundedVarcharType()) - .hiddenColumn("trino_relation_type", createUnboundedVarcharType()) .hiddenColumn("table_comment", createUnboundedVarcharType()) // MySQL compatible .build()), VIEWS(table("views") diff --git a/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java b/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java index 214130af3770d..dd00a554460a7 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java @@ -1052,15 +1052,15 @@ public void testMaterializedView() .containsAll("VALUES '" + view.getObjectName() + "'"); // information_schema.tables without table_name filter so that ConnectorMetadata.listViews is exercised assertThat(query( - "SELECT table_name, table_type, trino_relation_type FROM information_schema.tables " + + "SELECT table_name, table_type FROM information_schema.tables " + "WHERE table_schema = '" + view.getSchemaName() + "'")) .skippingTypesCheck() - .containsAll("VALUES ('" + view.getObjectName() + "', 'BASE TABLE', 'MATERIALIZED VIEW')"); + .containsAll("VALUES ('" + view.getObjectName() + "', 'BASE TABLE')"); // information_schema.tables with table_name filter assertQuery( - "SELECT table_name, table_type, trino_relation_type FROM information_schema.tables " + + "SELECT table_name, table_type FROM information_schema.tables " + "WHERE table_schema = '" + view.getSchemaName() + "' and table_name = '" + view.getObjectName() + "'", - "VALUES ('" + view.getObjectName() + "', 'BASE TABLE', 'MATERIALIZED VIEW')"); + "VALUES ('" + view.getObjectName() + "', 'BASE TABLE')"); // system.jdbc.tables without filter assertThat(query("SELECT table_schem, table_name, table_type FROM system.jdbc.tables")) From da534693ef6961c38cd59f0b2d1347d7a9784831 Mon Sep 17 00:00:00 2001 From: Manfred Moser Date: Fri, 1 Dec 2023 14:33:33 -0800 Subject: [PATCH 128/350] Add Trino 435 release notes --- docs/src/main/sphinx/release.md | 6 +- docs/src/main/sphinx/release/release-435.md | 82 +++++++++++++++++++++ 2 files changed, 83 insertions(+), 5 deletions(-) create mode 100644 docs/src/main/sphinx/release/release-435.md diff --git a/docs/src/main/sphinx/release.md b/docs/src/main/sphinx/release.md index 056cfe9d7f9b7..1457ac89a4a3c 100644 --- a/docs/src/main/sphinx/release.md +++ b/docs/src/main/sphinx/release.md @@ -1,12 +1,12 @@ # Release notes (releases-2023)= - ## 2023 ```{toctree} :maxdepth: 1 +release/release-435 release/release-434 release/release-433 release/release-432 @@ -39,7 +39,6 @@ release/release-406 ``` (releases-2022)= - ## 2022 ```{toctree} @@ -86,7 +85,6 @@ release/release-368 ``` (releases-2021)= - ## 2021 ```{toctree} @@ -112,7 +110,6 @@ release/release-351 ``` (releases-2020)= - ## 2020 ```{toctree} @@ -144,7 +141,6 @@ release/release-328 ``` (releases-2019)= - ## 2019 ```{toctree} diff --git a/docs/src/main/sphinx/release/release-435.md b/docs/src/main/sphinx/release/release-435.md new file mode 100644 index 0000000000000..ccb6aef6d5e89 --- /dev/null +++ b/docs/src/main/sphinx/release/release-435.md @@ -0,0 +1,82 @@ +# Release 435 (13 Dec 2023) + +## General + +* Add support for the `json_table` table function. ({issue}`18017`) +* Reduce coordinator memory usage. ({issue}`20018`, {issue}`20022`) +* Increase reliability and memory consumption of inserts. ({issue}`20040`) +* Fix incorrect results for `LIKE` with some strings containing repeated + substrings. ({issue}`20089`) +* Fix coordinator memory leak. ({issue}`20023`) +* Fix possible query failure for `MERGE` queries when `retry-policy` set to + `TASK` and `query.determine-partition-count-for-write-enabled` set to `true`. + ({issue}`19979`) +* Prevent hanging query processing with `retry.policy` set to `TASK` when a + worker node died. ({issue}`18603 `) +* Fix query failure when reading array columns. ({issue}`20065`) + +## Delta Lake connector + +* {{breaking}} Remove support for registering external tables with + `CREATE TABLE` and the `location` table property. Use the + `register_table` procedure as replacement. The property + `delta.legacy-create-table-with-existing-location.enabled` is + also removed. ({issue}`17016`) +* Improve query planning performance on Delta Lake tables. ({issue}`19795`) +* Ensure AWS access keys are used for connections to the AWS Security Token + Service. ({issue}`19982`) +* Reduce memory usage for inserts into partitioned tables. ({issue}`19649`) +* Improve reliability when reading from GCS. ({issue}`20003`) +* Fix failure when reading ORC data. ({issue}`19935`) + +## Elasticsearch connector + +* Ensure certificate validation is skipped when + `elasticsearch.tls.verify-hostnames` is `false`. ({issue}`20076`) + +## Hive connector + +* Add support for columns that changed from integer types to `decimal` type. ({issue}`19931`) +* Add support for columns that changed from `date` to `varchar` type. ({issue}`19500`) +* Rename `presto_version` table property to `trino_version`. ({issue}`19967`) +* Rename `presto_query_id` table property to `trino_query_id`. ({issue}`19967`) +* Ensure AWS access keys are used for connections to the AWS Security Token + Service. ({issue}`19982`) +* Improve query planning time on Hive tables without statistics. ({issue}`20034`) +* Reduce memory usage for inserts into partitioned tables. ({issue}`19649`) +* Improve reliability when reading from GCS. ({issue}`20003`) +* Fix failure when reading ORC data. ({issue}`19935`) + +## Hudi connector + +* Ensure AWS access keys are used for connections to the AWS Security Token + Service. ({issue}`19982`) +* Improve reliability when reading from GCS. ({issue}`20003`) +* Fix failure when reading ORC data. ({issue}`19935`) + +## Iceberg connector + +* Fix incorrect removal of statistics files when executing + `remove_orphan_files`. ({issue}`19965`) +* Ensure AWS access keys are used for connections to the AWS Security Token + Service. ({issue}`19982`) +* Improve performance of metadata queries involving materialized views. ({issue}`19939`) +* Reduce memory usage for inserts into partitioned tables. ({issue}`19649`) +* Improve reliability when reading from GCS. ({issue}`20003`) +* Fix failure when reading ORC data. ({issue}`19935`) + +## Ignite connector + +* Improve performance of queries involving `OR` with `IS NULL`, `IS NOT NULL` + predicates, or involving `NOT` expression by pushing predicate computation to + the Ignite database. ({issue}`19453`) + +## MongoDB connector + +* Allow configuration to use local scheduling of MongoDB splits with + `mongodb.allow-local-scheduling`. ({issue}`20078`) + +## SQL Server connector + +* Fix incorrect results when reading dates between `1582-10-05` and + `1582-10-14`. ({issue}`20005`) From c11e77e08431999ec80c636224889ddcf8d6267d Mon Sep 17 00:00:00 2001 From: Martin Traverso Date: Thu, 14 Dec 2023 04:11:34 +0000 Subject: [PATCH 129/350] [maven-release-plugin] prepare release 435 --- client/trino-cli/pom.xml | 2 +- client/trino-client/pom.xml | 2 +- client/trino-jdbc/pom.xml | 2 +- core/trino-grammar/pom.xml | 2 +- core/trino-main/pom.xml | 2 +- core/trino-parser/pom.xml | 2 +- core/trino-server-main/pom.xml | 2 +- core/trino-server-rpm/pom.xml | 2 +- core/trino-server/pom.xml | 2 +- core/trino-spi/pom.xml | 2 +- docs/pom.xml | 2 +- lib/trino-array/pom.xml | 2 +- lib/trino-cache/pom.xml | 2 +- lib/trino-filesystem-azure/pom.xml | 2 +- lib/trino-filesystem-gcs/pom.xml | 2 +- lib/trino-filesystem-manager/pom.xml | 2 +- lib/trino-filesystem-s3/pom.xml | 2 +- lib/trino-filesystem/pom.xml | 2 +- lib/trino-geospatial-toolkit/pom.xml | 2 +- lib/trino-hdfs/pom.xml | 2 +- lib/trino-hive-formats/pom.xml | 2 +- lib/trino-ignite-patched/pom.xml | 2 +- lib/trino-matching/pom.xml | 2 +- lib/trino-memory-context/pom.xml | 2 +- lib/trino-orc/pom.xml | 2 +- lib/trino-parquet/pom.xml | 2 +- lib/trino-phoenix5-patched/pom.xml | 2 +- lib/trino-plugin-toolkit/pom.xml | 2 +- lib/trino-record-decoder/pom.xml | 2 +- plugin/trino-accumulo-iterators/pom.xml | 2 +- plugin/trino-accumulo/pom.xml | 2 +- plugin/trino-atop/pom.xml | 2 +- plugin/trino-base-jdbc/pom.xml | 2 +- plugin/trino-bigquery/pom.xml | 2 +- plugin/trino-blackhole/pom.xml | 2 +- plugin/trino-cassandra/pom.xml | 2 +- plugin/trino-clickhouse/pom.xml | 2 +- plugin/trino-delta-lake/pom.xml | 2 +- plugin/trino-druid/pom.xml | 2 +- plugin/trino-elasticsearch/pom.xml | 2 +- plugin/trino-example-http/pom.xml | 2 +- plugin/trino-example-jdbc/pom.xml | 2 +- plugin/trino-exchange-filesystem/pom.xml | 2 +- plugin/trino-exchange-hdfs/pom.xml | 2 +- plugin/trino-geospatial/pom.xml | 2 +- plugin/trino-google-sheets/pom.xml | 2 +- plugin/trino-hive/pom.xml | 2 +- plugin/trino-http-event-listener/pom.xml | 2 +- plugin/trino-hudi/pom.xml | 2 +- plugin/trino-iceberg/pom.xml | 2 +- plugin/trino-ignite/pom.xml | 2 +- plugin/trino-jmx/pom.xml | 2 +- plugin/trino-kafka/pom.xml | 2 +- plugin/trino-kinesis/pom.xml | 2 +- plugin/trino-kudu/pom.xml | 2 +- plugin/trino-local-file/pom.xml | 2 +- plugin/trino-mariadb/pom.xml | 2 +- plugin/trino-memory/pom.xml | 2 +- plugin/trino-ml/pom.xml | 2 +- plugin/trino-mongodb/pom.xml | 2 +- plugin/trino-mysql-event-listener/pom.xml | 2 +- plugin/trino-mysql/pom.xml | 2 +- plugin/trino-oracle/pom.xml | 2 +- plugin/trino-password-authenticators/pom.xml | 2 +- plugin/trino-phoenix5/pom.xml | 2 +- plugin/trino-pinot/pom.xml | 2 +- plugin/trino-postgresql/pom.xml | 2 +- plugin/trino-prometheus/pom.xml | 2 +- plugin/trino-raptor-legacy/pom.xml | 2 +- plugin/trino-redis/pom.xml | 2 +- plugin/trino-redshift/pom.xml | 2 +- plugin/trino-resource-group-managers/pom.xml | 2 +- plugin/trino-session-property-managers/pom.xml | 2 +- plugin/trino-singlestore/pom.xml | 2 +- plugin/trino-sqlserver/pom.xml | 2 +- plugin/trino-teradata-functions/pom.xml | 2 +- plugin/trino-thrift-api/pom.xml | 2 +- plugin/trino-thrift-testing-server/pom.xml | 2 +- plugin/trino-thrift/pom.xml | 2 +- plugin/trino-tpcds/pom.xml | 2 +- plugin/trino-tpch/pom.xml | 2 +- pom.xml | 4 ++-- service/trino-proxy/pom.xml | 2 +- service/trino-verifier/pom.xml | 2 +- testing/trino-benchmark-queries/pom.xml | 2 +- testing/trino-benchto-benchmarks/pom.xml | 2 +- testing/trino-faulttolerant-tests/pom.xml | 2 +- testing/trino-plugin-reader/pom.xml | 2 +- testing/trino-product-tests-launcher/pom.xml | 2 +- testing/trino-product-tests/pom.xml | 2 +- testing/trino-server-dev/pom.xml | 2 +- testing/trino-test-jdbc-compatibility-old-driver/pom.xml | 4 ++-- testing/trino-test-jdbc-compatibility-old-server/pom.xml | 2 +- testing/trino-testing-containers/pom.xml | 2 +- testing/trino-testing-kafka/pom.xml | 2 +- testing/trino-testing-resources/pom.xml | 2 +- testing/trino-testing-services/pom.xml | 2 +- testing/trino-testing/pom.xml | 2 +- testing/trino-tests/pom.xml | 2 +- 99 files changed, 101 insertions(+), 101 deletions(-) diff --git a/client/trino-cli/pom.xml b/client/trino-cli/pom.xml index 7720045b66874..e7f7ca0dff4b4 100644 --- a/client/trino-cli/pom.xml +++ b/client/trino-cli/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/client/trino-client/pom.xml b/client/trino-client/pom.xml index 27587ca34d0af..fe84dc67d8f28 100644 --- a/client/trino-client/pom.xml +++ b/client/trino-client/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/client/trino-jdbc/pom.xml b/client/trino-jdbc/pom.xml index b3115a4b63c8a..1e882f5a8b354 100644 --- a/client/trino-jdbc/pom.xml +++ b/client/trino-jdbc/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/core/trino-grammar/pom.xml b/core/trino-grammar/pom.xml index 75cf06246cc92..0c84295de7556 100644 --- a/core/trino-grammar/pom.xml +++ b/core/trino-grammar/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/core/trino-main/pom.xml b/core/trino-main/pom.xml index 1e0a8050611be..96c729be03970 100644 --- a/core/trino-main/pom.xml +++ b/core/trino-main/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/core/trino-parser/pom.xml b/core/trino-parser/pom.xml index fa2fe342cbccd..3de822d9da636 100644 --- a/core/trino-parser/pom.xml +++ b/core/trino-parser/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/core/trino-server-main/pom.xml b/core/trino-server-main/pom.xml index 247c271598537..edeaf26c22aeb 100644 --- a/core/trino-server-main/pom.xml +++ b/core/trino-server-main/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/core/trino-server-rpm/pom.xml b/core/trino-server-rpm/pom.xml index 29c66bd4f0fdd..49af423f6b2cd 100644 --- a/core/trino-server-rpm/pom.xml +++ b/core/trino-server-rpm/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/core/trino-server/pom.xml b/core/trino-server/pom.xml index b538b06c7047a..783d594eae75e 100644 --- a/core/trino-server/pom.xml +++ b/core/trino-server/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/core/trino-spi/pom.xml b/core/trino-spi/pom.xml index 4c0ffdc0393fc..583d5641b0771 100644 --- a/core/trino-spi/pom.xml +++ b/core/trino-spi/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/docs/pom.xml b/docs/pom.xml index fb14778bf5b9a..de9905d652818 100644 --- a/docs/pom.xml +++ b/docs/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 trino-docs diff --git a/lib/trino-array/pom.xml b/lib/trino-array/pom.xml index b3a566e618fbd..49f197d59cd37 100644 --- a/lib/trino-array/pom.xml +++ b/lib/trino-array/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/lib/trino-cache/pom.xml b/lib/trino-cache/pom.xml index 63848a176b006..2190504c84e8b 100644 --- a/lib/trino-cache/pom.xml +++ b/lib/trino-cache/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/lib/trino-filesystem-azure/pom.xml b/lib/trino-filesystem-azure/pom.xml index f637bfa330179..ec154178de574 100644 --- a/lib/trino-filesystem-azure/pom.xml +++ b/lib/trino-filesystem-azure/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/lib/trino-filesystem-gcs/pom.xml b/lib/trino-filesystem-gcs/pom.xml index a958870f39267..da21089f6b058 100644 --- a/lib/trino-filesystem-gcs/pom.xml +++ b/lib/trino-filesystem-gcs/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/lib/trino-filesystem-manager/pom.xml b/lib/trino-filesystem-manager/pom.xml index 902503eab98e9..b5fe87df132fc 100644 --- a/lib/trino-filesystem-manager/pom.xml +++ b/lib/trino-filesystem-manager/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/lib/trino-filesystem-s3/pom.xml b/lib/trino-filesystem-s3/pom.xml index b8c1165d8d114..d72dc0cec2398 100644 --- a/lib/trino-filesystem-s3/pom.xml +++ b/lib/trino-filesystem-s3/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/lib/trino-filesystem/pom.xml b/lib/trino-filesystem/pom.xml index 7f78e77bcd86b..ebc7c70284e5a 100644 --- a/lib/trino-filesystem/pom.xml +++ b/lib/trino-filesystem/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/lib/trino-geospatial-toolkit/pom.xml b/lib/trino-geospatial-toolkit/pom.xml index 181ce54f0bf31..ebb3e51bc76b6 100644 --- a/lib/trino-geospatial-toolkit/pom.xml +++ b/lib/trino-geospatial-toolkit/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/lib/trino-hdfs/pom.xml b/lib/trino-hdfs/pom.xml index 70fec6c06ea4d..40dca75268b5b 100644 --- a/lib/trino-hdfs/pom.xml +++ b/lib/trino-hdfs/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/lib/trino-hive-formats/pom.xml b/lib/trino-hive-formats/pom.xml index a96634b81a87b..55e497a09dcc7 100644 --- a/lib/trino-hive-formats/pom.xml +++ b/lib/trino-hive-formats/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/lib/trino-ignite-patched/pom.xml b/lib/trino-ignite-patched/pom.xml index 3b3c7d32d77e5..e086e2d49828c 100644 --- a/lib/trino-ignite-patched/pom.xml +++ b/lib/trino-ignite-patched/pom.xml @@ -6,7 +6,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/lib/trino-matching/pom.xml b/lib/trino-matching/pom.xml index c53931f7d6a90..eacedb134a690 100644 --- a/lib/trino-matching/pom.xml +++ b/lib/trino-matching/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/lib/trino-memory-context/pom.xml b/lib/trino-memory-context/pom.xml index 63f297577970d..0086feff1cdec 100644 --- a/lib/trino-memory-context/pom.xml +++ b/lib/trino-memory-context/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/lib/trino-orc/pom.xml b/lib/trino-orc/pom.xml index c18b8d87eda0b..869c4e40b58fc 100644 --- a/lib/trino-orc/pom.xml +++ b/lib/trino-orc/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/lib/trino-parquet/pom.xml b/lib/trino-parquet/pom.xml index 4bf90122f230b..61db7477a413f 100644 --- a/lib/trino-parquet/pom.xml +++ b/lib/trino-parquet/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/lib/trino-phoenix5-patched/pom.xml b/lib/trino-phoenix5-patched/pom.xml index b5d69023d0ef0..79291dcc88165 100644 --- a/lib/trino-phoenix5-patched/pom.xml +++ b/lib/trino-phoenix5-patched/pom.xml @@ -6,7 +6,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/lib/trino-plugin-toolkit/pom.xml b/lib/trino-plugin-toolkit/pom.xml index 74332aa4cb8e0..8a55f61d54842 100644 --- a/lib/trino-plugin-toolkit/pom.xml +++ b/lib/trino-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/lib/trino-record-decoder/pom.xml b/lib/trino-record-decoder/pom.xml index 5e7a0548029b0..eabdb5ded2b49 100644 --- a/lib/trino-record-decoder/pom.xml +++ b/lib/trino-record-decoder/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-accumulo-iterators/pom.xml b/plugin/trino-accumulo-iterators/pom.xml index dbe4589d0ac4e..d1a4613f2553d 100644 --- a/plugin/trino-accumulo-iterators/pom.xml +++ b/plugin/trino-accumulo-iterators/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-accumulo/pom.xml b/plugin/trino-accumulo/pom.xml index 5d07c63c19277..dbaa1d6448453 100644 --- a/plugin/trino-accumulo/pom.xml +++ b/plugin/trino-accumulo/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-atop/pom.xml b/plugin/trino-atop/pom.xml index 032fc348bb4f0..4b0db3e036a89 100644 --- a/plugin/trino-atop/pom.xml +++ b/plugin/trino-atop/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-base-jdbc/pom.xml b/plugin/trino-base-jdbc/pom.xml index 7d9e8a828e8e1..41a80c9d02933 100644 --- a/plugin/trino-base-jdbc/pom.xml +++ b/plugin/trino-base-jdbc/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-bigquery/pom.xml b/plugin/trino-bigquery/pom.xml index 446a8930c7a42..1cb09cf8c0837 100644 --- a/plugin/trino-bigquery/pom.xml +++ b/plugin/trino-bigquery/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-blackhole/pom.xml b/plugin/trino-blackhole/pom.xml index d76dc1989adc1..d273e7e1a5272 100644 --- a/plugin/trino-blackhole/pom.xml +++ b/plugin/trino-blackhole/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-cassandra/pom.xml b/plugin/trino-cassandra/pom.xml index 16c5f45ada009..a77dd84bf6d5f 100644 --- a/plugin/trino-cassandra/pom.xml +++ b/plugin/trino-cassandra/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-clickhouse/pom.xml b/plugin/trino-clickhouse/pom.xml index 90082d9a937bb..088575a04159b 100644 --- a/plugin/trino-clickhouse/pom.xml +++ b/plugin/trino-clickhouse/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-delta-lake/pom.xml b/plugin/trino-delta-lake/pom.xml index fe8c8910cde77..a01de83a16ef9 100644 --- a/plugin/trino-delta-lake/pom.xml +++ b/plugin/trino-delta-lake/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-druid/pom.xml b/plugin/trino-druid/pom.xml index cc686502ecb43..3fe3b35305041 100644 --- a/plugin/trino-druid/pom.xml +++ b/plugin/trino-druid/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-elasticsearch/pom.xml b/plugin/trino-elasticsearch/pom.xml index 326020108295b..b6c682ed46fa1 100644 --- a/plugin/trino-elasticsearch/pom.xml +++ b/plugin/trino-elasticsearch/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-example-http/pom.xml b/plugin/trino-example-http/pom.xml index fdb209737132a..1ca3db06c52b7 100644 --- a/plugin/trino-example-http/pom.xml +++ b/plugin/trino-example-http/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-example-jdbc/pom.xml b/plugin/trino-example-jdbc/pom.xml index 2e84219b1e0ee..72338b14158e1 100644 --- a/plugin/trino-example-jdbc/pom.xml +++ b/plugin/trino-example-jdbc/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-exchange-filesystem/pom.xml b/plugin/trino-exchange-filesystem/pom.xml index 2ebb14c8aeb3d..22b34c7a27c20 100644 --- a/plugin/trino-exchange-filesystem/pom.xml +++ b/plugin/trino-exchange-filesystem/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-exchange-hdfs/pom.xml b/plugin/trino-exchange-hdfs/pom.xml index a9300c086d2c4..9092f218cd211 100644 --- a/plugin/trino-exchange-hdfs/pom.xml +++ b/plugin/trino-exchange-hdfs/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-geospatial/pom.xml b/plugin/trino-geospatial/pom.xml index d11409deb6d2e..d0a5315a7f7c8 100644 --- a/plugin/trino-geospatial/pom.xml +++ b/plugin/trino-geospatial/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-google-sheets/pom.xml b/plugin/trino-google-sheets/pom.xml index 4ff813f9718e3..470073238e25e 100644 --- a/plugin/trino-google-sheets/pom.xml +++ b/plugin/trino-google-sheets/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-hive/pom.xml b/plugin/trino-hive/pom.xml index 653808044db12..46b7601c12f83 100644 --- a/plugin/trino-hive/pom.xml +++ b/plugin/trino-hive/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-http-event-listener/pom.xml b/plugin/trino-http-event-listener/pom.xml index 4f81b6d59a6ce..686cdc7f19d18 100644 --- a/plugin/trino-http-event-listener/pom.xml +++ b/plugin/trino-http-event-listener/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-hudi/pom.xml b/plugin/trino-hudi/pom.xml index d609063eb6fb8..0bf0cb322eccf 100644 --- a/plugin/trino-hudi/pom.xml +++ b/plugin/trino-hudi/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-iceberg/pom.xml b/plugin/trino-iceberg/pom.xml index 9e6e78c811fb7..0afe97ab8ae05 100644 --- a/plugin/trino-iceberg/pom.xml +++ b/plugin/trino-iceberg/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-ignite/pom.xml b/plugin/trino-ignite/pom.xml index 211ae7925dc08..71eeef3e9afd3 100644 --- a/plugin/trino-ignite/pom.xml +++ b/plugin/trino-ignite/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-jmx/pom.xml b/plugin/trino-jmx/pom.xml index ad2464afa78a2..ab60ecf898c6f 100644 --- a/plugin/trino-jmx/pom.xml +++ b/plugin/trino-jmx/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-kafka/pom.xml b/plugin/trino-kafka/pom.xml index 879684c3ced73..1880901559fe9 100644 --- a/plugin/trino-kafka/pom.xml +++ b/plugin/trino-kafka/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-kinesis/pom.xml b/plugin/trino-kinesis/pom.xml index 5988c20a55e20..9b998ffbf2029 100644 --- a/plugin/trino-kinesis/pom.xml +++ b/plugin/trino-kinesis/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-kudu/pom.xml b/plugin/trino-kudu/pom.xml index 0e944903a4f08..a27bbd0bc2630 100644 --- a/plugin/trino-kudu/pom.xml +++ b/plugin/trino-kudu/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-local-file/pom.xml b/plugin/trino-local-file/pom.xml index cea93e66eb001..e4e71503d59f4 100644 --- a/plugin/trino-local-file/pom.xml +++ b/plugin/trino-local-file/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-mariadb/pom.xml b/plugin/trino-mariadb/pom.xml index 4116f5e8ccd42..099f8e3288232 100644 --- a/plugin/trino-mariadb/pom.xml +++ b/plugin/trino-mariadb/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-memory/pom.xml b/plugin/trino-memory/pom.xml index 860f99a46ddd5..b8f76f11e0f2b 100644 --- a/plugin/trino-memory/pom.xml +++ b/plugin/trino-memory/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-ml/pom.xml b/plugin/trino-ml/pom.xml index 5e85c2fe5b0e5..58cd8bd3cdecb 100644 --- a/plugin/trino-ml/pom.xml +++ b/plugin/trino-ml/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-mongodb/pom.xml b/plugin/trino-mongodb/pom.xml index ef823ea6c80e0..68e3047ea82fe 100644 --- a/plugin/trino-mongodb/pom.xml +++ b/plugin/trino-mongodb/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-mysql-event-listener/pom.xml b/plugin/trino-mysql-event-listener/pom.xml index b7904f9a45165..068ee15da2387 100644 --- a/plugin/trino-mysql-event-listener/pom.xml +++ b/plugin/trino-mysql-event-listener/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-mysql/pom.xml b/plugin/trino-mysql/pom.xml index bdf9e9b22a3c5..4c186b0c34885 100644 --- a/plugin/trino-mysql/pom.xml +++ b/plugin/trino-mysql/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-oracle/pom.xml b/plugin/trino-oracle/pom.xml index 19f7be5c195ad..8c217f7988565 100644 --- a/plugin/trino-oracle/pom.xml +++ b/plugin/trino-oracle/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-password-authenticators/pom.xml b/plugin/trino-password-authenticators/pom.xml index 6306c9f15c13c..54fa2f579e9cf 100644 --- a/plugin/trino-password-authenticators/pom.xml +++ b/plugin/trino-password-authenticators/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-phoenix5/pom.xml b/plugin/trino-phoenix5/pom.xml index 97a834b0a75be..9d55e8fed1e32 100644 --- a/plugin/trino-phoenix5/pom.xml +++ b/plugin/trino-phoenix5/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-pinot/pom.xml b/plugin/trino-pinot/pom.xml index 7d2e84cf60606..6ba1f40d6ee38 100755 --- a/plugin/trino-pinot/pom.xml +++ b/plugin/trino-pinot/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-postgresql/pom.xml b/plugin/trino-postgresql/pom.xml index cd51a4ee159ef..f092ccd50e870 100644 --- a/plugin/trino-postgresql/pom.xml +++ b/plugin/trino-postgresql/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-prometheus/pom.xml b/plugin/trino-prometheus/pom.xml index 083b7ddf1644f..bd9cc3587f0ca 100644 --- a/plugin/trino-prometheus/pom.xml +++ b/plugin/trino-prometheus/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-raptor-legacy/pom.xml b/plugin/trino-raptor-legacy/pom.xml index 47e4c75853928..dd7549a86d12c 100644 --- a/plugin/trino-raptor-legacy/pom.xml +++ b/plugin/trino-raptor-legacy/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-redis/pom.xml b/plugin/trino-redis/pom.xml index e85e49cda7f93..634fe3df9be73 100644 --- a/plugin/trino-redis/pom.xml +++ b/plugin/trino-redis/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-redshift/pom.xml b/plugin/trino-redshift/pom.xml index 3535e4ab43bb9..0ecaa2f362fc4 100644 --- a/plugin/trino-redshift/pom.xml +++ b/plugin/trino-redshift/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-resource-group-managers/pom.xml b/plugin/trino-resource-group-managers/pom.xml index 97e276cf212d6..7c5adcd59c7fe 100644 --- a/plugin/trino-resource-group-managers/pom.xml +++ b/plugin/trino-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-session-property-managers/pom.xml b/plugin/trino-session-property-managers/pom.xml index 7838a5bb9aacb..86437911416f9 100644 --- a/plugin/trino-session-property-managers/pom.xml +++ b/plugin/trino-session-property-managers/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-singlestore/pom.xml b/plugin/trino-singlestore/pom.xml index 1b46a94680771..be83a37e5cc2f 100644 --- a/plugin/trino-singlestore/pom.xml +++ b/plugin/trino-singlestore/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-sqlserver/pom.xml b/plugin/trino-sqlserver/pom.xml index 7e6e87f22753a..92203437cf776 100644 --- a/plugin/trino-sqlserver/pom.xml +++ b/plugin/trino-sqlserver/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-teradata-functions/pom.xml b/plugin/trino-teradata-functions/pom.xml index 9f6091c6856fb..90d4037d4733d 100644 --- a/plugin/trino-teradata-functions/pom.xml +++ b/plugin/trino-teradata-functions/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-thrift-api/pom.xml b/plugin/trino-thrift-api/pom.xml index 1ccccd4bb56eb..859c04799e524 100644 --- a/plugin/trino-thrift-api/pom.xml +++ b/plugin/trino-thrift-api/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-thrift-testing-server/pom.xml b/plugin/trino-thrift-testing-server/pom.xml index 9ac075a82a1d0..980f997fa7b2b 100644 --- a/plugin/trino-thrift-testing-server/pom.xml +++ b/plugin/trino-thrift-testing-server/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-thrift/pom.xml b/plugin/trino-thrift/pom.xml index ba76784280563..2c993a8713dff 100644 --- a/plugin/trino-thrift/pom.xml +++ b/plugin/trino-thrift/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-tpcds/pom.xml b/plugin/trino-tpcds/pom.xml index eb484191d85b3..efaaf2f237e49 100644 --- a/plugin/trino-tpcds/pom.xml +++ b/plugin/trino-tpcds/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/plugin/trino-tpch/pom.xml b/plugin/trino-tpch/pom.xml index 721574b2eab7b..71a398d1956cf 100644 --- a/plugin/trino-tpch/pom.xml +++ b/plugin/trino-tpch/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/pom.xml b/pom.xml index db31fce963101..8c560b916e9dd 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 pom ${project.artifactId} @@ -132,7 +132,7 @@ scm:git:git://github.com/trinodb/trino.git - HEAD + 435 https://github.com/trinodb/trino diff --git a/service/trino-proxy/pom.xml b/service/trino-proxy/pom.xml index c147ba4838656..acc82f3934c6d 100644 --- a/service/trino-proxy/pom.xml +++ b/service/trino-proxy/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/service/trino-verifier/pom.xml b/service/trino-verifier/pom.xml index a8ab89b6bfa68..ec9a2627294be 100644 --- a/service/trino-verifier/pom.xml +++ b/service/trino-verifier/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/testing/trino-benchmark-queries/pom.xml b/testing/trino-benchmark-queries/pom.xml index 11777cbd2b339..8df4777876b14 100644 --- a/testing/trino-benchmark-queries/pom.xml +++ b/testing/trino-benchmark-queries/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/testing/trino-benchto-benchmarks/pom.xml b/testing/trino-benchto-benchmarks/pom.xml index 3c31fa6350dce..4cc437c22be34 100644 --- a/testing/trino-benchto-benchmarks/pom.xml +++ b/testing/trino-benchto-benchmarks/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/testing/trino-faulttolerant-tests/pom.xml b/testing/trino-faulttolerant-tests/pom.xml index b0135bbd5c0bd..f6ca7bc0b2eab 100644 --- a/testing/trino-faulttolerant-tests/pom.xml +++ b/testing/trino-faulttolerant-tests/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/testing/trino-plugin-reader/pom.xml b/testing/trino-plugin-reader/pom.xml index 078ca5cd85d58..97acb23e4f8eb 100644 --- a/testing/trino-plugin-reader/pom.xml +++ b/testing/trino-plugin-reader/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/testing/trino-product-tests-launcher/pom.xml b/testing/trino-product-tests-launcher/pom.xml index 03eb0334fe483..a3ff5e873e29e 100644 --- a/testing/trino-product-tests-launcher/pom.xml +++ b/testing/trino-product-tests-launcher/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/testing/trino-product-tests/pom.xml b/testing/trino-product-tests/pom.xml index 8aaf3b31cfbd8..d78e8e61a76c5 100644 --- a/testing/trino-product-tests/pom.xml +++ b/testing/trino-product-tests/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/testing/trino-server-dev/pom.xml b/testing/trino-server-dev/pom.xml index ac80013894a44..f783e13a63ba7 100644 --- a/testing/trino-server-dev/pom.xml +++ b/testing/trino-server-dev/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/testing/trino-test-jdbc-compatibility-old-driver/pom.xml b/testing/trino-test-jdbc-compatibility-old-driver/pom.xml index 999d56505a128..4a22666fe8f5e 100644 --- a/testing/trino-test-jdbc-compatibility-old-driver/pom.xml +++ b/testing/trino-test-jdbc-compatibility-old-driver/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml @@ -15,7 +15,7 @@ ${project.parent.basedir} - 435-SNAPSHOT + 435 diff --git a/testing/trino-test-jdbc-compatibility-old-server/pom.xml b/testing/trino-test-jdbc-compatibility-old-server/pom.xml index c8c837e69e739..7036fe29d2d51 100644 --- a/testing/trino-test-jdbc-compatibility-old-server/pom.xml +++ b/testing/trino-test-jdbc-compatibility-old-server/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/testing/trino-testing-containers/pom.xml b/testing/trino-testing-containers/pom.xml index ff83b97339de9..f3d807fc3a1b0 100644 --- a/testing/trino-testing-containers/pom.xml +++ b/testing/trino-testing-containers/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/testing/trino-testing-kafka/pom.xml b/testing/trino-testing-kafka/pom.xml index ccc9d5c586208..b2ae5d9ab11a5 100644 --- a/testing/trino-testing-kafka/pom.xml +++ b/testing/trino-testing-kafka/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/testing/trino-testing-resources/pom.xml b/testing/trino-testing-resources/pom.xml index 307463e255df4..9260dfcbf9efa 100644 --- a/testing/trino-testing-resources/pom.xml +++ b/testing/trino-testing-resources/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/testing/trino-testing-services/pom.xml b/testing/trino-testing-services/pom.xml index 0a29fa6069c80..fb6cec703fc16 100644 --- a/testing/trino-testing-services/pom.xml +++ b/testing/trino-testing-services/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/testing/trino-testing/pom.xml b/testing/trino-testing/pom.xml index 5325154710e6c..11bd90ce9f450 100644 --- a/testing/trino-testing/pom.xml +++ b/testing/trino-testing/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml diff --git a/testing/trino-tests/pom.xml b/testing/trino-tests/pom.xml index 2d1f6efffd065..dff00dc7536ce 100644 --- a/testing/trino-tests/pom.xml +++ b/testing/trino-tests/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 435 ../../pom.xml From 82a76c96e5ee66ca60d0e0ab9f926158c79536c3 Mon Sep 17 00:00:00 2001 From: Martin Traverso Date: Thu, 14 Dec 2023 04:11:35 +0000 Subject: [PATCH 130/350] [maven-release-plugin] prepare for next development iteration --- client/trino-cli/pom.xml | 2 +- client/trino-client/pom.xml | 2 +- client/trino-jdbc/pom.xml | 2 +- core/trino-grammar/pom.xml | 2 +- core/trino-main/pom.xml | 2 +- core/trino-parser/pom.xml | 2 +- core/trino-server-main/pom.xml | 2 +- core/trino-server-rpm/pom.xml | 2 +- core/trino-server/pom.xml | 2 +- core/trino-spi/pom.xml | 2 +- docs/pom.xml | 2 +- lib/trino-array/pom.xml | 2 +- lib/trino-cache/pom.xml | 2 +- lib/trino-filesystem-azure/pom.xml | 2 +- lib/trino-filesystem-gcs/pom.xml | 2 +- lib/trino-filesystem-manager/pom.xml | 2 +- lib/trino-filesystem-s3/pom.xml | 2 +- lib/trino-filesystem/pom.xml | 2 +- lib/trino-geospatial-toolkit/pom.xml | 2 +- lib/trino-hdfs/pom.xml | 2 +- lib/trino-hive-formats/pom.xml | 2 +- lib/trino-ignite-patched/pom.xml | 2 +- lib/trino-matching/pom.xml | 2 +- lib/trino-memory-context/pom.xml | 2 +- lib/trino-orc/pom.xml | 2 +- lib/trino-parquet/pom.xml | 2 +- lib/trino-phoenix5-patched/pom.xml | 2 +- lib/trino-plugin-toolkit/pom.xml | 2 +- lib/trino-record-decoder/pom.xml | 2 +- plugin/trino-accumulo-iterators/pom.xml | 2 +- plugin/trino-accumulo/pom.xml | 2 +- plugin/trino-atop/pom.xml | 2 +- plugin/trino-base-jdbc/pom.xml | 2 +- plugin/trino-bigquery/pom.xml | 2 +- plugin/trino-blackhole/pom.xml | 2 +- plugin/trino-cassandra/pom.xml | 2 +- plugin/trino-clickhouse/pom.xml | 2 +- plugin/trino-delta-lake/pom.xml | 2 +- plugin/trino-druid/pom.xml | 2 +- plugin/trino-elasticsearch/pom.xml | 2 +- plugin/trino-example-http/pom.xml | 2 +- plugin/trino-example-jdbc/pom.xml | 2 +- plugin/trino-exchange-filesystem/pom.xml | 2 +- plugin/trino-exchange-hdfs/pom.xml | 2 +- plugin/trino-geospatial/pom.xml | 2 +- plugin/trino-google-sheets/pom.xml | 2 +- plugin/trino-hive/pom.xml | 2 +- plugin/trino-http-event-listener/pom.xml | 2 +- plugin/trino-hudi/pom.xml | 2 +- plugin/trino-iceberg/pom.xml | 2 +- plugin/trino-ignite/pom.xml | 2 +- plugin/trino-jmx/pom.xml | 2 +- plugin/trino-kafka/pom.xml | 2 +- plugin/trino-kinesis/pom.xml | 2 +- plugin/trino-kudu/pom.xml | 2 +- plugin/trino-local-file/pom.xml | 2 +- plugin/trino-mariadb/pom.xml | 2 +- plugin/trino-memory/pom.xml | 2 +- plugin/trino-ml/pom.xml | 2 +- plugin/trino-mongodb/pom.xml | 2 +- plugin/trino-mysql-event-listener/pom.xml | 2 +- plugin/trino-mysql/pom.xml | 2 +- plugin/trino-oracle/pom.xml | 2 +- plugin/trino-password-authenticators/pom.xml | 2 +- plugin/trino-phoenix5/pom.xml | 2 +- plugin/trino-pinot/pom.xml | 2 +- plugin/trino-postgresql/pom.xml | 2 +- plugin/trino-prometheus/pom.xml | 2 +- plugin/trino-raptor-legacy/pom.xml | 2 +- plugin/trino-redis/pom.xml | 2 +- plugin/trino-redshift/pom.xml | 2 +- plugin/trino-resource-group-managers/pom.xml | 2 +- plugin/trino-session-property-managers/pom.xml | 2 +- plugin/trino-singlestore/pom.xml | 2 +- plugin/trino-sqlserver/pom.xml | 2 +- plugin/trino-teradata-functions/pom.xml | 2 +- plugin/trino-thrift-api/pom.xml | 2 +- plugin/trino-thrift-testing-server/pom.xml | 2 +- plugin/trino-thrift/pom.xml | 2 +- plugin/trino-tpcds/pom.xml | 2 +- plugin/trino-tpch/pom.xml | 2 +- pom.xml | 4 ++-- service/trino-proxy/pom.xml | 2 +- service/trino-verifier/pom.xml | 2 +- testing/trino-benchmark-queries/pom.xml | 2 +- testing/trino-benchto-benchmarks/pom.xml | 2 +- testing/trino-faulttolerant-tests/pom.xml | 2 +- testing/trino-plugin-reader/pom.xml | 2 +- testing/trino-product-tests-launcher/pom.xml | 2 +- testing/trino-product-tests/pom.xml | 2 +- testing/trino-server-dev/pom.xml | 2 +- testing/trino-test-jdbc-compatibility-old-driver/pom.xml | 4 ++-- testing/trino-test-jdbc-compatibility-old-server/pom.xml | 2 +- testing/trino-testing-containers/pom.xml | 2 +- testing/trino-testing-kafka/pom.xml | 2 +- testing/trino-testing-resources/pom.xml | 2 +- testing/trino-testing-services/pom.xml | 2 +- testing/trino-testing/pom.xml | 2 +- testing/trino-tests/pom.xml | 2 +- 99 files changed, 101 insertions(+), 101 deletions(-) diff --git a/client/trino-cli/pom.xml b/client/trino-cli/pom.xml index e7f7ca0dff4b4..80fc251f557e4 100644 --- a/client/trino-cli/pom.xml +++ b/client/trino-cli/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/client/trino-client/pom.xml b/client/trino-client/pom.xml index fe84dc67d8f28..074194851396f 100644 --- a/client/trino-client/pom.xml +++ b/client/trino-client/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/client/trino-jdbc/pom.xml b/client/trino-jdbc/pom.xml index 1e882f5a8b354..fbc7fbae9a1ae 100644 --- a/client/trino-jdbc/pom.xml +++ b/client/trino-jdbc/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/core/trino-grammar/pom.xml b/core/trino-grammar/pom.xml index 0c84295de7556..a77a6d86389c8 100644 --- a/core/trino-grammar/pom.xml +++ b/core/trino-grammar/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/core/trino-main/pom.xml b/core/trino-main/pom.xml index 96c729be03970..34b8648ccb90d 100644 --- a/core/trino-main/pom.xml +++ b/core/trino-main/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/core/trino-parser/pom.xml b/core/trino-parser/pom.xml index 3de822d9da636..b1b7fed9685d5 100644 --- a/core/trino-parser/pom.xml +++ b/core/trino-parser/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/core/trino-server-main/pom.xml b/core/trino-server-main/pom.xml index edeaf26c22aeb..ea33c72a0c0e4 100644 --- a/core/trino-server-main/pom.xml +++ b/core/trino-server-main/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/core/trino-server-rpm/pom.xml b/core/trino-server-rpm/pom.xml index 49af423f6b2cd..fde598314adbe 100644 --- a/core/trino-server-rpm/pom.xml +++ b/core/trino-server-rpm/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/core/trino-server/pom.xml b/core/trino-server/pom.xml index 783d594eae75e..c396e07ec6909 100644 --- a/core/trino-server/pom.xml +++ b/core/trino-server/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/core/trino-spi/pom.xml b/core/trino-spi/pom.xml index 583d5641b0771..6817dfaf7dd98 100644 --- a/core/trino-spi/pom.xml +++ b/core/trino-spi/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/docs/pom.xml b/docs/pom.xml index de9905d652818..bc732e679b0a8 100644 --- a/docs/pom.xml +++ b/docs/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT trino-docs diff --git a/lib/trino-array/pom.xml b/lib/trino-array/pom.xml index 49f197d59cd37..38862568d8672 100644 --- a/lib/trino-array/pom.xml +++ b/lib/trino-array/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/lib/trino-cache/pom.xml b/lib/trino-cache/pom.xml index 2190504c84e8b..7bc80bb2204d4 100644 --- a/lib/trino-cache/pom.xml +++ b/lib/trino-cache/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/lib/trino-filesystem-azure/pom.xml b/lib/trino-filesystem-azure/pom.xml index ec154178de574..63e155f093a79 100644 --- a/lib/trino-filesystem-azure/pom.xml +++ b/lib/trino-filesystem-azure/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/lib/trino-filesystem-gcs/pom.xml b/lib/trino-filesystem-gcs/pom.xml index da21089f6b058..7e9505d9c9392 100644 --- a/lib/trino-filesystem-gcs/pom.xml +++ b/lib/trino-filesystem-gcs/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/lib/trino-filesystem-manager/pom.xml b/lib/trino-filesystem-manager/pom.xml index b5fe87df132fc..e96fd45aef0df 100644 --- a/lib/trino-filesystem-manager/pom.xml +++ b/lib/trino-filesystem-manager/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/lib/trino-filesystem-s3/pom.xml b/lib/trino-filesystem-s3/pom.xml index d72dc0cec2398..1f9645ef738e6 100644 --- a/lib/trino-filesystem-s3/pom.xml +++ b/lib/trino-filesystem-s3/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/lib/trino-filesystem/pom.xml b/lib/trino-filesystem/pom.xml index ebc7c70284e5a..997ffdd7b110a 100644 --- a/lib/trino-filesystem/pom.xml +++ b/lib/trino-filesystem/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/lib/trino-geospatial-toolkit/pom.xml b/lib/trino-geospatial-toolkit/pom.xml index ebb3e51bc76b6..c3d1211654ef9 100644 --- a/lib/trino-geospatial-toolkit/pom.xml +++ b/lib/trino-geospatial-toolkit/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/lib/trino-hdfs/pom.xml b/lib/trino-hdfs/pom.xml index 40dca75268b5b..d36fa826048bf 100644 --- a/lib/trino-hdfs/pom.xml +++ b/lib/trino-hdfs/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/lib/trino-hive-formats/pom.xml b/lib/trino-hive-formats/pom.xml index 55e497a09dcc7..459979155d653 100644 --- a/lib/trino-hive-formats/pom.xml +++ b/lib/trino-hive-formats/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/lib/trino-ignite-patched/pom.xml b/lib/trino-ignite-patched/pom.xml index e086e2d49828c..6090d4cc14e06 100644 --- a/lib/trino-ignite-patched/pom.xml +++ b/lib/trino-ignite-patched/pom.xml @@ -6,7 +6,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/lib/trino-matching/pom.xml b/lib/trino-matching/pom.xml index eacedb134a690..754d7186b84c2 100644 --- a/lib/trino-matching/pom.xml +++ b/lib/trino-matching/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/lib/trino-memory-context/pom.xml b/lib/trino-memory-context/pom.xml index 0086feff1cdec..9141fc4be12c0 100644 --- a/lib/trino-memory-context/pom.xml +++ b/lib/trino-memory-context/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/lib/trino-orc/pom.xml b/lib/trino-orc/pom.xml index 869c4e40b58fc..931885bba9025 100644 --- a/lib/trino-orc/pom.xml +++ b/lib/trino-orc/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/lib/trino-parquet/pom.xml b/lib/trino-parquet/pom.xml index 61db7477a413f..92fc733c8c558 100644 --- a/lib/trino-parquet/pom.xml +++ b/lib/trino-parquet/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/lib/trino-phoenix5-patched/pom.xml b/lib/trino-phoenix5-patched/pom.xml index 79291dcc88165..8954ec63d723b 100644 --- a/lib/trino-phoenix5-patched/pom.xml +++ b/lib/trino-phoenix5-patched/pom.xml @@ -6,7 +6,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/lib/trino-plugin-toolkit/pom.xml b/lib/trino-plugin-toolkit/pom.xml index 8a55f61d54842..bffe213e00840 100644 --- a/lib/trino-plugin-toolkit/pom.xml +++ b/lib/trino-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/lib/trino-record-decoder/pom.xml b/lib/trino-record-decoder/pom.xml index eabdb5ded2b49..3ce83c8ff4b5a 100644 --- a/lib/trino-record-decoder/pom.xml +++ b/lib/trino-record-decoder/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-accumulo-iterators/pom.xml b/plugin/trino-accumulo-iterators/pom.xml index d1a4613f2553d..43ac42b4a82cf 100644 --- a/plugin/trino-accumulo-iterators/pom.xml +++ b/plugin/trino-accumulo-iterators/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-accumulo/pom.xml b/plugin/trino-accumulo/pom.xml index dbaa1d6448453..02347a6284bbd 100644 --- a/plugin/trino-accumulo/pom.xml +++ b/plugin/trino-accumulo/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-atop/pom.xml b/plugin/trino-atop/pom.xml index 4b0db3e036a89..208167a5f3033 100644 --- a/plugin/trino-atop/pom.xml +++ b/plugin/trino-atop/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-base-jdbc/pom.xml b/plugin/trino-base-jdbc/pom.xml index 41a80c9d02933..bae0cfc8b9cc6 100644 --- a/plugin/trino-base-jdbc/pom.xml +++ b/plugin/trino-base-jdbc/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-bigquery/pom.xml b/plugin/trino-bigquery/pom.xml index 1cb09cf8c0837..dab8f20a180c9 100644 --- a/plugin/trino-bigquery/pom.xml +++ b/plugin/trino-bigquery/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-blackhole/pom.xml b/plugin/trino-blackhole/pom.xml index d273e7e1a5272..cd9ac64f101f1 100644 --- a/plugin/trino-blackhole/pom.xml +++ b/plugin/trino-blackhole/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-cassandra/pom.xml b/plugin/trino-cassandra/pom.xml index a77dd84bf6d5f..f8eaf4fc7c590 100644 --- a/plugin/trino-cassandra/pom.xml +++ b/plugin/trino-cassandra/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-clickhouse/pom.xml b/plugin/trino-clickhouse/pom.xml index 088575a04159b..1f7861124ddfe 100644 --- a/plugin/trino-clickhouse/pom.xml +++ b/plugin/trino-clickhouse/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-delta-lake/pom.xml b/plugin/trino-delta-lake/pom.xml index a01de83a16ef9..8132ca79a53ca 100644 --- a/plugin/trino-delta-lake/pom.xml +++ b/plugin/trino-delta-lake/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-druid/pom.xml b/plugin/trino-druid/pom.xml index 3fe3b35305041..c04d8bd4bffcb 100644 --- a/plugin/trino-druid/pom.xml +++ b/plugin/trino-druid/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-elasticsearch/pom.xml b/plugin/trino-elasticsearch/pom.xml index b6c682ed46fa1..0ad20c8aaecad 100644 --- a/plugin/trino-elasticsearch/pom.xml +++ b/plugin/trino-elasticsearch/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-example-http/pom.xml b/plugin/trino-example-http/pom.xml index 1ca3db06c52b7..3dd676a3cf55e 100644 --- a/plugin/trino-example-http/pom.xml +++ b/plugin/trino-example-http/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-example-jdbc/pom.xml b/plugin/trino-example-jdbc/pom.xml index 72338b14158e1..837e1be5fbeb2 100644 --- a/plugin/trino-example-jdbc/pom.xml +++ b/plugin/trino-example-jdbc/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-exchange-filesystem/pom.xml b/plugin/trino-exchange-filesystem/pom.xml index 22b34c7a27c20..c58b41f9bacac 100644 --- a/plugin/trino-exchange-filesystem/pom.xml +++ b/plugin/trino-exchange-filesystem/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-exchange-hdfs/pom.xml b/plugin/trino-exchange-hdfs/pom.xml index 9092f218cd211..3fc32e4148af5 100644 --- a/plugin/trino-exchange-hdfs/pom.xml +++ b/plugin/trino-exchange-hdfs/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-geospatial/pom.xml b/plugin/trino-geospatial/pom.xml index d0a5315a7f7c8..7c6b237485296 100644 --- a/plugin/trino-geospatial/pom.xml +++ b/plugin/trino-geospatial/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-google-sheets/pom.xml b/plugin/trino-google-sheets/pom.xml index 470073238e25e..e7396a711768a 100644 --- a/plugin/trino-google-sheets/pom.xml +++ b/plugin/trino-google-sheets/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-hive/pom.xml b/plugin/trino-hive/pom.xml index 46b7601c12f83..62a3ebca9725c 100644 --- a/plugin/trino-hive/pom.xml +++ b/plugin/trino-hive/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-http-event-listener/pom.xml b/plugin/trino-http-event-listener/pom.xml index 686cdc7f19d18..8e1fea2806ab6 100644 --- a/plugin/trino-http-event-listener/pom.xml +++ b/plugin/trino-http-event-listener/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-hudi/pom.xml b/plugin/trino-hudi/pom.xml index 0bf0cb322eccf..fc5de619884ae 100644 --- a/plugin/trino-hudi/pom.xml +++ b/plugin/trino-hudi/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-iceberg/pom.xml b/plugin/trino-iceberg/pom.xml index 0afe97ab8ae05..3facee721c37a 100644 --- a/plugin/trino-iceberg/pom.xml +++ b/plugin/trino-iceberg/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-ignite/pom.xml b/plugin/trino-ignite/pom.xml index 71eeef3e9afd3..a9a5ae86d8954 100644 --- a/plugin/trino-ignite/pom.xml +++ b/plugin/trino-ignite/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-jmx/pom.xml b/plugin/trino-jmx/pom.xml index ab60ecf898c6f..dd6973b4ac180 100644 --- a/plugin/trino-jmx/pom.xml +++ b/plugin/trino-jmx/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-kafka/pom.xml b/plugin/trino-kafka/pom.xml index 1880901559fe9..3fe24be379bdc 100644 --- a/plugin/trino-kafka/pom.xml +++ b/plugin/trino-kafka/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-kinesis/pom.xml b/plugin/trino-kinesis/pom.xml index 9b998ffbf2029..733d4706230a2 100644 --- a/plugin/trino-kinesis/pom.xml +++ b/plugin/trino-kinesis/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-kudu/pom.xml b/plugin/trino-kudu/pom.xml index a27bbd0bc2630..3b755040c959f 100644 --- a/plugin/trino-kudu/pom.xml +++ b/plugin/trino-kudu/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-local-file/pom.xml b/plugin/trino-local-file/pom.xml index e4e71503d59f4..e25ca18de4fcc 100644 --- a/plugin/trino-local-file/pom.xml +++ b/plugin/trino-local-file/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-mariadb/pom.xml b/plugin/trino-mariadb/pom.xml index 099f8e3288232..517b94b472b0f 100644 --- a/plugin/trino-mariadb/pom.xml +++ b/plugin/trino-mariadb/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-memory/pom.xml b/plugin/trino-memory/pom.xml index b8f76f11e0f2b..93add17f06dc9 100644 --- a/plugin/trino-memory/pom.xml +++ b/plugin/trino-memory/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-ml/pom.xml b/plugin/trino-ml/pom.xml index 58cd8bd3cdecb..75aeb4ec3b896 100644 --- a/plugin/trino-ml/pom.xml +++ b/plugin/trino-ml/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-mongodb/pom.xml b/plugin/trino-mongodb/pom.xml index 68e3047ea82fe..dfb821c671ef0 100644 --- a/plugin/trino-mongodb/pom.xml +++ b/plugin/trino-mongodb/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-mysql-event-listener/pom.xml b/plugin/trino-mysql-event-listener/pom.xml index 068ee15da2387..6dbafc9f4f074 100644 --- a/plugin/trino-mysql-event-listener/pom.xml +++ b/plugin/trino-mysql-event-listener/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-mysql/pom.xml b/plugin/trino-mysql/pom.xml index 4c186b0c34885..8b2d51b2d12bf 100644 --- a/plugin/trino-mysql/pom.xml +++ b/plugin/trino-mysql/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-oracle/pom.xml b/plugin/trino-oracle/pom.xml index 8c217f7988565..dc28a30560dd6 100644 --- a/plugin/trino-oracle/pom.xml +++ b/plugin/trino-oracle/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-password-authenticators/pom.xml b/plugin/trino-password-authenticators/pom.xml index 54fa2f579e9cf..a401dd4ebfc8c 100644 --- a/plugin/trino-password-authenticators/pom.xml +++ b/plugin/trino-password-authenticators/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-phoenix5/pom.xml b/plugin/trino-phoenix5/pom.xml index 9d55e8fed1e32..89c7ec6174365 100644 --- a/plugin/trino-phoenix5/pom.xml +++ b/plugin/trino-phoenix5/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-pinot/pom.xml b/plugin/trino-pinot/pom.xml index 6ba1f40d6ee38..b6ddeb1131521 100755 --- a/plugin/trino-pinot/pom.xml +++ b/plugin/trino-pinot/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-postgresql/pom.xml b/plugin/trino-postgresql/pom.xml index f092ccd50e870..15e205edd9196 100644 --- a/plugin/trino-postgresql/pom.xml +++ b/plugin/trino-postgresql/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-prometheus/pom.xml b/plugin/trino-prometheus/pom.xml index bd9cc3587f0ca..4e5a7ccd6a31a 100644 --- a/plugin/trino-prometheus/pom.xml +++ b/plugin/trino-prometheus/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-raptor-legacy/pom.xml b/plugin/trino-raptor-legacy/pom.xml index dd7549a86d12c..c827073f709eb 100644 --- a/plugin/trino-raptor-legacy/pom.xml +++ b/plugin/trino-raptor-legacy/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-redis/pom.xml b/plugin/trino-redis/pom.xml index 634fe3df9be73..831f54a920ff4 100644 --- a/plugin/trino-redis/pom.xml +++ b/plugin/trino-redis/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-redshift/pom.xml b/plugin/trino-redshift/pom.xml index 0ecaa2f362fc4..99c2967c09320 100644 --- a/plugin/trino-redshift/pom.xml +++ b/plugin/trino-redshift/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-resource-group-managers/pom.xml b/plugin/trino-resource-group-managers/pom.xml index 7c5adcd59c7fe..ec9819c0a4c6e 100644 --- a/plugin/trino-resource-group-managers/pom.xml +++ b/plugin/trino-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-session-property-managers/pom.xml b/plugin/trino-session-property-managers/pom.xml index 86437911416f9..a3c195e5c92f9 100644 --- a/plugin/trino-session-property-managers/pom.xml +++ b/plugin/trino-session-property-managers/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-singlestore/pom.xml b/plugin/trino-singlestore/pom.xml index be83a37e5cc2f..4f6a2d76605a5 100644 --- a/plugin/trino-singlestore/pom.xml +++ b/plugin/trino-singlestore/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-sqlserver/pom.xml b/plugin/trino-sqlserver/pom.xml index 92203437cf776..1117e536dfcee 100644 --- a/plugin/trino-sqlserver/pom.xml +++ b/plugin/trino-sqlserver/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-teradata-functions/pom.xml b/plugin/trino-teradata-functions/pom.xml index 90d4037d4733d..3246a4c5429c9 100644 --- a/plugin/trino-teradata-functions/pom.xml +++ b/plugin/trino-teradata-functions/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-thrift-api/pom.xml b/plugin/trino-thrift-api/pom.xml index 859c04799e524..116a381208a92 100644 --- a/plugin/trino-thrift-api/pom.xml +++ b/plugin/trino-thrift-api/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-thrift-testing-server/pom.xml b/plugin/trino-thrift-testing-server/pom.xml index 980f997fa7b2b..eded21b2756b2 100644 --- a/plugin/trino-thrift-testing-server/pom.xml +++ b/plugin/trino-thrift-testing-server/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-thrift/pom.xml b/plugin/trino-thrift/pom.xml index 2c993a8713dff..4456743199a8c 100644 --- a/plugin/trino-thrift/pom.xml +++ b/plugin/trino-thrift/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-tpcds/pom.xml b/plugin/trino-tpcds/pom.xml index efaaf2f237e49..4d9a87f765870 100644 --- a/plugin/trino-tpcds/pom.xml +++ b/plugin/trino-tpcds/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/plugin/trino-tpch/pom.xml b/plugin/trino-tpch/pom.xml index 71a398d1956cf..5144675d01ea5 100644 --- a/plugin/trino-tpch/pom.xml +++ b/plugin/trino-tpch/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/pom.xml b/pom.xml index 8c560b916e9dd..aa18b92a631b2 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT pom ${project.artifactId} @@ -132,7 +132,7 @@ scm:git:git://github.com/trinodb/trino.git - 435 + HEAD https://github.com/trinodb/trino diff --git a/service/trino-proxy/pom.xml b/service/trino-proxy/pom.xml index acc82f3934c6d..b4adbcea150b8 100644 --- a/service/trino-proxy/pom.xml +++ b/service/trino-proxy/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/service/trino-verifier/pom.xml b/service/trino-verifier/pom.xml index ec9a2627294be..61d627374ed71 100644 --- a/service/trino-verifier/pom.xml +++ b/service/trino-verifier/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/testing/trino-benchmark-queries/pom.xml b/testing/trino-benchmark-queries/pom.xml index 8df4777876b14..250e5c2c991a8 100644 --- a/testing/trino-benchmark-queries/pom.xml +++ b/testing/trino-benchmark-queries/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/testing/trino-benchto-benchmarks/pom.xml b/testing/trino-benchto-benchmarks/pom.xml index 4cc437c22be34..d4181beed75ee 100644 --- a/testing/trino-benchto-benchmarks/pom.xml +++ b/testing/trino-benchto-benchmarks/pom.xml @@ -4,7 +4,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/testing/trino-faulttolerant-tests/pom.xml b/testing/trino-faulttolerant-tests/pom.xml index f6ca7bc0b2eab..ebbeb2578c05c 100644 --- a/testing/trino-faulttolerant-tests/pom.xml +++ b/testing/trino-faulttolerant-tests/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/testing/trino-plugin-reader/pom.xml b/testing/trino-plugin-reader/pom.xml index 97acb23e4f8eb..a2defff381c46 100644 --- a/testing/trino-plugin-reader/pom.xml +++ b/testing/trino-plugin-reader/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/testing/trino-product-tests-launcher/pom.xml b/testing/trino-product-tests-launcher/pom.xml index a3ff5e873e29e..a808a275eb6ca 100644 --- a/testing/trino-product-tests-launcher/pom.xml +++ b/testing/trino-product-tests-launcher/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/testing/trino-product-tests/pom.xml b/testing/trino-product-tests/pom.xml index d78e8e61a76c5..315dbdf876808 100644 --- a/testing/trino-product-tests/pom.xml +++ b/testing/trino-product-tests/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/testing/trino-server-dev/pom.xml b/testing/trino-server-dev/pom.xml index f783e13a63ba7..a85cd990c12fa 100644 --- a/testing/trino-server-dev/pom.xml +++ b/testing/trino-server-dev/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/testing/trino-test-jdbc-compatibility-old-driver/pom.xml b/testing/trino-test-jdbc-compatibility-old-driver/pom.xml index 4a22666fe8f5e..9a81500a37ff5 100644 --- a/testing/trino-test-jdbc-compatibility-old-driver/pom.xml +++ b/testing/trino-test-jdbc-compatibility-old-driver/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml @@ -15,7 +15,7 @@ ${project.parent.basedir} - 435 + 436-SNAPSHOT diff --git a/testing/trino-test-jdbc-compatibility-old-server/pom.xml b/testing/trino-test-jdbc-compatibility-old-server/pom.xml index 7036fe29d2d51..49c596b8f2858 100644 --- a/testing/trino-test-jdbc-compatibility-old-server/pom.xml +++ b/testing/trino-test-jdbc-compatibility-old-server/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/testing/trino-testing-containers/pom.xml b/testing/trino-testing-containers/pom.xml index f3d807fc3a1b0..1541f026e5853 100644 --- a/testing/trino-testing-containers/pom.xml +++ b/testing/trino-testing-containers/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/testing/trino-testing-kafka/pom.xml b/testing/trino-testing-kafka/pom.xml index b2ae5d9ab11a5..0699206118605 100644 --- a/testing/trino-testing-kafka/pom.xml +++ b/testing/trino-testing-kafka/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/testing/trino-testing-resources/pom.xml b/testing/trino-testing-resources/pom.xml index 9260dfcbf9efa..1c338f235e44a 100644 --- a/testing/trino-testing-resources/pom.xml +++ b/testing/trino-testing-resources/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/testing/trino-testing-services/pom.xml b/testing/trino-testing-services/pom.xml index fb6cec703fc16..de395577465e0 100644 --- a/testing/trino-testing-services/pom.xml +++ b/testing/trino-testing-services/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/testing/trino-testing/pom.xml b/testing/trino-testing/pom.xml index 11bd90ce9f450..42567f7441ff3 100644 --- a/testing/trino-testing/pom.xml +++ b/testing/trino-testing/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml diff --git a/testing/trino-tests/pom.xml b/testing/trino-tests/pom.xml index dff00dc7536ce..98a777e62b10c 100644 --- a/testing/trino-tests/pom.xml +++ b/testing/trino-tests/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435 + 436-SNAPSHOT ../../pom.xml From 742401384d3dc6fddc0c580959478616de68fe92 Mon Sep 17 00:00:00 2001 From: Yuya Ebihara Date: Thu, 14 Dec 2023 09:22:56 +0900 Subject: [PATCH 131/350] Remove redundant code from Kudu TypeHelper.getObject --- .../java/io/trino/plugin/kudu/TypeHelper.java | 37 +------------------ 1 file changed, 2 insertions(+), 35 deletions(-) diff --git a/plugin/trino-kudu/src/main/java/io/trino/plugin/kudu/TypeHelper.java b/plugin/trino-kudu/src/main/java/io/trino/plugin/kudu/TypeHelper.java index 5fdd31d30ab66..a62b892b6ed68 100644 --- a/plugin/trino-kudu/src/main/java/io/trino/plugin/kudu/TypeHelper.java +++ b/plugin/trino-kudu/src/main/java/io/trino/plugin/kudu/TypeHelper.java @@ -175,41 +175,8 @@ public static Object getJavaValue(Type type, Object nativeValue) public static Object getObject(Type type, RowResult row, int field) { - if (row.isNull(field)) { - return null; - } - if (type instanceof VarcharType) { - return row.getString(field); - } - if (type.equals(TIMESTAMP_MILLIS)) { - return truncateEpochMicrosToMillis(row.getLong(field)); - } - if (type == BigintType.BIGINT) { - return row.getLong(field); - } - if (type == IntegerType.INTEGER) { - return row.getInt(field); - } - if (type == SmallintType.SMALLINT) { - return row.getShort(field); - } - if (type == TinyintType.TINYINT) { - return row.getByte(field); - } - if (type == DoubleType.DOUBLE) { - return row.getDouble(field); - } - if (type == RealType.REAL) { - return row.getFloat(field); - } - if (type == BooleanType.BOOLEAN) { - return row.getBoolean(field); - } - if (type instanceof VarbinaryType) { - return Slices.wrappedHeapBuffer(row.getBinary(field)); - } - if (type instanceof DecimalType) { - return Decimals.encodeScaledValue(row.getDecimal(field), ((DecimalType) type).getScale()); + if (type instanceof DecimalType decimalType) { + return Decimals.encodeScaledValue(row.getDecimal(field), decimalType.getScale()); } throw new IllegalStateException("getObject not implemented for " + type); } From 5235555f91c9cb91c9ff62609cbf60e2c1ca0aea Mon Sep 17 00:00:00 2001 From: Yuya Ebihara Date: Thu, 14 Dec 2023 09:28:59 +0900 Subject: [PATCH 132/350] Reorder conditions in Kudu TypeHelper --- .../java/io/trino/plugin/kudu/TypeHelper.java | 116 +++++++++--------- 1 file changed, 58 insertions(+), 58 deletions(-) diff --git a/plugin/trino-kudu/src/main/java/io/trino/plugin/kudu/TypeHelper.java b/plugin/trino-kudu/src/main/java/io/trino/plugin/kudu/TypeHelper.java index a62b892b6ed68..88763e9525e8d 100644 --- a/plugin/trino-kudu/src/main/java/io/trino/plugin/kudu/TypeHelper.java +++ b/plugin/trino-kudu/src/main/java/io/trino/plugin/kudu/TypeHelper.java @@ -50,23 +50,20 @@ private TypeHelper() {} public static org.apache.kudu.Type toKuduClientType(Type type) { - if (type instanceof VarcharType) { - return org.apache.kudu.Type.STRING; + if (type == BooleanType.BOOLEAN) { + return org.apache.kudu.Type.BOOL; } - if (type.equals(TIMESTAMP_MILLIS)) { - return org.apache.kudu.Type.UNIXTIME_MICROS; + if (type == TinyintType.TINYINT) { + return org.apache.kudu.Type.INT8; } - if (type == BigintType.BIGINT) { - return org.apache.kudu.Type.INT64; + if (type == SmallintType.SMALLINT) { + return org.apache.kudu.Type.INT16; } if (type == IntegerType.INTEGER) { return org.apache.kudu.Type.INT32; } - if (type == SmallintType.SMALLINT) { - return org.apache.kudu.Type.INT16; - } - if (type == TinyintType.TINYINT) { - return org.apache.kudu.Type.INT8; + if (type == BigintType.BIGINT) { + return org.apache.kudu.Type.INT64; } if (type == RealType.REAL) { return org.apache.kudu.Type.FLOAT; @@ -74,20 +71,23 @@ public static org.apache.kudu.Type toKuduClientType(Type type) if (type == DoubleType.DOUBLE) { return org.apache.kudu.Type.DOUBLE; } - if (type == BooleanType.BOOLEAN) { - return org.apache.kudu.Type.BOOL; + if (type instanceof DecimalType) { + return org.apache.kudu.Type.DECIMAL; + } + if (type instanceof CharType) { + return org.apache.kudu.Type.STRING; + } + if (type instanceof VarcharType) { + return org.apache.kudu.Type.STRING; } if (type instanceof VarbinaryType) { return org.apache.kudu.Type.BINARY; } - if (type instanceof DecimalType) { - return org.apache.kudu.Type.DECIMAL; - } if (type == DateType.DATE) { return org.apache.kudu.Type.STRING; } - if (type instanceof CharType) { - return org.apache.kudu.Type.STRING; + if (type.equals(TIMESTAMP_MILLIS)) { + return org.apache.kudu.Type.UNIXTIME_MICROS; } throw new TrinoException(NOT_SUPPORTED, "Unsupported type: " + type); } @@ -100,29 +100,29 @@ public static Type fromKuduColumn(ColumnSchema column) private static Type fromKuduClientType(org.apache.kudu.Type ktype, ColumnTypeAttributes attributes) { switch (ktype) { - case STRING: - return VarcharType.VARCHAR; - case UNIXTIME_MICROS: - return TIMESTAMP_MILLIS; - case INT64: - return BigintType.BIGINT; - case INT32: - return IntegerType.INTEGER; - case INT16: - return SmallintType.SMALLINT; + case BOOL: + return BooleanType.BOOLEAN; case INT8: return TinyintType.TINYINT; + case INT16: + return SmallintType.SMALLINT; + case INT32: + return IntegerType.INTEGER; + case INT64: + return BigintType.BIGINT; case FLOAT: return RealType.REAL; case DOUBLE: return DoubleType.DOUBLE; - case BOOL: - return BooleanType.BOOLEAN; - case BINARY: - return VarbinaryType.VARBINARY; case DECIMAL: return DecimalType.createDecimalType(attributes.getPrecision(), attributes.getScale()); // TODO: add support for varchar and date types: https://github.com/trinodb/trino/issues/11009 + case STRING: + return VarcharType.VARCHAR; + case BINARY: + return VarbinaryType.VARBINARY; + case UNIXTIME_MICROS: + return TIMESTAMP_MILLIS; case VARCHAR: case DATE: break; @@ -132,44 +132,44 @@ private static Type fromKuduClientType(org.apache.kudu.Type ktype, ColumnTypeAtt public static Object getJavaValue(Type type, Object nativeValue) { - if (type instanceof VarcharType) { - return ((Slice) nativeValue).toStringUtf8(); - } - if (type.equals(TIMESTAMP_MILLIS)) { - // Kudu's native format is in microseconds - return nativeValue; - } - if (type == BigintType.BIGINT) { + if (type == BooleanType.BOOLEAN) { return nativeValue; } - if (type == IntegerType.INTEGER) { - return ((Long) nativeValue).intValue(); + if (type == TinyintType.TINYINT) { + return ((Long) nativeValue).byteValue(); } if (type == SmallintType.SMALLINT) { return ((Long) nativeValue).shortValue(); } - if (type == TinyintType.TINYINT) { - return ((Long) nativeValue).byteValue(); + if (type == IntegerType.INTEGER) { + return ((Long) nativeValue).intValue(); } - if (type == DoubleType.DOUBLE) { + if (type == BigintType.BIGINT) { return nativeValue; } if (type == RealType.REAL) { // conversion can result in precision lost return intBitsToFloat(((Long) nativeValue).intValue()); } - if (type == BooleanType.BOOLEAN) { + if (type == DoubleType.DOUBLE) { return nativeValue; } - if (type instanceof VarbinaryType) { - return ((Slice) nativeValue).toByteBuffer(); - } if (type instanceof DecimalType decimalType) { if (decimalType.isShort()) { return new BigDecimal(BigInteger.valueOf((long) nativeValue), decimalType.getScale()); } return new BigDecimal(((Int128) nativeValue).toBigInteger(), decimalType.getScale()); } + if (type instanceof VarcharType) { + return ((Slice) nativeValue).toStringUtf8(); + } + if (type instanceof VarbinaryType) { + return ((Slice) nativeValue).toByteBuffer(); + } + if (type.equals(TIMESTAMP_MILLIS)) { + // Kudu's native format is in microseconds + return nativeValue; + } throw new IllegalStateException("Back conversion not implemented for " + type); } @@ -183,20 +183,17 @@ public static Object getObject(Type type, RowResult row, int field) public static long getLong(Type type, RowResult row, int field) { - if (type.equals(TIMESTAMP_MILLIS)) { - return truncateEpochMicrosToMillis(row.getLong(field)); + if (type == TinyintType.TINYINT) { + return row.getByte(field); } - if (type == BigintType.BIGINT) { - return row.getLong(field); + if (type == SmallintType.SMALLINT) { + return row.getShort(field); } if (type == IntegerType.INTEGER) { return row.getInt(field); } - if (type == SmallintType.SMALLINT) { - return row.getShort(field); - } - if (type == TinyintType.TINYINT) { - return row.getByte(field); + if (type == BigintType.BIGINT) { + return row.getLong(field); } if (type == RealType.REAL) { return floatToRawIntBits(row.getFloat(field)); @@ -207,6 +204,9 @@ public static long getLong(Type type, RowResult row, int field) } throw new IllegalStateException("getLong not supported for long decimal: " + type); } + if (type.equals(TIMESTAMP_MILLIS)) { + return truncateEpochMicrosToMillis(row.getLong(field)); + } throw new IllegalStateException("getLong not implemented for " + type); } From 96588521e42dbae91e631c3814102fd8a91703a7 Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Wed, 13 Dec 2023 13:56:57 +0100 Subject: [PATCH 133/350] Remove unnecessary test CREATE TABLE with supported type is tested in type mapping tests already --- .../bigquery/BaseBigQueryConnectorTest.java | 29 ------------------- 1 file changed, 29 deletions(-) diff --git a/plugin/trino-bigquery/src/test/java/io/trino/plugin/bigquery/BaseBigQueryConnectorTest.java b/plugin/trino-bigquery/src/test/java/io/trino/plugin/bigquery/BaseBigQueryConnectorTest.java index 22dc474133079..e2dd2ec973f59 100644 --- a/plugin/trino-bigquery/src/test/java/io/trino/plugin/bigquery/BaseBigQueryConnectorTest.java +++ b/plugin/trino-bigquery/src/test/java/io/trino/plugin/bigquery/BaseBigQueryConnectorTest.java @@ -154,35 +154,6 @@ private void testPredicatePushdown(@Language("SQL") String inputLiteral, @Langua } } - @Test - public void testCreateTableSupportedType() - { - testCreateTableSupportedType("boolean", "boolean"); - testCreateTableSupportedType("tinyint", "bigint"); - testCreateTableSupportedType("smallint", "bigint"); - testCreateTableSupportedType("integer", "bigint"); - testCreateTableSupportedType("bigint", "bigint"); - testCreateTableSupportedType("double", "double"); - testCreateTableSupportedType("decimal", "decimal(38,9)"); - testCreateTableSupportedType("date", "date"); - testCreateTableSupportedType("time with time zone", "time(6)"); - testCreateTableSupportedType("timestamp(6)", "timestamp(6)"); - testCreateTableSupportedType("timestamp(6) with time zone", "timestamp(6) with time zone"); - testCreateTableSupportedType("varchar", "varchar"); - testCreateTableSupportedType("varchar(65535)", "varchar"); - testCreateTableSupportedType("varbinary", "varbinary"); - testCreateTableSupportedType("array(bigint)", "array(bigint)"); - testCreateTableSupportedType("row(x bigint, y double)", "row(x bigint, y double)"); - testCreateTableSupportedType("row(x array(bigint))", "row(x array(bigint))"); - } - - private void testCreateTableSupportedType(String createType, String expectedType) - { - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_create_table_supported_type_" + createType.replaceAll("[^a-zA-Z0-9]", ""), format("(col1 %s)", createType))) { - assertThat(computeScalar("SELECT data_type FROM information_schema.columns WHERE table_name = '" + table.getName() + "' AND column_name = 'col1'")).isEqualTo(expectedType); - } - } - @Test public void testCreateTableUnsupportedType() { From d4621e2a5a2a26ff4d96b8a1b07dae3f50b0626e Mon Sep 17 00:00:00 2001 From: Marius Grama Date: Wed, 13 Dec 2023 14:04:11 +0100 Subject: [PATCH 134/350] Use default stats column filter for the retrieval of active files --- .../transactionlog/TransactionLogAccess.java | 14 +++++++------- .../plugin/deltalake/TestTransactionLogAccess.java | 13 +++++++++++++ 2 files changed, 20 insertions(+), 7 deletions(-) diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/TransactionLogAccess.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/TransactionLogAccess.java index f308865e90b45..5d46748db19b5 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/TransactionLogAccess.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/TransactionLogAccess.java @@ -236,7 +236,7 @@ public MetadataEntry getMetadataEntry(TableSnapshot tableSnapshot, ConnectorSess @Deprecated public List getActiveFiles(TableSnapshot tableSnapshot, MetadataEntry metadataEntry, ProtocolEntry protocolEntry, ConnectorSession session) { - return retrieveActiveFiles(tableSnapshot, metadataEntry, protocolEntry, TupleDomain.all(), Optional.empty(), session); + return retrieveActiveFiles(tableSnapshot, metadataEntry, protocolEntry, TupleDomain.all(), alwaysTrue(), session); } public List getActiveFiles( @@ -247,13 +247,13 @@ public List getActiveFiles( Optional> projectedColumns, ConnectorSession session) { - Optional> addStatsMinMaxColumnFilter = Optional.of(alwaysFalse()); + Predicate addStatsMinMaxColumnFilter = alwaysFalse(); if (projectedColumns.isPresent()) { Set baseColumnNames = projectedColumns.get().stream() .filter(DeltaLakeColumnHandle::isBaseColumn) // Only base column stats are supported .map(DeltaLakeColumnHandle::getColumnName) .collect(toImmutableSet()); - addStatsMinMaxColumnFilter = Optional.of(baseColumnNames::contains); + addStatsMinMaxColumnFilter = baseColumnNames::contains; } return retrieveActiveFiles(tableSnapshot, metadataEntry, protocolEntry, partitionConstraint, addStatsMinMaxColumnFilter, session); } @@ -263,7 +263,7 @@ private List retrieveActiveFiles( MetadataEntry metadataEntry, ProtocolEntry protocolEntry, TupleDomain partitionConstraint, - Optional> addStatsMinMaxColumnFilter, + Predicate addStatsMinMaxColumnFilter, ConnectorSession session) { try { @@ -299,7 +299,7 @@ private List retrieveActiveFiles( } } - List activeFiles = loadActiveFiles(tableSnapshot, metadataEntry, protocolEntry, TupleDomain.all(), Optional.of(alwaysTrue()), session); + List activeFiles = loadActiveFiles(tableSnapshot, metadataEntry, protocolEntry, TupleDomain.all(), alwaysTrue(), session); return new DeltaLakeDataFileCacheEntry(tableSnapshot.getVersion(), activeFiles); }); return cacheEntry.getActiveFiles(); @@ -314,7 +314,7 @@ private List loadActiveFiles( MetadataEntry metadataEntry, ProtocolEntry protocolEntry, TupleDomain partitionConstraint, - Optional> addStatsMinMaxColumnFilter, + Predicate addStatsMinMaxColumnFilter, ConnectorSession session) { List transactions = tableSnapshot.getTransactions(); @@ -327,7 +327,7 @@ private List loadActiveFiles( fileFormatDataSourceStats, Optional.of(new MetadataAndProtocolEntry(metadataEntry, protocolEntry)), partitionConstraint, - addStatsMinMaxColumnFilter)) { + Optional.of(addStatsMinMaxColumnFilter))) { return activeAddEntries(checkpointEntries, transactions) .filter(partitionConstraint.isAll() ? addAction -> true diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestTransactionLogAccess.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestTransactionLogAccess.java index 0a39614c6292d..08beafc3e94d4 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestTransactionLogAccess.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestTransactionLogAccess.java @@ -33,7 +33,9 @@ import io.trino.plugin.deltalake.transactionlog.statistics.DeltaLakeFileStatistics; import io.trino.plugin.hive.FileFormatDataSourceStats; import io.trino.plugin.hive.parquet.ParquetReaderConfig; +import io.trino.plugin.hive.parquet.ParquetWriterConfig; import io.trino.spi.connector.ColumnMetadata; +import io.trino.spi.connector.ConnectorSession; import io.trino.spi.connector.SchemaTableName; import io.trino.spi.predicate.TupleDomain; import io.trino.spi.type.DateTimeEncoding; @@ -41,6 +43,7 @@ import io.trino.spi.type.IntegerType; import io.trino.spi.type.TypeManager; import io.trino.testing.TestingConnectorContext; +import io.trino.testing.TestingConnectorSession; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.parallel.Execution; @@ -206,6 +209,16 @@ public void testGetActiveAddEntries() .collect(Collectors.toSet()); assertThat(paths).isEqualTo(EXPECTED_ADD_FILE_PATHS); + ConnectorSession checkpointFilteringSession = TestingConnectorSession.builder() + .setPropertyMetadata(new DeltaLakeSessionProperties( + new DeltaLakeConfig().setCheckpointFilteringEnabled(true), + new ParquetReaderConfig(), + new ParquetWriterConfig()) + .getSessionProperties()) + .build(); + List checkpointFilteredAddFileEntries = transactionLogAccess.getActiveFiles(tableSnapshot, metadataEntry, protocolEntry, checkpointFilteringSession); + assertThat(checkpointFilteredAddFileEntries).isEqualTo(addFileEntries); + AddFileEntry addFileEntry = addFileEntries .stream() .filter(entry -> entry.getPath().equals("age=42/part-00000-b26c891a-7288-4d96-9d3b-bef648f12a34.c000.snappy.parquet")) From 639dce9a56d21e22a76fbf8a87708cc97281cfc4 Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Thu, 14 Dec 2023 14:53:49 +0100 Subject: [PATCH 135/350] Disable failing bigquery test --- .../trino/plugin/bigquery/BaseBigQueryConnectorTest.java | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/plugin/trino-bigquery/src/test/java/io/trino/plugin/bigquery/BaseBigQueryConnectorTest.java b/plugin/trino-bigquery/src/test/java/io/trino/plugin/bigquery/BaseBigQueryConnectorTest.java index e2dd2ec973f59..96f3ffe23d6ba 100644 --- a/plugin/trino-bigquery/src/test/java/io/trino/plugin/bigquery/BaseBigQueryConnectorTest.java +++ b/plugin/trino-bigquery/src/test/java/io/trino/plugin/bigquery/BaseBigQueryConnectorTest.java @@ -808,6 +808,13 @@ protected OptionalInt maxSchemaNameLength() return OptionalInt.of(1024); } + @Override + @Test + public void testCreateSchemaWithLongName() + { + abort("Dropping schema with long name causes BigQuery to return code 500"); + } + @Override protected void verifySchemaNameLengthFailurePermissible(Throwable e) { From 0fc93899a3bb5f77843848acbd1f6f76f9f3b63a Mon Sep 17 00:00:00 2001 From: Martin Traverso Date: Thu, 14 Dec 2023 10:15:45 -0800 Subject: [PATCH 136/350] Fix serialization failure when invoking json_table The concrete type declared in the JsonTableFunctionHandle is not directly serializable. The deserializer binding is only declared for the Type class. --- .../io/trino/operator/table/json/JsonTable.java | 8 ++++++-- .../testing/AbstractTestEngineOnlyQueries.java | 16 ++++++++++++++++ 2 files changed, 22 insertions(+), 2 deletions(-) diff --git a/core/trino-main/src/main/java/io/trino/operator/table/json/JsonTable.java b/core/trino-main/src/main/java/io/trino/operator/table/json/JsonTable.java index 3b64b58b34f88..1e1aff0faf19d 100644 --- a/core/trino-main/src/main/java/io/trino/operator/table/json/JsonTable.java +++ b/core/trino-main/src/main/java/io/trino/operator/table/json/JsonTable.java @@ -34,6 +34,7 @@ import java.util.List; import java.util.Optional; +import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.collect.Iterables.getOnlyElement; import static io.trino.operator.scalar.json.ParameterUtil.getParametersArray; import static io.trino.operator.table.json.execution.ExecutionPlanner.getExecutionPlan; @@ -65,7 +66,7 @@ private JsonTable() {} * knows the indexes of its default channels. * @param outputTypes types of the proper columns produced by the function */ - public record JsonTableFunctionHandle(JsonTablePlanNode processingPlan, boolean outer, boolean errorOnError, RowType parametersType, Type[] outputTypes) + public record JsonTableFunctionHandle(JsonTablePlanNode processingPlan, boolean outer, boolean errorOnError, Type parametersType, Type[] outputTypes) implements ConnectorTableFunctionHandle { public JsonTableFunctionHandle @@ -73,6 +74,9 @@ public record JsonTableFunctionHandle(JsonTablePlanNode processingPlan, boolean requireNonNull(processingPlan, "processingPlan is null"); requireNonNull(parametersType, "parametersType is null"); requireNonNull(outputTypes, "outputTypes is null"); + + // We can't use RowType in the public interface because it's not directly deserializeable from JSON. See TypeDeserializerModule. + checkArgument(parametersType instanceof RowType, "parametersType is not a row type"); } } @@ -94,7 +98,7 @@ public TableFunctionDataProcessor getDataProcessor(ConnectorSession session, Con metadata, typeManager, functionManager); - return new JsonTableFunctionProcessor(executionPlan, newRow, jsonTableFunctionHandle.outputTypes(), jsonTableFunctionHandle.parametersType(), jsonTableFunctionHandle.outer()); + return new JsonTableFunctionProcessor(executionPlan, newRow, jsonTableFunctionHandle.outputTypes(), (RowType) jsonTableFunctionHandle.parametersType(), jsonTableFunctionHandle.outer()); } }; } diff --git a/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestEngineOnlyQueries.java b/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestEngineOnlyQueries.java index 7fdd84268ac78..e3699a15e98fb 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestEngineOnlyQueries.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestEngineOnlyQueries.java @@ -6694,6 +6694,22 @@ SELECT a(10) .hasMessage("line 3:8: Recursive language functions are not supported: a(integer):integer"); } + // ensure that JSON_TABLE runs properly in distributed mode (i.e., serialization of handles works correctly, etc) + @Test + public void testJsonTable() + { + assertThat(query(""" + SELECT first, last + FROM (SELECT '{"a" : [1, 2, 3], "b" : [4, 5, 6]}') t(json_col), JSON_TABLE( + json_col, + 'lax $.a' + COLUMNS( + first bigint PATH 'lax $[0]', + last bigint PATH 'lax $[last]')) + """)) + .matches("VALUES (BIGINT '1', BIGINT '3')"); + } + private static ZonedDateTime zonedDateTime(String value) { return ZONED_DATE_TIME_FORMAT.parse(value, ZonedDateTime::from); From 1f2ac222a275ef53db4357f362eb801d24176220 Mon Sep 17 00:00:00 2001 From: Marius Grama Date: Wed, 13 Dec 2023 22:13:08 +0100 Subject: [PATCH 137/350] Disallow changing the location of a table in CREATE OR REPLACE statement --- .../trino/plugin/iceberg/IcebergMetadata.java | 17 ++++++- .../io/trino/plugin/iceberg/IcebergUtil.java | 9 ++-- .../iceberg/catalog/AbstractTrinoCatalog.java | 4 +- .../iceberg/BaseIcebergConnectorTest.java | 49 +++++++++++++++++++ 4 files changed, 71 insertions(+), 8 deletions(-) diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java index 9595bb202cde9..e6e4ca12a869d 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java @@ -234,6 +234,7 @@ import static io.trino.plugin.iceberg.IcebergTableProperties.PARTITIONING_PROPERTY; import static io.trino.plugin.iceberg.IcebergTableProperties.SORTED_BY_PROPERTY; import static io.trino.plugin.iceberg.IcebergTableProperties.getPartitioning; +import static io.trino.plugin.iceberg.IcebergTableProperties.getTableLocation; import static io.trino.plugin.iceberg.IcebergUtil.canEnforceColumnConstraintInSpecs; import static io.trino.plugin.iceberg.IcebergUtil.commit; import static io.trino.plugin.iceberg.IcebergUtil.deserializePartitionValue; @@ -266,6 +267,7 @@ import static io.trino.spi.StandardErrorCode.COLUMN_ALREADY_EXISTS; import static io.trino.spi.StandardErrorCode.INVALID_ANALYZE_PROPERTY; import static io.trino.spi.StandardErrorCode.INVALID_ARGUMENTS; +import static io.trino.spi.StandardErrorCode.INVALID_TABLE_PROPERTY; import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; import static io.trino.spi.StandardErrorCode.QUERY_REJECTED; import static io.trino.spi.StandardErrorCode.TABLE_NOT_FOUND; @@ -297,6 +299,7 @@ import static org.apache.iceberg.TableProperties.FORMAT_VERSION; import static org.apache.iceberg.TableProperties.WRITE_LOCATION_PROVIDER_IMPL; import static org.apache.iceberg.types.TypeUtil.indexParents; +import static org.apache.iceberg.util.LocationUtil.stripTrailingSlash; import static org.apache.iceberg.util.SnapshotUtil.schemaFor; public class IcebergMetadata @@ -934,15 +937,27 @@ public ConnectorOutputTableHandle beginCreateTable(ConnectorSession session, Con if (!schemaExists(session, schemaName)) { throw new SchemaNotFoundException(schemaName); } + + String tableLocation = null; if (replace) { IcebergTableHandle table = (IcebergTableHandle) getTableHandle(session, tableMetadata.getTableSchema().getTable(), Optional.empty(), Optional.empty()); if (table != null) { verifyTableVersionForUpdate(table); Table icebergTable = catalog.loadTable(session, table.getSchemaTableName()); + Optional providedTableLocation = getTableLocation(tableMetadata.getProperties()); + if (providedTableLocation.isPresent() && !stripTrailingSlash(providedTableLocation.get()).equals(icebergTable.location())) { + throw new TrinoException(INVALID_TABLE_PROPERTY, format("The provided location '%s' does not match the existing table location '%s'", providedTableLocation.get(), icebergTable.location())); + } validateNotModifyingOldSnapshot(table, icebergTable); + tableLocation = icebergTable.location(); } } - transaction = newCreateTableTransaction(catalog, tableMetadata, session, replace); + + if (tableLocation == null) { + tableLocation = getTableLocation(tableMetadata.getProperties()) + .orElseGet(() -> catalog.defaultTableLocation(session, tableMetadata.getTable())); + } + transaction = newCreateTableTransaction(catalog, tableMetadata, session, replace, tableLocation); Location location = Location.of(transaction.table().location()); TrinoFileSystem fileSystem = fileSystemFactory.create(session); try { diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergUtil.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergUtil.java index fe71cdbad39b4..3613ec3023d86 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergUtil.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergUtil.java @@ -116,7 +116,6 @@ import static io.trino.plugin.iceberg.IcebergTableProperties.getOrcBloomFilterFpp; import static io.trino.plugin.iceberg.IcebergTableProperties.getPartitioning; import static io.trino.plugin.iceberg.IcebergTableProperties.getSortOrder; -import static io.trino.plugin.iceberg.IcebergTableProperties.getTableLocation; import static io.trino.plugin.iceberg.PartitionFields.parsePartitionFields; import static io.trino.plugin.iceberg.PartitionFields.toPartitionFields; import static io.trino.plugin.iceberg.SortFieldUtils.parseSortFields; @@ -638,19 +637,17 @@ public static Schema schemaFromMetadata(List columns) return new Schema(icebergSchema.asStructType().fields()); } - public static Transaction newCreateTableTransaction(TrinoCatalog catalog, ConnectorTableMetadata tableMetadata, ConnectorSession session, boolean replace) + public static Transaction newCreateTableTransaction(TrinoCatalog catalog, ConnectorTableMetadata tableMetadata, ConnectorSession session, boolean replace, String tableLocation) { SchemaTableName schemaTableName = tableMetadata.getTable(); Schema schema = schemaFromMetadata(tableMetadata.getColumns()); PartitionSpec partitionSpec = parsePartitionFields(schema, getPartitioning(tableMetadata.getProperties())); SortOrder sortOrder = parseSortFields(schema, getSortOrder(tableMetadata.getProperties())); - String targetPath = getTableLocation(tableMetadata.getProperties()) - .orElseGet(() -> catalog.defaultTableLocation(session, schemaTableName)); if (replace) { - return catalog.newCreateOrReplaceTableTransaction(session, schemaTableName, schema, partitionSpec, sortOrder, targetPath, createTableProperties(tableMetadata)); + return catalog.newCreateOrReplaceTableTransaction(session, schemaTableName, schema, partitionSpec, sortOrder, tableLocation, createTableProperties(tableMetadata)); } - return catalog.newCreateTableTransaction(session, schemaTableName, schema, partitionSpec, sortOrder, targetPath, createTableProperties(tableMetadata)); + return catalog.newCreateTableTransaction(session, schemaTableName, schema, partitionSpec, sortOrder, tableLocation, createTableProperties(tableMetadata)); } public static Map createTableProperties(ConnectorTableMetadata tableMetadata) diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/AbstractTrinoCatalog.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/AbstractTrinoCatalog.java index 9f3627088aa0d..d461164a5e707 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/AbstractTrinoCatalog.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/AbstractTrinoCatalog.java @@ -346,7 +346,9 @@ protected SchemaTableName createMaterializedViewStorageTable( List columns = columnsForMaterializedView(definition, materializedViewProperties); ConnectorTableMetadata tableMetadata = new ConnectorTableMetadata(storageTable, columns, materializedViewProperties, Optional.empty()); - Transaction transaction = IcebergUtil.newCreateTableTransaction(this, tableMetadata, session, false); + String tableLocation = getTableLocation(tableMetadata.getProperties()) + .orElseGet(() -> defaultTableLocation(session, tableMetadata.getTable())); + Transaction transaction = IcebergUtil.newCreateTableTransaction(this, tableMetadata, session, false, tableLocation); AppendFiles appendFiles = transaction.newAppend(); commit(appendFiles, session); transaction.commitTransaction(); diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java index 6821a6aaeb990..9fc269172197a 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java @@ -6354,6 +6354,55 @@ public void testCreateOrReplaceTableWithComments() } } + @Test + public void testCreateOrReplaceTableWithSameLocation() + { + try (TestTable table = new TestTable( + getQueryRunner()::execute, + "test_create_or_replace_with_same_location_", + "(a integer)")) { + String initialTableLocation = getTableLocation(table.getName()); + assertUpdate("INSERT INTO " + table.getName() + " VALUES 1", 1); + assertThat(query("SELECT * FROM " + table.getName())) + .matches("VALUES 1"); + long v1SnapshotId = getCurrentSnapshotId(table.getName()); + assertUpdate("CREATE OR REPLACE TABLE " + table.getName() + " (a integer)"); + assertThat(getTableLocation(table.getName())) + .isEqualTo(initialTableLocation); + assertUpdate("CREATE OR REPLACE TABLE " + table.getName() + " (a integer) WITH (location = '" + initialTableLocation + "')"); + String initialTableLocationWithTrailingSlash = initialTableLocation.endsWith("/") ? initialTableLocation : initialTableLocation + "/"; + assertUpdate("CREATE OR REPLACE TABLE " + table.getName() + " (a integer) WITH (location = '" + initialTableLocationWithTrailingSlash + "')"); + assertThat(getTableLocation(table.getName())) + .isEqualTo(initialTableLocation); + assertThat(query("SELECT * FROM " + table.getName())) + .returnsEmptyResult(); + assertUpdate("CREATE OR REPLACE TABLE " + table.getName() + " WITH (location = '" + initialTableLocation + "') AS SELECT 2 as a", 1); + assertThat(query("SELECT * FROM " + table.getName())) + .matches("VALUES 2"); + assertThat(getTableLocation(table.getName())) + .isEqualTo(initialTableLocation); + assertThat(query("SELECT * FROM " + table.getName() + " FOR VERSION AS OF " + v1SnapshotId)) + .matches("VALUES 1"); + } + } + + @Test + public void testCreateOrReplaceTableWithChangeInLocation() + { + try (TestTable table = new TestTable(getQueryRunner()::execute, "test_create_or_replace_change_location_", "(a integer) ")) { + String initialTableLocation = getTableLocation(table.getName()) + randomNameSuffix(); + long v1SnapshotId = getCurrentSnapshotId(table.getName()); + assertQueryFails( + "CREATE OR REPLACE TABLE " + table.getName() + " (a integer) WITH (location = '%s')".formatted(initialTableLocation), + "The provided location '%s' does not match the existing table location '.*'".formatted(initialTableLocation)); + assertQueryFails( + "CREATE OR REPLACE TABLE " + table.getName() + " WITH (location = '%s') AS SELECT 1 AS a".formatted(initialTableLocation), + "The provided location '%s' does not match the existing table location '.*'".formatted(initialTableLocation)); + assertThat(getCurrentSnapshotId(table.getName())) + .isEqualTo(v1SnapshotId); + } + } + @Test public void testMergeSimpleSelectPartitioned() { From a56752ed2d3719129e0770d9d612a3cb68cd95e3 Mon Sep 17 00:00:00 2001 From: James Petty Date: Fri, 17 Nov 2023 10:47:03 -0500 Subject: [PATCH 138/350] Flush dictionaries before releasing partitioners PagePartitioners should either flatten their dictionary mode appenders into direct mode, or force flush their current page to preserve the dictionary encoding before being released to the pool for reuse since it is not possible for the appender to observe the same dictionary input when used from a different driver and the dictionary appenders do not accurately report their size and therefore may have been preventing a flush from occurring up until this point. Also fixes an issue where dictionary block outputs were severely under reporting their output size in bytes. --- .../operator/output/PagePartitioner.java | 82 +++++++++++++++---- .../output/PartitionedOutputOperator.java | 1 + .../output/PositionsAppenderPageBuilder.java | 27 ++++++ .../output/UnnestingPositionsAppender.java | 24 ++++++ .../output/TestPartitionedOutputOperator.java | 5 +- .../TestPositionsAppenderPageBuilder.java | 61 ++++++++++++++ 6 files changed, 185 insertions(+), 15 deletions(-) diff --git a/core/trino-main/src/main/java/io/trino/operator/output/PagePartitioner.java b/core/trino-main/src/main/java/io/trino/operator/output/PagePartitioner.java index 4a58724a729a4..86fd777a98970 100644 --- a/core/trino-main/src/main/java/io/trino/operator/output/PagePartitioner.java +++ b/core/trino-main/src/main/java/io/trino/operator/output/PagePartitioner.java @@ -69,6 +69,9 @@ public class PagePartitioner private final int nullChannel; // when >= 0, send the position to every partition if this channel is null private boolean hasAnyRowBeenReplicated; + // outputSizeInBytes that has already been reported to the operator stats during release and should be subtracted + // from future stats reporting to avoid double counting + private long outputSizeReportedBeforeRelease; public PagePartitioner( PartitionFunction partitionFunction, @@ -135,7 +138,6 @@ public void partitionPage(Page page, OperatorContext operatorContext) } int outputPositionCount = replicatesAnyRow && !hasAnyRowBeenReplicated ? page.getPositionCount() + positionsAppenders.length - 1 : page.getPositionCount(); - long positionsAppendersSizeBefore = getPositionsAppendersSizeInBytes(); if (page.getPositionCount() < partitionFunction.partitionCount() * COLUMNAR_STRATEGY_COEFFICIENT) { // Partition will have on average less than COLUMNAR_STRATEGY_COEFFICIENT rows. // Doing it column-wise would degrade performance, so we fall back to row-wise approach. @@ -146,11 +148,68 @@ public void partitionPage(Page page, OperatorContext operatorContext) else { partitionPageByColumn(page); } - long positionsAppendersSizeAfter = getPositionsAppendersSizeInBytes(); - flushPositionsAppenders(false); + long outputSizeInBytes = flushPositionsAppenders(false); updateMemoryUsage(); + operatorContext.recordOutput(outputSizeInBytes, outputPositionCount); + } + + private long adjustFlushedOutputSizeWithEagerlyReportedBytes(long flushedOutputSize) + { + // Reduce the flushed output size by the previously eagerly reported amount to avoid double counting + if (outputSizeReportedBeforeRelease > 0) { + long adjustmentAmount = min(flushedOutputSize, outputSizeReportedBeforeRelease); + outputSizeReportedBeforeRelease -= adjustmentAmount; + flushedOutputSize -= adjustmentAmount; + } + return flushedOutputSize; + } - operatorContext.recordOutput(positionsAppendersSizeAfter - positionsAppendersSizeBefore, outputPositionCount); + private long adjustEagerlyReportedBytesWithBufferedBytesOnRelease(long bufferedBytesOnRelease) + { + // adjust the amount to eagerly report as output by the amount already eagerly reported if the new value + // is larger, since this indicates that no data was flushed and only the delta between the two values should + // be reported eagerly + if (outputSizeReportedBeforeRelease > 0 && bufferedBytesOnRelease >= outputSizeReportedBeforeRelease) { + bufferedBytesOnRelease -= outputSizeReportedBeforeRelease; + outputSizeReportedBeforeRelease += bufferedBytesOnRelease; + } + return bufferedBytesOnRelease; + } + + /** + * Prepares this {@link PagePartitioner} for release to the pool by checking for dictionary mode appenders and either flattening + * them into direct appenders or forcing their current pages to flush to preserve a valuable dictionary encoded representation. This + * is done before release because we know that after reuse, the appenders will not observe any more inputs using the same dictionary. + *

+ * When a {@link PagePartitioner} is released back to the {@link PagePartitionerPool} we don't know if it will ever be reused. If it is not + * reused, then we have no {@link OperatorContext} we can use to report the output size of the final flushed page, so instead we report the + * buffered bytes still in the partitioner after {@link PagePartitioner#prepareForRelease(OperatorContext)} as output bytes eagerly and record + * that amount in {@link #outputSizeReportedBeforeRelease}. If the {@link PagePartitioner} is reused after having reported buffered bytes eagerly, + * we then have to subtract that same amount from the subsequent output bytes to avoid double counting them. + */ + public void prepareForRelease(OperatorContext operatorContext) + { + long bufferedSizeInBytes = 0; + long outputSizeInBytes = 0; + for (int partition = 0; partition < positionsAppenders.length; partition++) { + PositionsAppenderPageBuilder positionsAppender = positionsAppenders[partition]; + Optional flushedPage = positionsAppender.flushOrFlattenBeforeRelease(); + if (flushedPage.isPresent()) { + Page page = flushedPage.get(); + outputSizeInBytes += page.getSizeInBytes(); + enqueuePage(page, partition); + } + else { + // Dictionaries have now been flattened, so the new reported size is trustworthy to report + // eagerly + bufferedSizeInBytes += positionsAppender.getSizeInBytes(); + } + } + updateMemoryUsage(); + // Adjust flushed and buffered values against the previously eagerly reported sizes + outputSizeInBytes = adjustFlushedOutputSizeWithEagerlyReportedBytes(outputSizeInBytes); + bufferedSizeInBytes = adjustEagerlyReportedBytesWithBufferedBytesOnRelease(bufferedSizeInBytes); + operatorContext.recordOutput(outputSizeInBytes + bufferedSizeInBytes, 0 /* no new positions */); } public void partitionPageByRow(Page page) @@ -210,15 +269,6 @@ public void partitionPageByColumn(Page page) } } - private long getPositionsAppendersSizeInBytes() - { - long sizeInBytes = 0; - for (PositionsAppenderPageBuilder pageBuilder : positionsAppenders) { - sizeInBytes += pageBuilder.getSizeInBytes(); - } - return sizeInBytes; - } - private IntArrayList[] partitionPositions(Page page) { verify(page.getPositionCount() > 0, "position count is 0"); @@ -424,6 +474,7 @@ public void close() { try { flushPositionsAppenders(true); + outputSizeReportedBeforeRelease = 0; } finally { // clear buffers before memory release @@ -432,16 +483,19 @@ public void close() } } - private void flushPositionsAppenders(boolean force) + private long flushPositionsAppenders(boolean force) { + long outputSizeInBytes = 0; // add all full pages to output buffer for (int partition = 0; partition < positionsAppenders.length; partition++) { PositionsAppenderPageBuilder partitionPageBuilder = positionsAppenders[partition]; if (!partitionPageBuilder.isEmpty() && (force || partitionPageBuilder.isFull())) { Page pagePartition = partitionPageBuilder.build(); + outputSizeInBytes += pagePartition.getSizeInBytes(); enqueuePage(pagePartition, partition); } } + return adjustFlushedOutputSizeWithEagerlyReportedBytes(outputSizeInBytes); } private void enqueuePage(Page pagePartition, int partition) diff --git a/core/trino-main/src/main/java/io/trino/operator/output/PartitionedOutputOperator.java b/core/trino-main/src/main/java/io/trino/operator/output/PartitionedOutputOperator.java index 0bc28fee83302..fd683e126352e 100644 --- a/core/trino-main/src/main/java/io/trino/operator/output/PartitionedOutputOperator.java +++ b/core/trino-main/src/main/java/io/trino/operator/output/PartitionedOutputOperator.java @@ -284,6 +284,7 @@ public OperatorContext getOperatorContext() public void finish() { if (!finished) { + pagePartitioner.prepareForRelease(operatorContext); pagePartitionerPool.release(pagePartitioner); finished = true; } diff --git a/core/trino-main/src/main/java/io/trino/operator/output/PositionsAppenderPageBuilder.java b/core/trino-main/src/main/java/io/trino/operator/output/PositionsAppenderPageBuilder.java index 4ba6fd3361dfb..91948beec7611 100644 --- a/core/trino-main/src/main/java/io/trino/operator/output/PositionsAppenderPageBuilder.java +++ b/core/trino-main/src/main/java/io/trino/operator/output/PositionsAppenderPageBuilder.java @@ -21,6 +21,7 @@ import it.unimi.dsi.fastutil.ints.IntArrayList; import java.util.List; +import java.util.Optional; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; @@ -144,6 +145,32 @@ public boolean isEmpty() return declaredPositions == 0; } + public Optional flushOrFlattenBeforeRelease() + { + if (declaredPositions == 0) { + return Optional.empty(); + } + + for (UnnestingPositionsAppender positionsAppender : channelAppenders) { + if (positionsAppender.shouldForceFlushBeforeRelease()) { + // dictionary encoding will be preserved, so force the current page to be flushed + return Optional.of(build()); + } + } + + // transition from dictionary to direct mode if necessary, since we won't be able to reuse the + // same dictionary from the new operator + for (UnnestingPositionsAppender positionsAppender : channelAppenders) { + positionsAppender.flattenPendingDictionary(); + } + + // flush the current page if forced or if the builder is now full as a result of transitioning dictionaries to direct mode + if (isFull()) { + return Optional.of(build()); + } + return Optional.empty(); + } + public Page build() { Block[] blocks = new Block[channelAppenders.length]; diff --git a/core/trino-main/src/main/java/io/trino/operator/output/UnnestingPositionsAppender.java b/core/trino-main/src/main/java/io/trino/operator/output/UnnestingPositionsAppender.java index 23d2c11478615..258aeb54bd5e8 100644 --- a/core/trino-main/src/main/java/io/trino/operator/output/UnnestingPositionsAppender.java +++ b/core/trino-main/src/main/java/io/trino/operator/output/UnnestingPositionsAppender.java @@ -20,6 +20,7 @@ import io.trino.type.BlockTypeOperators.BlockPositionIsDistinctFrom; import it.unimi.dsi.fastutil.ints.IntArrayList; import it.unimi.dsi.fastutil.ints.IntArrays; +import it.unimi.dsi.fastutil.ints.IntOpenHashSet; import jakarta.annotation.Nullable; import java.util.Optional; @@ -52,6 +53,7 @@ private enum State private State state = State.UNINITIALIZED; + @Nullable private ValueBlock dictionary; private DictionaryIdsBuilder dictionaryIdsBuilder; @@ -219,6 +221,28 @@ void addSizesToAccumulator(PositionsAppenderSizeAccumulator accumulator) accumulator.accumulate(sizeInBytes, directSizeInBytes); } + public void flattenPendingDictionary() + { + if (state == State.DICTIONARY && dictionary != null) { + transitionToDirect(); + } + } + + public boolean shouldForceFlushBeforeRelease() + { + if (state == State.DICTIONARY && dictionary != null) { + IntOpenHashSet uniqueIdsSet = new IntOpenHashSet(); + int[] dictionaryIds = dictionaryIdsBuilder.getDictionaryIds(); + for (int i = 0; i < dictionaryIdsBuilder.size(); i++) { + // At least one position is referenced multiple times, preserve the dictionary encoding and force the current page to flush + if (!uniqueIdsSet.add(dictionaryIds[i])) { + return true; + } + } + } + return false; + } + private static class DictionaryIdsBuilder { private static final int INSTANCE_SIZE = instanceSize(DictionaryIdsBuilder.class); diff --git a/core/trino-main/src/test/java/io/trino/operator/output/TestPartitionedOutputOperator.java b/core/trino-main/src/test/java/io/trino/operator/output/TestPartitionedOutputOperator.java index a8adb0ddbab23..9dd54666e942b 100644 --- a/core/trino-main/src/test/java/io/trino/operator/output/TestPartitionedOutputOperator.java +++ b/core/trino-main/src/test/java/io/trino/operator/output/TestPartitionedOutputOperator.java @@ -68,7 +68,10 @@ public void testOperatorContextStats() partitionedOutputOperator.addInput(page); OperatorContext operatorContext = partitionedOutputOperator.getOperatorContext(); - assertThat(operatorContext.getOutputDataSize().getTotalCount()).isEqualTo(page.getSizeInBytes()); + assertThat(operatorContext.getOutputDataSize().getTotalCount()).isEqualTo(0); assertThat(operatorContext.getOutputPositions().getTotalCount()).isEqualTo(page.getPositionCount()); + + partitionedOutputOperator.finish(); + assertThat(operatorContext.getOutputDataSize().getTotalCount()).isEqualTo(page.getSizeInBytes()); } } diff --git a/core/trino-main/src/test/java/io/trino/operator/output/TestPositionsAppenderPageBuilder.java b/core/trino-main/src/test/java/io/trino/operator/output/TestPositionsAppenderPageBuilder.java index ca4cf5f5125fc..542cea8944be0 100644 --- a/core/trino-main/src/test/java/io/trino/operator/output/TestPositionsAppenderPageBuilder.java +++ b/core/trino-main/src/test/java/io/trino/operator/output/TestPositionsAppenderPageBuilder.java @@ -16,14 +16,20 @@ import io.airlift.slice.Slices; import io.trino.spi.Page; import io.trino.spi.block.Block; +import io.trino.spi.block.DictionaryBlock; import io.trino.spi.block.RunLengthEncodedBlock; +import io.trino.spi.block.ValueBlock; +import io.trino.spi.predicate.Utils; import io.trino.type.BlockTypeOperators; import it.unimi.dsi.fastutil.ints.IntArrayList; import org.junit.jupiter.api.Test; import java.util.List; +import java.util.Optional; +import static io.trino.block.BlockAssertions.createRandomBlockForType; import static io.trino.spi.type.VarcharType.VARCHAR; +import static java.lang.Math.toIntExact; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -102,4 +108,59 @@ public void testFullOnDirectSizeInBytes() assertEquals(120, result.getPositionCount(), "result positions should be below the 8192 maximum"); assertTrue(result.getBlock(0) instanceof RunLengthEncodedBlock, "result block is RLE encoded"); } + + @Test + public void testFlushUsefulDictionariesOnRelease() + { + int maxPageBytes = 100; + int maxDirectSize = 1000; + PositionsAppenderPageBuilder pageBuilder = PositionsAppenderPageBuilder.withMaxPageSize( + maxPageBytes, + maxDirectSize, + List.of(VARCHAR), + new PositionsAppenderFactory(new BlockTypeOperators())); + + Block valueBlock = Utils.nativeValueToBlock(VARCHAR, Slices.utf8Slice("test")); + Block dictionaryBlock = DictionaryBlock.create(10, valueBlock, new int[10]); + Page inputPage = new Page(dictionaryBlock); + + pageBuilder.appendToOutputPartition(inputPage, IntArrayList.wrap(new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})); + // Dictionary mode appender should report the size of the ID's, but doesn't currently track + // the per-position size at all because it would be inefficient + assertEquals(Integer.BYTES * 10, pageBuilder.getSizeInBytes()); + assertFalse(pageBuilder.isFull()); + + Optional flushedPage = pageBuilder.flushOrFlattenBeforeRelease(); + assertTrue(flushedPage.isPresent(), "pageBuilder should force flush the dictionary"); + assertTrue(flushedPage.get().getBlock(0) instanceof DictionaryBlock, "result should be dictionary encoded"); + } + + @Test + public void testFlattenUnhelpfulDictionariesOnRelease() + { + // Create unhelpful dictionary wrapping + Block valueBlock = createRandomBlockForType(VARCHAR, 10, 0.25f); + Block dictionaryBlock = DictionaryBlock.create(10, valueBlock, new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}); + Page inputPage = new Page(dictionaryBlock); + + // Ensure the builder allows the entire value block to be inserted without being full + int maxPageBytes = toIntExact(valueBlock.getSizeInBytes() * 10); + int maxDirectSize = maxPageBytes * 10; + PositionsAppenderPageBuilder pageBuilder = PositionsAppenderPageBuilder.withMaxPageSize( + maxPageBytes, + maxDirectSize, + List.of(VARCHAR), + new PositionsAppenderFactory(new BlockTypeOperators())); + + pageBuilder.appendToOutputPartition(inputPage, IntArrayList.wrap(new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})); + assertEquals(Integer.BYTES * 10, pageBuilder.getSizeInBytes()); + assertFalse(pageBuilder.isFull()); + + assertEquals(Optional.empty(), pageBuilder.flushOrFlattenBeforeRelease(), "pageBuilder should not force a flush"); + assertFalse(pageBuilder.isFull()); + assertEquals(valueBlock.getSizeInBytes(), pageBuilder.getSizeInBytes(), "pageBuilder should have transitioned to direct mode"); + + Page result = pageBuilder.build(); + assertTrue(result.getBlock(0) instanceof ValueBlock, "result should not be a dictionary block"); + } } From face6d8660e2dd2da5d9082c19c930bf353c259c Mon Sep 17 00:00:00 2001 From: ajantha-bhat Date: Wed, 25 Oct 2023 13:21:35 +0530 Subject: [PATCH 139/350] Ensure "nessie.commit.id" table property is set when updating the table Spark sets the table property NESSIE_COMMIT_ID_PROPERTY in NessieTableOperations#loadTableMetadata. Then NessieIcebergClient.commitTable uses this property. In Trino, this property is never set but used in NessieIcebergClient.commitTable as it is a common code. Hence, the commit id is old and doesn't allow new commits. Use the common code (available From Iceberg 1.4.0) NessieUtil.updateTableMetadataWithNessieSpecificProperties in Trino, which handles setting the property like "nessie.commit.id". --- .../AbstractIcebergTableOperations.java | 10 +++++++++- .../nessie/IcebergNessieTableOperations.java | 12 +++++++++++ .../TestIcebergSparkCompatibility.java | 20 ++++++++++++++++++- 3 files changed, 40 insertions(+), 2 deletions(-) diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/AbstractIcebergTableOperations.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/AbstractIcebergTableOperations.java index 5920798b8a106..708d48d45a304 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/AbstractIcebergTableOperations.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/AbstractIcebergTableOperations.java @@ -40,6 +40,7 @@ import java.util.Objects; import java.util.Optional; import java.util.OptionalInt; +import java.util.function.Function; import static com.google.common.base.Preconditions.checkState; import static com.google.common.collect.ImmutableList.toImmutableList; @@ -233,6 +234,13 @@ protected String writeNewMetadata(TableMetadata metadata, int newVersion) } protected void refreshFromMetadataLocation(String newLocation) + { + refreshFromMetadataLocation( + newLocation, + metadataLocation -> TableMetadataParser.read(fileIo, fileIo.newInputFile(metadataLocation))); + } + + protected void refreshFromMetadataLocation(String newLocation, Function metadataLoader) { // use null-safe equality check because new tables have a null metadata location if (Objects.equals(currentMetadataLocation, newLocation)) { @@ -254,7 +262,7 @@ protected void refreshFromMetadataLocation(String newLocation) .withMaxDuration(Duration.ofMinutes(10)) .abortOn(failure -> failure instanceof ValidationException || isNotFoundException(failure)) .build()) - .get(() -> TableMetadataParser.read(fileIo, io().newInputFile(newLocation))); + .get(() -> metadataLoader.apply(newLocation)); } catch (Throwable failure) { if (isNotFoundException(failure)) { diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/nessie/IcebergNessieTableOperations.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/nessie/IcebergNessieTableOperations.java index 602e526a213e0..3fe3c5b33dc44 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/nessie/IcebergNessieTableOperations.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/nessie/IcebergNessieTableOperations.java @@ -19,9 +19,11 @@ import io.trino.spi.connector.SchemaTableName; import io.trino.spi.connector.TableNotFoundException; import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableMetadataParser; import org.apache.iceberg.exceptions.CommitFailedException; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.nessie.NessieIcebergClient; +import org.apache.iceberg.nessie.NessieUtil; import org.projectnessie.error.NessieConflictException; import org.projectnessie.error.NessieNotFoundException; import org.projectnessie.model.ContentKey; @@ -80,6 +82,16 @@ public TableMetadata refresh(boolean invalidateCaches) return super.refresh(invalidateCaches); } + @Override + protected void refreshFromMetadataLocation(String newLocation) + { + super.refreshFromMetadataLocation( + newLocation, + location -> NessieUtil.updateTableMetadataWithNessieSpecificProperties( + TableMetadataParser.read(fileIo, location), + location, table, getSchemaTableName().toString(), nessieClient.getReference())); + } + @Override protected String getRefreshedLocation(boolean invalidateCaches) { diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergSparkCompatibility.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergSparkCompatibility.java index 43c77e287ed74..921bb4daf4204 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergSparkCompatibility.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergSparkCompatibility.java @@ -590,7 +590,7 @@ public void testSparkReadingCompositeTrinoData(StorageFormat storageFormat) onTrino().executeQuery("DROP TABLE " + trinoTableName); } - @Test(groups = {ICEBERG, PROFILE_SPECIFIC_TESTS, ICEBERG_REST, ICEBERG_JDBC, ICEBERG_NESSIE}, dataProvider = "storageFormatsWithSpecVersion") + @Test(groups = {ICEBERG, PROFILE_SPECIFIC_TESTS, ICEBERG_REST, ICEBERG_JDBC}, dataProvider = "storageFormatsWithSpecVersion") public void testTrinoReadingSparkIcebergTablePropertiesData(StorageFormat storageFormat, int specVersion) { String baseTableName = toLowerCase("test_trino_reading_spark_iceberg_table_properties_" + storageFormat); @@ -986,6 +986,24 @@ public void testCreateAndDropTableWithSameLocationFailsOnTrino(int specVersion) onTrino().executeQuery(format("DROP TABLE %s", trinoTableName(tableSameLocation2))); } + @Test(groups = {ICEBERG, PROFILE_SPECIFIC_TESTS, ICEBERG_REST, ICEBERG_JDBC, ICEBERG_NESSIE}) + public void testTrinoWritingDataAfterSpark() + { + String baseTableName = toLowerCase("test_trino_write_after_spark"); + String sparkTableName = sparkTableName(baseTableName); + String trinoTableName = trinoTableName(baseTableName); + + onSpark().executeQuery("CREATE TABLE " + sparkTableName + " (a INT) USING ICEBERG"); + onSpark().executeQuery("INSERT INTO " + sparkTableName + " VALUES 1"); + + onTrino().executeQuery("INSERT INTO " + trinoTableName + " VALUES 2"); + + List expected = ImmutableList.of(row(1), row(2)); + assertThat(onTrino().executeQuery("SELECT * FROM " + trinoTableName)).containsOnly(expected); + assertThat(onSpark().executeQuery("SELECT * FROM " + sparkTableName)).containsOnly(expected); + onSpark().executeQuery("DROP TABLE " + sparkTableName); + } + @Test(groups = {ICEBERG, ICEBERG_JDBC, PROFILE_SPECIFIC_TESTS, ICEBERG_NESSIE}, dataProvider = "storageFormatsWithSpecVersion") public void testTrinoWritingDataWithObjectStorageLocationProvider(StorageFormat storageFormat, int specVersion) { From dfdbc7dda76e2f1a01f1a8b2e3be8a537325e91e Mon Sep 17 00:00:00 2001 From: chenjian2664 Date: Fri, 15 Dec 2023 12:50:01 +0800 Subject: [PATCH 140/350] Make logPaths and listeners final --- .../io/trino/tests/product/launcher/env/DockerContainer.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/DockerContainer.java b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/DockerContainer.java index d204eed5854dd..0a9f567579a95 100644 --- a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/DockerContainer.java +++ b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/DockerContainer.java @@ -87,8 +87,8 @@ public class DockerContainer @GuardedBy("this") private OptionalLong lastStartFinishTimeNanos = OptionalLong.empty(); - private List logPaths = new ArrayList<>(); - private List listeners = new ArrayList<>(); + private final List logPaths = new ArrayList<>(); + private final List listeners = new ArrayList<>(); private boolean temporary; private static final ImagePullPolicy pullPolicy = new ConditionalPullPolicy(); From 3aea15e26d51f555ea8a78cc073a80bb1564f9ef Mon Sep 17 00:00:00 2001 From: Marius Grama Date: Mon, 11 Dec 2023 12:57:09 +0100 Subject: [PATCH 141/350] Process only files smaller than the threshold for OPTIMIZE Filter for processing the files which are smaller than the threshold and consider a partition for optimization if and only if it contains at least two such files. --- .../deltalake/DeltaLakeSplitManager.java | 41 ++++++++++++------- .../BaseDeltaLakeConnectorSmokeTest.java | 32 +++++++++++++++ 2 files changed, 59 insertions(+), 14 deletions(-) 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 8f6f48f247db3..cb9e031ea0a36 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,7 +14,6 @@ package io.trino.plugin.deltalake; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableSet; import com.google.inject.Inject; import io.airlift.units.DataSize; import io.trino.filesystem.Location; @@ -46,6 +45,7 @@ import java.net.URI; import java.net.URLDecoder; import java.time.Instant; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; @@ -54,6 +54,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Stream; +import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.collect.ImmutableList.toImmutableList; import static com.google.common.collect.ImmutableMap.toImmutableMap; import static com.google.common.collect.ImmutableSet.toImmutableSet; @@ -70,8 +71,6 @@ import static java.nio.charset.StandardCharsets.UTF_8; import static java.util.Objects.requireNonNull; import static java.util.function.Function.identity; -import static java.util.stream.Collectors.counting; -import static java.util.stream.Collectors.groupingBy; public class DeltaLakeSplitManager implements ConnectorSplitManager @@ -177,7 +176,10 @@ private Stream getSplits( MetadataEntry metadataEntry = tableHandle.getMetadataEntry(); boolean isOptimize = tableHandle.isOptimize(); - Set>> partitionsWithAtMostOneFile = isOptimize ? findPartitionsWithAtMostOneFile(validDataFiles) : ImmutableSet.of(); + if (isOptimize) { + checkArgument(maxScannedFileSizeInBytes.isPresent(), "maxScannedFileSizeInBytes must be provided when performing OPTIMIZE"); + validDataFiles = filterValidDataFilesForOptimize(validDataFiles, maxScannedFileSizeInBytes.get()); + } Set predicatedColumnNames = Stream.concat( nonPartitionConstraint.getDomains().orElseThrow().keySet().stream(), @@ -210,11 +212,6 @@ private Stream getSplits( return Stream.empty(); } - // no need to rewrite small file that is the only one in its partition - if (isOptimize && partitionsWithAtMostOneFile.contains(addAction.getCanonicalPartitionValues()) && maxScannedFileSizeInBytes.isPresent() && addAction.getSize() < maxScannedFileSizeInBytes.get()) { - return Stream.empty(); - } - Map enforcedDomains = enforcedPartitionConstraint.getDomains().orElseThrow(); if (!partitionMatchesPredicate(addAction.getCanonicalPartitionValues(), enforcedDomains)) { return Stream.empty(); @@ -253,12 +250,28 @@ private Stream getSplits( }); } - private Set>> findPartitionsWithAtMostOneFile(List addFileEntries) + private static List filterValidDataFilesForOptimize(List validDataFiles, long maxScannedFileSizeInBytes) { - return addFileEntries.stream().collect(groupingBy(AddFileEntry::getCanonicalPartitionValues, counting())).entrySet().stream() - .filter(entry -> entry.getValue() <= 1) - .map(Map.Entry::getKey) - .collect(toImmutableSet()); + // Value being present is a pending file (potentially the only one) for a given partition. + // Value being empty is a tombstone, indicates that there were in the stream previously at least 2 files selected for processing for a given partition. + Map>, Optional> pendingAddFileEntriesMap = new HashMap<>(); + return validDataFiles.stream() + .filter(addFileEntry -> addFileEntry.getSize() < maxScannedFileSizeInBytes) + .flatMap(addFileEntry -> { + Map> canonicalPartitionValues = addFileEntry.getCanonicalPartitionValues(); + if (pendingAddFileEntriesMap.containsKey(canonicalPartitionValues)) { + Optional alreadyQueuedAddFileEntry = pendingAddFileEntriesMap.get(canonicalPartitionValues); + if (alreadyQueuedAddFileEntry.isEmpty()) { + return Stream.of(addFileEntry); + } + pendingAddFileEntriesMap.put(canonicalPartitionValues, Optional.empty()); + return Stream.of(alreadyQueuedAddFileEntry.get(), addFileEntry); + } + + pendingAddFileEntriesMap.put(canonicalPartitionValues, Optional.of(addFileEntry)); + return Stream.empty(); + }) + .collect(toImmutableList()); } private static boolean mayAnyDataColumnProjected(DeltaLakeTableHandle tableHandle) diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeConnectorSmokeTest.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeConnectorSmokeTest.java index ea8c1f301b675..659c19caf2e9a 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeConnectorSmokeTest.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeConnectorSmokeTest.java @@ -51,6 +51,7 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; +import static com.google.common.base.Strings.repeat; import static com.google.common.base.Verify.verify; import static com.google.common.collect.ImmutableList.toImmutableList; import static com.google.common.collect.ImmutableSet.toImmutableSet; @@ -391,6 +392,37 @@ public void testOptimizeRewritesTable() } } + @Test + public void testOptimizeTableWithSmallFileAndLargeFiles() + { + String tableName = "test_optimize_rewrites_table_with_small_and_large_file" + randomNameSuffix(); + String tableLocation = getLocationForTable(bucketName, tableName); + assertUpdate("CREATE TABLE " + tableName + " (key integer, value varchar) WITH (location = '" + tableLocation + "')"); + try { + // Adds a small file of size < 1 kB + assertUpdate("INSERT INTO " + tableName + " VALUES (1, 'one')", 1); + // Adds other "large" files of size greater than 1 kB + assertUpdate("INSERT INTO " + tableName + " VALUES (2, '" + repeat("two", 1000) + "')", 1); + assertUpdate("INSERT INTO " + tableName + " VALUES (3, '" + repeat("three", 1000) + "')", 1); + + Set initialFiles = getActiveFiles(tableName); + assertThat(initialFiles).hasSize(3); + + for (int i = 0; i < 3; i++) { + computeActual("ALTER TABLE " + tableName + " EXECUTE OPTIMIZE (file_size_threshold => '1kB')"); + Set filesAfterOptimize = getActiveFiles(tableName); + assertThat(filesAfterOptimize) + .containsExactlyInAnyOrderElementsOf(initialFiles); + } + assertQuery( + "SELECT * FROM " + tableName, + "VALUES (1, 'one'), (2, '%s'), (3, '%s')".formatted(repeat("two", 1000), repeat("three", 1000))); + } + finally { + assertUpdate("DROP TABLE " + tableName); + } + } + @Test public void testOptimizeRewritesPartitionedTable() { From 5653e83e88f31be1677a21b21081692e6530752b Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Thu, 14 Dec 2023 09:52:12 +0100 Subject: [PATCH 142/350] Fix identation --- .../planner/AbstractPredicatePushdownTest.java | 10 +++++----- .../io/trino/sql/planner/TestLogicalPlanner.java | 16 ++++++++-------- .../trino/sql/planner/TestPredicatePushdown.java | 2 +- 3 files changed, 14 insertions(+), 14 deletions(-) diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/AbstractPredicatePushdownTest.java b/core/trino-main/src/test/java/io/trino/sql/planner/AbstractPredicatePushdownTest.java index 5d6ebc84b220d..eed65c2900754 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/AbstractPredicatePushdownTest.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/AbstractPredicatePushdownTest.java @@ -458,11 +458,11 @@ public void testOnlyNullPredicateIsPushDownThroughJoinFilters() { assertPlan( """ - WITH t(a) AS (VALUES 'a', 'b') - SELECT * - FROM t t1 JOIN t t2 ON true - WHERE t1.a = 'aa' - """, + WITH t(a) AS (VALUES 'a', 'b') + SELECT * + FROM t t1 JOIN t t2 ON true + WHERE t1.a = 'aa' + """, output(values("field", "field_0"))); } diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/TestLogicalPlanner.java b/core/trino-main/src/test/java/io/trino/sql/planner/TestLogicalPlanner.java index bf20f8dc6edff..0f8be1031b44c 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/TestLogicalPlanner.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/TestLogicalPlanner.java @@ -1691,14 +1691,14 @@ public void testLimitPushdownThroughUnionNesting() { assertPlan( """ - SELECT col FROM ( - SELECT nationkey FROM nation - UNION ALL - SELECT nationkey FROM nation - UNION ALL - SELECT nationkey FROM nation - ) AS t(col) - LIMIT 2""", + SELECT col FROM ( + SELECT nationkey FROM nation + UNION ALL + SELECT nationkey FROM nation + UNION ALL + SELECT nationkey FROM nation + ) AS t(col) + LIMIT 2""", output( limit( 2, diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/TestPredicatePushdown.java b/core/trino-main/src/test/java/io/trino/sql/planner/TestPredicatePushdown.java index 9ff43423109f0..aa1202f94af74 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/TestPredicatePushdown.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/TestPredicatePushdown.java @@ -155,7 +155,7 @@ public void testNonDeterministicPredicateDoesNotPropagateFromFilteringSideToSour semiJoin("LINE_ORDER_KEY", "ORDERS_ORDER_KEY", "SEMI_JOIN_RESULT", true, anyTree( tableScan("lineitem", ImmutableMap.of( - "LINE_ORDER_KEY", "orderkey"))), + "LINE_ORDER_KEY", "orderkey"))), node(ExchangeNode.class, filter("ORDERS_ORDER_KEY = CAST(random(5) AS bigint)", tableScan("orders", ImmutableMap.of("ORDERS_ORDER_KEY", "orderkey"))))))); From cfaa1dfc17a1e31488de445a89288a55db275fed Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 13 Dec 2023 13:40:37 +0100 Subject: [PATCH 143/350] Fix test comment typo --- .../src/test/java/io/trino/cache/TestEvictableCache.java | 2 +- .../src/test/java/io/trino/cache/TestEvictableLoadingCache.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/lib/trino-cache/src/test/java/io/trino/cache/TestEvictableCache.java b/lib/trino-cache/src/test/java/io/trino/cache/TestEvictableCache.java index 0c589503bea79..ce3788a3b8bba 100644 --- a/lib/trino-cache/src/test/java/io/trino/cache/TestEvictableCache.java +++ b/lib/trino-cache/src/test/java/io/trino/cache/TestEvictableCache.java @@ -360,7 +360,7 @@ private static Integer newInteger(int value) /** * Test that the loader is invoked only once for concurrent invocations of {{@link LoadingCache#get(Object, Callable)} with equal keys. - * This is a behavior of Guava Cache as well. While this is necessarily desirable behavior (see + * This is a behavior of Guava Cache as well. While this is not necessarily desirable behavior (see * https://github.com/trinodb/trino/issues/11067), * the test exists primarily to document current state and support discussion, should the current state change. */ diff --git a/lib/trino-cache/src/test/java/io/trino/cache/TestEvictableLoadingCache.java b/lib/trino-cache/src/test/java/io/trino/cache/TestEvictableLoadingCache.java index fea26d863d688..bb2e2f72f06a8 100644 --- a/lib/trino-cache/src/test/java/io/trino/cache/TestEvictableLoadingCache.java +++ b/lib/trino-cache/src/test/java/io/trino/cache/TestEvictableLoadingCache.java @@ -411,7 +411,7 @@ public Map loadAll(Iterablehttps://github.com/trinodb/trino/issues/11067), * the test exists primarily to document current state and support discussion, should the current state change. */ From d4dd6cf53d976c5de2155001df65c49886560ce6 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 8 Dec 2023 16:20:22 +0100 Subject: [PATCH 144/350] Fix negative cost estimates in LocalQueryRunner Fixes negative cost estimated failure in `TestQueryPlanDeterminism.testUnionAllAboveBroadcastJoin`. In `LocalQueryRunner` we were assuming 0 "node count for stats", and getting negative cost in the formula (somewhere deep in the stats calculator): double cpuCost = buildSideSize * (estimatedSourceDistributedTaskCount - 1); where `estimatedSourceDistributedTaskCount` is the node count. --- .../io/trino/testing/LocalQueryRunner.java | 2 +- .../AbstractPredicatePushdownTest.java | 19 ++-- .../planner/TestAddDynamicFilterSource.java | 12 +-- .../trino/sql/planner/TestLogicalPlanner.java | 96 ++++++++++--------- .../sql/planner/TestPredicatePushdown.java | 13 ++- ...PredicatePushdownWithoutDynamicFilter.java | 12 +-- .../sql/planner/TestQuantifiedComparison.java | 6 +- .../TestIcebergProjectionPushdownPlans.java | 16 ++-- .../trino/tests/TestQueryPlanDeterminism.java | 10 -- 9 files changed, 91 insertions(+), 95 deletions(-) diff --git a/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java b/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java index 36c2514e6eb5c..baa0cafe48e3f 100644 --- a/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java +++ b/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java @@ -1208,7 +1208,7 @@ public static class Builder private boolean alwaysRevokeMemory; private Map>> defaultSessionProperties = ImmutableMap.of(); private Set extraSessionProperties = ImmutableSet.of(); - private int nodeCountForStats; + private int nodeCountForStats = 1; private Function metadataDecorator = Function.identity(); private Builder(Session defaultSession) diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/AbstractPredicatePushdownTest.java b/core/trino-main/src/test/java/io/trino/sql/planner/AbstractPredicatePushdownTest.java index eed65c2900754..a1803ba70ec26 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/AbstractPredicatePushdownTest.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/AbstractPredicatePushdownTest.java @@ -40,6 +40,7 @@ import static io.trino.sql.planner.assertions.PlanMatchPattern.tableScan; import static io.trino.sql.planner.assertions.PlanMatchPattern.values; import static io.trino.sql.planner.plan.JoinNode.Type.LEFT; +import static io.trino.sql.tree.BooleanLiteral.TRUE_LITERAL; public abstract class AbstractPredicatePushdownTest extends BasePlanTest @@ -424,33 +425,31 @@ public void testRemovesRedundantTableScanPredicate() @Test public void testTablePredicateIsExtracted() { + PlanMatchPattern ordersTableScan = tableScan("orders", ImmutableMap.of("ORDERSTATUS", "orderstatus")); + if (enableDynamicFiltering) { + ordersTableScan = filter(TRUE_LITERAL, ordersTableScan); + } assertPlan( "SELECT * FROM orders, nation WHERE orderstatus = CAST(nation.name AS varchar(1)) AND orderstatus BETWEEN 'A' AND 'O'", anyTree( node(JoinNode.class, + ordersTableScan, anyTree( filter("CAST(NAME AS varchar(1)) IN ('F', 'O')", tableScan( "nation", - ImmutableMap.of("NAME", "name")))), - anyTree( - tableScan( - "orders", - ImmutableMap.of("ORDERSTATUS", "orderstatus")))))); + ImmutableMap.of("NAME", "name"))))))); assertPlan( "SELECT * FROM orders JOIN nation ON orderstatus = CAST(nation.name AS varchar(1))", anyTree( node(JoinNode.class, + ordersTableScan, anyTree( filter("CAST(NAME AS varchar(1)) IN ('F', 'O', 'P')", tableScan( "nation", - ImmutableMap.of("NAME", "name")))), - anyTree( - tableScan( - "orders", - ImmutableMap.of("ORDERSTATUS", "orderstatus")))))); + ImmutableMap.of("NAME", "name"))))))); } @Test diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/TestAddDynamicFilterSource.java b/core/trino-main/src/test/java/io/trino/sql/planner/TestAddDynamicFilterSource.java index 77335e39a2141..b0ffa8505aada 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/TestAddDynamicFilterSource.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/TestAddDynamicFilterSource.java @@ -225,24 +225,24 @@ public void testCrossJoinInequality() filter("O_ORDERKEY BETWEEN L_ORDERKEY AND L_PARTKEY", join(INNER, builder -> builder .dynamicFilter(ImmutableList.of( - new DynamicFilterPattern("O_ORDERKEY", GREATER_THAN_OR_EQUAL, "L_ORDERKEY"), - new DynamicFilterPattern("O_ORDERKEY", LESS_THAN_OR_EQUAL, "L_PARTKEY"))) + new DynamicFilterPattern("L_ORDERKEY", LESS_THAN_OR_EQUAL, "O_ORDERKEY"), + new DynamicFilterPattern("L_PARTKEY", GREATER_THAN_OR_EQUAL, "O_ORDERKEY"))) .left( filter( TRUE_LITERAL, - tableScan("orders", ImmutableMap.of("O_ORDERKEY", "orderkey")))) + tableScan("lineitem", ImmutableMap.of("L_ORDERKEY", "orderkey", "L_PARTKEY", "partkey")))) .right( exchange( LOCAL, exchange( REMOTE, - node( - DynamicFilterSourceNode.class, - tableScan("lineitem", ImmutableMap.of("L_ORDERKEY", "orderkey", "L_PARTKEY", "partkey")))))))))); + node(DynamicFilterSourceNode.class, + tableScan("orders", ImmutableMap.of("O_ORDERKEY", "orderkey")))))))))); // TODO: Add support for dynamic filters in the below case assertDistributedPlan( "SELECT o.orderkey FROM orders o, lineitem l WHERE o.orderkey >= l.orderkey AND o.orderkey <= l.partkey - 1", + withJoinDistributionType(PARTITIONED), anyTree( filter("O_ORDERKEY >= L_ORDERKEY AND O_ORDERKEY <= expr", join(INNER, builder -> builder diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/TestLogicalPlanner.java b/core/trino-main/src/test/java/io/trino/sql/planner/TestLogicalPlanner.java index 0f8be1031b44c..55f5099d848ec 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/TestLogicalPlanner.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/TestLogicalPlanner.java @@ -397,24 +397,24 @@ public void testInnerInequalityJoinWithEquiJoinConjuncts() anyTree( anyNot(FilterNode.class, join(INNER, builder -> builder - .equiCriteria("O_SHIPPRIORITY", "L_LINENUMBER") + .equiCriteria("L_LINENUMBER", "O_SHIPPRIORITY") .filter("O_ORDERKEY < L_ORDERKEY") .dynamicFilter( ImmutableList.of( - new DynamicFilterPattern("O_SHIPPRIORITY", EQUAL, "L_LINENUMBER"), - new DynamicFilterPattern("O_ORDERKEY", LESS_THAN, "L_ORDERKEY"))) + new DynamicFilterPattern("L_LINENUMBER", EQUAL, "O_SHIPPRIORITY"), + new DynamicFilterPattern("L_ORDERKEY", GREATER_THAN, "O_ORDERKEY"))) .left( filter(TRUE_LITERAL, - tableScan("orders", + tableScan("lineitem", ImmutableMap.of( - "O_SHIPPRIORITY", "shippriority", - "O_ORDERKEY", "orderkey")))) + "L_LINENUMBER", "linenumber", + "L_ORDERKEY", "orderkey")))) .right( anyTree( - tableScan("lineitem", + tableScan("orders", ImmutableMap.of( - "L_LINENUMBER", "linenumber", - "L_ORDERKEY", "orderkey")))))))); + "O_SHIPPRIORITY", "shippriority", + "O_ORDERKEY", "orderkey")))))))); } @Test @@ -439,13 +439,13 @@ public void testJoin() assertPlan("SELECT o.orderkey FROM orders o, lineitem l WHERE l.orderkey = o.orderkey", anyTree( join(INNER, builder -> builder - .equiCriteria("ORDERS_OK", "LINEITEM_OK") + .equiCriteria("LINEITEM_OK", "ORDERS_OK") .left( anyTree( - tableScan("orders", ImmutableMap.of("ORDERS_OK", "orderkey")))) + tableScan("lineitem", ImmutableMap.of("LINEITEM_OK", "orderkey")))) .right( anyTree( - tableScan("lineitem", ImmutableMap.of("LINEITEM_OK", "orderkey"))))))); + tableScan("orders", ImmutableMap.of("ORDERS_OK", "orderkey"))))))); } @Test @@ -454,13 +454,13 @@ public void testJoinWithOrderBySameKey() assertPlan("SELECT o.orderkey FROM orders o, lineitem l WHERE l.orderkey = o.orderkey ORDER BY l.orderkey ASC, o.orderkey ASC", anyTree( join(INNER, builder -> builder - .equiCriteria("ORDERS_OK", "LINEITEM_OK") + .equiCriteria("LINEITEM_OK", "ORDERS_OK") .left( anyTree( - tableScan("orders", ImmutableMap.of("ORDERS_OK", "orderkey")))) + tableScan("lineitem", ImmutableMap.of("LINEITEM_OK", "orderkey")))) .right( anyTree( - tableScan("lineitem", ImmutableMap.of("LINEITEM_OK", "orderkey"))))))); + tableScan("orders", ImmutableMap.of("ORDERS_OK", "orderkey"))))))); } @Test @@ -902,19 +902,19 @@ public void testStreamingAggregationOverJoin() assertPlan("SELECT o.orderkey, count(*) FROM orders o, lineitem l WHERE o.orderkey=l.orderkey GROUP BY 1", anyTree( aggregation( - singleGroupingSet("o_orderkey"), + singleGroupingSet("l_orderkey"), ImmutableMap.of(Optional.empty(), functionCall("count", ImmutableList.of())), - ImmutableList.of("o_orderkey"), // streaming + ImmutableList.of("l_orderkey"), // streaming Optional.empty(), SINGLE, join(INNER, builder -> builder - .equiCriteria("o_orderkey", "l_orderkey") + .equiCriteria("l_orderkey", "o_orderkey") .left( anyTree( - tableScan("orders", ImmutableMap.of("o_orderkey", "orderkey")))) + tableScan("lineitem", ImmutableMap.of("l_orderkey", "orderkey")))) .right( anyTree( - tableScan("lineitem", ImmutableMap.of("l_orderkey", "orderkey")))))))); + tableScan("orders", ImmutableMap.of("o_orderkey", "orderkey")))))))); // left join -> streaming aggregation assertPlan("SELECT o.orderkey, count(*) FROM orders o LEFT JOIN lineitem l ON o.orderkey=l.orderkey GROUP BY 1", @@ -1324,7 +1324,9 @@ public void testUsesDistributedJoinIfNaturallyPartitionedOnProbeSymbols() // replicated join is preserved if probe side is single node assertPlanWithSession( "SELECT * FROM (VALUES 1, 2, 3) t(a), region r WHERE r.regionkey = t.a", - broadcastJoin, + Session.builder(broadcastJoin) + .setSystemProperty(JOIN_REORDERING_STRATEGY, JoinReorderingStrategy.NONE.name()) + .build(), false, anyTree( node(JoinNode.class, @@ -1433,18 +1435,18 @@ public void testFilteringSemiJoinRewriteToInnerJoin() "SELECT custkey FROM orders WHERE custkey IN (SELECT custkey FROM customer)", any( join(INNER, builder -> builder - .equiCriteria("CUSTOMER_CUSTKEY", "ORDER_CUSTKEY") + .equiCriteria("ORDER_CUSTKEY", "CUSTOMER_CUSTKEY") .left( + anyTree( + tableScan("orders", ImmutableMap.of("ORDER_CUSTKEY", "custkey")))) + .right( aggregation( singleGroupingSet("CUSTOMER_CUSTKEY"), ImmutableMap.of(), Optional.empty(), FINAL, anyTree( - tableScan("customer", ImmutableMap.of("CUSTOMER_CUSTKEY", "custkey"))))) - .right( - anyTree( - tableScan("orders", ImmutableMap.of("ORDER_CUSTKEY", "custkey"))))))); + tableScan("customer", ImmutableMap.of("CUSTOMER_CUSTKEY", "custkey")))))))); } @Test @@ -1861,13 +1863,19 @@ public void testRemoveRedundantFilter() "ON orders.orderstatus = t2.s", any( join(INNER, builder -> builder - .equiCriteria("expr", "ORDER_STATUS") - .left(anyTree(values(ImmutableList.of("expr"), ImmutableList.of(ImmutableList.of(new StringLiteral("O")), ImmutableList.of(new StringLiteral("F")))))) + .equiCriteria("ORDER_STATUS", "expr") + .left( + filter(TRUE_LITERAL, + strictConstrainedTableScan( + "orders", + ImmutableMap.of("ORDER_STATUS", "orderstatus", "ORDER_KEY", "orderkey"), + ImmutableMap.of("orderstatus", multipleValues(createVarcharType(1), ImmutableList.of(utf8Slice("F"), utf8Slice("O"))))))) .right( - exchange(strictConstrainedTableScan( - "orders", - ImmutableMap.of("ORDER_STATUS", "orderstatus", "ORDER_KEY", "orderkey"), - ImmutableMap.of("orderstatus", multipleValues(createVarcharType(1), ImmutableList.of(utf8Slice("F"), utf8Slice("O")))))))))); + filter( + "expr IN ('F', 'O')", + values( + ImmutableList.of("expr"), + ImmutableList.of(ImmutableList.of(new StringLiteral("O")), ImmutableList.of(new StringLiteral("F"))))))))); } @Test @@ -1956,16 +1964,16 @@ public void testMergeProjectWithValues() "ON orders.orderstatus = t2.s", anyTree( join(INNER, builder -> builder - .equiCriteria("expr", "ORDER_STATUS") + .equiCriteria("ORDER_STATUS", "expr") .left( - filter("expr IN ('F', 'O')", - values(ImmutableList.of("expr"), ImmutableList.of(ImmutableList.of(new StringLiteral("O")), ImmutableList.of(new StringLiteral("F")))))) - .right( - exchange( + filter(TRUE_LITERAL, strictConstrainedTableScan( "orders", ImmutableMap.of("ORDER_STATUS", "orderstatus", "ORDER_KEY", "orderkey"), - ImmutableMap.of("orderstatus", multipleValues(createVarcharType(1), ImmutableList.of(utf8Slice("F"), utf8Slice("O")))))))))); + ImmutableMap.of("orderstatus", multipleValues(createVarcharType(1), ImmutableList.of(utf8Slice("F"), utf8Slice("O"))))))) + .right( + filter("expr IN ('F', 'O')", + values(ImmutableList.of("expr"), ImmutableList.of(ImmutableList.of(new StringLiteral("O")), ImmutableList.of(new StringLiteral("F"))))))))); // Constraint for the table is derived, based on constant values in the other branch of the join. // It is not accepted by the connector, and remains in form of a filter over TableScan. @@ -1976,18 +1984,18 @@ public void testMergeProjectWithValues() "ON orders.orderkey = t2.s", anyTree( join(INNER, builder -> builder - .equiCriteria("expr", "ORDER_KEY") + .equiCriteria("ORDER_KEY", "expr") .left( filter( - "expr IN (BIGINT '1', BIGINT '2')", - values(ImmutableList.of("expr"), ImmutableList.of(ImmutableList.of(new GenericLiteral("BIGINT", "1")), ImmutableList.of(new GenericLiteral("BIGINT", "2")))))) - .right( - anyTree(filter( "ORDER_KEY IN (BIGINT '1', BIGINT '2')", strictConstrainedTableScan( "orders", ImmutableMap.of("ORDER_STATUS", "orderstatus", "ORDER_KEY", "orderkey"), - ImmutableMap.of()))))))); + ImmutableMap.of()))) + .right( + filter( + "expr IN (BIGINT '1', BIGINT '2')", + values(ImmutableList.of("expr"), ImmutableList.of(ImmutableList.of(new GenericLiteral("BIGINT", "1")), ImmutableList.of(new GenericLiteral("BIGINT", "2"))))))))); } @Test diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/TestPredicatePushdown.java b/core/trino-main/src/test/java/io/trino/sql/planner/TestPredicatePushdown.java index aa1202f94af74..934be3f14f777 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/TestPredicatePushdown.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/TestPredicatePushdown.java @@ -168,15 +168,14 @@ public void testNonStraddlingJoinExpression() "SELECT * FROM orders JOIN lineitem ON orders.orderkey = lineitem.orderkey AND cast(lineitem.linenumber AS varchar) = '2'", anyTree( join(INNER, builder -> builder - .equiCriteria("ORDERS_OK", "LINEITEM_OK") + .equiCriteria("LINEITEM_OK", "ORDERS_OK") .left( - anyTree( - tableScan("orders", ImmutableMap.of("ORDERS_OK", "orderkey")))) + filter("cast(LINEITEM_LINENUMBER as varchar) = VARCHAR '2'", + tableScan("lineitem", ImmutableMap.of( + "LINEITEM_OK", "orderkey", + "LINEITEM_LINENUMBER", "linenumber")))) .right( anyTree( - filter("cast(LINEITEM_LINENUMBER as varchar) = VARCHAR '2'", - tableScan("lineitem", ImmutableMap.of( - "LINEITEM_OK", "orderkey", - "LINEITEM_LINENUMBER", "linenumber")))))))); + tableScan("orders", ImmutableMap.of("ORDERS_OK", "orderkey"))))))); } } diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/TestPredicatePushdownWithoutDynamicFilter.java b/core/trino-main/src/test/java/io/trino/sql/planner/TestPredicatePushdownWithoutDynamicFilter.java index b6ded5e0c49f0..1c4e627e47429 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/TestPredicatePushdownWithoutDynamicFilter.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/TestPredicatePushdownWithoutDynamicFilter.java @@ -163,14 +163,14 @@ public void testNonStraddlingJoinExpression() "SELECT * FROM orders JOIN lineitem ON orders.orderkey = lineitem.orderkey AND cast(lineitem.linenumber AS varchar) = '2'", anyTree( join(INNER, builder -> builder - .equiCriteria("ORDERS_OK", "LINEITEM_OK") + .equiCriteria("LINEITEM_OK", "ORDERS_OK") .left( - tableScan("orders", ImmutableMap.of("ORDERS_OK", "orderkey"))) + filter("cast(LINEITEM_LINENUMBER as varchar) = VARCHAR '2'", + tableScan("lineitem", ImmutableMap.of( + "LINEITEM_OK", "orderkey", + "LINEITEM_LINENUMBER", "linenumber")))) .right( anyTree( - filter("cast(LINEITEM_LINENUMBER as varchar) = VARCHAR '2'", - tableScan("lineitem", ImmutableMap.of( - "LINEITEM_OK", "orderkey", - "LINEITEM_LINENUMBER", "linenumber")))))))); + tableScan("orders", ImmutableMap.of("ORDERS_OK", "orderkey"))))))); } } diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/TestQuantifiedComparison.java b/core/trino-main/src/test/java/io/trino/sql/planner/TestQuantifiedComparison.java index e85181388c774..6261d4ab2a7d5 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/TestQuantifiedComparison.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/TestQuantifiedComparison.java @@ -38,9 +38,9 @@ public void testQuantifiedComparisonEqualsAny() String query = "SELECT orderkey, custkey FROM orders WHERE orderkey = ANY (VALUES ROW(CAST(5 as BIGINT)), ROW(CAST(3 as BIGINT)))"; assertPlan(query, anyTree( join(INNER, builder -> builder - .equiCriteria("Y", "X") - .left(anyTree(values(ImmutableMap.of("Y", 0)))) - .right(anyTree(tableScan("orders", ImmutableMap.of("X", "orderkey"))))))); + .equiCriteria("X", "Y") + .left(anyTree(tableScan("orders", ImmutableMap.of("X", "orderkey")))) + .right(anyTree(values(ImmutableMap.of("Y", 0))))))); } @Test diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergProjectionPushdownPlans.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergProjectionPushdownPlans.java index bd2953f7646bd..cd5de6b18f3a3 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergProjectionPushdownPlans.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergProjectionPushdownPlans.java @@ -218,8 +218,14 @@ public void testDereferencePushdown() "expr_0", expression("expr_0"), "expr_0_y", expression("expr_0[2]")), join(INNER, builder -> builder - .equiCriteria("t_expr_1", "s_expr_1") + .equiCriteria("s_expr_1", "t_expr_1") .left( + anyTree( + tableScan( + equalTo(((IcebergTableHandle) tableHandle.get().getConnectorHandle()).withProjectedColumns(Set.of(column1Handle))), + TupleDomain.all(), + ImmutableMap.of("s_expr_1", equalTo(column1Handle))))) + .right( anyTree( filter( "x = BIGINT '2'", @@ -234,12 +240,6 @@ public void testDereferencePushdown() unenforcedConstraint.equals(expectedUnenforcedConstraint); }, TupleDomain.all(), - ImmutableMap.of("x", equalTo(columnX), "expr_0", equalTo(column0Handle), "t_expr_1", equalTo(column1Handle)))))) - .right( - anyTree( - tableScan( - equalTo(((IcebergTableHandle) tableHandle.get().getConnectorHandle()).withProjectedColumns(Set.of(column1Handle))), - TupleDomain.all(), - ImmutableMap.of("s_expr_1", equalTo(column1Handle))))))))); + ImmutableMap.of("x", equalTo(columnX), "expr_0", equalTo(column0Handle), "t_expr_1", equalTo(column1Handle)))))))))); } } diff --git a/testing/trino-tests/src/test/java/io/trino/tests/TestQueryPlanDeterminism.java b/testing/trino-tests/src/test/java/io/trino/tests/TestQueryPlanDeterminism.java index 488696ac7ac65..9a6bd296c921b 100644 --- a/testing/trino-tests/src/test/java/io/trino/tests/TestQueryPlanDeterminism.java +++ b/testing/trino-tests/src/test/java/io/trino/tests/TestQueryPlanDeterminism.java @@ -34,7 +34,6 @@ import static io.trino.plugin.tpch.TpchMetadata.TINY_SCHEMA_NAME; import static io.trino.testing.TestingSession.testSessionBuilder; -import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; @TestInstance(PER_CLASS) @@ -275,13 +274,4 @@ public void testLargeIn() // testLargeIn is expensive Assumptions.abort("Skipping testLargeIn"); } - - @Test - @Override - public void testUnionAllAboveBroadcastJoin() - { - // TODO: https://github.com/trinodb/trino/issues/20043 - assertThatThrownBy(super::testUnionAllAboveBroadcastJoin) - .hasMessageContaining("bytes is negative"); - } } From 7e8ad5a5823913062362e2934998d339a0d19dbe Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 8 Dec 2023 16:34:23 +0100 Subject: [PATCH 145/350] Add guards for 0 #nodes in CBO Current formulas like double cpuCost = buildSideSize * (estimatedSourceDistributedTaskCount - 1); would produce negative values when node count is 0. In fact, 0 nodes cannot execute a query, so 0 is not a valid #nodes estimate. --- .../io/trino/SystemSessionProperties.java | 1 + .../CostCalculatorWithEstimatedExchanges.java | 2 ++ .../io/trino/cost/TaskCountEstimator.java | 19 ++++++++++++++----- 3 files changed, 17 insertions(+), 5 deletions(-) diff --git a/core/trino-main/src/main/java/io/trino/SystemSessionProperties.java b/core/trino-main/src/main/java/io/trino/SystemSessionProperties.java index c08a13a91391e..44d2805976aa8 100644 --- a/core/trino-main/src/main/java/io/trino/SystemSessionProperties.java +++ b/core/trino-main/src/main/java/io/trino/SystemSessionProperties.java @@ -715,6 +715,7 @@ public SystemSessionProperties( COST_ESTIMATION_WORKER_COUNT, "Set the estimate count of workers while planning", null, + value -> validateIntegerValue(value, COST_ESTIMATION_WORKER_COUNT, 1, true), true), booleanProperty( OMIT_DATETIME_TYPE_PRECISION, diff --git a/core/trino-main/src/main/java/io/trino/cost/CostCalculatorWithEstimatedExchanges.java b/core/trino-main/src/main/java/io/trino/cost/CostCalculatorWithEstimatedExchanges.java index d536ca27c102c..cfc5ee9fb1ce7 100644 --- a/core/trino-main/src/main/java/io/trino/cost/CostCalculatorWithEstimatedExchanges.java +++ b/core/trino-main/src/main/java/io/trino/cost/CostCalculatorWithEstimatedExchanges.java @@ -32,6 +32,7 @@ import java.util.Objects; import java.util.Optional; +import static com.google.common.base.Preconditions.checkArgument; import static io.trino.cost.LocalCostEstimate.addPartialComponents; import static java.util.Objects.requireNonNull; @@ -206,6 +207,7 @@ public static LocalCostEstimate calculateJoinCostWithoutOutput( boolean replicated, int estimatedSourceDistributedTaskCount) { + checkArgument(estimatedSourceDistributedTaskCount > 0, "estimatedSourceDistributedTaskCount must be positive: %s", estimatedSourceDistributedTaskCount); LocalCostEstimate exchangesCost = calculateJoinExchangeCost( probe, build, diff --git a/core/trino-main/src/main/java/io/trino/cost/TaskCountEstimator.java b/core/trino-main/src/main/java/io/trino/cost/TaskCountEstimator.java index 21a460d5a8b3b..c78a11de5bb3a 100644 --- a/core/trino-main/src/main/java/io/trino/cost/TaskCountEstimator.java +++ b/core/trino-main/src/main/java/io/trino/cost/TaskCountEstimator.java @@ -23,6 +23,7 @@ import java.util.Set; import java.util.function.IntSupplier; +import static com.google.common.base.Preconditions.checkState; import static io.trino.SystemSessionProperties.getCostEstimationWorkerCount; import static io.trino.SystemSessionProperties.getFaultTolerantExecutionMaxPartitionCount; import static io.trino.SystemSessionProperties.getMaxHashPartitionCount; @@ -42,12 +43,17 @@ public TaskCountEstimator(NodeSchedulerConfig nodeSchedulerConfig, InternalNodeM requireNonNull(nodeManager, "nodeManager is null"); this.numberOfNodes = () -> { Set activeNodes = nodeManager.getAllNodes().getActiveNodes(); + int count; if (schedulerIncludeCoordinator) { - return activeNodes.size(); + count = activeNodes.size(); } - return toIntExact(activeNodes.stream() - .filter(node -> !node.isCoordinator()) - .count()); + else { + count = toIntExact(activeNodes.stream() + .filter(node -> !node.isCoordinator()) + .count()); + } + // At least 1 even if no worker nodes currently registered. This is to prevent underflow or other mis-estimations. + return Math.max(count, 1); }; } @@ -60,9 +66,12 @@ public int estimateSourceDistributedTaskCount(Session session) { Integer costEstimationWorkerCount = getCostEstimationWorkerCount(session); if (costEstimationWorkerCount != null) { + // validated to be at least 1 return costEstimationWorkerCount; } - return numberOfNodes.getAsInt(); + int count = numberOfNodes.getAsInt(); + checkState(count > 0, "%s should return positive number of nodes: %s", numberOfNodes, count); + return count; } public int estimateHashedTaskCount(Session session) From ab27cb88c2c23e7ada18c372550f92d204fcca64 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 8 Dec 2023 16:40:01 +0100 Subject: [PATCH 146/350] Reject negative cost components No work has negative cost. Previously such negative costs could go unnoticed, or noticed only if explain plan is produced. --- .../src/main/java/io/trino/cost/LocalCostEstimate.java | 5 +++++ .../java/io/trino/cost/PlanNodeStatsAndCostSummary.java | 8 ++++++++ 2 files changed, 13 insertions(+) diff --git a/core/trino-main/src/main/java/io/trino/cost/LocalCostEstimate.java b/core/trino-main/src/main/java/io/trino/cost/LocalCostEstimate.java index 62a575ed3057d..cbeaad3913540 100644 --- a/core/trino-main/src/main/java/io/trino/cost/LocalCostEstimate.java +++ b/core/trino-main/src/main/java/io/trino/cost/LocalCostEstimate.java @@ -21,7 +21,9 @@ import java.util.stream.Stream; import static com.google.common.base.MoreObjects.toStringHelper; +import static com.google.common.base.Preconditions.checkArgument; import static java.lang.Double.NaN; +import static java.lang.Double.isNaN; /** * Represents inherent cost of some plan node, not including cost of its sources. @@ -63,6 +65,9 @@ public LocalCostEstimate( @JsonProperty("maxMemory") double maxMemory, @JsonProperty("networkCost") double networkCost) { + checkArgument(isNaN(cpuCost) || cpuCost >= 0, "cpuCost cannot be negative: %s", cpuCost); + checkArgument(isNaN(maxMemory) || maxMemory >= 0, "maxMemory cannot be negative: %s", maxMemory); + checkArgument(isNaN(networkCost) || networkCost >= 0, "networkCost cannot be negative: %s", networkCost); this.cpuCost = cpuCost; this.maxMemory = maxMemory; this.networkCost = networkCost; diff --git a/core/trino-main/src/main/java/io/trino/cost/PlanNodeStatsAndCostSummary.java b/core/trino-main/src/main/java/io/trino/cost/PlanNodeStatsAndCostSummary.java index 9d151c0162e41..97b1f93855018 100644 --- a/core/trino-main/src/main/java/io/trino/cost/PlanNodeStatsAndCostSummary.java +++ b/core/trino-main/src/main/java/io/trino/cost/PlanNodeStatsAndCostSummary.java @@ -16,6 +16,9 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import static com.google.common.base.Preconditions.checkArgument; +import static java.lang.Double.isNaN; + public class PlanNodeStatsAndCostSummary { private final double outputRowCount; @@ -32,6 +35,11 @@ public PlanNodeStatsAndCostSummary( @JsonProperty("memoryCost") double memoryCost, @JsonProperty("networkCost") double networkCost) { + checkArgument(isNaN(outputRowCount) || outputRowCount >= 0, "outputRowCount cannot be negative: %s", outputRowCount); + checkArgument(isNaN(outputSizeInBytes) || outputSizeInBytes >= 0, "outputSizeInBytes cannot be negative: %s", outputSizeInBytes); + checkArgument(isNaN(cpuCost) || cpuCost >= 0, "cpuCost cannot be negative: %s", cpuCost); + checkArgument(isNaN(memoryCost) || memoryCost >= 0, "memoryCost cannot be negative: %s", memoryCost); + checkArgument(isNaN(networkCost) || networkCost >= 0, "networkCost cannot be negative: %s", networkCost); this.outputRowCount = outputRowCount; this.outputSizeInBytes = outputSizeInBytes; this.cpuCost = cpuCost; From 12c33fb6455bfbde411c1263488aac8e3d7c5369 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 8 Dec 2023 16:42:02 +0100 Subject: [PATCH 147/350] Remove unused field --- .../src/main/java/io/trino/testing/LocalQueryRunner.java | 1 - 1 file changed, 1 deletion(-) diff --git a/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java b/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java index baa0cafe48e3f..403a27fab0476 100644 --- a/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java +++ b/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java @@ -1204,7 +1204,6 @@ public static class Builder private final Session defaultSession; private FeaturesConfig featuresConfig = new FeaturesConfig(); private NodeSpillConfig nodeSpillConfig = new NodeSpillConfig(); - private boolean initialTransaction; private boolean alwaysRevokeMemory; private Map>> defaultSessionProperties = ImmutableMap.of(); private Set extraSessionProperties = ImmutableSet.of(); From 2407eb945a847469e876efa70f323939d74a60d3 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 8 Dec 2023 16:45:46 +0100 Subject: [PATCH 148/350] Make DistributedQueryRunner.withTracing idempotent --- .../trino/testing/DistributedQueryRunner.java | 37 +++++++++++-------- 1 file changed, 21 insertions(+), 16 deletions(-) diff --git a/testing/trino-testing/src/main/java/io/trino/testing/DistributedQueryRunner.java b/testing/trino-testing/src/main/java/io/trino/testing/DistributedQueryRunner.java index 5c62cb78a6e10..2ca6f3a1fde93 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/DistributedQueryRunner.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/DistributedQueryRunner.java @@ -81,6 +81,7 @@ import static com.google.common.base.MoreObjects.firstNonNull; import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkState; import static com.google.common.base.Throwables.throwIfUnchecked; import static com.google.common.base.Verify.verify; import static com.google.inject.util.Modules.EMPTY_MODULE; @@ -656,6 +657,7 @@ private static void closeUnchecked(AutoCloseable closeable) public static class Builder> { private Session defaultSession; + private boolean withTracing; private int nodeCount = 3; private Map extraProperties = ImmutableMap.of(); private Map coordinatorProperties = ImmutableMap.of(); @@ -673,6 +675,8 @@ public static class Builder> protected Builder(Session defaultSession) { this.defaultSession = requireNonNull(defaultSession, "defaultSession is null"); + String tracingEnabled = firstNonNull(getenv("TESTS_TRACING_ENABLED"), "false"); + this.withTracing = parseBoolean(tracingEnabled) || tracingEnabled.equals("1"); } @CanIgnoreReturnValue @@ -828,19 +832,7 @@ public SELF enableBackupCoordinator() public SELF withTracing() { - OpenTracingCollector collector = new OpenTracingCollector(); - collector.start(); - extraCloseables = ImmutableList.of(collector); - this.addExtraProperties(Map.of("tracing.enabled", "true", "tracing.exporter.endpoint", collector.getExporterEndpoint().toString())); - this.setEventListener(new EventListener() - { - @Override - public void queryCompleted(QueryCompletedEvent queryCompletedEvent) - { - String queryId = queryCompletedEvent.getMetadata().getQueryId(); - log.info("TRACING: %s :: %s", queryId, collector.searchForQueryId(queryId)); - } - }); + this.withTracing = true; return self(); } @@ -853,9 +845,22 @@ protected SELF self() public DistributedQueryRunner build() throws Exception { - String tracingEnabled = firstNonNull(getenv("TESTS_TRACING_ENABLED"), "false"); - if (parseBoolean(tracingEnabled) || tracingEnabled.equals("1")) { - withTracing(); + if (withTracing) { + checkState(extraCloseables.isEmpty(), "extraCloseables already set"); + OpenTracingCollector collector = new OpenTracingCollector(); + collector.start(); + extraCloseables = ImmutableList.of(collector); + addExtraProperties(Map.of("tracing.enabled", "true", "tracing.exporter.endpoint", collector.getExporterEndpoint().toString())); + checkState(eventListeners.isEmpty(), "eventListeners already set"); + setEventListener(new EventListener() + { + @Override + public void queryCompleted(QueryCompletedEvent queryCompletedEvent) + { + String queryId = queryCompletedEvent.getMetadata().getQueryId(); + log.info("TRACING: %s :: %s", queryId, collector.searchForQueryId(queryId)); + } + }); } Optional systemAccessControlConfiguration = this.systemAccessControlConfiguration; From 982be2a3e00c71fca6d83fee6557e28e40c2b5d4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C5=81ukasz=20Osipiuk?= Date: Fri, 15 Dec 2023 12:36:12 +0100 Subject: [PATCH 149/350] Log finalQueryInfo when FTE scheduler stalled --- .../faulttolerant/EventDrivenFaultTolerantQueryScheduler.java | 1 + 1 file changed, 1 insertion(+) diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/EventDrivenFaultTolerantQueryScheduler.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/EventDrivenFaultTolerantQueryScheduler.java index 1c8f611b7d236..3d451fac3687f 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/EventDrivenFaultTolerantQueryScheduler.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/EventDrivenFaultTolerantQueryScheduler.java @@ -936,6 +936,7 @@ private void logDebugInfo(String reason) log.debug("Scheduler debug info for %s START; reason=%s", queryStateMachine.getQueryId(), reason); log.debug("General state: %s", toStringHelper(this) .add("queryState", queryStateMachine.getQueryState()) + .add("finalQueryInfo", queryStateMachine.getFinalQueryInfo()) .add("maxTaskExecutionAttempts", maxTaskExecutionAttempts) .add("maxTasksWaitingForNode", maxTasksWaitingForNode) .add("maxTasksWaitingForExecution", maxTasksWaitingForExecution) From 23be8c9d478b3b8530939a69ca3c5ff26148f1d4 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 15 Dec 2023 12:34:05 +0100 Subject: [PATCH 150/350] Increase GCLocker retries for Maven The build was observed failing with Warning: [325.033s][warning][gc,alloc] mvn-builder-trino-parquet: Retried waiting for GCLocker too often allocating 4194306 words on CI. --- .mvn/jvm.config | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.mvn/jvm.config b/.mvn/jvm.config index 65c0bc4d12016..6a0272bcee2a2 100644 --- a/.mvn/jvm.config +++ b/.mvn/jvm.config @@ -9,3 +9,5 @@ --add-exports=jdk.compiler/com.sun.tools.javac.util=ALL-UNNAMED --add-opens=jdk.compiler/com.sun.tools.javac.code=ALL-UNNAMED --add-opens=jdk.compiler/com.sun.tools.javac.comp=ALL-UNNAMED +-XX:+UnlockDiagnosticVMOptions +-XX:GCLockerRetryAllocationCount=100 From b6eabd5704a68845fdd4015906ca37bac4e61158 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 15 Dec 2023 12:09:03 +0100 Subject: [PATCH 151/350] Fix table listing concurrency in FileHiveMetastore Before the change, `FileHiveMetastore` would rely on `fileSystem.listFiles` to find schemas or tables within a schema. `listFiles` is recursive and, on a local file system, fails when files/directories are being modified concurrently. This commit replaces recursive `listFiles` listing with non-recursive `listDirectories` leveraging the fact that listed entities are represented as directories. --- .../metastore/file/FileHiveMetastore.java | 26 +++++++------------ 1 file changed, 10 insertions(+), 16 deletions(-) diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/file/FileHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/file/FileHiveMetastore.java index efd6ab1b7d22b..bdcd2f1b4ac65 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/file/FileHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/file/FileHiveMetastore.java @@ -89,6 +89,7 @@ import java.util.stream.Collectors; import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Verify.verify; import static com.google.common.collect.ImmutableList.toImmutableList; import static com.google.common.collect.ImmutableMap.toImmutableMap; import static com.google.common.collect.ImmutableSet.toImmutableSet; @@ -324,6 +325,7 @@ public synchronized List getAllDatabases() String prefix = catalogDirectory.toString(); Set databases = new HashSet<>(); + // TODO this lists files recursively and may fail if e.g. table data being modified by other threads/processes FileIterator iterator = fileSystem.listFiles(catalogDirectory); while (iterator.hasNext()) { Location location = iterator.next().location(); @@ -598,24 +600,16 @@ private List doListAllTables(String databaseName) Location metadataDirectory = getDatabaseMetadataDirectory(databaseName); try { String prefix = metadataDirectory.toString(); + if (!prefix.endsWith("/")) { + prefix += "/"; + } Set tables = new HashSet<>(); - FileIterator iterator = fileSystem.listFiles(metadataDirectory); - while (iterator.hasNext()) { - Location location = iterator.next().location(); - - String child = location.toString().substring(prefix.length()); - if (child.startsWith("/")) { - child = child.substring(1); - } - - if (child.startsWith(".") || (child.indexOf('/') != child.lastIndexOf('/'))) { - continue; - } - - int length = child.length() - TRINO_SCHEMA_FILE_NAME_SUFFIX.length() - 1; - if ((length >= 1) && child.endsWith("/" + TRINO_SCHEMA_FILE_NAME_SUFFIX)) { - tables.add(child.substring(0, length)); + for (Location subdirectory : fileSystem.listDirectories(metadataDirectory)) { + String locationString = subdirectory.toString(); + verify(locationString.startsWith(prefix) && locationString.endsWith("/"), "Unexpected subdirectory %s when listing %s", subdirectory, metadataDirectory); + if (fileSystem.newInputFile(subdirectory.appendPath(TRINO_SCHEMA_FILE_NAME_SUFFIX)).exists()) { + tables.add(locationString.substring(prefix.length(), locationString.length() - 1)); } } From 0eb70febcfeaa43bb6774b2a343bd0f3da3f7cd9 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 15 Dec 2023 13:36:24 +0100 Subject: [PATCH 152/350] Increase timeout for TestWindow.testManyFunctionsWithSameWindow The test sometimes failed due to a timeout on CI. --- .../trino-main/src/test/java/io/trino/sql/query/TestWindow.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/trino-main/src/test/java/io/trino/sql/query/TestWindow.java b/core/trino-main/src/test/java/io/trino/sql/query/TestWindow.java index 9996aae42631e..0545ffc9eb2ac 100644 --- a/core/trino-main/src/test/java/io/trino/sql/query/TestWindow.java +++ b/core/trino-main/src/test/java/io/trino/sql/query/TestWindow.java @@ -36,7 +36,7 @@ public void teardown() } @Test - @Timeout(2) + @Timeout(5) public void testManyFunctionsWithSameWindow() { assertThat(assertions.query(""" From a16cfdd98512b9bb241de579d96b5933cad907a8 Mon Sep 17 00:00:00 2001 From: Yuya Ebihara Date: Wed, 29 Nov 2023 09:13:30 +0900 Subject: [PATCH 153/350] Adjust BigQuery CI profile --- .github/workflows/ci.yml | 22 +++++++++++----------- plugin/trino-bigquery/pom.xml | 16 ++++++++-------- 2 files changed, 19 insertions(+), 19 deletions(-) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 258ae6d0630f8..e7faf76eddf1d 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -511,7 +511,7 @@ jobs: - { modules: lib/trino-filesystem-gcs, profile: cloud-tests } - { modules: plugin/trino-accumulo } - { modules: plugin/trino-bigquery } - - { modules: plugin/trino-bigquery, profile: cloud-tests-arrow-and-fte } + - { modules: plugin/trino-bigquery, profile: cloud-tests-2 } - { modules: plugin/trino-cassandra } - { modules: plugin/trino-clickhouse } - { modules: plugin/trino-delta-lake } @@ -588,7 +588,7 @@ jobs: matrix.modules != 'plugin/trino-singlestore' && ! (contains(matrix.modules, 'trino-delta-lake') && contains(matrix.profile, 'cloud-tests')) && ! (contains(matrix.modules, 'trino-iceberg') && contains(matrix.profile, 'cloud-tests')) - && ! (contains(matrix.modules, 'trino-bigquery') && contains(matrix.profile, 'cloud-tests-arrow-and-fte')) + && ! (contains(matrix.modules, 'trino-bigquery') && contains(matrix.profile, 'cloud-tests-2')) && ! (contains(matrix.modules, 'trino-redshift') && contains(matrix.profile, 'cloud-tests')) && ! (contains(matrix.modules, 'trino-redshift') && contains(matrix.profile, 'fte-tests')) && ! (contains(matrix.modules, 'trino-filesystem-s3') && contains(matrix.profile, 'cloud-tests')) @@ -691,25 +691,25 @@ jobs: env: BIGQUERY_CREDENTIALS_KEY: ${{ secrets.BIGQUERY_CREDENTIALS_KEY }} GCP_STORAGE_BUCKET: ${{ vars.GCP_STORAGE_BUCKET }} - if: matrix.modules == 'plugin/trino-bigquery' && !contains(matrix.profile, 'cloud-tests-arrow-and-fte') && (env.CI_SKIP_SECRETS_PRESENCE_CHECKS != '' || env.BIGQUERY_CREDENTIALS_KEY != '') + if: matrix.modules == 'plugin/trino-bigquery' && !contains(matrix.profile, 'cloud-tests-2') && (env.CI_SKIP_SECRETS_PRESENCE_CHECKS != '' || env.BIGQUERY_CREDENTIALS_KEY != '') run: | - $MAVEN test ${MAVEN_TEST} -pl :trino-bigquery -Pcloud-tests \ + $MAVEN test ${MAVEN_TEST} -pl :trino-bigquery -Pcloud-tests-1 \ -Dbigquery.credentials-key="${BIGQUERY_CREDENTIALS_KEY}" \ - -Dtesting.gcp-storage-bucket="${GCP_STORAGE_BUCKET}" \ - -Dtesting.alternate-bq-project-id=bigquery-cicd-alternate - - name: Cloud BigQuery Arrow and FTE Tests + -Dtesting.gcp-storage-bucket="${GCP_STORAGE_BUCKET}" + - name: Cloud BigQuery Smoke Tests env: BIGQUERY_CREDENTIALS_KEY: ${{ secrets.BIGQUERY_CREDENTIALS_KEY }} GCP_STORAGE_BUCKET: ${{ vars.GCP_STORAGE_BUCKET }} - if: matrix.modules == 'plugin/trino-bigquery' && contains(matrix.profile, 'cloud-tests-arrow-and-fte') && (env.CI_SKIP_SECRETS_PRESENCE_CHECKS != '' || env.BIGQUERY_CREDENTIALS_KEY != '') + if: matrix.modules == 'plugin/trino-bigquery' && contains(matrix.profile, 'cloud-tests-2') && (env.CI_SKIP_SECRETS_PRESENCE_CHECKS != '' || env.BIGQUERY_CREDENTIALS_KEY != '') run: | - $MAVEN test ${MAVEN_TEST} -pl :trino-bigquery -Pcloud-tests-arrow-and-fte \ + $MAVEN test ${MAVEN_TEST} -pl :trino-bigquery -Pcloud-tests-2 \ -Dbigquery.credentials-key="${BIGQUERY_CREDENTIALS_KEY}" \ - -Dtesting.gcp-storage-bucket="${GCP_STORAGE_BUCKET}" + -Dtesting.gcp-storage-bucket="${GCP_STORAGE_BUCKET}" \ + -Dtesting.alternate-bq-project-id=bigquery-cicd-alternate - name: Cloud BigQuery Case Insensitive Mapping Tests env: BIGQUERY_CASE_INSENSITIVE_CREDENTIALS_KEY: ${{ secrets.BIGQUERY_CASE_INSENSITIVE_CREDENTIALS_KEY }} - if: matrix.modules == 'plugin/trino-bigquery' && !contains(matrix.profile, 'cloud-tests-arrow-and-fte') && (env.CI_SKIP_SECRETS_PRESENCE_CHECKS != '' || env.BIGQUERY_CASE_INSENSITIVE_CREDENTIALS_KEY != '') + if: matrix.modules == 'plugin/trino-bigquery' && !contains(matrix.profile, 'cloud-tests-2') && (env.CI_SKIP_SECRETS_PRESENCE_CHECKS != '' || env.BIGQUERY_CASE_INSENSITIVE_CREDENTIALS_KEY != '') run: | $MAVEN test ${MAVEN_TEST} -pl :trino-bigquery -Pcloud-tests-case-insensitive-mapping -Dbigquery.credentials-key="${BIGQUERY_CASE_INSENSITIVE_CREDENTIALS_KEY}" - name: Iceberg Cloud Tests diff --git a/plugin/trino-bigquery/pom.xml b/plugin/trino-bigquery/pom.xml index dab8f20a180c9..3570d09995032 100644 --- a/plugin/trino-bigquery/pom.xml +++ b/plugin/trino-bigquery/pom.xml @@ -539,7 +539,7 @@ - cloud-tests + cloud-tests-1 false @@ -551,12 +551,6 @@ **/TestBigQueryAvroConnectorTest.java - **/TestBigQueryWithDifferentProjectIdConnectorSmokeTest.java - **/TestBigQueryMetadataCaching.java - **/TestBigQueryAvroTypeMapping.java - **/TestBigQueryMetadata.java - **/TestBigQueryInstanceCleaner.java - **/TestBigQueryWithProxyConnectorSmokeTest.java @@ -589,7 +583,7 @@ - cloud-tests-arrow-and-fte + cloud-tests-2 false @@ -604,8 +598,14 @@ **/TestBigQueryArrowConnectorSmokeTest.java + **/TestBigQueryWithDifferentProjectIdConnectorSmokeTest.java + **/TestBigQueryWithProxyConnectorSmokeTest.java **/TestBigQueryArrowTypeMapping.java + **/TestBigQueryAvroTypeMapping.java + **/TestBigQueryMetadataCaching.java + **/TestBigQueryMetadata.java **/TestBigQuery*FailureRecoveryTest.java + **/TestBigQueryInstanceCleaner.java From 4d3c11b4cdfcbf8af8957efb871c9960dc250f91 Mon Sep 17 00:00:00 2001 From: Dejan Mircevski Date: Tue, 12 Dec 2023 22:27:16 -0500 Subject: [PATCH 154/350] Rearrange candidate-host logic in bin-packing allocator Results should be identical, but the new code is easier to extend for failover. --- .../BinPackingNodeAllocatorService.java | 24 +++++++++---------- 1 file changed, 11 insertions(+), 13 deletions(-) diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/BinPackingNodeAllocatorService.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/BinPackingNodeAllocatorService.java index d6757fbce2d2f..902ae541e47a2 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/BinPackingNodeAllocatorService.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/BinPackingNodeAllocatorService.java @@ -34,6 +34,7 @@ import io.trino.metadata.InternalNode; import io.trino.metadata.InternalNodeManager; import io.trino.metadata.InternalNodeManager.NodesSnapshot; +import io.trino.spi.HostAddress; import io.trino.spi.TrinoException; import io.trino.spi.memory.MemoryPoolInfo; import jakarta.annotation.PostConstruct; @@ -41,6 +42,7 @@ import org.assertj.core.util.VisibleForTesting; import java.time.Duration; +import java.util.ArrayList; import java.util.Comparator; import java.util.Deque; import java.util.HashMap; @@ -578,19 +580,15 @@ public ReserveResult tryReserve(PendingAcquire acquire) NodeRequirements requirements = acquire.getNodeRequirements(); Optional> catalogNodes = requirements.getCatalogHandle().map(nodesSnapshot::getConnectorNodes); - List candidates = allNodesSorted.stream() - .filter(node -> catalogNodes.isEmpty() || catalogNodes.get().contains(node)) - .filter(node -> { - // Allow using coordinator if explicitly requested - if (requirements.getAddresses().contains(node.getHostAndPort())) { - return true; - } - if (requirements.getAddresses().isEmpty()) { - return scheduleOnCoordinator || !node.isCoordinator(); - } - return false; - }) - .collect(toImmutableList()); + List candidates = new ArrayList<>(allNodesSorted); + catalogNodes.ifPresent(candidates::retainAll); // Drop non-catalog nodes, if any. + Set addresses = requirements.getAddresses(); + if (!addresses.isEmpty()) { + candidates = candidates.stream().filter(node -> addresses.contains(node.getHostAndPort())).collect(toImmutableList()); + } + else if (!scheduleOnCoordinator) { + candidates = candidates.stream().filter(node -> !node.isCoordinator()).collect(toImmutableList()); + } if (candidates.isEmpty()) { return ReserveResult.NONE_MATCHING; From f6819ac2bf938fd3d210598657f1599a7a5388b1 Mon Sep 17 00:00:00 2001 From: Dejan Mircevski Date: Tue, 12 Dec 2023 23:20:40 -0500 Subject: [PATCH 155/350] Factor out dropCoordinatorsIfNecessary --- .../faulttolerant/BinPackingNodeAllocatorService.java | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/BinPackingNodeAllocatorService.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/BinPackingNodeAllocatorService.java index 902ae541e47a2..2b8b874d488ba 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/BinPackingNodeAllocatorService.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/BinPackingNodeAllocatorService.java @@ -575,6 +575,11 @@ public BinPackingSimulation( } } + private List dropCoordinatorsIfNecessary(List candidates) + { + return scheduleOnCoordinator ? candidates : candidates.stream().filter(node -> !node.isCoordinator()).collect(toImmutableList()); + } + public ReserveResult tryReserve(PendingAcquire acquire) { NodeRequirements requirements = acquire.getNodeRequirements(); @@ -586,8 +591,8 @@ public ReserveResult tryReserve(PendingAcquire acquire) if (!addresses.isEmpty()) { candidates = candidates.stream().filter(node -> addresses.contains(node.getHostAndPort())).collect(toImmutableList()); } - else if (!scheduleOnCoordinator) { - candidates = candidates.stream().filter(node -> !node.isCoordinator()).collect(toImmutableList()); + else { + candidates = dropCoordinatorsIfNecessary(candidates); } if (candidates.isEmpty()) { From 5fb10501009721bb9d6aebcfe4ba1ffe74621513 Mon Sep 17 00:00:00 2001 From: Manfred Moser Date: Fri, 15 Dec 2023 12:38:42 -0800 Subject: [PATCH 156/350] Update requirement to Oracle 19 --- docs/src/main/sphinx/connector/oracle.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/src/main/sphinx/connector/oracle.md b/docs/src/main/sphinx/connector/oracle.md index ccbfb9a483741..b823e642dfef6 100644 --- a/docs/src/main/sphinx/connector/oracle.md +++ b/docs/src/main/sphinx/connector/oracle.md @@ -18,7 +18,7 @@ like Oracle and Hive, or different Oracle database instances. To connect to Oracle, you need: -- Oracle 12 or higher. +- Oracle 19 or higher. - Network access from the Trino coordinator and workers to Oracle. Port 1521 is the default port. @@ -43,7 +43,7 @@ to the JDBC driver. The Oracle connector uses the Oracle JDBC Thin driver, and the syntax of the URL may be different depending on your Oracle configuration. For example, the connection URL is different if you are connecting to an Oracle SID or an Oracle service name. See the [Oracle -Database JDBC driver documentation](https://docs.oracle.com/en/database/oracle/oracle-database/21/jjdbc/data-sources-and-URLs.html#GUID-088B1600-C6C2-4F19-A020-2DAF8FE1F1C3) +Database JDBC driver documentation](https://docs.oracle.com/en/database/oracle/oracle-database/19/jjdbc/data-sources-and-URLs.html) for more information. The `connection-user` and `connection-password` are typically required and @@ -54,7 +54,7 @@ properties files. :::{note} Oracle does not expose metadata comment via `REMARKS` column by default in JDBC driver. You can enable it using `oracle.remarks-reporting.enabled` -config option. See [Additional Oracle Performance Extensions](https://docs.oracle.com/en/database/oracle/oracle-database/19/jjdbc/performance-extensions.html#GUID-96A38C6D-A288-4E0B-9F03-E711C146632B) +config option. See [Additional Oracle Performance Extensions](https://docs.oracle.com/en/database/oracle/oracle-database/19/jjdbc/performance-extensions.html) for more details. ::: From 5912370f16057a420111cd61ccc3802174365a45 Mon Sep 17 00:00:00 2001 From: Elon Azoulay Date: Sat, 16 Dec 2023 08:44:46 -0800 Subject: [PATCH 157/350] Reduce repetitions of testCreateFileRetry --- .../java/io/trino/filesystem/gcs/TestGcsFileSystemGcs.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/lib/trino-filesystem-gcs/src/test/java/io/trino/filesystem/gcs/TestGcsFileSystemGcs.java b/lib/trino-filesystem-gcs/src/test/java/io/trino/filesystem/gcs/TestGcsFileSystemGcs.java index 852b30da274b9..00f08df91377d 100644 --- a/lib/trino-filesystem-gcs/src/test/java/io/trino/filesystem/gcs/TestGcsFileSystemGcs.java +++ b/lib/trino-filesystem-gcs/src/test/java/io/trino/filesystem/gcs/TestGcsFileSystemGcs.java @@ -38,8 +38,12 @@ void setup() @Test void testCreateFileRetry() { + // Note: this test is meant to expose flakiness + // Without retries it may fail non-deterministically. + // Retries are enabled in the default GcsFileSystemConfig. + // In practice this may happen between 7 and 20 retries. assertThatNoException().isThrownBy(() -> { - for (int i = 1; i <= 100; i++) { + for (int i = 1; i <= 30; i++) { TrinoOutputFile outputFile = getFileSystem().newOutputFile(getRootLocation().appendPath("testFile")); try (OutputStream out = outputFile.createOrOverwrite()) { out.write("test".getBytes(UTF_8)); From df3a63af556ec071ef89440f00a46c4760e9d564 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 15 Dec 2023 17:21:21 +0100 Subject: [PATCH 158/350] Convert PTL tests to JUnit Convert product tests launcher's test to JUnit. This does not change product tests themselves. --- testing/trino-product-tests-launcher/pom.xml | 16 ++++++++++++++-- .../product/launcher/cli/TestInvocations.java | 6 ++++-- .../product/launcher/cli/TestOptionsPrinter.java | 2 +- .../product/launcher/env/TestConfigurations.java | 2 +- .../launcher/local/TestManuallyJdbcOauth2.java | 15 ++++++++------- .../product/launcher/util/TestConsoleTable.java | 2 +- 6 files changed, 29 insertions(+), 14 deletions(-) diff --git a/testing/trino-product-tests-launcher/pom.xml b/testing/trino-product-tests-launcher/pom.xml index a808a275eb6ca..986f9a209202d 100644 --- a/testing/trino-product-tests-launcher/pom.xml +++ b/testing/trino-product-tests-launcher/pom.xml @@ -199,6 +199,12 @@ + + io.airlift + junit-extensions + test + + io.trino trino-jdbc @@ -212,8 +218,14 @@ - org.testng - testng + org.junit.jupiter + junit-jupiter-api + test + + + + org.junit.jupiter + junit-jupiter-engine test diff --git a/testing/trino-product-tests-launcher/src/test/java/io/trino/tests/product/launcher/cli/TestInvocations.java b/testing/trino-product-tests-launcher/src/test/java/io/trino/tests/product/launcher/cli/TestInvocations.java index c3fc55911bc43..4344639ac7e85 100644 --- a/testing/trino-product-tests-launcher/src/test/java/io/trino/tests/product/launcher/cli/TestInvocations.java +++ b/testing/trino-product-tests-launcher/src/test/java/io/trino/tests/product/launcher/cli/TestInvocations.java @@ -14,7 +14,8 @@ package io.trino.tests.product.launcher.cli; import com.google.common.base.Splitter; -import org.testng.annotations.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.parallel.Execution; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -24,8 +25,9 @@ import static io.trino.tests.product.launcher.cli.Launcher.execute; import static java.nio.charset.StandardCharsets.UTF_8; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; -@Test(singleThreaded = true) +@Execution(SAME_THREAD) public class TestInvocations { @Test diff --git a/testing/trino-product-tests-launcher/src/test/java/io/trino/tests/product/launcher/cli/TestOptionsPrinter.java b/testing/trino-product-tests-launcher/src/test/java/io/trino/tests/product/launcher/cli/TestOptionsPrinter.java index e1bd087930e09..a5728b650e79b 100644 --- a/testing/trino-product-tests-launcher/src/test/java/io/trino/tests/product/launcher/cli/TestOptionsPrinter.java +++ b/testing/trino-product-tests-launcher/src/test/java/io/trino/tests/product/launcher/cli/TestOptionsPrinter.java @@ -14,7 +14,7 @@ package io.trino.tests.product.launcher.cli; import com.google.common.collect.ImmutableList; -import org.testng.annotations.Test; +import org.junit.jupiter.api.Test; import picocli.CommandLine.Option; import picocli.CommandLine.Parameters; diff --git a/testing/trino-product-tests-launcher/src/test/java/io/trino/tests/product/launcher/env/TestConfigurations.java b/testing/trino-product-tests-launcher/src/test/java/io/trino/tests/product/launcher/env/TestConfigurations.java index 2f5a744d0153e..28016a171170c 100644 --- a/testing/trino-product-tests-launcher/src/test/java/io/trino/tests/product/launcher/env/TestConfigurations.java +++ b/testing/trino-product-tests-launcher/src/test/java/io/trino/tests/product/launcher/env/TestConfigurations.java @@ -17,7 +17,7 @@ import io.trino.tests.product.launcher.suite.suites.Suite1; import io.trino.tests.product.launcher.suite.suites.Suite6NonGeneric; import io.trino.tests.product.launcher.suite.suites.SuiteTpcds; -import org.testng.annotations.Test; +import org.junit.jupiter.api.Test; import static io.trino.tests.product.launcher.Configurations.canonicalEnvironmentName; import static io.trino.tests.product.launcher.Configurations.nameForSuiteClass; diff --git a/testing/trino-product-tests-launcher/src/test/java/io/trino/tests/product/launcher/local/TestManuallyJdbcOauth2.java b/testing/trino-product-tests-launcher/src/test/java/io/trino/tests/product/launcher/local/TestManuallyJdbcOauth2.java index 1a87ab9de2c8a..ffd092f1a8a64 100644 --- a/testing/trino-product-tests-launcher/src/test/java/io/trino/tests/product/launcher/local/TestManuallyJdbcOauth2.java +++ b/testing/trino-product-tests-launcher/src/test/java/io/trino/tests/product/launcher/local/TestManuallyJdbcOauth2.java @@ -13,8 +13,8 @@ */ package io.trino.tests.product.launcher.local; -import org.testng.annotations.BeforeClass; -import org.testng.annotations.Test; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; import java.net.InetAddress; import java.net.Socket; @@ -23,7 +23,6 @@ import java.sql.DriverManager; import java.sql.PreparedStatement; import java.sql.ResultSet; -import java.sql.SQLException; import java.util.Properties; import static java.lang.String.format; @@ -32,8 +31,7 @@ public class TestManuallyJdbcOauth2 { - @BeforeClass(alwaysRun = true) - public void verifyEtcHostsEntries() + private static void verifyEtcHostsEntries() throws UnknownHostException { assertThat(InetAddress.getByName("presto-master").isLoopbackAddress()).isTrue(); @@ -53,10 +51,13 @@ public void verifyEtcHostsEntries() * 127.0.0.1 hydra * 127.0.0.1 hydra-consent */ - @Test(enabled = false) + @Test + @Disabled public void shouldAuthenticateAndExecuteQuery() - throws SQLException + throws Exception { + verifyEtcHostsEntries(); + Properties properties = new Properties(); String jdbcUrl = format("jdbc:trino://presto-master:7778?" + "SSL=true&" diff --git a/testing/trino-product-tests-launcher/src/test/java/io/trino/tests/product/launcher/util/TestConsoleTable.java b/testing/trino-product-tests-launcher/src/test/java/io/trino/tests/product/launcher/util/TestConsoleTable.java index 99170caa82fea..91797f932ce04 100644 --- a/testing/trino-product-tests-launcher/src/test/java/io/trino/tests/product/launcher/util/TestConsoleTable.java +++ b/testing/trino-product-tests-launcher/src/test/java/io/trino/tests/product/launcher/util/TestConsoleTable.java @@ -13,7 +13,7 @@ */ package io.trino.tests.product.launcher.util; -import org.testng.annotations.Test; +import org.junit.jupiter.api.Test; import static org.assertj.core.api.Assertions.assertThat; From ed6e854f9ca7d2cf2c3baf2f550ee0f34bf858c4 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 15 Dec 2023 16:55:21 +0100 Subject: [PATCH 159/350] Remove TestNG from test classpath Remove TestNG from classpath of modules that do not use it anymore. --- client/trino-cli/pom.xml | 6 ------ client/trino-client/pom.xml | 6 ------ lib/trino-hdfs/pom.xml | 6 ------ lib/trino-plugin-toolkit/pom.xml | 6 ------ plugin/trino-accumulo/pom.xml | 6 ------ plugin/trino-base-jdbc/pom.xml | 6 ------ plugin/trino-bigquery/pom.xml | 6 ------ plugin/trino-cassandra/pom.xml | 6 ------ plugin/trino-clickhouse/pom.xml | 6 ------ plugin/trino-kudu/pom.xml | 6 ------ plugin/trino-password-authenticators/pom.xml | 6 ------ plugin/trino-pinot/pom.xml | 6 ------ plugin/trino-raptor-legacy/pom.xml | 6 ------ plugin/trino-resource-group-managers/pom.xml | 6 ------ plugin/trino-thrift-api/pom.xml | 6 ------ plugin/trino-thrift/pom.xml | 6 ------ 16 files changed, 96 deletions(-) diff --git a/client/trino-cli/pom.xml b/client/trino-cli/pom.xml index 80fc251f557e4..03ca229a1e534 100644 --- a/client/trino-cli/pom.xml +++ b/client/trino-cli/pom.xml @@ -147,12 +147,6 @@ junit-jupiter-engine test - - - org.testng - testng - test - diff --git a/client/trino-client/pom.xml b/client/trino-client/pom.xml index 074194851396f..875d625530ff7 100644 --- a/client/trino-client/pom.xml +++ b/client/trino-client/pom.xml @@ -122,11 +122,5 @@ junit-jupiter-api test - - - org.testng - testng - test - diff --git a/lib/trino-hdfs/pom.xml b/lib/trino-hdfs/pom.xml index d36fa826048bf..02167d3ce2608 100644 --- a/lib/trino-hdfs/pom.xml +++ b/lib/trino-hdfs/pom.xml @@ -261,12 +261,6 @@ testcontainers test - - - org.testng - testng - test - diff --git a/lib/trino-plugin-toolkit/pom.xml b/lib/trino-plugin-toolkit/pom.xml index bffe213e00840..01cd65a8fe6c4 100644 --- a/lib/trino-plugin-toolkit/pom.xml +++ b/lib/trino-plugin-toolkit/pom.xml @@ -187,12 +187,6 @@ junit-jupiter-engine test - - - org.testng - testng - test - diff --git a/plugin/trino-accumulo/pom.xml b/plugin/trino-accumulo/pom.xml index 02347a6284bbd..1f1c1697b4f74 100644 --- a/plugin/trino-accumulo/pom.xml +++ b/plugin/trino-accumulo/pom.xml @@ -314,12 +314,6 @@ testcontainers test - - - org.testng - testng - test - diff --git a/plugin/trino-base-jdbc/pom.xml b/plugin/trino-base-jdbc/pom.xml index bae0cfc8b9cc6..2952976342510 100644 --- a/plugin/trino-base-jdbc/pom.xml +++ b/plugin/trino-base-jdbc/pom.xml @@ -275,12 +275,6 @@ junit-jupiter-engine test - - - org.testng - testng - test - diff --git a/plugin/trino-bigquery/pom.xml b/plugin/trino-bigquery/pom.xml index 3570d09995032..fc7493cc7c795 100644 --- a/plugin/trino-bigquery/pom.xml +++ b/plugin/trino-bigquery/pom.xml @@ -484,12 +484,6 @@ 5.6.0 test - - - org.testng - testng - test - diff --git a/plugin/trino-cassandra/pom.xml b/plugin/trino-cassandra/pom.xml index f8eaf4fc7c590..5da45d4abc3c0 100644 --- a/plugin/trino-cassandra/pom.xml +++ b/plugin/trino-cassandra/pom.xml @@ -251,12 +251,6 @@ testcontainers test - - - org.testng - testng - test - diff --git a/plugin/trino-clickhouse/pom.xml b/plugin/trino-clickhouse/pom.xml index 1f7861124ddfe..b36e0422613b7 100644 --- a/plugin/trino-clickhouse/pom.xml +++ b/plugin/trino-clickhouse/pom.xml @@ -199,11 +199,5 @@ testcontainers test - - - org.testng - testng - test - diff --git a/plugin/trino-kudu/pom.xml b/plugin/trino-kudu/pom.xml index 3b755040c959f..61874946a4fc7 100644 --- a/plugin/trino-kudu/pom.xml +++ b/plugin/trino-kudu/pom.xml @@ -228,12 +228,6 @@ toxiproxy test - - - org.testng - testng - test - diff --git a/plugin/trino-password-authenticators/pom.xml b/plugin/trino-password-authenticators/pom.xml index a401dd4ebfc8c..3d0e2cbd98b79 100644 --- a/plugin/trino-password-authenticators/pom.xml +++ b/plugin/trino-password-authenticators/pom.xml @@ -164,11 +164,5 @@ toxiproxy test - - - org.testng - testng - test - diff --git a/plugin/trino-pinot/pom.xml b/plugin/trino-pinot/pom.xml index b6ddeb1131521..7ba6ff80a783f 100755 --- a/plugin/trino-pinot/pom.xml +++ b/plugin/trino-pinot/pom.xml @@ -658,12 +658,6 @@ testcontainers test - - - org.testng - testng - test - diff --git a/plugin/trino-raptor-legacy/pom.xml b/plugin/trino-raptor-legacy/pom.xml index c827073f709eb..eeff307695e7f 100644 --- a/plugin/trino-raptor-legacy/pom.xml +++ b/plugin/trino-raptor-legacy/pom.xml @@ -313,11 +313,5 @@ testcontainers test - - - org.testng - testng - test - diff --git a/plugin/trino-resource-group-managers/pom.xml b/plugin/trino-resource-group-managers/pom.xml index ec9819c0a4c6e..730877fe81ce6 100644 --- a/plugin/trino-resource-group-managers/pom.xml +++ b/plugin/trino-resource-group-managers/pom.xml @@ -257,11 +257,5 @@ testcontainers test - - - org.testng - testng - test - diff --git a/plugin/trino-thrift-api/pom.xml b/plugin/trino-thrift-api/pom.xml index 116a381208a92..fec2de0085624 100644 --- a/plugin/trino-thrift-api/pom.xml +++ b/plugin/trino-thrift-api/pom.xml @@ -83,12 +83,6 @@ junit-jupiter-engine test - - - org.testng - testng - test - diff --git a/plugin/trino-thrift/pom.xml b/plugin/trino-thrift/pom.xml index 4456743199a8c..0284ae57d8de9 100644 --- a/plugin/trino-thrift/pom.xml +++ b/plugin/trino-thrift/pom.xml @@ -241,12 +241,6 @@ junit-jupiter-engine test - - - org.testng - testng - test - From 3142333b19baac57507f674e98c5efce781be777 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 15 Dec 2023 17:27:45 +0100 Subject: [PATCH 160/350] Migrate trino-record-decoder off TestNG assertions --- lib/trino-record-decoder/pom.xml | 6 ------ .../test/java/io/trino/decoder/util/DecoderTestUtil.java | 4 ++-- 2 files changed, 2 insertions(+), 8 deletions(-) diff --git a/lib/trino-record-decoder/pom.xml b/lib/trino-record-decoder/pom.xml index 3ce83c8ff4b5a..8c838108c22c1 100644 --- a/lib/trino-record-decoder/pom.xml +++ b/lib/trino-record-decoder/pom.xml @@ -165,12 +165,6 @@ junit-jupiter-engine test - - - org.testng - testng - test - diff --git a/lib/trino-record-decoder/src/test/java/io/trino/decoder/util/DecoderTestUtil.java b/lib/trino-record-decoder/src/test/java/io/trino/decoder/util/DecoderTestUtil.java index 31b3a13dedea2..61d5d190b74ae 100644 --- a/lib/trino-record-decoder/src/test/java/io/trino/decoder/util/DecoderTestUtil.java +++ b/lib/trino-record-decoder/src/test/java/io/trino/decoder/util/DecoderTestUtil.java @@ -22,7 +22,7 @@ import java.util.Map; import static org.assertj.core.api.Assertions.assertThat; -import static org.testng.Assert.assertEquals; +import static org.assertj.core.data.Offset.offset; public final class DecoderTestUtil { @@ -55,7 +55,7 @@ public static void checkValue(Map decod { FieldValueProvider provider = decodedRow.get(handle); assertThat(provider).isNotNull(); - assertEquals(provider.getDouble(), value, 0.0001); + assertThat(provider.getDouble()).isCloseTo(value, offset(0.0001)); } public static void checkValue(Map decodedRow, DecoderColumnHandle handle, boolean value) From a16597bb6d9cea8a45a471ce60cb61867d613694 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 15 Dec 2023 17:33:02 +0100 Subject: [PATCH 161/350] Migrate trino-prometheus off TestNG assertions --- plugin/trino-prometheus/pom.xml | 6 ------ .../trino/plugin/prometheus/TestPrometheusIntegration.java | 4 +--- .../io/trino/plugin/prometheus/TestPrometheusSplit.java | 4 ++-- 3 files changed, 3 insertions(+), 11 deletions(-) diff --git a/plugin/trino-prometheus/pom.xml b/plugin/trino-prometheus/pom.xml index 4e5a7ccd6a31a..2b33f26c168d5 100644 --- a/plugin/trino-prometheus/pom.xml +++ b/plugin/trino-prometheus/pom.xml @@ -257,11 +257,5 @@ testcontainers test - - - org.testng - testng - test - diff --git a/plugin/trino-prometheus/src/test/java/io/trino/plugin/prometheus/TestPrometheusIntegration.java b/plugin/trino-prometheus/src/test/java/io/trino/plugin/prometheus/TestPrometheusIntegration.java index 9e771bb78791e..0fae916c36e5f 100644 --- a/plugin/trino-prometheus/src/test/java/io/trino/plugin/prometheus/TestPrometheusIntegration.java +++ b/plugin/trino-prometheus/src/test/java/io/trino/plugin/prometheus/TestPrometheusIntegration.java @@ -30,7 +30,6 @@ import static java.util.concurrent.TimeUnit.DAYS; import static java.util.concurrent.TimeUnit.SECONDS; import static org.assertj.core.api.Assertions.assertThat; -import static org.testng.Assert.assertEquals; public class TestPrometheusIntegration extends AbstractTestQueryFramework @@ -129,7 +128,6 @@ public void testCorrectNumberOfSplitsCreated() (DynamicFilter) null, Constraint.alwaysTrue()); int numSplits = splits.getNextBatch(NUMBER_MORE_THAN_EXPECTED_NUMBER_SPLITS).getNow(null).getSplits().size(); - assertEquals(numSplits, config.getMaxQueryRangeDuration().getValue(TimeUnit.SECONDS) / config.getQueryChunkSizeDuration().getValue(TimeUnit.SECONDS), - 0.001); + assertThat((double) numSplits).isEqualTo(config.getMaxQueryRangeDuration().getValue(TimeUnit.SECONDS) / config.getQueryChunkSizeDuration().getValue(TimeUnit.SECONDS)); } } diff --git a/plugin/trino-prometheus/src/test/java/io/trino/plugin/prometheus/TestPrometheusSplit.java b/plugin/trino-prometheus/src/test/java/io/trino/plugin/prometheus/TestPrometheusSplit.java index dc156c3fe0561..4ac86b8a344e8 100644 --- a/plugin/trino-prometheus/src/test/java/io/trino/plugin/prometheus/TestPrometheusSplit.java +++ b/plugin/trino-prometheus/src/test/java/io/trino/plugin/prometheus/TestPrometheusSplit.java @@ -62,9 +62,9 @@ import static java.util.concurrent.TimeUnit.SECONDS; import static org.apache.http.client.utils.URLEncodedUtils.parse; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.data.Offset.offset; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; -import static org.testng.Assert.assertEquals; @TestInstance(PER_CLASS) @Execution(CONCURRENT) @@ -210,7 +210,7 @@ public void testQueryDividedIntoSplitsShouldHaveCorrectSpacingBetweenTimes() assertThat(paramsMap1).containsEntry("query", "up[1d]"); assertThat(paramsMap2).containsEntry("query", "up[1d]"); long diff = Double.valueOf(paramsMap2.get("time")).longValue() - Double.valueOf(paramsMap1.get("time")).longValue(); - assertEquals(config.getQueryChunkSizeDuration().getValue(TimeUnit.SECONDS), diff, 0.0001); + assertThat(config.getQueryChunkSizeDuration().getValue(TimeUnit.SECONDS)).isCloseTo(diff, offset(0.0001)); } @Test From 73b5ccf56fda5d67b9a8abe7a4bbf34ad72722b2 Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Thu, 14 Dec 2023 20:39:58 +0100 Subject: [PATCH 162/350] Update flyway to 10.3.0 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index aa18b92a631b2..305e8555c8f94 100644 --- a/pom.xml +++ b/pom.xml @@ -183,7 +183,7 @@ 1.21 1.0.8 2.23.0 - 10.1.0 + 10.3.0 1.43.3 1.4.2 5.14.0 From 16917084a71357447543f2851c717df1af8682ac Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Thu, 14 Dec 2023 20:41:46 +0100 Subject: [PATCH 163/350] Update RoaringBitmap to 1.0.1 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 305e8555c8f94..12ae4554ae6d3 100644 --- a/pom.xml +++ b/pom.xml @@ -2042,7 +2042,7 @@ org.roaringbitmap RoaringBitmap - 1.0.0 + 1.0.1 From 95910473d08449933d62d46560cc702e9fd48c09 Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Thu, 14 Dec 2023 20:43:29 +0100 Subject: [PATCH 164/350] Update redshift-jdbc42 to 2.1.0.24 --- plugin/trino-redshift/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/plugin/trino-redshift/pom.xml b/plugin/trino-redshift/pom.xml index 99c2967c09320..b9defa27b006a 100644 --- a/plugin/trino-redshift/pom.xml +++ b/plugin/trino-redshift/pom.xml @@ -21,7 +21,7 @@ com.amazon.redshift redshift-jdbc42 - 2.1.0.23 + 2.1.0.24 From 7b7910db1c26868747ff8f9e5ad6cd08277fb53a Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Thu, 14 Dec 2023 20:45:04 +0100 Subject: [PATCH 165/350] Update metrics-core to 4.2.23 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 12ae4554ae6d3..8b36b3ea5e179 100644 --- a/pom.xml +++ b/pom.xml @@ -825,7 +825,7 @@ io.dropwizard.metrics metrics-core - 4.2.22 + 4.2.23 From af062c7feb3e1e3173937c7210bd4d892d903b49 Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Thu, 14 Dec 2023 20:47:10 +0100 Subject: [PATCH 166/350] Update nimbus-jose-jwt 9.37.3 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 8b36b3ea5e179..4256024fac003 100644 --- a/pom.xml +++ b/pom.xml @@ -586,7 +586,7 @@ com.nimbusds nimbus-jose-jwt - 9.37.2 + 9.37.3 From 82bde82d90e4aeed481743b337b3720bc7524875 Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Thu, 14 Dec 2023 20:47:33 +0100 Subject: [PATCH 167/350] Update oauth2-oidc-sdk to 11.8 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 4256024fac003..7d314cebea367 100644 --- a/pom.xml +++ b/pom.xml @@ -592,7 +592,7 @@ com.nimbusds oauth2-oidc-sdk - 11.7 + 11.8 jdk11 From 7af5892fe56cd8fd374112eebdeaf10120a579f6 Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Thu, 14 Dec 2023 20:52:57 +0100 Subject: [PATCH 168/350] Update AWS SDK v1 to 1.12.618 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 7d314cebea367..0964b113b4e79 100644 --- a/pom.xml +++ b/pom.xml @@ -176,7 +176,7 @@ 4.13.1 14.0.1 1.11.3 - 1.12.610 + 1.12.618 4.17.0 7.5.1 87 From 6b32e9107bb414a66306d00f7dbb8d473f2af11d Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Thu, 14 Dec 2023 20:53:36 +0100 Subject: [PATCH 169/350] Update AWS SDK v2 to 2.21.45 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 0964b113b4e79..f1cbac2a40026 100644 --- a/pom.xml +++ b/pom.xml @@ -295,7 +295,7 @@ software.amazon.awssdk bom - 2.21.43 + 2.21.45 pom import From 81ea01196925a7c741b4745b93bfc3ef550fd2d1 Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Thu, 14 Dec 2023 20:55:11 +0100 Subject: [PATCH 170/350] Update plexus-xml to 4.0.3 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index f1cbac2a40026..2284eccc29681 100644 --- a/pom.xml +++ b/pom.xml @@ -1951,7 +1951,7 @@ org.codehaus.plexus plexus-xml - 4.0.2 + 4.0.3 From 40117ea499f9a34cf2a33b844f6130b0d95f0d77 Mon Sep 17 00:00:00 2001 From: Gaurav Sehgal Date: Tue, 19 Dec 2023 11:49:03 +0530 Subject: [PATCH 171/350] Use sf1 table for testMultipleWritersWithSkewedData --- .../java/io/trino/plugin/hive/BaseHiveConnectorTest.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java index a977ce6a996c1..18bc7a699f55b 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java @@ -4137,11 +4137,11 @@ public void testMultipleWriters() public void testMultipleWritersWithSkewedData() { try { - // We need to use large table (sf2) to see the effect. Otherwise, a single writer will write the entire + // We need to use large table (sf1) to see the effect. Otherwise, a single writer will write the entire // data before ScaledWriterScheduler is able to scale it to multiple machines. // Skewed table that will scale writers to multiple machines. - String selectSql = "SELECT t1.* FROM (SELECT *, case when orderkey >= 0 then 1 else orderkey end as join_key FROM tpch.sf2.orders) t1 " + - "INNER JOIN (SELECT orderkey FROM tpch.sf2.orders) t2 " + + String selectSql = "SELECT t1.* FROM (SELECT *, case when orderkey >= 0 then 1 else orderkey end as join_key FROM tpch.sf1.orders) t1 " + + "INNER JOIN (SELECT orderkey FROM tpch.sf1.orders) t2 " + "ON t1.join_key = t2.orderkey"; @Language("SQL") String createTableSql = "CREATE TABLE scale_writers_skewed WITH (format = 'PARQUET') AS " + selectSql; assertUpdate( @@ -4149,7 +4149,7 @@ public void testMultipleWritersWithSkewedData() .setSystemProperty("task_min_writer_count", "1") .setSystemProperty("scale_writers", "true") .setSystemProperty("task_scale_writers_enabled", "false") - .setSystemProperty("writer_scaling_min_data_processed", "1MB") + .setSystemProperty("writer_scaling_min_data_processed", "0.5MB") .setSystemProperty("join_distribution_type", "PARTITIONED") .build(), createTableSql, From a38ce40a390a9326383e32b8249f2a8803b4ead3 Mon Sep 17 00:00:00 2001 From: dahn Date: Thu, 7 Dec 2023 13:38:56 -0500 Subject: [PATCH 172/350] Add example for listagg with filter --- docs/src/main/sphinx/functions/aggregate.md | 30 ++++++++++++++++++--- 1 file changed, 27 insertions(+), 3 deletions(-) diff --git a/docs/src/main/sphinx/functions/aggregate.md b/docs/src/main/sphinx/functions/aggregate.md index fc59ce710c20c..20af7dd7462cd 100644 --- a/docs/src/main/sphinx/functions/aggregate.md +++ b/docs/src/main/sphinx/functions/aggregate.md @@ -213,7 +213,7 @@ of omitted non-null values in case that the length of the output of the function exceeds `1048576` bytes: ``` -SELECT LISTAGG(value, ',' ON OVERFLOW TRUNCATE '.....' WITH COUNT) WITHIN GROUP (ORDER BY value) +SELECT listagg(value, ',' ON OVERFLOW TRUNCATE '.....' WITH COUNT) WITHIN GROUP (ORDER BY value) FROM (VALUES 'a', 'b', 'c') t(value); ``` @@ -222,7 +222,7 @@ If not specified, the truncation filler string is by default `'...'`. This aggregation function can be also used in a scenario involving grouping: ``` -SELECT id, LISTAGG(value, ',') WITHIN GROUP (ORDER BY o) csv_value +SELECT id, listagg(value, ',') WITHIN GROUP (ORDER BY o) csv_value FROM (VALUES (100, 1, 'a'), (200, 3, 'c'), @@ -241,7 +241,31 @@ results in: 200 | b,c ``` -The current implementation of `LISTAGG` function does not support window frames. +This aggregation function can be also used with the `FILTER` keyword to specify +which rows are processed during the `listagg` aggregation: + +```sql +SELECT listagg(value, ',') + WITHIN GROUP (ORDER BY id) + FILTER (WHERE id % 2 = 0) csv_value +FROM (VALUES + (1, 'a'), + (2, 'b'), + (3, 'c'), + (4, 'd') +) t(id, value) +``` + +The example aggregates rows that have even-numbered `id`, and concatenates +`value` to a comma-separated string: + +``` + csv_value +----------- + b,d +``` + +The current implementation of `listagg` function does not support window frames. ::: :::{function} max(x) -> [same as input] From 8a3d1f95780b19ccce422fc87b48b9f2608f9a30 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jan=20Wa=C5=9B?= Date: Fri, 8 Dec 2023 17:41:08 +0100 Subject: [PATCH 173/350] Add example SQL routines for charts --- docs/src/main/sphinx/routines/examples.md | 149 ++++++++++++++++++++++ 1 file changed, 149 insertions(+) diff --git a/docs/src/main/sphinx/routines/examples.md b/docs/src/main/sphinx/routines/examples.md index 801307b40ff2a..3b6eab0162058 100644 --- a/docs/src/main/sphinx/routines/examples.md +++ b/docs/src/main/sphinx/routines/examples.md @@ -622,3 +622,152 @@ The preceding query produces the following output: 100000000000000000 | 88.8PB 1000000000000000000 | 888PB ``` + + +## Charts + +Trino already has a built-in `bar()` [color function](/functions/color), but +it's using ANSI escape codes to output colors, and thus is only usable for +displaying results in a terminal. The following example shows a similar +function, that only uses ASCII characters. + +```sql +FUNCTION ascii_bar(value DOUBLE) +RETURNS VARCHAR +BEGIN + DECLARE max_width DOUBLE DEFAULT 40.0; + RETURN array_join( + repeat('█', + greatest(0, CAST(floor(max_width * value) AS integer) - 1)), '') + || ARRAY[' ', '▏', '▎', '▍', '▌', '▋', '▊', '▉', '█'][cast((value % (cast(1 as double) / max_width)) * max_width * 8 + 1 as int)]; +END; +``` + +It can be used to visualize a value. + +```sql +WITH +data AS ( + SELECT + cast(s.num as double) / 100.0 AS x, + sin(cast(s.num as double) / 100.0) AS y + FROM table(sequence(start=>0, stop=>314, step=>10)) AS s(num) +) +SELECT + data.x, + round(data.y, 4) AS y, + ascii_bar(data.y) AS chart +FROM data +ORDER BY data.x; +``` + +The preceding query produces the following output: + +```text + x | y | chart +-----+--------+----------------------------------------- + 0.0 | 0.0 | + 0.1 | 0.0998 | ███ + 0.2 | 0.1987 | ███████ + 0.3 | 0.2955 | ██████████▉ + 0.4 | 0.3894 | ██████████████▋ + 0.5 | 0.4794 | ██████████████████▏ + 0.6 | 0.5646 | █████████████████████▋ + 0.7 | 0.6442 | ████████████████████████▊ + 0.8 | 0.7174 | ███████████████████████████▊ + 0.9 | 0.7833 | ██████████████████████████████▍ + 1.0 | 0.8415 | ████████████████████████████████▋ + 1.1 | 0.8912 | ██████████████████████████████████▋ + 1.2 | 0.932 | ████████████████████████████████████▎ + 1.3 | 0.9636 | █████████████████████████████████████▌ + 1.4 | 0.9854 | ██████████████████████████████████████▍ + 1.5 | 0.9975 | ██████████████████████████████████████▉ + 1.6 | 0.9996 | ███████████████████████████████████████ + 1.7 | 0.9917 | ██████████████████████████████████████▋ + 1.8 | 0.9738 | ██████████████████████████████████████ + 1.9 | 0.9463 | ████████████████████████████████████▉ + 2.0 | 0.9093 | ███████████████████████████████████▍ + 2.1 | 0.8632 | █████████████████████████████████▌ + 2.2 | 0.8085 | ███████████████████████████████▍ + 2.3 | 0.7457 | ████████████████████████████▉ + 2.4 | 0.6755 | ██████████████████████████ + 2.5 | 0.5985 | ███████████████████████ + 2.6 | 0.5155 | ███████████████████▋ + 2.7 | 0.4274 | ████████████████▏ + 2.8 | 0.335 | ████████████▍ + 2.9 | 0.2392 | ████████▋ + 3.0 | 0.1411 | ████▋ + 3.1 | 0.0416 | ▋ +``` + +It's also possible to draw more compacted charts. Following is a function +drawing vertical bars: + +```sql +FUNCTION vertical_bar(value DOUBLE) +RETURNS VARCHAR +RETURN ARRAY[' ', '▁', '▂', '▃', '▄', '▅', '▆', '▇', '█'][cast(value * 8 + 1 as int)]; +``` + +It can be used to draw a distribution of values, in a single column. + +```sql +WITH +measurements(sensor_id, recorded_at, value) AS ( + VALUES + ('A', date '2023-01-01', 5.0) + , ('A', date '2023-01-03', 7.0) + , ('A', date '2023-01-04', 15.0) + , ('A', date '2023-01-05', 14.0) + , ('A', date '2023-01-08', 10.0) + , ('A', date '2023-01-09', 1.0) + , ('A', date '2023-01-10', 7.0) + , ('A', date '2023-01-11', 8.0) + , ('B', date '2023-01-03', 2.0) + , ('B', date '2023-01-04', 3.0) + , ('B', date '2023-01-05', 2.5) + , ('B', date '2023-01-07', 2.75) + , ('B', date '2023-01-09', 4.0) + , ('B', date '2023-01-10', 1.5) + , ('B', date '2023-01-11', 1.0) +), +days AS ( + SELECT date_add('day', s.num, date '2023-01-01') AS day + -- table function arguments need to be constant but range could be calculated + -- using: SELECT date_diff('day', max(recorded_at), min(recorded_at)) FROM measurements + FROM table(sequence(start=>0, stop=>10)) AS s(num) +), +sensors(id) AS (VALUES ('A'), ('B')), +normalized AS ( + SELECT + sensors.id AS sensor_id, + days.day, + value, + value / max(value) OVER (PARTITION BY sensor_id) AS normalized + FROM days + CROSS JOIN sensors + LEFT JOIN measurements m ON day = recorded_at AND m.sensor_id = sensors.id +) +SELECT + sensor_id, + min(day) AS start, + max(day) AS stop, + count(value) AS num_values, + min(value) AS min_value, + max(value) AS max_value, + avg(value) AS avg_value, + array_join(array_agg(coalesce(vertical_bar(normalized), ' ') ORDER BY day), '') AS distribution +FROM normalized +WHERE sensor_id IS NOT NULL +GROUP BY sensor_id +ORDER BY sensor_id; +``` + +The preceding query produces the following output: + +```text + sensor_id | start | stop | num_values | min_value | max_value | avg_value | distribution +-----------+------------+------------+------------+-----------+-----------+-----------+-------------- + A | 2023-01-01 | 2023-01-11 | 8 | 1.00 | 15.00 | 8.38 | ▃ ▄█▇ ▅▁▄▄ + B | 2023-01-01 | 2023-01-11 | 7 | 1.00 | 4.00 | 2.39 | ▄▆▅ ▆ █▃▂ +``` From 69315d9df545c3b9cd034845398d85b4963abeb9 Mon Sep 17 00:00:00 2001 From: Yuya Ebihara Date: Tue, 19 Dec 2023 23:17:17 +0900 Subject: [PATCH 174/350] Add comment why disabling pushdown on MariaDB REAL type --- .../src/main/java/io/trino/plugin/mariadb/MariaDbClient.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/plugin/trino-mariadb/src/main/java/io/trino/plugin/mariadb/MariaDbClient.java b/plugin/trino-mariadb/src/main/java/io/trino/plugin/mariadb/MariaDbClient.java index 23dbb45e3310f..af3a6c0ecca88 100644 --- a/plugin/trino-mariadb/src/main/java/io/trino/plugin/mariadb/MariaDbClient.java +++ b/plugin/trino-mariadb/src/main/java/io/trino/plugin/mariadb/MariaDbClient.java @@ -339,6 +339,8 @@ public Optional toColumnMapping(ConnectorSession session, Connect case Types.BIGINT: return Optional.of(bigintColumnMapping()); case Types.REAL: + // Disable pushdown because floating-point values are approximate and not stored as exact values, + // attempts to treat them as exact in comparisons may lead to problems return Optional.of(ColumnMapping.longMapping( REAL, (resultSet, columnIndex) -> floatToRawIntBits(resultSet.getFloat(columnIndex)), From 4a36c3449e57caf7564a5e8bf2e9148b4d7b97e5 Mon Sep 17 00:00:00 2001 From: Manfred Moser Date: Tue, 19 Dec 2023 13:16:13 -0800 Subject: [PATCH 175/350] Expand catalog routine docs Add explicit mention what connector support storage so readers dont have to go looking through all connectors to find out. --- docs/src/main/sphinx/routines/introduction.md | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/docs/src/main/sphinx/routines/introduction.md b/docs/src/main/sphinx/routines/introduction.md index ab3377ba75a18..0174a3d2f2121 100644 --- a/docs/src/main/sphinx/routines/introduction.md +++ b/docs/src/main/sphinx/routines/introduction.md @@ -55,8 +55,13 @@ SELECT abs(-10); -- -20, not 10! ## Catalog routines You can store a routine in the context of a catalog, if the connector used in -the catalog supports routine storage. In this scenario, the following commands -can be used: +the catalog supports routine storage. The following connectors support catalog +routine storage: + +* [](/connector/hive) +* [](/connector/memory) + +In this scenario, the following commands can be used: * [](/sql/create-function) to create and store a routine. * [](/sql/drop-function) to remove a routine. From 33dd20a8c104d358f5b6d38e8a0405f8b0ced944 Mon Sep 17 00:00:00 2001 From: Marius Grama Date: Tue, 19 Dec 2023 05:48:19 +0100 Subject: [PATCH 176/350] Document filtering during aggregation for `listagg` --- docs/src/main/sphinx/functions/aggregate.md | 46 ++++++++++++--------- 1 file changed, 27 insertions(+), 19 deletions(-) diff --git a/docs/src/main/sphinx/functions/aggregate.md b/docs/src/main/sphinx/functions/aggregate.md index 20af7dd7462cd..109a25a1a5770 100644 --- a/docs/src/main/sphinx/functions/aggregate.md +++ b/docs/src/main/sphinx/functions/aggregate.md @@ -180,7 +180,7 @@ Synopsis: ``` LISTAGG( expression [, separator] [ON OVERFLOW overflow_behaviour]) - WITHIN GROUP (ORDER BY sort_item, [...]) + WITHIN GROUP (ORDER BY sort_item, [...]) [FILTER (WHERE condition)] ``` If `separator` is not specified, the empty string will be used as `separator`. @@ -241,28 +241,36 @@ results in: 200 | b,c ``` -This aggregation function can be also used with the `FILTER` keyword to specify -which rows are processed during the `listagg` aggregation: +This aggregation function supports +[filtering during aggregation](aggregate-function-filtering-during-aggregation) +for scenarios where the aggregation for the data not matching the filter +condition still needs to show up in the output: -```sql -SELECT listagg(value, ',') - WITHIN GROUP (ORDER BY id) - FILTER (WHERE id % 2 = 0) csv_value -FROM (VALUES - (1, 'a'), - (2, 'b'), - (3, 'c'), - (4, 'd') -) t(id, value) +``` +SELECT + country, + listagg(city, ',') + WITHIN GROUP (ORDER BY population DESC) + FILTER (WHERE population >= 10_000_000) megacities +FROM (VALUES + ('India', 'Bangalore', 13_700_000), + ('India', 'Chennai', 12_200_000), + ('India', 'Ranchi', 1_547_000), + ('Austria', 'Vienna', 1_897_000), + ('Poland', 'Warsaw', 1_765_000) +) t(country, city, population) +GROUP BY country +ORDER BY country; ``` -The example aggregates rows that have even-numbered `id`, and concatenates -`value` to a comma-separated string: +results in: -``` - csv_value ------------ - b,d +```text + country | megacities +---------+------------------- + Austria | NULL + India | Bangalore,Chennai + Poland | NULL ``` The current implementation of `listagg` function does not support window frames. From 6d08816502ab31a4060d78380608dd285fea44c0 Mon Sep 17 00:00:00 2001 From: Yuya Ebihara Date: Wed, 20 Dec 2023 15:36:04 +0900 Subject: [PATCH 177/350] Randomize identifiers in BaseSqlServerConnectorTest --- .../sqlserver/BaseSqlServerConnectorTest.java | 113 ++++++++++-------- 1 file changed, 64 insertions(+), 49 deletions(-) diff --git a/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/BaseSqlServerConnectorTest.java b/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/BaseSqlServerConnectorTest.java index 1ed1ec95b8eea..779d10220df45 100644 --- a/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/BaseSqlServerConnectorTest.java +++ b/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/BaseSqlServerConnectorTest.java @@ -24,6 +24,7 @@ import io.trino.sql.planner.plan.FilterNode; import io.trino.testing.TestingConnectorBehavior; import io.trino.testing.sql.TestTable; +import io.trino.testing.sql.TestView; import org.junit.jupiter.api.Test; import java.util.List; @@ -111,10 +112,10 @@ protected Optional filterDataMappingSmokeTestData(DataMapp @Test public void testReadFromView() { - onRemoteDatabase().execute("CREATE VIEW test_view AS SELECT * FROM orders"); - assertThat(getQueryRunner().tableExists(getSession(), "test_view")).isTrue(); - assertQuery("SELECT orderkey FROM test_view", "SELECT orderkey FROM orders"); - onRemoteDatabase().execute("DROP VIEW IF EXISTS test_view"); + try (TestView view = new TestView(onRemoteDatabase(), "test_view", "SELECT * FROM orders")) { + assertThat(getQueryRunner().tableExists(getSession(), view.getName())).isTrue(); + assertQuery("SELECT orderkey FROM " + view.getName(), "SELECT orderkey FROM orders"); + } } @Override @@ -459,31 +460,40 @@ private void testCreateWithDataCompression(DataCompression dataCompression) @Test public void testShowCreateForPartitionedTablesWithDataCompression() { - onRemoteDatabase().execute("CREATE PARTITION FUNCTION pfSales (DATE)\n" + - "AS RANGE LEFT FOR VALUES \n" + - "('2013-01-01', '2014-01-01', '2015-01-01')"); - onRemoteDatabase().execute("CREATE PARTITION SCHEME psSales\n" + - "AS PARTITION pfSales \n" + - "ALL TO ([PRIMARY])"); - onRemoteDatabase().execute("CREATE TABLE partitionedsales (\n" + - " SalesDate DATE,\n" + - " Quantity INT\n" + - ") ON psSales(SalesDate) WITH (DATA_COMPRESSION = PAGE)"); - assertThat((String) computeActual("SHOW CREATE TABLE partitionedsales").getOnlyValue()) - .matches("CREATE TABLE \\w+\\.\\w+\\.partitionedsales \\Q(\n" + - " salesdate date,\n" + - " quantity integer\n" + - ")"); - assertUpdate("DROP TABLE partitionedSales"); - onRemoteDatabase().execute("DROP PARTITION SCHEME psSales"); - onRemoteDatabase().execute("DROP PARTITION FUNCTION pfSales"); + String partitionFunction = "pfSales" + randomNameSuffix(); + String partitionScheme = "psSales" + randomNameSuffix(); + String tableName = "partitionedsales" + randomNameSuffix(); + + try { + onRemoteDatabase().execute("CREATE PARTITION FUNCTION " + partitionFunction + " (DATE)\n" + + "AS RANGE LEFT FOR VALUES \n" + + "('2013-01-01', '2014-01-01', '2015-01-01')"); + onRemoteDatabase().execute("CREATE PARTITION SCHEME " + partitionScheme + "\n" + + "AS PARTITION " + partitionFunction + " \n" + + "ALL TO ([PRIMARY])"); + onRemoteDatabase().execute("CREATE TABLE " + tableName + " (\n" + + " SalesDate DATE,\n" + + " Quantity INT\n" + + ") ON " + partitionScheme + "(SalesDate) WITH (DATA_COMPRESSION = PAGE)"); + assertThat((String) computeActual("SHOW CREATE TABLE " + tableName).getOnlyValue()) + .matches("CREATE TABLE \\w+\\.\\w+\\." + tableName + " \\Q(\n" + + " salesdate date,\n" + + " quantity integer\n" + + ")"); + } + finally { + assertUpdate("DROP TABLE IF EXISTS " + tableName); + onRemoteDatabase().execute("DROP PARTITION SCHEME " + partitionScheme); + onRemoteDatabase().execute("DROP PARTITION FUNCTION " + partitionFunction); + } } @Test public void testShowCreateForIndexedAndCompressedTable() { // SHOW CREATE doesn't expose data compression for Indexed tables - onRemoteDatabase().execute("CREATE TABLE test_show_indexed_table (\n" + + String tableName = "test_show_indexed_table" + randomNameSuffix(); + onRemoteDatabase().execute("CREATE TABLE " + tableName + " (\n" + " key1 BIGINT NOT NULL,\n" + " key2 BIGINT NOT NULL,\n" + " key3 BIGINT NOT NULL,\n" + @@ -493,23 +503,26 @@ public void testShowCreateForIndexedAndCompressedTable() " CONSTRAINT IX_IndexedTable UNIQUE (key2, key3),\n" + " INDEX IX_MyTable4 NONCLUSTERED (key4, key5))\n" + " WITH (DATA_COMPRESSION = PAGE)"); - - assertThat((String) computeActual("SHOW CREATE TABLE test_show_indexed_table").getOnlyValue()) - .isEqualTo("CREATE TABLE sqlserver.dbo.test_show_indexed_table (\n" + - " key1 bigint NOT NULL,\n" + - " key2 bigint NOT NULL,\n" + - " key3 bigint NOT NULL,\n" + - " key4 bigint NOT NULL,\n" + - " key5 bigint NOT NULL\n" + - ")"); - - assertUpdate("DROP TABLE test_show_indexed_table"); + try { + assertThat((String) computeActual("SHOW CREATE TABLE " + tableName).getOnlyValue()) + .isEqualTo("CREATE TABLE sqlserver.dbo." + tableName + " (\n" + + " key1 bigint NOT NULL,\n" + + " key2 bigint NOT NULL,\n" + + " key3 bigint NOT NULL,\n" + + " key4 bigint NOT NULL,\n" + + " key5 bigint NOT NULL\n" + + ")"); + } + finally { + assertUpdate("DROP TABLE " + tableName); + } } @Test public void testShowCreateForUniqueConstraintCompressedTable() { - onRemoteDatabase().execute("CREATE TABLE test_show_unique_constraint_table (\n" + + String tableName = "test_show_unique_constraint_table" + randomNameSuffix(); + onRemoteDatabase().execute("CREATE TABLE " + tableName + " (\n" + " key1 BIGINT NOT NULL,\n" + " key2 BIGINT NOT NULL,\n" + " key3 BIGINT NOT NULL,\n" + @@ -518,20 +531,22 @@ public void testShowCreateForUniqueConstraintCompressedTable() " UNIQUE (key1, key4),\n" + " UNIQUE (key2, key3))\n" + " WITH (DATA_COMPRESSION = PAGE)"); - - assertThat((String) computeActual("SHOW CREATE TABLE test_show_unique_constraint_table").getOnlyValue()) - .isEqualTo("CREATE TABLE sqlserver.dbo.test_show_unique_constraint_table (\n" + - " key1 bigint NOT NULL,\n" + - " key2 bigint NOT NULL,\n" + - " key3 bigint NOT NULL,\n" + - " key4 bigint NOT NULL,\n" + - " key5 bigint NOT NULL\n" + - ")\n" + - "WITH (\n" + - " data_compression = 'PAGE'\n" + - ")"); - - assertUpdate("DROP TABLE test_show_unique_constraint_table"); + try { + assertThat((String) computeActual("SHOW CREATE TABLE " + tableName).getOnlyValue()) + .isEqualTo("CREATE TABLE sqlserver.dbo." + tableName + " (\n" + + " key1 bigint NOT NULL,\n" + + " key2 bigint NOT NULL,\n" + + " key3 bigint NOT NULL,\n" + + " key4 bigint NOT NULL,\n" + + " key5 bigint NOT NULL\n" + + ")\n" + + "WITH (\n" + + " data_compression = 'PAGE'\n" + + ")"); + } + finally { + assertUpdate("DROP TABLE " + tableName); + } } @Test From cf77edb4b847e196d0fd13a6e2eccecf09ebc8ba Mon Sep 17 00:00:00 2001 From: Athul T R Date: Wed, 20 Dec 2023 22:50:18 +0530 Subject: [PATCH 178/350] Fail with proper error on overflow in from_unixtime The exception is due to the value user entered hence should be reported as a TrinoException rather than a GENERIC_INTERNAL_ERROR --- .../operator/scalar/DateTimeFunctions.java | 30 +++++++++++++++---- .../scalar/TestDateTimeFunctions.java | 21 +++++++++++++ 2 files changed, 46 insertions(+), 5 deletions(-) diff --git a/core/trino-main/src/main/java/io/trino/operator/scalar/DateTimeFunctions.java b/core/trino-main/src/main/java/io/trino/operator/scalar/DateTimeFunctions.java index 34708653f848b..37df8394fb58c 100644 --- a/core/trino-main/src/main/java/io/trino/operator/scalar/DateTimeFunctions.java +++ b/core/trino-main/src/main/java/io/trino/operator/scalar/DateTimeFunctions.java @@ -127,7 +127,12 @@ public static Slice currentTimeZone(ConnectorSession session) public static long fromUnixTime(ConnectorSession session, @SqlType(StandardTypes.DOUBLE) double unixTime) { // TODO (https://github.com/trinodb/trino/issues/5781) - return packDateTimeWithZone(Math.round(unixTime * 1000), session.getTimeZoneKey()); + try { + return packDateTimeWithZone(Math.round(unixTime * 1000), session.getTimeZoneKey()); + } + catch (IllegalArgumentException e) { + throw new TrinoException(INVALID_FUNCTION_ARGUMENT, e); + } } @ScalarFunction("from_unixtime") @@ -137,11 +142,11 @@ public static long fromUnixTime(@SqlType(StandardTypes.DOUBLE) double unixTime, TimeZoneKey timeZoneKey; try { timeZoneKey = getTimeZoneKeyForOffset(toIntExact(hoursOffset * 60 + minutesOffset)); + return packDateTimeWithZone(Math.round(unixTime * 1000), timeZoneKey); } catch (IllegalArgumentException e) { throw new TrinoException(INVALID_FUNCTION_ARGUMENT, e); } - return packDateTimeWithZone(Math.round(unixTime * 1000), timeZoneKey); } @ScalarFunction("from_unixtime") @@ -149,7 +154,12 @@ public static long fromUnixTime(@SqlType(StandardTypes.DOUBLE) double unixTime, @SqlType("timestamp(3) with time zone") public static long fromUnixTime(@SqlType(StandardTypes.DOUBLE) double unixTime, @SqlType("varchar(x)") Slice zoneId) { - return packDateTimeWithZone(Math.round(unixTime * 1000), zoneId.toStringUtf8()); + try { + return packDateTimeWithZone(Math.round(unixTime * 1000), zoneId.toStringUtf8()); + } + catch (IllegalArgumentException e) { + throw new TrinoException(INVALID_FUNCTION_ARGUMENT, e); + } } @ScalarFunction("from_unixtime_nanos") @@ -172,7 +182,12 @@ public static LongTimestampWithTimeZone fromLong(@LiteralParameter("s") long sca epochSeconds -= 1; picosOfSecond += PICOSECONDS_PER_SECOND; } - return DateTimes.longTimestampWithTimeZone(epochSeconds, picosOfSecond, session.getTimeZoneKey().getZoneId()); + try { + return DateTimes.longTimestampWithTimeZone(epochSeconds, picosOfSecond, session.getTimeZoneKey().getZoneId()); + } + catch (ArithmeticException e) { + throw new TrinoException(INVALID_FUNCTION_ARGUMENT, e); + } } @LiteralParameters({"p", "s"}) @@ -216,7 +231,12 @@ public static long fromISO8601Timestamp(ConnectorSession session, @SqlType("varc DateTimeFormatter formatter = ISODateTimeFormat.dateTimeParser() .withChronology(getChronology(session.getTimeZoneKey())) .withOffsetParsed(); - return packDateTimeWithZone(parseDateTimeHelper(formatter, iso8601DateTime.toStringUtf8())); + try { + return packDateTimeWithZone(parseDateTimeHelper(formatter, iso8601DateTime.toStringUtf8())); + } + catch (IllegalArgumentException e) { + throw new TrinoException(INVALID_FUNCTION_ARGUMENT, e); + } } @ScalarFunction("from_iso8601_timestamp_nanos") diff --git a/core/trino-main/src/test/java/io/trino/operator/scalar/TestDateTimeFunctions.java b/core/trino-main/src/test/java/io/trino/operator/scalar/TestDateTimeFunctions.java index 0e7fd5af99718..9af8140ed98e5 100644 --- a/core/trino-main/src/test/java/io/trino/operator/scalar/TestDateTimeFunctions.java +++ b/core/trino-main/src/test/java/io/trino/operator/scalar/TestDateTimeFunctions.java @@ -130,6 +130,10 @@ public void testFromUnixTime() assertThat(assertions.function("from_unixtime", "980172245.888")) .matches("TIMESTAMP '2001-01-22 03:04:05.888 Pacific/Apia'"); + + assertTrinoExceptionThrownBy(assertions.function("from_unixtime", "123456789123456789")::evaluate) + .hasErrorCode(INVALID_FUNCTION_ARGUMENT) + .hasMessage("Millis overflow: 9223372036854775807"); } @Test @@ -197,6 +201,10 @@ public void testFromUnixTimeNanos() assertThat(assertions.function("from_unixtime_nanos", "DECIMAL '-12345678900123456789.500'")) .matches("TIMESTAMP '1578-10-13 17:18:03.876543210 Pacific/Apia'"); + + assertTrinoExceptionThrownBy(assertions.function("from_unixtime_nanos", "DECIMAL '123456789123456789000000000'")::evaluate) + .hasErrorCode(INVALID_FUNCTION_ARGUMENT) + .hasMessage("long overflow"); } @Test @@ -214,6 +222,11 @@ public void testFromUnixTimeWithOffset() assertTrinoExceptionThrownBy(assertions.function("from_unixtime", "0", "-100", "100")::evaluate) .hasErrorCode(INVALID_FUNCTION_ARGUMENT); + + // test millisecond overflow + assertTrinoExceptionThrownBy(assertions.function("from_unixtime", "123456789123456789", "1", "1")::evaluate) + .hasErrorCode(INVALID_FUNCTION_ARGUMENT) + .hasMessage("Millis overflow: 9223372036854775807"); } @Test @@ -236,6 +249,10 @@ public void testFromUnixTimeWithTimeZone() assertThat(assertions.function("from_unixtime", "7200", "'America/Los_Angeles'")) .matches("TIMESTAMP '1969-12-31 18:00:00.000 America/Los_Angeles'"); + + assertTrinoExceptionThrownBy(assertions.function("from_unixtime", "123456789123456789", "'Asia/Kolkata'")::evaluate) + .hasErrorCode(INVALID_FUNCTION_ARGUMENT) + .hasMessage("Millis overflow: 9223372036854775807"); } @Test @@ -262,6 +279,10 @@ public void testFromISO8601() assertThat(assertions.function("from_iso8601_date", "'2001-08-22'")) .matches("DATE '2001-08-22'"); + + assertTrinoExceptionThrownBy(assertions.function("from_iso8601_timestamp", "'115023-03-21T10:45:30.00Z'")::evaluate) + .hasErrorCode(INVALID_FUNCTION_ARGUMENT) + .hasMessage("Millis overflow: 3567614928330000"); } @Test From f7369f674384b7cde8b5eb97931d679674930c2f Mon Sep 17 00:00:00 2001 From: James Petty Date: Wed, 20 Dec 2023 13:07:38 -0500 Subject: [PATCH 179/350] Fix page partitioner output bytes on release handling --- .../operator/output/PagePartitioner.java | 13 ++++++--- .../operator/output/TestPagePartitioner.java | 27 +++++++++++++++++++ 2 files changed, 36 insertions(+), 4 deletions(-) diff --git a/core/trino-main/src/main/java/io/trino/operator/output/PagePartitioner.java b/core/trino-main/src/main/java/io/trino/operator/output/PagePartitioner.java index 86fd777a98970..a9446d3aba047 100644 --- a/core/trino-main/src/main/java/io/trino/operator/output/PagePartitioner.java +++ b/core/trino-main/src/main/java/io/trino/operator/output/PagePartitioner.java @@ -169,11 +169,16 @@ private long adjustEagerlyReportedBytesWithBufferedBytesOnRelease(long bufferedB // adjust the amount to eagerly report as output by the amount already eagerly reported if the new value // is larger, since this indicates that no data was flushed and only the delta between the two values should // be reported eagerly - if (outputSizeReportedBeforeRelease > 0 && bufferedBytesOnRelease >= outputSizeReportedBeforeRelease) { - bufferedBytesOnRelease -= outputSizeReportedBeforeRelease; - outputSizeReportedBeforeRelease += bufferedBytesOnRelease; + if (bufferedBytesOnRelease > outputSizeReportedBeforeRelease) { + long additionalBufferedBytes = bufferedBytesOnRelease - outputSizeReportedBeforeRelease; + outputSizeReportedBeforeRelease = bufferedBytesOnRelease; + return additionalBufferedBytes; + } + else { + // buffered size is unchanged or reduced (as a result of flushing) since last release, so + // do not report any additional bytes as output eagerly + return 0; } - return bufferedBytesOnRelease; } /** diff --git a/core/trino-main/src/test/java/io/trino/operator/output/TestPagePartitioner.java b/core/trino-main/src/test/java/io/trino/operator/output/TestPagePartitioner.java index c16c6c200c7bb..898f5802e0f6b 100644 --- a/core/trino-main/src/test/java/io/trino/operator/output/TestPagePartitioner.java +++ b/core/trino-main/src/test/java/io/trino/operator/output/TestPagePartitioner.java @@ -523,6 +523,33 @@ public void testOutputWithMixedRowWiseAndColumnarPartitioning() testOutputEqualsInput(IPADDRESS, PartitioningMode.ROW_WISE, PartitioningMode.COLUMNAR); } + @Test + public void testOutputBytesWhenReused() + { + TestOutputBuffer outputBuffer = new TestOutputBuffer(); + PagePartitioner pagePartitioner = pagePartitioner(outputBuffer, BIGINT).build(); + OperatorContext operatorContext = operatorContext(); + + Page page = new Page(createLongsBlock(1, 1, 1, 1, 1, 1)); + + pagePartitioner.partitionPage(page, operatorContext); + assertThat(operatorContext.getOutputDataSize().getTotalCount()).isEqualTo(0); + pagePartitioner.prepareForRelease(operatorContext); + assertThat(operatorContext.getOutputDataSize().getTotalCount()).isEqualTo(page.getSizeInBytes()); + // release again with no additional input, size should not change + pagePartitioner.prepareForRelease(operatorContext); + assertThat(operatorContext.getOutputDataSize().getTotalCount()).isEqualTo(page.getSizeInBytes()); + + pagePartitioner.partitionPage(page, operatorContext); + pagePartitioner.prepareForRelease(operatorContext); + assertThat(operatorContext.getOutputDataSize().getTotalCount()).isEqualTo(page.getSizeInBytes() * 2); + + pagePartitioner.close(); + List output = outputBuffer.getEnqueued(); + // only a single page was flushed after the partitioner is closed, all output bytes were reported eagerly on release + assertThat(output.size()).isEqualTo(1); + } + @Test public void testMemoryReleased() { From 3a8faca1ca0d22d3abded3611df50f604447a08d Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 20 Dec 2023 17:10:58 +0100 Subject: [PATCH 180/350] Inline pom property for test profiles When defining test profiles, we don't use properties. --- lib/trino-hdfs/pom.xml | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/lib/trino-hdfs/pom.xml b/lib/trino-hdfs/pom.xml index 02167d3ce2608..820fb5c8372dd 100644 --- a/lib/trino-hdfs/pom.xml +++ b/lib/trino-hdfs/pom.xml @@ -13,9 +13,6 @@ ${project.parent.basedir} - - **/TestFileSystemCache.java @@ -295,7 +292,7 @@ maven-surefire-plugin - ${isolatedJvmTests} + **/TestFileSystemCache.java **/TestTrinoS3FileSystemAwsS3.java @@ -313,7 +310,7 @@ maven-surefire-plugin - ${isolatedJvmTests} + **/TestFileSystemCache.java false 1 From 4829f69019029c521c8454161dfe278dc267a662 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 20 Dec 2023 14:56:22 +0100 Subject: [PATCH 181/350] Make createExclusive atomic `GcsTransactionLogSynchronizer` requires that the new transaction log file is created exclusively and atomically. Otherwise concurrent readers of a table may see a file partially written (or not written yet, just empty) and fail. This commit: - fixes the GCS native filesystem implementation so that it's atomic - changes the method signature to indicate atomic creation and remove default not atomic implementation. - makes it clear in-memory buffering occurs (previously it was implicitly done in `HdfsOutputFile` which could be considered surprising) - in `AbstractTestTrinoFileSystem` decouples "is create() exclusive" and "supports createExclusive" behaviors. For example local filesystem has the former, GCS filesystem has both and S3 filesystem has none. --- .../filesystem/azure/AzureOutputFile.java | 7 -- lib/trino-filesystem-gcs/pom.xml | 5 + .../trino/filesystem/gcs/GcsOutputFile.java | 19 ++- .../gcs/AbstractTestGcsFileSystem.java | 6 + .../io/trino/filesystem/s3/S3OutputFile.java | 8 -- .../s3/AbstractTestS3FileSystem.java | 2 +- lib/trino-filesystem/pom.xml | 6 + .../io/trino/filesystem/TrinoOutputFile.java | 18 ++- .../filesystem/local/LocalOutputFile.java | 7 -- .../filesystem/memory/MemoryOutputFile.java | 4 +- .../filesystem/tracing/TracingOutputFile.java | 9 +- .../AbstractTestTrinoFileSystem.java | 112 +++++++++++++++--- .../filesystem/TrackingFileSystemFactory.java | 5 +- .../memory/TestMemoryFileSystem.java | 6 + .../trino/filesystem/hdfs/HdfsOutputFile.java | 16 +-- ...Stream.java => GcsAtomicOutputStream.java} | 4 +- .../hdfs/TestHdfsFileSystemS3Mock.java | 2 +- .../writer/GcsTransactionLogSynchronizer.java | 6 +- 18 files changed, 175 insertions(+), 67 deletions(-) rename lib/trino-hdfs/src/main/java/io/trino/hdfs/gcs/{GcsExclusiveOutputStream.java => GcsAtomicOutputStream.java} (93%) diff --git a/lib/trino-filesystem-azure/src/main/java/io/trino/filesystem/azure/AzureOutputFile.java b/lib/trino-filesystem-azure/src/main/java/io/trino/filesystem/azure/AzureOutputFile.java index 17840e85388a3..261601c197e71 100644 --- a/lib/trino-filesystem-azure/src/main/java/io/trino/filesystem/azure/AzureOutputFile.java +++ b/lib/trino-filesystem-azure/src/main/java/io/trino/filesystem/azure/AzureOutputFile.java @@ -71,13 +71,6 @@ public OutputStream createOrOverwrite(AggregatedMemoryContext memoryContext) return createOutputStream(memoryContext, true); } - @Override - public OutputStream createExclusive(AggregatedMemoryContext memoryContext) - throws IOException - { - return create(memoryContext); - } - private AzureOutputStream createOutputStream(AggregatedMemoryContext memoryContext, boolean overwrite) throws IOException { diff --git a/lib/trino-filesystem-gcs/pom.xml b/lib/trino-filesystem-gcs/pom.xml index 7e9505d9c9392..a5732aeffb7ee 100644 --- a/lib/trino-filesystem-gcs/pom.xml +++ b/lib/trino-filesystem-gcs/pom.xml @@ -98,6 +98,11 @@ configuration + + io.airlift + slice + + io.airlift units diff --git a/lib/trino-filesystem-gcs/src/main/java/io/trino/filesystem/gcs/GcsOutputFile.java b/lib/trino-filesystem-gcs/src/main/java/io/trino/filesystem/gcs/GcsOutputFile.java index 12344c9a0f98b..0131299daebda 100644 --- a/lib/trino-filesystem-gcs/src/main/java/io/trino/filesystem/gcs/GcsOutputFile.java +++ b/lib/trino-filesystem-gcs/src/main/java/io/trino/filesystem/gcs/GcsOutputFile.java @@ -18,6 +18,7 @@ import com.google.cloud.storage.BlobInfo; import com.google.cloud.storage.Storage; import com.google.cloud.storage.Storage.BlobTargetOption; +import io.airlift.slice.Slice; import io.trino.filesystem.Location; import io.trino.filesystem.TrinoOutputFile; import io.trino.memory.context.AggregatedMemoryContext; @@ -64,10 +65,24 @@ public OutputStream createOrOverwrite(AggregatedMemoryContext memoryContext) } @Override - public OutputStream createExclusive(AggregatedMemoryContext memoryContext) + public void createExclusive(Slice content, AggregatedMemoryContext memoryContext) throws IOException { - return create(memoryContext); + try { + if (getBlob(storage, location).isPresent()) { + throw new FileAlreadyExistsException("File %s already exists".formatted(location)); + } + storage.create( + BlobInfo.newBuilder(BlobId.of(location.bucket(), location.path())).build(), + content.getBytes(), + DOES_NOT_EXIST_TARGET_OPTION); + } + catch (FileAlreadyExistsException e) { + throw e; + } + catch (RuntimeException e) { + throw handleGcsException(e, "writing file", location); + } } private OutputStream createOutputStream(AggregatedMemoryContext memoryContext, boolean overwrite) diff --git a/lib/trino-filesystem-gcs/src/test/java/io/trino/filesystem/gcs/AbstractTestGcsFileSystem.java b/lib/trino-filesystem-gcs/src/test/java/io/trino/filesystem/gcs/AbstractTestGcsFileSystem.java index b830f893a9043..ec0ae9836f5b6 100644 --- a/lib/trino-filesystem-gcs/src/test/java/io/trino/filesystem/gcs/AbstractTestGcsFileSystem.java +++ b/lib/trino-filesystem-gcs/src/test/java/io/trino/filesystem/gcs/AbstractTestGcsFileSystem.java @@ -125,6 +125,12 @@ protected void verifyFileSystemIsEmpty() assertThat(storage.list(bucket).iterateAll()).isEmpty(); } + @Override + protected final boolean supportsCreateExclusive() + { + return true; + } + @Override protected final boolean supportsRenameFile() { diff --git a/lib/trino-filesystem-s3/src/main/java/io/trino/filesystem/s3/S3OutputFile.java b/lib/trino-filesystem-s3/src/main/java/io/trino/filesystem/s3/S3OutputFile.java index a388bcb6d2874..5a6bf934c1b86 100644 --- a/lib/trino-filesystem-s3/src/main/java/io/trino/filesystem/s3/S3OutputFile.java +++ b/lib/trino-filesystem-s3/src/main/java/io/trino/filesystem/s3/S3OutputFile.java @@ -18,7 +18,6 @@ import io.trino.memory.context.AggregatedMemoryContext; import software.amazon.awssdk.services.s3.S3Client; -import java.io.IOException; import java.io.OutputStream; import static java.util.Objects.requireNonNull; @@ -51,13 +50,6 @@ public OutputStream createOrOverwrite(AggregatedMemoryContext memoryContext) return new S3OutputStream(memoryContext, client, context, location); } - @Override - public OutputStream createExclusive(AggregatedMemoryContext memoryContext) - throws IOException - { - throw new IOException("S3 does not support exclusive create"); - } - @Override public Location location() { diff --git a/lib/trino-filesystem-s3/src/test/java/io/trino/filesystem/s3/AbstractTestS3FileSystem.java b/lib/trino-filesystem-s3/src/test/java/io/trino/filesystem/s3/AbstractTestS3FileSystem.java index 9f68d3b5efe84..56b17fae8d825 100644 --- a/lib/trino-filesystem-s3/src/test/java/io/trino/filesystem/s3/AbstractTestS3FileSystem.java +++ b/lib/trino-filesystem-s3/src/test/java/io/trino/filesystem/s3/AbstractTestS3FileSystem.java @@ -82,7 +82,7 @@ protected final Location getRootLocation() } @Override - protected final boolean supportsCreateExclusive() + protected boolean isCreateExclusive() { return false; } diff --git a/lib/trino-filesystem/pom.xml b/lib/trino-filesystem/pom.xml index 997ffdd7b110a..9b01a4bb67280 100644 --- a/lib/trino-filesystem/pom.xml +++ b/lib/trino-filesystem/pom.xml @@ -58,6 +58,12 @@ provided + + io.airlift + concurrent + test + + io.airlift junit-extensions diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/TrinoOutputFile.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/TrinoOutputFile.java index eea75c0291fb9..2f20f3d1fba4f 100644 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/TrinoOutputFile.java +++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/TrinoOutputFile.java @@ -14,6 +14,7 @@ package io.trino.filesystem; +import io.airlift.slice.Slice; import io.trino.memory.context.AggregatedMemoryContext; import java.io.IOException; @@ -35,10 +36,13 @@ default OutputStream createOrOverwrite() return createOrOverwrite(newSimpleAggregatedMemoryContext()); } - default OutputStream createExclusive() + /** + * Create file exclusively and atomically with specified contents. + */ + default void createExclusive(Slice content) throws IOException { - return createExclusive(newSimpleAggregatedMemoryContext()); + createExclusive(content, newSimpleAggregatedMemoryContext()); } OutputStream create(AggregatedMemoryContext memoryContext) @@ -47,8 +51,14 @@ OutputStream create(AggregatedMemoryContext memoryContext) OutputStream createOrOverwrite(AggregatedMemoryContext memoryContext) throws IOException; - OutputStream createExclusive(AggregatedMemoryContext memoryContext) - throws IOException; + /** + * Create file exclusively and atomically with specified contents. + */ + default void createExclusive(Slice content, AggregatedMemoryContext memoryContext) + throws IOException + { + throw new UnsupportedOperationException("createExclusive not supported by " + getClass()); + } Location location(); } diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalOutputFile.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalOutputFile.java index 2af7daf0602da..87deb5fd11fff 100644 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalOutputFile.java +++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalOutputFile.java @@ -72,13 +72,6 @@ public OutputStream createOrOverwrite(AggregatedMemoryContext memoryContext) } } - @Override - public OutputStream createExclusive(AggregatedMemoryContext memoryContext) - throws IOException - { - return create(memoryContext); - } - @Override public Location location() { diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryOutputFile.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryOutputFile.java index b937377f3ca45..7d4373242e165 100644 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryOutputFile.java +++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryOutputFile.java @@ -64,10 +64,10 @@ public OutputStream createOrOverwrite(AggregatedMemoryContext memoryContext) } @Override - public OutputStream createExclusive(AggregatedMemoryContext memoryContext) + public void createExclusive(Slice content, AggregatedMemoryContext memoryContext) throws IOException { - return create(memoryContext); + outputBlob.createBlob(content); } @Override diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/tracing/TracingOutputFile.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/tracing/TracingOutputFile.java index de0123b21b2a9..89125f49d102d 100644 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/tracing/TracingOutputFile.java +++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/tracing/TracingOutputFile.java @@ -13,6 +13,7 @@ */ package io.trino.filesystem.tracing; +import io.airlift.slice.Slice; import io.opentelemetry.api.trace.Span; import io.opentelemetry.api.trace.Tracer; import io.trino.filesystem.Location; @@ -58,13 +59,13 @@ public OutputStream createOrOverwrite() } @Override - public OutputStream createExclusive() + public void createExclusive(Slice content) throws IOException { Span span = tracer.spanBuilder("OutputFile.createExclusive") .setAttribute(FileSystemAttributes.FILE_LOCATION, toString()) .startSpan(); - return withTracing(span, () -> delegate.createExclusive()); + withTracing(span, () -> delegate.createExclusive(content)); } @Override @@ -88,13 +89,13 @@ public OutputStream createOrOverwrite(AggregatedMemoryContext memoryContext) } @Override - public OutputStream createExclusive(AggregatedMemoryContext memoryContext) + public void createExclusive(Slice content, AggregatedMemoryContext memoryContext) throws IOException { Span span = tracer.spanBuilder("OutputFile.createExclusive") .setAttribute(FileSystemAttributes.FILE_LOCATION, toString()) .startSpan(); - return withTracing(span, () -> delegate.createExclusive(memoryContext)); + withTracing(span, () -> delegate.createExclusive(content, memoryContext)); } @Override diff --git a/lib/trino-filesystem/src/test/java/io/trino/filesystem/AbstractTestTrinoFileSystem.java b/lib/trino-filesystem/src/test/java/io/trino/filesystem/AbstractTestTrinoFileSystem.java index 4f18850ad0633..fa6ead06f44fe 100644 --- a/lib/trino-filesystem/src/test/java/io/trino/filesystem/AbstractTestTrinoFileSystem.java +++ b/lib/trino-filesystem/src/test/java/io/trino/filesystem/AbstractTestTrinoFileSystem.java @@ -17,7 +17,6 @@ import com.google.common.io.ByteStreams; import com.google.common.io.Closer; import io.airlift.slice.Slice; -import io.airlift.slice.Slices; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.TestInstance; @@ -40,10 +39,19 @@ import java.util.Optional; import java.util.Set; import java.util.UUID; - +import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicBoolean; + +import static io.airlift.slice.Slices.EMPTY_SLICE; +import static io.airlift.slice.Slices.wrappedBuffer; import static java.lang.Math.min; +import static java.nio.charset.StandardCharsets.US_ASCII; import static java.nio.charset.StandardCharsets.UTF_8; import static java.util.Objects.requireNonNull; +import static java.util.concurrent.TimeUnit.SECONDS; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; @@ -63,11 +71,22 @@ public abstract class AbstractTestTrinoFileSystem protected abstract void verifyFileSystemIsEmpty(); - protected boolean supportsCreateExclusive() + /** + * Specifies whether implementation {@link TrinoOutputFile#create()} is exclusive. + */ + protected boolean isCreateExclusive() { return true; } + /** + * Specifies whether implementation supports {@link TrinoOutputFile#createExclusive(Slice)}. + */ + protected boolean supportsCreateExclusive() + { + return false; + } + protected boolean supportsRenameFile() { return true; @@ -226,7 +245,7 @@ public void testInputFile() // write a 16 MB file try (OutputStream outputStream = tempBlob.outputFile().create()) { byte[] bytes = new byte[4]; - Slice slice = Slices.wrappedBuffer(bytes); + Slice slice = wrappedBuffer(bytes); for (int i = 0; i < 4 * MEGABYTE; i++) { slice.setInt(0, i); outputStream.write(bytes); @@ -239,7 +258,7 @@ public void testInputFile() try (TrinoInputStream inputStream = inputFile.newStream()) { byte[] bytes = new byte[4]; - Slice slice = Slices.wrappedBuffer(bytes); + Slice slice = wrappedBuffer(bytes); // read int at a time for (int intPosition = 0; intPosition < 4 * MEGABYTE; intPosition++) { @@ -393,14 +412,14 @@ public void testInputFile() try (TrinoInput trinoInput = inputFile.newInput()) { byte[] bytes = new byte[4 * 10]; - Slice slice = Slices.wrappedBuffer(bytes); + Slice slice = wrappedBuffer(bytes); // positioned read trinoInput.readFully(0, bytes, 0, bytes.length); for (int i = 0; i < 10; i++) { assertThat(slice.getInt(i * 4)).isEqualTo(i); } - assertThat(trinoInput.readFully(0, bytes.length)).isEqualTo(Slices.wrappedBuffer(bytes)); + assertThat(trinoInput.readFully(0, bytes.length)).isEqualTo(wrappedBuffer(bytes)); trinoInput.readFully(0, bytes, 2, bytes.length - 2); for (int i = 0; i < 9; i++) { @@ -411,7 +430,7 @@ public void testInputFile() for (int i = 0; i < 10; i++) { assertThat(slice.getInt(i * 4)).isEqualTo(i + MEGABYTE / 4); } - assertThat(trinoInput.readFully(MEGABYTE, bytes.length)).isEqualTo(Slices.wrappedBuffer(bytes)); + assertThat(trinoInput.readFully(MEGABYTE, bytes.length)).isEqualTo(wrappedBuffer(bytes)); assertThatThrownBy(() -> trinoInput.readFully(fileSize - bytes.length + 1, bytes, 0, bytes.length)) .isInstanceOf(IOException.class) .hasMessageContaining(tempBlob.location().toString()); @@ -423,7 +442,7 @@ public void testInputFile() assertThat(slice.getInt(i * 4)).isEqualTo(totalPositions - 10 + i); } - assertThat(trinoInput.readTail(bytes.length)).isEqualTo(Slices.wrappedBuffer(bytes)); + assertThat(trinoInput.readTail(bytes.length)).isEqualTo(wrappedBuffer(bytes)); trinoInput.readTail(bytes, 2, bytes.length - 2); for (int i = 0; i < 9; i++) { @@ -474,7 +493,7 @@ void testOutputFile() outputStream.write("initial".getBytes(UTF_8)); } - if (supportsCreateExclusive()) { + if (isCreateExclusive()) { // re-create without overwrite is an error assertThatThrownBy(outputFile::create) .isInstanceOf(FileAlreadyExistsException.class) @@ -484,9 +503,16 @@ void testOutputFile() assertThat(tempBlob.read()).isEqualTo("initial"); // re-create exclusive is an error - assertThatThrownBy(outputFile::createExclusive) - .isInstanceOf(FileAlreadyExistsException.class) - .hasMessageContaining(tempBlob.location().toString()); + if (supportsCreateExclusive()) { + assertThatThrownBy(() -> outputFile.createExclusive(EMPTY_SLICE)) + .isInstanceOf(FileAlreadyExistsException.class) + .hasMessageContaining(tempBlob.location().toString()); + } + else { + assertThatThrownBy(() -> outputFile.createExclusive(EMPTY_SLICE)) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessageStartingWith("createExclusive not supported"); + } // verify nothing changed assertThat(tempBlob.read()).isEqualTo("initial"); @@ -501,9 +527,16 @@ void testOutputFile() assertThat(tempBlob.read()).isEqualTo("replaced"); // create exclusive is an error - assertThatThrownBy(outputFile::createExclusive) - .isInstanceOf(IOException.class) - .hasMessageContaining("does not support exclusive create"); + if (supportsCreateExclusive()) { + assertThatThrownBy(() -> outputFile.createExclusive(EMPTY_SLICE)) + .isInstanceOf(FileAlreadyExistsException.class) + .hasMessageContaining(tempBlob.location().toString()); + } + else { + assertThatThrownBy(() -> outputFile.createExclusive(EMPTY_SLICE)) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessageStartingWith("createExclusive not supported"); + } } // overwrite file @@ -516,6 +549,53 @@ void testOutputFile() } } + @Test + void testCreateExclusiveIsAtomic() + throws Exception + { + if (!supportsCreateExclusive()) { + return; + } + + int timeoutSeconds = 20; + ExecutorService executor = Executors.newCachedThreadPool(io.airlift.concurrent.Threads.daemonThreadsNamed("testCreateExclusiveIsAtomic-%s")); + AtomicBoolean finishing = new AtomicBoolean(false); + try (TempBlob tempBlob = randomBlobLocation("outputFile")) { + TrinoFileSystem fileSystem = getFileSystem(); + Slice content = wrappedBuffer("a".repeat(MEGABYTE).getBytes(US_ASCII)); + + fileSystem.deleteFile(tempBlob.location()); + CyclicBarrier barrier = new CyclicBarrier(2); + Future write = executor.submit(() -> { + barrier.await(timeoutSeconds, SECONDS); + fileSystem.newOutputFile(tempBlob.location()).createExclusive(content); + return null; + }); + Future read = executor.submit(() -> { + TrinoInputFile inputFile = fileSystem.newInputFile(tempBlob.location()); + assertThat(inputFile.exists()).as("inputFile.exists()").isFalse(); + barrier.await(timeoutSeconds, SECONDS); + + while (!finishing.get()) { + try (TrinoInput input = inputFile.newInput()) { + return input.readFully(0, content.length()); + } + catch (FileNotFoundException expected) { + } + } + throw new RuntimeException("File not created"); + }); + + assertThat(read.get(timeoutSeconds, SECONDS)).as("read content").isEqualTo(content); + write.get(timeoutSeconds, SECONDS); + } + finally { + finishing.set(true); + executor.shutdownNow(); + assertThat(executor.awaitTermination(timeoutSeconds, SECONDS)).as("executor terminated").isTrue(); + } + } + @Test void testOutputStreamByteAtATime() throws IOException diff --git a/lib/trino-filesystem/src/test/java/io/trino/filesystem/TrackingFileSystemFactory.java b/lib/trino-filesystem/src/test/java/io/trino/filesystem/TrackingFileSystemFactory.java index c9f4bae6e02ce..195b3eede7576 100644 --- a/lib/trino-filesystem/src/test/java/io/trino/filesystem/TrackingFileSystemFactory.java +++ b/lib/trino-filesystem/src/test/java/io/trino/filesystem/TrackingFileSystemFactory.java @@ -14,6 +14,7 @@ package io.trino.filesystem; import com.google.common.collect.ImmutableMap; +import io.airlift.slice.Slice; import io.trino.memory.context.AggregatedMemoryContext; import io.trino.spi.security.ConnectorIdentity; @@ -306,11 +307,11 @@ public OutputStream createOrOverwrite(AggregatedMemoryContext memoryContext) } @Override - public OutputStream createExclusive(AggregatedMemoryContext memoryContext) + public void createExclusive(Slice content, AggregatedMemoryContext memoryContext) throws IOException { tracker.accept(OUTPUT_FILE_CREATE_EXCLUSIVE); - return delegate.createExclusive(memoryContext); + delegate.createExclusive(content, memoryContext); } @Override diff --git a/lib/trino-filesystem/src/test/java/io/trino/filesystem/memory/TestMemoryFileSystem.java b/lib/trino-filesystem/src/test/java/io/trino/filesystem/memory/TestMemoryFileSystem.java index a1014dc898097..b11aaeac7f62a 100644 --- a/lib/trino-filesystem/src/test/java/io/trino/filesystem/memory/TestMemoryFileSystem.java +++ b/lib/trino-filesystem/src/test/java/io/trino/filesystem/memory/TestMemoryFileSystem.java @@ -44,6 +44,12 @@ protected boolean isHierarchical() return false; } + @Override + protected boolean supportsCreateExclusive() + { + return true; + } + @Override protected TrinoFileSystem getFileSystem() { diff --git a/lib/trino-hdfs/src/main/java/io/trino/filesystem/hdfs/HdfsOutputFile.java b/lib/trino-hdfs/src/main/java/io/trino/filesystem/hdfs/HdfsOutputFile.java index d09b28baeb563..b913425283011 100644 --- a/lib/trino-hdfs/src/main/java/io/trino/filesystem/hdfs/HdfsOutputFile.java +++ b/lib/trino-hdfs/src/main/java/io/trino/filesystem/hdfs/HdfsOutputFile.java @@ -14,6 +14,7 @@ package io.trino.filesystem.hdfs; import com.google.cloud.hadoop.fs.gcs.GoogleHadoopFileSystem; +import io.airlift.slice.Slice; import io.airlift.stats.TimeStat; import io.trino.filesystem.Location; import io.trino.filesystem.TrinoOutputFile; @@ -22,8 +23,7 @@ import io.trino.hdfs.HdfsEnvironment; import io.trino.hdfs.MemoryAwareFileSystem; import io.trino.hdfs.authentication.GenericExceptionAction; -import io.trino.hdfs.gcs.GcsExclusiveOutputStream; -import io.trino.hdfs.s3.TrinoS3FileSystem; +import io.trino.hdfs.gcs.GcsAtomicOutputStream; import io.trino.memory.context.AggregatedMemoryContext; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; @@ -70,18 +70,18 @@ public OutputStream createOrOverwrite(AggregatedMemoryContext memoryContext) } @Override - public OutputStream createExclusive(AggregatedMemoryContext memoryContext) + public void createExclusive(Slice content, AggregatedMemoryContext memoryContext) throws IOException { Path file = hadoopPath(location); FileSystem fileSystem = getRawFileSystem(environment.getFileSystem(context, file)); - if (fileSystem instanceof TrinoS3FileSystem) { - throw new IOException("S3 does not support exclusive create"); - } if (fileSystem instanceof GoogleHadoopFileSystem) { - return new GcsExclusiveOutputStream(environment, context, file); + GcsAtomicOutputStream atomicOutputStream = new GcsAtomicOutputStream(environment, context, file); + atomicOutputStream.write(content.getBytes()); + atomicOutputStream.close(); + return; } - return create(memoryContext); + throw new UnsupportedOperationException("createExclusive not supported for " + fileSystem); } private OutputStream create(boolean overwrite, AggregatedMemoryContext memoryContext) diff --git a/lib/trino-hdfs/src/main/java/io/trino/hdfs/gcs/GcsExclusiveOutputStream.java b/lib/trino-hdfs/src/main/java/io/trino/hdfs/gcs/GcsAtomicOutputStream.java similarity index 93% rename from lib/trino-hdfs/src/main/java/io/trino/hdfs/gcs/GcsExclusiveOutputStream.java rename to lib/trino-hdfs/src/main/java/io/trino/hdfs/gcs/GcsAtomicOutputStream.java index b53e0b4e082da..3ee56648304f5 100644 --- a/lib/trino-hdfs/src/main/java/io/trino/hdfs/gcs/GcsExclusiveOutputStream.java +++ b/lib/trino-hdfs/src/main/java/io/trino/hdfs/gcs/GcsAtomicOutputStream.java @@ -24,14 +24,14 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; -public class GcsExclusiveOutputStream +public class GcsAtomicOutputStream extends ByteArrayOutputStream { private final Storage storage; private final Path path; private boolean closed; - public GcsExclusiveOutputStream(HdfsEnvironment environment, HdfsContext context, Path path) + public GcsAtomicOutputStream(HdfsEnvironment environment, HdfsContext context, Path path) { this.storage = environment.createGcsStorage(context, path); this.path = path; diff --git a/lib/trino-hdfs/src/test/java/io/trino/filesystem/hdfs/TestHdfsFileSystemS3Mock.java b/lib/trino-hdfs/src/test/java/io/trino/filesystem/hdfs/TestHdfsFileSystemS3Mock.java index f477a75e793ca..25fe9a824169c 100644 --- a/lib/trino-hdfs/src/test/java/io/trino/filesystem/hdfs/TestHdfsFileSystemS3Mock.java +++ b/lib/trino-hdfs/src/test/java/io/trino/filesystem/hdfs/TestHdfsFileSystemS3Mock.java @@ -109,7 +109,7 @@ protected Location getRootLocation() } @Override - protected final boolean supportsCreateExclusive() + protected boolean isCreateExclusive() { return false; } diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/writer/GcsTransactionLogSynchronizer.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/writer/GcsTransactionLogSynchronizer.java index 4c373e9de2237..e2a70cb542b04 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/writer/GcsTransactionLogSynchronizer.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/writer/GcsTransactionLogSynchronizer.java @@ -20,9 +20,9 @@ import io.trino.spi.connector.ConnectorSession; import java.io.IOException; -import java.io.OutputStream; import java.io.UncheckedIOException; +import static io.airlift.slice.Slices.wrappedBuffer; import static java.util.Objects.requireNonNull; public class GcsTransactionLogSynchronizer @@ -42,8 +42,8 @@ public GcsTransactionLogSynchronizer(TrinoFileSystemFactory fileSystemFactory) public void write(ConnectorSession session, String clusterId, Location newLogEntryPath, byte[] entryContents) { TrinoFileSystem fileSystem = fileSystemFactory.create(session); - try (OutputStream outputStream = fileSystem.newOutputFile(newLogEntryPath).createExclusive()) { - outputStream.write(entryContents); + try { + fileSystem.newOutputFile(newLogEntryPath).createExclusive(wrappedBuffer(entryContents)); } catch (IOException e) { throw new UncheckedIOException(e); From 699fe621834e477eed60cad5a4640d83085a3b23 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 20 Dec 2023 22:23:34 +0100 Subject: [PATCH 182/350] Remove redundant classloader wrapper from exclude_columns Engine-provided functions do not need `ClassLoaderSafeConnectorTableFunction` wrapper. --- .../system/SystemConnectorModule.java | 4 +- .../trino/metadata/GlobalFunctionCatalog.java | 4 +- .../trino/operator/table/ExcludeColumns.java | 174 ------------------ .../table/ExcludeColumnsFunction.java | 161 ++++++++++++++++ .../io/trino/testing/LocalQueryRunner.java | 2 +- 5 files changed, 166 insertions(+), 179 deletions(-) delete mode 100644 core/trino-main/src/main/java/io/trino/operator/table/ExcludeColumns.java create mode 100644 core/trino-main/src/main/java/io/trino/operator/table/ExcludeColumnsFunction.java diff --git a/core/trino-main/src/main/java/io/trino/connector/system/SystemConnectorModule.java b/core/trino-main/src/main/java/io/trino/connector/system/SystemConnectorModule.java index 306fb4347c39d..8500f3e645b46 100644 --- a/core/trino-main/src/main/java/io/trino/connector/system/SystemConnectorModule.java +++ b/core/trino-main/src/main/java/io/trino/connector/system/SystemConnectorModule.java @@ -30,7 +30,7 @@ import io.trino.connector.system.jdbc.TableTypeJdbcTable; import io.trino.connector.system.jdbc.TypesJdbcTable; import io.trino.connector.system.jdbc.UdtJdbcTable; -import io.trino.operator.table.ExcludeColumns; +import io.trino.operator.table.ExcludeColumnsFunction; import io.trino.operator.table.Sequence; import io.trino.spi.connector.SystemTable; import io.trino.spi.function.table.ConnectorTableFunction; @@ -79,7 +79,7 @@ public void configure(Binder binder) binder.bind(GlobalSystemConnector.class).in(Scopes.SINGLETON); Multibinder tableFunctions = Multibinder.newSetBinder(binder, ConnectorTableFunction.class); - tableFunctions.addBinding().toProvider(ExcludeColumns.class).in(Scopes.SINGLETON); + tableFunctions.addBinding().to(ExcludeColumnsFunction.class).in(Scopes.SINGLETON); tableFunctions.addBinding().toProvider(Sequence.class).in(Scopes.SINGLETON); } } diff --git a/core/trino-main/src/main/java/io/trino/metadata/GlobalFunctionCatalog.java b/core/trino-main/src/main/java/io/trino/metadata/GlobalFunctionCatalog.java index 71a8e17eb5b85..af84b2dd38b3e 100644 --- a/core/trino-main/src/main/java/io/trino/metadata/GlobalFunctionCatalog.java +++ b/core/trino-main/src/main/java/io/trino/metadata/GlobalFunctionCatalog.java @@ -21,7 +21,7 @@ import com.google.inject.Inject; import com.google.inject.Provider; import io.trino.connector.system.GlobalSystemConnector; -import io.trino.operator.table.ExcludeColumns.ExcludeColumnsFunctionHandle; +import io.trino.operator.table.ExcludeColumnsFunction.ExcludeColumnsFunctionHandle; import io.trino.operator.table.Sequence.SequenceFunctionHandle; import io.trino.operator.table.json.JsonTable.JsonTableFunctionHandle; import io.trino.spi.function.AggregationFunctionMetadata; @@ -55,7 +55,7 @@ import static io.trino.metadata.OperatorNameUtil.isOperatorName; import static io.trino.metadata.OperatorNameUtil.mangleOperatorName; import static io.trino.metadata.OperatorNameUtil.unmangleOperator; -import static io.trino.operator.table.ExcludeColumns.getExcludeColumnsFunctionProcessorProvider; +import static io.trino.operator.table.ExcludeColumnsFunction.getExcludeColumnsFunctionProcessorProvider; import static io.trino.operator.table.Sequence.getSequenceFunctionProcessorProvider; import static io.trino.operator.table.json.JsonTable.getJsonTableFunctionProcessorProvider; import static io.trino.spi.function.FunctionKind.AGGREGATE; diff --git a/core/trino-main/src/main/java/io/trino/operator/table/ExcludeColumns.java b/core/trino-main/src/main/java/io/trino/operator/table/ExcludeColumns.java deleted file mode 100644 index d650c97b2b987..0000000000000 --- a/core/trino-main/src/main/java/io/trino/operator/table/ExcludeColumns.java +++ /dev/null @@ -1,174 +0,0 @@ -/* - * 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.operator.table; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Sets; -import com.google.inject.Provider; -import io.trino.plugin.base.classloader.ClassLoaderSafeConnectorTableFunction; -import io.trino.spi.TrinoException; -import io.trino.spi.connector.ConnectorAccessControl; -import io.trino.spi.connector.ConnectorSession; -import io.trino.spi.connector.ConnectorTransactionHandle; -import io.trino.spi.function.table.AbstractConnectorTableFunction; -import io.trino.spi.function.table.Argument; -import io.trino.spi.function.table.ConnectorTableFunction; -import io.trino.spi.function.table.ConnectorTableFunctionHandle; -import io.trino.spi.function.table.Descriptor; -import io.trino.spi.function.table.DescriptorArgument; -import io.trino.spi.function.table.DescriptorArgumentSpecification; -import io.trino.spi.function.table.TableArgument; -import io.trino.spi.function.table.TableArgumentSpecification; -import io.trino.spi.function.table.TableFunctionAnalysis; -import io.trino.spi.function.table.TableFunctionDataProcessor; -import io.trino.spi.function.table.TableFunctionProcessorProvider; -import io.trino.spi.type.RowType; - -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.Set; - -import static com.google.common.collect.ImmutableSet.toImmutableSet; -import static com.google.common.collect.Iterables.getOnlyElement; -import static io.trino.metadata.GlobalFunctionCatalog.BUILTIN_SCHEMA; -import static io.trino.spi.StandardErrorCode.INVALID_FUNCTION_ARGUMENT; -import static io.trino.spi.function.table.DescriptorArgument.NULL_DESCRIPTOR; -import static io.trino.spi.function.table.ReturnTypeSpecification.GenericTable.GENERIC_TABLE; -import static io.trino.spi.function.table.TableFunctionProcessorState.Finished.FINISHED; -import static io.trino.spi.function.table.TableFunctionProcessorState.Processed.usedInputAndProduced; -import static java.lang.String.format; -import static java.util.Locale.ENGLISH; -import static java.util.stream.Collectors.joining; - -public class ExcludeColumns - implements Provider -{ - public static final String NAME = "exclude_columns"; - - @Override - public ConnectorTableFunction get() - { - return new ClassLoaderSafeConnectorTableFunction(new ExcludeColumnsFunction(), getClass().getClassLoader()); - } - - public static class ExcludeColumnsFunction - extends AbstractConnectorTableFunction - { - private static final String TABLE_ARGUMENT_NAME = "INPUT"; - private static final String DESCRIPTOR_ARGUMENT_NAME = "COLUMNS"; - - public ExcludeColumnsFunction() - { - super( - BUILTIN_SCHEMA, - NAME, - ImmutableList.of( - TableArgumentSpecification.builder() - .name(TABLE_ARGUMENT_NAME) - .rowSemantics() - .build(), - DescriptorArgumentSpecification.builder() - .name(DESCRIPTOR_ARGUMENT_NAME) - .build()), - GENERIC_TABLE); - } - - @Override - public TableFunctionAnalysis analyze( - ConnectorSession session, - ConnectorTransactionHandle transaction, - Map arguments, - ConnectorAccessControl accessControl) - { - DescriptorArgument excludedColumns = (DescriptorArgument) arguments.get(DESCRIPTOR_ARGUMENT_NAME); - if (excludedColumns.equals(NULL_DESCRIPTOR)) { - throw new TrinoException(INVALID_FUNCTION_ARGUMENT, "COLUMNS descriptor is null"); - } - Descriptor excludedColumnsDescriptor = excludedColumns.getDescriptor().orElseThrow(); - if (excludedColumnsDescriptor.getFields().stream().anyMatch(field -> field.getType().isPresent())) { - throw new TrinoException(INVALID_FUNCTION_ARGUMENT, "COLUMNS descriptor contains types"); - } - - // column names in DescriptorArgument are canonical wrt SQL identifier semantics. - // column names in TableArgument are not canonical wrt SQL identifier semantics, as they are taken from the corresponding RelationType. - // because of that, we match the excluded columns names case-insensitive - // TODO apply proper identifier semantics - Set excludedNames = excludedColumnsDescriptor.getFields().stream() - .map(Descriptor.Field::getName) - .map(name -> name.orElseThrow().toLowerCase(ENGLISH)) - .collect(toImmutableSet()); - - List inputSchema = ((TableArgument) arguments.get(TABLE_ARGUMENT_NAME)).getRowType().getFields(); - Set inputNames = inputSchema.stream() - .map(RowType.Field::getName) - .filter(Optional::isPresent) - .map(Optional::get) - .map(name -> name.toLowerCase(ENGLISH)) - .collect(toImmutableSet()); - - if (!inputNames.containsAll(excludedNames)) { - String missingColumns = Sets.difference(excludedNames, inputNames).stream() - .collect(joining(", ", "[", "]")); - throw new TrinoException(INVALID_FUNCTION_ARGUMENT, format("Excluded columns: %s not present in the table", missingColumns)); - } - - ImmutableList.Builder requiredColumns = ImmutableList.builder(); - ImmutableList.Builder returnedColumns = ImmutableList.builder(); - - for (int i = 0; i < inputSchema.size(); i++) { - Optional name = inputSchema.get(i).getName(); - if (name.isEmpty() || !excludedNames.contains(name.orElseThrow().toLowerCase(ENGLISH))) { - requiredColumns.add(i); - // per SQL standard, all columns produced by a table function must be named. We allow anonymous columns. - returnedColumns.add(new Descriptor.Field(name, Optional.of(inputSchema.get(i).getType()))); - } - } - - List returnedType = returnedColumns.build(); - if (returnedType.isEmpty()) { - throw new TrinoException(INVALID_FUNCTION_ARGUMENT, "All columns are excluded"); - } - - return TableFunctionAnalysis.builder() - .requiredColumns(TABLE_ARGUMENT_NAME, requiredColumns.build()) - .returnedType(new Descriptor(returnedType)) - .handle(new ExcludeColumnsFunctionHandle()) - .build(); - } - } - - public static TableFunctionProcessorProvider getExcludeColumnsFunctionProcessorProvider() - { - return new TableFunctionProcessorProvider() - { - @Override - public TableFunctionDataProcessor getDataProcessor(ConnectorSession session, ConnectorTableFunctionHandle handle) - { - return input -> { - if (input == null) { - return FINISHED; - } - return usedInputAndProduced(getOnlyElement(input).orElseThrow()); - }; - } - }; - } - - public record ExcludeColumnsFunctionHandle() - implements ConnectorTableFunctionHandle - { - // there's no information to remember. All logic is effectively delegated to the engine via `requiredColumns`. - } -} diff --git a/core/trino-main/src/main/java/io/trino/operator/table/ExcludeColumnsFunction.java b/core/trino-main/src/main/java/io/trino/operator/table/ExcludeColumnsFunction.java new file mode 100644 index 0000000000000..0417b91ffa64f --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/operator/table/ExcludeColumnsFunction.java @@ -0,0 +1,161 @@ +/* + * 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.operator.table; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Sets; +import io.trino.spi.TrinoException; +import io.trino.spi.connector.ConnectorAccessControl; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.ConnectorTransactionHandle; +import io.trino.spi.function.table.AbstractConnectorTableFunction; +import io.trino.spi.function.table.Argument; +import io.trino.spi.function.table.ConnectorTableFunctionHandle; +import io.trino.spi.function.table.Descriptor; +import io.trino.spi.function.table.DescriptorArgument; +import io.trino.spi.function.table.DescriptorArgumentSpecification; +import io.trino.spi.function.table.TableArgument; +import io.trino.spi.function.table.TableArgumentSpecification; +import io.trino.spi.function.table.TableFunctionAnalysis; +import io.trino.spi.function.table.TableFunctionDataProcessor; +import io.trino.spi.function.table.TableFunctionProcessorProvider; +import io.trino.spi.type.RowType; + +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +import static com.google.common.collect.ImmutableSet.toImmutableSet; +import static com.google.common.collect.Iterables.getOnlyElement; +import static io.trino.metadata.GlobalFunctionCatalog.BUILTIN_SCHEMA; +import static io.trino.spi.StandardErrorCode.INVALID_FUNCTION_ARGUMENT; +import static io.trino.spi.function.table.DescriptorArgument.NULL_DESCRIPTOR; +import static io.trino.spi.function.table.ReturnTypeSpecification.GenericTable.GENERIC_TABLE; +import static io.trino.spi.function.table.TableFunctionProcessorState.Finished.FINISHED; +import static io.trino.spi.function.table.TableFunctionProcessorState.Processed.usedInputAndProduced; +import static java.lang.String.format; +import static java.util.Locale.ENGLISH; +import static java.util.stream.Collectors.joining; + +public class ExcludeColumnsFunction + extends AbstractConnectorTableFunction +{ + public static final String NAME = "exclude_columns"; + + private static final String TABLE_ARGUMENT_NAME = "INPUT"; + private static final String DESCRIPTOR_ARGUMENT_NAME = "COLUMNS"; + + public ExcludeColumnsFunction() + { + super( + BUILTIN_SCHEMA, + NAME, + ImmutableList.of( + TableArgumentSpecification.builder() + .name(TABLE_ARGUMENT_NAME) + .rowSemantics() + .build(), + DescriptorArgumentSpecification.builder() + .name(DESCRIPTOR_ARGUMENT_NAME) + .build()), + GENERIC_TABLE); + } + + @Override + public TableFunctionAnalysis analyze( + ConnectorSession session, + ConnectorTransactionHandle transaction, + Map arguments, + ConnectorAccessControl accessControl) + { + DescriptorArgument excludedColumns = (DescriptorArgument) arguments.get(DESCRIPTOR_ARGUMENT_NAME); + if (excludedColumns.equals(NULL_DESCRIPTOR)) { + throw new TrinoException(INVALID_FUNCTION_ARGUMENT, "COLUMNS descriptor is null"); + } + Descriptor excludedColumnsDescriptor = excludedColumns.getDescriptor().orElseThrow(); + if (excludedColumnsDescriptor.getFields().stream().anyMatch(field -> field.getType().isPresent())) { + throw new TrinoException(INVALID_FUNCTION_ARGUMENT, "COLUMNS descriptor contains types"); + } + + // column names in DescriptorArgument are canonical wrt SQL identifier semantics. + // column names in TableArgument are not canonical wrt SQL identifier semantics, as they are taken from the corresponding RelationType. + // because of that, we match the excluded columns names case-insensitive + // TODO apply proper identifier semantics + Set excludedNames = excludedColumnsDescriptor.getFields().stream() + .map(Descriptor.Field::getName) + .map(name -> name.orElseThrow().toLowerCase(ENGLISH)) + .collect(toImmutableSet()); + + List inputSchema = ((TableArgument) arguments.get(TABLE_ARGUMENT_NAME)).getRowType().getFields(); + Set inputNames = inputSchema.stream() + .map(RowType.Field::getName) + .filter(Optional::isPresent) + .map(Optional::get) + .map(name -> name.toLowerCase(ENGLISH)) + .collect(toImmutableSet()); + + if (!inputNames.containsAll(excludedNames)) { + String missingColumns = Sets.difference(excludedNames, inputNames).stream() + .collect(joining(", ", "[", "]")); + throw new TrinoException(INVALID_FUNCTION_ARGUMENT, format("Excluded columns: %s not present in the table", missingColumns)); + } + + ImmutableList.Builder requiredColumns = ImmutableList.builder(); + ImmutableList.Builder returnedColumns = ImmutableList.builder(); + + for (int i = 0; i < inputSchema.size(); i++) { + Optional name = inputSchema.get(i).getName(); + if (name.isEmpty() || !excludedNames.contains(name.orElseThrow().toLowerCase(ENGLISH))) { + requiredColumns.add(i); + // per SQL standard, all columns produced by a table function must be named. We allow anonymous columns. + returnedColumns.add(new Descriptor.Field(name, Optional.of(inputSchema.get(i).getType()))); + } + } + + List returnedType = returnedColumns.build(); + if (returnedType.isEmpty()) { + throw new TrinoException(INVALID_FUNCTION_ARGUMENT, "All columns are excluded"); + } + + return TableFunctionAnalysis.builder() + .requiredColumns(TABLE_ARGUMENT_NAME, requiredColumns.build()) + .returnedType(new Descriptor(returnedType)) + .handle(new ExcludeColumnsFunctionHandle()) + .build(); + } + + public static TableFunctionProcessorProvider getExcludeColumnsFunctionProcessorProvider() + { + return new TableFunctionProcessorProvider() + { + @Override + public TableFunctionDataProcessor getDataProcessor(ConnectorSession session, ConnectorTableFunctionHandle handle) + { + return input -> { + if (input == null) { + return FINISHED; + } + return usedInputAndProduced(getOnlyElement(input).orElseThrow()); + }; + } + }; + } + + public record ExcludeColumnsFunctionHandle() + implements ConnectorTableFunctionHandle + { + // there's no information to remember. All logic is effectively delegated to the engine via `requiredColumns`. + } +} diff --git a/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java b/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java index 403a27fab0476..b2119dc59ab25 100644 --- a/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java +++ b/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java @@ -124,7 +124,7 @@ import io.trino.operator.scalar.json.JsonExistsFunction; import io.trino.operator.scalar.json.JsonQueryFunction; import io.trino.operator.scalar.json.JsonValueFunction; -import io.trino.operator.table.ExcludeColumns.ExcludeColumnsFunction; +import io.trino.operator.table.ExcludeColumnsFunction; import io.trino.plugin.base.security.AllowAllSystemAccessControl; import io.trino.security.GroupProviderManager; import io.trino.server.PluginManager; From a8ac255028418d6d0050b5d70904b237e5ebcab5 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 20 Dec 2023 22:27:46 +0100 Subject: [PATCH 183/350] Remove redundant classloader wrapper from sequence Engine-provided functions do not need `ClassLoaderSafeConnectorTableFunction` wrapper. --- .../system/GlobalSystemConnector.java | 4 +- .../system/SystemConnectorModule.java | 4 +- .../trino/metadata/GlobalFunctionCatalog.java | 4 +- .../{Sequence.java => SequenceFunction.java} | 125 ++++++++---------- .../io/trino/tests/TestSequenceFunction.java | 2 +- 5 files changed, 63 insertions(+), 76 deletions(-) rename core/trino-main/src/main/java/io/trino/operator/table/{Sequence.java => SequenceFunction.java} (68%) diff --git a/core/trino-main/src/main/java/io/trino/connector/system/GlobalSystemConnector.java b/core/trino-main/src/main/java/io/trino/connector/system/GlobalSystemConnector.java index 3912bbe262c98..6fb93e229023e 100644 --- a/core/trino-main/src/main/java/io/trino/connector/system/GlobalSystemConnector.java +++ b/core/trino-main/src/main/java/io/trino/connector/system/GlobalSystemConnector.java @@ -15,7 +15,7 @@ import com.google.common.collect.ImmutableSet; import com.google.inject.Inject; -import io.trino.operator.table.Sequence.SequenceFunctionHandle; +import io.trino.operator.table.SequenceFunction.SequenceFunctionHandle; import io.trino.spi.connector.CatalogHandle; import io.trino.spi.connector.CatalogHandle.CatalogVersion; import io.trino.spi.connector.ConnectorMetadata; @@ -33,7 +33,7 @@ import java.util.Set; -import static io.trino.operator.table.Sequence.getSequenceFunctionSplitSource; +import static io.trino.operator.table.SequenceFunction.getSequenceFunctionSplitSource; import static io.trino.spi.connector.CatalogHandle.createRootCatalogHandle; import static java.util.Objects.requireNonNull; diff --git a/core/trino-main/src/main/java/io/trino/connector/system/SystemConnectorModule.java b/core/trino-main/src/main/java/io/trino/connector/system/SystemConnectorModule.java index 8500f3e645b46..7d6bb634309bc 100644 --- a/core/trino-main/src/main/java/io/trino/connector/system/SystemConnectorModule.java +++ b/core/trino-main/src/main/java/io/trino/connector/system/SystemConnectorModule.java @@ -31,7 +31,7 @@ import io.trino.connector.system.jdbc.TypesJdbcTable; import io.trino.connector.system.jdbc.UdtJdbcTable; import io.trino.operator.table.ExcludeColumnsFunction; -import io.trino.operator.table.Sequence; +import io.trino.operator.table.SequenceFunction; import io.trino.spi.connector.SystemTable; import io.trino.spi.function.table.ConnectorTableFunction; import io.trino.spi.procedure.Procedure; @@ -80,6 +80,6 @@ public void configure(Binder binder) Multibinder tableFunctions = Multibinder.newSetBinder(binder, ConnectorTableFunction.class); tableFunctions.addBinding().to(ExcludeColumnsFunction.class).in(Scopes.SINGLETON); - tableFunctions.addBinding().toProvider(Sequence.class).in(Scopes.SINGLETON); + tableFunctions.addBinding().to(SequenceFunction.class).in(Scopes.SINGLETON); } } diff --git a/core/trino-main/src/main/java/io/trino/metadata/GlobalFunctionCatalog.java b/core/trino-main/src/main/java/io/trino/metadata/GlobalFunctionCatalog.java index af84b2dd38b3e..6cdd80f483000 100644 --- a/core/trino-main/src/main/java/io/trino/metadata/GlobalFunctionCatalog.java +++ b/core/trino-main/src/main/java/io/trino/metadata/GlobalFunctionCatalog.java @@ -22,7 +22,7 @@ import com.google.inject.Provider; import io.trino.connector.system.GlobalSystemConnector; import io.trino.operator.table.ExcludeColumnsFunction.ExcludeColumnsFunctionHandle; -import io.trino.operator.table.Sequence.SequenceFunctionHandle; +import io.trino.operator.table.SequenceFunction.SequenceFunctionHandle; import io.trino.operator.table.json.JsonTable.JsonTableFunctionHandle; import io.trino.spi.function.AggregationFunctionMetadata; import io.trino.spi.function.AggregationImplementation; @@ -56,7 +56,7 @@ import static io.trino.metadata.OperatorNameUtil.mangleOperatorName; import static io.trino.metadata.OperatorNameUtil.unmangleOperator; import static io.trino.operator.table.ExcludeColumnsFunction.getExcludeColumnsFunctionProcessorProvider; -import static io.trino.operator.table.Sequence.getSequenceFunctionProcessorProvider; +import static io.trino.operator.table.SequenceFunction.getSequenceFunctionProcessorProvider; import static io.trino.operator.table.json.JsonTable.getJsonTableFunctionProcessorProvider; import static io.trino.spi.function.FunctionKind.AGGREGATE; import static io.trino.spi.type.BigintType.BIGINT; diff --git a/core/trino-main/src/main/java/io/trino/operator/table/Sequence.java b/core/trino-main/src/main/java/io/trino/operator/table/SequenceFunction.java similarity index 68% rename from core/trino-main/src/main/java/io/trino/operator/table/Sequence.java rename to core/trino-main/src/main/java/io/trino/operator/table/SequenceFunction.java index c111f86c51401..2a1b176c95928 100644 --- a/core/trino-main/src/main/java/io/trino/operator/table/Sequence.java +++ b/core/trino-main/src/main/java/io/trino/operator/table/SequenceFunction.java @@ -17,8 +17,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.inject.Provider; -import io.trino.plugin.base.classloader.ClassLoaderSafeConnectorTableFunction; import io.trino.spi.Page; import io.trino.spi.PageBuilder; import io.trino.spi.TrinoException; @@ -31,7 +29,6 @@ import io.trino.spi.connector.FixedSplitSource; import io.trino.spi.function.table.AbstractConnectorTableFunction; import io.trino.spi.function.table.Argument; -import io.trino.spi.function.table.ConnectorTableFunction; import io.trino.spi.function.table.ConnectorTableFunctionHandle; import io.trino.spi.function.table.ReturnTypeSpecification.DescribedTable; import io.trino.spi.function.table.ScalarArgument; @@ -48,7 +45,7 @@ import static com.google.common.base.Preconditions.checkState; import static io.airlift.slice.SizeOf.instanceSize; import static io.trino.metadata.GlobalFunctionCatalog.BUILTIN_SCHEMA; -import static io.trino.operator.table.Sequence.SequenceFunctionSplit.MAX_SPLIT_SIZE; +import static io.trino.operator.table.SequenceFunction.SequenceFunctionSplit.MAX_SPLIT_SIZE; import static io.trino.spi.StandardErrorCode.INVALID_FUNCTION_ARGUMENT; import static io.trino.spi.function.table.Descriptor.descriptor; import static io.trino.spi.function.table.TableFunctionProcessorState.Finished.FINISHED; @@ -56,85 +53,75 @@ import static io.trino.spi.type.BigintType.BIGINT; import static java.lang.String.format; -public class Sequence - implements Provider +public class SequenceFunction + extends AbstractConnectorTableFunction { public static final String NAME = "sequence"; - @Override - public ConnectorTableFunction get() + private static final String START_ARGUMENT_NAME = "START"; + private static final String STOP_ARGUMENT_NAME = "STOP"; + private static final String STEP_ARGUMENT_NAME = "STEP"; + + public SequenceFunction() { - return new ClassLoaderSafeConnectorTableFunction(new SequenceFunction(), getClass().getClassLoader()); + super( + BUILTIN_SCHEMA, + NAME, + ImmutableList.of( + ScalarArgumentSpecification.builder() + .name(START_ARGUMENT_NAME) + .type(BIGINT) + .defaultValue(0L) + .build(), + ScalarArgumentSpecification.builder() + .name(STOP_ARGUMENT_NAME) + .type(BIGINT) + .build(), + ScalarArgumentSpecification.builder() + .name(STEP_ARGUMENT_NAME) + .type(BIGINT) + .defaultValue(1L) + .build()), + new DescribedTable(descriptor(ImmutableList.of("sequential_number"), ImmutableList.of(BIGINT)))); } - public static class SequenceFunction - extends AbstractConnectorTableFunction + @Override + public TableFunctionAnalysis analyze( + ConnectorSession session, + ConnectorTransactionHandle transaction, + Map arguments, + ConnectorAccessControl accessControl) { - private static final String START_ARGUMENT_NAME = "START"; - private static final String STOP_ARGUMENT_NAME = "STOP"; - private static final String STEP_ARGUMENT_NAME = "STEP"; - - public SequenceFunction() - { - super( - BUILTIN_SCHEMA, - NAME, - ImmutableList.of( - ScalarArgumentSpecification.builder() - .name(START_ARGUMENT_NAME) - .type(BIGINT) - .defaultValue(0L) - .build(), - ScalarArgumentSpecification.builder() - .name(STOP_ARGUMENT_NAME) - .type(BIGINT) - .build(), - ScalarArgumentSpecification.builder() - .name(STEP_ARGUMENT_NAME) - .type(BIGINT) - .defaultValue(1L) - .build()), - new DescribedTable(descriptor(ImmutableList.of("sequential_number"), ImmutableList.of(BIGINT)))); + Object startValue = ((ScalarArgument) arguments.get(START_ARGUMENT_NAME)).getValue(); + if (startValue == null) { + throw new TrinoException(INVALID_FUNCTION_ARGUMENT, "Start is null"); } - @Override - public TableFunctionAnalysis analyze( - ConnectorSession session, - ConnectorTransactionHandle transaction, - Map arguments, - ConnectorAccessControl accessControl) - { - Object startValue = ((ScalarArgument) arguments.get(START_ARGUMENT_NAME)).getValue(); - if (startValue == null) { - throw new TrinoException(INVALID_FUNCTION_ARGUMENT, "Start is null"); - } - - Object stopValue = ((ScalarArgument) arguments.get(STOP_ARGUMENT_NAME)).getValue(); - if (stopValue == null) { - throw new TrinoException(INVALID_FUNCTION_ARGUMENT, "Stop is null"); - } + Object stopValue = ((ScalarArgument) arguments.get(STOP_ARGUMENT_NAME)).getValue(); + if (stopValue == null) { + throw new TrinoException(INVALID_FUNCTION_ARGUMENT, "Stop is null"); + } - Object stepValue = ((ScalarArgument) arguments.get(STEP_ARGUMENT_NAME)).getValue(); - if (stepValue == null) { - throw new TrinoException(INVALID_FUNCTION_ARGUMENT, "Step is null"); - } + Object stepValue = ((ScalarArgument) arguments.get(STEP_ARGUMENT_NAME)).getValue(); + if (stepValue == null) { + throw new TrinoException(INVALID_FUNCTION_ARGUMENT, "Step is null"); + } - long start = (long) startValue; - long stop = (long) stopValue; - long step = (long) stepValue; + long start = (long) startValue; + long stop = (long) stopValue; + long step = (long) stepValue; - if (start < stop && step <= 0) { - throw new TrinoException(INVALID_FUNCTION_ARGUMENT, format("Step must be positive for sequence [%s, %s]", start, stop)); - } - - if (start > stop && step >= 0) { - throw new TrinoException(INVALID_FUNCTION_ARGUMENT, format("Step must be negative for sequence [%s, %s]", start, stop)); - } + if (start < stop && step <= 0) { + throw new TrinoException(INVALID_FUNCTION_ARGUMENT, format("Step must be positive for sequence [%s, %s]", start, stop)); + } - return TableFunctionAnalysis.builder() - .handle(new SequenceFunctionHandle(start, stop, start == stop ? 0 : step)) - .build(); + if (start > stop && step >= 0) { + throw new TrinoException(INVALID_FUNCTION_ARGUMENT, format("Step must be negative for sequence [%s, %s]", start, stop)); } + + return TableFunctionAnalysis.builder() + .handle(new SequenceFunctionHandle(start, stop, start == stop ? 0 : step)) + .build(); } public record SequenceFunctionHandle(long start, long stop, long step) diff --git a/testing/trino-tests/src/test/java/io/trino/tests/TestSequenceFunction.java b/testing/trino-tests/src/test/java/io/trino/tests/TestSequenceFunction.java index 1be55965f6745..b1688f3e3a23c 100644 --- a/testing/trino-tests/src/test/java/io/trino/tests/TestSequenceFunction.java +++ b/testing/trino-tests/src/test/java/io/trino/tests/TestSequenceFunction.java @@ -18,7 +18,7 @@ import io.trino.testing.QueryRunner; import org.junit.jupiter.api.Test; -import static io.trino.operator.table.Sequence.SequenceFunctionSplit.DEFAULT_SPLIT_SIZE; +import static io.trino.operator.table.SequenceFunction.SequenceFunctionSplit.DEFAULT_SPLIT_SIZE; import static io.trino.testing.TestingSession.testSessionBuilder; import static java.lang.String.format; import static org.assertj.core.api.Assertions.assertThat; From 95d59fe9280e11beeae72f5f8903f43ae03159c2 Mon Sep 17 00:00:00 2001 From: Yuya Ebihara Date: Thu, 21 Dec 2023 15:19:31 +0900 Subject: [PATCH 184/350] Remove redundant air.test.parallel property --- plugin/trino-cassandra/pom.xml | 1 - plugin/trino-elasticsearch/pom.xml | 1 - plugin/trino-pinot/pom.xml | 7 ------- plugin/trino-sqlserver/pom.xml | 9 --------- 4 files changed, 18 deletions(-) diff --git a/plugin/trino-cassandra/pom.xml b/plugin/trino-cassandra/pom.xml index 5da45d4abc3c0..5aa4d6099a687 100644 --- a/plugin/trino-cassandra/pom.xml +++ b/plugin/trino-cassandra/pom.xml @@ -15,7 +15,6 @@ ${project.parent.basedir} 1.5.1 - instances diff --git a/plugin/trino-elasticsearch/pom.xml b/plugin/trino-elasticsearch/pom.xml index 0ad20c8aaecad..c742a69a3f331 100644 --- a/plugin/trino-elasticsearch/pom.xml +++ b/plugin/trino-elasticsearch/pom.xml @@ -16,7 +16,6 @@ ${project.parent.basedir} 6.8.23 - instances diff --git a/plugin/trino-pinot/pom.xml b/plugin/trino-pinot/pom.xml index 7ba6ff80a783f..f530198220fb5 100755 --- a/plugin/trino-pinot/pom.xml +++ b/plugin/trino-pinot/pom.xml @@ -15,13 +15,6 @@ ${project.parent.basedir} 0.12.1 - - instances diff --git a/plugin/trino-sqlserver/pom.xml b/plugin/trino-sqlserver/pom.xml index 1117e536dfcee..441ee718262a0 100644 --- a/plugin/trino-sqlserver/pom.xml +++ b/plugin/trino-sqlserver/pom.xml @@ -15,15 +15,6 @@ ${project.parent.basedir} - - - instances From 05404daab0bae2dfa4de4c899824374ef59a54a6 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 20 Dec 2023 17:05:14 +0100 Subject: [PATCH 185/350] Fix ConstantExpression serialization When `ConstantExpression` was created with a Trino type and a value that didn't match this type's java type (for example Trino INTEGER with value being Integer, instead of Long), the `ConstantExpression` would fail to serialize to JSON. Such failure happens during sending task status updates to workers and is currently logged and ignored, leading to query hang. The problem could be triggered with SQL routines, which utilize `RowExpression` (including `ConstantExpression`) serialization. Thus, this fixes execution of SQL routines involving Row field dereference. --- .../sql/gen/DereferenceCodeGenerator.java | 3 ++- .../sql/relational/ConstantExpression.java | 8 ++++++++ .../SqlToRowExpressionTranslator.java | 2 +- .../io/trino/sql/gen/TestInCodeGenerator.java | 18 +++++++++--------- .../testing/AbstractTestEngineOnlyQueries.java | 9 +++++++++ 5 files changed, 29 insertions(+), 11 deletions(-) diff --git a/core/trino-main/src/main/java/io/trino/sql/gen/DereferenceCodeGenerator.java b/core/trino-main/src/main/java/io/trino/sql/gen/DereferenceCodeGenerator.java index 6f57363567913..8770e094926d1 100644 --- a/core/trino-main/src/main/java/io/trino/sql/gen/DereferenceCodeGenerator.java +++ b/core/trino-main/src/main/java/io/trino/sql/gen/DereferenceCodeGenerator.java @@ -29,6 +29,7 @@ import static com.google.common.base.Preconditions.checkArgument; import static io.airlift.bytecode.expression.BytecodeExpressions.constantInt; import static io.trino.sql.gen.SqlTypeBytecodeExpression.constantType; +import static java.lang.Math.toIntExact; import static java.util.Objects.requireNonNull; public class DereferenceCodeGenerator @@ -44,7 +45,7 @@ public DereferenceCodeGenerator(SpecialForm specialForm) returnType = specialForm.getType(); checkArgument(specialForm.getArguments().size() == 2); base = specialForm.getArguments().get(0); - index = (int) ((ConstantExpression) specialForm.getArguments().get(1)).getValue(); + index = toIntExact((long) ((ConstantExpression) specialForm.getArguments().get(1)).getValue()); } @Override diff --git a/core/trino-main/src/main/java/io/trino/sql/relational/ConstantExpression.java b/core/trino-main/src/main/java/io/trino/sql/relational/ConstantExpression.java index 9cbf9fd8c8243..fa7f804120647 100644 --- a/core/trino-main/src/main/java/io/trino/sql/relational/ConstantExpression.java +++ b/core/trino-main/src/main/java/io/trino/sql/relational/ConstantExpression.java @@ -15,6 +15,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.primitives.Primitives; import com.google.errorprone.annotations.DoNotCall; import io.airlift.slice.Slice; import io.trino.spi.block.Block; @@ -48,6 +49,13 @@ public static ConstantExpression fromJson( public ConstantExpression(Object value, Type type) { requireNonNull(type, "type is null"); + if (value != null && !Primitives.wrap(type.getJavaType()).isInstance(value)) { + throw new IllegalArgumentException("Invalid value %s of Java type %s for Trino type %s, expected instance of %s".formatted( + value, + value.getClass(), + type, + type.getJavaType())); + } this.value = value; this.type = type; diff --git a/core/trino-main/src/main/java/io/trino/sql/relational/SqlToRowExpressionTranslator.java b/core/trino-main/src/main/java/io/trino/sql/relational/SqlToRowExpressionTranslator.java index 8fd3e2efd5b81..dfedb6720c67b 100644 --- a/core/trino-main/src/main/java/io/trino/sql/relational/SqlToRowExpressionTranslator.java +++ b/core/trino-main/src/main/java/io/trino/sql/relational/SqlToRowExpressionTranslator.java @@ -702,7 +702,7 @@ protected RowExpression visitSubscriptExpression(SubscriptExpression node, Void if (getType(node.getBase()) instanceof RowType) { long value = (Long) ((ConstantExpression) index).getValue(); - return new SpecialForm(DEREFERENCE, getType(node), base, constant((int) value - 1, INTEGER)); + return new SpecialForm(DEREFERENCE, getType(node), base, constant(value - 1, INTEGER)); } return call( diff --git a/core/trino-main/src/test/java/io/trino/sql/gen/TestInCodeGenerator.java b/core/trino-main/src/test/java/io/trino/sql/gen/TestInCodeGenerator.java index 2fcaa99882b5c..9bfb5af900252 100644 --- a/core/trino-main/src/test/java/io/trino/sql/gen/TestInCodeGenerator.java +++ b/core/trino-main/src/test/java/io/trino/sql/gen/TestInCodeGenerator.java @@ -43,9 +43,9 @@ public class TestInCodeGenerator public void testInteger() { List values = new ArrayList<>(); - values.add(constant(Integer.MIN_VALUE, INTEGER)); - values.add(constant(Integer.MAX_VALUE, INTEGER)); - values.add(constant(3, INTEGER)); + values.add(constant((long) Integer.MIN_VALUE, INTEGER)); + values.add(constant((long) Integer.MAX_VALUE, INTEGER)); + values.add(constant(3L, INTEGER)); assertThat(checkSwitchGenerationCase(INTEGER, values)).isEqualTo(DIRECT_SWITCH); values.add(constant(null, INTEGER)); @@ -55,11 +55,11 @@ public void testInteger() Collections.singletonList(constant(12345678901234.0, DOUBLE)))); assertThat(checkSwitchGenerationCase(INTEGER, values)).isEqualTo(DIRECT_SWITCH); - values.add(constant(6, BIGINT)); - values.add(constant(7, BIGINT)); + values.add(constant(6L, BIGINT)); + values.add(constant(7L, BIGINT)); assertThat(checkSwitchGenerationCase(INTEGER, values)).isEqualTo(DIRECT_SWITCH); - values.add(constant(8, INTEGER)); + values.add(constant(8L, INTEGER)); assertThat(checkSwitchGenerationCase(INTEGER, values)).isEqualTo(SET_CONTAINS); } @@ -130,9 +130,9 @@ public void testDouble() public void testVarchar() { List values = new ArrayList<>(); - values.add(constant(Slices.utf8Slice("1"), DOUBLE)); - values.add(constant(Slices.utf8Slice("2"), DOUBLE)); - values.add(constant(Slices.utf8Slice("3"), DOUBLE)); + values.add(constant(Slices.utf8Slice("1"), VARCHAR)); + values.add(constant(Slices.utf8Slice("2"), VARCHAR)); + values.add(constant(Slices.utf8Slice("3"), VARCHAR)); assertThat(checkSwitchGenerationCase(VARCHAR, values)).isEqualTo(HASH_SWITCH); values.add(constant(null, VARCHAR)); diff --git a/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestEngineOnlyQueries.java b/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestEngineOnlyQueries.java index e3699a15e98fb..734cd082242f2 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestEngineOnlyQueries.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestEngineOnlyQueries.java @@ -6642,6 +6642,15 @@ SELECT my_pow(2, 8) """)) .matches("VALUES 256"); + // function with dereference + assertThat(query(""" + WITH FUNCTION get(input row(varchar)) + RETURNS varchar + RETURN input[1] + SELECT get(ROW('abc')) + """)) + .matches("VALUES VARCHAR 'abc'"); + // validations for inline functions assertQueryFails("WITH FUNCTION a.b() RETURNS int RETURN 42 SELECT a.b()", "line 1:6: Inline function names cannot be qualified: a.b"); From 2754c0bb8f1f427006ac991835bc788148593716 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Thu, 21 Dec 2023 12:28:19 +0100 Subject: [PATCH 186/350] Fix TestAccessControl execution The test must run single-threaded and was run like that until 9a7cf10d16253d165253218ae8345f22e6d480c8. --- .../src/test/java/io/trino/security/TestAccessControl.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/testing/trino-tests/src/test/java/io/trino/security/TestAccessControl.java b/testing/trino-tests/src/test/java/io/trino/security/TestAccessControl.java index e86fe06d1d5d6..33961bf0cd525 100644 --- a/testing/trino-tests/src/test/java/io/trino/security/TestAccessControl.java +++ b/testing/trino-tests/src/test/java/io/trino/security/TestAccessControl.java @@ -65,6 +65,7 @@ import io.trino.testing.TestingGroupProvider; import io.trino.testing.TestingSession; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.parallel.Execution; import java.lang.invoke.MethodHandles; import java.time.Duration; @@ -118,7 +119,9 @@ import static java.util.stream.Collectors.toList; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; +@Execution(SAME_THREAD) public class TestAccessControl extends AbstractTestQueryFramework { From 11e85304d052960eba50695df9d55a09cb3f8414 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Thu, 21 Dec 2023 12:01:15 +0100 Subject: [PATCH 187/350] Add tracing for query start Query start may be time consuming operation and should be traced. Especially in case of `DataDefinitionExecution`, the start may cover all of the query execution. --- .../java/io/trino/execution/SqlQueryManager.java | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/core/trino-main/src/main/java/io/trino/execution/SqlQueryManager.java b/core/trino-main/src/main/java/io/trino/execution/SqlQueryManager.java index fa3ee37b94545..acce256889772 100644 --- a/core/trino-main/src/main/java/io/trino/execution/SqlQueryManager.java +++ b/core/trino-main/src/main/java/io/trino/execution/SqlQueryManager.java @@ -22,6 +22,8 @@ import io.airlift.log.Logger; import io.airlift.units.DataSize; import io.airlift.units.Duration; +import io.opentelemetry.api.trace.Tracer; +import io.opentelemetry.context.Context; import io.trino.ExceededCpuLimitException; import io.trino.ExceededScanLimitException; import io.trino.Session; @@ -55,6 +57,7 @@ import static io.trino.SystemSessionProperties.getQueryMaxScanPhysicalBytes; import static io.trino.execution.QueryState.RUNNING; import static io.trino.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR; +import static io.trino.tracing.ScopedSpan.scopedSpan; import static java.lang.String.format; import static java.util.Objects.requireNonNull; import static java.util.concurrent.Executors.newCachedThreadPool; @@ -67,6 +70,7 @@ public class SqlQueryManager private static final Logger log = Logger.get(SqlQueryManager.class); private final ClusterMemoryManager memoryManager; + private final Tracer tracer; private final QueryTracker queryTracker; private final Duration maxQueryCpuTime; @@ -79,9 +83,10 @@ public class SqlQueryManager private final ThreadPoolExecutorMBean queryManagementExecutorMBean; @Inject - public SqlQueryManager(ClusterMemoryManager memoryManager, QueryManagerConfig queryManagerConfig) + public SqlQueryManager(ClusterMemoryManager memoryManager, Tracer tracer, QueryManagerConfig queryManagerConfig) { this.memoryManager = requireNonNull(memoryManager, "memoryManager is null"); + this.tracer = requireNonNull(tracer, "tracer is null"); this.maxQueryCpuTime = queryManagerConfig.getQueryMaxCpuTime(); this.maxQueryScanPhysicalBytes = queryManagerConfig.getQueryMaxScanPhysicalBytes(); @@ -253,7 +258,11 @@ public void createQuery(QueryExecution queryExecution) }); try (SetThreadName ignored = new SetThreadName("Query-%s", queryExecution.getQueryId())) { - queryExecution.start(); + try (var ignoredStartScope = scopedSpan(tracer.spanBuilder("query-start") + .setParent(Context.current().with(queryExecution.getSession().getQuerySpan())) + .startSpan())) { + queryExecution.start(); + } } } From 03011754d058a4bff1ded6eebf8bc83d2ce97a75 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Thu, 21 Dec 2023 12:38:37 +0100 Subject: [PATCH 188/350] Fix code indentation --- .../plugin/deltalake/TestDeltaLakeBasic.java | 68 +++++++++---------- 1 file changed, 34 insertions(+), 34 deletions(-) diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeBasic.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeBasic.java index 2d74baea49f71..0d649b1ed22ef 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeBasic.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeBasic.java @@ -309,8 +309,8 @@ private void testOptimizeWithColumnMappingMode(String columnMappingMode) // Verify optimized parquet file contains the expected physical id and name TrinoInputFile inputFile = new LocalInputFile(tableLocation.resolve(addFileEntry.getPath()).toFile()); ParquetMetadata parquetMetadata = MetadataReader.readFooter( - new TrinoParquetDataSource(inputFile, new ParquetReaderOptions(), new FileFormatDataSourceStats()), - Optional.empty()); + new TrinoParquetDataSource(inputFile, new ParquetReaderOptions(), new FileFormatDataSourceStats()), + Optional.empty()); FileMetaData fileMetaData = parquetMetadata.getFileMetaData(); PrimitiveType physicalType = getOnlyElement(fileMetaData.getSchema().getColumns().iterator()).getPrimitiveType(); assertThat(physicalType.getName()).isEqualTo(physicalName); @@ -590,23 +590,23 @@ private void testDeltaTimestampNtz(ZoneId sessionZone) assertThat(query(session, "SELECT * FROM " + tableName)) .matches(""" - VALUES - NULL, - TIMESTAMP '-9999-12-31 23:59:59.999999', - TIMESTAMP '-0001-01-01 00:00:00', - TIMESTAMP '0000-01-01 00:00:00', - TIMESTAMP '1582-10-05 00:00:00', - TIMESTAMP '1582-10-14 23:59:59.999999', - TIMESTAMP '2020-12-31 01:02:03.123456', - TIMESTAMP '9999-12-31 23:59:59.999999' - """); + VALUES + NULL, + TIMESTAMP '-9999-12-31 23:59:59.999999', + TIMESTAMP '-0001-01-01 00:00:00', + TIMESTAMP '0000-01-01 00:00:00', + TIMESTAMP '1582-10-05 00:00:00', + TIMESTAMP '1582-10-14 23:59:59.999999', + TIMESTAMP '2020-12-31 01:02:03.123456', + TIMESTAMP '9999-12-31 23:59:59.999999' + """); assertQuery( "SHOW STATS FOR " + tableName, """ - VALUES - ('x', null, null, 0.125, null, null, null), - (null, null, null, null, 8.0, null, null) - """); + VALUES + ('x', null, null, 0.125, null, null, null), + (null, null, null, null, 8.0, null, null) + """); // Verify the connector can insert into tables created by Databricks assertUpdate(session, "INSERT INTO " + tableName + " VALUES TIMESTAMP '2023-01-02 03:04:05.123456'", 1); @@ -833,11 +833,11 @@ private void testTimestampNtzPartitioned(ZoneId sessionZone) assertQuery( "SHOW STATS FOR " + tableName, """ - VALUES - ('id', null, null, 0.0, null, 1, 8), - ('part', null, 7.0, 0.125, null, null, null), - (null, null, null, null, 8.0, null, null) - """); + VALUES + ('id', null, null, 0.0, null, 1, 8), + ('part', null, 7.0, 0.125, null, null, null), + (null, null, null, null, 8.0, null, null) + """); // Verify the connector can insert into tables created by Databricks assertUpdate(session, "INSERT INTO " + tableName + " VALUES (9, TIMESTAMP '2023-01-02 03:04:05.123456')", 1); @@ -997,22 +997,22 @@ public void testStatsWithMinMaxValuesAsNulls() assertQuery( "SELECT * FROM stats_with_minmax_nulls", """ - VALUES - (0, 1), - (1, 2), - (3, 4), - (3, 7), - (NULL, NULL), - (NULL, NULL) - """); + VALUES + (0, 1), + (1, 2), + (3, 4), + (3, 7), + (NULL, NULL), + (NULL, NULL) + """); assertQuery( "SHOW STATS FOR stats_with_minmax_nulls", """ - VALUES - ('id', null, null, 0.3333333333333333, null, 0, 3), - ('id2', null, null, 0.3333333333333333, null, 1, 7), - (null, null, null, null, 6.0, null, null) - """); + VALUES + ('id', null, null, 0.3333333333333333, null, 0, 3), + ('id2', null, null, 0.3333333333333333, null, 1, 7), + (null, null, null, null, 6.0, null, null) + """); } /** From f6ea5b452c0d77e150ff2b07cf30a3be5bec5e0c Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Thu, 21 Dec 2023 12:37:26 +0100 Subject: [PATCH 189/350] Remove redundant PER_CLASS & CONCURRENT annotations Tests extending `AbstractTestQueryFramework` inherit `PER_CLASS` lifecycle and `CONCURRENT` behavior and that should be expected given how `AbstractTestQueryFramework` works. No need to define that in subclasses, and indeed most subclasses do not define these behaviors again. --- .../io/trino/plugin/cassandra/TestCassandraTypeMapping.java | 3 --- .../io/trino/plugin/deltalake/TestDeltaLakeAdlsStorage.java | 3 --- .../java/io/trino/plugin/deltalake/TestDeltaLakeBasic.java | 3 --- .../deltalake/TestDeltaLakeFlushMetadataCacheProcedure.java | 3 --- .../trino/plugin/deltalake/TestDeltaLakePartitioning.java | 3 --- .../trino/plugin/deltalake/TestDeltaLakeReadTimestamps.java | 3 --- .../TestDeltaLakeSharedHiveMetastoreWithViews.java | 3 --- .../plugin/deltalake/TestDeltaLakeTableStatistics.java | 3 --- .../java/io/trino/plugin/deltalake/TestSplitPruning.java | 3 --- .../TestDeltaLakeConcurrentModificationGlueMetastore.java | 3 --- .../glue/TestDeltaLakeRenameToWithGlueMetastore.java | 3 --- .../metastore/glue/TestDeltaLakeViewsGlueMetastore.java | 3 --- .../java/io/trino/plugin/druid/TestDruidTypeMapping.java | 3 --- .../plugin/elasticsearch/TestElasticsearchBackpressure.java | 3 --- .../test/java/io/trino/plugin/hive/TestHive3OnDataLake.java | 3 --- .../hive/TestHiveConcurrentModificationGlueMetastore.java | 3 --- .../src/test/java/io/trino/plugin/hive/TestShowStats.java | 3 --- .../cache/TestCachingHiveMetastoreWithQueryRunner.java | 3 --- .../plugin/hive/s3/TestS3FileSystemAccessOperations.java | 3 --- .../iceberg/TestIcebergGetTableStatisticsOperations.java | 3 --- .../io/trino/plugin/iceberg/TestIcebergMetadataListing.java | 3 --- .../trino/plugin/iceberg/TestIcebergReadVersionedTable.java | 3 --- .../iceberg/TestIcebergReadVersionedTableByTemporal.java | 3 --- .../io/trino/plugin/iceberg/TestIcebergSplitSource.java | 3 --- .../iceberg/TestIcebergTableWithExternalLocation.java | 3 --- .../test/java/io/trino/plugin/iceberg/TestIcebergV2.java | 3 --- .../catalog/glue/TestIcebergGlueCatalogSkipArchive.java | 3 --- .../glue/TestIcebergGlueTableOperationsInsertFailure.java | 3 --- .../plugin/kudu/TestKuduIntegrationDecimalColumns.java | 3 --- .../io/trino/plugin/mariadb/TestMariaDbTypeMapping.java | 6 ------ .../plugin/mongodb/TestMongoCaseInsensitiveMapping.java | 6 ------ .../java/io/trino/plugin/mysql/TestMySqlTypeMapping.java | 6 ------ .../io/trino/plugin/phoenix5/TestPhoenixTypeMapping.java | 6 ------ .../trino/plugin/postgresql/TestPostgreSqlTypeMapping.java | 6 ------ .../plugin/singlestore/TestSingleStoreTypeMapping.java | 6 ------ .../informationschema/TestInformationSchemaConnector.java | 3 --- .../io/trino/connector/system/runtime/TestKillQuery.java | 3 --- .../java/io/trino/execution/TestEventListenerBasic.java | 3 --- .../src/test/java/io/trino/execution/TestQueryTracker.java | 3 --- .../io/trino/execution/TestRefreshMaterializedView.java | 3 --- .../io/trino/tests/TestGetTableStatisticsOperations.java | 3 --- .../src/test/java/io/trino/tests/TestProcedureCall.java | 3 --- 42 files changed, 144 deletions(-) diff --git a/plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/TestCassandraTypeMapping.java b/plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/TestCassandraTypeMapping.java index 6894781f83697..887ce0291569c 100644 --- a/plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/TestCassandraTypeMapping.java +++ b/plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/TestCassandraTypeMapping.java @@ -31,7 +31,6 @@ import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; import java.time.LocalDate; import java.time.LocalDateTime; @@ -68,9 +67,7 @@ import static java.lang.String.format; import static java.time.ZoneOffset.UTC; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -@TestInstance(PER_CLASS) public class TestCassandraTypeMapping extends AbstractTestQueryFramework { diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeAdlsStorage.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeAdlsStorage.java index 066260afe5573..1c08f5bd22577 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeAdlsStorage.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeAdlsStorage.java @@ -22,7 +22,6 @@ import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; import org.testcontainers.containers.Network; import java.nio.file.Files; @@ -43,9 +42,7 @@ import static java.nio.charset.StandardCharsets.UTF_8; import static java.util.Objects.requireNonNull; import static java.util.UUID.randomUUID; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -@TestInstance(PER_CLASS) public class TestDeltaLakeAdlsStorage extends AbstractTestQueryFramework { diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeBasic.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeBasic.java index 0d649b1ed22ef..c1256054224a0 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeBasic.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeBasic.java @@ -46,7 +46,6 @@ import org.intellij.lang.annotations.Language; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; import org.junit.jupiter.api.parallel.Execution; import java.io.File; @@ -81,10 +80,8 @@ import static java.time.ZoneOffset.UTC; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.entry; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; -@TestInstance(PER_CLASS) @Execution(SAME_THREAD) public class TestDeltaLakeBasic extends AbstractTestQueryFramework diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeFlushMetadataCacheProcedure.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeFlushMetadataCacheProcedure.java index 4bbe568ba3a70..b0f6af428db21 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeFlushMetadataCacheProcedure.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeFlushMetadataCacheProcedure.java @@ -21,7 +21,6 @@ import io.trino.testing.QueryRunner; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; import static io.trino.plugin.deltalake.DeltaLakeQueryRunner.DELTA_CATALOG; import static io.trino.plugin.deltalake.DeltaLakeQueryRunner.createS3DeltaLakeQueryRunner; @@ -29,9 +28,7 @@ import static io.trino.plugin.hive.containers.HiveHadoop.HIVE3_IMAGE; import static io.trino.testing.TestingNames.randomNameSuffix; import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -@TestInstance(PER_CLASS) public class TestDeltaLakeFlushMetadataCacheProcedure extends AbstractTestQueryFramework { diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakePartitioning.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakePartitioning.java index d614ebc19c035..24d01476e1c10 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakePartitioning.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakePartitioning.java @@ -18,15 +18,12 @@ import io.trino.testing.QueryRunner; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; import static io.trino.plugin.deltalake.DeltaLakeQueryRunner.DELTA_CATALOG; import static io.trino.plugin.deltalake.DeltaLakeQueryRunner.createDeltaLakeQueryRunner; import static io.trino.testing.TestingNames.randomNameSuffix; import static java.lang.String.format; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -@TestInstance(PER_CLASS) public class TestDeltaLakePartitioning extends AbstractTestQueryFramework { diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeReadTimestamps.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeReadTimestamps.java index 7201465bf0432..8e1708a3dfb10 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeReadTimestamps.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeReadTimestamps.java @@ -21,7 +21,6 @@ import io.trino.testing.QueryRunner; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; import java.time.Instant; import java.time.LocalDateTime; @@ -49,9 +48,7 @@ import static java.time.temporal.ChronoField.MONTH_OF_YEAR; import static java.time.temporal.ChronoField.YEAR; import static java.util.stream.Collectors.joining; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -@TestInstance(PER_CLASS) public class TestDeltaLakeReadTimestamps extends AbstractTestQueryFramework { diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSharedHiveMetastoreWithViews.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSharedHiveMetastoreWithViews.java index e84783eb94a81..6cd482390b335 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSharedHiveMetastoreWithViews.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSharedHiveMetastoreWithViews.java @@ -21,7 +21,6 @@ import io.trino.testing.QueryRunner; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; import java.util.Map; @@ -31,9 +30,7 @@ import static io.trino.testing.containers.Minio.MINIO_SECRET_KEY; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -@TestInstance(PER_CLASS) public class TestDeltaLakeSharedHiveMetastoreWithViews extends AbstractTestQueryFramework { diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeTableStatistics.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeTableStatistics.java index 9807cdc6a37e3..5562072fc5dd5 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeTableStatistics.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeTableStatistics.java @@ -19,15 +19,12 @@ import io.trino.testing.QueryRunner; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; import static io.trino.plugin.deltalake.DeltaLakeQueryRunner.DELTA_CATALOG; import static io.trino.plugin.deltalake.DeltaLakeQueryRunner.createDeltaLakeQueryRunner; import static io.trino.testing.TestingNames.randomNameSuffix; import static java.lang.String.format; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -@TestInstance(PER_CLASS) public class TestDeltaLakeTableStatistics extends AbstractTestQueryFramework { diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestSplitPruning.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestSplitPruning.java index 99ff36ebfa627..b2546c7e76505 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestSplitPruning.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestSplitPruning.java @@ -24,7 +24,6 @@ import org.intellij.lang.annotations.Language; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; import java.util.Arrays; import java.util.List; @@ -36,9 +35,7 @@ import static io.trino.plugin.deltalake.DeltaLakeQueryRunner.createDeltaLakeQueryRunner; import static java.lang.String.format; import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -@TestInstance(PER_CLASS) public class TestSplitPruning extends AbstractTestQueryFramework { diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeConcurrentModificationGlueMetastore.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeConcurrentModificationGlueMetastore.java index 04092fe237db9..c596157b662b3 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeConcurrentModificationGlueMetastore.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeConcurrentModificationGlueMetastore.java @@ -28,7 +28,6 @@ import io.trino.testing.QueryRunner; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; import java.io.IOException; import java.lang.reflect.InvocationTargetException; @@ -47,9 +46,7 @@ import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.testing.TestingSession.testSessionBuilder; import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -@TestInstance(PER_CLASS) public class TestDeltaLakeConcurrentModificationGlueMetastore extends AbstractTestQueryFramework { diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeRenameToWithGlueMetastore.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeRenameToWithGlueMetastore.java index 4e15c1e55eef9..f7a2ff52b92b5 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeRenameToWithGlueMetastore.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeRenameToWithGlueMetastore.java @@ -21,16 +21,13 @@ import io.trino.testing.QueryRunner; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; import java.nio.file.Path; import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.testing.TestingSession.testSessionBuilder; import static java.lang.String.format; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -@TestInstance(PER_CLASS) public class TestDeltaLakeRenameToWithGlueMetastore extends AbstractTestQueryFramework { diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeViewsGlueMetastore.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeViewsGlueMetastore.java index 9fee324abe606..6a4cc5a7a1e67 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeViewsGlueMetastore.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeViewsGlueMetastore.java @@ -24,7 +24,6 @@ import io.trino.testing.sql.TestView; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; import java.io.IOException; import java.nio.file.Path; @@ -37,9 +36,7 @@ import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.testing.TestingSession.testSessionBuilder; import static java.lang.String.format; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -@TestInstance(PER_CLASS) public class TestDeltaLakeViewsGlueMetastore extends AbstractTestQueryFramework { diff --git a/plugin/trino-druid/src/test/java/io/trino/plugin/druid/TestDruidTypeMapping.java b/plugin/trino-druid/src/test/java/io/trino/plugin/druid/TestDruidTypeMapping.java index a13cea76c9b01..c9e9255e1b6ac 100644 --- a/plugin/trino-druid/src/test/java/io/trino/plugin/druid/TestDruidTypeMapping.java +++ b/plugin/trino-druid/src/test/java/io/trino/plugin/druid/TestDruidTypeMapping.java @@ -23,7 +23,6 @@ import io.trino.testing.datatype.SqlDataTypeTest; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; import org.junit.jupiter.api.parallel.Isolated; import java.io.BufferedWriter; @@ -42,9 +41,7 @@ import static java.util.Objects.requireNonNull; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -@TestInstance(PER_CLASS) @Isolated public class TestDruidTypeMapping extends AbstractTestQueryFramework diff --git a/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchBackpressure.java b/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchBackpressure.java index dbaf931760664..bad037ce2bd80 100644 --- a/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchBackpressure.java +++ b/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchBackpressure.java @@ -19,16 +19,13 @@ import io.trino.testing.QueryRunner; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; import org.testcontainers.containers.Network; import java.io.IOException; import static io.trino.plugin.elasticsearch.ElasticsearchQueryRunner.createElasticsearchQueryRunner; import static io.trino.tpch.TpchTable.ORDERS; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -@TestInstance(PER_CLASS) public class TestElasticsearchBackpressure extends AbstractTestQueryFramework { diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHive3OnDataLake.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHive3OnDataLake.java index 24d40204c58f1..79bb9c2e42bdb 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHive3OnDataLake.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHive3OnDataLake.java @@ -34,7 +34,6 @@ import io.trino.testing.minio.MinioClient; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; import java.text.DateFormat; import java.text.SimpleDateFormat; @@ -64,9 +63,7 @@ import static java.util.stream.Collectors.joining; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -@TestInstance(PER_CLASS) public class TestHive3OnDataLake extends AbstractTestQueryFramework { diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveConcurrentModificationGlueMetastore.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveConcurrentModificationGlueMetastore.java index fb08016d7200a..b2bb2f0a7b3aa 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveConcurrentModificationGlueMetastore.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveConcurrentModificationGlueMetastore.java @@ -26,7 +26,6 @@ import io.trino.testing.QueryRunner; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; import java.io.IOException; import java.lang.reflect.InvocationTargetException; @@ -44,9 +43,7 @@ import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.testing.TestingSession.testSessionBuilder; import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -@TestInstance(PER_CLASS) public class TestHiveConcurrentModificationGlueMetastore extends AbstractTestQueryFramework { diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestShowStats.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestShowStats.java index 4ed0fd12490b0..844cc1aef3f44 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestShowStats.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestShowStats.java @@ -19,15 +19,12 @@ import io.trino.testing.QueryRunner; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; import static io.trino.SystemSessionProperties.PREFER_PARTIAL_AGGREGATION; import static io.trino.SystemSessionProperties.USE_PARTIAL_DISTINCT_LIMIT; import static io.trino.SystemSessionProperties.USE_PARTIAL_TOPN; import static io.trino.tpch.TpchTable.NATION; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -@TestInstance(PER_CLASS) public class TestShowStats extends AbstractTestQueryFramework { diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastoreWithQueryRunner.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastoreWithQueryRunner.java index 93b184573fd70..f75c6cc7edded 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastoreWithQueryRunner.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastoreWithQueryRunner.java @@ -28,7 +28,6 @@ import io.trino.testing.DistributedQueryRunner; import io.trino.testing.QueryRunner; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; import org.junit.jupiter.api.parallel.Execution; import java.util.List; @@ -40,10 +39,8 @@ import static io.trino.testing.TestingSession.testSessionBuilder; import static java.util.Collections.nCopies; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; -@TestInstance(PER_CLASS) @Execution(SAME_THREAD) public class TestCachingHiveMetastoreWithQueryRunner extends AbstractTestQueryFramework diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/TestS3FileSystemAccessOperations.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/TestS3FileSystemAccessOperations.java index 4e41f64437152..8a7e8b6e2361e 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/TestS3FileSystemAccessOperations.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/TestS3FileSystemAccessOperations.java @@ -28,7 +28,6 @@ import org.intellij.lang.annotations.Language; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; import org.junit.jupiter.api.parallel.Execution; import org.junit.jupiter.api.parallel.ExecutionMode; @@ -45,9 +44,7 @@ import static io.trino.testing.containers.Minio.MINIO_REGION; import static io.trino.testing.containers.Minio.MINIO_SECRET_KEY; import static java.util.stream.Collectors.toCollection; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -@TestInstance(PER_CLASS) @Execution(ExecutionMode.SAME_THREAD) // S3 request counters shares mutable state so can't be run from many threads simultaneously public class TestS3FileSystemAccessOperations extends AbstractTestQueryFramework diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergGetTableStatisticsOperations.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergGetTableStatisticsOperations.java index 36bdc4ba43ae7..7762e6cbb9bfd 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergGetTableStatisticsOperations.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergGetTableStatisticsOperations.java @@ -29,7 +29,6 @@ import org.intellij.lang.annotations.Language; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; import org.junit.jupiter.api.extension.RegisterExtension; import org.junit.jupiter.api.parallel.Execution; @@ -46,12 +45,10 @@ import static io.trino.sql.planner.LogicalPlanner.Stage.OPTIMIZED_AND_VALIDATED; import static io.trino.testing.TestingSession.testSessionBuilder; import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; // Cost-based optimizers' behaviors are affected by the statistics returned by the Connectors. Here is to count the getTableStatistics calls // when CBOs work with Iceberg Connector. -@TestInstance(PER_CLASS) @Execution(SAME_THREAD) public class TestIcebergGetTableStatisticsOperations extends AbstractTestQueryFramework diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetadataListing.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetadataListing.java index b34ca4a5e5e8d..43cb906c88067 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetadataListing.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetadataListing.java @@ -30,7 +30,6 @@ import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; import java.io.File; import java.util.Optional; @@ -39,9 +38,7 @@ import static io.trino.spi.security.SelectedRole.Type.ROLE; import static io.trino.testing.TestingSession.testSessionBuilder; import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -@TestInstance(PER_CLASS) public class TestIcebergMetadataListing extends AbstractTestQueryFramework { diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergReadVersionedTable.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergReadVersionedTable.java index 92374ea8742aa..056470675c346 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergReadVersionedTable.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergReadVersionedTable.java @@ -17,7 +17,6 @@ import io.trino.testing.DistributedQueryRunner; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; import java.time.Instant; import java.time.ZonedDateTime; @@ -26,9 +25,7 @@ import static io.trino.plugin.iceberg.IcebergQueryRunner.createIcebergQueryRunner; import static java.lang.String.format; import static java.time.ZoneOffset.UTC; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -@TestInstance(PER_CLASS) public class TestIcebergReadVersionedTable extends AbstractTestQueryFramework { diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergReadVersionedTableByTemporal.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergReadVersionedTableByTemporal.java index f03fb453f92a3..68b1748b441d4 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergReadVersionedTableByTemporal.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergReadVersionedTableByTemporal.java @@ -21,7 +21,6 @@ import io.trino.testing.containers.Minio; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; import static io.trino.plugin.iceberg.IcebergQueryRunner.ICEBERG_CATALOG; import static io.trino.testing.TestingNames.randomNameSuffix; @@ -29,9 +28,7 @@ import static io.trino.testing.containers.Minio.MINIO_SECRET_KEY; import static java.lang.String.format; import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -@TestInstance(PER_CLASS) public class TestIcebergReadVersionedTableByTemporal extends AbstractTestQueryFramework { diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergSplitSource.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergSplitSource.java index 98504649f63ce..8cbc01068fc9e 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergSplitSource.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergSplitSource.java @@ -53,7 +53,6 @@ import org.apache.iceberg.types.Types; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; import org.junit.jupiter.api.Timeout; import java.io.File; @@ -77,9 +76,7 @@ import static io.trino.tpch.TpchTable.NATION; import static java.util.concurrent.TimeUnit.SECONDS; import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -@TestInstance(PER_CLASS) public class TestIcebergSplitSource extends AbstractTestQueryFramework { diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergTableWithExternalLocation.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergTableWithExternalLocation.java index 1c297f3edac57..bf5f549480cab 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergTableWithExternalLocation.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergTableWithExternalLocation.java @@ -23,7 +23,6 @@ import io.trino.testing.MaterializedResult; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; import java.io.File; import java.io.IOException; @@ -37,9 +36,7 @@ import static io.trino.testing.TestingNames.randomNameSuffix; import static java.lang.String.format; import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -@TestInstance(PER_CLASS) public class TestIcebergTableWithExternalLocation extends AbstractTestQueryFramework { diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergV2.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergV2.java index a7e9b8f42650a..7fb65254de97d 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergV2.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergV2.java @@ -64,7 +64,6 @@ import org.apache.iceberg.types.Types; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; import java.io.Closeable; import java.nio.ByteBuffer; @@ -99,9 +98,7 @@ import static org.apache.iceberg.TableProperties.SPLIT_SIZE; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -@TestInstance(PER_CLASS) public class TestIcebergV2 extends AbstractTestQueryFramework { diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogSkipArchive.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogSkipArchive.java index 178b5f7ae50e8..05c385273f750 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogSkipArchive.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogSkipArchive.java @@ -35,7 +35,6 @@ import org.apache.iceberg.io.FileIO; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; import java.io.File; import java.nio.file.Files; @@ -56,14 +55,12 @@ import static io.trino.type.InternalTypeManager.TESTING_TYPE_MANAGER; import static org.apache.iceberg.BaseMetastoreTableOperations.METADATA_LOCATION_PROP; import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; /* * The test currently uses AWS Default Credential Provider Chain, * See https://docs.aws.amazon.com/sdk-for-java/v1/developer-guide/credentials.html#credentials-default * on ways to set your AWS credentials which will be needed to run this test. */ -@TestInstance(PER_CLASS) public class TestIcebergGlueCatalogSkipArchive extends AbstractTestQueryFramework { diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueTableOperationsInsertFailure.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueTableOperationsInsertFailure.java index 10a3447da4354..db5bc565e2e14 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueTableOperationsInsertFailure.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueTableOperationsInsertFailure.java @@ -28,7 +28,6 @@ import org.apache.iceberg.exceptions.CommitStateUnknownException; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; import java.lang.reflect.InvocationTargetException; import java.nio.file.Files; @@ -42,14 +41,12 @@ import static io.trino.testing.TestingSession.testSessionBuilder; import static java.lang.String.format; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; /* * The test currently uses AWS Default Credential Provider Chain, * See https://docs.aws.amazon.com/sdk-for-java/v1/developer-guide/credentials.html#credentials-default * on ways to set your AWS credentials which will be needed to run this test. */ -@TestInstance(PER_CLASS) public class TestIcebergGlueTableOperationsInsertFailure extends AbstractTestQueryFramework { diff --git a/plugin/trino-kudu/src/test/java/io/trino/plugin/kudu/TestKuduIntegrationDecimalColumns.java b/plugin/trino-kudu/src/test/java/io/trino/plugin/kudu/TestKuduIntegrationDecimalColumns.java index c2420d76be866..b326915c61ef2 100644 --- a/plugin/trino-kudu/src/test/java/io/trino/plugin/kudu/TestKuduIntegrationDecimalColumns.java +++ b/plugin/trino-kudu/src/test/java/io/trino/plugin/kudu/TestKuduIntegrationDecimalColumns.java @@ -20,15 +20,12 @@ import io.trino.testing.sql.TrinoSqlExecutor; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; import static io.trino.plugin.kudu.KuduQueryRunnerFactory.createKuduQueryRunner; import static java.lang.String.format; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.offset; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -@TestInstance(PER_CLASS) public class TestKuduIntegrationDecimalColumns extends AbstractTestQueryFramework { diff --git a/plugin/trino-mariadb/src/test/java/io/trino/plugin/mariadb/TestMariaDbTypeMapping.java b/plugin/trino-mariadb/src/test/java/io/trino/plugin/mariadb/TestMariaDbTypeMapping.java index ad153999d3c93..853b126a12132 100644 --- a/plugin/trino-mariadb/src/test/java/io/trino/plugin/mariadb/TestMariaDbTypeMapping.java +++ b/plugin/trino-mariadb/src/test/java/io/trino/plugin/mariadb/TestMariaDbTypeMapping.java @@ -30,8 +30,6 @@ import io.trino.testing.sql.TrinoSqlExecutor; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; -import org.junit.jupiter.api.parallel.Execution; import java.math.RoundingMode; import java.time.LocalDate; @@ -66,14 +64,10 @@ import static java.math.RoundingMode.UNNECESSARY; import static java.time.ZoneOffset.UTC; import static java.util.Arrays.asList; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; /** * @see MariaDB data types */ -@TestInstance(PER_CLASS) -@Execution(CONCURRENT) public class TestMariaDbTypeMapping extends AbstractTestQueryFramework { diff --git a/plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/TestMongoCaseInsensitiveMapping.java b/plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/TestMongoCaseInsensitiveMapping.java index d12cda7b4e31b..1bb94415198bc 100644 --- a/plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/TestMongoCaseInsensitiveMapping.java +++ b/plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/TestMongoCaseInsensitiveMapping.java @@ -22,19 +22,13 @@ import org.bson.Document; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; -import org.junit.jupiter.api.parallel.Execution; import static io.trino.plugin.mongodb.MongoQueryRunner.createMongoClient; import static io.trino.plugin.mongodb.MongoQueryRunner.createMongoQueryRunner; import static io.trino.testing.TestingNames.randomNameSuffix; import static java.util.Locale.ENGLISH; import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; -@TestInstance(PER_CLASS) -@Execution(CONCURRENT) public class TestMongoCaseInsensitiveMapping extends AbstractTestQueryFramework { diff --git a/plugin/trino-mysql/src/test/java/io/trino/plugin/mysql/TestMySqlTypeMapping.java b/plugin/trino-mysql/src/test/java/io/trino/plugin/mysql/TestMySqlTypeMapping.java index 2f99cd90d93b4..ff565235e4b4c 100644 --- a/plugin/trino-mysql/src/test/java/io/trino/plugin/mysql/TestMySqlTypeMapping.java +++ b/plugin/trino-mysql/src/test/java/io/trino/plugin/mysql/TestMySqlTypeMapping.java @@ -32,8 +32,6 @@ import org.intellij.lang.annotations.Language; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; -import org.junit.jupiter.api.parallel.Execution; import java.math.RoundingMode; import java.sql.Connection; @@ -77,11 +75,7 @@ import static java.util.Arrays.asList; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; -@TestInstance(PER_CLASS) -@Execution(CONCURRENT) public class TestMySqlTypeMapping extends AbstractTestQueryFramework { diff --git a/plugin/trino-phoenix5/src/test/java/io/trino/plugin/phoenix5/TestPhoenixTypeMapping.java b/plugin/trino-phoenix5/src/test/java/io/trino/plugin/phoenix5/TestPhoenixTypeMapping.java index 672b4e75fea41..9bd734daff9be 100644 --- a/plugin/trino-phoenix5/src/test/java/io/trino/plugin/phoenix5/TestPhoenixTypeMapping.java +++ b/plugin/trino-phoenix5/src/test/java/io/trino/plugin/phoenix5/TestPhoenixTypeMapping.java @@ -31,8 +31,6 @@ import org.intellij.lang.annotations.Language; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; -import org.junit.jupiter.api.parallel.Execution; import java.math.RoundingMode; import java.time.LocalDate; @@ -70,14 +68,10 @@ import static java.util.Arrays.asList; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; /** * @see Phoenix data types */ -@TestInstance(PER_CLASS) -@Execution(CONCURRENT) public class TestPhoenixTypeMapping extends AbstractTestQueryFramework { diff --git a/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestPostgreSqlTypeMapping.java b/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestPostgreSqlTypeMapping.java index 49db947ecb560..c467790e36492 100644 --- a/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestPostgreSqlTypeMapping.java +++ b/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestPostgreSqlTypeMapping.java @@ -40,8 +40,6 @@ import org.intellij.lang.annotations.Language; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; -import org.junit.jupiter.api.parallel.Execution; import java.math.BigDecimal; import java.math.RoundingMode; @@ -107,11 +105,7 @@ import static java.util.stream.Collectors.toList; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; -@TestInstance(PER_CLASS) -@Execution(CONCURRENT) public class TestPostgreSqlTypeMapping extends AbstractTestQueryFramework { diff --git a/plugin/trino-singlestore/src/test/java/io/trino/plugin/singlestore/TestSingleStoreTypeMapping.java b/plugin/trino-singlestore/src/test/java/io/trino/plugin/singlestore/TestSingleStoreTypeMapping.java index 9c89484014d83..d924cdd7ce3e5 100644 --- a/plugin/trino-singlestore/src/test/java/io/trino/plugin/singlestore/TestSingleStoreTypeMapping.java +++ b/plugin/trino-singlestore/src/test/java/io/trino/plugin/singlestore/TestSingleStoreTypeMapping.java @@ -31,8 +31,6 @@ import io.trino.testing.sql.TrinoSqlExecutor; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; -import org.junit.jupiter.api.parallel.Execution; import java.math.RoundingMode; import java.time.LocalDate; @@ -74,14 +72,10 @@ import static java.time.ZoneOffset.UTC; import static java.util.Arrays.asList; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; /** * @see SingleStore (MemSQL) data types */ -@TestInstance(PER_CLASS) -@Execution(CONCURRENT) public class TestSingleStoreTypeMapping extends AbstractTestQueryFramework { diff --git a/testing/trino-tests/src/test/java/io/trino/connector/informationschema/TestInformationSchemaConnector.java b/testing/trino-tests/src/test/java/io/trino/connector/informationschema/TestInformationSchemaConnector.java index 18e23395d0c04..94a99fa107a9b 100644 --- a/testing/trino-tests/src/test/java/io/trino/connector/informationschema/TestInformationSchemaConnector.java +++ b/testing/trino-tests/src/test/java/io/trino/connector/informationschema/TestInformationSchemaConnector.java @@ -26,7 +26,6 @@ import org.intellij.lang.annotations.Language; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; import org.junit.jupiter.api.Timeout; import java.util.stream.IntStream; @@ -35,9 +34,7 @@ import static io.trino.testing.MultisetAssertions.assertMultisetsEqual; import static io.trino.testing.TestingSession.testSessionBuilder; import static java.util.stream.Collectors.joining; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -@TestInstance(PER_CLASS) public class TestInformationSchemaConnector extends AbstractTestQueryFramework { diff --git a/testing/trino-tests/src/test/java/io/trino/connector/system/runtime/TestKillQuery.java b/testing/trino-tests/src/test/java/io/trino/connector/system/runtime/TestKillQuery.java index 096a98f95f834..17c0a83e04a9a 100644 --- a/testing/trino-tests/src/test/java/io/trino/connector/system/runtime/TestKillQuery.java +++ b/testing/trino-tests/src/test/java/io/trino/connector/system/runtime/TestKillQuery.java @@ -23,7 +23,6 @@ import io.trino.testng.services.ReportOrphanedExecutors; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; import org.junit.jupiter.api.Timeout; import java.util.Optional; @@ -44,9 +43,7 @@ import static java.util.UUID.randomUUID; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -@TestInstance(PER_CLASS) public class TestKillQuery extends AbstractTestQueryFramework { diff --git a/testing/trino-tests/src/test/java/io/trino/execution/TestEventListenerBasic.java b/testing/trino-tests/src/test/java/io/trino/execution/TestEventListenerBasic.java index a583cfc40ec05..57cf979dcc8a5 100644 --- a/testing/trino-tests/src/test/java/io/trino/execution/TestEventListenerBasic.java +++ b/testing/trino-tests/src/test/java/io/trino/execution/TestEventListenerBasic.java @@ -53,7 +53,6 @@ import io.trino.testing.QueryRunner; import org.intellij.lang.annotations.Language; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.parallel.Execution; @@ -89,10 +88,8 @@ import static java.lang.String.format; import static java.util.UUID.randomUUID; import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; -@TestInstance(PER_CLASS) @Execution(SAME_THREAD) public class TestEventListenerBasic extends AbstractTestQueryFramework diff --git a/testing/trino-tests/src/test/java/io/trino/execution/TestQueryTracker.java b/testing/trino-tests/src/test/java/io/trino/execution/TestQueryTracker.java index 6e1d3bfe17e4e..14a810c9a6464 100644 --- a/testing/trino-tests/src/test/java/io/trino/execution/TestQueryTracker.java +++ b/testing/trino-tests/src/test/java/io/trino/execution/TestQueryTracker.java @@ -24,7 +24,6 @@ import io.trino.testing.QueryRunner; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; import org.junit.jupiter.api.Timeout; import java.util.concurrent.CountDownLatch; @@ -33,11 +32,9 @@ import static io.trino.spi.type.VarcharType.VARCHAR; import static io.trino.testing.TestingSession.testSessionBuilder; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; // Tests need to finish before strict timeouts. Any background work // may make them flaky -@TestInstance(PER_CLASS) public class TestQueryTracker extends AbstractTestQueryFramework { diff --git a/testing/trino-tests/src/test/java/io/trino/execution/TestRefreshMaterializedView.java b/testing/trino-tests/src/test/java/io/trino/execution/TestRefreshMaterializedView.java index 316f87b8b5134..d1dce7c43f18f 100644 --- a/testing/trino-tests/src/test/java/io/trino/execution/TestRefreshMaterializedView.java +++ b/testing/trino-tests/src/test/java/io/trino/execution/TestRefreshMaterializedView.java @@ -35,7 +35,6 @@ import org.intellij.lang.annotations.Language; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.parallel.Execution; @@ -54,10 +53,8 @@ import static java.util.concurrent.Executors.newCachedThreadPool; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; -@TestInstance(PER_CLASS) @Execution(SAME_THREAD) public class TestRefreshMaterializedView extends AbstractTestQueryFramework diff --git a/testing/trino-tests/src/test/java/io/trino/tests/TestGetTableStatisticsOperations.java b/testing/trino-tests/src/test/java/io/trino/tests/TestGetTableStatisticsOperations.java index 73591178989e8..a1f4817fa9e44 100644 --- a/testing/trino-tests/src/test/java/io/trino/tests/TestGetTableStatisticsOperations.java +++ b/testing/trino-tests/src/test/java/io/trino/tests/TestGetTableStatisticsOperations.java @@ -25,7 +25,6 @@ import org.intellij.lang.annotations.Language; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; import org.junit.jupiter.api.extension.RegisterExtension; import org.junit.jupiter.api.parallel.Execution; @@ -34,10 +33,8 @@ import static io.trino.testing.TestingSession.testSessionBuilder; import static io.trino.testing.TransactionBuilder.transaction; import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; -@TestInstance(PER_CLASS) @Execution(SAME_THREAD) public class TestGetTableStatisticsOperations extends AbstractTestQueryFramework diff --git a/testing/trino-tests/src/test/java/io/trino/tests/TestProcedureCall.java b/testing/trino-tests/src/test/java/io/trino/tests/TestProcedureCall.java index 8854489ed17c7..66eec5e26eb2e 100644 --- a/testing/trino-tests/src/test/java/io/trino/tests/TestProcedureCall.java +++ b/testing/trino-tests/src/test/java/io/trino/tests/TestProcedureCall.java @@ -26,7 +26,6 @@ import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; import java.util.List; @@ -35,9 +34,7 @@ import static java.util.Arrays.asList; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -@TestInstance(PER_CLASS) public class TestProcedureCall extends AbstractTestQueryFramework { From 6afc644ff8640741c74126ae9828c024034a45fe Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Thu, 21 Dec 2023 12:53:01 +0100 Subject: [PATCH 190/350] Remove unused field from TestIcebergMetadataListing --- .../iceberg/TestIcebergMetadataListing.java | 18 ------------------ 1 file changed, 18 deletions(-) diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetadataListing.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetadataListing.java index 43cb906c88067..cf2b0e92a40ab 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetadataListing.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetadataListing.java @@ -15,18 +15,13 @@ import com.google.common.collect.ImmutableMap; import io.trino.Session; -import io.trino.metadata.MaterializedViewDefinition; -import io.trino.metadata.QualifiedObjectName; import io.trino.plugin.hive.TestingHivePlugin; import io.trino.plugin.hive.metastore.HiveMetastore; import io.trino.plugin.hive.metastore.HiveMetastoreFactory; -import io.trino.spi.connector.SchemaTableName; import io.trino.spi.security.Identity; import io.trino.spi.security.SelectedRole; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.DistributedQueryRunner; -import io.trino.transaction.TransactionId; -import io.trino.transaction.TransactionManager; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; @@ -43,7 +38,6 @@ public class TestIcebergMetadataListing extends AbstractTestQueryFramework { private HiveMetastore metastore; - private SchemaTableName storageTable; @Override protected DistributedQueryRunner createQueryRunner() @@ -77,7 +71,6 @@ public void setUp() assertQuerySucceeds("CREATE TABLE iceberg.test_schema.iceberg_table2 (_double DOUBLE) WITH (partitioning = ARRAY['_double'])"); assertQuerySucceeds("CREATE MATERIALIZED VIEW iceberg.test_schema.iceberg_materialized_view AS " + "SELECT * FROM iceberg.test_schema.iceberg_table1"); - storageTable = getStorageTable("iceberg", "test_schema", "iceberg_materialized_view"); assertQuerySucceeds("CREATE VIEW iceberg.test_schema.iceberg_view AS SELECT * FROM iceberg.test_schema.iceberg_table1"); assertQuerySucceeds("CREATE TABLE hive.test_schema.hive_table (_double DOUBLE)"); @@ -148,15 +141,4 @@ public void testTableValidation() assertQuerySucceeds("SELECT * FROM iceberg.test_schema.iceberg_table1"); assertQueryFails("SELECT * FROM iceberg.test_schema.hive_table", "Not an Iceberg table: test_schema.hive_table"); } - - private SchemaTableName getStorageTable(String catalogName, String schemaName, String objectName) - { - TransactionManager transactionManager = getQueryRunner().getTransactionManager(); - TransactionId transactionId = transactionManager.beginTransaction(false); - Session session = getSession().beginTransactionId(transactionId, transactionManager, getQueryRunner().getAccessControl()); - Optional materializedView = getQueryRunner().getMetadata() - .getMaterializedView(session, new QualifiedObjectName(catalogName, schemaName, objectName)); - assertThat(materializedView).isPresent(); - return materializedView.get().getStorageTable().get().getSchemaTableName(); - } } From 2b3008060f18c638279bdde06d3d544824f8b12e Mon Sep 17 00:00:00 2001 From: Dejan Mircevski Date: Thu, 14 Dec 2023 19:09:09 -0500 Subject: [PATCH 191/350] Factor out common code in TestUniformNodeSelector --- .../scheduler/TestUniformNodeSelector.java | 26 ++++--------------- 1 file changed, 5 insertions(+), 21 deletions(-) diff --git a/core/trino-main/src/test/java/io/trino/execution/scheduler/TestUniformNodeSelector.java b/core/trino-main/src/test/java/io/trino/execution/scheduler/TestUniformNodeSelector.java index 299f8b6f153a4..d3565e3322f00 100644 --- a/core/trino-main/src/test/java/io/trino/execution/scheduler/TestUniformNodeSelector.java +++ b/core/trino-main/src/test/java/io/trino/execution/scheduler/TestUniformNodeSelector.java @@ -64,6 +64,9 @@ @TestInstance(PER_METHOD) public class TestUniformNodeSelector { + private static final InternalNode node1 = new InternalNode("node1", URI.create("http://10.0.0.1:13"), NodeVersion.UNKNOWN, false); + private static final InternalNode node2 = new InternalNode("node2", URI.create("http://10.0.0.1:12"), NodeVersion.UNKNOWN, false); + private final Set splits = new LinkedHashSet<>(); private FinalizerService finalizerService; private NodeTaskMap nodeTaskMap; private InMemoryNodeManager nodeManager; @@ -82,6 +85,8 @@ public void setUp() finalizerService = new FinalizerService(); nodeTaskMap = new NodeTaskMap(finalizerService); nodeManager = new InMemoryNodeManager(); + nodeManager.addNodes(node1); + nodeManager.addNodes(node2); nodeSchedulerConfig = new NodeSchedulerConfig() .setMaxSplitsPerNode(20) @@ -116,11 +121,6 @@ public void tearDown() @Test public void testQueueSizeAdjustmentScaleDown() { - InternalNode node1 = new InternalNode("node1", URI.create("http://10.0.0.1:13"), NodeVersion.UNKNOWN, false); - nodeManager.addNodes(node1); - InternalNode node2 = new InternalNode("node2", URI.create("http://10.0.0.1:12"), NodeVersion.UNKNOWN, false); - nodeManager.addNodes(node2); - TestingTicker ticker = new TestingTicker(); UniformNodeSelector.QueueSizeAdjuster queueSizeAdjuster = new UniformNodeSelector.QueueSizeAdjuster(10, 100, ticker); @@ -137,8 +137,6 @@ public void testQueueSizeAdjustmentScaleDown() false, queueSizeAdjuster); - Set splits = new LinkedHashSet<>(); - for (int i = 0; i < 20; i++) { splits.add(new Split(TEST_CATALOG_HANDLE, TestingSplit.createRemoteSplit())); } @@ -187,13 +185,6 @@ public void testQueueSizeAdjustmentScaleDown() @Test public void testQueueSizeAdjustmentAllNodes() { - InternalNode node1 = new InternalNode("node1", URI.create("http://10.0.0.1:13"), NodeVersion.UNKNOWN, false); - nodeManager.addNodes(node1); - InternalNode node2 = new InternalNode("node2", URI.create("http://10.0.0.1:12"), NodeVersion.UNKNOWN, false); - nodeManager.addNodes(node2); - - Set splits = new LinkedHashSet<>(); - for (int i = 0; i < 20 * 9; i++) { splits.add(new Split(TEST_CATALOG_HANDLE, TestingSplit.createRemoteSplit())); } @@ -246,13 +237,6 @@ public void testQueueSizeAdjustmentAllNodes() @Test public void testQueueSizeAdjustmentOneOfAll() { - InternalNode node1 = new InternalNode("node1", URI.create("http://10.0.0.1:13"), NodeVersion.UNKNOWN, false); - nodeManager.addNodes(node1); - InternalNode node2 = new InternalNode("node2", URI.create("http://10.0.0.1:12"), NodeVersion.UNKNOWN, false); - nodeManager.addNodes(node2); - - Set splits = new LinkedHashSet<>(); - for (int i = 0; i < 20 * 9; i++) { splits.add(new Split(TEST_CATALOG_HANDLE, TestingSplit.createRemoteSplit())); } From 83fdd35cefcab0791d273888ef932c8ea1bc1857 Mon Sep 17 00:00:00 2001 From: Dejan Mircevski Date: Thu, 21 Dec 2023 11:37:35 -0500 Subject: [PATCH 192/350] Warn about unexpected task updates We observed duplicate attempts to drop spoolingOutputStats in some logs. These warnings will help diagnose what's going on. --- .../trino/server/remotetask/TaskInfoFetcher.java | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/core/trino-main/src/main/java/io/trino/server/remotetask/TaskInfoFetcher.java b/core/trino-main/src/main/java/io/trino/server/remotetask/TaskInfoFetcher.java index 3d1b170521bd5..548ab94337980 100644 --- a/core/trino-main/src/main/java/io/trino/server/remotetask/TaskInfoFetcher.java +++ b/core/trino-main/src/main/java/io/trino/server/remotetask/TaskInfoFetcher.java @@ -250,9 +250,18 @@ synchronized void updateTaskInfo(TaskInfo newTaskInfo) TaskStatus localTaskStatus = taskStatusFetcher.getTaskStatus(); TaskStatus newRemoteTaskStatus = newTaskInfo.getTaskStatus(); + if (!newRemoteTaskStatus.getTaskId().equals(taskId)) { + log.warn("Task ID mismatch on remote task status. Member task ID is %s, but remote task ID is %s. This will confuse finalTaskInfo listeners.", + taskId.toString(), newRemoteTaskStatus.getTaskId().toString()); + } + if (localTaskStatus.getState().isDone() && newRemoteTaskStatus.getState().isDone() && localTaskStatus.getState() != newRemoteTaskStatus.getState()) { // prefer local newTaskInfo = newTaskInfo.withTaskStatus(localTaskStatus); + if (!localTaskStatus.getTaskId().equals(taskId)) { + log.warn("Task ID mismatch on local task status. Member task ID is %s, but status-fetcher ID is %s. This will confuse finalTaskInfo listeners.", + taskId.toString(), newRemoteTaskStatus.getTaskId().toString()); + } } if (estimatedMemory.isPresent()) { @@ -260,7 +269,10 @@ synchronized void updateTaskInfo(TaskInfo newTaskInfo) } if (newTaskInfo.getTaskStatus().getState().isDone()) { - spoolingOutputStats.compareAndSet(null, newTaskInfo.getOutputBuffers().getSpoolingOutputStats().orElse(null)); + boolean wasSet = spoolingOutputStats.compareAndSet(null, newTaskInfo.getOutputBuffers().getSpoolingOutputStats().orElse(null)); + if (wasSet && spoolingOutputStats.get() == null) { + log.warn("Task %s was updated to null spoolingOutputStats. Future calls to retrieveAndDropSpoolingOutputStats will fail.", taskId.toString()); + } newTaskInfo = newTaskInfo.pruneSpoolingOutputStats(); } From 797403f3438df28bcc24883cf46b3ddb315de597 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jan=20Wa=C5=9B?= Date: Fri, 8 Dec 2023 18:01:53 +0100 Subject: [PATCH 193/350] Add example SQL routine for formatting topn results --- docs/src/main/sphinx/routines/examples.md | 59 +++++++++++++++++++++++ 1 file changed, 59 insertions(+) diff --git a/docs/src/main/sphinx/routines/examples.md b/docs/src/main/sphinx/routines/examples.md index 3b6eab0162058..f586b2988b7c4 100644 --- a/docs/src/main/sphinx/routines/examples.md +++ b/docs/src/main/sphinx/routines/examples.md @@ -771,3 +771,62 @@ The preceding query produces the following output: A | 2023-01-01 | 2023-01-11 | 8 | 1.00 | 15.00 | 8.38 | ▃ ▄█▇ ▅▁▄▄ B | 2023-01-01 | 2023-01-11 | 7 | 1.00 | 4.00 | 2.39 | ▄▆▅ ▆ █▃▂ ``` + +## Top-N + +Trino already has a built-in [aggregate function](/functions/aggregate) called +`approx_most_frequent()`, that can calculate most frequently occurring values. +It returns a map with values as keys and number of occurrences as values. Maps +are not ordered, so when displayed, the entries can change places on subsequent +runs of the same query, and readers must still compare all frequencies to find +the one most frequent value. The following is a routine returns ordered results +as a string. + +```sql +FUNCTION format_topn(input map) +RETURNS VARCHAR +NOT DETERMINISTIC +BEGIN + DECLARE freq_separator VARCHAR DEFAULT '='; + DECLARE entry_separator VARCHAR DEFAULT ', '; + RETURN array_join(transform( + reverse(array_sort(transform( + transform( + map_entries(input), + r -> cast(r AS row(key varchar, value bigint)) + ), + r -> cast(row(r.value, r.key) AS row(value bigint, key varchar))) + )), + r -> r.key || freq_separator || cast(r.value as varchar)), + entry_separator); +END; +``` + +Following is an example query to count generated strings: + +```sql +WITH +data AS ( + SELECT lpad('', 3, chr(65+(s.num / 3))) AS value + FROM table(sequence(start=>1, stop=>10)) AS s(num) +), +aggregated AS ( + SELECT + array_agg(data.value ORDER BY data.value) AS all_values, + approx_most_frequent(3, data.value, 1000) AS top3 + FROM data +) +SELECT + a.all_values, + a.top3, + format_topn(a.top3) AS top3_formatted +FROM aggregated a; +``` + +The preceding query produces the following result: + +```text + all_values | top3 | top3_formatted +----------------------------------------------------+-----------------------+--------------------- + [AAA, AAA, BBB, BBB, BBB, CCC, CCC, CCC, DDD, DDD] | {AAA=2, CCC=3, BBB=3} | CCC=3, BBB=3, AAA=2 +``` From 259702fa21becd3d8cb8c56a21f17da6a4b92bdf Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Thu, 21 Dec 2023 17:09:49 +0100 Subject: [PATCH 194/350] Fix incorrectly concurrent tests Some tests being fixed simply fail because they do not support concurrent test method execution. Others worked, but could allocate lots of resources, potentially leading to OOM failures on CI. The tests where found by looking at 9a7cf10d16253d165253218ae8345f22e6d480c8 commit: git show 9a7cf10d16253d165253218ae8345f22e6d480c8 | grep -A 3 '@Test(singleThreaded = true)' --- .../test/java/io/trino/operator/TestExchangeOperator.java | 4 ++-- .../io/trino/plugin/redis/TestMinimalFunctionality.java | 3 +++ .../informationschema/TestInformationSchemaConnector.java | 3 +++ .../io/trino/connector/system/runtime/TestKillQuery.java | 3 +++ .../src/test/java/io/trino/execution/TestBeginQuery.java | 3 +++ .../io/trino/execution/TestCompletedEventWarnings.java | 4 ++-- .../io/trino/execution/TestEventListenerWithSplits.java | 3 +++ .../src/test/java/io/trino/execution/TestQueryTracker.java | 3 +++ .../src/test/java/io/trino/execution/TestQueues.java | 4 +++- .../execution/resourcegroups/db/TestEnvironments.java | 3 +++ .../io/trino/execution/resourcegroups/db/TestQueuesDb.java | 7 +++---- .../src/test/java/io/trino/memory/TestMemoryManager.java | 4 ++-- .../java/io/trino/security/TestSystemSecurityMetadata.java | 3 +++ .../src/test/java/io/trino/tests/TestGracefulShutdown.java | 4 ++-- .../src/test/java/io/trino/tests/TestMetadataManager.java | 4 ++-- .../src/test/java/io/trino/tests/TestProcedureCall.java | 3 +++ .../src/test/java/io/trino/tests/TestQueryManager.java | 4 ++-- 17 files changed, 45 insertions(+), 17 deletions(-) diff --git a/core/trino-main/src/test/java/io/trino/operator/TestExchangeOperator.java b/core/trino-main/src/test/java/io/trino/operator/TestExchangeOperator.java index bc6ed291249dc..01aa3b10e3bf4 100644 --- a/core/trino-main/src/test/java/io/trino/operator/TestExchangeOperator.java +++ b/core/trino-main/src/test/java/io/trino/operator/TestExchangeOperator.java @@ -60,10 +60,10 @@ import static java.util.concurrent.Executors.newScheduledThreadPool; import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; +import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; @TestInstance(PER_CLASS) -@Execution(CONCURRENT) +@Execution(SAME_THREAD) public class TestExchangeOperator { private static final List TYPES = ImmutableList.of(VARCHAR); diff --git a/plugin/trino-redis/src/test/java/io/trino/plugin/redis/TestMinimalFunctionality.java b/plugin/trino-redis/src/test/java/io/trino/plugin/redis/TestMinimalFunctionality.java index 6ebcdf32c74ee..1f85511d69450 100644 --- a/plugin/trino-redis/src/test/java/io/trino/plugin/redis/TestMinimalFunctionality.java +++ b/plugin/trino-redis/src/test/java/io/trino/plugin/redis/TestMinimalFunctionality.java @@ -18,6 +18,7 @@ import io.trino.metadata.TableHandle; import io.trino.security.AllowAllAccessControl; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.parallel.Execution; import java.util.Map; import java.util.Optional; @@ -25,7 +26,9 @@ import static io.trino.testing.TransactionBuilder.transaction; import static java.lang.String.format; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; +@Execution(SAME_THREAD) // clearData(), populateData() looks like shared mutable state public class TestMinimalFunctionality extends AbstractTestMinimalFunctionality { diff --git a/testing/trino-tests/src/test/java/io/trino/connector/informationschema/TestInformationSchemaConnector.java b/testing/trino-tests/src/test/java/io/trino/connector/informationschema/TestInformationSchemaConnector.java index 94a99fa107a9b..209a4e9ec639f 100644 --- a/testing/trino-tests/src/test/java/io/trino/connector/informationschema/TestInformationSchemaConnector.java +++ b/testing/trino-tests/src/test/java/io/trino/connector/informationschema/TestInformationSchemaConnector.java @@ -27,6 +27,7 @@ import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.parallel.Execution; import java.util.stream.IntStream; import java.util.stream.Stream; @@ -34,7 +35,9 @@ import static io.trino.testing.MultisetAssertions.assertMultisetsEqual; import static io.trino.testing.TestingSession.testSessionBuilder; import static java.util.stream.Collectors.joining; +import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; +@Execution(SAME_THREAD) // countingMockConnector is shared mutable state public class TestInformationSchemaConnector extends AbstractTestQueryFramework { diff --git a/testing/trino-tests/src/test/java/io/trino/connector/system/runtime/TestKillQuery.java b/testing/trino-tests/src/test/java/io/trino/connector/system/runtime/TestKillQuery.java index 17c0a83e04a9a..e3ff85aeab1a5 100644 --- a/testing/trino-tests/src/test/java/io/trino/connector/system/runtime/TestKillQuery.java +++ b/testing/trino-tests/src/test/java/io/trino/connector/system/runtime/TestKillQuery.java @@ -24,6 +24,7 @@ import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.parallel.Execution; import java.util.Optional; import java.util.concurrent.ExecutionException; @@ -43,7 +44,9 @@ import static java.util.UUID.randomUUID; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; +@Execution(SAME_THREAD) // e.g. some tests methods modify AC configuration public class TestKillQuery extends AbstractTestQueryFramework { diff --git a/testing/trino-tests/src/test/java/io/trino/execution/TestBeginQuery.java b/testing/trino-tests/src/test/java/io/trino/execution/TestBeginQuery.java index 05d982ce03bdd..fae3329fcdf5b 100644 --- a/testing/trino-tests/src/test/java/io/trino/execution/TestBeginQuery.java +++ b/testing/trino-tests/src/test/java/io/trino/execution/TestBeginQuery.java @@ -42,6 +42,7 @@ import io.trino.testing.TestingSplitManager; import io.trino.testing.TestingTransactionHandle; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.parallel.Execution; import java.util.List; import java.util.Map; @@ -50,7 +51,9 @@ import static io.trino.testing.TestingSession.testSessionBuilder; import static java.util.Objects.requireNonNull; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; +@Execution(SAME_THREAD) // TestMetadata is shared mutable state public class TestBeginQuery extends AbstractTestQueryFramework { diff --git a/testing/trino-tests/src/test/java/io/trino/execution/TestCompletedEventWarnings.java b/testing/trino-tests/src/test/java/io/trino/execution/TestCompletedEventWarnings.java index 003675efacbbd..f0586b7248421 100644 --- a/testing/trino-tests/src/test/java/io/trino/execution/TestCompletedEventWarnings.java +++ b/testing/trino-tests/src/test/java/io/trino/execution/TestCompletedEventWarnings.java @@ -39,10 +39,10 @@ import static io.trino.SessionTestUtils.TEST_SESSION; import static org.assertj.core.api.Fail.fail; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; +import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; @TestInstance(PER_CLASS) -@Execution(CONCURRENT) +@Execution(SAME_THREAD) // EventsAwaitingQueries is shared mutable state public class TestCompletedEventWarnings { private static final int TEST_WARNINGS = 5; diff --git a/testing/trino-tests/src/test/java/io/trino/execution/TestEventListenerWithSplits.java b/testing/trino-tests/src/test/java/io/trino/execution/TestEventListenerWithSplits.java index 956aa7f794af1..5eaabe2e1716b 100644 --- a/testing/trino-tests/src/test/java/io/trino/execution/TestEventListenerWithSplits.java +++ b/testing/trino-tests/src/test/java/io/trino/execution/TestEventListenerWithSplits.java @@ -36,6 +36,7 @@ import io.trino.testing.QueryRunner; import org.intellij.lang.annotations.Language; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.parallel.Execution; import java.util.List; import java.util.Optional; @@ -48,7 +49,9 @@ import static java.util.concurrent.TimeUnit.SECONDS; import static java.util.stream.Collectors.toSet; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; +@Execution(SAME_THREAD) // EventsAwaitingQueries is shared mutable state public class TestEventListenerWithSplits extends AbstractTestQueryFramework { diff --git a/testing/trino-tests/src/test/java/io/trino/execution/TestQueryTracker.java b/testing/trino-tests/src/test/java/io/trino/execution/TestQueryTracker.java index 14a810c9a6464..7d02faaa31a1f 100644 --- a/testing/trino-tests/src/test/java/io/trino/execution/TestQueryTracker.java +++ b/testing/trino-tests/src/test/java/io/trino/execution/TestQueryTracker.java @@ -25,6 +25,7 @@ import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.parallel.Execution; import java.util.concurrent.CountDownLatch; @@ -32,9 +33,11 @@ import static io.trino.spi.type.VarcharType.VARCHAR; import static io.trino.testing.TestingSession.testSessionBuilder; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; // Tests need to finish before strict timeouts. Any background work // may make them flaky +@Execution(SAME_THREAD) // CountDownLatches are shared mutable state public class TestQueryTracker extends AbstractTestQueryFramework { diff --git a/testing/trino-tests/src/test/java/io/trino/execution/TestQueues.java b/testing/trino-tests/src/test/java/io/trino/execution/TestQueues.java index 9f343c2693a93..906d7dab8382a 100644 --- a/testing/trino-tests/src/test/java/io/trino/execution/TestQueues.java +++ b/testing/trino-tests/src/test/java/io/trino/execution/TestQueues.java @@ -28,6 +28,7 @@ import io.trino.tests.tpch.TpchQueryRunnerBuilder; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.parallel.Execution; import java.util.Optional; import java.util.Set; @@ -49,8 +50,9 @@ import static java.util.Arrays.asList; import static java.util.Objects.requireNonNull; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; -// run single threaded to avoid creating multiple query runners at once +@Execution(SAME_THREAD) // run single threaded to avoid creating multiple query runners at once public class TestQueues { private static final String LONG_LASTING_QUERY = "SELECT COUNT(*) FROM lineitem"; diff --git a/testing/trino-tests/src/test/java/io/trino/execution/resourcegroups/db/TestEnvironments.java b/testing/trino-tests/src/test/java/io/trino/execution/resourcegroups/db/TestEnvironments.java index e826087d2ed82..d94ad84f5ad6b 100644 --- a/testing/trino-tests/src/test/java/io/trino/execution/resourcegroups/db/TestEnvironments.java +++ b/testing/trino-tests/src/test/java/io/trino/execution/resourcegroups/db/TestEnvironments.java @@ -18,6 +18,7 @@ import io.trino.testing.DistributedQueryRunner; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.parallel.Execution; import static io.trino.execution.QueryRunnerUtil.createQuery; import static io.trino.execution.QueryRunnerUtil.waitForQueryState; @@ -29,7 +30,9 @@ import static io.trino.execution.resourcegroups.db.H2TestUtil.createQueryRunner; import static io.trino.execution.resourcegroups.db.H2TestUtil.getDao; import static io.trino.execution.resourcegroups.db.H2TestUtil.getDbConfigUrl; +import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; +@Execution(SAME_THREAD) // run single threaded to avoid creating multiple query runners at once public class TestEnvironments { private static final String LONG_LASTING_QUERY = "SELECT COUNT(*) FROM lineitem"; diff --git a/testing/trino-tests/src/test/java/io/trino/execution/resourcegroups/db/TestQueuesDb.java b/testing/trino-tests/src/test/java/io/trino/execution/resourcegroups/db/TestQueuesDb.java index d61984362bbe2..3c857f8426dc6 100644 --- a/testing/trino-tests/src/test/java/io/trino/execution/resourcegroups/db/TestQueuesDb.java +++ b/testing/trino-tests/src/test/java/io/trino/execution/resourcegroups/db/TestQueuesDb.java @@ -32,8 +32,8 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.parallel.Execution; import java.util.Optional; import java.util.Set; @@ -69,10 +69,9 @@ import static java.lang.String.format; import static java.util.concurrent.TimeUnit.MILLISECONDS; import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_METHOD; +import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; -// run single threaded to avoid creating multiple query runners at once -@TestInstance(PER_METHOD) +@Execution(SAME_THREAD) // run single threaded to avoid creating multiple query runners at once public class TestQueuesDb { // Copy of TestQueues with tests for db reconfiguration of resource groups diff --git a/testing/trino-tests/src/test/java/io/trino/memory/TestMemoryManager.java b/testing/trino-tests/src/test/java/io/trino/memory/TestMemoryManager.java index c9b9d54f4adeb..dfe071fc43dd6 100644 --- a/testing/trino-tests/src/test/java/io/trino/memory/TestMemoryManager.java +++ b/testing/trino-tests/src/test/java/io/trino/memory/TestMemoryManager.java @@ -57,10 +57,10 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; +import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; @TestInstance(PER_CLASS) -@Execution(CONCURRENT) +@Execution(SAME_THREAD) // run single threaded to avoid creating multiple query runners at once public class TestMemoryManager { private static final Session SESSION = testSessionBuilder() diff --git a/testing/trino-tests/src/test/java/io/trino/security/TestSystemSecurityMetadata.java b/testing/trino-tests/src/test/java/io/trino/security/TestSystemSecurityMetadata.java index 0136d6bf0f1e8..5891d68c8f520 100644 --- a/testing/trino-tests/src/test/java/io/trino/security/TestSystemSecurityMetadata.java +++ b/testing/trino-tests/src/test/java/io/trino/security/TestSystemSecurityMetadata.java @@ -21,10 +21,13 @@ import io.trino.testing.DistributedQueryRunner; import io.trino.testing.QueryRunner; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.parallel.Execution; import static com.google.inject.multibindings.OptionalBinder.newOptionalBinder; import static io.trino.testing.TestingSession.testSessionBuilder; +import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; +@Execution(SAME_THREAD) // TestingSystemSecurityMetadata is shared mutable state public class TestSystemSecurityMetadata extends AbstractTestQueryFramework { diff --git a/testing/trino-tests/src/test/java/io/trino/tests/TestGracefulShutdown.java b/testing/trino-tests/src/test/java/io/trino/tests/TestGracefulShutdown.java index 9fbf1065fc931..ecb52a979e8c5 100644 --- a/testing/trino-tests/src/test/java/io/trino/tests/TestGracefulShutdown.java +++ b/testing/trino-tests/src/test/java/io/trino/tests/TestGracefulShutdown.java @@ -45,10 +45,10 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; +import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; @TestInstance(PER_CLASS) -@Execution(CONCURRENT) +@Execution(SAME_THREAD) // run single threaded to avoid creating multiple query runners at once public class TestGracefulShutdown { private static final long SHUTDOWN_TIMEOUT_MILLIS = 240_000; diff --git a/testing/trino-tests/src/test/java/io/trino/tests/TestMetadataManager.java b/testing/trino-tests/src/test/java/io/trino/tests/TestMetadataManager.java index aaae7d41b06cf..edbca15904bf4 100644 --- a/testing/trino-tests/src/test/java/io/trino/tests/TestMetadataManager.java +++ b/testing/trino-tests/src/test/java/io/trino/tests/TestMetadataManager.java @@ -50,7 +50,7 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; +import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; /** * This is integration / unit test suite. @@ -59,7 +59,7 @@ * This mapping has to be manually cleaned when query finishes execution (Metadata#cleanupQuery method). */ @TestInstance(PER_CLASS) -@Execution(CONCURRENT) +@Execution(SAME_THREAD) // metadataManager.getActiveQueryIds() is shared mutable state that affects the test outcome public class TestMetadataManager { private DistributedQueryRunner queryRunner; diff --git a/testing/trino-tests/src/test/java/io/trino/tests/TestProcedureCall.java b/testing/trino-tests/src/test/java/io/trino/tests/TestProcedureCall.java index 66eec5e26eb2e..0bd33ad0c04f0 100644 --- a/testing/trino-tests/src/test/java/io/trino/tests/TestProcedureCall.java +++ b/testing/trino-tests/src/test/java/io/trino/tests/TestProcedureCall.java @@ -26,6 +26,7 @@ import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.parallel.Execution; import java.util.List; @@ -34,7 +35,9 @@ import static java.util.Arrays.asList; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; +@Execution(SAME_THREAD) // ProcedureTester is shared mutable state public class TestProcedureCall extends AbstractTestQueryFramework { diff --git a/testing/trino-tests/src/test/java/io/trino/tests/TestQueryManager.java b/testing/trino-tests/src/test/java/io/trino/tests/TestQueryManager.java index 091a4a951b0fe..4b41a0f303bf0 100644 --- a/testing/trino-tests/src/test/java/io/trino/tests/TestQueryManager.java +++ b/testing/trino-tests/src/test/java/io/trino/tests/TestQueryManager.java @@ -49,10 +49,10 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Fail.fail; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; +import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; @TestInstance(PER_CLASS) -@Execution(CONCURRENT) +@Execution(SAME_THREAD) // run single threaded to avoid creating multiple query runners at once public class TestQueryManager { private DistributedQueryRunner queryRunner; From 35ad5c164b924f30b579d126a6ab10036acd642e Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Thu, 21 Dec 2023 17:15:39 +0100 Subject: [PATCH 195/350] Set up only one concurrent query runner in TestQueryManager `TestQueryManager` defined a default `queryRunner` used by one test method, but other test methods created additional runners. This commit moves the default one into the only test method that used it. In the result, there is at most 1 query runner at the same time created during test execution. --- .../java/io/trino/tests/TestQueryManager.java | 77 +++++++------------ 1 file changed, 29 insertions(+), 48 deletions(-) diff --git a/testing/trino-tests/src/test/java/io/trino/tests/TestQueryManager.java b/testing/trino-tests/src/test/java/io/trino/tests/TestQueryManager.java index 4b41a0f303bf0..f7a7a91a4dbb1 100644 --- a/testing/trino-tests/src/test/java/io/trino/tests/TestQueryManager.java +++ b/testing/trino-tests/src/test/java/io/trino/tests/TestQueryManager.java @@ -27,10 +27,7 @@ import io.trino.testing.DistributedQueryRunner; import io.trino.testing.TestingSessionContext; import io.trino.tests.tpch.TpchQueryRunnerBuilder; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.parallel.Execution; @@ -48,64 +45,48 @@ import static java.util.Arrays.stream; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Fail.fail; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; -@TestInstance(PER_CLASS) @Execution(SAME_THREAD) // run single threaded to avoid creating multiple query runners at once public class TestQueryManager { - private DistributedQueryRunner queryRunner; - - @BeforeAll - public void setUp() - throws Exception - { - queryRunner = TpchQueryRunnerBuilder.builder().build(); - } - - @AfterAll - public void tearDown() - { - queryRunner.close(); - queryRunner = null; - } - @Test @Timeout(60) public void testFailQuery() throws Exception { - DispatchManager dispatchManager = queryRunner.getCoordinator().getDispatchManager(); - QueryId queryId = dispatchManager.createQueryId(); - dispatchManager.createQuery( - queryId, - Span.getInvalid(), - Slug.createNew(), - TestingSessionContext.fromSession(TEST_SESSION), - "SELECT * FROM lineitem") - .get(); + try (DistributedQueryRunner queryRunner = TpchQueryRunnerBuilder.builder().build()) { + DispatchManager dispatchManager = queryRunner.getCoordinator().getDispatchManager(); + QueryId queryId = dispatchManager.createQueryId(); + dispatchManager.createQuery( + queryId, + Span.getInvalid(), + Slug.createNew(), + TestingSessionContext.fromSession(TEST_SESSION), + "SELECT * FROM lineitem") + .get(); - // wait until query starts running - while (true) { - QueryState state = dispatchManager.getQueryInfo(queryId).getState(); - if (state.isDone()) { - fail("unexpected query state: " + state); + // wait until query starts running + while (true) { + QueryState state = dispatchManager.getQueryInfo(queryId).getState(); + if (state.isDone()) { + fail("unexpected query state: " + state); + } + if (state == RUNNING) { + break; + } + Thread.sleep(100); } - if (state == RUNNING) { - break; - } - Thread.sleep(100); - } - // cancel query - QueryManager queryManager = queryRunner.getCoordinator().getQueryManager(); - queryManager.failQuery(queryId, new TrinoException(GENERIC_INTERNAL_ERROR, "mock exception")); - QueryInfo queryInfo = queryManager.getFullQueryInfo(queryId); - assertThat(queryInfo.getState()).isEqualTo(FAILED); - assertThat(queryInfo.getErrorCode()).isEqualTo(GENERIC_INTERNAL_ERROR.toErrorCode()); - assertThat(queryInfo.getFailureInfo()).isNotNull(); - assertThat(queryInfo.getFailureInfo().getMessage()).isEqualTo("mock exception"); + // cancel query + QueryManager queryManager = queryRunner.getCoordinator().getQueryManager(); + queryManager.failQuery(queryId, new TrinoException(GENERIC_INTERNAL_ERROR, "mock exception")); + QueryInfo queryInfo = queryManager.getFullQueryInfo(queryId); + assertThat(queryInfo.getState()).isEqualTo(FAILED); + assertThat(queryInfo.getErrorCode()).isEqualTo(GENERIC_INTERNAL_ERROR.toErrorCode()); + assertThat(queryInfo.getFailureInfo()).isNotNull(); + assertThat(queryInfo.getFailureInfo().getMessage()).isEqualTo("mock exception"); + } } @Test From de79cdc405acd95fa04cc6c86e5776337141416a Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Thu, 21 Dec 2023 11:31:41 +0100 Subject: [PATCH 196/350] Add tracing for SplitManager.getSplits Sometimes, `ConnectorSplitManager.getSplits` can take long to construct `ConnectorSplitSource`. For example, in Delta, there is IO work being done before `ConnectorSplitSource` is returned. This work would better be delayed until `ConnectorSplitSource.getNextBatch` is invoked, but currently this is not the case. Let's add tracing so that time spent in `ConnectorSplitManager.getSplits` is attributable. --- .../java/io/trino/split/SplitManager.java | 33 +++++++++++++------ 1 file changed, 23 insertions(+), 10 deletions(-) diff --git a/core/trino-main/src/main/java/io/trino/split/SplitManager.java b/core/trino-main/src/main/java/io/trino/split/SplitManager.java index 4ec43a8670a73..e6e61b27d11f1 100644 --- a/core/trino-main/src/main/java/io/trino/split/SplitManager.java +++ b/core/trino-main/src/main/java/io/trino/split/SplitManager.java @@ -38,6 +38,7 @@ import static io.airlift.concurrent.Threads.daemonThreadsNamed; import static io.trino.SystemSessionProperties.isAllowPushdownIntoConnectors; +import static io.trino.tracing.ScopedSpan.scopedSpan; import static java.util.Objects.requireNonNull; import static java.util.concurrent.Executors.newCachedThreadPool; @@ -80,12 +81,18 @@ public SplitSource getSplits( ConnectorSession connectorSession = session.toConnectorSession(catalogHandle); - ConnectorSplitSource source = splitManager.getSplits( - table.getTransaction(), - connectorSession, - table.getConnectorHandle(), - dynamicFilter, - constraint); + ConnectorSplitSource source; + try (var ignore = scopedSpan(tracer.spanBuilder("SplitManager.getSplits") + .setParent(Context.current().with(parentSpan)) + .setAttribute(TrinoAttributes.TABLE, table.getConnectorHandle().toString()) + .startSpan())) { + source = splitManager.getSplits( + table.getTransaction(), + connectorSession, + table.getConnectorHandle(), + dynamicFilter, + constraint); + } SplitSource splitSource = new ConnectorAwareSplitSource(catalogHandle, source); @@ -108,10 +115,16 @@ public SplitSource getSplits(Session session, Span parentSpan, TableFunctionHand CatalogHandle catalogHandle = function.getCatalogHandle(); ConnectorSplitManager splitManager = splitManagerProvider.getService(catalogHandle); - ConnectorSplitSource source = splitManager.getSplits( - function.getTransactionHandle(), - session.toConnectorSession(catalogHandle), - function.getFunctionHandle()); + ConnectorSplitSource source; + try (var ignore = scopedSpan(tracer.spanBuilder("SplitManager.getSplits") + .setParent(Context.current().with(parentSpan)) + .setAttribute(TrinoAttributes.FUNCTION, function.getFunctionHandle().toString()) + .startSpan())) { + source = splitManager.getSplits( + function.getTransactionHandle(), + session.toConnectorSession(catalogHandle), + function.getFunctionHandle()); + } SplitSource splitSource = new ConnectorAwareSplitSource(catalogHandle, source); From 506da6006ce06b699b928622a017a94653322eab Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Thu, 21 Dec 2023 11:34:35 +0100 Subject: [PATCH 197/350] Rename parameter in helper method The parameter refers to "a stage span", not to the top level "query span". --- .../trino-main/src/main/java/io/trino/split/SplitManager.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/trino-main/src/main/java/io/trino/split/SplitManager.java b/core/trino-main/src/main/java/io/trino/split/SplitManager.java index e6e61b27d11f1..c35bfa05debd5 100644 --- a/core/trino-main/src/main/java/io/trino/split/SplitManager.java +++ b/core/trino-main/src/main/java/io/trino/split/SplitManager.java @@ -132,10 +132,10 @@ public SplitSource getSplits(Session session, Span parentSpan, TableFunctionHand return new TracingSplitSource(splitSource, tracer, Optional.of(span), "split-buffer"); } - private Span splitSourceSpan(Span querySpan, CatalogHandle catalogHandle) + private Span splitSourceSpan(Span parentSpan, CatalogHandle catalogHandle) { return tracer.spanBuilder("split-source") - .setParent(Context.current().with(querySpan)) + .setParent(Context.current().with(parentSpan)) .setAttribute(TrinoAttributes.CATALOG, catalogHandle.getCatalogName()) .startSpan(); } From 7af05cb122fa82f2df7e25f345bb5a62074b42ad Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 20 Dec 2023 22:33:08 +0100 Subject: [PATCH 198/350] Hide FTE tests configs Failure injection is a for-tests functionality. Hide its config, they are not meant to be used. --- .../main/java/io/trino/execution/FailureInjectionConfig.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/core/trino-main/src/main/java/io/trino/execution/FailureInjectionConfig.java b/core/trino-main/src/main/java/io/trino/execution/FailureInjectionConfig.java index 2ee459b524842..b8d0a5b36aab7 100644 --- a/core/trino-main/src/main/java/io/trino/execution/FailureInjectionConfig.java +++ b/core/trino-main/src/main/java/io/trino/execution/FailureInjectionConfig.java @@ -15,6 +15,7 @@ import io.airlift.configuration.Config; import io.airlift.configuration.ConfigDescription; +import io.airlift.configuration.ConfigHidden; import io.airlift.units.Duration; import jakarta.validation.constraints.NotNull; @@ -31,6 +32,7 @@ public Duration getExpirationPeriod() return expirationPeriod; } + @ConfigHidden // not supposed to be used outside of tests @Config("failure-injection.expiration-period") @ConfigDescription("Period after which an injected failure is considered expired and will no longer be triggering a failure") public FailureInjectionConfig setExpirationPeriod(Duration expirationPeriod) @@ -45,6 +47,7 @@ public Duration getRequestTimeout() return requestTimeout; } + @ConfigHidden // not supposed to be used outside of tests @Config("failure-injection.request-timeout") @ConfigDescription("Period after which requests blocked to emulate a timeout are released") public FailureInjectionConfig setRequestTimeout(Duration requestTimeout) From 5f4e7b6041e5c802b624150e5b2badcf18609de2 Mon Sep 17 00:00:00 2001 From: Naoki Takezoe Date: Fri, 22 Dec 2023 10:45:22 +0900 Subject: [PATCH 199/350] Fix rendering of time/timestamp with timezone type name --- .../io/trino/client/ClientTypeSignature.java | 11 +++++++ .../io/trino/jdbc/BaseTestJdbcResultSet.java | 33 ++++++++----------- 2 files changed, 24 insertions(+), 20 deletions(-) diff --git a/client/trino-client/src/main/java/io/trino/client/ClientTypeSignature.java b/client/trino-client/src/main/java/io/trino/client/ClientTypeSignature.java index b408006b9949a..3e964f279b47a 100644 --- a/client/trino-client/src/main/java/io/trino/client/ClientTypeSignature.java +++ b/client/trino-client/src/main/java/io/trino/client/ClientTypeSignature.java @@ -29,6 +29,8 @@ import static com.google.common.base.Preconditions.checkState; import static com.google.common.collect.ImmutableList.toImmutableList; import static io.trino.client.ClientStandardTypes.ROW; +import static io.trino.client.ClientStandardTypes.TIMESTAMP_WITH_TIME_ZONE; +import static io.trino.client.ClientStandardTypes.TIME_WITH_TIME_ZONE; import static io.trino.client.ClientStandardTypes.VARCHAR; import static java.util.Collections.unmodifiableList; import static java.util.Objects.requireNonNull; @@ -94,6 +96,15 @@ public String toString() if (arguments.isEmpty()) { return rawType; } + + if (rawType.equals(TIME_WITH_TIME_ZONE)) { + return "time(" + arguments.get(0) + ") with time zone"; + } + + if (rawType.equals(TIMESTAMP_WITH_TIME_ZONE)) { + return "timestamp(" + arguments.get(0) + ") with time zone"; + } + return rawType + arguments.stream() .map(ClientTypeSignatureParameter::toString) .collect(joining(",", "(", ")")); diff --git a/client/trino-jdbc/src/test/java/io/trino/jdbc/BaseTestJdbcResultSet.java b/client/trino-jdbc/src/test/java/io/trino/jdbc/BaseTestJdbcResultSet.java index 6bae338a8cbb8..776d4b50e395e 100644 --- a/client/trino-jdbc/src/test/java/io/trino/jdbc/BaseTestJdbcResultSet.java +++ b/client/trino-jdbc/src/test/java/io/trino/jdbc/BaseTestJdbcResultSet.java @@ -441,10 +441,9 @@ public void testTimeWithTimeZone() .isInstanceOf(SQLException.class) .hasMessage("Expected value to be a date but is: 09:39:07+01:00"); assertThat(rs.getTime(column)).isEqualTo(Time.valueOf(LocalTime.of(1, 39, 7))); // TODO this should fail, or represent TIME '09:39:07' - // TODO (https://github.com/trinodb/trino/issues/5317) placement of precision parameter assertThatThrownBy(() -> rs.getTimestamp(column)) .isInstanceOf(IllegalArgumentException.class) // TODO (https://github.com/trinodb/trino/issues/5315) SQLException - .hasMessage("Expected column to be a timestamp type but is time with time zone(0)"); + .hasMessage("Expected column to be a timestamp type but is time(0) with time zone"); }); checkRepresentation(connectedStatement.getStatement(), "TIME '01:39:07 +01:00'", Types.TIME_WITH_TIMEZONE, (rs, column) -> { @@ -458,10 +457,9 @@ public void testTimeWithTimeZone() .isInstanceOf(SQLException.class) .hasMessage("Expected value to be a date but is: 01:39:07+01:00"); assertThat(rs.getTime(column)).isEqualTo(someBogusValue); // TODO this should fail, or represent TIME '01:39:07' - // TODO (https://github.com/trinodb/trino/issues/5317) placement of precision parameter assertThatThrownBy(() -> rs.getTimestamp(column)) .isInstanceOf(IllegalArgumentException.class) // TODO (https://github.com/trinodb/trino/issues/5315) SQLException - .hasMessage("Expected column to be a timestamp type but is time with time zone(0)"); + .hasMessage("Expected column to be a timestamp type but is time(0) with time zone"); }); checkRepresentation(connectedStatement.getStatement(), "TIME '00:39:07 +01:00'", Types.TIME_WITH_TIMEZONE, (rs, column) -> { @@ -475,10 +473,9 @@ public void testTimeWithTimeZone() .isInstanceOf(SQLException.class) .hasMessage("Expected value to be a date but is: 00:39:07+01:00"); assertThat(rs.getTime(column)).isEqualTo(someBogusValue); // TODO this should fail, as there no java.sql.Time representation for TIME '00:39:07' in America/Bahia_Banderas - // TODO (https://github.com/trinodb/trino/issues/5317) placement of precision parameter assertThatThrownBy(() -> rs.getTimestamp(column)) .isInstanceOf(IllegalArgumentException.class) // TODO (https://github.com/trinodb/trino/issues/5315) SQLException - .hasMessage("Expected column to be a timestamp type but is time with time zone(0)"); + .hasMessage("Expected column to be a timestamp type but is time(0) with time zone"); }); } } @@ -642,10 +639,9 @@ public void testTimestampWithTimeZone() assertThatThrownBy(() -> rs.getDate(column)) .isInstanceOf(SQLException.class) .hasMessage("Expected value to be a date but is: 1970-01-01 00:00:00.000 UTC"); - // TODO (https://github.com/trinodb/trino/issues/5317) placement of precision parameter assertThatThrownBy(() -> rs.getTime(column)) .isInstanceOf(IllegalArgumentException.class) // TODO (https://github.com/trinodb/trino/issues/5315) SQLException - .hasMessage("Expected column to be a time type but is timestamp with time zone(3)"); + .hasMessage("Expected column to be a time type but is timestamp(3) with time zone"); assertThat(rs.getTimestamp(column)).isEqualTo(timestampForPointInTime); }); @@ -657,10 +653,9 @@ public void testTimestampWithTimeZone() assertThatThrownBy(() -> rs.getDate(column)) .isInstanceOf(SQLException.class) .hasMessage("Expected value to be a date but is: 2018-02-13 13:14:15.227 Europe/Warsaw"); - // TODO (https://github.com/trinodb/trino/issues/5317) placement of precision parameter assertThatThrownBy(() -> rs.getTime(column)) .isInstanceOf(IllegalArgumentException.class) // TODO (https://github.com/trinodb/trino/issues/5315) SQLException - .hasMessage("Expected column to be a time type but is timestamp with time zone(3)"); + .hasMessage("Expected column to be a time type but is timestamp(3) with time zone"); assertThat(rs.getTimestamp(column)).isEqualTo(timestampForPointInTime); }); @@ -675,7 +670,7 @@ public void testTimestampWithTimeZone() .hasMessage("Expected value to be a date but is: 2019-12-31 23:59:59.999999999999 Europe/Warsaw"); assertThatThrownBy(() -> rs.getTime(column)) .isInstanceOf(IllegalArgumentException.class) // TODO (https://github.com/trinodb/trino/issues/5315) SQLException - .hasMessage("Expected column to be a time type but is timestamp with time zone(12)"); // TODO (https://github.com/trinodb/trino/issues/5317) placement of precision parameter + .hasMessage("Expected column to be a time type but is timestamp(12) with time zone"); assertThat(rs.getTimestamp(column)).isEqualTo(timestampForPointInTime); }); @@ -694,7 +689,7 @@ public void testTimestampWithTimeZone() .hasMessage("Expected value to be a date but is: 2019-12-31 23:59:59.999999999999 America/Bahia_Banderas"); assertThatThrownBy(() -> rs.getTime(column)) .isInstanceOf(IllegalArgumentException.class) // TODO (https://github.com/trinodb/trino/issues/5315) SQLException - .hasMessage("Expected column to be a time type but is timestamp with time zone(12)"); // TODO (https://github.com/trinodb/trino/issues/5317) placement of precision parameter + .hasMessage("Expected column to be a time type but is timestamp(12) with time zone"); assertThat(rs.getTimestamp(column)).isEqualTo(timestampForPointInTime); }); @@ -708,7 +703,7 @@ public void testTimestampWithTimeZone() .hasMessage("Expected value to be a date but is: 1957-12-31 23:59:59.999999999999 Europe/Warsaw"); assertThatThrownBy(() -> rs.getTime(column)) .isInstanceOf(IllegalArgumentException.class) // TODO (https://github.com/trinodb/trino/issues/5315) SQLException - .hasMessage("Expected column to be a time type but is timestamp with time zone(12)"); // TODO (https://github.com/trinodb/trino/issues/5317) placement of precision parameter + .hasMessage("Expected column to be a time type but is timestamp(12) with time zone"); assertThat(rs.getTimestamp(column)).isEqualTo(timestampForPointInTime); }); @@ -720,10 +715,9 @@ public void testTimestampWithTimeZone() assertThatThrownBy(() -> rs.getDate(column)) .isInstanceOf(SQLException.class) .hasMessage("Expected value to be a date but is: 1970-01-01 09:14:15.227 Europe/Warsaw"); - // TODO (https://github.com/trinodb/trino/issues/5317) placement of precision parameter assertThatThrownBy(() -> rs.getTime(column)) .isInstanceOf(IllegalArgumentException.class) // TODO (https://github.com/trinodb/trino/issues/5315) SQLException - .hasMessage("Expected column to be a time type but is timestamp with time zone(3)"); + .hasMessage("Expected column to be a time type but is timestamp(3) with time zone"); assertThat(rs.getTimestamp(column)).isEqualTo(timestampForPointInTime); }); @@ -735,10 +729,9 @@ public void testTimestampWithTimeZone() assertThatThrownBy(() -> rs.getDate(column)) .isInstanceOf(SQLException.class) .hasMessage("Expected value to be a date but is: 1970-01-01 00:14:15.227 Europe/Warsaw"); - // TODO (https://github.com/trinodb/trino/issues/5317) placement of precision parameter assertThatThrownBy(() -> rs.getTime(column)) .isInstanceOf(IllegalArgumentException.class) // TODO (https://github.com/trinodb/trino/issues/5315) SQLException - .hasMessage("Expected column to be a time type but is timestamp with time zone(3)"); + .hasMessage("Expected column to be a time type but is timestamp(3) with time zone"); assertThat(rs.getTimestamp(column)).isEqualTo(timestampForPointInTime); }); @@ -757,7 +750,7 @@ public void testTimestampWithTimeZone() .hasMessage("Expected value to be a date but is: +12345-01-23 01:23:45.123456789 Europe/Warsaw"); assertThatThrownBy(() -> rs.getTime(column)) .isInstanceOf(IllegalArgumentException.class) // TODO (https://github.com/trinodb/trino/issues/5315) SQLException - .hasMessage("Expected column to be a time type but is timestamp with time zone(9)"); // TODO (https://github.com/trinodb/trino/issues/5317) placement of precision parameter + .hasMessage("Expected column to be a time type but is timestamp(9) with time zone"); assertThat(rs.getTimestamp(column)).isEqualTo(timestampForPointInTime); }); } @@ -810,8 +803,8 @@ public void testArray() checkArrayRepresentation(connectedStatement.getStatement(), "TIMESTAMP '2017-01-02 09:00:00.123'", Types.TIMESTAMP, "timestamp(3)"); checkArrayRepresentation(connectedStatement.getStatement(), "TIMESTAMP '2017-01-02 09:00:00.123456789'", Types.TIMESTAMP, "timestamp(9)"); - checkArrayRepresentation(connectedStatement.getStatement(), "TIMESTAMP '2017-01-02 09:00:00.123 Europe/Warsaw'", Types.TIMESTAMP_WITH_TIMEZONE, "timestamp with time zone(3)"); - checkArrayRepresentation(connectedStatement.getStatement(), "TIMESTAMP '2017-01-02 09:00:00.123456789 Europe/Warsaw'", Types.TIMESTAMP_WITH_TIMEZONE, "timestamp with time zone(9)"); + checkArrayRepresentation(connectedStatement.getStatement(), "TIMESTAMP '2017-01-02 09:00:00.123 Europe/Warsaw'", Types.TIMESTAMP_WITH_TIMEZONE, "timestamp(3) with time zone"); + checkArrayRepresentation(connectedStatement.getStatement(), "TIMESTAMP '2017-01-02 09:00:00.123456789 Europe/Warsaw'", Types.TIMESTAMP_WITH_TIMEZONE, "timestamp(9) with time zone"); // array or array checkRepresentation(connectedStatement.getStatement(), "ARRAY[NULL, ARRAY[NULL, BIGINT '1', 2]]", Types.ARRAY, (rs, column) -> { From 261322f75190f0f744a5fce59d1f8496accb9e40 Mon Sep 17 00:00:00 2001 From: Vikash Kumar Date: Mon, 27 Nov 2023 12:28:30 +0530 Subject: [PATCH 200/350] Update hudi test resource for hudi_non_part_cow table The change is to make the schema sync with hudi_cow_pt_tbl. --- .../trino/plugin/hudi/TestHudiSmokeTest.java | 4 +- .../ResourceHudiTablesInitializer.java | 11 ++-- .../trino-hudi/src/test/resources/README.md | 47 ++++++++++++++++ .../.hoodie/20211217110514527.commit | 50 ------------------ .../.hoodie/20231127051653361.commit | 37 +++++++++++++ ...ted => 20231127051653361.commit.requested} | 0 ...27.inflight => 20231127051653361.inflight} | 25 ++------- .../.hoodie/hoodie.properties | 23 ++++---- .../.hoodie_partition_metadata | 4 +- ...da93b-0_0-27-28_20231127051653361.parquet} | Bin 436273 -> 435338 bytes 10 files changed, 109 insertions(+), 92 deletions(-) create mode 100644 plugin/trino-hudi/src/test/resources/README.md delete mode 100644 plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/.hoodie/20211217110514527.commit create mode 100644 plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/.hoodie/20231127051653361.commit rename plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/.hoodie/{20211217110514527.commit.requested => 20231127051653361.commit.requested} (100%) rename plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/.hoodie/{20211217110514527.inflight => 20231127051653361.inflight} (56%) rename plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/{d0875d00-483d-4e8b-bbbe-c520366c47a0-0_0-6-11_20211217110514527.parquet => 05b0f4ec-00fb-49f2-a1e2-7f510f3da93b-0_0-27-28_20231127051653361.parquet} (98%) 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 5dee9b2d7f8e0..565cf9d1fed6b 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 @@ -46,8 +46,8 @@ protected QueryRunner createQueryRunner() public void testReadNonPartitionedTable() { assertQuery( - "SELECT rowid, name FROM " + HUDI_NON_PART_COW, - "SELECT * FROM VALUES ('row_1', 'bob'), ('row_2', 'john'), ('row_3', 'tom')"); + "SELECT id, name FROM " + HUDI_NON_PART_COW, + "SELECT * FROM VALUES (1, 'a1'), (2, 'a2')"); } @Test 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 b55cfe52c265a..9c664c055f6f0 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 @@ -220,14 +220,11 @@ public Map getPartitions() private static List nonPartitionRegularColumns() { return ImmutableList.of( - column("rowid", HIVE_STRING), - column("partitionid", HIVE_STRING), - column("precomb", HIVE_LONG), + column("id", HIVE_LONG), column("name", HIVE_STRING), - column("versionid", HIVE_STRING), - column("tobedeletedstr", HIVE_STRING), - column("inttolong", HIVE_INT), - column("longtoint", HIVE_LONG)); + column("ts", HIVE_LONG), + column("dt", HIVE_STRING), + column("hh", HIVE_STRING)); } private static List stockTicksRegularColumns() diff --git a/plugin/trino-hudi/src/test/resources/README.md b/plugin/trino-hudi/src/test/resources/README.md new file mode 100644 index 0000000000000..26f65f3fcd78f --- /dev/null +++ b/plugin/trino-hudi/src/test/resources/README.md @@ -0,0 +1,47 @@ +# Hudi Test Resources + +## Generating Hudi Resources + +Follow these steps to create the `hudi_non_part_cow` test table and utilize it for testing. `hudi_non_part_cow` resource is generated using `423` trino version. + +### Start the Hudi environment + +Execute the following command in the terminal to initiate the Hudi environment: + +```shell +testing/bin/ptl env up --environment singlenode-hudi +``` + +### Generate Resources + +* Open the `spark-sql` terminal and initiate the `spark-sql` shell in the `ptl-spark` container. +* Execute the following Spark SQL queries to create the `hudi_non_part_cow` table: + +``` +spark-sql> CREATE TABLE default.hudi_non_part_cow ( + id bigint, + name string, + ts bigint, + dt string, + hh string + ) + USING hudi + TBLPROPERTIES ( + type = 'cow', + primaryKey = 'id', + preCombineField = 'ts' + ) + LOCATION 's3://test-bucket/hudi_non_part_cow'; + +spark-sql> INSERT INTO default.hudi_non_part_cow (id, name, ts, dt, hh) VALUES + (1, 'a1', 1000, '2021-12-09', '10'), + (2, 'a2', 2000, '2021-12-09', '11'); +``` + +### Download Resources + +Download the `hudi_non_part_cow` table from the MinIO client http://localhost:9001/buckets/test-bucket/browse. + +### Use Resources + +Unzip the downloaded `hudi_non_part_cow.zip`. Remove any unnecessary files obtained after unzipping to prepare the resource for testing. diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/.hoodie/20211217110514527.commit b/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/.hoodie/20211217110514527.commit deleted file mode 100644 index f77eeb137f026..0000000000000 --- a/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/.hoodie/20211217110514527.commit +++ /dev/null @@ -1,50 +0,0 @@ -{ - "partitionToWriteStats" : { - "" : [ { - "fileId" : "d0875d00-483d-4e8b-bbbe-c520366c47a0-0", - "path" : "d0875d00-483d-4e8b-bbbe-c520366c47a0-0_0-6-11_20211217110514527.parquet", - "prevCommit" : "null", - "numWrites" : 3, - "numDeletes" : 0, - "numUpdateWrites" : 0, - "numInserts" : 3, - "totalWriteBytes" : 436273, - "totalWriteErrors" : 0, - "tempPath" : null, - "partitionPath" : "", - "totalLogRecords" : 0, - "totalLogFilesCompacted" : 0, - "totalLogSizeCompacted" : 0, - "totalUpdatedRecordsCompacted" : 0, - "totalLogBlocks" : 0, - "totalCorruptLogBlock" : 0, - "totalRollbackBlocks" : 0, - "fileSizeInBytes" : 436273, - "minEventTime" : null, - "maxEventTime" : null - } ] - }, - "compacted" : false, - "extraMetadata" : { - "schema" : "{\"type\":\"record\",\"name\":\"hudi_non_part_cow_record\",\"namespace\":\"hoodie.hudi_non_part_cow\",\"fields\":[{\"name\":\"rowId\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"partitionId\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"preComb\",\"type\":[\"null\",\"long\"],\"default\":null},{\"name\":\"name\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"versionId\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"toBeDeletedStr\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"intToLong\",\"type\":[\"null\",\"int\"],\"default\":null},{\"name\":\"longToInt\",\"type\":[\"null\",\"long\"],\"default\":null}]}" - }, - "operationType" : "INSERT", - "writePartitionPaths" : [ "" ], - "fileIdAndRelativePaths" : { - "d0875d00-483d-4e8b-bbbe-c520366c47a0-0" : "d0875d00-483d-4e8b-bbbe-c520366c47a0-0_0-6-11_20211217110514527.parquet" - }, - "totalRecordsDeleted" : 0, - "totalLogRecordsCompacted" : 0, - "totalLogFilesCompacted" : 0, - "totalCompactedRecordsUpdated" : 0, - "totalLogFilesSize" : 0, - "totalScanTime" : 0, - "totalCreateTime" : 1743, - "totalUpsertTime" : 0, - "minAndMaxEventTime" : { - "Optional.empty" : { - "val" : null, - "present" : false - } - } -} \ No newline at end of file diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/.hoodie/20231127051653361.commit b/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/.hoodie/20231127051653361.commit new file mode 100644 index 0000000000000..9fc9470ff41a2 --- /dev/null +++ b/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/.hoodie/20231127051653361.commit @@ -0,0 +1,37 @@ +{ + "partitionToWriteStats" : { + "" : [ { + "fileId" : "05b0f4ec-00fb-49f2-a1e2-7f510f3da93b-0", + "path" : "05b0f4ec-00fb-49f2-a1e2-7f510f3da93b-0_0-27-28_20231127051653361.parquet", + "prevCommit" : "null", + "numWrites" : 2, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 2, + "totalWriteBytes" : 435338, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 435338, + "minEventTime" : null, + "maxEventTime" : null, + "runtimeStats" : { + "totalScanTime" : 0, + "totalUpsertTime" : 0, + "totalCreateTime" : 856 + } + } ] + }, + "compacted" : false, + "extraMetadata" : { + "schema" : "{\"type\":\"record\",\"name\":\"hudi_non_part_cow_record\",\"namespace\":\"hoodie.hudi_non_part_cow\",\"fields\":[{\"name\":\"id\",\"type\":[\"null\",\"long\"],\"default\":null},{\"name\":\"name\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"ts\",\"type\":[\"null\",\"long\"],\"default\":null},{\"name\":\"dt\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"hh\",\"type\":[\"null\",\"string\"],\"default\":null}]}" + }, + "operationType" : "UPSERT" +} \ No newline at end of file diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/.hoodie/20211217110514527.commit.requested b/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/.hoodie/20231127051653361.commit.requested similarity index 100% rename from plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/.hoodie/20211217110514527.commit.requested rename to plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/.hoodie/20231127051653361.commit.requested diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/.hoodie/20211217110514527.inflight b/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/.hoodie/20231127051653361.inflight similarity index 56% rename from plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/.hoodie/20211217110514527.inflight rename to plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/.hoodie/20231127051653361.inflight index 6605bcaf9b36c..dad745d91bd72 100644 --- a/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/.hoodie/20211217110514527.inflight +++ b/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/.hoodie/20231127051653361.inflight @@ -7,7 +7,7 @@ "numWrites" : 0, "numDeletes" : 0, "numUpdateWrites" : 0, - "numInserts" : 3, + "numInserts" : 2, "totalWriteBytes" : 0, "totalWriteErrors" : 0, "tempPath" : null, @@ -21,28 +21,11 @@ "totalRollbackBlocks" : 0, "fileSizeInBytes" : 0, "minEventTime" : null, - "maxEventTime" : null + "maxEventTime" : null, + "runtimeStats" : null } ] }, "compacted" : false, "extraMetadata" : { }, - "operationType" : "INSERT", - "writePartitionPaths" : [ "" ], - "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 - } - } + "operationType" : "UPSERT" } \ No newline at end of file diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/.hoodie/hoodie.properties b/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/.hoodie/hoodie.properties index 3d03fa7915c39..aa323696b299e 100644 --- a/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/.hoodie/hoodie.properties +++ b/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/.hoodie/hoodie.properties @@ -1,14 +1,17 @@ -#Properties saved on Fri Dec 17 11:05:14 UTC 2021 -#Fri Dec 17 11:05:14 UTC 2021 -hoodie.table.precombine.field=preComb -hoodie.table.partition.fields= +#Updated at 2023-11-27T05:16:58.380652Z +#Mon Nov 27 05:16:58 UTC 2023 hoodie.table.type=COPY_ON_WRITE +hoodie.table.metadata.partitions=files +hoodie.table.precombine.field=ts hoodie.archivelog.folder=archived -hoodie.populate.meta.fields=true +hoodie.table.create.schema={"type"\:"record","name"\:"hudi_non_part_cow_record","namespace"\:"hoodie.hudi_non_part_cow","fields"\:[{"name"\:"_hoodie_commit_time","type"\:["string","null"]},{"name"\:"_hoodie_commit_seqno","type"\:["string","null"]},{"name"\:"_hoodie_record_key","type"\:["string","null"]},{"name"\:"_hoodie_partition_path","type"\:["string","null"]},{"name"\:"_hoodie_file_name","type"\:["string","null"]},{"name"\:"id","type"\:["long","null"]},{"name"\:"name","type"\:["string","null"]},{"name"\:"ts","type"\:["long","null"]},{"name"\:"dt","type"\:["string","null"]},{"name"\:"hh","type"\:["string","null"]}]} hoodie.timeline.layout.version=1 -hoodie.table.version=3 -hoodie.table.recordkey.fields=rowId -hoodie.table.base.file.format=PARQUET -hoodie.table.keygenerator.class=org.apache.hudi.keygen.NonpartitionedKeyGenerator +hoodie.table.checksum=2968816715 +hoodie.datasource.write.drop.partition.columns=false +hoodie.table.recordkey.fields=id hoodie.table.name=hudi_non_part_cow -hoodie.datasource.write.hive_style_partitioning=false +hoodie.datasource.write.hive_style_partitioning=true +hoodie.table.keygenerator.class=org.apache.hudi.keygen.NonpartitionedKeyGenerator +hoodie.database.name=default +hoodie.datasource.write.partitionpath.urlencode=false +hoodie.table.version=5 diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/.hoodie_partition_metadata b/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/.hoodie_partition_metadata index f2149eb6cd5a3..e9de1b96c3ff3 100644 --- a/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/.hoodie_partition_metadata +++ b/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/.hoodie_partition_metadata @@ -1,4 +1,4 @@ #partition metadata -#Fri Dec 17 11:05:23 UTC 2021 -commitTime=20211217110514527 +#Mon Nov 27 05:16:59 UTC 2023 +commitTime=20231127051653361 partitionDepth=0 diff --git a/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/d0875d00-483d-4e8b-bbbe-c520366c47a0-0_0-6-11_20211217110514527.parquet b/plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/05b0f4ec-00fb-49f2-a1e2-7f510f3da93b-0_0-27-28_20231127051653361.parquet similarity index 98% rename from plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/d0875d00-483d-4e8b-bbbe-c520366c47a0-0_0-6-11_20211217110514527.parquet rename to plugin/trino-hudi/src/test/resources/hudi-testing-data/hudi_non_part_cow/05b0f4ec-00fb-49f2-a1e2-7f510f3da93b-0_0-27-28_20231127051653361.parquet index 52de8719bf62d7a762237ff4fae1887ee250f57a..c8368bd7c33cdbd5bedde73daab1af052e537fb6 100644 GIT binary patch delta 3472 zcmcIneQXnD7=Q12y)>Ndkcm3EF76sI42vNj8j3&Bu{m3R`8*M>N z2(3gG83`df!UzN~sQ85oI^#r#VZNkK6M; z@ALaTAMf)%cYEV#WzR>I$;I=RNSML1@eIFwfq&PygbNFN}8Gee%>c^KeNJ zC;R8le=xFk>GFYHpB&$LVcRdI;|acf(REjN4pe2sh3)H^>8cV=A*aiX9&dGhp5)W6 zs_+7QaLpU6Rf{?`C{qBpkS7?)Dx=rccZnpWD z1+Bcj{CC~eME(*oTVR(S-bBe*P*B~o`1@$?QIPGt4qP@r2|GfSQk?cyG1 z3T~}Q^s@}7ba3J$48MWm&rqOta`pmUTtLMm*VJyGYNwnlF5~&VWtiuGHQ^TI9M5=0 zoN|e)Ofviqir-9ux*>pEh11FQ#SG8z z_wgNI2Obx@C|RJSe5x}f)(=YXO~`owW@5TZqQGa*s4Eytv?ba)JDR$JiB^@aogbpW z*g2uIE2oThp8tbzjF;e<5DuH=P^h8`$TvmvLo}$JyY+VA-v4?_YapEEDh5vj1CH^0m()WqxsGlJ6;piat|{chJZ=MkX^fFF9ej3+fR(0w+3VXvobLK^r)b zk2Y~~+WaGbj*#!N;@gq)D3|LV^?Ir7$t8RjY>^g)83l7{Qpdba;9py@6FSD27g{2o z2`vVC2rWCr1TH(-NN7{F`Jo*m+}%kViES%-q!1~SS_nyk(K_iC{!PF=Wl#uy z$jCEJjz4QEhhf_;lx5nqCb%-zSrZSnMwSJIHS9oj$z=+TP&s0CDR)?llu=6^orq_b zkfh?W_`o;(nXUX~T}$0Lu*$Zwd|qpIm?Km~~sm$!Eg1wLR8s~mi6Fo*^P+vlx zFPHWK+m3h>aSG_i<-qfs*?$G)^w9zGtFh$pToyE;_-)4un!orpofb!(%HqT z5<>eDg#E>nE1si#G367tI- oZzviJguK3%+DO#vl_jq%hw6Q|h&a7@qpiYCE#*#@B|G!hNQ%>`%SxPumNtabk2}dC zlFm7u?S+yplp(e@Y{P{9(SO>EkaijLuMtKGgTe|eW!>2FM=62Obj$wPMl%XqG8lW` z(~mprPPU|+WS#CKjQsw1p6~O0KF|BU&nxNPyT^tf&xGya{o&(}hHYVc5d7*_4+qO% z$8+Q3>Hc*8z(D`N(7-_Y>_Fyh|DoW|Zv5?+!>B|Ug!hE!PKO;)^^WjxSL#?)Hk0Yk z3}rIuvzdXj{X=y&C4*C&Di<|S$z*DBe0+R*X1pW#e)^Yx8Rr)@{uhbxO5A!-ZhbH~ z71p$$O=r)f>Tb@|iT}}qHy-JDWNGP2H4hg)3vJ1Hp}N zyf|=ax$Tkgvo|$;uvoYrHC6tvUg(*FH!i;SukS9UmyW$KaU}SD__6QY7e{Ij4~OX* z_^GJ3LD2Qq!}mqwQR^I}f=Ds?9(~=?s6G5zBK&ruCjPmfc|KjtX{@?5W{pcrp2OhZVo~NJvlmAG5Ma<@V`Z`rOdjTsqx5aPCa5cOZXmvUhTFGT%FOwm*Gl zaBym1Xd>O4&Zc_@do!7A&4~3~nJ7MUHD5|KDo`ZVs89{d>2~85gpE7Cy29V88ULS0 zr&D}lzcCct7yM-I%l99xHwd2Ovdt;{`y8n}( z`n9|NuJ?!ka4;zT)rntrMMa}?_Gma*+l}e*@l#(N3WC9Jw*A|oxZL{7K2+1#&!Y>(sUKed%+;~KkFGX%-T0$FJyNg5!==JV{^9&wI<>YG zf^xQ)IBjuryyWZAm9kTX>$O|5#r#yEmkun zJkoi#<8XUtU7xT$*^&HOTk@T@p2HXF+%weK6bZX-HRM<;B1*cP9U@qa(nihoqd(t8K~uNknBY3~u$#qO&V%E24?pdZ)e8R-F+pxAez}HldEeYlkY)YLB0Cpw~K*zv$?B?a+mW&3g?C6pjAg zo7C$lzjCxP`pf5H)quZp^wtCZ-McG8RDJ4}L;PbR`SrG*@7{f(dIGLqj)*pzrt2j7 z#<8xjedP>Yy(E;|`RhdT{Y1|j$1YT6+R9SYM!lK1PW&GquRxS9GUedc;^Y3u$76=d zWL$mBzkP3opmI4X*ZoQ&`O`$t+xK25PpP%@YMmrrPgbI>yOfnfzMe?F(AM*MGU{vX z6dCumc56_lnIELO!nPI5%B8W~!V8Jyn~9zuq%JH^i*+Z*IuTz!Q5l`e#kCwU8lV45 z^t^l`%DHx?i*v5sS|{h^eJ{Oa(;dy_0NaC)LnY^O1}HS z?&Z0u`*e3aQAF!C^W(~FxKd*A%EVN;*aov{RBU=SKbKqVe(=dJt;SrP&8jBQsN8Y_ zPj)X{otukR7fZ$2g_-WBPIu=DQ&A$(rd)n{;_6(f`@y)#bEh|oSDthm60kB+HzeHJ z>>S-}lrA?U|_2pZW+N$yWMS18qXt#WD-s+Iqpx)XI zPi>P`$J+*#Z@xF$PK34FsM^ZWu3L3@?S`wiW__!to(odPt`}!Z`C{LEA(x-)i_KE) z@o68VKDKfTJ2_V<%x9y6C@PkXPr&G=HkVI*_~9>ncKCP4Mzfzw2dQ5v6leM-qVxRn z^4)S@xzci-)Qhj4cp(0|K0G`;GB7ecnmhm4%<%Bwe6&71{BZgEbX?$EyqF)pG`ukV z#6WqoZv+SsAV7cs0RjXF5FkK+009C72oTu1z{RasQG);h0t5&UAV7cs0RjXF5FkK+ z009C72oNAZfB=C82#hu$h7be@5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs z0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0((@z8^7++fsqyg0t5&UAV7cs0RjXF z5FkLH1q8-gppUvpfB*pk1PBlyK!5-N0=pqFvm1?Qg8%^n1PBlyK!5-N0t5&UAV7cs z0RjXF5FkK+009C72oNAZfB=CF1-ygFhBS5%AV7cs0RjXF5FoHl;L^Hvs{{xTAV7cs z0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72y9ls z(>a@!QjP!t0t5&UAV6RQ0RaL82oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+ z009C72oNAZ;DZW`eb4~v6Cgl<009C72y8;YBgC6PQiK2j0t5&UAV7cs0RjXF5Fl_X zftg$Fw2uG*0t5&UAV7cs0RnqMz&ix)iD8fg0RjXF5FkK+K$U=pMyfC@5+Fc;z)lNz z)%^B^{Yq(jZfX%AK!5-N0t5&UAV7e?CI$SSc#}|y5g3;3nMz8_*8BtU=w0RjXFG@Zam(`77S0tA{~ zz=fwthtMO_O*(MWCP09|CIn2*n?O>8009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs z0RjXF5FoH#z>6-{i&!T>fB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZ zfB*pk1PBly&>RAnnj=kF5+Fc;009C72oNAZfB=DgC*b2e_uY`{5CH-N2oNAZfB*pk z1PBlyK!5-N0t5&UAV7cs0RjXFw7kGf%L@P?K!5-N0vicQN z0t5&UAV7e?9TOP2V=Zf#009C72oNAZfB*pk1PBn=tbn&X*sPRt1PBlyK!5-N0t5&U zAV7cs0RjXF>_Gu9QQCupA|(O@2oNAZfB*pk1PBlyK!5-N0$UgG>0MhlQ-c5j0t5&U zAV7cs0RjYeR=|^YJL@Yt0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs z0RjXF5FkLH*#&$sce9V6{0R^sK!5-N0t5&UAV7cs0RjXF5FkK+009C7nq6R1A6-;_ z(^Kyz)9hm{e*y&dhJdT--WUg25FkK+009C72oNAZfB*pk1PHW>z)Y)jR6hw2AVA=b z2<(Ty{P+~qJ96k~lmGz&1PBlyK!5-N0t5&UAV7cs0RjYeOJGxv&+S$++95!I0D=7^ zV2asKqp2?h2oNAZfB=Ek5%B3CtuxkoN`L?X0t5&U*t)=ecuHpLy;g$&0RjXF5FkK+ z009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBly zK!5-N0t5&&Sm0uV;|WKA009C72oNAZfB*pk1PBmlfPhb0X#k=S1PBlyK!5-N0t5&U zAV7cs0RjXF5FkK+009C72<()=%uWRsmjD3*1PBlyK!5-N0t5&UAV7cs0RjZJCosCb zYHAT6K!5-N0t5&UAV7csfmRXlqjRf_w0;sGK!Ct*3wT}eZi{P=009C72oNAZfB*pk z1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+Kr0Ef?uWUxZqHlk_|{hf1PBlyK!5-N0t5&U zxbp&@j=J+>AOivf2oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZ zfB*pk1PBlyK!5-N0t5&UAV7e?jtKZD+Z|yQlmGz&1PBlyK!5-N0t5&UAV7cs0RjXF z5FkK+009C72oNB!PT=CYb*lsjY(?PGRthUifB*pk1PBlyK!5-N0t5&UAV7cs0RjXF z5FkKc{|JoipRVW$0RjXF5FkK+009C72oNAZfB*pk1PJV~z}ODQ7oGrt))w%PWNVMV z-V-1|fB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfWU4Fc-7u+YHz14 zZ=Js#wY^ryEAaQ)h{%Tk0RjXF5FkK+009ESe&BWx5FkK+009C72oNAZfB*pk1PBly zK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyKwwV_c&Cm%IV_c=W-5!# zzqGG9)8xb9%y04mls*9h1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB=CV z7VsLQ9R?Pj009C72oNAZfB*pk1PBlyK!5-N0?jBe+KfrdnE-)p3HW2tw%Al6K!5-N z0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+ zK$8kwYSO%=O@M%Bz#Rhw2oNAZfB*pk1PBly&|ra)2FDYQz|8{Q;Ob@%YXk@oAV7cs z0RjXF5FoJ61-vF_pAWF^5g%=*2oNAZfB*pk1PBly zK!5-N0t5&UAaLgdX6{_mS|&h%009C72viC9JijUoiv$P|AV7cs0RjXF5FkK+009C7 z2oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N z0t5&UAV7cs0RjYCPhhO|x~#`dDDay>D!g1M|E~T#JTgi>2uTP z&!6d^9L!G-4xTxaOAn?8&*#skr_N6fogYe1<|lKRPjtT76$G347o=Y52p$`LJoEp) CsCG~Q From 1afaa520693832b9d762f87d679b64d34f68de2f Mon Sep 17 00:00:00 2001 From: Vikash Kumar Date: Tue, 21 Nov 2023 18:20:08 +0530 Subject: [PATCH 201/350] Optional check for query partition filter for Hudi --- docs/src/main/sphinx/connector/hudi.md | 9 + .../java/io/trino/plugin/hudi/HudiConfig.java | 14 ++ .../io/trino/plugin/hudi/HudiMetadata.java | 52 ++++- .../plugin/hudi/HudiSessionProperties.java | 11 + .../io/trino/plugin/hudi/HudiTableHandle.java | 40 ++++ .../io/trino/plugin/hudi/TestHudiConfig.java | 7 +- .../trino/plugin/hudi/TestHudiSmokeTest.java | 198 ++++++++++++++++++ 7 files changed, 326 insertions(+), 5 deletions(-) diff --git a/docs/src/main/sphinx/connector/hudi.md b/docs/src/main/sphinx/connector/hudi.md index 8289e6fb6eb1f..2f8be98a3f3d7 100644 --- a/docs/src/main/sphinx/connector/hudi.md +++ b/docs/src/main/sphinx/connector/hudi.md @@ -82,6 +82,15 @@ Additionally, following configuration properties can be set depending on the use - Maximum number of metastore data objects per transaction in the Hive metastore cache. - `2000` +* - `hudi.query-partition-filter-required` + - Set to `true` to force a query to use a partition column in the filter condition. + The equivalent catalog session property is `query_partition_filter_required`. + Enabling this property causes query failures if the partition column used + in the filter condition doesn't effectively reduce the number of data files read. + Example: Complex filter expressions such as `id = 1 OR part_key = '100'` + or `CAST(part_key AS INTEGER) % 2 = 0` are not recognized as partition filters, + and queries using such expressions fail if the property is set to `true`. + - `false` ::: 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 c7fd2f84e4bbc..306287f9ff9fd 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 @@ -50,6 +50,7 @@ public class HudiConfig private int splitLoaderParallelism = 4; private int splitGeneratorParallelism = 4; private long perTransactionMetastoreCacheMaximumSize = 2000; + private boolean queryPartitionFilterRequired; public List getColumnsToHide() { @@ -193,4 +194,17 @@ public HudiConfig setPerTransactionMetastoreCacheMaximumSize(long perTransaction this.perTransactionMetastoreCacheMaximumSize = perTransactionMetastoreCacheMaximumSize; return this; } + + @Config("hudi.query-partition-filter-required") + @ConfigDescription("Require a filter on at least one partition column") + public HudiConfig setQueryPartitionFilterRequired(boolean queryPartitionFilterRequired) + { + this.queryPartitionFilterRequired = queryPartitionFilterRequired; + return this; + } + + public boolean isQueryPartitionFilterRequired() + { + return queryPartitionFilterRequired; + } } 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 152ca336bd8b0..e732474cae63c 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 @@ -45,23 +45,29 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.function.Function; +import java.util.stream.Stream; import static com.google.common.base.Strings.isNullOrEmpty; import static com.google.common.collect.ImmutableList.toImmutableList; import static com.google.common.collect.ImmutableMap.toImmutableMap; +import static com.google.common.collect.ImmutableSet.toImmutableSet; import static io.trino.plugin.hive.HiveMetadata.TABLE_COMMENT; import static io.trino.plugin.hive.HiveTimestampPrecision.NANOSECONDS; import static io.trino.plugin.hive.util.HiveUtil.columnMetadataGetter; +import static io.trino.plugin.hive.util.HiveUtil.getPartitionKeyColumnHandles; import static io.trino.plugin.hive.util.HiveUtil.hiveColumnHandles; import static io.trino.plugin.hive.util.HiveUtil.isHiveSystemSchema; import static io.trino.plugin.hive.util.HiveUtil.isHudiTable; import static io.trino.plugin.hudi.HudiErrorCode.HUDI_BAD_DATA; import static io.trino.plugin.hudi.HudiSessionProperties.getColumnsToHide; +import static io.trino.plugin.hudi.HudiSessionProperties.isQueryPartitionFilterRequired; import static io.trino.plugin.hudi.HudiTableProperties.LOCATION_PROPERTY; import static io.trino.plugin.hudi.HudiTableProperties.PARTITIONED_BY_PROPERTY; import static io.trino.plugin.hudi.HudiUtil.hudiMetadataExists; import static io.trino.plugin.hudi.model.HudiTableType.COPY_ON_WRITE; +import static io.trino.spi.StandardErrorCode.QUERY_REJECTED; import static io.trino.spi.StandardErrorCode.UNSUPPORTED_TABLE_TYPE; import static io.trino.spi.connector.SchemaTableName.schemaTableName; import static java.lang.String.format; @@ -114,6 +120,7 @@ public HudiTableHandle getTableHandle(ConnectorSession session, SchemaTableName tableName.getTableName(), table.get().getStorage().getLocation(), COPY_ON_WRITE, + getPartitionKeyColumnHandles(table.get(), typeManager), TupleDomain.all(), TupleDomain.all()); } @@ -162,12 +169,30 @@ public Optional> applyFilter(C { HudiTableHandle handle = (HudiTableHandle) tableHandle; HudiPredicates predicates = HudiPredicates.from(constraint.getSummary()); + TupleDomain regularColumnPredicates = predicates.getRegularColumnPredicates(); + TupleDomain partitionColumnPredicates = predicates.getPartitionColumnPredicates(); + + // TODO Since the constraint#predicate isn't utilized during split generation. So, + // Let's not add constraint#predicateColumns to newConstraintColumns. + Set newConstraintColumns = Stream.concat( + Stream.concat( + regularColumnPredicates.getDomains().stream() + .map(Map::keySet) + .flatMap(Collection::stream), + partitionColumnPredicates.getDomains().stream() + .map(Map::keySet) + .flatMap(Collection::stream)), + handle.getConstraintColumns().stream()) + .collect(toImmutableSet()); + HudiTableHandle newHudiTableHandle = handle.applyPredicates( - predicates.getPartitionColumnPredicates(), - predicates.getRegularColumnPredicates()); + newConstraintColumns, + partitionColumnPredicates, + regularColumnPredicates); if (handle.getPartitionPredicates().equals(newHudiTableHandle.getPartitionPredicates()) - && handle.getRegularPredicates().equals(newHudiTableHandle.getRegularPredicates())) { + && handle.getRegularPredicates().equals(newHudiTableHandle.getRegularPredicates()) + && handle.getConstraintColumns().equals(newHudiTableHandle.getConstraintColumns())) { return Optional.empty(); } @@ -224,6 +249,27 @@ public Iterator streamTableColumns(ConnectorSession sessio .iterator(); } + @Override + public void validateScan(ConnectorSession session, ConnectorTableHandle handle) + { + HudiTableHandle hudiTableHandle = (HudiTableHandle) handle; + if (isQueryPartitionFilterRequired(session)) { + if (!hudiTableHandle.getPartitionColumns().isEmpty()) { + Set partitionColumns = hudiTableHandle.getPartitionColumns().stream() + .map(HiveColumnHandle::getName) + .collect(toImmutableSet()); + Set constraintColumns = hudiTableHandle.getConstraintColumns().stream() + .map(HiveColumnHandle::getBaseColumnName) + .collect(toImmutableSet()); + if (Collections.disjoint(constraintColumns, partitionColumns)) { + throw new TrinoException( + QUERY_REJECTED, + format("Filter required on %s for at least one of the partition columns: %s", hudiTableHandle.getSchemaTableName(), String.join(", ", partitionColumns))); + } + } + } + } + HiveMetastore getMetastore() { return metastore; 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 ede43ec3386b6..f7946ff3ef9fa 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 @@ -50,6 +50,7 @@ public class HudiSessionProperties 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 SPLIT_GENERATOR_PARALLELISM = "split_generator_parallelism"; + private static final String QUERY_PARTITION_FILTER_REQUIRED = "query_partition_filter_required"; private final List> sessionProperties; @@ -113,6 +114,11 @@ public HudiSessionProperties(HudiConfig hudiConfig, ParquetReaderConfig parquetR SPLIT_GENERATOR_PARALLELISM, "Number of threads to generate splits from partitions", hudiConfig.getSplitGeneratorParallelism(), + false), + booleanProperty( + QUERY_PARTITION_FILTER_REQUIRED, + "Require a filter on at least one partition column", + hudiConfig.isQueryPartitionFilterRequired(), false)); } @@ -167,4 +173,9 @@ public static int getSplitGeneratorParallelism(ConnectorSession session) { return session.getProperty(SPLIT_GENERATOR_PARALLELISM, Integer.class); } + + public static boolean isQueryPartitionFilterRequired(ConnectorSession session) + { + return session.getProperty(QUERY_PARTITION_FILTER_REQUIRED, Boolean.class); + } } diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiTableHandle.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiTableHandle.java index 0da9f2d897a77..9101deb4de0bc 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiTableHandle.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiTableHandle.java @@ -14,13 +14,18 @@ package io.trino.plugin.hudi; import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableSet; import io.trino.plugin.hive.HiveColumnHandle; import io.trino.plugin.hudi.model.HudiTableType; import io.trino.spi.connector.ConnectorTableHandle; import io.trino.spi.connector.SchemaTableName; import io.trino.spi.predicate.TupleDomain; +import java.util.List; +import java.util.Set; + import static io.trino.spi.connector.SchemaTableName.schemaTableName; import static java.util.Objects.requireNonNull; @@ -31,6 +36,9 @@ public class HudiTableHandle private final String tableName; private final String basePath; private final HudiTableType tableType; + private final List partitionColumns; + // Used only for validation when config property hudi.query-partition-filter-required is enabled + private final Set constraintColumns; private final TupleDomain partitionPredicates; private final TupleDomain regularPredicates; @@ -40,13 +48,29 @@ public HudiTableHandle( @JsonProperty("tableName") String tableName, @JsonProperty("basePath") String basePath, @JsonProperty("tableType") HudiTableType tableType, + @JsonProperty("partitionColumns") List partitionColumns, @JsonProperty("partitionPredicates") TupleDomain partitionPredicates, @JsonProperty("regularPredicates") TupleDomain regularPredicates) + { + this(schemaName, tableName, basePath, tableType, partitionColumns, ImmutableSet.of(), partitionPredicates, regularPredicates); + } + + public HudiTableHandle( + String schemaName, + String tableName, + String basePath, + HudiTableType tableType, + List partitionColumns, + Set constraintColumns, + TupleDomain partitionPredicates, + TupleDomain regularPredicates) { this.schemaName = requireNonNull(schemaName, "schemaName is null"); this.tableName = requireNonNull(tableName, "tableName is null"); this.basePath = requireNonNull(basePath, "basePath is null"); this.tableType = requireNonNull(tableType, "tableType is null"); + this.partitionColumns = requireNonNull(partitionColumns, "partitionColumns is null"); + this.constraintColumns = requireNonNull(constraintColumns, "constraintColumns is null"); this.partitionPredicates = requireNonNull(partitionPredicates, "partitionPredicates is null"); this.regularPredicates = requireNonNull(regularPredicates, "regularPredicates is null"); } @@ -81,6 +105,19 @@ public TupleDomain getPartitionPredicates() return partitionPredicates; } + @JsonProperty + public List getPartitionColumns() + { + return partitionColumns; + } + + // do not serialize constraint columns as they are not needed on workers + @JsonIgnore + public Set getConstraintColumns() + { + return constraintColumns; + } + @JsonProperty public TupleDomain getRegularPredicates() { @@ -93,6 +130,7 @@ public SchemaTableName getSchemaTableName() } HudiTableHandle applyPredicates( + Set constraintColumns, TupleDomain partitionTupleDomain, TupleDomain regularTupleDomain) { @@ -101,6 +139,8 @@ HudiTableHandle applyPredicates( tableName, basePath, tableType, + partitionColumns, + constraintColumns, partitionPredicates.intersect(partitionTupleDomain), regularPredicates.intersect(regularTupleDomain)); } diff --git a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiConfig.java b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiConfig.java index 2aaed93bcad8a..719ef64bce1a5 100644 --- a/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiConfig.java +++ b/plugin/trino-hudi/src/test/java/io/trino/plugin/hudi/TestHudiConfig.java @@ -39,7 +39,8 @@ public void testDefaults() .setMaxOutstandingSplits(1000) .setSplitLoaderParallelism(4) .setSplitGeneratorParallelism(4) - .setPerTransactionMetastoreCacheMaximumSize(2000)); + .setPerTransactionMetastoreCacheMaximumSize(2000) + .setQueryPartitionFilterRequired(false)); } @Test @@ -56,6 +57,7 @@ public void testExplicitPropertyMappings() .put("hudi.split-loader-parallelism", "16") .put("hudi.split-generator-parallelism", "32") .put("hudi.per-transaction-metastore-cache-maximum-size", "1000") + .put("hudi.query-partition-filter-required", "true") .buildOrThrow(); HudiConfig expected = new HudiConfig() @@ -68,7 +70,8 @@ public void testExplicitPropertyMappings() .setMaxOutstandingSplits(100) .setSplitLoaderParallelism(16) .setSplitGeneratorParallelism(32) - .setPerTransactionMetastoreCacheMaximumSize(1000); + .setPerTransactionMetastoreCacheMaximumSize(1000) + .setQueryPartitionFilterRequired(true); assertFullMapping(properties, expected); } 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 565cf9d1fed6b..6313e921cdf2a 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 @@ -14,6 +14,7 @@ package io.trino.plugin.hudi; import com.google.common.collect.ImmutableMap; +import io.trino.Session; import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.filesystem.TrinoInputFile; @@ -21,6 +22,7 @@ import io.trino.spi.security.ConnectorIdentity; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.QueryRunner; +import org.intellij.lang.annotations.Language; import org.junit.jupiter.api.Test; import java.time.ZonedDateTime; @@ -168,6 +170,202 @@ public void testPartitionColumn() assertQueryFails("SELECT \"$partition\" FROM " + HUDI_NON_PART_COW, ".* Column '\\$partition' cannot be resolved"); } + @Test + public void testPartitionFilterRequired() + { + Session session = withPartitionFilterRequired(getSession()); + + assertQueryFails( + session, + "SELECT * FROM " + HUDI_COW_PT_TBL, + "Filter required on tests." + HUDI_COW_PT_TBL.getTableName() + " for at least one of the partition columns: dt, hh"); + } + + @Test + public void testPartitionFilterRequiredPredicateOnNonPartitionColumn() + { + Session session = withPartitionFilterRequired(getSession()); + + assertQueryFails( + session, + "SELECT * FROM " + HUDI_COW_PT_TBL + " WHERE id = 1", + "Filter required on tests." + HUDI_COW_PT_TBL.getTableName() + " for at least one of the partition columns: dt, hh"); + } + + @Test + public void testPartitionFilterRequiredNestedQueryWithInnerPartitionPredicate() + { + Session session = withPartitionFilterRequired(getSession()); + + assertQuery(session, "SELECT name FROM (SELECT * FROM " + HUDI_COW_PT_TBL + " WHERE dt = '2021-12-09') WHERE id = 1", "VALUES 'a1'"); + } + + @Test + public void testPartitionFilterRequiredNestedQueryWithOuterPartitionPredicate() + { + Session session = withPartitionFilterRequired(getSession()); + + assertQuery(session, "SELECT name FROM (SELECT * FROM " + HUDI_COW_PT_TBL + " WHERE id = 1) WHERE dt = '2021-12-09'", "VALUES 'a1'"); + } + + @Test + public void testPartitionFilterRequiredNestedWithIsNotNullFilter() + { + Session session = withPartitionFilterRequired(getSession()); + + assertQuery(session, "SELECT name FROM " + HUDI_COW_PT_TBL + " WHERE dt IS NOT null", "VALUES 'a1', 'a2'"); + } + + @Test + public void testPartitionFilterRequiredFilterRemovedByPlanner() + { + Session session = withPartitionFilterRequired(getSession()); + + assertQueryFails( + session, + "SELECT id FROM " + HUDI_COW_PT_TBL + " WHERE dt IS NOT null OR true", + "Filter required on tests." + HUDI_COW_PT_TBL.getTableName() + " for at least one of the partition columns: dt, hh"); + } + + @Test + public void testPartitionFilterRequiredOnJoin() + { + Session session = withPartitionFilterRequired(getSession()); + @Language("RegExp") String errorMessage = "Filter required on tests." + HUDI_COW_PT_TBL.getTableName() + " for at least one of the partition columns: dt, hh"; + + // ON with partition column + assertQueryFails( + session, + "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_NON_PART_COW + " t2 ON (t1.dt = t2.dt)", + errorMessage); + // ON with partition column and WHERE with same left table's partition column + assertQuery( + session, + "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_NON_PART_COW + " t2 ON (t1.dt = t2.dt) WHERE t1.dt = '2021-12-09'", + "VALUES ('a1', 'a1'), ('a2', 'a2'), ('a1', 'a2'), ('a2', 'a1')"); + // ON with partition column and WHERE with same right table's regular column + assertQuery( + session, + "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_NON_PART_COW + " t2 ON (t1.dt = t2.dt) WHERE t2.dt = '2021-12-09'", + "VALUES ('a1', 'a1'), ('a2', 'a2'), ('a1', 'a2'), ('a2', 'a1')"); + // ON with partition column and WHERE with different left table's partition column + assertQuery( + session, + "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_NON_PART_COW + " t2 ON (t1.dt = t2.dt) WHERE t1.hh = '10'", + "VALUES ('a1', 'a1'), ('a1', 'a2')"); + // ON with partition column and WHERE with different regular column + assertQueryFails( + session, + "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_NON_PART_COW + " t2 ON (t1.dt = t2.dt) WHERE t2.hh = '10'", + errorMessage); + // ON with partition column and WHERE with regular column + assertQueryFails( + session, + "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_NON_PART_COW + " t2 ON (t1.dt = t2.dt) WHERE t1.id = 1", + errorMessage); + + // ON with regular column + assertQueryFails( + session, + "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_NON_PART_COW + " t2 ON (t1.id = t2.id)", + errorMessage); + // ON with regular column and WHERE with left table's partition column + assertQuery( + session, + "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_NON_PART_COW + " t2 ON (t1.id = t2.id) WHERE t1.dt = '2021-12-09'", + "VALUES ('a1', 'a1'), ('a2', 'a2')"); + // ON with partition column and WHERE with right table's regular column + assertQueryFails( + session, + "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_NON_PART_COW + " t2 ON (t1.dt = t2.dt) WHERE t2.id = 1", + errorMessage); + } + + @Test + public void testPartitionFilterRequiredOnJoinBothTablePartitioned() + { + Session session = withPartitionFilterRequired(getSession()); + + // ON with partition column + assertQueryFails( + session, + "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_COW_PT_TBL + " t2 ON (t1.dt = t2.dt)", + "Filter required on tests." + HUDI_COW_PT_TBL.getTableName() + " for at least one of the partition columns: dt, hh"); + // ON with partition column and WHERE with same left table's partition column + assertQuery( + session, + "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_COW_PT_TBL + " t2 ON (t1.dt = t2.dt) WHERE t1.dt = '2021-12-09'", + "VALUES ('a1', 'a1'), ('a2', 'a2'), ('a1', 'a2'), ('a2', 'a1')"); + // ON with partition column and WHERE with same right table's partition column + assertQuery( + session, + "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_COW_PT_TBL + " t2 ON (t1.dt = t2.dt) WHERE t2.dt = '2021-12-09'", + "VALUES ('a1', 'a1'), ('a2', 'a2'), ('a1', 'a2'), ('a2', 'a1')"); + + @Language("RegExp") String errorMessage = "Filter required on tests." + HUDI_COW_PT_TBL.getTableName() + " for at least one of the partition columns: dt, hh"; + // ON with partition column and WHERE with different left table's partition column + assertQueryFails(session, "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_COW_PT_TBL + " t2 ON (t1.dt = t2.dt) WHERE t1.hh = '10'", errorMessage); + // ON with partition column and WHERE with different right table's partition column + assertQueryFails(session, "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_COW_PT_TBL + " t2 ON (t1.dt = t2.dt) WHERE t2.hh = '10'", errorMessage); + // ON with partition column and WHERE with regular column + assertQueryFails(session, "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_COW_PT_TBL + " t2 ON (t1.dt = t2.dt) WHERE t2.id = 1", errorMessage); + + // ON with regular column + assertQueryFails(session, "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_COW_PT_TBL + " t2 ON (t1.id = t2.id)", errorMessage); + // ON with regular column and WHERE with regular column + assertQueryFails(session, "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_COW_PT_TBL + " t2 ON (t1.id = t2.id) WHERE t1.id = 1", errorMessage); + // ON with regular column and WHERE with left table's partition column + assertQueryFails(session, "SELECT t1.name, t2.name FROM " + HUDI_COW_PT_TBL + " t1 JOIN " + HUDI_COW_PT_TBL + " t2 ON (t1.id = t2.id) WHERE t1.dt = '2021-12-09'", errorMessage); + } + + @Test + public void testPartitionFilterRequiredWithLike() + { + Session session = withPartitionFilterRequired(getSession()); + assertQueryFails( + session, + "SELECT name FROM " + HUDI_COW_PT_TBL + " WHERE name LIKE '%1'", + "Filter required on tests." + HUDI_COW_PT_TBL.getTableName() + " for at least one of the partition columns: dt, hh"); + } + + @Test + public void testPartitionFilterRequiredFilterIncluded() + { + Session session = withPartitionFilterRequired(getSession()); + assertQuery(session, "SELECT name FROM " + HUDI_COW_PT_TBL + " WHERE hh = '10'", "VALUES 'a1'"); + assertQuery(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE hh < '12'", "VALUES 2"); + assertQuery(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE Hh < '11'", "VALUES 1"); + assertQuery(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE HH < '10'", "VALUES 0"); + assertQuery(session, "SELECT name FROM " + HUDI_COW_PT_TBL + " WHERE CAST(hh AS INTEGER) % 2 = 1 and hh IS NOT NULL", "VALUES 'a2'"); + assertQuery(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE hh IS NULL", "VALUES 0"); + assertQuery(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE hh IS NOT NULL", "VALUES 2"); + assertQuery(session, "SELECT name FROM " + HUDI_COW_PT_TBL + " WHERE hh LIKE '10'", "VALUES 'a1'"); + assertQuery(session, "SELECT name FROM " + HUDI_COW_PT_TBL + " WHERE hh LIKE '1%'", "VALUES 'a1', 'a2'"); + assertQuery(session, "SELECT name FROM " + HUDI_COW_PT_TBL + " WHERE id = 1 AND dt = '2021-12-09'", "VALUES 'a1'"); + assertQuery(session, "SELECT name FROM " + HUDI_COW_PT_TBL + " WHERE hh = '11' AND dt = '2021-12-09'", "VALUES 'a2'"); + assertQuery(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE hh = '12' AND dt = '2021-12-19'", "VALUES 0"); + + // Predicate which could not be translated into tuple domain + @Language("RegExp") String errorMessage = "Filter required on tests." + HUDI_COW_PT_TBL.getTableName() + " for at least one of the partition columns: dt, hh"; + assertQueryFails(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE CAST(hh AS INTEGER) % 2 = 0", errorMessage); + assertQueryFails(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE CAST(hh AS INTEGER) - 11 = 0", errorMessage); + assertQueryFails(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE CAST(hh AS INTEGER) * 2 = 20", errorMessage); + assertQueryFails(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE CAST(hh AS INTEGER) % 2 > 0", errorMessage); + assertQueryFails(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE name LIKE '%1' OR hh LIKE '%1'", errorMessage); + assertQueryFails(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE name LIKE '%1' AND hh LIKE '%0'", errorMessage); + assertQueryFails(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE id = 1 OR dt = '2021-12-09'", errorMessage); + assertQueryFails(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE hh = '11' OR dt = '2021-12-09'", errorMessage); + assertQueryFails(session, "SELECT count(*) FROM " + HUDI_COW_PT_TBL + " WHERE hh = '12' OR dt = '2021-12-19'", errorMessage); + assertQueryFails(session, "SELECT count(*) AS COUNT FROM " + HUDI_COW_PT_TBL + " WHERE CAST(hh AS INTEGER) > 2 GROUP BY name ", errorMessage); + } + + private static Session withPartitionFilterRequired(Session session) + { + return Session.builder(session) + .setCatalogSessionProperty(session.getCatalog().orElseThrow(), "query_partition_filter_required", "true") + .build(); + } + private TrinoInputFile toInputFile(String path) { return ((HudiConnector) getDistributedQueryRunner().getCoordinator().getConnector("hudi")).getInjector() From 1236521cc0fc6b73aa65833f59ef40abd6f59379 Mon Sep 17 00:00:00 2001 From: Jonas Irgens Kylling Date: Sat, 18 Nov 2023 14:41:08 +0100 Subject: [PATCH 202/350] Throw IndexOutOfBoundsException in InputStream implementations --- .../java/io/trino/filesystem/memory/MemoryInputStream.java | 2 ++ .../io/trino/filesystem/AbstractTestTrinoFileSystem.java | 7 +++++++ .../io/trino/filesystem/hdfs/HdfsTrinoInputStream.java | 2 ++ 3 files changed, 11 insertions(+) diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryInputStream.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryInputStream.java index 573093a9a14af..05830d68aa037 100644 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryInputStream.java +++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryInputStream.java @@ -20,6 +20,7 @@ import java.io.IOException; +import static java.util.Objects.checkFromIndexSize; import static java.util.Objects.requireNonNull; class MemoryInputStream @@ -78,6 +79,7 @@ public int read(byte[] destination, int destinationIndex, int length) throws IOException { ensureOpen(); + checkFromIndexSize(destinationIndex, length, destination.length); return input.read(destination, destinationIndex, length); } diff --git a/lib/trino-filesystem/src/test/java/io/trino/filesystem/AbstractTestTrinoFileSystem.java b/lib/trino-filesystem/src/test/java/io/trino/filesystem/AbstractTestTrinoFileSystem.java index fa6ead06f44fe..ee5a7c03b8b7d 100644 --- a/lib/trino-filesystem/src/test/java/io/trino/filesystem/AbstractTestTrinoFileSystem.java +++ b/lib/trino-filesystem/src/test/java/io/trino/filesystem/AbstractTestTrinoFileSystem.java @@ -391,6 +391,13 @@ public void testInputFile() assertThat(inputStream.getPosition()).isEqualTo(fileSize + 100); } + assertThatThrownBy(() -> inputStream.read(new byte[1], -1, 0)) + .isInstanceOf(IndexOutOfBoundsException.class); + assertThatThrownBy(() -> inputStream.read(new byte[1], 0, -1)) + .isInstanceOf(IndexOutOfBoundsException.class); + assertThatThrownBy(() -> inputStream.read(new byte[1], 1, 3)) + .isInstanceOf(IndexOutOfBoundsException.class); + // verify all the methods throw after close inputStream.close(); assertThatThrownBy(inputStream::available) diff --git a/lib/trino-hdfs/src/main/java/io/trino/filesystem/hdfs/HdfsTrinoInputStream.java b/lib/trino-hdfs/src/main/java/io/trino/filesystem/hdfs/HdfsTrinoInputStream.java index 8e1fa57b06e4b..f5374dc751a66 100644 --- a/lib/trino-hdfs/src/main/java/io/trino/filesystem/hdfs/HdfsTrinoInputStream.java +++ b/lib/trino-hdfs/src/main/java/io/trino/filesystem/hdfs/HdfsTrinoInputStream.java @@ -21,6 +21,7 @@ import java.io.IOException; import static io.trino.filesystem.hdfs.HdfsFileSystem.withCause; +import static java.util.Objects.checkFromIndexSize; import static java.util.Objects.requireNonNull; class HdfsTrinoInputStream @@ -96,6 +97,7 @@ public int read(byte[] b, int off, int len) throws IOException { ensureOpen(); + checkFromIndexSize(off, len, b.length); try { return stream.read(b, off, len); } From 3a31d3ab6bf808c487960213dfb80c005e493b4b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C5=81ukasz=20Osipiuk?= Date: Thu, 21 Dec 2023 14:16:00 +0100 Subject: [PATCH 203/350] Catch exception in sendUpdate Catch exception in sendUpdate. Currently in case we get unexpected exception from that method query would fail as thread responsible for updating task state will terminate. --- .../io/trino/server/remotetask/HttpRemoteTask.java | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/core/trino-main/src/main/java/io/trino/server/remotetask/HttpRemoteTask.java b/core/trino-main/src/main/java/io/trino/server/remotetask/HttpRemoteTask.java index 1bfbfaf5919d7..af868a76358f3 100644 --- a/core/trino-main/src/main/java/io/trino/server/remotetask/HttpRemoteTask.java +++ b/core/trino-main/src/main/java/io/trino/server/remotetask/HttpRemoteTask.java @@ -117,6 +117,7 @@ import static io.trino.execution.TaskStatus.failWith; import static io.trino.server.remotetask.RequestErrorTracker.logError; import static io.trino.spi.HostAddress.fromUri; +import static io.trino.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR; import static io.trino.spi.StandardErrorCode.REMOTE_TASK_ERROR; import static io.trino.util.Failures.toFailure; import static java.lang.Math.addExact; @@ -714,6 +715,16 @@ boolean adjustSplitBatchSize(List splitAssignments, long reques } private void sendUpdate() + { + try { + sendUpdateInternal(); + } + catch (Throwable e) { + fatalUnacknowledgedFailure(new TrinoException(GENERIC_INTERNAL_ERROR, "unexpected error calling sendUpdate()", e)); + } + } + + private void sendUpdateInternal() { TaskStatus taskStatus = getTaskStatus(); // don't update if the task is already finishing or finished, or if we have sent a termination command From f5def2df0ad9539351335a385db37d6db5c71614 Mon Sep 17 00:00:00 2001 From: Martin Traverso Date: Thu, 21 Dec 2023 07:35:32 -0800 Subject: [PATCH 204/350] Restore lifecycle annotations in tests They are not redundant, as the tests contain lifecycle annotations such as BeforeAll and AfterAll. The fact that AbstractTestQueries also has them is an implementation detail to manage its own private internal state, and subclasses should not rely on them being present. This reverts commit f6ea5b452c0d77e150ff2b07cf30a3be5bec5e0c. --- .../io/trino/plugin/cassandra/TestCassandraTypeMapping.java | 3 +++ .../io/trino/plugin/deltalake/TestDeltaLakeAdlsStorage.java | 3 +++ .../java/io/trino/plugin/deltalake/TestDeltaLakeBasic.java | 3 +++ .../deltalake/TestDeltaLakeFlushMetadataCacheProcedure.java | 3 +++ .../trino/plugin/deltalake/TestDeltaLakePartitioning.java | 3 +++ .../trino/plugin/deltalake/TestDeltaLakeReadTimestamps.java | 3 +++ .../TestDeltaLakeSharedHiveMetastoreWithViews.java | 3 +++ .../plugin/deltalake/TestDeltaLakeTableStatistics.java | 3 +++ .../java/io/trino/plugin/deltalake/TestSplitPruning.java | 3 +++ .../TestDeltaLakeConcurrentModificationGlueMetastore.java | 3 +++ .../glue/TestDeltaLakeRenameToWithGlueMetastore.java | 3 +++ .../metastore/glue/TestDeltaLakeViewsGlueMetastore.java | 3 +++ .../java/io/trino/plugin/druid/TestDruidTypeMapping.java | 3 +++ .../plugin/elasticsearch/TestElasticsearchBackpressure.java | 3 +++ .../test/java/io/trino/plugin/hive/TestHive3OnDataLake.java | 3 +++ .../hive/TestHiveConcurrentModificationGlueMetastore.java | 3 +++ .../src/test/java/io/trino/plugin/hive/TestShowStats.java | 3 +++ .../cache/TestCachingHiveMetastoreWithQueryRunner.java | 3 +++ .../plugin/hive/s3/TestS3FileSystemAccessOperations.java | 3 +++ .../iceberg/TestIcebergGetTableStatisticsOperations.java | 3 +++ .../io/trino/plugin/iceberg/TestIcebergMetadataListing.java | 3 +++ .../trino/plugin/iceberg/TestIcebergReadVersionedTable.java | 3 +++ .../iceberg/TestIcebergReadVersionedTableByTemporal.java | 3 +++ .../io/trino/plugin/iceberg/TestIcebergSplitSource.java | 3 +++ .../iceberg/TestIcebergTableWithExternalLocation.java | 3 +++ .../test/java/io/trino/plugin/iceberg/TestIcebergV2.java | 3 +++ .../catalog/glue/TestIcebergGlueCatalogSkipArchive.java | 3 +++ .../glue/TestIcebergGlueTableOperationsInsertFailure.java | 3 +++ .../plugin/kudu/TestKuduIntegrationDecimalColumns.java | 3 +++ .../io/trino/plugin/mariadb/TestMariaDbTypeMapping.java | 6 ++++++ .../plugin/mongodb/TestMongoCaseInsensitiveMapping.java | 6 ++++++ .../java/io/trino/plugin/mysql/TestMySqlTypeMapping.java | 6 ++++++ .../io/trino/plugin/phoenix5/TestPhoenixTypeMapping.java | 6 ++++++ .../trino/plugin/postgresql/TestPostgreSqlTypeMapping.java | 6 ++++++ .../plugin/singlestore/TestSingleStoreTypeMapping.java | 6 ++++++ .../informationschema/TestInformationSchemaConnector.java | 3 +++ .../io/trino/connector/system/runtime/TestKillQuery.java | 3 +++ .../java/io/trino/execution/TestEventListenerBasic.java | 3 +++ .../src/test/java/io/trino/execution/TestQueryTracker.java | 3 +++ .../io/trino/execution/TestRefreshMaterializedView.java | 3 +++ .../io/trino/tests/TestGetTableStatisticsOperations.java | 3 +++ .../src/test/java/io/trino/tests/TestProcedureCall.java | 3 +++ 42 files changed, 144 insertions(+) diff --git a/plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/TestCassandraTypeMapping.java b/plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/TestCassandraTypeMapping.java index 887ce0291569c..6894781f83697 100644 --- a/plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/TestCassandraTypeMapping.java +++ b/plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/TestCassandraTypeMapping.java @@ -31,6 +31,7 @@ import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; import java.time.LocalDate; import java.time.LocalDateTime; @@ -67,7 +68,9 @@ import static java.lang.String.format; import static java.time.ZoneOffset.UTC; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +@TestInstance(PER_CLASS) public class TestCassandraTypeMapping extends AbstractTestQueryFramework { diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeAdlsStorage.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeAdlsStorage.java index 1c08f5bd22577..066260afe5573 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeAdlsStorage.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeAdlsStorage.java @@ -22,6 +22,7 @@ import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; import org.testcontainers.containers.Network; import java.nio.file.Files; @@ -42,7 +43,9 @@ import static java.nio.charset.StandardCharsets.UTF_8; import static java.util.Objects.requireNonNull; import static java.util.UUID.randomUUID; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +@TestInstance(PER_CLASS) public class TestDeltaLakeAdlsStorage extends AbstractTestQueryFramework { diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeBasic.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeBasic.java index c1256054224a0..0d649b1ed22ef 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeBasic.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeBasic.java @@ -46,6 +46,7 @@ import org.intellij.lang.annotations.Language; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; import org.junit.jupiter.api.parallel.Execution; import java.io.File; @@ -80,8 +81,10 @@ import static java.time.ZoneOffset.UTC; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.entry; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; +@TestInstance(PER_CLASS) @Execution(SAME_THREAD) public class TestDeltaLakeBasic extends AbstractTestQueryFramework diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeFlushMetadataCacheProcedure.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeFlushMetadataCacheProcedure.java index b0f6af428db21..4bbe568ba3a70 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeFlushMetadataCacheProcedure.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeFlushMetadataCacheProcedure.java @@ -21,6 +21,7 @@ import io.trino.testing.QueryRunner; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; import static io.trino.plugin.deltalake.DeltaLakeQueryRunner.DELTA_CATALOG; import static io.trino.plugin.deltalake.DeltaLakeQueryRunner.createS3DeltaLakeQueryRunner; @@ -28,7 +29,9 @@ import static io.trino.plugin.hive.containers.HiveHadoop.HIVE3_IMAGE; import static io.trino.testing.TestingNames.randomNameSuffix; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +@TestInstance(PER_CLASS) public class TestDeltaLakeFlushMetadataCacheProcedure extends AbstractTestQueryFramework { diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakePartitioning.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakePartitioning.java index 24d01476e1c10..d614ebc19c035 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakePartitioning.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakePartitioning.java @@ -18,12 +18,15 @@ import io.trino.testing.QueryRunner; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; import static io.trino.plugin.deltalake.DeltaLakeQueryRunner.DELTA_CATALOG; import static io.trino.plugin.deltalake.DeltaLakeQueryRunner.createDeltaLakeQueryRunner; import static io.trino.testing.TestingNames.randomNameSuffix; import static java.lang.String.format; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +@TestInstance(PER_CLASS) public class TestDeltaLakePartitioning extends AbstractTestQueryFramework { diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeReadTimestamps.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeReadTimestamps.java index 8e1708a3dfb10..7201465bf0432 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeReadTimestamps.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeReadTimestamps.java @@ -21,6 +21,7 @@ import io.trino.testing.QueryRunner; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; import java.time.Instant; import java.time.LocalDateTime; @@ -48,7 +49,9 @@ import static java.time.temporal.ChronoField.MONTH_OF_YEAR; import static java.time.temporal.ChronoField.YEAR; import static java.util.stream.Collectors.joining; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +@TestInstance(PER_CLASS) public class TestDeltaLakeReadTimestamps extends AbstractTestQueryFramework { diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSharedHiveMetastoreWithViews.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSharedHiveMetastoreWithViews.java index 6cd482390b335..e84783eb94a81 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSharedHiveMetastoreWithViews.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSharedHiveMetastoreWithViews.java @@ -21,6 +21,7 @@ import io.trino.testing.QueryRunner; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; import java.util.Map; @@ -30,7 +31,9 @@ import static io.trino.testing.containers.Minio.MINIO_SECRET_KEY; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +@TestInstance(PER_CLASS) public class TestDeltaLakeSharedHiveMetastoreWithViews extends AbstractTestQueryFramework { diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeTableStatistics.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeTableStatistics.java index 5562072fc5dd5..9807cdc6a37e3 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeTableStatistics.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeTableStatistics.java @@ -19,12 +19,15 @@ import io.trino.testing.QueryRunner; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; import static io.trino.plugin.deltalake.DeltaLakeQueryRunner.DELTA_CATALOG; import static io.trino.plugin.deltalake.DeltaLakeQueryRunner.createDeltaLakeQueryRunner; import static io.trino.testing.TestingNames.randomNameSuffix; import static java.lang.String.format; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +@TestInstance(PER_CLASS) public class TestDeltaLakeTableStatistics extends AbstractTestQueryFramework { diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestSplitPruning.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestSplitPruning.java index b2546c7e76505..99ff36ebfa627 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestSplitPruning.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestSplitPruning.java @@ -24,6 +24,7 @@ import org.intellij.lang.annotations.Language; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; import java.util.Arrays; import java.util.List; @@ -35,7 +36,9 @@ import static io.trino.plugin.deltalake.DeltaLakeQueryRunner.createDeltaLakeQueryRunner; import static java.lang.String.format; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +@TestInstance(PER_CLASS) public class TestSplitPruning extends AbstractTestQueryFramework { diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeConcurrentModificationGlueMetastore.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeConcurrentModificationGlueMetastore.java index c596157b662b3..04092fe237db9 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeConcurrentModificationGlueMetastore.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeConcurrentModificationGlueMetastore.java @@ -28,6 +28,7 @@ import io.trino.testing.QueryRunner; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; import java.io.IOException; import java.lang.reflect.InvocationTargetException; @@ -46,7 +47,9 @@ import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.testing.TestingSession.testSessionBuilder; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +@TestInstance(PER_CLASS) public class TestDeltaLakeConcurrentModificationGlueMetastore extends AbstractTestQueryFramework { diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeRenameToWithGlueMetastore.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeRenameToWithGlueMetastore.java index f7a2ff52b92b5..4e15c1e55eef9 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeRenameToWithGlueMetastore.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeRenameToWithGlueMetastore.java @@ -21,13 +21,16 @@ import io.trino.testing.QueryRunner; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; import java.nio.file.Path; import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.testing.TestingSession.testSessionBuilder; import static java.lang.String.format; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +@TestInstance(PER_CLASS) public class TestDeltaLakeRenameToWithGlueMetastore extends AbstractTestQueryFramework { diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeViewsGlueMetastore.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeViewsGlueMetastore.java index 6a4cc5a7a1e67..9fee324abe606 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeViewsGlueMetastore.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/metastore/glue/TestDeltaLakeViewsGlueMetastore.java @@ -24,6 +24,7 @@ import io.trino.testing.sql.TestView; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; import java.io.IOException; import java.nio.file.Path; @@ -36,7 +37,9 @@ import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.testing.TestingSession.testSessionBuilder; import static java.lang.String.format; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +@TestInstance(PER_CLASS) public class TestDeltaLakeViewsGlueMetastore extends AbstractTestQueryFramework { diff --git a/plugin/trino-druid/src/test/java/io/trino/plugin/druid/TestDruidTypeMapping.java b/plugin/trino-druid/src/test/java/io/trino/plugin/druid/TestDruidTypeMapping.java index c9e9255e1b6ac..a13cea76c9b01 100644 --- a/plugin/trino-druid/src/test/java/io/trino/plugin/druid/TestDruidTypeMapping.java +++ b/plugin/trino-druid/src/test/java/io/trino/plugin/druid/TestDruidTypeMapping.java @@ -23,6 +23,7 @@ import io.trino.testing.datatype.SqlDataTypeTest; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; import org.junit.jupiter.api.parallel.Isolated; import java.io.BufferedWriter; @@ -41,7 +42,9 @@ import static java.util.Objects.requireNonNull; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +@TestInstance(PER_CLASS) @Isolated public class TestDruidTypeMapping extends AbstractTestQueryFramework diff --git a/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchBackpressure.java b/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchBackpressure.java index bad037ce2bd80..dbaf931760664 100644 --- a/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchBackpressure.java +++ b/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchBackpressure.java @@ -19,13 +19,16 @@ import io.trino.testing.QueryRunner; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; import org.testcontainers.containers.Network; import java.io.IOException; import static io.trino.plugin.elasticsearch.ElasticsearchQueryRunner.createElasticsearchQueryRunner; import static io.trino.tpch.TpchTable.ORDERS; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +@TestInstance(PER_CLASS) public class TestElasticsearchBackpressure extends AbstractTestQueryFramework { diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHive3OnDataLake.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHive3OnDataLake.java index 79bb9c2e42bdb..24d40204c58f1 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHive3OnDataLake.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHive3OnDataLake.java @@ -34,6 +34,7 @@ import io.trino.testing.minio.MinioClient; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; import java.text.DateFormat; import java.text.SimpleDateFormat; @@ -63,7 +64,9 @@ import static java.util.stream.Collectors.joining; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +@TestInstance(PER_CLASS) public class TestHive3OnDataLake extends AbstractTestQueryFramework { diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveConcurrentModificationGlueMetastore.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveConcurrentModificationGlueMetastore.java index b2bb2f0a7b3aa..fb08016d7200a 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveConcurrentModificationGlueMetastore.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveConcurrentModificationGlueMetastore.java @@ -26,6 +26,7 @@ import io.trino.testing.QueryRunner; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; import java.io.IOException; import java.lang.reflect.InvocationTargetException; @@ -43,7 +44,9 @@ import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.testing.TestingSession.testSessionBuilder; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +@TestInstance(PER_CLASS) public class TestHiveConcurrentModificationGlueMetastore extends AbstractTestQueryFramework { diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestShowStats.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestShowStats.java index 844cc1aef3f44..4ed0fd12490b0 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestShowStats.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestShowStats.java @@ -19,12 +19,15 @@ import io.trino.testing.QueryRunner; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; import static io.trino.SystemSessionProperties.PREFER_PARTIAL_AGGREGATION; import static io.trino.SystemSessionProperties.USE_PARTIAL_DISTINCT_LIMIT; import static io.trino.SystemSessionProperties.USE_PARTIAL_TOPN; import static io.trino.tpch.TpchTable.NATION; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +@TestInstance(PER_CLASS) public class TestShowStats extends AbstractTestQueryFramework { diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastoreWithQueryRunner.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastoreWithQueryRunner.java index f75c6cc7edded..93b184573fd70 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastoreWithQueryRunner.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastoreWithQueryRunner.java @@ -28,6 +28,7 @@ import io.trino.testing.DistributedQueryRunner; import io.trino.testing.QueryRunner; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; import org.junit.jupiter.api.parallel.Execution; import java.util.List; @@ -39,8 +40,10 @@ import static io.trino.testing.TestingSession.testSessionBuilder; import static java.util.Collections.nCopies; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; +@TestInstance(PER_CLASS) @Execution(SAME_THREAD) public class TestCachingHiveMetastoreWithQueryRunner extends AbstractTestQueryFramework diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/TestS3FileSystemAccessOperations.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/TestS3FileSystemAccessOperations.java index 8a7e8b6e2361e..4e41f64437152 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/TestS3FileSystemAccessOperations.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/s3/TestS3FileSystemAccessOperations.java @@ -28,6 +28,7 @@ import org.intellij.lang.annotations.Language; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; import org.junit.jupiter.api.parallel.Execution; import org.junit.jupiter.api.parallel.ExecutionMode; @@ -44,7 +45,9 @@ import static io.trino.testing.containers.Minio.MINIO_REGION; import static io.trino.testing.containers.Minio.MINIO_SECRET_KEY; import static java.util.stream.Collectors.toCollection; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +@TestInstance(PER_CLASS) @Execution(ExecutionMode.SAME_THREAD) // S3 request counters shares mutable state so can't be run from many threads simultaneously public class TestS3FileSystemAccessOperations extends AbstractTestQueryFramework diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergGetTableStatisticsOperations.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergGetTableStatisticsOperations.java index 7762e6cbb9bfd..36bdc4ba43ae7 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergGetTableStatisticsOperations.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergGetTableStatisticsOperations.java @@ -29,6 +29,7 @@ import org.intellij.lang.annotations.Language; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; import org.junit.jupiter.api.extension.RegisterExtension; import org.junit.jupiter.api.parallel.Execution; @@ -45,10 +46,12 @@ import static io.trino.sql.planner.LogicalPlanner.Stage.OPTIMIZED_AND_VALIDATED; import static io.trino.testing.TestingSession.testSessionBuilder; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; // Cost-based optimizers' behaviors are affected by the statistics returned by the Connectors. Here is to count the getTableStatistics calls // when CBOs work with Iceberg Connector. +@TestInstance(PER_CLASS) @Execution(SAME_THREAD) public class TestIcebergGetTableStatisticsOperations extends AbstractTestQueryFramework diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetadataListing.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetadataListing.java index cf2b0e92a40ab..98e5aa57a4a23 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetadataListing.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetadataListing.java @@ -25,6 +25,7 @@ import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; import java.io.File; import java.util.Optional; @@ -33,7 +34,9 @@ import static io.trino.spi.security.SelectedRole.Type.ROLE; import static io.trino.testing.TestingSession.testSessionBuilder; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +@TestInstance(PER_CLASS) public class TestIcebergMetadataListing extends AbstractTestQueryFramework { diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergReadVersionedTable.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergReadVersionedTable.java index 056470675c346..92374ea8742aa 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergReadVersionedTable.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergReadVersionedTable.java @@ -17,6 +17,7 @@ import io.trino.testing.DistributedQueryRunner; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; import java.time.Instant; import java.time.ZonedDateTime; @@ -25,7 +26,9 @@ import static io.trino.plugin.iceberg.IcebergQueryRunner.createIcebergQueryRunner; import static java.lang.String.format; import static java.time.ZoneOffset.UTC; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +@TestInstance(PER_CLASS) public class TestIcebergReadVersionedTable extends AbstractTestQueryFramework { diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergReadVersionedTableByTemporal.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergReadVersionedTableByTemporal.java index 68b1748b441d4..f03fb453f92a3 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergReadVersionedTableByTemporal.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergReadVersionedTableByTemporal.java @@ -21,6 +21,7 @@ import io.trino.testing.containers.Minio; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; import static io.trino.plugin.iceberg.IcebergQueryRunner.ICEBERG_CATALOG; import static io.trino.testing.TestingNames.randomNameSuffix; @@ -28,7 +29,9 @@ import static io.trino.testing.containers.Minio.MINIO_SECRET_KEY; import static java.lang.String.format; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +@TestInstance(PER_CLASS) public class TestIcebergReadVersionedTableByTemporal extends AbstractTestQueryFramework { diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergSplitSource.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergSplitSource.java index 8cbc01068fc9e..98504649f63ce 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergSplitSource.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergSplitSource.java @@ -53,6 +53,7 @@ import org.apache.iceberg.types.Types; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; import org.junit.jupiter.api.Timeout; import java.io.File; @@ -76,7 +77,9 @@ import static io.trino.tpch.TpchTable.NATION; import static java.util.concurrent.TimeUnit.SECONDS; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +@TestInstance(PER_CLASS) public class TestIcebergSplitSource extends AbstractTestQueryFramework { diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergTableWithExternalLocation.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergTableWithExternalLocation.java index bf5f549480cab..1c297f3edac57 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergTableWithExternalLocation.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergTableWithExternalLocation.java @@ -23,6 +23,7 @@ import io.trino.testing.MaterializedResult; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; import java.io.File; import java.io.IOException; @@ -36,7 +37,9 @@ import static io.trino.testing.TestingNames.randomNameSuffix; import static java.lang.String.format; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +@TestInstance(PER_CLASS) public class TestIcebergTableWithExternalLocation extends AbstractTestQueryFramework { diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergV2.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergV2.java index 7fb65254de97d..a7e9b8f42650a 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergV2.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergV2.java @@ -64,6 +64,7 @@ import org.apache.iceberg.types.Types; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; import java.io.Closeable; import java.nio.ByteBuffer; @@ -98,7 +99,9 @@ import static org.apache.iceberg.TableProperties.SPLIT_SIZE; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +@TestInstance(PER_CLASS) public class TestIcebergV2 extends AbstractTestQueryFramework { diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogSkipArchive.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogSkipArchive.java index 05c385273f750..178b5f7ae50e8 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogSkipArchive.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogSkipArchive.java @@ -35,6 +35,7 @@ import org.apache.iceberg.io.FileIO; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; import java.io.File; import java.nio.file.Files; @@ -55,12 +56,14 @@ import static io.trino.type.InternalTypeManager.TESTING_TYPE_MANAGER; import static org.apache.iceberg.BaseMetastoreTableOperations.METADATA_LOCATION_PROP; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; /* * The test currently uses AWS Default Credential Provider Chain, * See https://docs.aws.amazon.com/sdk-for-java/v1/developer-guide/credentials.html#credentials-default * on ways to set your AWS credentials which will be needed to run this test. */ +@TestInstance(PER_CLASS) public class TestIcebergGlueCatalogSkipArchive extends AbstractTestQueryFramework { diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueTableOperationsInsertFailure.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueTableOperationsInsertFailure.java index db5bc565e2e14..10a3447da4354 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueTableOperationsInsertFailure.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueTableOperationsInsertFailure.java @@ -28,6 +28,7 @@ import org.apache.iceberg.exceptions.CommitStateUnknownException; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; import java.lang.reflect.InvocationTargetException; import java.nio.file.Files; @@ -41,12 +42,14 @@ import static io.trino.testing.TestingSession.testSessionBuilder; import static java.lang.String.format; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; /* * The test currently uses AWS Default Credential Provider Chain, * See https://docs.aws.amazon.com/sdk-for-java/v1/developer-guide/credentials.html#credentials-default * on ways to set your AWS credentials which will be needed to run this test. */ +@TestInstance(PER_CLASS) public class TestIcebergGlueTableOperationsInsertFailure extends AbstractTestQueryFramework { diff --git a/plugin/trino-kudu/src/test/java/io/trino/plugin/kudu/TestKuduIntegrationDecimalColumns.java b/plugin/trino-kudu/src/test/java/io/trino/plugin/kudu/TestKuduIntegrationDecimalColumns.java index b326915c61ef2..c2420d76be866 100644 --- a/plugin/trino-kudu/src/test/java/io/trino/plugin/kudu/TestKuduIntegrationDecimalColumns.java +++ b/plugin/trino-kudu/src/test/java/io/trino/plugin/kudu/TestKuduIntegrationDecimalColumns.java @@ -20,12 +20,15 @@ import io.trino.testing.sql.TrinoSqlExecutor; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; import static io.trino.plugin.kudu.KuduQueryRunnerFactory.createKuduQueryRunner; import static java.lang.String.format; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.offset; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +@TestInstance(PER_CLASS) public class TestKuduIntegrationDecimalColumns extends AbstractTestQueryFramework { diff --git a/plugin/trino-mariadb/src/test/java/io/trino/plugin/mariadb/TestMariaDbTypeMapping.java b/plugin/trino-mariadb/src/test/java/io/trino/plugin/mariadb/TestMariaDbTypeMapping.java index 853b126a12132..ad153999d3c93 100644 --- a/plugin/trino-mariadb/src/test/java/io/trino/plugin/mariadb/TestMariaDbTypeMapping.java +++ b/plugin/trino-mariadb/src/test/java/io/trino/plugin/mariadb/TestMariaDbTypeMapping.java @@ -30,6 +30,8 @@ import io.trino.testing.sql.TrinoSqlExecutor; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.parallel.Execution; import java.math.RoundingMode; import java.time.LocalDate; @@ -64,10 +66,14 @@ import static java.math.RoundingMode.UNNECESSARY; import static java.time.ZoneOffset.UTC; import static java.util.Arrays.asList; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; /** * @see MariaDB data types */ +@TestInstance(PER_CLASS) +@Execution(CONCURRENT) public class TestMariaDbTypeMapping extends AbstractTestQueryFramework { diff --git a/plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/TestMongoCaseInsensitiveMapping.java b/plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/TestMongoCaseInsensitiveMapping.java index 1bb94415198bc..d12cda7b4e31b 100644 --- a/plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/TestMongoCaseInsensitiveMapping.java +++ b/plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/TestMongoCaseInsensitiveMapping.java @@ -22,13 +22,19 @@ import org.bson.Document; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.parallel.Execution; import static io.trino.plugin.mongodb.MongoQueryRunner.createMongoClient; import static io.trino.plugin.mongodb.MongoQueryRunner.createMongoQueryRunner; import static io.trino.testing.TestingNames.randomNameSuffix; import static java.util.Locale.ENGLISH; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; +@TestInstance(PER_CLASS) +@Execution(CONCURRENT) public class TestMongoCaseInsensitiveMapping extends AbstractTestQueryFramework { diff --git a/plugin/trino-mysql/src/test/java/io/trino/plugin/mysql/TestMySqlTypeMapping.java b/plugin/trino-mysql/src/test/java/io/trino/plugin/mysql/TestMySqlTypeMapping.java index ff565235e4b4c..2f99cd90d93b4 100644 --- a/plugin/trino-mysql/src/test/java/io/trino/plugin/mysql/TestMySqlTypeMapping.java +++ b/plugin/trino-mysql/src/test/java/io/trino/plugin/mysql/TestMySqlTypeMapping.java @@ -32,6 +32,8 @@ import org.intellij.lang.annotations.Language; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.parallel.Execution; import java.math.RoundingMode; import java.sql.Connection; @@ -75,7 +77,11 @@ import static java.util.Arrays.asList; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; +@TestInstance(PER_CLASS) +@Execution(CONCURRENT) public class TestMySqlTypeMapping extends AbstractTestQueryFramework { diff --git a/plugin/trino-phoenix5/src/test/java/io/trino/plugin/phoenix5/TestPhoenixTypeMapping.java b/plugin/trino-phoenix5/src/test/java/io/trino/plugin/phoenix5/TestPhoenixTypeMapping.java index 9bd734daff9be..672b4e75fea41 100644 --- a/plugin/trino-phoenix5/src/test/java/io/trino/plugin/phoenix5/TestPhoenixTypeMapping.java +++ b/plugin/trino-phoenix5/src/test/java/io/trino/plugin/phoenix5/TestPhoenixTypeMapping.java @@ -31,6 +31,8 @@ import org.intellij.lang.annotations.Language; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.parallel.Execution; import java.math.RoundingMode; import java.time.LocalDate; @@ -68,10 +70,14 @@ import static java.util.Arrays.asList; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; /** * @see Phoenix data types */ +@TestInstance(PER_CLASS) +@Execution(CONCURRENT) public class TestPhoenixTypeMapping extends AbstractTestQueryFramework { diff --git a/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestPostgreSqlTypeMapping.java b/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestPostgreSqlTypeMapping.java index c467790e36492..49db947ecb560 100644 --- a/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestPostgreSqlTypeMapping.java +++ b/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestPostgreSqlTypeMapping.java @@ -40,6 +40,8 @@ import org.intellij.lang.annotations.Language; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.parallel.Execution; import java.math.BigDecimal; import java.math.RoundingMode; @@ -105,7 +107,11 @@ import static java.util.stream.Collectors.toList; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; +@TestInstance(PER_CLASS) +@Execution(CONCURRENT) public class TestPostgreSqlTypeMapping extends AbstractTestQueryFramework { diff --git a/plugin/trino-singlestore/src/test/java/io/trino/plugin/singlestore/TestSingleStoreTypeMapping.java b/plugin/trino-singlestore/src/test/java/io/trino/plugin/singlestore/TestSingleStoreTypeMapping.java index d924cdd7ce3e5..9c89484014d83 100644 --- a/plugin/trino-singlestore/src/test/java/io/trino/plugin/singlestore/TestSingleStoreTypeMapping.java +++ b/plugin/trino-singlestore/src/test/java/io/trino/plugin/singlestore/TestSingleStoreTypeMapping.java @@ -31,6 +31,8 @@ import io.trino.testing.sql.TrinoSqlExecutor; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.parallel.Execution; import java.math.RoundingMode; import java.time.LocalDate; @@ -72,10 +74,14 @@ import static java.time.ZoneOffset.UTC; import static java.util.Arrays.asList; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; /** * @see SingleStore (MemSQL) data types */ +@TestInstance(PER_CLASS) +@Execution(CONCURRENT) public class TestSingleStoreTypeMapping extends AbstractTestQueryFramework { diff --git a/testing/trino-tests/src/test/java/io/trino/connector/informationschema/TestInformationSchemaConnector.java b/testing/trino-tests/src/test/java/io/trino/connector/informationschema/TestInformationSchemaConnector.java index 209a4e9ec639f..88236fbdf4786 100644 --- a/testing/trino-tests/src/test/java/io/trino/connector/informationschema/TestInformationSchemaConnector.java +++ b/testing/trino-tests/src/test/java/io/trino/connector/informationschema/TestInformationSchemaConnector.java @@ -26,6 +26,7 @@ import org.intellij.lang.annotations.Language; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.parallel.Execution; @@ -35,8 +36,10 @@ import static io.trino.testing.MultisetAssertions.assertMultisetsEqual; import static io.trino.testing.TestingSession.testSessionBuilder; import static java.util.stream.Collectors.joining; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; +@TestInstance(PER_CLASS) @Execution(SAME_THREAD) // countingMockConnector is shared mutable state public class TestInformationSchemaConnector extends AbstractTestQueryFramework diff --git a/testing/trino-tests/src/test/java/io/trino/connector/system/runtime/TestKillQuery.java b/testing/trino-tests/src/test/java/io/trino/connector/system/runtime/TestKillQuery.java index e3ff85aeab1a5..dae215d396f83 100644 --- a/testing/trino-tests/src/test/java/io/trino/connector/system/runtime/TestKillQuery.java +++ b/testing/trino-tests/src/test/java/io/trino/connector/system/runtime/TestKillQuery.java @@ -23,6 +23,7 @@ import io.trino.testng.services.ReportOrphanedExecutors; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.parallel.Execution; @@ -44,8 +45,10 @@ import static java.util.UUID.randomUUID; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; +@TestInstance(PER_CLASS) @Execution(SAME_THREAD) // e.g. some tests methods modify AC configuration public class TestKillQuery extends AbstractTestQueryFramework diff --git a/testing/trino-tests/src/test/java/io/trino/execution/TestEventListenerBasic.java b/testing/trino-tests/src/test/java/io/trino/execution/TestEventListenerBasic.java index 57cf979dcc8a5..a583cfc40ec05 100644 --- a/testing/trino-tests/src/test/java/io/trino/execution/TestEventListenerBasic.java +++ b/testing/trino-tests/src/test/java/io/trino/execution/TestEventListenerBasic.java @@ -53,6 +53,7 @@ import io.trino.testing.QueryRunner; import org.intellij.lang.annotations.Language; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.parallel.Execution; @@ -88,8 +89,10 @@ import static java.lang.String.format; import static java.util.UUID.randomUUID; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; +@TestInstance(PER_CLASS) @Execution(SAME_THREAD) public class TestEventListenerBasic extends AbstractTestQueryFramework diff --git a/testing/trino-tests/src/test/java/io/trino/execution/TestQueryTracker.java b/testing/trino-tests/src/test/java/io/trino/execution/TestQueryTracker.java index 7d02faaa31a1f..e65d5ed38bbc4 100644 --- a/testing/trino-tests/src/test/java/io/trino/execution/TestQueryTracker.java +++ b/testing/trino-tests/src/test/java/io/trino/execution/TestQueryTracker.java @@ -24,6 +24,7 @@ import io.trino.testing.QueryRunner; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.parallel.Execution; @@ -33,10 +34,12 @@ import static io.trino.spi.type.VarcharType.VARCHAR; import static io.trino.testing.TestingSession.testSessionBuilder; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; // Tests need to finish before strict timeouts. Any background work // may make them flaky +@TestInstance(PER_CLASS) @Execution(SAME_THREAD) // CountDownLatches are shared mutable state public class TestQueryTracker extends AbstractTestQueryFramework diff --git a/testing/trino-tests/src/test/java/io/trino/execution/TestRefreshMaterializedView.java b/testing/trino-tests/src/test/java/io/trino/execution/TestRefreshMaterializedView.java index d1dce7c43f18f..316f87b8b5134 100644 --- a/testing/trino-tests/src/test/java/io/trino/execution/TestRefreshMaterializedView.java +++ b/testing/trino-tests/src/test/java/io/trino/execution/TestRefreshMaterializedView.java @@ -35,6 +35,7 @@ import org.intellij.lang.annotations.Language; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.parallel.Execution; @@ -53,8 +54,10 @@ import static java.util.concurrent.Executors.newCachedThreadPool; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; +@TestInstance(PER_CLASS) @Execution(SAME_THREAD) public class TestRefreshMaterializedView extends AbstractTestQueryFramework diff --git a/testing/trino-tests/src/test/java/io/trino/tests/TestGetTableStatisticsOperations.java b/testing/trino-tests/src/test/java/io/trino/tests/TestGetTableStatisticsOperations.java index a1f4817fa9e44..73591178989e8 100644 --- a/testing/trino-tests/src/test/java/io/trino/tests/TestGetTableStatisticsOperations.java +++ b/testing/trino-tests/src/test/java/io/trino/tests/TestGetTableStatisticsOperations.java @@ -25,6 +25,7 @@ import org.intellij.lang.annotations.Language; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; import org.junit.jupiter.api.extension.RegisterExtension; import org.junit.jupiter.api.parallel.Execution; @@ -33,8 +34,10 @@ import static io.trino.testing.TestingSession.testSessionBuilder; import static io.trino.testing.TransactionBuilder.transaction; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; +@TestInstance(PER_CLASS) @Execution(SAME_THREAD) public class TestGetTableStatisticsOperations extends AbstractTestQueryFramework diff --git a/testing/trino-tests/src/test/java/io/trino/tests/TestProcedureCall.java b/testing/trino-tests/src/test/java/io/trino/tests/TestProcedureCall.java index 0bd33ad0c04f0..a95ad64b1e594 100644 --- a/testing/trino-tests/src/test/java/io/trino/tests/TestProcedureCall.java +++ b/testing/trino-tests/src/test/java/io/trino/tests/TestProcedureCall.java @@ -26,6 +26,7 @@ import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; import org.junit.jupiter.api.parallel.Execution; import java.util.List; @@ -35,8 +36,10 @@ import static java.util.Arrays.asList; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; +@TestInstance(PER_CLASS) @Execution(SAME_THREAD) // ProcedureTester is shared mutable state public class TestProcedureCall extends AbstractTestQueryFramework From 0579c95b02204934d36d2d45c4bd9e1723de2ea0 Mon Sep 17 00:00:00 2001 From: Martin Traverso Date: Fri, 22 Dec 2023 11:49:28 -0800 Subject: [PATCH 205/350] Adjust logging level to debug They result in very verbose logs in tests and are not actionable for anyone managing a Trino deployment --- .../java/io/trino/server/remotetask/TaskInfoFetcher.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/trino-main/src/main/java/io/trino/server/remotetask/TaskInfoFetcher.java b/core/trino-main/src/main/java/io/trino/server/remotetask/TaskInfoFetcher.java index 548ab94337980..6453b7e10f0c5 100644 --- a/core/trino-main/src/main/java/io/trino/server/remotetask/TaskInfoFetcher.java +++ b/core/trino-main/src/main/java/io/trino/server/remotetask/TaskInfoFetcher.java @@ -251,7 +251,7 @@ synchronized void updateTaskInfo(TaskInfo newTaskInfo) TaskStatus newRemoteTaskStatus = newTaskInfo.getTaskStatus(); if (!newRemoteTaskStatus.getTaskId().equals(taskId)) { - log.warn("Task ID mismatch on remote task status. Member task ID is %s, but remote task ID is %s. This will confuse finalTaskInfo listeners.", + log.debug("Task ID mismatch on remote task status. Member task ID is %s, but remote task ID is %s. This will confuse finalTaskInfo listeners.", taskId.toString(), newRemoteTaskStatus.getTaskId().toString()); } @@ -259,7 +259,7 @@ synchronized void updateTaskInfo(TaskInfo newTaskInfo) // prefer local newTaskInfo = newTaskInfo.withTaskStatus(localTaskStatus); if (!localTaskStatus.getTaskId().equals(taskId)) { - log.warn("Task ID mismatch on local task status. Member task ID is %s, but status-fetcher ID is %s. This will confuse finalTaskInfo listeners.", + log.debug("Task ID mismatch on local task status. Member task ID is %s, but status-fetcher ID is %s. This will confuse finalTaskInfo listeners.", taskId.toString(), newRemoteTaskStatus.getTaskId().toString()); } } @@ -271,7 +271,7 @@ synchronized void updateTaskInfo(TaskInfo newTaskInfo) if (newTaskInfo.getTaskStatus().getState().isDone()) { boolean wasSet = spoolingOutputStats.compareAndSet(null, newTaskInfo.getOutputBuffers().getSpoolingOutputStats().orElse(null)); if (wasSet && spoolingOutputStats.get() == null) { - log.warn("Task %s was updated to null spoolingOutputStats. Future calls to retrieveAndDropSpoolingOutputStats will fail.", taskId.toString()); + log.debug("Task %s was updated to null spoolingOutputStats. Future calls to retrieveAndDropSpoolingOutputStats will fail.", taskId.toString()); } newTaskInfo = newTaskInfo.pruneSpoolingOutputStats(); } From 7b660f1578b1b92d732b65e2f9a12012cd794248 Mon Sep 17 00:00:00 2001 From: Martin Traverso Date: Fri, 22 Dec 2023 11:53:46 -0800 Subject: [PATCH 206/350] Fix formatting Arguments should go on the same line or each on a separate line. --- .../java/io/trino/server/remotetask/TaskInfoFetcher.java | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/core/trino-main/src/main/java/io/trino/server/remotetask/TaskInfoFetcher.java b/core/trino-main/src/main/java/io/trino/server/remotetask/TaskInfoFetcher.java index 6453b7e10f0c5..4bca142ec7bc5 100644 --- a/core/trino-main/src/main/java/io/trino/server/remotetask/TaskInfoFetcher.java +++ b/core/trino-main/src/main/java/io/trino/server/remotetask/TaskInfoFetcher.java @@ -251,16 +251,14 @@ synchronized void updateTaskInfo(TaskInfo newTaskInfo) TaskStatus newRemoteTaskStatus = newTaskInfo.getTaskStatus(); if (!newRemoteTaskStatus.getTaskId().equals(taskId)) { - log.debug("Task ID mismatch on remote task status. Member task ID is %s, but remote task ID is %s. This will confuse finalTaskInfo listeners.", - taskId.toString(), newRemoteTaskStatus.getTaskId().toString()); + log.debug("Task ID mismatch on remote task status. Member task ID is %s, but remote task ID is %s. This will confuse finalTaskInfo listeners.", taskId.toString(), newRemoteTaskStatus.getTaskId().toString()); } if (localTaskStatus.getState().isDone() && newRemoteTaskStatus.getState().isDone() && localTaskStatus.getState() != newRemoteTaskStatus.getState()) { // prefer local newTaskInfo = newTaskInfo.withTaskStatus(localTaskStatus); if (!localTaskStatus.getTaskId().equals(taskId)) { - log.debug("Task ID mismatch on local task status. Member task ID is %s, but status-fetcher ID is %s. This will confuse finalTaskInfo listeners.", - taskId.toString(), newRemoteTaskStatus.getTaskId().toString()); + log.debug("Task ID mismatch on local task status. Member task ID is %s, but status-fetcher ID is %s. This will confuse finalTaskInfo listeners.", taskId.toString(), newRemoteTaskStatus.getTaskId().toString()); } } @@ -271,7 +269,7 @@ synchronized void updateTaskInfo(TaskInfo newTaskInfo) if (newTaskInfo.getTaskStatus().getState().isDone()) { boolean wasSet = spoolingOutputStats.compareAndSet(null, newTaskInfo.getOutputBuffers().getSpoolingOutputStats().orElse(null)); if (wasSet && spoolingOutputStats.get() == null) { - log.debug("Task %s was updated to null spoolingOutputStats. Future calls to retrieveAndDropSpoolingOutputStats will fail.", taskId.toString()); + log.debug("Task %s was updated to null spoolingOutputStats. Future calls to retrieveAndDropSpoolingOutputStats will fail.", taskId.toString()); } newTaskInfo = newTaskInfo.pruneSpoolingOutputStats(); } From b2e98e9d4f1bf9ae6d3f81909e6cc38c98b65e3f Mon Sep 17 00:00:00 2001 From: Martin Traverso Date: Fri, 22 Dec 2023 11:54:54 -0800 Subject: [PATCH 207/350] Remove unnecessary calls to toString() --- .../java/io/trino/server/remotetask/TaskInfoFetcher.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/trino-main/src/main/java/io/trino/server/remotetask/TaskInfoFetcher.java b/core/trino-main/src/main/java/io/trino/server/remotetask/TaskInfoFetcher.java index 4bca142ec7bc5..73b14cade4ba9 100644 --- a/core/trino-main/src/main/java/io/trino/server/remotetask/TaskInfoFetcher.java +++ b/core/trino-main/src/main/java/io/trino/server/remotetask/TaskInfoFetcher.java @@ -251,14 +251,14 @@ synchronized void updateTaskInfo(TaskInfo newTaskInfo) TaskStatus newRemoteTaskStatus = newTaskInfo.getTaskStatus(); if (!newRemoteTaskStatus.getTaskId().equals(taskId)) { - log.debug("Task ID mismatch on remote task status. Member task ID is %s, but remote task ID is %s. This will confuse finalTaskInfo listeners.", taskId.toString(), newRemoteTaskStatus.getTaskId().toString()); + log.debug("Task ID mismatch on remote task status. Member task ID is %s, but remote task ID is %s. This will confuse finalTaskInfo listeners.", taskId, newRemoteTaskStatus.getTaskId()); } if (localTaskStatus.getState().isDone() && newRemoteTaskStatus.getState().isDone() && localTaskStatus.getState() != newRemoteTaskStatus.getState()) { // prefer local newTaskInfo = newTaskInfo.withTaskStatus(localTaskStatus); if (!localTaskStatus.getTaskId().equals(taskId)) { - log.debug("Task ID mismatch on local task status. Member task ID is %s, but status-fetcher ID is %s. This will confuse finalTaskInfo listeners.", taskId.toString(), newRemoteTaskStatus.getTaskId().toString()); + log.debug("Task ID mismatch on local task status. Member task ID is %s, but status-fetcher ID is %s. This will confuse finalTaskInfo listeners.", taskId, newRemoteTaskStatus.getTaskId()); } } @@ -269,7 +269,7 @@ synchronized void updateTaskInfo(TaskInfo newTaskInfo) if (newTaskInfo.getTaskStatus().getState().isDone()) { boolean wasSet = spoolingOutputStats.compareAndSet(null, newTaskInfo.getOutputBuffers().getSpoolingOutputStats().orElse(null)); if (wasSet && spoolingOutputStats.get() == null) { - log.debug("Task %s was updated to null spoolingOutputStats. Future calls to retrieveAndDropSpoolingOutputStats will fail.", taskId.toString()); + log.debug("Task %s was updated to null spoolingOutputStats. Future calls to retrieveAndDropSpoolingOutputStats will fail.", taskId); } newTaskInfo = newTaskInfo.pruneSpoolingOutputStats(); } From 355f5c866e20926cb5530479d2bf95c54e8a0776 Mon Sep 17 00:00:00 2001 From: Karol Sobczak Date: Fri, 10 Mar 2023 13:22:54 +0100 Subject: [PATCH 208/350] Remove unnecessary simplification step This step is not needed anymore as corresponding tests do not fail. --- .../planner/optimizations/PredicatePushDown.java | 16 +++++----------- 1 file changed, 5 insertions(+), 11 deletions(-) diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/PredicatePushDown.java b/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/PredicatePushDown.java index e4bb453baf727..2321e456fa1e1 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/PredicatePushDown.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/PredicatePushDown.java @@ -1008,16 +1008,10 @@ private InnerJoinPushDownResult processInnerJoin( ImmutableSet leftScope = ImmutableSet.copyOf(leftSymbols); ImmutableSet rightScope = ImmutableSet.copyOf(rightSymbols); - // Attempt to simplify the effective left/right predicates with the predicate we're pushing down - // This, effectively, inlines any constants derived from such predicate - EqualityInference predicateInference = new EqualityInference(metadata, inheritedPredicate); - Expression simplifiedLeftEffectivePredicate = predicateInference.rewrite(leftEffectivePredicate, leftScope); - Expression simplifiedRightEffectivePredicate = predicateInference.rewrite(rightEffectivePredicate, rightScope); - // Generate equality inferences - EqualityInference allInference = new EqualityInference(metadata, inheritedPredicate, leftEffectivePredicate, rightEffectivePredicate, joinPredicate, simplifiedLeftEffectivePredicate, simplifiedRightEffectivePredicate); - EqualityInference allInferenceWithoutLeftInferred = new EqualityInference(metadata, inheritedPredicate, rightEffectivePredicate, joinPredicate, simplifiedRightEffectivePredicate); - EqualityInference allInferenceWithoutRightInferred = new EqualityInference(metadata, inheritedPredicate, leftEffectivePredicate, joinPredicate, simplifiedLeftEffectivePredicate); + EqualityInference allInference = new EqualityInference(metadata, inheritedPredicate, leftEffectivePredicate, rightEffectivePredicate, joinPredicate); + EqualityInference allInferenceWithoutLeftInferred = new EqualityInference(metadata, inheritedPredicate, rightEffectivePredicate, joinPredicate); + EqualityInference allInferenceWithoutRightInferred = new EqualityInference(metadata, inheritedPredicate, leftEffectivePredicate, joinPredicate); // Add equalities from the inference back in leftPushDownConjuncts.addAll(allInferenceWithoutLeftInferred.generateEqualitiesPartitionedBy(leftScope).getScopeEqualities()); @@ -1043,13 +1037,13 @@ private InnerJoinPushDownResult processInnerJoin( }); // See if we can push the right effective predicate to the left side - EqualityInference.nonInferrableConjuncts(metadata, simplifiedRightEffectivePredicate) + EqualityInference.nonInferrableConjuncts(metadata, rightEffectivePredicate) .map(conjunct -> allInference.rewrite(conjunct, leftScope)) .filter(Objects::nonNull) .forEach(leftPushDownConjuncts::add); // See if we can push the left effective predicate to the right side - EqualityInference.nonInferrableConjuncts(metadata, simplifiedLeftEffectivePredicate) + EqualityInference.nonInferrableConjuncts(metadata, leftEffectivePredicate) .map(conjunct -> allInference.rewrite(conjunct, rightScope)) .filter(Objects::nonNull) .forEach(rightPushDownConjuncts::add); From 17098af4d5a91af56818827716fd7220994098b5 Mon Sep 17 00:00:00 2001 From: Karol Sobczak Date: Fri, 10 Mar 2023 13:05:13 +0100 Subject: [PATCH 209/350] Do not generate redundant straddling predicates by equality inference When there are predicates like a1 = b1 and a2 = a1 + 1, then equality inference would derive staddling predicate for a2 = b1 + 1, which is redundant to a1 = b1, a2 = a1 + 1. This commit makes sure that redundant straddling predicates are not generated. --- .../trino/sql/planner/EqualityInference.java | 40 ++++++++---- .../AbstractPredicatePushdownTest.java | 62 +++++++++++++++++++ .../sql/planner/TestEqualityInference.java | 23 +++++++ ...PredicatePushdownWithoutDynamicFilter.java | 29 +++++++++ 4 files changed, 142 insertions(+), 12 deletions(-) diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/EqualityInference.java b/core/trino-main/src/main/java/io/trino/sql/planner/EqualityInference.java index a9cfc0289d35e..ecbdcf0bd599c 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/EqualityInference.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/EqualityInference.java @@ -34,11 +34,10 @@ import java.util.LinkedHashSet; import java.util.List; import java.util.Map; -import java.util.Objects; +import java.util.Optional; import java.util.Set; import java.util.function.Predicate; import java.util.function.ToIntFunction; -import java.util.stream.Collectors; import java.util.stream.Stream; import static com.google.common.collect.ImmutableList.toImmutableList; @@ -215,17 +214,34 @@ public EqualityPartition generateEqualitiesPartitionedBy(Set scope) .forEach(scopeComplementEqualities::add); } - // Compile the scope straddling equality expressions - List connectingExpressions = new ArrayList<>(); - connectingExpressions.add(matchingCanonical); - connectingExpressions.add(complementCanonical); - connectingExpressions.addAll(scopeStraddlingExpressions); - connectingExpressions = connectingExpressions.stream() - .filter(Objects::nonNull) - .collect(Collectors.toList()); - Expression connectingCanonical = getCanonical(connectingExpressions.stream()); + // Compile single equality between matching and complement scope. + // Only consider expressions that don't have derived expression in other scope. + // Otherwise, redundant equality would be generated. + Optional matchingConnecting = scopeExpressions.stream() + .filter(expression -> SymbolsExtractor.extractAll(expression).isEmpty() || rewrite(expression, symbol -> !scope.contains(symbol), false) == null) + .min(canonicalComparator); + Optional complementConnecting = scopeComplementExpressions.stream() + .filter(expression -> SymbolsExtractor.extractAll(expression).isEmpty() || rewrite(expression, scope::contains, false) == null) + .min(canonicalComparator); + if (matchingConnecting.isPresent() && complementConnecting.isPresent() && !matchingConnecting.equals(complementConnecting)) { + scopeStraddlingEqualities.add(new ComparisonExpression(ComparisonExpression.Operator.EQUAL, matchingConnecting.get(), complementConnecting.get())); + } + + // Compile the scope straddling equality expressions. + // scopeStraddlingExpressions couldn't be pushed to either side, + // therefore there needs to be an equality generated with + // one of the scopes (either matching or complement). + List straddlingExpressions = new ArrayList<>(); + if (matchingCanonical != null) { + straddlingExpressions.add(matchingCanonical); + } + else if (complementCanonical != null) { + straddlingExpressions.add(complementCanonical); + } + straddlingExpressions.addAll(scopeStraddlingExpressions); + Expression connectingCanonical = getCanonical(straddlingExpressions.stream()); if (connectingCanonical != null) { - connectingExpressions.stream() + straddlingExpressions.stream() .filter(expression -> !expression.equals(connectingCanonical)) .map(expression -> new ComparisonExpression(ComparisonExpression.Operator.EQUAL, connectingCanonical, expression)) .forEach(scopeStraddlingEqualities::add); diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/AbstractPredicatePushdownTest.java b/core/trino-main/src/test/java/io/trino/sql/planner/AbstractPredicatePushdownTest.java index a1803ba70ec26..5fb02b62ae8f7 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/AbstractPredicatePushdownTest.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/AbstractPredicatePushdownTest.java @@ -13,6 +13,7 @@ */ package io.trino.sql.planner; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import io.trino.Session; import io.trino.sql.planner.assertions.BasePlanTest; @@ -39,6 +40,7 @@ import static io.trino.sql.planner.assertions.PlanMatchPattern.semiJoin; import static io.trino.sql.planner.assertions.PlanMatchPattern.tableScan; import static io.trino.sql.planner.assertions.PlanMatchPattern.values; +import static io.trino.sql.planner.plan.JoinNode.Type.INNER; import static io.trino.sql.planner.plan.JoinNode.Type.LEFT; import static io.trino.sql.tree.BooleanLiteral.TRUE_LITERAL; @@ -465,6 +467,66 @@ WITH t(a) AS (VALUES 'a', 'b') output(values("field", "field_0"))); } + @Test + public void testSimplifyNonInferrableInheritedPredicate() + { + assertPlan("SELECT * FROM (SELECT * FROM nation WHERE nationkey = regionkey AND regionkey = 5) a, nation b WHERE a.nationkey = b.nationkey AND a.nationkey + 11 > 15", + output( + join(INNER, builder -> builder + .equiCriteria(ImmutableList.of()) + .left( + filter("((L_NATIONKEY = L_REGIONKEY) AND (L_REGIONKEY = BIGINT '5'))", + tableScan("nation", ImmutableMap.of("L_NATIONKEY", "nationkey", "L_REGIONKEY", "regionkey")))) + .right( + anyTree( + filter("R_NATIONKEY = BIGINT '5'", + tableScan("nation", ImmutableMap.of("R_NATIONKEY", "nationkey")))))))); + } + + @Test + public void testDoesNotCreatePredicateFromInferredPredicate() + { + assertPlan("SELECT * FROM (SELECT *, nationkey + 1 as nationkey2 FROM nation) a JOIN nation b ON a.nationkey = b.nationkey", + output( + join(INNER, builder -> builder + .equiCriteria("L_NATIONKEY", "R_NATIONKEY") + .left( + filter("true", // DF filter + tableScan("nation", ImmutableMap.of("L_NATIONKEY", "nationkey")))) + .right( + anyTree( + tableScan("nation", ImmutableMap.of("R_NATIONKEY", "nationkey"))))))); + + assertPlan("SELECT * FROM (SELECT * FROM nation WHERE nationkey = 5) a JOIN (SELECT * FROM nation WHERE nationkey = 5) b ON a.nationkey = b.nationkey", + output( + join(INNER, builder -> builder + .equiCriteria(ImmutableList.of()) + .left( + filter("L_NATIONKEY = BIGINT '5'", + tableScan("nation", ImmutableMap.of("L_NATIONKEY", "nationkey")))) + .right( + anyTree( + filter("R_NATIONKEY = BIGINT '5'", + tableScan("nation", ImmutableMap.of("R_NATIONKEY", "nationkey")))))))); + } + + @Test + public void testSimplifiesStraddlingPredicate() + { + assertPlan("SELECT * FROM (SELECT * FROM NATION WHERE nationkey = 5) a JOIN nation b ON a.nationkey = b.nationkey AND a.nationkey = a.regionkey + b.regionkey", + output( + filter("L_REGIONKEY + R_REGIONKEY = BIGINT '5'", + join(INNER, builder -> builder + .equiCriteria(ImmutableList.of()) + .left( + filter("L_NATIONKEY = BIGINT '5'", + tableScan("nation", ImmutableMap.of("L_NATIONKEY", "nationkey", "L_REGIONKEY", "regionkey")))) + .right( + anyTree( + filter("R_NATIONKEY = BIGINT '5'", + tableScan("nation", ImmutableMap.of("R_NATIONKEY", "nationkey", "R_REGIONKEY", "regionkey"))))))))); + } + protected Session noSemiJoinRewrite() { return Session.builder(getQueryRunner().getDefaultSession()) diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/TestEqualityInference.java b/core/trino-main/src/test/java/io/trino/sql/planner/TestEqualityInference.java index b3174c7a48a7b..2c03203834d54 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/TestEqualityInference.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/TestEqualityInference.java @@ -65,6 +65,29 @@ public class TestEqualityInference private final TestingFunctionResolution functionResolution = new TestingFunctionResolution(); private final Metadata metadata = functionResolution.getMetadata(); + @Test + public void testDoesNotInferRedundantStraddlingPredicates() + { + EqualityInference inference = new EqualityInference( + metadata, + equals("a1", "b1"), + equals(add(nameReference("a1"), number(1)), number(0)), + equals(nameReference("a2"), add(nameReference("a1"), number(2))), + equals(nameReference("a1"), add("a3", "b3")), + equals(nameReference("b2"), add("a4", "b4"))); + EqualityInference.EqualityPartition partition = inference.generateEqualitiesPartitionedBy(symbols("a1", "a2", "a3", "a4")); + assertThat(partition.getScopeEqualities()).containsExactly( + equals(number(0), add(nameReference("a1"), number(1))), + equals(nameReference("a2"), add(nameReference("a1"), number(2)))); + assertThat(partition.getScopeComplementEqualities()).containsExactly( + equals(number(0), add(nameReference("b1"), number(1)))); + // there shouldn't be equality a2 = b1 + 1 as it can be derived from a2 = a1 + 1, a1 = b1 + assertThat(partition.getScopeStraddlingEqualities()).containsExactly( + equals("a1", "b1"), + equals(nameReference("a1"), add("a3", "b3")), + equals(nameReference("b2"), add("a4", "b4"))); + } + @Test public void testTransitivity() { diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/TestPredicatePushdownWithoutDynamicFilter.java b/core/trino-main/src/test/java/io/trino/sql/planner/TestPredicatePushdownWithoutDynamicFilter.java index 1c4e627e47429..92fa340168ef6 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/TestPredicatePushdownWithoutDynamicFilter.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/TestPredicatePushdownWithoutDynamicFilter.java @@ -13,6 +13,7 @@ */ package io.trino.sql.planner; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import io.trino.Session; import io.trino.sql.planner.plan.ExchangeNode; @@ -23,6 +24,7 @@ import static io.trino.sql.planner.assertions.PlanMatchPattern.filter; import static io.trino.sql.planner.assertions.PlanMatchPattern.join; import static io.trino.sql.planner.assertions.PlanMatchPattern.node; +import static io.trino.sql.planner.assertions.PlanMatchPattern.output; import static io.trino.sql.planner.assertions.PlanMatchPattern.project; import static io.trino.sql.planner.assertions.PlanMatchPattern.semiJoin; import static io.trino.sql.planner.assertions.PlanMatchPattern.tableScan; @@ -173,4 +175,31 @@ public void testNonStraddlingJoinExpression() anyTree( tableScan("orders", ImmutableMap.of("ORDERS_OK", "orderkey"))))))); } + + @Override + @Test + public void testDoesNotCreatePredicateFromInferredPredicate() + { + assertPlan("SELECT * FROM (SELECT *, nationkey + 1 as nationkey2 FROM nation) a JOIN nation b ON a.nationkey = b.nationkey", + output( + join(INNER, builder -> builder + .equiCriteria("L_NATIONKEY", "R_NATIONKEY") + .left( + tableScan("nation", ImmutableMap.of("L_NATIONKEY", "nationkey"))) + .right( + anyTree( + tableScan("nation", ImmutableMap.of("R_NATIONKEY", "nationkey"))))))); + + assertPlan("SELECT * FROM (SELECT * FROM nation WHERE nationkey = 5) a JOIN (SELECT * FROM nation WHERE nationkey = 5) b ON a.nationkey = b.nationkey", + output( + join(INNER, builder -> builder + .equiCriteria(ImmutableList.of()) + .left( + filter("L_NATIONKEY = BIGINT '5'", + tableScan("nation", ImmutableMap.of("L_NATIONKEY", "nationkey")))) + .right( + anyTree( + filter("R_NATIONKEY = BIGINT '5'", + tableScan("nation", ImmutableMap.of("R_NATIONKEY", "nationkey")))))))); + } } From f0d59e64479b7c0c3d04f79931e8a78890419476 Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Sun, 24 Dec 2023 11:46:05 +0100 Subject: [PATCH 210/350] Update JLine to 3.25.0 --- client/trino-cli/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/client/trino-cli/pom.xml b/client/trino-cli/pom.xml index 03ca229a1e534..d64a685f4b199 100644 --- a/client/trino-cli/pom.xml +++ b/client/trino-cli/pom.xml @@ -15,7 +15,7 @@ ${project.parent.basedir} 8 io.trino.cli.Trino - 3.24.1 + 3.25.0 From 457df6e2403c85a0c81b6d57d905807f50b1ffc7 Mon Sep 17 00:00:00 2001 From: "Adam J. Shook" Date: Fri, 3 Nov 2023 17:34:42 +0000 Subject: [PATCH 211/350] Fix SQL Server DATETIMEOFFSET for old dates The value returned via the microsoft.sql.DateTimeOffset when converted to an OffsetDateTime is changed for old dates due to an issue in the JDBC driver. This changes retrieving datetimeoffset types from SQL Server to use getString instead of getObject and OffsetDateTime. Predicate pushdown is now disabled for this type due to test failures with IS NOT DISTINCT FROM predicates if the value is before the year 1583. --- .../plugin/sqlserver/SqlServerClient.java | 71 ++++++++- .../sqlserver/BaseSqlServerTypeMapping.java | 135 ++++++++++++++++++ 2 files changed, 200 insertions(+), 6 deletions(-) diff --git a/plugin/trino-sqlserver/src/main/java/io/trino/plugin/sqlserver/SqlServerClient.java b/plugin/trino-sqlserver/src/main/java/io/trino/plugin/sqlserver/SqlServerClient.java index a9dfb09b7d4a7..c1cdd10770273 100644 --- a/plugin/trino-sqlserver/src/main/java/io/trino/plugin/sqlserver/SqlServerClient.java +++ b/plugin/trino-sqlserver/src/main/java/io/trino/plugin/sqlserver/SqlServerClient.java @@ -76,6 +76,7 @@ import io.trino.spi.connector.TableNotFoundException; import io.trino.spi.expression.ConnectorExpression; import io.trino.spi.predicate.Domain; +import io.trino.spi.predicate.Range; import io.trino.spi.predicate.ValueSet; import io.trino.spi.statistics.ColumnStatistics; import io.trino.spi.statistics.Estimate; @@ -91,6 +92,7 @@ import io.trino.spi.type.Type; import io.trino.spi.type.VarbinaryType; import io.trino.spi.type.VarcharType; +import microsoft.sql.DateTimeOffset; import org.jdbi.v3.core.Handle; import org.jdbi.v3.core.Jdbi; @@ -102,6 +104,7 @@ import java.sql.ResultSetMetaData; import java.sql.SQLException; import java.sql.Statement; +import java.sql.Timestamp; import java.sql.Types; import java.time.Instant; import java.time.LocalDate; @@ -110,6 +113,7 @@ import java.time.ZoneId; import java.time.ZonedDateTime; import java.time.format.DateTimeFormatter; +import java.time.format.DateTimeFormatterBuilder; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -179,6 +183,7 @@ import static io.trino.spi.type.RealType.REAL; import static io.trino.spi.type.SmallintType.SMALLINT; import static io.trino.spi.type.TimeType.createTimeType; +import static io.trino.spi.type.TimeZoneKey.UTC_KEY; import static io.trino.spi.type.TimeZoneKey.getTimeZoneKey; import static io.trino.spi.type.TimestampType.MAX_SHORT_PRECISION; import static io.trino.spi.type.TimestampType.createTimestampType; @@ -199,6 +204,7 @@ import static java.lang.String.format; import static java.lang.String.join; import static java.math.RoundingMode.UNNECESSARY; +import static java.time.temporal.ChronoField.NANO_OF_SECOND; import static java.util.Objects.requireNonNull; import static java.util.stream.Collectors.joining; @@ -225,6 +231,13 @@ public class SqlServerClient private static final int MAX_SUPPORTED_TEMPORAL_PRECISION = 7; + private static final DateTimeFormatter DATE_TIME_OFFSET_FORMATTER = new DateTimeFormatterBuilder() + .appendPattern("yyyy-MM-dd HH:mm:ss") + .appendFraction(NANO_OF_SECOND, 0, MAX_SUPPORTED_TEMPORAL_PRECISION, true) + .appendPattern(" ") + .appendZoneId() + .toFormatter(); + private static final PredicatePushdownController SQLSERVER_CHARACTER_PUSHDOWN = (session, domain) -> { if (domain.isNullableSingleValue()) { return FULL_PUSHDOWN.apply(session, domain); @@ -239,12 +252,39 @@ public class SqlServerClient } // Domain#simplify can turn a discrete set into a range predicate // Push down of range predicate for varchar/char types could lead to incorrect results - // when the remote database is case insensitive + // when the remote database is case-insensitive return DISABLE_PUSHDOWN.apply(session, domain); } return FULL_PUSHDOWN.apply(session, simplifiedDomain); }; + // Dates prior to the Gregorian calendar switch in 1582 can cause incorrect results when pushed down, + // so we disable predicate push down when the domain contains values prior to 1583 + private static final Instant GREGORIAN_SWITCH_INSTANT = Instant.parse("1583-01-01T00:00:00Z"); + private static final DateTimeOffset GREGORIAN_SWITCH_DATETIMEOFFSET = DateTimeOffset.valueOf(new Timestamp(GREGORIAN_SWITCH_INSTANT.toEpochMilli()), 0); + private static final LongTimestampWithTimeZone LONG_DATETIMEOFFSET_DISABLE_VALUE = + LongTimestampWithTimeZone.fromEpochSecondsAndFraction( + GREGORIAN_SWITCH_INSTANT.getEpochSecond(), + (long) GREGORIAN_SWITCH_INSTANT.getNano() * PICOSECONDS_PER_NANOSECOND, + UTC_KEY); + private static final long SHORT_DATETIMEOFFSET_DISABLE_VALUE = GREGORIAN_SWITCH_INSTANT.toEpochMilli(); + + private static final PredicatePushdownController SQLSERVER_DATE_TIME_PUSHDOWN = (session, domain) -> { + Domain simplifiedDomain = domain.simplify(getDomainCompactionThreshold(session)); + for (Range range : simplifiedDomain.getValues().getRanges().getOrderedRanges()) { + Range disableRange = range.getType().getJavaType().equals(LongTimestampWithTimeZone.class) + ? Range.lessThan(range.getType(), LONG_DATETIMEOFFSET_DISABLE_VALUE) + : Range.lessThan(range.getType(), SHORT_DATETIMEOFFSET_DISABLE_VALUE); + + // If there is any overlap of any predicate range and (-inf, 1583), disable push down + if (range.overlaps(disableRange)) { + return DISABLE_PUSHDOWN.apply(session, domain); + } + } + + return FULL_PUSHDOWN.apply(session, domain); + }; + @Inject public SqlServerClient( BaseJdbcConfig config, @@ -894,19 +934,29 @@ private static ColumnMapping timestampWithTimeZoneColumnMapping(int precision) return ColumnMapping.longMapping( createTimestampWithTimeZoneType(precision), shortTimestampWithTimeZoneReadFunction(), - shortTimestampWithTimeZoneWriteFunction()); + shortTimestampWithTimeZoneWriteFunction(), + SQLSERVER_DATE_TIME_PUSHDOWN); } return ColumnMapping.objectMapping( createTimestampWithTimeZoneType(precision), longTimestampWithTimeZoneReadFunction(), - longTimestampWithTimeZoneWriteFunction()); + longTimestampWithTimeZoneWriteFunction(), + SQLSERVER_DATE_TIME_PUSHDOWN); } private static LongReadFunction shortTimestampWithTimeZoneReadFunction() { return (resultSet, columnIndex) -> { - OffsetDateTime offsetDateTime = resultSet.getObject(columnIndex, OffsetDateTime.class); - ZonedDateTime zonedDateTime = offsetDateTime.toZonedDateTime(); + ZonedDateTime zonedDateTime; + DateTimeOffset dateTimeOffset = resultSet.getObject(columnIndex, DateTimeOffset.class); + if (dateTimeOffset.compareTo(GREGORIAN_SWITCH_DATETIMEOFFSET) < 0) { + String stringValue = resultSet.getString(columnIndex); + zonedDateTime = ZonedDateTime.from(DATE_TIME_OFFSET_FORMATTER.parse(stringValue)); + } + else { + zonedDateTime = dateTimeOffset.getOffsetDateTime().toZonedDateTime(); + } + return packDateTimeWithZone(zonedDateTime.toInstant().toEpochMilli(), zonedDateTime.getZone().getId()); }; } @@ -925,7 +975,16 @@ private static ObjectReadFunction longTimestampWithTimeZoneReadFunction() return ObjectReadFunction.of( LongTimestampWithTimeZone.class, (resultSet, columnIndex) -> { - OffsetDateTime offsetDateTime = resultSet.getObject(columnIndex, OffsetDateTime.class); + OffsetDateTime offsetDateTime; + DateTimeOffset dateTimeOffset = resultSet.getObject(columnIndex, DateTimeOffset.class); + if (dateTimeOffset.compareTo(GREGORIAN_SWITCH_DATETIMEOFFSET) < 0) { + String stringValue = resultSet.getString(columnIndex); + offsetDateTime = ZonedDateTime.from(DATE_TIME_OFFSET_FORMATTER.parse(stringValue)).toOffsetDateTime(); + } + else { + offsetDateTime = dateTimeOffset.getOffsetDateTime(); + } + return LongTimestampWithTimeZone.fromEpochSecondsAndFraction( offsetDateTime.toEpochSecond(), (long) offsetDateTime.getNano() * PICOSECONDS_PER_NANOSECOND, diff --git a/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/BaseSqlServerTypeMapping.java b/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/BaseSqlServerTypeMapping.java index f6aae0bb6f1d7..97144a7092816 100644 --- a/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/BaseSqlServerTypeMapping.java +++ b/plugin/trino-sqlserver/src/test/java/io/trino/plugin/sqlserver/BaseSqlServerTypeMapping.java @@ -34,6 +34,7 @@ import java.time.LocalDate; import java.time.LocalDateTime; import java.time.ZoneId; +import java.util.List; import java.util.function.Function; import static com.google.common.base.Preconditions.checkState; @@ -53,9 +54,11 @@ import static io.trino.spi.type.VarbinaryType.VARBINARY; import static io.trino.spi.type.VarcharType.createUnboundedVarcharType; import static io.trino.spi.type.VarcharType.createVarcharType; +import static io.trino.sql.planner.assertions.PlanMatchPattern.tableScan; import static io.trino.testing.TestingNames.randomNameSuffix; import static java.lang.String.format; import static java.time.ZoneOffset.UTC; +import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; @@ -836,6 +839,138 @@ private void testSqlServerDatetimeOffset(ZoneId sessionZone) .execute(getQueryRunner(), session, sqlServerCreateAndInsert("test_sqlserver_datetimeoffset")); } + @Test + public void testSqlServerDatetimeOffsetHistoricalDates() + { + testSqlServerDatetimeOffsetHistoricalDates(UTC); + testSqlServerDatetimeOffsetHistoricalDates(ZoneId.systemDefault()); + // using two non-JVM zones so that we don't need to worry what SQL Server system zone is + // no DST in 1970, but has DST in later years (e.g. 2018) + testSqlServerDatetimeOffsetHistoricalDates(ZoneId.of("Europe/Vilnius")); + // minutes offset change since 1970-01-01, no DST + testSqlServerDatetimeOffsetHistoricalDates(ZoneId.of("Asia/Kathmandu")); + testSqlServerDatetimeOffsetHistoricalDates(TestingSession.DEFAULT_TIME_ZONE_KEY.getZoneId()); + } + + public void testSqlServerDatetimeOffsetHistoricalDates(ZoneId sessionZone) + { + Session session = Session.builder(getSession()) + .setTimeZoneKey(TimeZoneKey.getTimeZoneKey(sessionZone.getId())) + .build(); + + SqlDataTypeTest.create() + .addRoundTrip("DATETIMEOFFSET(0)", "'1400-09-27 00:00:00+07:00'", createTimestampWithTimeZoneType(0), "TIMESTAMP '1400-09-27 00:00:00+07:00'") + .addRoundTrip("DATETIMEOFFSET(1)", "'1400-09-27 00:00:00.1+07:00'", createTimestampWithTimeZoneType(1), "TIMESTAMP '1400-09-27 00:00:00.1+07:00'") + .addRoundTrip("DATETIMEOFFSET(2)", "'1400-09-27 00:00:00.12+07:00'", createTimestampWithTimeZoneType(2), "TIMESTAMP '1400-09-27 00:00:00.12+07:00'") + .addRoundTrip("DATETIMEOFFSET(3)", "'1400-09-27 00:00:00.123+07:00'", createTimestampWithTimeZoneType(3), "TIMESTAMP '1400-09-27 00:00:00.123+07:00'") + .addRoundTrip("DATETIMEOFFSET(4)", "'1400-09-27 00:00:00.1234+07:00'", createTimestampWithTimeZoneType(4), "TIMESTAMP '1400-09-27 00:00:00.1234+07:00'") + .addRoundTrip("DATETIMEOFFSET(5)", "'1400-09-27 00:00:00.12345+07:00'", createTimestampWithTimeZoneType(5), "TIMESTAMP '1400-09-27 00:00:00.12345+07:00'") + .addRoundTrip("DATETIMEOFFSET(6)", "'1400-09-27 00:00:00.123456+07:00'", createTimestampWithTimeZoneType(6), "TIMESTAMP '1400-09-27 00:00:00.123456+07:00'") + .addRoundTrip("DATETIMEOFFSET(7)", "'1400-09-27 00:00:00.1234567+07:00'", createTimestampWithTimeZoneType(7), "TIMESTAMP '1400-09-27 00:00:00.1234567+07:00'") + .execute(getQueryRunner(), session, sqlServerCreateAndInsert("test_sqlserver_datetimeoffset_historical_date")); + } + + @Test + public void testSqlServerDatetimeOffsetHistoricalDatesRangeQuery() + { + // Tests the custom predicate push down controller for DATETIMEOFFSET types with values before and after 1583 + List dateTimeOffsetValues = List.of( + "'1400-01-01 00:00:00.1234567+00:00'", + "'1500-01-01 00:00:00.1234567+00:00'", + "'1582-12-31 23:59:59.9999999+00:00'", + "'1583-01-01 00:00:00+00:00'", + "'1583-01-01 00:00:00.1234567+00:00'", + "'1600-01-01 00:00:00.1234567+00:00'", + "'1700-01-01 00:00:00.1234567+00:00'", + "'1800-01-01 00:00:00.1234567+00:00'", + "'1900-01-01 00:00:00.1234567+00:00'"); + + try (TestTable table = new TestTable(onRemoteDatabase(), "test_sqlserver_datetimeoffset_historical_date_range_query", "(col0 datetimeoffset(7))", dateTimeOffsetValues)) { + assertThat(query("SELECT count(*) FROM " + table.getName())) + .matches("SELECT CAST(9 AS BIGINT)") + .isFullyPushedDown(); + + assertThat(query("SELECT * FROM " + table.getName() + " WHERE col0 <= TIMESTAMP '1582-12-31 23:59:59.9999999+00:00'")) + .matches(""" + VALUES (TIMESTAMP '1400-01-01 00:00:00.1234567+00:00'), + (TIMESTAMP '1500-01-01 00:00:00.1234567+00:00'), + (TIMESTAMP '1582-12-31 23:59:59.9999999+00:00')""") + .isNotFullyPushedDown(tableScan(table.getName())); + + assertThat(query("SELECT * FROM " + table.getName() + " WHERE col0 >= TIMESTAMP '1583-01-01 00:00:00+00:00'")) + .matches(""" + VALUES (TIMESTAMP '1583-01-01 00:00:00+00:00'), + (TIMESTAMP '1583-01-01 00:00:00.1234567+00:00'), + (TIMESTAMP '1600-01-01 00:00:00.1234567+00:00'), + (TIMESTAMP '1700-01-01 00:00:00.1234567+00:00'), + (TIMESTAMP '1800-01-01 00:00:00.1234567+00:00'), + (TIMESTAMP '1900-01-01 00:00:00.1234567+00:00')""") + .isFullyPushedDown(); + + assertThat(query("SELECT * FROM " + table.getName() + " WHERE col0 IN (TIMESTAMP '1582-12-31 23:59:59.9999999+00:00', TIMESTAMP '1583-01-01 00:00:00+00:00')")) + .matches(""" + VALUES (TIMESTAMP '1582-12-31 23:59:59.9999999+00:00'), + (TIMESTAMP '1583-01-01 00:00:00+00:00')""") + .isNotFullyPushedDown(tableScan(table.getName())); + + assertThat(query("SELECT * FROM " + table.getName() + " WHERE col0 IN (TIMESTAMP '1583-01-01 00:00:00+00:00', TIMESTAMP '1600-01-01 00:00:00.1234567+00:00')")) + .matches(""" + VALUES (TIMESTAMP '1583-01-01 00:00:00+00:00'), + (TIMESTAMP '1600-01-01 00:00:00.1234567+00:00')""") + .isFullyPushedDown(); + + assertThat(query("SELECT * FROM " + table.getName() + " WHERE col0 NOT IN (TIMESTAMP '1582-12-31 23:59:59.9999999+00:00', TIMESTAMP '1600-01-01 00:00:00.1234567+00:00')")) + .matches(""" + VALUES (TIMESTAMP '1400-01-01 00:00:00.1234567+00:00'), + (TIMESTAMP '1500-01-01 00:00:00.1234567+00:00'), + (TIMESTAMP '1583-01-01 00:00:00+00:00'), + (TIMESTAMP '1583-01-01 00:00:00.1234567+00:00'), + (TIMESTAMP '1700-01-01 00:00:00.1234567+00:00'), + (TIMESTAMP '1800-01-01 00:00:00.1234567+00:00'), + (TIMESTAMP '1900-01-01 00:00:00.1234567+00:00')""") + .isNotFullyPushedDown(tableScan(table.getName())); + + assertThat(query("SELECT * FROM " + table.getName() + " WHERE col0 NOT IN (TIMESTAMP '1583-01-01 00:00:00+00:00', TIMESTAMP '1600-01-01 00:00:00.1234567+00:00')")) + .matches(""" + VALUES (TIMESTAMP '1400-01-01 00:00:00.1234567+00:00'), + (TIMESTAMP '1500-01-01 00:00:00.1234567+00:00'), + (TIMESTAMP '1582-12-31 23:59:59.9999999+00:00'), + (TIMESTAMP '1583-01-01 00:00:00.1234567+00:00'), + (TIMESTAMP '1700-01-01 00:00:00.1234567+00:00'), + (TIMESTAMP '1800-01-01 00:00:00.1234567+00:00'), + (TIMESTAMP '1900-01-01 00:00:00.1234567+00:00')""") + .isNotFullyPushedDown(tableScan(table.getName())); + + assertThat(query("SELECT * FROM " + table.getName() + " WHERE col0 BETWEEN TIMESTAMP '1582-12-31 23:59:59.9999999+00:00' AND TIMESTAMP '1600-01-01 00:00:00.1234567+00:00'")) + .matches(""" + VALUES (TIMESTAMP '1582-12-31 23:59:59.9999999+00:00'), + (TIMESTAMP '1583-01-01 00:00:00+00:00'), + (TIMESTAMP '1583-01-01 00:00:00.1234567+00:00'), + (TIMESTAMP '1600-01-01 00:00:00.1234567+00:00')""") + .isNotFullyPushedDown(tableScan(table.getName())); + + assertThat(query("SELECT * FROM " + table.getName() + " WHERE col0 BETWEEN TIMESTAMP '1583-01-01 00:00:00+00:00' AND TIMESTAMP '1600-01-01 00:00:00.1234567+00:00'")) + .matches(""" + VALUES (TIMESTAMP '1583-01-01 00:00:00+00:00'), + (TIMESTAMP '1583-01-01 00:00:00.1234567+00:00'), + (TIMESTAMP '1600-01-01 00:00:00.1234567+00:00')""") + .isFullyPushedDown(); + + assertThat(query("SELECT * FROM " + table.getName() + " WHERE col0 <= TIMESTAMP '1990-01-01 00:00:00+00:00'")) + .matches(""" + VALUES (TIMESTAMP '1400-01-01 00:00:00.1234567+00:00'), + (TIMESTAMP '1500-01-01 00:00:00.1234567+00:00'), + (TIMESTAMP '1582-12-31 23:59:59.9999999+00:00'), + (TIMESTAMP '1583-01-01 00:00:00+00:00'), + (TIMESTAMP '1583-01-01 00:00:00.1234567+00:00'), + (TIMESTAMP '1600-01-01 00:00:00.1234567+00:00'), + (TIMESTAMP '1700-01-01 00:00:00.1234567+00:00'), + (TIMESTAMP '1800-01-01 00:00:00.1234567+00:00'), + (TIMESTAMP '1900-01-01 00:00:00.1234567+00:00')""") + .isNotFullyPushedDown(tableScan(table.getName())); + } + } + protected DataSetup trinoCreateAsSelect(String tableNamePrefix) { return trinoCreateAsSelect(getSession(), tableNamePrefix); From c106cf5de5cae8ac0bb94544b12700701401b8a0 Mon Sep 17 00:00:00 2001 From: Dejan Mircevski Date: Fri, 22 Dec 2023 18:44:16 -0500 Subject: [PATCH 212/350] Only log null spooling stats in FTE mode In pipelined mode, the spooling stats are always null, which fills the log needlessly. https://github.com/trinodb/trino/pull/20214#discussion_r1435339288 --- .../io/trino/server/remotetask/HttpRemoteTask.java | 6 +++++- .../io/trino/server/remotetask/TaskInfoFetcher.java | 10 ++++++++-- 2 files changed, 13 insertions(+), 3 deletions(-) diff --git a/core/trino-main/src/main/java/io/trino/server/remotetask/HttpRemoteTask.java b/core/trino-main/src/main/java/io/trino/server/remotetask/HttpRemoteTask.java index af868a76358f3..d767cc62505cc 100644 --- a/core/trino-main/src/main/java/io/trino/server/remotetask/HttpRemoteTask.java +++ b/core/trino-main/src/main/java/io/trino/server/remotetask/HttpRemoteTask.java @@ -58,6 +58,7 @@ import io.trino.execution.buffer.PipelinedOutputBuffers; import io.trino.execution.buffer.SpoolingOutputStats; import io.trino.metadata.Split; +import io.trino.operator.RetryPolicy; import io.trino.operator.TaskStats; import io.trino.server.DynamicFilterService; import io.trino.server.FailTaskRequest; @@ -110,6 +111,7 @@ import static io.trino.SystemSessionProperties.getMaxUnacknowledgedSplitsPerTask; import static io.trino.SystemSessionProperties.getRemoteTaskGuaranteedSplitsPerRequest; import static io.trino.SystemSessionProperties.getRemoteTaskRequestSizeHeadroom; +import static io.trino.SystemSessionProperties.getRetryPolicy; import static io.trino.SystemSessionProperties.isRemoteTaskAdaptiveUpdateRequestSizeEnabled; import static io.trino.execution.DynamicFiltersCollector.INITIAL_DYNAMIC_FILTERS_VERSION; import static io.trino.execution.TaskInfo.createInitialTask; @@ -343,6 +345,7 @@ public HttpRemoteTask( errorScheduledExecutor, stats); + RetryPolicy retryPolicy = getRetryPolicy(session); this.taskInfoFetcher = new TaskInfoFetcher( this::fatalUnacknowledgedFailure, taskStatusFetcher, @@ -357,7 +360,8 @@ public HttpRemoteTask( updateScheduledExecutor, errorScheduledExecutor, stats, - estimatedMemory); + estimatedMemory, + retryPolicy); taskStatusFetcher.addStateChangeListener(newStatus -> { TaskState state = newStatus.getState(); diff --git a/core/trino-main/src/main/java/io/trino/server/remotetask/TaskInfoFetcher.java b/core/trino-main/src/main/java/io/trino/server/remotetask/TaskInfoFetcher.java index 73b14cade4ba9..7f87b2c5c94d1 100644 --- a/core/trino-main/src/main/java/io/trino/server/remotetask/TaskInfoFetcher.java +++ b/core/trino-main/src/main/java/io/trino/server/remotetask/TaskInfoFetcher.java @@ -33,6 +33,7 @@ import io.trino.execution.TaskState; import io.trino.execution.TaskStatus; import io.trino.execution.buffer.SpoolingOutputStats; +import io.trino.operator.RetryPolicy; import java.net.URI; import java.util.Optional; @@ -52,6 +53,7 @@ import static io.airlift.http.client.HttpUriBuilder.uriBuilderFrom; import static io.airlift.http.client.Request.Builder.prepareGet; import static io.airlift.units.Duration.nanosSince; +import static io.trino.operator.RetryPolicy.TASK; import static java.util.Objects.requireNonNull; import static java.util.concurrent.TimeUnit.MILLISECONDS; @@ -81,6 +83,8 @@ public class TaskInfoFetcher private final AtomicReference spoolingOutputStats = new AtomicReference<>(); + private final RetryPolicy retryPolicy; + @GuardedBy("this") private boolean running; @@ -104,7 +108,8 @@ public TaskInfoFetcher( ScheduledExecutorService updateScheduledExecutor, ScheduledExecutorService errorScheduledExecutor, RemoteTaskStats stats, - Optional estimatedMemory) + Optional estimatedMemory, + RetryPolicy retryPolicy) { requireNonNull(initialTask, "initialTask is null"); requireNonNull(errorScheduledExecutor, "errorScheduledExecutor is null"); @@ -127,6 +132,7 @@ public TaskInfoFetcher( this.spanBuilderFactory = requireNonNull(spanBuilderFactory, "spanBuilderFactory is null"); this.stats = requireNonNull(stats, "stats is null"); this.estimatedMemory = requireNonNull(estimatedMemory, "estimatedMemory is null"); + this.retryPolicy = requireNonNull(retryPolicy, "retryPolicy is null"); } public TaskInfo getTaskInfo() @@ -268,7 +274,7 @@ synchronized void updateTaskInfo(TaskInfo newTaskInfo) if (newTaskInfo.getTaskStatus().getState().isDone()) { boolean wasSet = spoolingOutputStats.compareAndSet(null, newTaskInfo.getOutputBuffers().getSpoolingOutputStats().orElse(null)); - if (wasSet && spoolingOutputStats.get() == null) { + if (retryPolicy == TASK && wasSet && spoolingOutputStats.get() == null) { log.debug("Task %s was updated to null spoolingOutputStats. Future calls to retrieveAndDropSpoolingOutputStats will fail.", taskId); } newTaskInfo = newTaskInfo.pruneSpoolingOutputStats(); From ee7046b07f7ba8a633f25460bd77683c63cd0f9c Mon Sep 17 00:00:00 2001 From: Karol Sobczak Date: Tue, 26 Dec 2023 23:45:52 +0100 Subject: [PATCH 213/350] Fix failing test on master --- .../AbstractPredicatePushdownTest.java | 6 ++-- ...PredicatePushdownWithoutDynamicFilter.java | 29 ------------------- 2 files changed, 3 insertions(+), 32 deletions(-) diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/AbstractPredicatePushdownTest.java b/core/trino-main/src/test/java/io/trino/sql/planner/AbstractPredicatePushdownTest.java index 5fb02b62ae8f7..476b697ffc14f 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/AbstractPredicatePushdownTest.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/AbstractPredicatePushdownTest.java @@ -486,12 +486,12 @@ public void testSimplifyNonInferrableInheritedPredicate() @Test public void testDoesNotCreatePredicateFromInferredPredicate() { - assertPlan("SELECT * FROM (SELECT *, nationkey + 1 as nationkey2 FROM nation) a JOIN nation b ON a.nationkey = b.nationkey", + assertPlan("SELECT * FROM (SELECT *, nationkey + 1 as nationkey2 FROM nation) a JOIN nation b ON a.nationkey2 = b.nationkey", output( join(INNER, builder -> builder - .equiCriteria("L_NATIONKEY", "R_NATIONKEY") + .equiCriteria("L_NATIONKEY2", "R_NATIONKEY") .left( - filter("true", // DF filter + project(ImmutableMap.of("L_NATIONKEY2", expression("L_NATIONKEY + BIGINT '1'")), tableScan("nation", ImmutableMap.of("L_NATIONKEY", "nationkey")))) .right( anyTree( diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/TestPredicatePushdownWithoutDynamicFilter.java b/core/trino-main/src/test/java/io/trino/sql/planner/TestPredicatePushdownWithoutDynamicFilter.java index 92fa340168ef6..1c4e627e47429 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/TestPredicatePushdownWithoutDynamicFilter.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/TestPredicatePushdownWithoutDynamicFilter.java @@ -13,7 +13,6 @@ */ package io.trino.sql.planner; -import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import io.trino.Session; import io.trino.sql.planner.plan.ExchangeNode; @@ -24,7 +23,6 @@ import static io.trino.sql.planner.assertions.PlanMatchPattern.filter; import static io.trino.sql.planner.assertions.PlanMatchPattern.join; import static io.trino.sql.planner.assertions.PlanMatchPattern.node; -import static io.trino.sql.planner.assertions.PlanMatchPattern.output; import static io.trino.sql.planner.assertions.PlanMatchPattern.project; import static io.trino.sql.planner.assertions.PlanMatchPattern.semiJoin; import static io.trino.sql.planner.assertions.PlanMatchPattern.tableScan; @@ -175,31 +173,4 @@ public void testNonStraddlingJoinExpression() anyTree( tableScan("orders", ImmutableMap.of("ORDERS_OK", "orderkey"))))))); } - - @Override - @Test - public void testDoesNotCreatePredicateFromInferredPredicate() - { - assertPlan("SELECT * FROM (SELECT *, nationkey + 1 as nationkey2 FROM nation) a JOIN nation b ON a.nationkey = b.nationkey", - output( - join(INNER, builder -> builder - .equiCriteria("L_NATIONKEY", "R_NATIONKEY") - .left( - tableScan("nation", ImmutableMap.of("L_NATIONKEY", "nationkey"))) - .right( - anyTree( - tableScan("nation", ImmutableMap.of("R_NATIONKEY", "nationkey"))))))); - - assertPlan("SELECT * FROM (SELECT * FROM nation WHERE nationkey = 5) a JOIN (SELECT * FROM nation WHERE nationkey = 5) b ON a.nationkey = b.nationkey", - output( - join(INNER, builder -> builder - .equiCriteria(ImmutableList.of()) - .left( - filter("L_NATIONKEY = BIGINT '5'", - tableScan("nation", ImmutableMap.of("L_NATIONKEY", "nationkey")))) - .right( - anyTree( - filter("R_NATIONKEY = BIGINT '5'", - tableScan("nation", ImmutableMap.of("R_NATIONKEY", "nationkey")))))))); - } } From f124589b17bd2fc4abf3ddde499f3ead82c1398f Mon Sep 17 00:00:00 2001 From: Ashhar Hasan Date: Thu, 21 Dec 2023 14:15:45 +0530 Subject: [PATCH 214/350] Use explicit values for MaxResults in all Glue APIs Most listing Glue APIs accept a MaxResults parameter which decides how many objects are returned in a single API call. The default values are not documented but observed behaviour is that the default values change on some basis. This commit adds explicit values for MaxResults in the APIs which support it to the maximum possible values. This possibly reduces the number of Glue calls when listing tables, databases or functions in some cases. This is similar to 4f22b0ebb52c71e07370b41f785fa88dde289c22 and 45dc37d4b68b6e7e72d2b50810fe96ebab08c0c3. --- .../hive/metastore/glue/GlueHiveMetastore.java | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueHiveMetastore.java index e7ba1fab181c7..e25d21898776e 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/glue/GlueHiveMetastore.java @@ -185,6 +185,9 @@ public class GlueHiveMetastore private static final int BATCH_CREATE_PARTITION_MAX_PAGE_SIZE = 100; private static final int BATCH_UPDATE_PARTITION_MAX_PAGE_SIZE = 100; private static final int AWS_GLUE_GET_PARTITIONS_MAX_RESULTS = 1000; + private static final int AWS_GLUE_GET_DATABASES_MAX_RESULTS = 100; + private static final int AWS_GLUE_GET_FUNCTIONS_MAX_RESULTS = 100; + private static final int AWS_GLUE_GET_TABLES_MAX_RESULTS = 100; private static final Comparator> PARTITION_VALUE_COMPARATOR = lexicographical(String.CASE_INSENSITIVE_ORDER); private static final Predicate SOME_KIND_OF_VIEW_FILTER = table -> VIRTUAL_VIEW.name().equals(getTableTypeNullable(table)); private static final RetryPolicy CONCURRENT_MODIFICATION_EXCEPTION_RETRY_POLICY = RetryPolicy.builder() @@ -253,7 +256,8 @@ public List getAllDatabases() try { List databaseNames = getPaginatedResults( glueClient::getDatabases, - new GetDatabasesRequest(), + new GetDatabasesRequest() + .withMaxResults(AWS_GLUE_GET_DATABASES_MAX_RESULTS), GetDatabasesRequest::setNextToken, GetDatabasesResult::getNextToken, stats.getGetDatabases()) @@ -1291,7 +1295,8 @@ private Collection getFunctionsByPattern(String databaseName, glueClient::getUserDefinedFunctions, new GetUserDefinedFunctionsRequest() .withDatabaseName(databaseName) - .withPattern(functionNamePattern), + .withPattern(functionNamePattern) + .withMaxResults(AWS_GLUE_GET_FUNCTIONS_MAX_RESULTS), GetUserDefinedFunctionsRequest::setNextToken, GetUserDefinedFunctionsResult::getNextToken, stats.getGetUserDefinedFunctions()) @@ -1370,7 +1375,8 @@ private Stream getGlueTables(String dat return getPaginatedResults( glueClient::getTables, new GetTablesRequest() - .withDatabaseName(databaseName), + .withDatabaseName(databaseName) + .withMaxResults(AWS_GLUE_GET_TABLES_MAX_RESULTS), GetTablesRequest::setNextToken, GetTablesResult::getNextToken, stats.getGetTables()) From a9d1e03efc4a30fd2a527224544133b8f03c318b Mon Sep 17 00:00:00 2001 From: Jonas Irgens Kylling Date: Fri, 29 Dec 2023 12:57:50 +0100 Subject: [PATCH 215/350] Add Delta table read version to connectorInfo --- .../trino/plugin/deltalake/DeltaLakeInputInfo.java | 14 +++++++++++--- .../trino/plugin/deltalake/DeltaLakeMetadata.java | 5 +++-- .../plugin/deltalake/TestDeltaLakeMetadata.java | 4 ++-- 3 files changed, 16 insertions(+), 7 deletions(-) diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeInputInfo.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeInputInfo.java index e65b1cf0c3a3e..9bc5fc99fe577 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeInputInfo.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeInputInfo.java @@ -21,11 +21,13 @@ public class DeltaLakeInputInfo { private final boolean partitioned; + private final long version; @JsonCreator - public DeltaLakeInputInfo(@JsonProperty("partitioned") boolean partitioned) + public DeltaLakeInputInfo(@JsonProperty("partitioned") boolean partitioned, @JsonProperty("version") long version) { this.partitioned = partitioned; + this.version = version; } @JsonProperty @@ -34,6 +36,12 @@ public boolean isPartitioned() return partitioned; } + @JsonProperty + public long getVersion() + { + return version; + } + @Override public boolean equals(Object o) { @@ -43,12 +51,12 @@ public boolean equals(Object o) if (!(o instanceof DeltaLakeInputInfo that)) { return false; } - return partitioned == that.partitioned; + return partitioned == that.partitioned && version == that.version; } @Override public int hashCode() { - return Objects.hash(partitioned); + return Objects.hash(partitioned, version); } } 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 49a639f1c0fba..ee79488b7fe0d 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 @@ -2362,8 +2362,9 @@ private void cleanupFailedWrite(ConnectorSession session, String tableLocation, @Override public Optional getInfo(ConnectorTableHandle table) { - boolean isPartitioned = !((DeltaLakeTableHandle) table).getMetadataEntry().getLowercasePartitionColumns().isEmpty(); - return Optional.of(new DeltaLakeInputInfo(isPartitioned)); + DeltaLakeTableHandle handle = (DeltaLakeTableHandle) table; + boolean isPartitioned = !handle.getMetadataEntry().getLowercasePartitionColumns().isEmpty(); + return Optional.of(new DeltaLakeInputInfo(isPartitioned, handle.getReadVersion())); } @Override diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeMetadata.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeMetadata.java index 8b006a851dc3a..2392455a57b21 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeMetadata.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeMetadata.java @@ -475,7 +475,7 @@ public void testGetInputInfoForPartitionedTable() ImmutableList.of(BIGINT_COLUMN_1)); deltaLakeMetadata.createTable(SESSION, tableMetadata, false); DeltaLakeTableHandle tableHandle = (DeltaLakeTableHandle) deltaLakeMetadata.getTableHandle(SESSION, tableMetadata.getTable()); - assertThat(deltaLakeMetadata.getInfo(tableHandle)).isEqualTo(Optional.of(new DeltaLakeInputInfo(true))); + assertThat(deltaLakeMetadata.getInfo(tableHandle)).isEqualTo(Optional.of(new DeltaLakeInputInfo(true, 0))); deltaLakeMetadata.cleanupQuery(SESSION); } @@ -488,7 +488,7 @@ public void testGetInputInfoForUnPartitionedTable() ImmutableList.of()); deltaLakeMetadata.createTable(SESSION, tableMetadata, false); DeltaLakeTableHandle tableHandle = (DeltaLakeTableHandle) deltaLakeMetadata.getTableHandle(SESSION, tableMetadata.getTable()); - assertThat(deltaLakeMetadata.getInfo(tableHandle)).isEqualTo(Optional.of(new DeltaLakeInputInfo(false))); + assertThat(deltaLakeMetadata.getInfo(tableHandle)).isEqualTo(Optional.of(new DeltaLakeInputInfo(false, 0))); deltaLakeMetadata.cleanupQuery(SESSION); } From b6eb50c07daa0fea8e9e85c943f30bad97979574 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 29 Dec 2023 09:31:38 +0100 Subject: [PATCH 216/350] Fix case in RN section header --- docs/src/main/sphinx/release/release-434.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/src/main/sphinx/release/release-434.md b/docs/src/main/sphinx/release/release-434.md index 2c62dcd348ee6..c7c3c4d2488d6 100644 --- a/docs/src/main/sphinx/release/release-434.md +++ b/docs/src/main/sphinx/release/release-434.md @@ -67,7 +67,7 @@ * Add support for separate metadata caching configuration for schemas, tables, and metadata. ({issue}`19859`) -## MySQl connector +## MySQL connector * Add support for separate metadata caching configuration for schemas, tables, and metadata. ({issue}`19859`) From 2a844f2f591f78627b5a7ed6f260ab565f9374e2 Mon Sep 17 00:00:00 2001 From: Raunaq Morarka Date: Wed, 27 Dec 2023 11:44:42 +0530 Subject: [PATCH 217/350] Refactor IcebergSplitSource to reduce per split operations We can apply pruning logic at file level before splitting it into splits to avoid redundant operations per split --- .../plugin/iceberg/IcebergSplitSource.java | 121 +++++++++--------- 1 file changed, 59 insertions(+), 62 deletions(-) diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplitSource.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplitSource.java index b18feb4c66767..6432e6a8df195 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplitSource.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplitSource.java @@ -124,7 +124,6 @@ public class IcebergSplitSource private long targetSplitSize; private CloseableIterator fileScanIterator; private Iterator fileTasksIterator = emptyIterator(); - private boolean fileHasAnyDeletions; private final boolean recordScannedFiles; private final ImmutableSet.Builder scannedFiles = ImmutableSet.builder(); @@ -229,81 +228,79 @@ public CompletableFuture getNextBatch(int maxSize) List splits = new ArrayList<>(maxSize); while (splits.size() < maxSize && (fileTasksIterator.hasNext() || fileScanIterator.hasNext())) { if (!fileTasksIterator.hasNext()) { - FileScanTask wholeFileTask = fileScanIterator.next(); - if (wholeFileTask.deletes().isEmpty() && noDataColumnsProjected(wholeFileTask)) { - fileTasksIterator = List.of(wholeFileTask).iterator(); + if (limit.isPresent() && limit.getAsLong() <= outputRowsLowerBound) { + finish(); + break; } - else { - fileTasksIterator = wholeFileTask.split(targetSplitSize).iterator(); + FileScanTask wholeFileTask = fileScanIterator.next(); + boolean fileHasNoDeletions = wholeFileTask.deletes().isEmpty(); + + if (fileHasNoDeletions && + maxScannedFileSizeInBytes.isPresent() && + wholeFileTask.file().fileSizeInBytes() > maxScannedFileSizeInBytes.get()) { + continue; } - fileHasAnyDeletions = false; - // In theory, .split() could produce empty iterator, so let's evaluate the outer loop condition again. - continue; - } - FileScanTask scanTask = fileTasksIterator.next(); - fileHasAnyDeletions = fileHasAnyDeletions || !scanTask.deletes().isEmpty(); - if (scanTask.deletes().isEmpty() && - maxScannedFileSizeInBytes.isPresent() && - scanTask.file().fileSizeInBytes() > maxScannedFileSizeInBytes.get()) { - continue; - } - if (!pathDomain.includesNullableValue(utf8Slice(scanTask.file().path().toString()))) { - continue; - } - if (!fileModifiedTimeDomain.isAll()) { - long fileModifiedTime = getModificationTime(scanTask.file().path().toString()); - if (!fileModifiedTimeDomain.includesNullableValue(packDateTimeWithZone(fileModifiedTime, UTC_KEY))) { + if (!pathDomain.includesNullableValue(utf8Slice(wholeFileTask.file().path().toString()))) { continue; } - } - IcebergSplit icebergSplit = toIcebergSplit(scanTask); + if (!fileModifiedTimeDomain.isAll()) { + long fileModifiedTime = getModificationTime(wholeFileTask.file().path().toString()); + if (!fileModifiedTimeDomain.includesNullableValue(packDateTimeWithZone(fileModifiedTime, UTC_KEY))) { + continue; + } + } - Schema fileSchema = scanTask.spec().schema(); - Map> partitionKeys = getPartitionKeys(scanTask); + Schema fileSchema = wholeFileTask.spec().schema(); + Map> partitionKeys = getPartitionKeys(wholeFileTask); - Set identityPartitionColumns = partitionKeys.keySet().stream() - .map(fieldId -> getColumnHandle(fileSchema.findField(fieldId), typeManager)) - .collect(toImmutableSet()); + Set identityPartitionColumns = partitionKeys.keySet().stream() + .map(fieldId -> getColumnHandle(fileSchema.findField(fieldId), typeManager)) + .collect(toImmutableSet()); - Supplier> partitionValues = memoize(() -> getPartitionValues(identityPartitionColumns, partitionKeys)); + Supplier> partitionValues = memoize(() -> getPartitionValues(identityPartitionColumns, partitionKeys)); - if (!dynamicFilterPredicate.isAll() && !dynamicFilterPredicate.equals(pushedDownDynamicFilterPredicate)) { - if (!partitionMatchesPredicate( - identityPartitionColumns, - partitionValues, - dynamicFilterPredicate)) { - continue; + if (!dynamicFilterPredicate.isAll() && !dynamicFilterPredicate.equals(pushedDownDynamicFilterPredicate)) { + if (!partitionMatchesPredicate( + identityPartitionColumns, + partitionValues, + dynamicFilterPredicate)) { + continue; + } + if (!fileMatchesPredicate( + fieldIdToType, + dynamicFilterPredicate, + wholeFileTask.file().lowerBounds(), + wholeFileTask.file().upperBounds(), + wholeFileTask.file().nullValueCounts())) { + continue; + } } - if (!fileMatchesPredicate( - fieldIdToType, - dynamicFilterPredicate, - scanTask.file().lowerBounds(), - scanTask.file().upperBounds(), - scanTask.file().nullValueCounts())) { + if (!partitionMatchesConstraint(identityPartitionColumns, partitionValues, constraint)) { continue; } - } - if (!partitionMatchesConstraint(identityPartitionColumns, partitionValues, constraint)) { - continue; - } - if (recordScannedFiles) { - // Positional and Equality deletes can only be cleaned up if the whole table has been optimized. - // Equality deletes may apply to many files, and position deletes may be grouped together. This makes it difficult to know if they are obsolete. - List fullyAppliedDeletes = tableHandle.getEnforcedPredicate().isAll() ? scanTask.deletes() : ImmutableList.of(); - scannedFiles.add(new DataFileWithDeleteFiles(scanTask.file(), fullyAppliedDeletes)); - } - if (!fileTasksIterator.hasNext()) { - // This is the last task for this file - if (!fileHasAnyDeletions) { - // There were no deletions, so we produced splits covering the whole file - outputRowsLowerBound = saturatedAdd(outputRowsLowerBound, scanTask.file().recordCount()); - if (limit.isPresent() && limit.getAsLong() <= outputRowsLowerBound) { - finish(); - } + if (recordScannedFiles) { + // Positional and Equality deletes can only be cleaned up if the whole table has been optimized. + // Equality deletes may apply to many files, and position deletes may be grouped together. This makes it difficult to know if they are obsolete. + List fullyAppliedDeletes = tableHandle.getEnforcedPredicate().isAll() ? wholeFileTask.deletes() : ImmutableList.of(); + scannedFiles.add(new DataFileWithDeleteFiles(wholeFileTask.file(), fullyAppliedDeletes)); + } + + if (fileHasNoDeletions) { + // There were no deletions, so we will produce splits covering the whole file + outputRowsLowerBound = saturatedAdd(outputRowsLowerBound, wholeFileTask.file().recordCount()); } + + if (fileHasNoDeletions && noDataColumnsProjected(wholeFileTask)) { + fileTasksIterator = List.of(wholeFileTask).iterator(); + } + else { + fileTasksIterator = wholeFileTask.split(targetSplitSize).iterator(); + } + // In theory, .split() could produce empty iterator, so let's evaluate the outer loop condition again. + continue; } - splits.add(icebergSplit); + splits.add(toIcebergSplit(fileTasksIterator.next())); } return completedFuture(new ConnectorSplitBatch(splits, isFinished())); } From 8a14adc4ce269b5498c1370928168f3ed1d84b9f Mon Sep 17 00:00:00 2001 From: Raunaq Morarka Date: Wed, 27 Dec 2023 17:09:12 +0530 Subject: [PATCH 218/350] Extract pruneFileScanTask in IcebergSplitSource --- .../plugin/iceberg/IcebergSplitSource.java | 89 ++++++++++--------- 1 file changed, 48 insertions(+), 41 deletions(-) diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplitSource.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplitSource.java index 6432e6a8df195..9f12d7621d99e 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplitSource.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplitSource.java @@ -235,50 +235,10 @@ public CompletableFuture getNextBatch(int maxSize) FileScanTask wholeFileTask = fileScanIterator.next(); boolean fileHasNoDeletions = wholeFileTask.deletes().isEmpty(); - if (fileHasNoDeletions && - maxScannedFileSizeInBytes.isPresent() && - wholeFileTask.file().fileSizeInBytes() > maxScannedFileSizeInBytes.get()) { + if (pruneFileScanTask(wholeFileTask, fileHasNoDeletions, dynamicFilterPredicate)) { continue; } - if (!pathDomain.includesNullableValue(utf8Slice(wholeFileTask.file().path().toString()))) { - continue; - } - if (!fileModifiedTimeDomain.isAll()) { - long fileModifiedTime = getModificationTime(wholeFileTask.file().path().toString()); - if (!fileModifiedTimeDomain.includesNullableValue(packDateTimeWithZone(fileModifiedTime, UTC_KEY))) { - continue; - } - } - - Schema fileSchema = wholeFileTask.spec().schema(); - Map> partitionKeys = getPartitionKeys(wholeFileTask); - - Set identityPartitionColumns = partitionKeys.keySet().stream() - .map(fieldId -> getColumnHandle(fileSchema.findField(fieldId), typeManager)) - .collect(toImmutableSet()); - - Supplier> partitionValues = memoize(() -> getPartitionValues(identityPartitionColumns, partitionKeys)); - - if (!dynamicFilterPredicate.isAll() && !dynamicFilterPredicate.equals(pushedDownDynamicFilterPredicate)) { - if (!partitionMatchesPredicate( - identityPartitionColumns, - partitionValues, - dynamicFilterPredicate)) { - continue; - } - if (!fileMatchesPredicate( - fieldIdToType, - dynamicFilterPredicate, - wholeFileTask.file().lowerBounds(), - wholeFileTask.file().upperBounds(), - wholeFileTask.file().nullValueCounts())) { - continue; - } - } - if (!partitionMatchesConstraint(identityPartitionColumns, partitionValues, constraint)) { - continue; - } if (recordScannedFiles) { // Positional and Equality deletes can only be cleaned up if the whole table has been optimized. // Equality deletes may apply to many files, and position deletes may be grouped together. This makes it difficult to know if they are obsolete. @@ -305,6 +265,53 @@ public CompletableFuture getNextBatch(int maxSize) return completedFuture(new ConnectorSplitBatch(splits, isFinished())); } + private boolean pruneFileScanTask(FileScanTask fileScanTask, boolean fileHasNoDeletions, TupleDomain dynamicFilterPredicate) + { + if (fileHasNoDeletions && + maxScannedFileSizeInBytes.isPresent() && + fileScanTask.file().fileSizeInBytes() > maxScannedFileSizeInBytes.get()) { + return true; + } + + if (!pathDomain.includesNullableValue(utf8Slice(fileScanTask.file().path().toString()))) { + return true; + } + if (!fileModifiedTimeDomain.isAll()) { + long fileModifiedTime = getModificationTime(fileScanTask.file().path().toString()); + if (!fileModifiedTimeDomain.includesNullableValue(packDateTimeWithZone(fileModifiedTime, UTC_KEY))) { + return true; + } + } + + Schema fileSchema = fileScanTask.spec().schema(); + Map> partitionKeys = getPartitionKeys(fileScanTask); + + Set identityPartitionColumns = partitionKeys.keySet().stream() + .map(fieldId -> getColumnHandle(fileSchema.findField(fieldId), typeManager)) + .collect(toImmutableSet()); + + Supplier> partitionValues = memoize(() -> getPartitionValues(identityPartitionColumns, partitionKeys)); + + if (!dynamicFilterPredicate.isAll() && !dynamicFilterPredicate.equals(pushedDownDynamicFilterPredicate)) { + if (!partitionMatchesPredicate( + identityPartitionColumns, + partitionValues, + dynamicFilterPredicate)) { + return true; + } + if (!fileMatchesPredicate( + fieldIdToType, + dynamicFilterPredicate, + fileScanTask.file().lowerBounds(), + fileScanTask.file().upperBounds(), + fileScanTask.file().nullValueCounts())) { + return true; + } + } + + return !partitionMatchesConstraint(identityPartitionColumns, partitionValues, constraint); + } + private boolean noDataColumnsProjected(FileScanTask fileScanTask) { return fileScanTask.spec().fields().stream() From 2caf83df3f0571c38f7a2859abb48fb2c44fa8c6 Mon Sep 17 00:00:00 2001 From: Raunaq Morarka Date: Thu, 28 Dec 2023 11:04:27 +0530 Subject: [PATCH 219/350] Avoid unnecessary string to slice conversion --- .../main/java/io/trino/plugin/iceberg/IcebergSplitSource.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplitSource.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplitSource.java index 9f12d7621d99e..1adab5feb5e76 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplitSource.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplitSource.java @@ -273,7 +273,7 @@ private boolean pruneFileScanTask(FileScanTask fileScanTask, boolean fileHasNoDe return true; } - if (!pathDomain.includesNullableValue(utf8Slice(fileScanTask.file().path().toString()))) { + if (!pathDomain.isAll() && !pathDomain.includesNullableValue(utf8Slice(fileScanTask.file().path().toString()))) { return true; } if (!fileModifiedTimeDomain.isAll()) { From e88011482898201b8ad371079f7f1b0aa7a7fb7e Mon Sep 17 00:00:00 2001 From: Marius Grama Date: Fri, 29 Dec 2023 02:40:15 +0100 Subject: [PATCH 220/350] Assert position count of page instead of individual blocks --- .../iceberg/TestIcebergNodeLocalDynamicSplitPruning.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergNodeLocalDynamicSplitPruning.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergNodeLocalDynamicSplitPruning.java index 686e76eed7b07..a492386d57986 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergNodeLocalDynamicSplitPruning.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergNodeLocalDynamicSplitPruning.java @@ -118,9 +118,8 @@ public void testDynamicSplitPruning() try (ConnectorPageSource nonEmptyPageSource = createTestingPageSource(transaction, icebergConfig, inputFile, getDynamicFilter(getNonSelectiveTupleDomain()))) { Page page = nonEmptyPageSource.getNextPage(); assertThat(page).isNotNull(); - assertThat(page.getBlock(0).getPositionCount()).isEqualTo(1); + assertThat(page.getPositionCount()).isEqualTo(1); assertThat(page.getBlock(0).getInt(0, 0)).isEqualTo(KEY_COLUMN_VALUE); - assertThat(page.getBlock(1).getPositionCount()).isEqualTo(1); assertThat(page.getBlock(1).getSlice(0, 0, page.getBlock(1).getSliceLength(0)).toStringUtf8()).isEqualTo(DATA_COLUMN_VALUE); } } From 31e74bc2730718f332cedd83e09ccc11912cdfae Mon Sep 17 00:00:00 2001 From: Marius Grama Date: Wed, 27 Dec 2023 12:55:31 +0100 Subject: [PATCH 221/350] Refactor test class to be able to host eventually multiple tests --- ...stIcebergNodeLocalDynamicSplitPruning.java | 194 ++++++++---------- 1 file changed, 91 insertions(+), 103 deletions(-) diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergNodeLocalDynamicSplitPruning.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergNodeLocalDynamicSplitPruning.java index a492386d57986..1041cd6ade43d 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergNodeLocalDynamicSplitPruning.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergNodeLocalDynamicSplitPruning.java @@ -30,7 +30,6 @@ import io.trino.orc.OutputStreamOrcDataSink; import io.trino.plugin.hive.FileFormatDataSourceStats; import io.trino.plugin.hive.HiveTransactionHandle; -import io.trino.plugin.hive.metastore.Column; import io.trino.plugin.hive.orc.OrcReaderConfig; import io.trino.plugin.hive.orc.OrcWriterConfig; import io.trino.plugin.hive.parquet.ParquetReaderConfig; @@ -65,8 +64,6 @@ import static io.trino.orc.metadata.CompressionKind.NONE; import static io.trino.plugin.hive.HiveTestUtils.HDFS_ENVIRONMENT; import static io.trino.plugin.hive.HiveTestUtils.HDFS_FILE_SYSTEM_STATS; -import static io.trino.plugin.hive.HiveType.HIVE_INT; -import static io.trino.plugin.hive.HiveType.HIVE_STRING; import static io.trino.plugin.iceberg.ColumnIdentity.TypeCategory.PRIMITIVE; import static io.trino.plugin.iceberg.IcebergFileFormat.ORC; import static io.trino.plugin.iceberg.util.OrcTypeConverter.toOrcType; @@ -80,28 +77,28 @@ public class TestIcebergNodeLocalDynamicSplitPruning { - private static final String SCHEMA_NAME = "test"; - private static final String TABLE_NAME = "test"; - private static final Column KEY_COLUMN = new Column("a_integer", HIVE_INT, Optional.empty(), Map.of()); - private static final ColumnIdentity KEY_COLUMN_IDENTITY = new ColumnIdentity(1, KEY_COLUMN.getName(), PRIMITIVE, ImmutableList.of()); - private static final IcebergColumnHandle KEY_ICEBERG_COLUMN_HANDLE = new IcebergColumnHandle(KEY_COLUMN_IDENTITY, INTEGER, ImmutableList.of(), INTEGER, Optional.empty()); - private static final int KEY_COLUMN_VALUE = 42; - private static final Column DATA_COLUMN = new Column("a_varchar", HIVE_STRING, Optional.empty(), Map.of()); - private static final ColumnIdentity DATA_COLUMN_IDENTITY = new ColumnIdentity(2, DATA_COLUMN.getName(), PRIMITIVE, ImmutableList.of()); - private static final IcebergColumnHandle DATA_ICEBERG_COLUMN_HANDLE = new IcebergColumnHandle(DATA_COLUMN_IDENTITY, VARCHAR, ImmutableList.of(), VARCHAR, Optional.empty()); - private static final String DATA_COLUMN_VALUE = "hello world"; - private static final Schema TABLE_SCHEMA = new Schema( - optional(KEY_COLUMN_IDENTITY.getId(), KEY_COLUMN.getName(), Types.IntegerType.get()), - optional(DATA_COLUMN_IDENTITY.getId(), DATA_COLUMN.getName(), Types.StringType.get())); private static final OrcReaderConfig ORC_READER_CONFIG = new OrcReaderConfig(); private static final OrcWriterConfig ORC_WRITER_CONFIG = new OrcWriterConfig(); private static final ParquetReaderConfig PARQUET_READER_CONFIG = new ParquetReaderConfig(); private static final ParquetWriterConfig PARQUET_WRITER_CONFIG = new ParquetWriterConfig(); @Test - public void testDynamicSplitPruning() + public void testDynamicSplitPruningOnUnpartitionedTable() throws IOException { + String tableName = "unpartitioned_table"; + String keyColumnName = "a_integer"; + ColumnIdentity keyColumnIdentity = new ColumnIdentity(1, keyColumnName, PRIMITIVE, ImmutableList.of()); + IcebergColumnHandle keyColumnHandle = new IcebergColumnHandle(keyColumnIdentity, INTEGER, ImmutableList.of(), INTEGER, Optional.empty()); + int keyColumnValue = 42; + String dataColumnName = "a_varchar"; + ColumnIdentity dataColumnIdentity = new ColumnIdentity(2, dataColumnName, PRIMITIVE, ImmutableList.of()); + IcebergColumnHandle dataColumnHandle = new IcebergColumnHandle(dataColumnIdentity, VARCHAR, ImmutableList.of(), VARCHAR, Optional.empty()); + String dataColumnValue = "hello world"; + Schema tableSchema = new Schema( + optional(keyColumnIdentity.getId(), keyColumnName, Types.IntegerType.get()), + optional(dataColumnIdentity.getId(), dataColumnName, Types.StringType.get())); + IcebergConfig icebergConfig = new IcebergConfig(); HiveTransactionHandle transaction = new HiveTransactionHandle(false); try (TempFile file = new TempFile()) { @@ -109,87 +106,94 @@ public void testDynamicSplitPruning() TrinoOutputFile outputFile = new LocalOutputFile(file.file()); TrinoInputFile inputFile = new LocalInputFile(file.file()); - writeOrcContent(outputFile); + List columnNames = ImmutableList.of(keyColumnName, dataColumnName); + List types = ImmutableList.of(INTEGER, VARCHAR); + + try (OrcWriter writer = new OrcWriter( + OutputStreamOrcDataSink.create(outputFile), + columnNames, + types, + toOrcType(tableSchema), + NONE, + new OrcWriterOptions(), + ImmutableMap.of(), + true, + OrcWriteValidation.OrcWriteValidationMode.BOTH, + new OrcWriterStats())) { + BlockBuilder keyBuilder = INTEGER.createBlockBuilder(null, 1); + INTEGER.writeLong(keyBuilder, keyColumnValue); + BlockBuilder dataBuilder = VARCHAR.createBlockBuilder(null, 1); + VARCHAR.writeString(dataBuilder, dataColumnValue); + writer.write(new Page(keyBuilder.build(), dataBuilder.build())); + } + + IcebergSplit split = new IcebergSplit( + inputFile.toString(), + 0, + inputFile.length(), + inputFile.length(), + -1, // invalid; normally known + ORC, + PartitionSpecParser.toJson(PartitionSpec.unpartitioned()), + PartitionData.toJson(new PartitionData(new Object[] {})), + ImmutableList.of(), + SplitWeight.standard()); + + String tablePath = inputFile.location().fileName(); + TableHandle tableHandle = new TableHandle( + TEST_CATALOG_HANDLE, + new IcebergTableHandle( + CatalogHandle.fromId("iceberg:NORMAL:v12345"), + "test_schema", + tableName, + TableType.DATA, + Optional.empty(), + SchemaParser.toJson(tableSchema), + Optional.of(PartitionSpecParser.toJson(PartitionSpec.unpartitioned())), + 2, + TupleDomain.withColumnDomains(ImmutableMap.of(keyColumnHandle, Domain.singleValue(INTEGER, (long) keyColumnValue))), + TupleDomain.all(), + OptionalLong.empty(), + ImmutableSet.of(keyColumnHandle), + Optional.empty(), + tablePath, + ImmutableMap.of(), + false, + Optional.empty(), + ImmutableSet.of(), + Optional.of(false)), + transaction); - try (ConnectorPageSource emptyPageSource = createTestingPageSource(transaction, icebergConfig, inputFile, getDynamicFilter(getTupleDomainForSplitPruning()))) { + TupleDomain splitPruningPredicate = TupleDomain.withColumnDomains( + ImmutableMap.of( + keyColumnHandle, + Domain.singleValue(INTEGER, 1L))); + try (ConnectorPageSource emptyPageSource = createTestingPageSource(transaction, icebergConfig, split, tableHandle, ImmutableList.of(keyColumnHandle, dataColumnHandle), getDynamicFilter(splitPruningPredicate))) { assertThat(emptyPageSource.getNextPage()).isNull(); } - try (ConnectorPageSource nonEmptyPageSource = createTestingPageSource(transaction, icebergConfig, inputFile, getDynamicFilter(getNonSelectiveTupleDomain()))) { + TupleDomain nonSelectivePredicate = TupleDomain.withColumnDomains( + ImmutableMap.of( + keyColumnHandle, + Domain.singleValue(INTEGER, (long) keyColumnValue))); + try (ConnectorPageSource nonEmptyPageSource = createTestingPageSource(transaction, icebergConfig, split, tableHandle, ImmutableList.of(keyColumnHandle, dataColumnHandle), getDynamicFilter(nonSelectivePredicate))) { Page page = nonEmptyPageSource.getNextPage(); assertThat(page).isNotNull(); assertThat(page.getPositionCount()).isEqualTo(1); - assertThat(page.getBlock(0).getInt(0, 0)).isEqualTo(KEY_COLUMN_VALUE); - assertThat(page.getBlock(1).getSlice(0, 0, page.getBlock(1).getSliceLength(0)).toStringUtf8()).isEqualTo(DATA_COLUMN_VALUE); + assertThat(page.getBlock(0).getInt(0, 0)).isEqualTo(keyColumnValue); + assertThat(page.getBlock(1).getSlice(0, 0, page.getBlock(1).getSliceLength(0)).toStringUtf8()).isEqualTo(dataColumnValue); } } } - private static void writeOrcContent(TrinoOutputFile outputFile) - throws IOException - { - List columnNames = ImmutableList.of(KEY_COLUMN.getName(), DATA_COLUMN.getName()); - List types = ImmutableList.of(INTEGER, VARCHAR); - - try (OrcWriter writer = new OrcWriter( - OutputStreamOrcDataSink.create(outputFile), - columnNames, - types, - toOrcType(TABLE_SCHEMA), - NONE, - new OrcWriterOptions(), - ImmutableMap.of(), - true, - OrcWriteValidation.OrcWriteValidationMode.BOTH, - new OrcWriterStats())) { - BlockBuilder keyBuilder = INTEGER.createBlockBuilder(null, 1); - INTEGER.writeLong(keyBuilder, KEY_COLUMN_VALUE); - BlockBuilder dataBuilder = VARCHAR.createBlockBuilder(null, 1); - VARCHAR.writeString(dataBuilder, DATA_COLUMN_VALUE); - writer.write(new Page(keyBuilder.build(), dataBuilder.build())); - } - } - - private static ConnectorPageSource createTestingPageSource(HiveTransactionHandle transaction, IcebergConfig icebergConfig, TrinoInputFile inputFile, DynamicFilter dynamicFilter) - throws IOException + private static ConnectorPageSource createTestingPageSource( + HiveTransactionHandle transaction, + IcebergConfig icebergConfig, + IcebergSplit split, + TableHandle tableHandle, + List columns, + DynamicFilter dynamicFilter) { - IcebergSplit split = new IcebergSplit( - inputFile.toString(), - 0, - inputFile.length(), - inputFile.length(), - -1, // invalid; normally known - ORC, - PartitionSpecParser.toJson(PartitionSpec.unpartitioned()), - PartitionData.toJson(new PartitionData(new Object[] {})), - ImmutableList.of(), - SplitWeight.standard()); - - String tablePath = inputFile.location().fileName(); - TableHandle tableHandle = new TableHandle( - TEST_CATALOG_HANDLE, - new IcebergTableHandle( - CatalogHandle.fromId("iceberg:NORMAL:v12345"), - SCHEMA_NAME, - TABLE_NAME, - TableType.DATA, - Optional.empty(), - SchemaParser.toJson(TABLE_SCHEMA), - Optional.of(PartitionSpecParser.toJson(PartitionSpec.unpartitioned())), - 2, - TupleDomain.withColumnDomains(ImmutableMap.of(KEY_ICEBERG_COLUMN_HANDLE, Domain.singleValue(INTEGER, (long) KEY_COLUMN_VALUE))), - TupleDomain.all(), - OptionalLong.empty(), - ImmutableSet.of(KEY_ICEBERG_COLUMN_HANDLE), - Optional.empty(), - tablePath, - ImmutableMap.of(), - false, - Optional.empty(), - ImmutableSet.of(), - Optional.of(false)), - transaction); - FileFormatDataSourceStats stats = new FileFormatDataSourceStats(); IcebergPageSourceProvider provider = new IcebergPageSourceProvider( new HdfsFileSystemFactory(HDFS_ENVIRONMENT, HDFS_FILE_SYSTEM_STATS), @@ -203,26 +207,10 @@ private static ConnectorPageSource createTestingPageSource(HiveTransactionHandle getSession(icebergConfig), split, tableHandle.getConnectorHandle(), - ImmutableList.of(KEY_ICEBERG_COLUMN_HANDLE, DATA_ICEBERG_COLUMN_HANDLE), + columns, dynamicFilter); } - private static TupleDomain getTupleDomainForSplitPruning() - { - return TupleDomain.withColumnDomains( - ImmutableMap.of( - KEY_ICEBERG_COLUMN_HANDLE, - Domain.singleValue(INTEGER, 1L))); - } - - private static TupleDomain getNonSelectiveTupleDomain() - { - return TupleDomain.withColumnDomains( - ImmutableMap.of( - KEY_ICEBERG_COLUMN_HANDLE, - Domain.singleValue(INTEGER, (long) KEY_COLUMN_VALUE))); - } - private static TestingConnectorSession getSession(IcebergConfig icebergConfig) { return TestingConnectorSession.builder() From 89105bde15cd6ce0b5ad7e3b08f77e3e5ec27f69 Mon Sep 17 00:00:00 2001 From: Marius Grama Date: Fri, 22 Dec 2023 17:04:31 +0100 Subject: [PATCH 222/350] Short circuit page source in case of partition mismatch In case that the dynamic filter completes after scheduling of split on the worker, the results in the split will be getting pruned in the situation that there is a partition predicate mismatch. --- .../iceberg/IcebergPageSourceProvider.java | 33 +++- .../iceberg/BaseIcebergConnectorTest.java | 61 +++++++ ...stIcebergNodeLocalDynamicSplitPruning.java | 151 ++++++++++++++++++ 3 files changed, 241 insertions(+), 4 deletions(-) diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSourceProvider.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSourceProvider.java index 19bde2a570789..678c53282d635 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSourceProvider.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSourceProvider.java @@ -13,7 +13,6 @@ */ package io.trino.plugin.iceberg; -import com.google.common.base.Suppliers; import com.google.common.base.VerifyException; import com.google.common.collect.AbstractIterator; import com.google.common.collect.BiMap; @@ -133,6 +132,7 @@ import java.util.function.Supplier; import static com.google.common.base.Preconditions.checkState; +import static com.google.common.base.Suppliers.memoize; import static com.google.common.base.Verify.verify; import static com.google.common.collect.ImmutableList.toImmutableList; import static com.google.common.collect.ImmutableMap.toImmutableMap; @@ -170,9 +170,11 @@ import static io.trino.plugin.iceberg.IcebergSessionProperties.isUseFileSizeFromMetadata; import static io.trino.plugin.iceberg.IcebergSessionProperties.useParquetBloomFilter; import static io.trino.plugin.iceberg.IcebergSplitManager.ICEBERG_DOMAIN_COMPACTION_THRESHOLD; +import static io.trino.plugin.iceberg.IcebergSplitSource.partitionMatchesPredicate; import static io.trino.plugin.iceberg.IcebergUtil.deserializePartitionValue; import static io.trino.plugin.iceberg.IcebergUtil.getColumnHandle; import static io.trino.plugin.iceberg.IcebergUtil.getPartitionKeys; +import static io.trino.plugin.iceberg.IcebergUtil.getPartitionValues; import static io.trino.plugin.iceberg.IcebergUtil.schemaFromHandles; import static io.trino.plugin.iceberg.delete.EqualityDeleteFilter.readEqualityDeletes; import static io.trino.plugin.iceberg.delete.PositionDeleteFilter.readPositionDeletes; @@ -332,8 +334,11 @@ else if (identity.getId() == TRINO_MERGE_PARTITION_DATA) { } }); - TupleDomain effectivePredicate = unenforcedPredicate - .intersect(dynamicFilter.getCurrentPredicate().transformKeys(IcebergColumnHandle.class::cast)) + TupleDomain effectivePredicate = getEffectivePredicate( + tableSchema, + partitionKeys, + dynamicFilter.getCurrentPredicate().transformKeys(IcebergColumnHandle.class::cast), + unenforcedPredicate) .simplify(ICEBERG_DOMAIN_COMPACTION_THRESHOLD); if (effectivePredicate.isNone()) { return new EmptyPageSource(); @@ -386,7 +391,7 @@ else if (identity.getId() == TRINO_MERGE_PARTITION_DATA) { .map(readerColumns -> readerColumns.get().stream().map(IcebergColumnHandle.class::cast).collect(toList())) .orElse(requiredColumns); - Supplier> deletePredicate = Suppliers.memoize(() -> { + Supplier> deletePredicate = memoize(() -> { List deleteFilters = readDeletes( session, tableSchema, @@ -408,6 +413,26 @@ else if (identity.getId() == TRINO_MERGE_PARTITION_DATA) { deletePredicate); } + private TupleDomain getEffectivePredicate( + Schema tableSchema, + Map> partitionKeys, + TupleDomain dynamicFilterPredicate, + TupleDomain unenforcedPredicate) + { + if (dynamicFilterPredicate.isAll() || dynamicFilterPredicate.isNone() || partitionKeys.isEmpty()) { + return unenforcedPredicate.intersect(dynamicFilterPredicate); + } + Set partitionColumns = partitionKeys.keySet().stream() + .map(fieldId -> getColumnHandle(tableSchema.findField(fieldId), typeManager)) + .collect(toImmutableSet()); + Supplier> partitionValues = memoize(() -> getPartitionValues(partitionColumns, partitionKeys)); + if (!partitionMatchesPredicate(partitionColumns, partitionValues, dynamicFilterPredicate)) { + return TupleDomain.none(); + } + + return unenforcedPredicate.intersect(dynamicFilterPredicate); + } + private Set requiredColumnsForDeletes(Schema schema, List deletes) { ImmutableSet.Builder requiredColumns = ImmutableSet.builder(); diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java index 9fc269172197a..ff39da34da8bc 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java @@ -31,6 +31,7 @@ import io.trino.operator.OperatorStats; import io.trino.plugin.hive.TestingHivePlugin; import io.trino.plugin.iceberg.fileio.ForwardingFileIo; +import io.trino.server.DynamicFilterService; import io.trino.spi.QueryId; import io.trino.spi.connector.ColumnHandle; import io.trino.spi.connector.Constraint; @@ -101,6 +102,7 @@ import static com.google.common.collect.MoreCollectors.onlyElement; import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly; import static io.trino.SystemSessionProperties.DETERMINE_PARTITION_COUNT_FOR_WRITE_ENABLED; +import static io.trino.SystemSessionProperties.ENABLE_DYNAMIC_FILTERING; import static io.trino.SystemSessionProperties.SCALE_WRITERS; import static io.trino.SystemSessionProperties.TASK_MAX_WRITER_COUNT; import static io.trino.SystemSessionProperties.TASK_MIN_WRITER_COUNT; @@ -7353,6 +7355,65 @@ public void testUuidDynamicFilter() } } + @Test + public void testDynamicFilterWithExplicitPartitionFilter() + { + String catalog = getSession().getCatalog().orElseThrow(); + try (TestTable salesTable = new TestTable(getQueryRunner()::execute, "sales_table", "(date date, receipt_id varchar, amount decimal(10,2)) with (partitioning=array['date'])"); + TestTable dimensionTable = new TestTable(getQueryRunner()::execute, "dimension_table", "(date date, following_holiday boolean, year int)")) { + assertUpdate(""" + INSERT INTO %s + VALUES + (DATE '2023-01-01' , false, 2023), + (DATE '2023-01-02' , true, 2023), + (DATE '2023-01-03' , false, 2023)""".formatted(dimensionTable.getName()), 3); + assertUpdate(""" + INSERT INTO %s + VALUES + (DATE '2023-01-02' , '#2023#1', DECIMAL '122.12'), + (DATE '2023-01-02' , '#2023#2', DECIMAL '124.12'), + (DATE '2023-01-02' , '#2023#3', DECIMAL '99.99'), + (DATE '2023-01-02' , '#2023#4', DECIMAL '95.12'), + (DATE '2023-01-03' , '#2023#5', DECIMAL '199.12'), + (DATE '2023-01-04' , '#2023#6', DECIMAL '99.55'), + (DATE '2023-01-05' , '#2023#7', DECIMAL '50.11'), + (DATE '2023-01-05' , '#2023#8', DECIMAL '60.20'), + (DATE '2023-01-05' , '#2023#9', DECIMAL '70.75'), + (DATE '2023-01-05' , '#2023#10', DECIMAL '80.12')""".formatted(salesTable.getName()), 10); + + String selectQuery = """ + SELECT receipt_id + FROM %s s + JOIN %s d + ON s.date = d.date + WHERE + d.following_holiday = true AND + d.date BETWEEN DATE '2023-01-01' AND DATE '2024-01-01'""".formatted(salesTable.getName(), dimensionTable.getName()); + MaterializedResultWithQueryId result = getDistributedQueryRunner().executeWithQueryId( + Session.builder(getSession()) + .setCatalogSessionProperty(catalog, DYNAMIC_FILTERING_WAIT_TIMEOUT, "10s") + .build(), + selectQuery); + MaterializedResult expected = computeActual( + Session.builder(getSession()) + .setSystemProperty(ENABLE_DYNAMIC_FILTERING, "false") + .build(), + selectQuery); + assertEqualsIgnoreOrder(result.getResult(), expected); + + DynamicFilterService.DynamicFiltersStats dynamicFiltersStats = getDistributedQueryRunner().getCoordinator() + .getQueryManager() + .getFullQueryInfo(result.getQueryId()) + .getQueryStats() + .getDynamicFiltersStats(); + // The dynamic filter reduces the range specified for the partition column `date` from `date :: [[2023-01-01, 2024-01-01]]` to `date :: {[2023-01-02]}` + assertThat(dynamicFiltersStats.getTotalDynamicFilters()).isEqualTo(1L); + assertThat(dynamicFiltersStats.getLazyDynamicFilters()).isEqualTo(1L); + assertThat(dynamicFiltersStats.getReplicatedDynamicFilters()).isEqualTo(0L); + assertThat(dynamicFiltersStats.getDynamicFiltersCompleted()).isEqualTo(1L); + } + } + @Override protected void verifyTableNameLengthFailurePermissible(Throwable e) { diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergNodeLocalDynamicSplitPruning.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergNodeLocalDynamicSplitPruning.java index 1041cd6ade43d..5ca4ca56f427d 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergNodeLocalDynamicSplitPruning.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergNodeLocalDynamicSplitPruning.java @@ -42,7 +42,11 @@ import io.trino.spi.connector.ConnectorPageSource; import io.trino.spi.connector.DynamicFilter; import io.trino.spi.predicate.Domain; +import io.trino.spi.predicate.Range; import io.trino.spi.predicate.TupleDomain; +import io.trino.spi.predicate.ValueSet; +import io.trino.spi.type.DecimalType; +import io.trino.spi.type.SqlDecimal; import io.trino.spi.type.Type; import io.trino.testing.TestingConnectorSession; import org.apache.iceberg.PartitionSpec; @@ -53,7 +57,9 @@ import org.junit.jupiter.api.Test; import java.io.IOException; +import java.math.BigDecimal; import java.nio.file.Files; +import java.time.LocalDate; import java.util.List; import java.util.Map; import java.util.Optional; @@ -67,6 +73,8 @@ import static io.trino.plugin.iceberg.ColumnIdentity.TypeCategory.PRIMITIVE; import static io.trino.plugin.iceberg.IcebergFileFormat.ORC; import static io.trino.plugin.iceberg.util.OrcTypeConverter.toOrcType; +import static io.trino.spi.type.DateType.DATE; +import static io.trino.spi.type.Decimals.writeShortDecimal; import static io.trino.spi.type.IntegerType.INTEGER; import static io.trino.spi.type.VarcharType.VARCHAR; import static io.trino.testing.TestingHandles.TEST_CATALOG_HANDLE; @@ -186,6 +194,149 @@ public void testDynamicSplitPruningOnUnpartitionedTable() } } + @Test + public void testDynamicSplitPruningWithExplicitPartitionFilter() + throws IOException + { + String tableName = "sales_table"; + String dateColumnName = "date"; + ColumnIdentity dateColumnIdentity = new ColumnIdentity(1, dateColumnName, PRIMITIVE, ImmutableList.of()); + IcebergColumnHandle dateColumnHandle = new IcebergColumnHandle(dateColumnIdentity, DATE, ImmutableList.of(), DATE, Optional.empty()); + long dateColumnValue = LocalDate.of(2023, 1, 10).toEpochDay(); + String receiptColumnName = "receipt"; + ColumnIdentity receiptColumnIdentity = new ColumnIdentity(2, receiptColumnName, PRIMITIVE, ImmutableList.of()); + IcebergColumnHandle receiptColumnHandle = new IcebergColumnHandle(receiptColumnIdentity, VARCHAR, ImmutableList.of(), VARCHAR, Optional.empty()); + String receiptColumnValue = "#12345"; + String amountColumnName = "amount"; + ColumnIdentity amountColumnIdentity = new ColumnIdentity(3, amountColumnName, PRIMITIVE, ImmutableList.of()); + DecimalType amountColumnType = DecimalType.createDecimalType(10, 2); + IcebergColumnHandle amountColumnHandle = new IcebergColumnHandle(amountColumnIdentity, amountColumnType, ImmutableList.of(), amountColumnType, Optional.empty()); + BigDecimal amountColumnValue = new BigDecimal("1234567.65"); + Schema tableSchema = new Schema( + optional(dateColumnIdentity.getId(), dateColumnName, Types.DateType.get()), + optional(receiptColumnIdentity.getId(), receiptColumnName, Types.StringType.get()), + optional(amountColumnIdentity.getId(), amountColumnName, Types.DecimalType.of(10, 2))); + PartitionSpec partitionSpec = PartitionSpec.builderFor(tableSchema) + .identity(dateColumnName) + .build(); + + IcebergConfig icebergConfig = new IcebergConfig(); + HiveTransactionHandle transaction = new HiveTransactionHandle(false); + try (TempFile file = new TempFile()) { + Files.delete(file.path()); + + TrinoOutputFile outputFile = new LocalOutputFile(file.file()); + TrinoInputFile inputFile = new LocalInputFile(file.file()); + List columnNames = ImmutableList.of(dateColumnName, receiptColumnName, amountColumnName); + List types = ImmutableList.of(DATE, VARCHAR, amountColumnType); + + try (OrcWriter writer = new OrcWriter( + OutputStreamOrcDataSink.create(outputFile), + columnNames, + types, + toOrcType(tableSchema), + NONE, + new OrcWriterOptions(), + ImmutableMap.of(), + true, + OrcWriteValidation.OrcWriteValidationMode.BOTH, + new OrcWriterStats())) { + BlockBuilder dateBuilder = DATE.createBlockBuilder(null, 1); + DATE.writeLong(dateBuilder, dateColumnValue); + BlockBuilder receiptBuilder = VARCHAR.createBlockBuilder(null, 1); + VARCHAR.writeString(receiptBuilder, receiptColumnValue); + BlockBuilder amountBuilder = amountColumnType.createBlockBuilder(null, 1); + writeShortDecimal(amountBuilder, amountColumnValue.unscaledValue().longValueExact()); + writer.write(new Page(dateBuilder.build(), receiptBuilder.build(), amountBuilder.build())); + } + + IcebergSplit split = new IcebergSplit( + inputFile.toString(), + 0, + inputFile.length(), + inputFile.length(), + -1, // invalid; normally known + ORC, + PartitionSpecParser.toJson(partitionSpec), + PartitionData.toJson(new PartitionData(new Object[] {dateColumnValue})), + ImmutableList.of(), + SplitWeight.standard()); + + String tablePath = inputFile.location().fileName(); + TableHandle tableHandle = new TableHandle( + TEST_CATALOG_HANDLE, + new IcebergTableHandle( + CatalogHandle.fromId("iceberg:NORMAL:v12345"), + "test_schema", + tableName, + TableType.DATA, + Optional.empty(), + SchemaParser.toJson(tableSchema), + Optional.of(PartitionSpecParser.toJson(partitionSpec)), + 2, + TupleDomain.all(), + TupleDomain.all(), + OptionalLong.empty(), + ImmutableSet.of(dateColumnHandle), + Optional.empty(), + tablePath, + ImmutableMap.of(), + false, + Optional.empty(), + ImmutableSet.of(), + Optional.of(false)), + transaction); + + // Simulate situations where the dynamic filter (e.g.: while performing a JOIN with another table) reduces considerably + // the amount of data to be processed from the current table + + TupleDomain differentDatePredicate = TupleDomain.withColumnDomains( + ImmutableMap.of( + dateColumnHandle, + Domain.singleValue(DATE, LocalDate.of(2023, 2, 2).toEpochDay()))); + TupleDomain nonOverlappingDatePredicate = TupleDomain.withColumnDomains( + ImmutableMap.of( + dateColumnHandle, + Domain.create(ValueSet.ofRanges(Range.greaterThanOrEqual(DATE, LocalDate.of(2023, 2, 2).toEpochDay())), true))); + for (TupleDomain partitionPredicate : List.of(differentDatePredicate, nonOverlappingDatePredicate)) { + try (ConnectorPageSource emptyPageSource = createTestingPageSource( + transaction, + icebergConfig, + split, + tableHandle, + ImmutableList.of(dateColumnHandle, receiptColumnHandle, amountColumnHandle), + getDynamicFilter(partitionPredicate))) { + assertThat(emptyPageSource.getNextPage()).isNull(); + } + } + + TupleDomain sameDatePredicate = TupleDomain.withColumnDomains( + ImmutableMap.of( + dateColumnHandle, + Domain.singleValue(DATE, dateColumnValue))); + TupleDomain overlappingDatePredicate = TupleDomain.withColumnDomains( + ImmutableMap.of( + dateColumnHandle, + Domain.create(ValueSet.ofRanges(Range.range(DATE, LocalDate.of(2023, 1, 1).toEpochDay(), true, LocalDate.of(2023, 2, 1).toEpochDay(), false)), true))); + for (TupleDomain partitionPredicate : List.of(sameDatePredicate, overlappingDatePredicate)) { + try (ConnectorPageSource nonEmptyPageSource = createTestingPageSource( + transaction, + icebergConfig, + split, + tableHandle, + ImmutableList.of(dateColumnHandle, receiptColumnHandle, amountColumnHandle), + getDynamicFilter(partitionPredicate))) { + Page page = nonEmptyPageSource.getNextPage(); + assertThat(page).isNotNull(); + assertThat(page.getPositionCount()).isEqualTo(1); + assertThat(page.getBlock(0).getInt(0, 0)).isEqualTo(dateColumnValue); + assertThat(page.getBlock(1).getSlice(0, 0, page.getBlock(1).getSliceLength(0)).toStringUtf8()).isEqualTo(receiptColumnValue); + assertThat(((SqlDecimal) amountColumnType.getObjectValue(null, page.getBlock(2), 0)).toBigDecimal()).isEqualTo(amountColumnValue); + } + } + } + } + private static ConnectorPageSource createTestingPageSource( HiveTransactionHandle transaction, IcebergConfig icebergConfig, From b4f280392d09233264c657c74171c09ca0806553 Mon Sep 17 00:00:00 2001 From: Marius Grama Date: Wed, 27 Dec 2023 14:53:35 +0100 Subject: [PATCH 223/350] Strip the partition columns domains from the effective predicate --- .../io/trino/plugin/iceberg/IcebergPageSourceProvider.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSourceProvider.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSourceProvider.java index 678c53282d635..22cef9576b14b 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSourceProvider.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSourceProvider.java @@ -429,7 +429,9 @@ private TupleDomain getEffectivePredicate( if (!partitionMatchesPredicate(partitionColumns, partitionValues, dynamicFilterPredicate)) { return TupleDomain.none(); } - + // Filter out partition columns domains from the dynamic filter because they should be irrelevant at data file level + dynamicFilterPredicate = dynamicFilterPredicate + .filter((columnHandle, domain) -> !partitionKeys.containsKey(columnHandle.getId())); return unenforcedPredicate.intersect(dynamicFilterPredicate); } From e45ccafe25ea12a3f2406777c4be0c03e8372f3d Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 29 Dec 2023 14:23:23 +0100 Subject: [PATCH 224/350] Unignore re2j dependency conflicts `trino-re2j` is a fork of `com.google.re2j:re2j`. The forks diverged over time. It is not safe to ignore this dependency conflict e.g. because `com.google` version has `com.google.re2j.Pattern.matcher` accepting `CharSequence` or `byte[]` while Trino version accepts only `Slice`. They cannot be used interchangeably at all. --- client/trino-jdbc/pom.xml | 12 ++++++++++++ plugin/trino-geospatial/pom.xml | 12 ++++++++++++ plugin/trino-hudi/pom.xml | 12 ++++++++++++ plugin/trino-session-property-managers/pom.xml | 12 ++++++++++++ pom.xml | 15 --------------- 5 files changed, 48 insertions(+), 15 deletions(-) diff --git a/client/trino-jdbc/pom.xml b/client/trino-jdbc/pom.xml index fbc7fbae9a1ae..c5472b875de37 100644 --- a/client/trino-jdbc/pom.xml +++ b/client/trino-jdbc/pom.xml @@ -207,6 +207,12 @@ io.trino trino-main test + + + io.trino + re2j + + @@ -237,6 +243,12 @@ io.trino trino-testing test + + + io.trino + re2j + + diff --git a/plugin/trino-geospatial/pom.xml b/plugin/trino-geospatial/pom.xml index 7c6b237485296..b859355ed7a26 100644 --- a/plugin/trino-geospatial/pom.xml +++ b/plugin/trino-geospatial/pom.xml @@ -148,6 +148,12 @@ io.trino trino-main test + + + io.trino + re2j + + @@ -155,6 +161,12 @@ trino-main test-jar test + + + io.trino + re2j + + diff --git a/plugin/trino-hudi/pom.xml b/plugin/trino-hudi/pom.xml index fc5de619884ae..0f08bcd4cc63b 100644 --- a/plugin/trino-hudi/pom.xml +++ b/plugin/trino-hudi/pom.xml @@ -225,6 +225,12 @@ io.trino trino-main test + + + io.trino + re2j + + @@ -232,6 +238,12 @@ trino-main test-jar test + + + io.trino + re2j + + diff --git a/plugin/trino-session-property-managers/pom.xml b/plugin/trino-session-property-managers/pom.xml index a3c195e5c92f9..fb09231697036 100644 --- a/plugin/trino-session-property-managers/pom.xml +++ b/plugin/trino-session-property-managers/pom.xml @@ -180,12 +180,24 @@ io.trino trino-main test + + + io.trino + re2j + + io.trino trino-testing test + + + io.trino + re2j + + diff --git a/pom.xml b/pom.xml index 2284eccc29681..07857774af95b 100644 --- a/pom.xml +++ b/pom.xml @@ -2434,21 +2434,6 @@ android.annotation.TargetApi - - - - com.google.re2j - re2j - - - io.trino - re2j - - - - com.google.re2j - - From b293de3f830fe5309ca6d9577f5032d33c0a35a7 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 29 Dec 2023 14:45:46 +0100 Subject: [PATCH 225/350] Prune dependency management for alluxio-shaded-client Last use of this dependency was removed in 15cc6e58656d4c77b7f8460024d42c9f02e9a8d3, while the main use was removed probably in 59d48594d79c70548d8a89cc110741cdca5ab239. --- pom.xml | 83 --------------------------------------------------------- 1 file changed, 83 deletions(-) diff --git a/pom.xml b/pom.xml index 07857774af95b..8049973b331dc 100644 --- a/pom.xml +++ b/pom.xml @@ -1645,30 +1645,6 @@ 2.3 - - org.alluxio - alluxio-shaded-client - 2.9.3 - - - commons-logging - commons-logging - - - log4j - log4j - - - org.slf4j - slf4j-api - - - org.slf4j - slf4j-log4j12 - - - - org.antlr antlr4-runtime @@ -2400,65 +2376,6 @@ opencensus/proto/trace/v1/trace_config.proto - - - - io.grpc - grpc-services - - - org.alluxio - alluxio-shaded-client - - - - grpc/binlog/v1/binarylog.proto - grpc/health/v1/health.proto - grpc/reflection/v1alpha/reflection.proto - grpc/channelz/v1/channelz.proto - - - - - - com.google.android - annotations - - - org.alluxio - alluxio-shaded-client - - - - android.annotation.SuppressLint - android.annotation.TargetApi - - - - - - org.alluxio - alluxio-shaded-client - - - com.google.protobuf - protobuf-java - - - - google/protobuf/any.proto - google/protobuf/api.proto - google/protobuf/descriptor.proto - google/protobuf/duration.proto - google/protobuf/empty.proto - google/protobuf/field_mask.proto - google/protobuf/source_context.proto - google/protobuf/struct.proto - google/protobuf/timestamp.proto - google/protobuf/type.proto - google/protobuf/wrappers.proto - - From 8504cf732f3900f35543a9ec01aef0d36d622e21 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 29 Dec 2023 15:02:55 +0100 Subject: [PATCH 226/350] Put related modernizer exclusions together --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 8049973b331dc..753d9a0d9f6f8 100644 --- a/pom.xml +++ b/pom.xml @@ -2182,6 +2182,7 @@ com/google/common/collect/Iterables.getOnlyElement:(Ljava/lang/Iterable;)Ljava/lang/Object; + com/google/common/collect/Iterables.getOnlyElement:(Ljava/lang/Iterable;Ljava/lang/Object;)Ljava/lang/Object; com/google/common/collect/Iterables.getLast:(Ljava/lang/Iterable;)Ljava/lang/Object; com/google/common/collect/Iterables.getLast:(Ljava/lang/Iterable;Ljava/lang/Object;)Ljava/lang/Object; @@ -2202,7 +2203,6 @@ com/google/common/collect/Iterables.getLast:(Ljava/lang/Iterable;)Ljava/lang/Object; com/google/common/collect/Iterables.cycle:(Ljava/lang/Iterable;)Ljava/lang/Iterable; com/google/common/collect/Iterables.cycle:([Ljava/lang/Object;)Ljava/lang/Iterable; - com/google/common/collect/Iterables.getOnlyElement:(Ljava/lang/Iterable;Ljava/lang/Object;)Ljava/lang/Object; com/google/common/io/BaseEncoding.base64:()Lcom/google/common/io/BaseEncoding; From 875fe1b8a891e3293f0baf48cb4eea7b815b8b3d Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 29 Dec 2023 19:09:30 +0100 Subject: [PATCH 227/350] Remove unused test helper method --- .../src/test/java/io/trino/jdbc/BaseTestJdbcResultSet.java | 2 -- .../src/test/java/io/trino/jdbc/TestJdbcResultSet.java | 7 ------- .../io/trino/TestJdbcResultSetCompatibilityOldServer.java | 6 ------ 3 files changed, 15 deletions(-) diff --git a/client/trino-jdbc/src/test/java/io/trino/jdbc/BaseTestJdbcResultSet.java b/client/trino-jdbc/src/test/java/io/trino/jdbc/BaseTestJdbcResultSet.java index 776d4b50e395e..705b50a8e15d9 100644 --- a/client/trino-jdbc/src/test/java/io/trino/jdbc/BaseTestJdbcResultSet.java +++ b/client/trino-jdbc/src/test/java/io/trino/jdbc/BaseTestJdbcResultSet.java @@ -58,8 +58,6 @@ public abstract class BaseTestJdbcResultSet protected abstract Connection createConnection() throws SQLException; - protected abstract int getTestedServerVersion(); - @Test public void testDuplicateColumnLabels() throws Exception diff --git a/client/trino-jdbc/src/test/java/io/trino/jdbc/TestJdbcResultSet.java b/client/trino-jdbc/src/test/java/io/trino/jdbc/TestJdbcResultSet.java index 95c3378ed18cb..1a159b45ed55a 100644 --- a/client/trino-jdbc/src/test/java/io/trino/jdbc/TestJdbcResultSet.java +++ b/client/trino-jdbc/src/test/java/io/trino/jdbc/TestJdbcResultSet.java @@ -59,11 +59,4 @@ protected Connection createConnection() String url = format("jdbc:trino://%s", server.getAddress()); return DriverManager.getConnection(url, "test", null); } - - @Override - protected int getTestedServerVersion() - { - // Latest version - return Integer.MAX_VALUE; - } } diff --git a/testing/trino-test-jdbc-compatibility-old-server/src/test/java/io/trino/TestJdbcResultSetCompatibilityOldServer.java b/testing/trino-test-jdbc-compatibility-old-server/src/test/java/io/trino/TestJdbcResultSetCompatibilityOldServer.java index 3adc54e4ced0b..c4c5047550de0 100644 --- a/testing/trino-test-jdbc-compatibility-old-server/src/test/java/io/trino/TestJdbcResultSetCompatibilityOldServer.java +++ b/testing/trino-test-jdbc-compatibility-old-server/src/test/java/io/trino/TestJdbcResultSetCompatibilityOldServer.java @@ -139,12 +139,6 @@ protected Connection createConnection() return DriverManager.getConnection(trinoContainer.getJdbcUrl(), "test", null); } - @Override - protected int getTestedServerVersion() - { - return parseInt(getTestedTrinoVersion()); - } - @Override public String toString() { From 71499d531e61f1ab73ac896ff709d83588fb923d Mon Sep 17 00:00:00 2001 From: XuPengfei-1020 Date: Wed, 20 Dec 2023 15:55:26 +0800 Subject: [PATCH 228/350] Mark wether a PlanFragement is contains TableScanNode or not There may be a little bit performance issue while response for api of cluster stats, if cluster has huge amounts of tasks. Most of time was spent on distinguish the type of fragements temporarily. --- .../main/java/io/trino/execution/QueryStateMachine.java | 3 +-- .../main/java/io/trino/execution/StageStateMachine.java | 3 +-- .../src/main/java/io/trino/sql/planner/PlanFragment.java | 9 +++++++++ 3 files changed, 11 insertions(+), 4 deletions(-) diff --git a/core/trino-main/src/main/java/io/trino/execution/QueryStateMachine.java b/core/trino-main/src/main/java/io/trino/execution/QueryStateMachine.java index d83d74f074dc0..e5b6052855a72 100644 --- a/core/trino-main/src/main/java/io/trino/execution/QueryStateMachine.java +++ b/core/trino-main/src/main/java/io/trino/execution/QueryStateMachine.java @@ -54,7 +54,6 @@ import io.trino.spi.type.Type; import io.trino.sql.analyzer.Output; import io.trino.sql.planner.PlanFragment; -import io.trino.sql.planner.plan.TableScanNode; import io.trino.tracing.TrinoAttributes; import io.trino.transaction.TransactionId; import io.trino.transaction.TransactionInfo; @@ -669,7 +668,7 @@ private QueryStats getQueryStats(Optional rootStage, List failedInternalNetworkInputPositions += stageStats.getFailedInternalNetworkInputPositions(); PlanFragment plan = stageInfo.getPlan(); - if (plan != null && plan.getPartitionedSourceNodes().stream().anyMatch(TableScanNode.class::isInstance)) { + if (plan != null && plan.containsTableScanNode()) { rawInputDataSize += stageStats.getRawInputDataSize().toBytes(); failedRawInputDataSize += stageStats.getFailedRawInputDataSize().toBytes(); rawInputPositions += stageStats.getRawInputPositions(); diff --git a/core/trino-main/src/main/java/io/trino/execution/StageStateMachine.java b/core/trino-main/src/main/java/io/trino/execution/StageStateMachine.java index 2ddc4a6c3a9e0..5205c42943428 100644 --- a/core/trino-main/src/main/java/io/trino/execution/StageStateMachine.java +++ b/core/trino-main/src/main/java/io/trino/execution/StageStateMachine.java @@ -33,7 +33,6 @@ import io.trino.spi.eventlistener.StageGcStatistics; import io.trino.sql.planner.PlanFragment; import io.trino.sql.planner.plan.PlanNodeId; -import io.trino.sql.planner.plan.TableScanNode; import io.trino.tracing.TrinoAttributes; import io.trino.util.Failures; import it.unimi.dsi.fastutil.longs.Long2ObjectOpenHashMap; @@ -338,7 +337,7 @@ public BasicStageStats getBasicStageStats(Supplier> taskInfos internalNetworkInputDataSize += taskStats.getInternalNetworkInputDataSize().toBytes(); internalNetworkInputPositions += taskStats.getInternalNetworkInputPositions(); - if (fragment.getPartitionedSourceNodes().stream().anyMatch(TableScanNode.class::isInstance)) { + if (fragment.containsTableScanNode()) { rawInputDataSize += taskStats.getRawInputDataSize().toBytes(); rawInputPositions += taskStats.getRawInputPositions(); } diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/PlanFragment.java b/core/trino-main/src/main/java/io/trino/sql/planner/PlanFragment.java index 42db56e5812ca..226149f770e4a 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/PlanFragment.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/PlanFragment.java @@ -26,6 +26,7 @@ import io.trino.sql.planner.plan.PlanNode; import io.trino.sql.planner.plan.PlanNodeId; import io.trino.sql.planner.plan.RemoteSourceNode; +import io.trino.sql.planner.plan.TableScanNode; import java.util.List; import java.util.Map; @@ -55,6 +56,7 @@ public class PlanFragment private final List activeCatalogs; private final List languageFunctions; private final Optional jsonRepresentation; + private final boolean containsTableScanNode; // Only for creating instances without the JSON representation embedded private PlanFragment( @@ -88,6 +90,7 @@ private PlanFragment( this.activeCatalogs = requireNonNull(activeCatalogs, "activeCatalogs is null"); this.languageFunctions = requireNonNull(languageFunctions, "languageFunctions is null"); this.jsonRepresentation = Optional.empty(); + this.containsTableScanNode = partitionedSourceNodes.stream().anyMatch(TableScanNode.class::isInstance); } @JsonCreator @@ -135,6 +138,7 @@ public PlanFragment( this.remoteSourceNodes = remoteSourceNodes.build(); this.outputPartitioningScheme = requireNonNull(outputPartitioningScheme, "partitioningScheme is null"); + this.containsTableScanNode = partitionedSourceNodes.stream().anyMatch(TableScanNode.class::isInstance); } @JsonProperty @@ -372,4 +376,9 @@ public PlanFragment withActiveCatalogs(List activeCatalogs) this.languageFunctions, this.jsonRepresentation); } + + public boolean containsTableScanNode() + { + return containsTableScanNode; + } } From 4155e9fdf031fc3d1f59a89d2b52bae6c251dab5 Mon Sep 17 00:00:00 2001 From: Chenren Shao <9207574+cshao239@users.noreply.github.com> Date: Thu, 28 Dec 2023 13:22:59 -0500 Subject: [PATCH 229/350] fix ST_Centroid and ST_Buffer for tiny geometries backporting from https://github.com/prestodb/presto/pull/13323/files --- docs/src/main/sphinx/functions/geospatial.md | 3 +- .../trino/plugin/geospatial/GeoFunctions.java | 134 +----------------- .../geospatial/TestBingTileFunctions.java | 2 +- .../plugin/geospatial/TestGeoFunctions.java | 44 ++++++ 4 files changed, 48 insertions(+), 135 deletions(-) diff --git a/docs/src/main/sphinx/functions/geospatial.md b/docs/src/main/sphinx/functions/geospatial.md index 448bc8f2e2027..19473464ada1e 100644 --- a/docs/src/main/sphinx/functions/geospatial.md +++ b/docs/src/main/sphinx/functions/geospatial.md @@ -174,7 +174,8 @@ Returns the closure of the combinatorial boundary of this geometry. :::{function} ST_Buffer(Geometry, distance) -> Geometry Returns the geometry that represents all points whose distance from the specified geometry -is less than or equal to the specified distance. +is less than or equal to the specified distance. If the points of the geometry are extremely +close together (``delta < 1e-8``), this might return an empty geometry. ::: :::{function} ST_Difference(Geometry, Geometry) -> Geometry diff --git a/plugin/trino-geospatial/src/main/java/io/trino/plugin/geospatial/GeoFunctions.java b/plugin/trino-geospatial/src/main/java/io/trino/plugin/geospatial/GeoFunctions.java index cd36a36e00445..74a7fad720cfa 100644 --- a/plugin/trino-geospatial/src/main/java/io/trino/plugin/geospatial/GeoFunctions.java +++ b/plugin/trino-geospatial/src/main/java/io/trino/plugin/geospatial/GeoFunctions.java @@ -33,7 +33,6 @@ import com.esri.core.geometry.ogc.OGCGeometry; import com.esri.core.geometry.ogc.OGCGeometryCollection; import com.esri.core.geometry.ogc.OGCLineString; -import com.esri.core.geometry.ogc.OGCMultiPolygon; import com.esri.core.geometry.ogc.OGCPoint; import com.esri.core.geometry.ogc.OGCPolygon; import com.google.common.base.Joiner; @@ -433,25 +432,7 @@ public static Slice stCentroid(@SqlType(GEOMETRY_TYPE_NAME) Slice input) return serialize(createFromEsriGeometry(new Point(), geometry.getEsriSpatialReference())); } - Point centroid; - switch (geometryType) { - case MULTI_POINT: - centroid = computePointsCentroid((MultiVertexGeometry) geometry.getEsriGeometry()); - break; - case LINE_STRING: - case MULTI_LINE_STRING: - centroid = computeLineCentroid((Polyline) geometry.getEsriGeometry()); - break; - case POLYGON: - centroid = computePolygonCentroid((Polygon) geometry.getEsriGeometry()); - break; - case MULTI_POLYGON: - centroid = computeMultiPolygonCentroid((OGCMultiPolygon) geometry); - break; - default: - throw new TrinoException(INVALID_FUNCTION_ARGUMENT, "Unexpected geometry type: " + geometryType); - } - return serialize(createFromEsriGeometry(centroid, geometry.getEsriSpatialReference())); + return serialize(geometry.centroid()); } @Description("Returns the minimum convex geometry that encloses all input geometries") @@ -1609,119 +1590,6 @@ private static void verifySameSpatialReference(OGCGeometry leftGeometry, OGCGeom checkArgument(Objects.equals(leftGeometry.getEsriSpatialReference(), rightGeometry.getEsriSpatialReference()), "Input geometries must have the same spatial reference"); } - // Points centroid is arithmetic mean of the input points - private static Point computePointsCentroid(MultiVertexGeometry multiVertex) - { - double xSum = 0; - double ySum = 0; - for (int i = 0; i < multiVertex.getPointCount(); i++) { - Point point = multiVertex.getPoint(i); - xSum += point.getX(); - ySum += point.getY(); - } - return new Point(xSum / multiVertex.getPointCount(), ySum / multiVertex.getPointCount()); - } - - // Lines centroid is weighted mean of each line segment, weight in terms of line length - private static Point computeLineCentroid(Polyline polyline) - { - double xSum = 0; - double ySum = 0; - double weightSum = 0; - for (int i = 0; i < polyline.getPathCount(); i++) { - Point startPoint = polyline.getPoint(polyline.getPathStart(i)); - Point endPoint = polyline.getPoint(polyline.getPathEnd(i) - 1); - double dx = endPoint.getX() - startPoint.getX(); - double dy = endPoint.getY() - startPoint.getY(); - double length = sqrt(dx * dx + dy * dy); - weightSum += length; - xSum += (startPoint.getX() + endPoint.getX()) * length / 2; - ySum += (startPoint.getY() + endPoint.getY()) * length / 2; - } - return new Point(xSum / weightSum, ySum / weightSum); - } - - // Polygon centroid: area weighted average of centroids in case of holes - private static Point computePolygonCentroid(Polygon polygon) - { - int pathCount = polygon.getPathCount(); - - if (pathCount == 1) { - return getPolygonSansHolesCentroid(polygon); - } - - double xSum = 0; - double ySum = 0; - double areaSum = 0; - - for (int i = 0; i < pathCount; i++) { - int startIndex = polygon.getPathStart(i); - int endIndex = polygon.getPathEnd(i); - - Polygon sansHoles = getSubPolygon(polygon, startIndex, endIndex); - - Point centroid = getPolygonSansHolesCentroid(sansHoles); - double area = sansHoles.calculateArea2D(); - - xSum += centroid.getX() * area; - ySum += centroid.getY() * area; - areaSum += area; - } - - return new Point(xSum / areaSum, ySum / areaSum); - } - - private static Polygon getSubPolygon(Polygon polygon, int startIndex, int endIndex) - { - Polyline boundary = new Polyline(); - boundary.startPath(polygon.getPoint(startIndex)); - for (int i = startIndex + 1; i < endIndex; i++) { - Point current = polygon.getPoint(i); - boundary.lineTo(current); - } - - Polygon newPolygon = new Polygon(); - newPolygon.add(boundary, false); - return newPolygon; - } - - // Polygon sans holes centroid: - // c[x] = (Sigma(x[i] + x[i + 1]) * (x[i] * y[i + 1] - x[i + 1] * y[i]), for i = 0 to N - 1) / (6 * signedArea) - // c[y] = (Sigma(y[i] + y[i + 1]) * (x[i] * y[i + 1] - x[i + 1] * y[i]), for i = 0 to N - 1) / (6 * signedArea) - private static Point getPolygonSansHolesCentroid(Polygon polygon) - { - int pointCount = polygon.getPointCount(); - double xSum = 0; - double ySum = 0; - double signedArea = 0; - for (int i = 0; i < pointCount; i++) { - Point current = polygon.getPoint(i); - Point next = polygon.getPoint((i + 1) % polygon.getPointCount()); - double ladder = current.getX() * next.getY() - next.getX() * current.getY(); - xSum += (current.getX() + next.getX()) * ladder; - ySum += (current.getY() + next.getY()) * ladder; - signedArea += ladder / 2; - } - return new Point(xSum / (signedArea * 6), ySum / (signedArea * 6)); - } - - // MultiPolygon centroid is weighted mean of each polygon, weight in terms of polygon area - private static Point computeMultiPolygonCentroid(OGCMultiPolygon multiPolygon) - { - double xSum = 0; - double ySum = 0; - double weightSum = 0; - for (int i = 0; i < multiPolygon.numGeometries(); i++) { - Point centroid = computePolygonCentroid((Polygon) multiPolygon.geometryN(i).getEsriGeometry()); - Polygon polygon = (Polygon) multiPolygon.geometryN(i).getEsriGeometry(); - double weight = polygon.calculateArea2D(); - weightSum += weight; - xSum += centroid.getX() * weight; - ySum += centroid.getY() * weight; - } - return new Point(xSum / weightSum, ySum / weightSum); - } - private static boolean envelopes(Slice left, Slice right, EnvelopesPredicate predicate) { Envelope leftEnvelope = deserializeEnvelope(left); diff --git a/plugin/trino-geospatial/src/test/java/io/trino/plugin/geospatial/TestBingTileFunctions.java b/plugin/trino-geospatial/src/test/java/io/trino/plugin/geospatial/TestBingTileFunctions.java index e38fe7fde0725..4044d3610577c 100644 --- a/plugin/trino-geospatial/src/test/java/io/trino/plugin/geospatial/TestBingTileFunctions.java +++ b/plugin/trino-geospatial/src/test/java/io/trino/plugin/geospatial/TestBingTileFunctions.java @@ -454,7 +454,7 @@ public void testBingTilePolygon() assertThat(assertions.function("ST_AsText", "ST_Centroid(bing_tile_polygon(bing_tile('123030123010121')))")) .hasType(VARCHAR) - .isEqualTo("POINT (60.0018310442288 30.121372968273892)"); + .isEqualTo("POINT (60.0018310546875 30.121372973521975)"); // Check bottom right corner of a stack of tiles at different zoom levels assertThat(assertions.function("ST_AsText", "apply(bing_tile_polygon(bing_tile(1, 1, 1)), g -> ST_Point(ST_XMax(g), ST_YMin(g)))")) diff --git a/plugin/trino-geospatial/src/test/java/io/trino/plugin/geospatial/TestGeoFunctions.java b/plugin/trino-geospatial/src/test/java/io/trino/plugin/geospatial/TestGeoFunctions.java index 96e7b57653161..bd86208503502 100644 --- a/plugin/trino-geospatial/src/test/java/io/trino/plugin/geospatial/TestGeoFunctions.java +++ b/plugin/trino-geospatial/src/test/java/io/trino/plugin/geospatial/TestGeoFunctions.java @@ -14,10 +14,12 @@ package io.trino.plugin.geospatial; import com.esri.core.geometry.Point; +import com.esri.core.geometry.ogc.OGCGeometry; import com.esri.core.geometry.ogc.OGCPoint; import com.google.common.collect.ImmutableList; import io.trino.geospatial.KdbTreeUtils; import io.trino.geospatial.Rectangle; +import io.trino.geospatial.serde.GeometrySerde; import io.trino.spi.block.Block; import io.trino.spi.block.BlockBuilder; import io.trino.spi.type.ArrayType; @@ -35,6 +37,7 @@ import static com.google.common.collect.ImmutableList.toImmutableList; import static io.trino.geospatial.KdbTree.buildKdbTree; +import static io.trino.plugin.geospatial.GeoFunctions.stCentroid; import static io.trino.plugin.geospatial.GeometryType.GEOMETRY; import static io.trino.spi.type.BooleanType.BOOLEAN; import static io.trino.spi.type.DoubleType.DOUBLE; @@ -42,6 +45,7 @@ import static io.trino.spi.type.VarcharType.VARCHAR; import static io.trino.testing.assertions.TrinoExceptionAssert.assertTrinoExceptionThrownBy; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; @@ -259,6 +263,19 @@ public void testSTBuffer() assertTrinoExceptionThrownBy(assertions.function("ST_Buffer", "ST_Point(0, 0)", "nan()")::evaluate) .hasMessage("distance is NaN"); + + // For small polygons, there was a bug in ESRI that throw an NPE. This + // was fixed (https://github.com/Esri/geometry-api-java/pull/243) to + // return an empty geometry instead. Ideally, these would return + // something approximately like `ST_Buffer(ST_Centroid(geometry))`. + assertThat(assertions.function("ST_IsEmpty", "ST_Buffer(ST_Buffer(ST_Point(177.50102959662, 64.726807421691), 0.0000000001), 0.00005)")) + .hasType(BOOLEAN) + .isEqualTo(true); + + assertThat(assertions.function("ST_IsEmpty", "ST_Buffer(ST_GeometryFromText(" + + "'POLYGON ((177.0 64.0, 177.0000000001 64.0, 177.0000000001 64.0000000001, 177.0 64.0000000001, 177.0 64.0))'), 0.01)")) + .hasType(BOOLEAN) + .isEqualTo(true); } @Test @@ -299,6 +316,33 @@ public void testSTCentroid() assertThat(assertions.function("ST_AsText", "ST_Centroid(ST_GeometryFromText('POLYGON ((0 0, 0 5, 5 5, 5 0, 0 0), (1 1, 1 2, 2 2, 2 1, 1 1))'))")) .hasType(VARCHAR) .isEqualTo("POINT (2.5416666666666665 2.5416666666666665)"); + + assertApproximateCentroid("MULTIPOLYGON (((4.903234300000006 52.08474289999999, 4.903234265193165 52.084742934806826, 4.903234299999999 52.08474289999999, 4.903234300000006 52.08474289999999)))", new Point(4.9032343, 52.0847429), 1e-7); + + // Numerical stability tests + assertApproximateCentroid( + "MULTIPOLYGON (((153.492818 -28.13729, 153.492821 -28.137291, 153.492816 -28.137289, 153.492818 -28.13729)))", + new Point(153.49282, -28.13729), 1e-5); + assertApproximateCentroid( + "MULTIPOLYGON (((153.112475 -28.360526, 153.1124759 -28.360527, 153.1124759 -28.360526, 153.112475 -28.360526)))", + new Point(153.112475, -28.360526), 1e-5); + assertApproximateCentroid( + "POLYGON ((4.903234300000006 52.08474289999999, 4.903234265193165 52.084742934806826, 4.903234299999999 52.08474289999999, 4.903234300000006 52.08474289999999))", + new Point(4.9032343, 52.0847429), 1e-6); + assertApproximateCentroid( + "MULTIPOLYGON (((4.903234300000006 52.08474289999999, 4.903234265193165 52.084742934806826, 4.903234299999999 52.08474289999999, 4.903234300000006 52.08474289999999)))", + new Point(4.9032343, 52.0847429), 1e-6); + assertApproximateCentroid( + "POLYGON ((-81.0387349 29.20822, -81.039974 29.210597, -81.0410331 29.2101579, -81.0404758 29.2090879, -81.0404618 29.2090609, -81.040433 29.209005, -81.0404269 29.208993, -81.0404161 29.2089729, -81.0398001 29.20779, -81.0387349 29.20822), (-81.0404229 29.208986, -81.04042 29.2089809, -81.0404269 29.208993, -81.0404229 29.208986))", + new Point(-81.039885, 29.209191), 1e-6); + } + + private void assertApproximateCentroid(String wkt, Point expectedCentroid, double epsilon) + { + OGCPoint actualCentroid = (OGCPoint) GeometrySerde.deserialize( + stCentroid(GeometrySerde.serialize(OGCGeometry.fromText(wkt)))); + assertEquals(actualCentroid.X(), expectedCentroid.getX(), epsilon); + assertEquals(actualCentroid.Y(), expectedCentroid.getY(), epsilon); } @Test From e0c0c01b2aa32826a4d191063d23e087ade86b95 Mon Sep 17 00:00:00 2001 From: Marius Grama Date: Tue, 2 Jan 2024 14:46:56 +0100 Subject: [PATCH 230/350] Use the effective predicate when doing partition matching Use the effective predicate instead of the dynamic filter predicate to check for partition matching. This change results in short circuiting the page source and not having to read anymore the data file footer in the exotic case where a partition filter acts as unenforced predicate due to table partition spec evolution. --- .../iceberg/IcebergPageSourceProvider.java | 8 +- ...stIcebergNodeLocalDynamicSplitPruning.java | 163 ++++++++++++++++++ 2 files changed, 167 insertions(+), 4 deletions(-) diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSourceProvider.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSourceProvider.java index 22cef9576b14b..9476836cd454b 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSourceProvider.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSourceProvider.java @@ -419,20 +419,20 @@ private TupleDomain getEffectivePredicate( TupleDomain dynamicFilterPredicate, TupleDomain unenforcedPredicate) { + TupleDomain effectivePredicate = unenforcedPredicate.intersect(dynamicFilterPredicate); if (dynamicFilterPredicate.isAll() || dynamicFilterPredicate.isNone() || partitionKeys.isEmpty()) { - return unenforcedPredicate.intersect(dynamicFilterPredicate); + return effectivePredicate; } Set partitionColumns = partitionKeys.keySet().stream() .map(fieldId -> getColumnHandle(tableSchema.findField(fieldId), typeManager)) .collect(toImmutableSet()); Supplier> partitionValues = memoize(() -> getPartitionValues(partitionColumns, partitionKeys)); - if (!partitionMatchesPredicate(partitionColumns, partitionValues, dynamicFilterPredicate)) { + if (!partitionMatchesPredicate(partitionColumns, partitionValues, effectivePredicate)) { return TupleDomain.none(); } // Filter out partition columns domains from the dynamic filter because they should be irrelevant at data file level - dynamicFilterPredicate = dynamicFilterPredicate + return effectivePredicate .filter((columnHandle, domain) -> !partitionKeys.containsKey(columnHandle.getId())); - return unenforcedPredicate.intersect(dynamicFilterPredicate); } private Set requiredColumnsForDeletes(Schema schema, List deletes) diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergNodeLocalDynamicSplitPruning.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergNodeLocalDynamicSplitPruning.java index 5ca4ca56f427d..5657b21b840c9 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergNodeLocalDynamicSplitPruning.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergNodeLocalDynamicSplitPruning.java @@ -337,6 +337,169 @@ public void testDynamicSplitPruningWithExplicitPartitionFilter() } } + @Test + public void testDynamicSplitPruningWithExplicitPartitionFilterPartitionEvolution() + throws IOException + { + String tableName = "sales_table"; + String yearColumnName = "year"; + ColumnIdentity yearColumnIdentity = new ColumnIdentity(1, yearColumnName, PRIMITIVE, ImmutableList.of()); + IcebergColumnHandle yearColumnHandle = new IcebergColumnHandle(yearColumnIdentity, INTEGER, ImmutableList.of(), INTEGER, Optional.empty()); + long yearColumnValue = 2023L; + String monthColumnName = "month"; + ColumnIdentity monthColumnIdentity = new ColumnIdentity(2, monthColumnName, PRIMITIVE, ImmutableList.of()); + IcebergColumnHandle monthColumnHandle = new IcebergColumnHandle(monthColumnIdentity, INTEGER, ImmutableList.of(), INTEGER, Optional.empty()); + long monthColumnValue = 1L; + String receiptColumnName = "receipt"; + ColumnIdentity receiptColumnIdentity = new ColumnIdentity(3, receiptColumnName, PRIMITIVE, ImmutableList.of()); + IcebergColumnHandle receiptColumnHandle = new IcebergColumnHandle(receiptColumnIdentity, VARCHAR, ImmutableList.of(), VARCHAR, Optional.empty()); + String receiptColumnValue = "#12345"; + String amountColumnName = "amount"; + ColumnIdentity amountColumnIdentity = new ColumnIdentity(4, amountColumnName, PRIMITIVE, ImmutableList.of()); + DecimalType amountColumnType = DecimalType.createDecimalType(10, 2); + IcebergColumnHandle amountColumnHandle = new IcebergColumnHandle(amountColumnIdentity, amountColumnType, ImmutableList.of(), amountColumnType, Optional.empty()); + BigDecimal amountColumnValue = new BigDecimal("1234567.65"); + Schema tableSchema = new Schema( + optional(yearColumnIdentity.getId(), yearColumnName, Types.IntegerType.get()), + optional(monthColumnIdentity.getId(), monthColumnName, Types.IntegerType.get()), + optional(receiptColumnIdentity.getId(), receiptColumnName, Types.StringType.get()), + optional(amountColumnIdentity.getId(), amountColumnName, Types.DecimalType.of(10, 2))); + PartitionSpec partitionSpec = PartitionSpec.builderFor(tableSchema) + .identity(yearColumnName) + .build(); + + IcebergConfig icebergConfig = new IcebergConfig(); + HiveTransactionHandle transaction = new HiveTransactionHandle(false); + try (TempFile file = new TempFile()) { + Files.delete(file.path()); + + TrinoOutputFile outputFile = new LocalOutputFile(file.file()); + TrinoInputFile inputFile = new LocalInputFile(file.file()); + List columnNames = ImmutableList.of(yearColumnName, monthColumnName, receiptColumnName, amountColumnName); + List types = ImmutableList.of(INTEGER, INTEGER, VARCHAR, amountColumnType); + + try (OrcWriter writer = new OrcWriter( + OutputStreamOrcDataSink.create(outputFile), + columnNames, + types, + toOrcType(tableSchema), + NONE, + new OrcWriterOptions(), + ImmutableMap.of(), + true, + OrcWriteValidation.OrcWriteValidationMode.BOTH, + new OrcWriterStats())) { + BlockBuilder yearBuilder = INTEGER.createBlockBuilder(null, 1); + INTEGER.writeLong(yearBuilder, yearColumnValue); + BlockBuilder monthBuilder = INTEGER.createBlockBuilder(null, 1); + INTEGER.writeLong(monthBuilder, monthColumnValue); + BlockBuilder receiptBuilder = VARCHAR.createBlockBuilder(null, 1); + VARCHAR.writeString(receiptBuilder, receiptColumnValue); + BlockBuilder amountBuilder = amountColumnType.createBlockBuilder(null, 1); + writeShortDecimal(amountBuilder, amountColumnValue.unscaledValue().longValueExact()); + writer.write(new Page(yearBuilder.build(), monthBuilder.build(), receiptBuilder.build(), amountBuilder.build())); + } + + IcebergSplit split = new IcebergSplit( + inputFile.toString(), + 0, + inputFile.length(), + inputFile.length(), + -1, // invalid; normally known + ORC, + PartitionSpecParser.toJson(partitionSpec), + PartitionData.toJson(new PartitionData(new Object[] {yearColumnValue})), + ImmutableList.of(), + SplitWeight.standard()); + + String tablePath = inputFile.location().fileName(); + // Simulate the situation where `month` column is added at a later phase as partitioning column + // in addition to the `year` column, which leads to use it as unenforced predicate in the table handle + // after applying the filter + TableHandle tableHandle = new TableHandle( + TEST_CATALOG_HANDLE, + new IcebergTableHandle( + CatalogHandle.fromId("iceberg:NORMAL:v12345"), + "test_schema", + tableName, + TableType.DATA, + Optional.empty(), + SchemaParser.toJson(tableSchema), + Optional.of(PartitionSpecParser.toJson(partitionSpec)), + 2, + TupleDomain.withColumnDomains( + ImmutableMap.of( + yearColumnHandle, + Domain.create(ValueSet.ofRanges(Range.range(INTEGER, 2023L, true, 2024L, true)), true))), + TupleDomain.withColumnDomains( + ImmutableMap.of( + monthColumnHandle, + Domain.create(ValueSet.ofRanges(Range.range(INTEGER, 1L, true, 12L, true)), true))), + OptionalLong.empty(), + ImmutableSet.of(yearColumnHandle, monthColumnHandle, receiptColumnHandle, amountColumnHandle), + Optional.empty(), + tablePath, + ImmutableMap.of(), + false, + Optional.empty(), + ImmutableSet.of(), + Optional.of(false)), + transaction); + + // Simulate situations where the dynamic filter (e.g.: while performing a JOIN with another table) reduces considerably + // the amount of data to be processed from the current table + TupleDomain differentYearPredicate = TupleDomain.withColumnDomains( + ImmutableMap.of( + yearColumnHandle, + Domain.singleValue(INTEGER, 2024L))); + TupleDomain sameYearAndDifferentMonthPredicate = TupleDomain.withColumnDomains( + ImmutableMap.of( + yearColumnHandle, + Domain.singleValue(INTEGER, 2023L), + monthColumnHandle, + Domain.singleValue(INTEGER, 2L))); + for (TupleDomain partitionPredicate : List.of(differentYearPredicate, sameYearAndDifferentMonthPredicate)) { + try (ConnectorPageSource emptyPageSource = createTestingPageSource( + transaction, + icebergConfig, + split, + tableHandle, + ImmutableList.of(yearColumnHandle, monthColumnHandle, receiptColumnHandle, amountColumnHandle), + getDynamicFilter(partitionPredicate))) { + assertThat(emptyPageSource.getNextPage()).isNull(); + } + } + + TupleDomain sameYearPredicate = TupleDomain.withColumnDomains( + ImmutableMap.of( + yearColumnHandle, + Domain.singleValue(INTEGER, 2023L))); + TupleDomain sameYearAndMonthPredicate = TupleDomain.withColumnDomains( + ImmutableMap.of( + yearColumnHandle, + Domain.singleValue(INTEGER, 2023L), + monthColumnHandle, + Domain.singleValue(INTEGER, 1L))); + for (TupleDomain partitionPredicate : List.of(sameYearPredicate, sameYearAndMonthPredicate)) { + try (ConnectorPageSource nonEmptyPageSource = createTestingPageSource( + transaction, + icebergConfig, + split, + tableHandle, + ImmutableList.of(yearColumnHandle, monthColumnHandle, receiptColumnHandle, amountColumnHandle), + getDynamicFilter(partitionPredicate))) { + Page page = nonEmptyPageSource.getNextPage(); + assertThat(page).isNotNull(); + assertThat(page.getPositionCount()).isEqualTo(1); + assertThat(page.getBlock(0).getInt(0, 0)).isEqualTo(2023L); + assertThat(page.getBlock(1).getInt(0, 0)).isEqualTo(1L); + assertThat(page.getBlock(2).getSlice(0, 0, page.getBlock(2).getSliceLength(0)).toStringUtf8()).isEqualTo(receiptColumnValue); + assertThat(((SqlDecimal) amountColumnType.getObjectValue(null, page.getBlock(3), 0)).toBigDecimal()).isEqualTo(amountColumnValue); + } + } + } + } + private static ConnectorPageSource createTestingPageSource( HiveTransactionHandle transaction, IcebergConfig icebergConfig, From 6106814efaa64d4d877bf1f53315ddc15f3f28c6 Mon Sep 17 00:00:00 2001 From: Raunaq Morarka Date: Wed, 27 Dec 2023 16:10:20 +0530 Subject: [PATCH 231/350] Remove Deprecated annotation from parquet.ignore-statistics Deprecation was originally intended for parquet.fail-on-corrupted-statistics which is no longer in the code base. Reading files with incorrect/corrupted statistics by ignoring statistics is a valid use case and safe to use. --- .../java/io/trino/plugin/hive/parquet/ParquetReaderConfig.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/parquet/ParquetReaderConfig.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/parquet/ParquetReaderConfig.java index b4b1841f6e8e8..47d06429226e2 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/parquet/ParquetReaderConfig.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/parquet/ParquetReaderConfig.java @@ -37,13 +37,11 @@ public class ParquetReaderConfig private ParquetReaderOptions options = new ParquetReaderOptions(); - @Deprecated public boolean isIgnoreStatistics() { return options.isIgnoreStatistics(); } - @Deprecated @Config("parquet.ignore-statistics") @ConfigDescription("Ignore statistics from Parquet to allow querying files with corrupted or incorrect statistics") public ParquetReaderConfig setIgnoreStatistics(boolean ignoreStatistics) From 9c25df599a8e55a46a98ebf368b85ea9d90a8531 Mon Sep 17 00:00:00 2001 From: Raunaq Morarka Date: Wed, 27 Dec 2023 16:25:01 +0530 Subject: [PATCH 232/350] Add parquet_ignore_statistics session property to iceberg --- .../iceberg/IcebergPageSourceProvider.java | 4 +- .../iceberg/IcebergSessionProperties.java | 11 ++++++ .../iceberg/BaseIcebergConnectorTest.java | 2 +- .../TestIcebergParquetConnectorTest.java | 39 +++++++++++++++++++ 4 files changed, 54 insertions(+), 2 deletions(-) diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSourceProvider.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSourceProvider.java index 9476836cd454b..aceaf809b65d3 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSourceProvider.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSourceProvider.java @@ -167,6 +167,7 @@ import static io.trino.plugin.iceberg.IcebergSessionProperties.getParquetSmallFileThreshold; import static io.trino.plugin.iceberg.IcebergSessionProperties.isOrcBloomFiltersEnabled; import static io.trino.plugin.iceberg.IcebergSessionProperties.isOrcNestedLazy; +import static io.trino.plugin.iceberg.IcebergSessionProperties.isParquetIgnoreStatistics; import static io.trino.plugin.iceberg.IcebergSessionProperties.isUseFileSizeFromMetadata; import static io.trino.plugin.iceberg.IcebergSessionProperties.useParquetBloomFilter; import static io.trino.plugin.iceberg.IcebergSplitManager.ICEBERG_DOMAIN_COMPACTION_THRESHOLD; @@ -611,6 +612,7 @@ public ReaderPageSourceWithRowPositions createDataPageSource( .withMaxReadBlockSize(getParquetMaxReadBlockSize(session)) .withMaxReadBlockRowCount(getParquetMaxReadBlockRowCount(session)) .withSmallFileThreshold(getParquetSmallFileThreshold(session)) + .withIgnoreStatistics(isParquetIgnoreStatistics(session)) .withBloomFilter(useParquetBloomFilter(session)) // TODO https://github.com/trinodb/trino/issues/11000 .withUseColumnIndex(false), @@ -984,7 +986,7 @@ private static ReaderPageSourceWithRowPositions createParquetPageSource( MessageType requestedSchema = getMessageType(regularColumns, fileSchema.getName(), parquetIdToField); Map, ColumnDescriptor> descriptorsByPath = getDescriptors(fileSchema, requestedSchema); - TupleDomain parquetTupleDomain = getParquetTupleDomain(descriptorsByPath, effectivePredicate); + TupleDomain parquetTupleDomain = options.isIgnoreStatistics() ? TupleDomain.all() : getParquetTupleDomain(descriptorsByPath, effectivePredicate); TupleDomainParquetPredicate parquetPredicate = buildPredicate(requestedSchema, parquetTupleDomain, descriptorsByPath, UTC); List rowGroups = getFilteredRowGroups( diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSessionProperties.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSessionProperties.java index 613eea702275c..b499cd212ed1d 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSessionProperties.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSessionProperties.java @@ -76,6 +76,7 @@ public final class IcebergSessionProperties private static final String PARQUET_USE_BLOOM_FILTER = "parquet_use_bloom_filter"; private static final String PARQUET_MAX_READ_BLOCK_ROW_COUNT = "parquet_max_read_block_row_count"; private static final String PARQUET_SMALL_FILE_THRESHOLD = "parquet_small_file_threshold"; + private static final String PARQUET_IGNORE_STATISTICS = "parquet_ignore_statistics"; private static final String PARQUET_WRITER_BLOCK_SIZE = "parquet_writer_block_size"; private static final String PARQUET_WRITER_PAGE_SIZE = "parquet_writer_page_size"; private static final String PARQUET_WRITER_BATCH_SIZE = "parquet_writer_batch_size"; @@ -235,6 +236,11 @@ public IcebergSessionProperties( parquetReaderConfig.getSmallFileThreshold(), value -> validateMaxDataSize(PARQUET_SMALL_FILE_THRESHOLD, value, DataSize.valueOf(PARQUET_READER_MAX_SMALL_FILE_THRESHOLD)), false)) + .add(booleanProperty( + PARQUET_IGNORE_STATISTICS, + "Ignore statistics from Parquet to allow querying files with corrupted or incorrect statistics", + parquetReaderConfig.isIgnoreStatistics(), + false)) .add(dataSizeProperty( PARQUET_WRITER_BLOCK_SIZE, "Parquet: Writer block size", @@ -448,6 +454,11 @@ public static DataSize getParquetSmallFileThreshold(ConnectorSession session) return session.getProperty(PARQUET_SMALL_FILE_THRESHOLD, DataSize.class); } + public static boolean isParquetIgnoreStatistics(ConnectorSession session) + { + return session.getProperty(PARQUET_IGNORE_STATISTICS, Boolean.class); + } + public static DataSize getParquetWriterPageSize(ConnectorSession session) { return session.getProperty(PARQUET_WRITER_PAGE_SIZE, DataSize.class); diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java index ff39da34da8bc..ba732ec9dacf0 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java @@ -4776,7 +4776,7 @@ private void verifySplitCount(QueryId queryId, long expectedSplitCount) } } - private OperatorStats getOperatorStats(QueryId queryId) + protected OperatorStats getOperatorStats(QueryId queryId) { try { return getDistributedQueryRunner().getCoordinator() diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergParquetConnectorTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergParquetConnectorTest.java index 6f6b2d83f663e..67fbbf6c65ccc 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergParquetConnectorTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergParquetConnectorTest.java @@ -13,9 +13,14 @@ */ package io.trino.plugin.iceberg; +import io.trino.Session; import io.trino.filesystem.Location; +import io.trino.operator.OperatorStats; +import io.trino.testing.DistributedQueryRunner; import io.trino.testing.MaterializedResult; +import io.trino.testing.MaterializedResultWithQueryId; import io.trino.testing.sql.TestTable; +import org.intellij.lang.annotations.Language; import org.junit.jupiter.api.Test; import java.util.Optional; @@ -25,6 +30,7 @@ import static io.trino.plugin.iceberg.IcebergFileFormat.PARQUET; import static io.trino.plugin.iceberg.IcebergTestUtils.checkParquetFileSorting; import static io.trino.plugin.iceberg.IcebergTestUtils.withSmallRowGroups; +import static io.trino.testing.QueryAssertions.assertEqualsIgnoreOrder; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -91,6 +97,39 @@ public void testDropAmbiguousRowFieldCaseSensitivity() .hasStackTraceContaining("Multiple entries with same key"); } + @Test + public void testIgnoreParquetStatistics() + { + try (TestTable table = new TestTable( + getQueryRunner()::execute, + "test_ignore_parquet_statistics", + "WITH (sorted_by = ARRAY['custkey']) AS TABLE tpch.tiny.customer WITH NO DATA")) { + assertUpdate( + withSmallRowGroups(getSession()), + "INSERT INTO " + table.getName() + " TABLE tpch.tiny.customer", + "VALUES 1500"); + + @Language("SQL") String query = "SELECT * FROM " + table.getName() + " WHERE custkey = 100"; + + DistributedQueryRunner queryRunner = getDistributedQueryRunner(); + MaterializedResultWithQueryId resultWithoutParquetStatistics = queryRunner.executeWithQueryId( + Session.builder(getSession()) + .setCatalogSessionProperty(getSession().getCatalog().orElseThrow(), "parquet_ignore_statistics", "true") + .build(), + query); + OperatorStats queryStatsWithoutParquetStatistics = getOperatorStats(resultWithoutParquetStatistics.getQueryId()); + assertThat(queryStatsWithoutParquetStatistics.getPhysicalInputPositions()).isGreaterThan(0); + + MaterializedResultWithQueryId resultWithParquetStatistics = queryRunner.executeWithQueryId(getSession(), query); + OperatorStats queryStatsWithParquetStatistics = getOperatorStats(resultWithParquetStatistics.getQueryId()); + assertThat(queryStatsWithParquetStatistics.getPhysicalInputPositions()).isGreaterThan(0); + assertThat(queryStatsWithParquetStatistics.getPhysicalInputPositions()) + .isLessThan(queryStatsWithoutParquetStatistics.getPhysicalInputPositions()); + + assertEqualsIgnoreOrder(resultWithParquetStatistics.getResult(), resultWithoutParquetStatistics.getResult()); + } + } + @Override protected boolean isFileSorted(String path, String sortColumnName) { From cf7e7e537f7234e2c61f7ae44682098b10e0f838 Mon Sep 17 00:00:00 2001 From: Raunaq Morarka Date: Wed, 27 Dec 2023 16:28:57 +0530 Subject: [PATCH 233/350] Add parquet_ignore_statistics session property to delta lake --- .../DeltaLakePageSourceProvider.java | 4 +- .../deltalake/DeltaLakeSessionProperties.java | 11 +++++ .../TableChangesFunctionProcessor.java | 4 +- .../deltalake/TestPredicatePushdown.java | 42 +++++++++++++++++++ 4 files changed, 59 insertions(+), 2 deletions(-) diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakePageSourceProvider.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakePageSourceProvider.java index 4a56b213cd53d..0b0b0d27c8bca 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakePageSourceProvider.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakePageSourceProvider.java @@ -86,6 +86,7 @@ import static io.trino.plugin.deltalake.DeltaLakeSessionProperties.getParquetMaxReadBlockRowCount; import static io.trino.plugin.deltalake.DeltaLakeSessionProperties.getParquetMaxReadBlockSize; import static io.trino.plugin.deltalake.DeltaLakeSessionProperties.getParquetSmallFileThreshold; +import static io.trino.plugin.deltalake.DeltaLakeSessionProperties.isParquetIgnoreStatistics; import static io.trino.plugin.deltalake.DeltaLakeSessionProperties.isParquetUseColumnIndex; import static io.trino.plugin.deltalake.delete.DeletionVectors.readDeletionVectors; import static io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.extractSchema; @@ -204,7 +205,8 @@ public ConnectorPageSource createPageSource( ParquetReaderOptions options = parquetReaderOptions.withMaxReadBlockSize(getParquetMaxReadBlockSize(session)) .withMaxReadBlockRowCount(getParquetMaxReadBlockRowCount(session)) .withSmallFileThreshold(getParquetSmallFileThreshold(session)) - .withUseColumnIndex(isParquetUseColumnIndex(session)); + .withUseColumnIndex(isParquetUseColumnIndex(session)) + .withIgnoreStatistics(isParquetIgnoreStatistics(session)); Map parquetFieldIdToName = columnMappingMode == ColumnMappingMode.ID ? loadParquetIdAndNameMapping(inputFile, options) : ImmutableMap.of(); diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSessionProperties.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSessionProperties.java index 753ca07a80692..065ac2e187fad 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSessionProperties.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSessionProperties.java @@ -56,6 +56,7 @@ public final class DeltaLakeSessionProperties private static final String PARQUET_MAX_READ_BLOCK_ROW_COUNT = "parquet_max_read_block_row_count"; private static final String PARQUET_SMALL_FILE_THRESHOLD = "parquet_small_file_threshold"; private static final String PARQUET_USE_COLUMN_INDEX = "parquet_use_column_index"; + private static final String PARQUET_IGNORE_STATISTICS = "parquet_ignore_statistics"; private static final String PARQUET_WRITER_BLOCK_SIZE = "parquet_writer_block_size"; private static final String PARQUET_WRITER_PAGE_SIZE = "parquet_writer_page_size"; private static final String TARGET_MAX_FILE_SIZE = "target_max_file_size"; @@ -131,6 +132,11 @@ public DeltaLakeSessionProperties( "Use Parquet column index", parquetReaderConfig.isUseColumnIndex(), false), + booleanProperty( + PARQUET_IGNORE_STATISTICS, + "Ignore statistics from Parquet to allow querying files with corrupted or incorrect statistics", + parquetReaderConfig.isIgnoreStatistics(), + false), dataSizeProperty( PARQUET_WRITER_BLOCK_SIZE, "Parquet: Writer block size", @@ -257,6 +263,11 @@ public static boolean isParquetUseColumnIndex(ConnectorSession session) return session.getProperty(PARQUET_USE_COLUMN_INDEX, Boolean.class); } + public static boolean isParquetIgnoreStatistics(ConnectorSession session) + { + return session.getProperty(PARQUET_IGNORE_STATISTICS, Boolean.class); + } + public static DataSize getParquetWriterBlockSize(ConnectorSession session) { return session.getProperty(PARQUET_WRITER_BLOCK_SIZE, DataSize.class); diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/functions/tablechanges/TableChangesFunctionProcessor.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/functions/tablechanges/TableChangesFunctionProcessor.java index d5cf1f482538e..d18e32462a177 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/functions/tablechanges/TableChangesFunctionProcessor.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/functions/tablechanges/TableChangesFunctionProcessor.java @@ -47,6 +47,7 @@ import static io.trino.plugin.deltalake.DeltaLakeColumnType.REGULAR; import static io.trino.plugin.deltalake.DeltaLakeSessionProperties.getParquetMaxReadBlockRowCount; import static io.trino.plugin.deltalake.DeltaLakeSessionProperties.getParquetMaxReadBlockSize; +import static io.trino.plugin.deltalake.DeltaLakeSessionProperties.isParquetIgnoreStatistics; import static io.trino.plugin.deltalake.DeltaLakeSessionProperties.isParquetUseColumnIndex; import static io.trino.plugin.deltalake.functions.tablechanges.TableChangesFileType.CDF_FILE; import static io.trino.spi.function.table.TableFunctionProcessorState.Finished.FINISHED; @@ -175,7 +176,8 @@ private static DeltaLakePageSource createDeltaLakePageSource( parquetReaderOptions = parquetReaderOptions .withMaxReadBlockSize(getParquetMaxReadBlockSize(session)) .withMaxReadBlockRowCount(getParquetMaxReadBlockRowCount(session)) - .withUseColumnIndex(isParquetUseColumnIndex(session)); + .withUseColumnIndex(isParquetUseColumnIndex(session)) + .withIgnoreStatistics(isParquetIgnoreStatistics(session)); List splitColumns = switch (split.fileType()) { case CDF_FILE -> ImmutableList.builder().addAll(handle.columns()) diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestPredicatePushdown.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestPredicatePushdown.java index ec95becbea822..f09ef7ed3d2fc 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestPredicatePushdown.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestPredicatePushdown.java @@ -15,21 +15,27 @@ import com.google.common.collect.ContiguousSet; import com.google.common.collect.ImmutableMap; +import io.trino.Session; +import io.trino.operator.OperatorStats; import io.trino.plugin.hive.containers.HiveMinioDataLake; import io.trino.spi.QueryId; import io.trino.testing.AbstractTestQueryFramework; +import io.trino.testing.DistributedQueryRunner; import io.trino.testing.MaterializedResult; import io.trino.testing.MaterializedResultWithQueryId; import io.trino.testing.MaterializedRow; import io.trino.testing.QueryRunner; +import org.intellij.lang.annotations.Language; import org.junit.jupiter.api.Test; import java.nio.file.Path; import java.util.OptionalLong; import java.util.Set; +import static com.google.common.collect.MoreCollectors.onlyElement; import static io.trino.plugin.deltalake.DeltaLakeQueryRunner.DELTA_CATALOG; import static io.trino.plugin.deltalake.DeltaLakeQueryRunner.createS3DeltaLakeQueryRunner; +import static io.trino.testing.QueryAssertions.assertEqualsIgnoreOrder; import static io.trino.testing.TestingNames.randomNameSuffix; import static java.lang.String.format; import static org.assertj.core.api.Assertions.assertThat; @@ -130,6 +136,42 @@ public void testUpdatePushdown() table)); } + @Test + public void testIgnoreParquetStatistics() + { + String table = testTable.register("ignore_parquet_statistics"); + @Language("SQL") String query = "SELECT * FROM " + table + " WHERE custkey = 1450"; + + DistributedQueryRunner queryRunner = getDistributedQueryRunner(); + MaterializedResultWithQueryId resultWithoutParquetStatistics = queryRunner.executeWithQueryId( + Session.builder(getSession()) + .setCatalogSessionProperty(getSession().getCatalog().orElseThrow(), "parquet_ignore_statistics", "true") + .build(), + query); + OperatorStats queryStatsWithoutParquetStatistics = getOperatorStats(resultWithoutParquetStatistics.getQueryId()); + assertThat(queryStatsWithoutParquetStatistics.getPhysicalInputPositions()).isGreaterThan(0); + + MaterializedResultWithQueryId resultWithParquetStatistics = queryRunner.executeWithQueryId(getSession(), query); + OperatorStats queryStatsWithParquetStatistics = getOperatorStats(resultWithParquetStatistics.getQueryId()); + assertThat(queryStatsWithParquetStatistics.getPhysicalInputPositions()).isGreaterThan(0); + assertThat(queryStatsWithParquetStatistics.getPhysicalInputPositions()) + .isLessThan(queryStatsWithoutParquetStatistics.getPhysicalInputPositions()); + + assertEqualsIgnoreOrder(resultWithParquetStatistics.getResult(), resultWithoutParquetStatistics.getResult()); + } + + private OperatorStats getOperatorStats(QueryId queryId) + { + return getDistributedQueryRunner().getCoordinator() + .getQueryManager() + .getFullQueryInfo(queryId) + .getQueryStats() + .getOperatorSummaries() + .stream() + .filter(summary -> summary.getOperatorType().startsWith("TableScan") || summary.getOperatorType().startsWith("Scan")) + .collect(onlyElement()); + } + /** * Assert on the number of rows read and updated by a read operation * From b087a3cdb06308cb503d92230446a0032bc85c86 Mon Sep 17 00:00:00 2001 From: Raunaq Morarka Date: Wed, 27 Dec 2023 16:38:09 +0530 Subject: [PATCH 234/350] Add documentation for parquet.ignore-statistics --- .../src/main/sphinx/connector/object-storage-file-formats.md | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/docs/src/main/sphinx/connector/object-storage-file-formats.md b/docs/src/main/sphinx/connector/object-storage-file-formats.md index 45e9f59c34e52..b47be16835fc3 100644 --- a/docs/src/main/sphinx/connector/object-storage-file-formats.md +++ b/docs/src/main/sphinx/connector/object-storage-file-formats.md @@ -91,6 +91,11 @@ with Parquet files performed by supported object storage connectors: catalog session property is `parquet_use_column_index`. Only supported by the Delta Lake and Hive connectors. - `true` +* - `parquet.ignore-statistics` + - Ignore statistics from Parquet to allow querying files with corrupted or + incorrect statistics. The equivalent catalog session property is + `parquet_ignore_statistics`. + - `false` * - `parquet.max-read-block-row-count` - Sets the maximum number of rows read in a batch. The equivalent catalog session property is named `parquet_max_read_block_row_count` and supported From 1706df06e79c7b2ed9dc865ef46937124e471f06 Mon Sep 17 00:00:00 2001 From: Jaeho Yoo Date: Wed, 3 Jan 2024 11:23:43 +0900 Subject: [PATCH 235/350] Update docs for java 17.0.5 --- README.md | 2 +- docs/src/main/sphinx/installation/deployment.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index b59e4ace4b18f..6c2ee3efeda0d 100644 --- a/README.md +++ b/README.md @@ -34,7 +34,7 @@ information about reporting vulnerabilities. ## Build requirements * Mac OS X or Linux -* Java 17.0.4+, 64-bit +* Java 17.0.5+, 64-bit * Docker * Turn SELinux or other systems disabling write access to the local checkout off, to allow containers to mount parts of the Trino source tree diff --git a/docs/src/main/sphinx/installation/deployment.md b/docs/src/main/sphinx/installation/deployment.md index e7f723c5ede7f..22d7b635b36b0 100644 --- a/docs/src/main/sphinx/installation/deployment.md +++ b/docs/src/main/sphinx/installation/deployment.md @@ -35,7 +35,7 @@ ### Java runtime environment -Trino requires a 64-bit version of Java 17, with a minimum required version of 17.0.3. +Trino requires a 64-bit version of Java 17, with a minimum required version of 17.0.5. Earlier major versions such as Java 8 or Java 11 do not work. Newer major versions such as Java 18 or 19, are not supported -- they may work, but are not tested. From 19a86960704621ad554fe80796b4156fada309b5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C5=81ukasz=20Osipiuk?= Date: Wed, 6 Dec 2023 12:07:07 +0100 Subject: [PATCH 236/350] Increase threshold for stalled FTE scheduler logging --- .../faulttolerant/EventDrivenFaultTolerantQueryScheduler.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/EventDrivenFaultTolerantQueryScheduler.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/EventDrivenFaultTolerantQueryScheduler.java index 3d451fac3687f..f34e4f547af4c 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/EventDrivenFaultTolerantQueryScheduler.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/EventDrivenFaultTolerantQueryScheduler.java @@ -653,7 +653,7 @@ private static class Scheduler // If scheduler is stalled for SCHEDULER_STALLED_DURATION_THRESHOLD debug log will be emitted. // This value must be larger than EVENT_PROCESSING_ENFORCED_FREQUENCY as prerequiste for processing is // that there are no events in the event queue. - private static final long SCHEDULER_STALLED_DURATION_THRESHOLD_MILLIS = MINUTES.toMillis(5); + private static final long SCHEDULER_STALLED_DURATION_THRESHOLD_MILLIS = MINUTES.toMillis(10); private static final long SCHEDULER_STALLED_DURATION_ON_TIME_EXCEEDED_THRESHOLD_MILLIS = SECONDS.toMillis(30); private static final int EVENTS_DEBUG_INFOS_PER_BUCKET = 10; From aa84aea6c1d5f44bd4f5758313d9df254fa2e9ae Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C5=81ukasz=20Osipiuk?= Date: Thu, 14 Dec 2023 15:16:25 +0100 Subject: [PATCH 237/350] Decouple debug logging frequency and scheduler stalled time Count separately how much time passed since FTE scheduler processed a meaningful event and how much time passed since debug information was last logged. --- ...ventDrivenFaultTolerantQueryScheduler.java | 24 ++++++++++++------- 1 file changed, 15 insertions(+), 9 deletions(-) diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/EventDrivenFaultTolerantQueryScheduler.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/EventDrivenFaultTolerantQueryScheduler.java index f34e4f547af4c..2132a0b95c345 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/EventDrivenFaultTolerantQueryScheduler.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/faulttolerant/EventDrivenFaultTolerantQueryScheduler.java @@ -651,9 +651,11 @@ private static class Scheduler private static final int EVENT_BUFFER_CAPACITY = 100; private static final long EVENT_PROCESSING_ENFORCED_FREQUENCY_MILLIS = MINUTES.toMillis(1); // If scheduler is stalled for SCHEDULER_STALLED_DURATION_THRESHOLD debug log will be emitted. - // This value must be larger than EVENT_PROCESSING_ENFORCED_FREQUENCY as prerequiste for processing is + // If situation persists event logs will be emitted at SCHEDULER_MAX_DEBUG_INFO_FREQUENCY. + // SCHEDULER_STALLED_DURATION_THRESHOLD must be larger than EVENT_PROCESSING_ENFORCED_FREQUENCY as prerequiste for processing is // that there are no events in the event queue. private static final long SCHEDULER_STALLED_DURATION_THRESHOLD_MILLIS = MINUTES.toMillis(10); + private static final long SCHEDULER_MAX_DEBUG_INFO_FREQUENCY_MILLIS = MINUTES.toMillis(10); private static final long SCHEDULER_STALLED_DURATION_ON_TIME_EXCEEDED_THRESHOLD_MILLIS = SECONDS.toMillis(30); private static final int EVENTS_DEBUG_INFOS_PER_BUCKET = 10; @@ -688,7 +690,8 @@ private static class Scheduler private final BlockingQueue eventQueue = new LinkedBlockingQueue<>(); private final List eventBuffer = new ArrayList<>(EVENT_BUFFER_CAPACITY); - private final Stopwatch eventDebugInfoStopwatch = Stopwatch.createUnstarted(); + private final Stopwatch noEventsStopwatch = Stopwatch.createUnstarted(); + private final Stopwatch debugInfoStopwatch = Stopwatch.createUnstarted(); private final Optional eventDebugInfos; private boolean started; @@ -781,7 +784,7 @@ public Scheduler( } planInTopologicalOrder = sortPlanInTopologicalOrder(plan); - eventDebugInfoStopwatch.start(); + noEventsStopwatch.start(); } public void run() @@ -801,8 +804,8 @@ public void run() } if (queryInfo.getState() == QueryState.FAILED && queryInfo.getErrorCode() == EXCEEDED_TIME_LIMIT.toErrorCode() - && eventDebugInfoStopwatch.elapsed().toMillis() > SCHEDULER_STALLED_DURATION_ON_TIME_EXCEEDED_THRESHOLD_MILLIS) { - logDebugInfoSafe(format("Scheduler stalled for %s on EXCEEDED_TIME_LIMIT", eventDebugInfoStopwatch.elapsed())); + && noEventsStopwatch.elapsed().toMillis() > SCHEDULER_STALLED_DURATION_ON_TIME_EXCEEDED_THRESHOLD_MILLIS) { + logDebugInfoSafe(format("Scheduler stalled for %s on EXCEEDED_TIME_LIMIT", noEventsStopwatch.elapsed())); } }); @@ -896,13 +899,16 @@ private boolean processEvents() if (eventDebugInfoRecorded) { // mark that we processed some events; we filter out some no-op events. // If only no-op events appear in event queue we still treat scheduler as stuck - eventDebugInfoStopwatch.reset().start(); + noEventsStopwatch.reset().start(); + debugInfoStopwatch.reset(); } else { // if no events were recorded there is a chance scheduler is stalled - if (log.isDebugEnabled() && eventDebugInfoStopwatch.elapsed().toMillis() > SCHEDULER_STALLED_DURATION_THRESHOLD_MILLIS) { - logDebugInfoSafe("Scheduler stalled for %s".formatted(eventDebugInfoStopwatch.elapsed())); - eventDebugInfoStopwatch.reset().start(); // reset to prevent extensive logging + if (log.isDebugEnabled() + && (!debugInfoStopwatch.isRunning() || debugInfoStopwatch.elapsed().toMillis() > SCHEDULER_MAX_DEBUG_INFO_FREQUENCY_MILLIS) + && noEventsStopwatch.elapsed().toMillis() > SCHEDULER_STALLED_DURATION_THRESHOLD_MILLIS) { + logDebugInfoSafe("Scheduler stalled for %s".formatted(noEventsStopwatch.elapsed())); + debugInfoStopwatch.reset().start(); // reset to prevent extensive logging } } From 8aab3631bc08d922ec7e241a6f7d63d91113011d Mon Sep 17 00:00:00 2001 From: Dominik Zalewski Date: Wed, 27 Dec 2023 16:57:27 +0100 Subject: [PATCH 238/350] Sometimes EventListenerMangager.queryCompleted() called too early For happy path and parsable queries, we have the following order of calls: - query is added to the QueryTracker - EventListener.queryCompleted() is called For really bad queries, the ones that do not even parse to a proper SQL, the order is inverted. As a result, when implementing an EventListener that wants to access query detailed information we have to use a workaround for this edge case and artificially invert the sequence of calls. EventListener will typically use a retry mechanism combined with switching to a separate thread, which adds unnecessary complexity and multi-threading to a code that otherwise would be single-threaded. --- .../java/io/trino/dispatcher/DispatchManager.java | 12 +++++++++++- .../trino/dispatcher/FailedDispatchQueryFactory.java | 2 -- 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/core/trino-main/src/main/java/io/trino/dispatcher/DispatchManager.java b/core/trino-main/src/main/java/io/trino/dispatcher/DispatchManager.java index fd3e9b3563c65..39b7d1795a0d3 100644 --- a/core/trino-main/src/main/java/io/trino/dispatcher/DispatchManager.java +++ b/core/trino-main/src/main/java/io/trino/dispatcher/DispatchManager.java @@ -22,6 +22,7 @@ import io.opentelemetry.api.trace.Tracer; import io.opentelemetry.context.Context; import io.trino.Session; +import io.trino.event.QueryMonitor; import io.trino.execution.QueryIdGenerator; import io.trino.execution.QueryInfo; import io.trino.execution.QueryManagerConfig; @@ -56,6 +57,7 @@ import static io.trino.execution.QueryState.RUNNING; import static io.trino.spi.StandardErrorCode.QUERY_TEXT_TOO_LARGE; import static io.trino.tracing.ScopedSpan.scopedSpan; +import static io.trino.util.Failures.toFailure; import static io.trino.util.StatementUtils.getQueryType; import static java.lang.String.format; import static java.util.Objects.requireNonNull; @@ -80,6 +82,7 @@ public class DispatchManager private final QueryTracker queryTracker; private final QueryManagerStats stats = new QueryManagerStats(); + private final QueryMonitor queryMonitor; @Inject public DispatchManager( @@ -94,7 +97,8 @@ public DispatchManager( SessionPropertyManager sessionPropertyManager, Tracer tracer, QueryManagerConfig queryManagerConfig, - DispatchExecutor dispatchExecutor) + DispatchExecutor dispatchExecutor, + QueryMonitor queryMonitor) { this.queryIdGenerator = requireNonNull(queryIdGenerator, "queryIdGenerator is null"); this.queryPreparer = requireNonNull(queryPreparer, "queryPreparer is null"); @@ -112,6 +116,7 @@ public DispatchManager( this.dispatchExecutor = dispatchExecutor.getExecutor(); this.queryTracker = new QueryTracker<>(queryManagerConfig, dispatchExecutor.getScheduledExecutor()); + this.queryMonitor = requireNonNull(queryMonitor, "queryMonitor is null"); } @PostConstruct @@ -236,6 +241,11 @@ private void createQueryInternal(QueryId queryId, Span querySpan, Slug slug, Optional preparedSql = Optional.ofNullable(preparedQuery).flatMap(PreparedQuery::getPrepareSql); DispatchQuery failedDispatchQuery = failedDispatchQueryFactory.createFailedDispatchQuery(session, query, preparedSql, Optional.empty(), throwable); queryCreated(failedDispatchQuery); + // maintain proper order of calls such that EventListener has access to QueryInfo + // - add query to tracker + // - fire query created event + // - fire query completed event + queryMonitor.queryImmediateFailureEvent(failedDispatchQuery.getBasicQueryInfo(), toFailure(throwable)); querySpan.setStatus(StatusCode.ERROR, throwable.getMessage()) .recordException(throwable) .end(); diff --git a/core/trino-main/src/main/java/io/trino/dispatcher/FailedDispatchQueryFactory.java b/core/trino-main/src/main/java/io/trino/dispatcher/FailedDispatchQueryFactory.java index 3eeab6198ef39..65c75d35780e3 100644 --- a/core/trino-main/src/main/java/io/trino/dispatcher/FailedDispatchQueryFactory.java +++ b/core/trino-main/src/main/java/io/trino/dispatcher/FailedDispatchQueryFactory.java @@ -24,7 +24,6 @@ import java.util.Optional; import java.util.concurrent.ExecutorService; -import static io.trino.util.Failures.toFailure; import static java.util.Objects.requireNonNull; public class FailedDispatchQueryFactory @@ -58,7 +57,6 @@ public FailedDispatchQuery createFailedDispatchQuery(Session session, String que BasicQueryInfo queryInfo = failedDispatchQuery.getBasicQueryInfo(); queryMonitor.queryCreatedEvent(queryInfo); - queryMonitor.queryImmediateFailureEvent(queryInfo, toFailure(throwable)); return failedDispatchQuery; } From 7037df73968861f3ba235f8ffa3d2968569e146c Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Thu, 21 Dec 2023 17:19:27 +0100 Subject: [PATCH 239/350] Update to Iceberg 1.4.3 This e.g. fixes data loss on retried commit under some circumstances. --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 753d9a0d9f6f8..cbf17b622194d 100644 --- a/pom.xml +++ b/pom.xml @@ -185,7 +185,7 @@ 2.23.0 10.3.0 1.43.3 - 1.4.2 + 1.4.3 5.14.0 0.12.3 3.6.1 From d269fad2d2f872e5200fc33da72acd58e21838d4 Mon Sep 17 00:00:00 2001 From: Marius Grama Date: Tue, 2 Jan 2024 22:35:13 +0100 Subject: [PATCH 240/350] Add scaffolding for unit testing dynamic filtering --- ...DeltaLakeNodeLocalDynamicSplitPruning.java | 248 ++++++++++++++++++ 1 file changed, 248 insertions(+) create mode 100644 plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeNodeLocalDynamicSplitPruning.java diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeNodeLocalDynamicSplitPruning.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeNodeLocalDynamicSplitPruning.java new file mode 100644 index 0000000000000..880a1e5465419 --- /dev/null +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeNodeLocalDynamicSplitPruning.java @@ -0,0 +1,248 @@ +/* + * 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.deltalake; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import io.airlift.testing.TempFile; +import io.trino.filesystem.TrinoInputFile; +import io.trino.filesystem.TrinoOutputFile; +import io.trino.filesystem.hdfs.HdfsFileSystemFactory; +import io.trino.filesystem.local.LocalInputFile; +import io.trino.filesystem.local.LocalOutputFile; +import io.trino.metadata.TableHandle; +import io.trino.parquet.writer.ParquetSchemaConverter; +import io.trino.parquet.writer.ParquetWriter; +import io.trino.parquet.writer.ParquetWriterOptions; +import io.trino.plugin.deltalake.transactionlog.MetadataEntry; +import io.trino.plugin.deltalake.transactionlog.ProtocolEntry; +import io.trino.plugin.hive.FileFormatDataSourceStats; +import io.trino.plugin.hive.HiveTransactionHandle; +import io.trino.plugin.hive.parquet.ParquetReaderConfig; +import io.trino.plugin.hive.parquet.ParquetWriterConfig; +import io.trino.spi.Page; +import io.trino.spi.SplitWeight; +import io.trino.spi.block.BlockBuilder; +import io.trino.spi.connector.ColumnHandle; +import io.trino.spi.connector.ConnectorPageSource; +import io.trino.spi.connector.DynamicFilter; +import io.trino.spi.predicate.Domain; +import io.trino.spi.predicate.TupleDomain; +import io.trino.testing.TestingConnectorSession; +import org.apache.parquet.format.CompressionCodec; +import org.joda.time.DateTimeZone; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.nio.file.Files; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.OptionalInt; +import java.util.Set; +import java.util.concurrent.CompletableFuture; + +import static io.trino.plugin.deltalake.DeltaLakeColumnType.REGULAR; +import static io.trino.plugin.hive.HiveTestUtils.HDFS_ENVIRONMENT; +import static io.trino.plugin.hive.HiveTestUtils.HDFS_FILE_SYSTEM_STATS; +import static io.trino.spi.type.IntegerType.INTEGER; +import static io.trino.spi.type.VarcharType.VARCHAR; +import static io.trino.testing.TestingHandles.TEST_CATALOG_HANDLE; +import static io.trino.type.InternalTypeManager.TESTING_TYPE_MANAGER; +import static java.util.concurrent.CompletableFuture.completedFuture; +import static org.assertj.core.api.Assertions.assertThat; + +public class TestDeltaLakeNodeLocalDynamicSplitPruning +{ + private static final ParquetReaderConfig PARQUET_READER_CONFIG = new ParquetReaderConfig(); + private static final ParquetWriterConfig PARQUET_WRITER_CONFIG = new ParquetWriterConfig(); + + @Test + public void testDynamicSplitPruningOnUnpartitionedTable() + throws IOException + { + String keyColumnName = "a_integer"; + DeltaLakeColumnHandle keyColumnHandle = new DeltaLakeColumnHandle(keyColumnName, INTEGER, OptionalInt.empty(), keyColumnName, INTEGER, REGULAR, Optional.empty()); + int keyColumnValue = 42; + String dataColumnName = "a_varchar"; + String dataColumnValue = "hello world"; + DeltaLakeColumnHandle dataColumnHandle = new DeltaLakeColumnHandle(dataColumnName, VARCHAR, OptionalInt.empty(), dataColumnName, VARCHAR, REGULAR, Optional.empty()); + ParquetSchemaConverter schemaConverter = new ParquetSchemaConverter( + ImmutableList.of(INTEGER, VARCHAR), + ImmutableList.of(keyColumnName, dataColumnName), + false, + false); + + DeltaLakeConfig deltaLakeConfig = new DeltaLakeConfig(); + HiveTransactionHandle transaction = new HiveTransactionHandle(false); + try (TempFile file = new TempFile()) { + Files.delete(file.path()); + + TrinoOutputFile outputFile = new LocalOutputFile(file.file()); + TrinoInputFile inputFile = new LocalInputFile(file.file()); + + try (ParquetWriter writer = createParquetWriter(outputFile, schemaConverter)) { + BlockBuilder keyBuilder = INTEGER.createBlockBuilder(null, 1); + INTEGER.writeLong(keyBuilder, keyColumnValue); + BlockBuilder dataBuilder = VARCHAR.createBlockBuilder(null, 1); + VARCHAR.writeString(dataBuilder, dataColumnValue); + writer.write(new Page(keyBuilder.build(), dataBuilder.build())); + } + + DeltaLakeSplit split = new DeltaLakeSplit( + inputFile.location().toString(), + 0, + inputFile.length(), + inputFile.length(), + Optional.empty(), + 0, + Optional.empty(), + SplitWeight.standard(), + TupleDomain.all(), + ImmutableMap.of()); + + MetadataEntry metadataEntry = new MetadataEntry( + "id", + "name", + "description", + new MetadataEntry.Format("provider", ImmutableMap.of()), + "{\"type\":\"struct\",\"fields\":[{\"name\":\"a_integer\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"a_varchar\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]}", + ImmutableList.of(), + ImmutableMap.of(), + 0); + TableHandle tableHandle = new TableHandle( + TEST_CATALOG_HANDLE, + new DeltaLakeTableHandle( + "test_schema_name", + "unpartitioned_table", + true, + "test_location", + metadataEntry, + new ProtocolEntry(1, 2, Optional.empty(), Optional.empty()), + TupleDomain.all(), + TupleDomain.all(), + Optional.empty(), + Optional.of(Set.of(keyColumnHandle, dataColumnHandle)), + Optional.empty(), + Optional.empty(), + Optional.empty(), + 0), + transaction); + + TupleDomain splitPruningPredicate = TupleDomain.withColumnDomains( + ImmutableMap.of( + keyColumnHandle, + Domain.singleValue(INTEGER, 1L))); + try (ConnectorPageSource emptyPageSource = createTestingPageSource(transaction, deltaLakeConfig, split, tableHandle, ImmutableList.of(keyColumnHandle, dataColumnHandle), getDynamicFilter(splitPruningPredicate))) { + assertThat(emptyPageSource.getNextPage()).isNull(); + } + + TupleDomain nonSelectivePredicate = TupleDomain.withColumnDomains( + ImmutableMap.of( + keyColumnHandle, + Domain.singleValue(INTEGER, (long) keyColumnValue))); + try (ConnectorPageSource nonEmptyPageSource = createTestingPageSource(transaction, deltaLakeConfig, split, tableHandle, ImmutableList.of(keyColumnHandle, dataColumnHandle), getDynamicFilter(nonSelectivePredicate))) { + Page page = nonEmptyPageSource.getNextPage(); + assertThat(page).isNotNull(); + assertThat(page.getPositionCount()).isEqualTo(1); + assertThat(page.getBlock(0).getInt(0, 0)).isEqualTo(keyColumnValue); + assertThat(page.getBlock(1).getSlice(0, 0, page.getBlock(1).getSliceLength(0)).toStringUtf8()).isEqualTo(dataColumnValue); + } + } + } + + private static ParquetWriter createParquetWriter(TrinoOutputFile outputFile, ParquetSchemaConverter schemaConverter) + throws IOException + { + return new ParquetWriter( + outputFile.create(), + schemaConverter.getMessageType(), + schemaConverter.getPrimitiveTypes(), + ParquetWriterOptions.builder().build(), + CompressionCodec.SNAPPY, + "test", + Optional.of(DateTimeZone.UTC), + Optional.empty()); + } + + private static ConnectorPageSource createTestingPageSource( + HiveTransactionHandle transaction, + DeltaLakeConfig deltaLakeConfig, + DeltaLakeSplit split, + TableHandle tableHandle, + List columns, + DynamicFilter dynamicFilter) + { + FileFormatDataSourceStats stats = new FileFormatDataSourceStats(); + DeltaLakePageSourceProvider provider = new DeltaLakePageSourceProvider( + new HdfsFileSystemFactory(HDFS_ENVIRONMENT, HDFS_FILE_SYSTEM_STATS), + stats, + PARQUET_READER_CONFIG, + deltaLakeConfig, + TESTING_TYPE_MANAGER); + + return provider.createPageSource( + transaction, + getSession(deltaLakeConfig), + split, + tableHandle.getConnectorHandle(), + columns, + dynamicFilter); + } + + private static TestingConnectorSession getSession(DeltaLakeConfig deltaLakeConfig) + { + return TestingConnectorSession.builder() + .setPropertyMetadata(new DeltaLakeSessionProperties(deltaLakeConfig, PARQUET_READER_CONFIG, PARQUET_WRITER_CONFIG).getSessionProperties()) + .build(); + } + + private static DynamicFilter getDynamicFilter(TupleDomain tupleDomain) + { + return new DynamicFilter() + { + @Override + public Set getColumnsCovered() + { + return tupleDomain.getDomains().map(Map::keySet) + .orElseGet(ImmutableSet::of); + } + + @Override + public CompletableFuture isBlocked() + { + return completedFuture(null); + } + + @Override + public boolean isComplete() + { + return true; + } + + @Override + public boolean isAwaitable() + { + return false; + } + + @Override + public TupleDomain getCurrentPredicate() + { + return tupleDomain; + } + }; + } +} From 5ac064541e62dd7253bdaac86e11a447cb427052 Mon Sep 17 00:00:00 2001 From: Marius Grama Date: Tue, 2 Jan 2024 23:20:09 +0100 Subject: [PATCH 241/350] Short circuit page source in case of partition mismatch In case that the dynamic filter completes after scheduling of split on the worker, the results in the split will be getting pruned in the situation that there is a partition predicate mismatch. --- .../DeltaLakePageSourceProvider.java | 7 + ...DeltaLakeNodeLocalDynamicSplitPruning.java | 136 ++++++++++++++++++ 2 files changed, 143 insertions(+) diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakePageSourceProvider.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakePageSourceProvider.java index 0b0b0d27c8bca..0882ad8100a79 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakePageSourceProvider.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakePageSourceProvider.java @@ -88,6 +88,7 @@ import static io.trino.plugin.deltalake.DeltaLakeSessionProperties.getParquetSmallFileThreshold; import static io.trino.plugin.deltalake.DeltaLakeSessionProperties.isParquetIgnoreStatistics; import static io.trino.plugin.deltalake.DeltaLakeSessionProperties.isParquetUseColumnIndex; +import static io.trino.plugin.deltalake.DeltaLakeSplitManager.partitionMatchesPredicate; import static io.trino.plugin.deltalake.delete.DeletionVectors.readDeletionVectors; import static io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.extractSchema; import static io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.getColumnMappingMode; @@ -181,6 +182,12 @@ public ConnectorPageSource createPageSource( if (filteredSplitPredicate.isNone()) { return new EmptyPageSource(); } + Map partitionColumnDomains = filteredSplitPredicate.getDomains().orElseThrow().entrySet().stream() + .filter(entry -> entry.getKey().getColumnType() == DeltaLakeColumnType.PARTITION_KEY) + .collect(toImmutableMap(Map.Entry::getKey, Map.Entry::getValue)); + if (!partitionMatchesPredicate(split.getPartitionKeys(), partitionColumnDomains)) { + return new EmptyPageSource(); + } if (filteredSplitPredicate.isAll() && split.getStart() == 0 && split.getLength() == split.getFileSize() && split.getFileRowCount().isPresent() && diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeNodeLocalDynamicSplitPruning.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeNodeLocalDynamicSplitPruning.java index 880a1e5465419..df81d9e5a5364 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeNodeLocalDynamicSplitPruning.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeNodeLocalDynamicSplitPruning.java @@ -39,14 +39,20 @@ import io.trino.spi.connector.ConnectorPageSource; import io.trino.spi.connector.DynamicFilter; import io.trino.spi.predicate.Domain; +import io.trino.spi.predicate.Range; import io.trino.spi.predicate.TupleDomain; +import io.trino.spi.predicate.ValueSet; +import io.trino.spi.type.DecimalType; +import io.trino.spi.type.SqlDecimal; import io.trino.testing.TestingConnectorSession; import org.apache.parquet.format.CompressionCodec; import org.joda.time.DateTimeZone; import org.junit.jupiter.api.Test; import java.io.IOException; +import java.math.BigDecimal; import java.nio.file.Files; +import java.time.LocalDate; import java.util.List; import java.util.Map; import java.util.Optional; @@ -54,9 +60,12 @@ import java.util.Set; import java.util.concurrent.CompletableFuture; +import static io.trino.plugin.deltalake.DeltaLakeColumnType.PARTITION_KEY; import static io.trino.plugin.deltalake.DeltaLakeColumnType.REGULAR; import static io.trino.plugin.hive.HiveTestUtils.HDFS_ENVIRONMENT; import static io.trino.plugin.hive.HiveTestUtils.HDFS_FILE_SYSTEM_STATS; +import static io.trino.spi.type.DateType.DATE; +import static io.trino.spi.type.Decimals.writeShortDecimal; import static io.trino.spi.type.IntegerType.INTEGER; import static io.trino.spi.type.VarcharType.VARCHAR; import static io.trino.testing.TestingHandles.TEST_CATALOG_HANDLE; @@ -163,6 +172,133 @@ public void testDynamicSplitPruningOnUnpartitionedTable() } } + @Test + public void testDynamicSplitPruningWithExplicitPartitionFilter() + throws IOException + { + String dateColumnName = "date"; + DeltaLakeColumnHandle dateColumnHandle = new DeltaLakeColumnHandle(dateColumnName, DATE, OptionalInt.empty(), dateColumnName, DATE, PARTITION_KEY, Optional.empty()); + long dateColumnValue = LocalDate.of(2023, 1, 10).toEpochDay(); + String receiptColumnName = "receipt"; + DeltaLakeColumnHandle receiptColumnHandle = new DeltaLakeColumnHandle(receiptColumnName, VARCHAR, OptionalInt.empty(), receiptColumnName, VARCHAR, REGULAR, Optional.empty()); + String receiptColumnValue = "#12345"; + String amountColumnName = "amount"; + DecimalType amountColumnType = DecimalType.createDecimalType(10, 2); + DeltaLakeColumnHandle amountColumnHandle = new DeltaLakeColumnHandle(amountColumnName, amountColumnType, OptionalInt.empty(), amountColumnName, amountColumnType, REGULAR, Optional.empty()); + BigDecimal amountColumnValue = new BigDecimal("1234567.65"); + ParquetSchemaConverter schemaConverter = new ParquetSchemaConverter( + ImmutableList.of(VARCHAR, amountColumnType), + ImmutableList.of(receiptColumnName, amountColumnName), + false, + false); + + DeltaLakeConfig icebergConfig = new DeltaLakeConfig(); + HiveTransactionHandle transaction = new HiveTransactionHandle(false); + try (TempFile file = new TempFile()) { + Files.delete(file.path()); + + TrinoOutputFile outputFile = new LocalOutputFile(file.file()); + TrinoInputFile inputFile = new LocalInputFile(file.file()); + + try (ParquetWriter writer = createParquetWriter(outputFile, schemaConverter)) { + BlockBuilder receiptBuilder = VARCHAR.createBlockBuilder(null, 1); + VARCHAR.writeString(receiptBuilder, receiptColumnValue); + BlockBuilder amountBuilder = amountColumnType.createBlockBuilder(null, 1); + writeShortDecimal(amountBuilder, amountColumnValue.unscaledValue().longValueExact()); + writer.write(new Page(receiptBuilder.build(), amountBuilder.build())); + } + + DeltaLakeSplit split = new DeltaLakeSplit( + inputFile.location().toString(), + 0, + inputFile.length(), + inputFile.length(), + Optional.empty(), + 0, + Optional.empty(), + SplitWeight.standard(), + TupleDomain.all(), + ImmutableMap.of(dateColumnName, Optional.of("2023-01-10"))); + + MetadataEntry metadataEntry = new MetadataEntry( + "id", + "name", + "description", + new MetadataEntry.Format("provider", ImmutableMap.of()), + "{\"type\":\"struct\",\"fields\":[{\"name\":\"date\",\"type\":\"date\",\"nullable\":true,\"metadata\":{}},{\"name\":\"receipt\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"amount\",\"type\":\"decimal(10,2)\",\"nullable\":true,\"metadata\":{}}]}", + ImmutableList.of(dateColumnName), + ImmutableMap.of(), + 0); + + TableHandle tableHandle = new TableHandle( + TEST_CATALOG_HANDLE, + new DeltaLakeTableHandle( + "test_schema_name", + "unpartitioned_table", + true, + "test_location", + metadataEntry, + new ProtocolEntry(1, 2, Optional.empty(), Optional.empty()), + TupleDomain.all(), + TupleDomain.all(), + Optional.empty(), + Optional.of(Set.of(dateColumnHandle, receiptColumnHandle, amountColumnHandle)), + Optional.empty(), + Optional.empty(), + Optional.empty(), + 0), + transaction); + + // Simulate situations where the dynamic filter (e.g.: while performing a JOIN with another table) reduces considerably + // the amount of data to be processed from the current table + + TupleDomain differentDatePredicate = TupleDomain.withColumnDomains( + ImmutableMap.of( + dateColumnHandle, + Domain.singleValue(DATE, LocalDate.of(2023, 2, 2).toEpochDay()))); + TupleDomain nonOverlappingDatePredicate = TupleDomain.withColumnDomains( + ImmutableMap.of( + dateColumnHandle, + Domain.create(ValueSet.ofRanges(Range.greaterThanOrEqual(DATE, LocalDate.of(2023, 2, 2).toEpochDay())), true))); + for (TupleDomain partitionPredicate : List.of(differentDatePredicate, nonOverlappingDatePredicate)) { + try (ConnectorPageSource emptyPageSource = createTestingPageSource( + transaction, + icebergConfig, + split, + tableHandle, + ImmutableList.of(dateColumnHandle, receiptColumnHandle, amountColumnHandle), + getDynamicFilter(partitionPredicate))) { + assertThat(emptyPageSource.getNextPage()).isNull(); + } + } + + TupleDomain sameDatePredicate = TupleDomain.withColumnDomains( + ImmutableMap.of( + dateColumnHandle, + Domain.singleValue(DATE, dateColumnValue))); + TupleDomain overlappingDatePredicate = TupleDomain.withColumnDomains( + ImmutableMap.of( + dateColumnHandle, + Domain.create(ValueSet.ofRanges(Range.range(DATE, LocalDate.of(2023, 1, 1).toEpochDay(), true, LocalDate.of(2023, 2, 1).toEpochDay(), false)), true))); + for (TupleDomain partitionPredicate : List.of(sameDatePredicate, overlappingDatePredicate)) { + try (ConnectorPageSource nonEmptyPageSource = createTestingPageSource( + transaction, + icebergConfig, + split, + tableHandle, + ImmutableList.of(dateColumnHandle, receiptColumnHandle, amountColumnHandle), + getDynamicFilter(partitionPredicate))) { + Page page = nonEmptyPageSource.getNextPage(); + assertThat(page).isNotNull(); + assertThat(page.getPositionCount()).isEqualTo(1); + assertThat(page.getBlock(0).getInt(0, 0)).isEqualTo(dateColumnValue); + assertThat(page.getBlock(1).getSlice(0, 0, page.getBlock(1).getSliceLength(0)).toStringUtf8()).isEqualTo(receiptColumnValue); + assertThat(((SqlDecimal) amountColumnType.getObjectValue(null, page.getBlock(2), 0)).toBigDecimal()).isEqualTo(amountColumnValue); + } + } + } + } + private static ParquetWriter createParquetWriter(TrinoOutputFile outputFile, ParquetSchemaConverter schemaConverter) throws IOException { From 77a4e949816e009f968300dc0417b2cbda4e5958 Mon Sep 17 00:00:00 2001 From: Yuya Ebihara Date: Thu, 4 Jan 2024 09:57:12 +0900 Subject: [PATCH 242/350] Fix TestRedshiftTypeMapping --- .../io/trino/plugin/redshift/TestRedshiftTypeMapping.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/plugin/trino-redshift/src/test/java/io/trino/plugin/redshift/TestRedshiftTypeMapping.java b/plugin/trino-redshift/src/test/java/io/trino/plugin/redshift/TestRedshiftTypeMapping.java index b2ea6798863b9..52a96949499cc 100644 --- a/plugin/trino-redshift/src/test/java/io/trino/plugin/redshift/TestRedshiftTypeMapping.java +++ b/plugin/trino-redshift/src/test/java/io/trino/plugin/redshift/TestRedshiftTypeMapping.java @@ -462,8 +462,7 @@ public void testTimestamp() testTimestamp(testZone); } - @Test - public void testTimestamp(ZoneId sessionZone) + private void testTimestamp(ZoneId sessionZone) { Session session = Session.builder(getSession()) .setTimeZoneKey(getTimeZoneKey(sessionZone.getId())) @@ -837,7 +836,7 @@ private void runTestCases(String tableName, List testCases) } @Test - public static void checkIllegalRedshiftTimePrecision() + public void checkIllegalRedshiftTimePrecision() { assertRedshiftCreateFails( "check_redshift_time_precision_error", @@ -846,7 +845,7 @@ public static void checkIllegalRedshiftTimePrecision() } @Test - public static void checkIllegalRedshiftTimestampPrecision() + public void checkIllegalRedshiftTimestampPrecision() { assertRedshiftCreateFails( "check_redshift_timestamp_precision_error", From cddf0d4a2b5547c7ffa92d00d80b1b9e36afb947 Mon Sep 17 00:00:00 2001 From: Raunaq Morarka Date: Wed, 3 Jan 2024 15:44:30 +0530 Subject: [PATCH 243/350] Add DomainUserDefinedPredicate#toString --- .../parquet/predicate/TupleDomainParquetPredicate.java | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/lib/trino-parquet/src/main/java/io/trino/parquet/predicate/TupleDomainParquetPredicate.java b/lib/trino-parquet/src/main/java/io/trino/parquet/predicate/TupleDomainParquetPredicate.java index 6dcc6b0743ffc..09fedb188ae61 100644 --- a/lib/trino-parquet/src/main/java/io/trino/parquet/predicate/TupleDomainParquetPredicate.java +++ b/lib/trino-parquet/src/main/java/io/trino/parquet/predicate/TupleDomainParquetPredicate.java @@ -62,6 +62,7 @@ import java.util.Optional; import java.util.function.Function; +import static com.google.common.base.MoreObjects.toStringHelper; import static com.google.common.base.Preconditions.checkArgument; import static io.trino.parquet.ParquetTimestampUtils.decodeInt64Timestamp; import static io.trino.parquet.ParquetTimestampUtils.decodeInt96Timestamp; @@ -763,6 +764,15 @@ public boolean inverseCanDrop(org.apache.parquet.filter2.predicate.Statistics // To be safe, we just keep the record by returning false. return false; } + + @Override + public String toString() + { + return toStringHelper(this) + .add("columnDescriptor", columnDescriptor) + .add("columnDomain", columnDomain) + .toString(); + } } private static class ColumnIndexValueConverter From 4931f0dedc76c9aa00809ef5af1aa6b3a6fbe518 Mon Sep 17 00:00:00 2001 From: Raunaq Morarka Date: Thu, 4 Jan 2024 14:22:15 +0530 Subject: [PATCH 244/350] Remove unnecessary forceTestNgToRespectSingleThreaded --- .../trino/plugin/hive/parquet/TestFullParquetReader.java | 9 --------- .../io/trino/plugin/hive/parquet/TestParquetReader.java | 9 --------- 2 files changed, 18 deletions(-) diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestFullParquetReader.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestFullParquetReader.java index 29ba7bae611d2..6bf6e6243fcc6 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestFullParquetReader.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestFullParquetReader.java @@ -13,7 +13,6 @@ */ package io.trino.plugin.hive.parquet; -import org.junit.jupiter.api.Test; import org.junit.jupiter.api.parallel.Execution; import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; @@ -29,12 +28,4 @@ public TestFullParquetReader() { super(ParquetTester.fullParquetTester()); } - - @Test - public void forceTestNgToRespectSingleThreaded() - { - // TODO: Remove after updating TestNG to 7.4.0+ (https://github.com/trinodb/trino/issues/8571) - // TestNG doesn't enforce @Test(singleThreaded = true) when tests are defined in base class. According to - // https://github.com/cbeust/testng/issues/2361#issuecomment-688393166 a workaround it to add a dummy test to the leaf test class. - } } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestParquetReader.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestParquetReader.java index 458f75690af97..c2dfd1e9d7ee5 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestParquetReader.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestParquetReader.java @@ -13,7 +13,6 @@ */ package io.trino.plugin.hive.parquet; -import org.junit.jupiter.api.Test; import org.junit.jupiter.api.parallel.Execution; import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; @@ -29,12 +28,4 @@ public TestParquetReader() { super(ParquetTester.quickParquetTester()); } - - @Test - public void forceTestNgToRespectSingleThreaded() - { - // TODO: Remove after updating TestNG to 7.4.0+ (https://github.com/trinodb/trino/issues/8571) - // TestNG doesn't enforce @Test(singleThreaded = true) when tests are defined in base class. According to - // https://github.com/cbeust/testng/issues/2361#issuecomment-688393166 a workaround it to add a dummy test to the leaf test class. - } } From 87cf65b210a0fc5e7a096657809945a55a3a46b1 Mon Sep 17 00:00:00 2001 From: Raunaq Morarka Date: Thu, 4 Jan 2024 14:23:19 +0530 Subject: [PATCH 245/350] Rename io.trino.plugin.hive.parquet.TestParquetReader --- .../{TestParquetReader.java => TestQuickParquetReader.java} | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) rename plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/{TestParquetReader.java => TestQuickParquetReader.java} (94%) diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestParquetReader.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestQuickParquetReader.java similarity index 94% rename from plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestParquetReader.java rename to plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestQuickParquetReader.java index c2dfd1e9d7ee5..9488344d59185 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestParquetReader.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestQuickParquetReader.java @@ -21,10 +21,10 @@ // uses a single record writer across all threads. // For example org.apache.parquet.column.values.factory.DefaultValuesWriterFactory#DEFAULT_V1_WRITER_FACTORY is shared mutable state. @Execution(SAME_THREAD) -public class TestParquetReader +public class TestQuickParquetReader extends AbstractTestParquetReader { - public TestParquetReader() + public TestQuickParquetReader() { super(ParquetTester.quickParquetTester()); } From 0ff524bb5aa44ffbd11e241eeff0ae75537c07da Mon Sep 17 00:00:00 2001 From: Raunaq Morarka Date: Wed, 3 Jan 2024 15:46:57 +0530 Subject: [PATCH 246/350] Rename TestParquetReaderMemoryUsage --- ...TestParquetReaderMemoryUsage.java => TestParquetReader.java} | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) rename lib/trino-parquet/src/test/java/io/trino/parquet/reader/{TestParquetReaderMemoryUsage.java => TestParquetReader.java} (99%) diff --git a/lib/trino-parquet/src/test/java/io/trino/parquet/reader/TestParquetReaderMemoryUsage.java b/lib/trino-parquet/src/test/java/io/trino/parquet/reader/TestParquetReader.java similarity index 99% rename from lib/trino-parquet/src/test/java/io/trino/parquet/reader/TestParquetReaderMemoryUsage.java rename to lib/trino-parquet/src/test/java/io/trino/parquet/reader/TestParquetReader.java index fab5d62849861..67bf1a963e60a 100644 --- a/lib/trino-parquet/src/test/java/io/trino/parquet/reader/TestParquetReaderMemoryUsage.java +++ b/lib/trino-parquet/src/test/java/io/trino/parquet/reader/TestParquetReader.java @@ -37,7 +37,7 @@ import static io.trino.spi.type.IntegerType.INTEGER; import static org.assertj.core.api.Assertions.assertThat; -public class TestParquetReaderMemoryUsage +public class TestParquetReader { @Test public void testColumnReaderMemoryUsage() From 9932acd8e77e78f6b7d79237ed6af989c81d03da Mon Sep 17 00:00:00 2001 From: Raunaq Morarka Date: Wed, 3 Jan 2024 16:17:29 +0530 Subject: [PATCH 247/350] Fix handling of pruned parquet row groups with column indexes Filters on multiple columns with page indexes may yield non-overlapping row ranges and eliminate a parquet row group. In this scenario, the reader is fixed to advance to the next non-pruned row group in the split instead of stopping early and missing rows from the remaining row groups --- .../trino/parquet/reader/ParquetReader.java | 4 +- .../io/trino/parquet/ParquetTestUtils.java | 53 ++++++++++++++---- .../parquet/reader/TestParquetReader.java | 49 ++++++++++++++++ .../lineitem_sorted_by_shipdate/README.md | 11 ++++ .../lineitem_sorted_by_shipdate/data.parquet | Bin 0 -> 1582678 bytes 5 files changed, 105 insertions(+), 12 deletions(-) create mode 100644 lib/trino-parquet/src/test/resources/lineitem_sorted_by_shipdate/README.md create mode 100644 lib/trino-parquet/src/test/resources/lineitem_sorted_by_shipdate/data.parquet diff --git a/lib/trino-parquet/src/main/java/io/trino/parquet/reader/ParquetReader.java b/lib/trino-parquet/src/main/java/io/trino/parquet/reader/ParquetReader.java index aa9633f2763be..d6884c3b14146 100644 --- a/lib/trino-parquet/src/main/java/io/trino/parquet/reader/ParquetReader.java +++ b/lib/trino-parquet/src/main/java/io/trino/parquet/reader/ParquetReader.java @@ -306,7 +306,9 @@ private boolean advanceToNextRowGroup() long rowCount = currentGroupRowRanges.getRowCount(); columnIndexRowsFiltered += currentGroupRowCount - rowCount; if (rowCount == 0) { - return false; + // Filters on multiple columns with page indexes may yield non-overlapping row ranges and eliminate the entire row group. + // Advance to next row group to ensure that we don't return a null Page and close the page source before all row groups are processed + return advanceToNextRowGroup(); } currentGroupRowCount = rowCount; } diff --git a/lib/trino-parquet/src/test/java/io/trino/parquet/ParquetTestUtils.java b/lib/trino-parquet/src/test/java/io/trino/parquet/ParquetTestUtils.java index 47dcbd151d251..a12422bfbbe7e 100644 --- a/lib/trino-parquet/src/test/java/io/trino/parquet/ParquetTestUtils.java +++ b/lib/trino-parquet/src/test/java/io/trino/parquet/ParquetTestUtils.java @@ -17,6 +17,7 @@ import io.airlift.slice.Slice; import io.airlift.slice.Slices; import io.trino.memory.context.AggregatedMemoryContext; +import io.trino.parquet.predicate.TupleDomainParquetPredicate; import io.trino.parquet.reader.ParquetReader; import io.trino.parquet.reader.RowGroupInfo; import io.trino.parquet.writer.ParquetSchemaConverter; @@ -27,13 +28,15 @@ import io.trino.spi.block.BlockBuilder; import io.trino.spi.block.LongArrayBlock; import io.trino.spi.block.RowBlock; +import io.trino.spi.predicate.TupleDomain; import io.trino.spi.type.MapType; import io.trino.spi.type.Type; import io.trino.spi.type.TypeOperators; +import org.apache.parquet.column.ColumnDescriptor; import org.apache.parquet.format.CompressionCodec; -import org.apache.parquet.hadoop.metadata.BlockMetaData; import org.apache.parquet.hadoop.metadata.ParquetMetadata; import org.apache.parquet.io.MessageColumnIO; +import org.apache.parquet.schema.MessageType; import org.joda.time.DateTimeZone; import java.io.ByteArrayOutputStream; @@ -42,6 +45,7 @@ import java.io.UncheckedIOException; import java.util.Arrays; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.Random; @@ -50,12 +54,16 @@ import static com.google.common.collect.ImmutableList.toImmutableList; import static io.trino.parquet.ParquetTypeUtils.constructField; import static io.trino.parquet.ParquetTypeUtils.getColumnIO; +import static io.trino.parquet.ParquetTypeUtils.getDescriptors; import static io.trino.parquet.ParquetTypeUtils.getParquetEncoding; import static io.trino.parquet.ParquetTypeUtils.lookupColumnByName; +import static io.trino.parquet.predicate.PredicateUtils.buildPredicate; +import static io.trino.parquet.predicate.PredicateUtils.getFilteredRowGroups; import static io.trino.spi.block.ArrayBlock.fromElementBlock; import static io.trino.spi.block.MapBlock.fromKeyValueBlock; import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.spi.type.TypeUtils.writeNativeValue; +import static java.util.Locale.ENGLISH; import static org.joda.time.DateTimeZone.UTC; public class ParquetTestUtils @@ -101,9 +109,22 @@ public static ParquetReader createParquetReader( List types, List columnNames) throws IOException + { + return createParquetReader(input, parquetMetadata, memoryContext, types, columnNames, TupleDomain.all()); + } + + public static ParquetReader createParquetReader( + ParquetDataSource input, + ParquetMetadata parquetMetadata, + AggregatedMemoryContext memoryContext, + List types, + List columnNames, + TupleDomain predicate) + throws IOException { org.apache.parquet.hadoop.metadata.FileMetaData fileMetaData = parquetMetadata.getFileMetaData(); - MessageColumnIO messageColumnIO = getColumnIO(fileMetaData.getSchema(), fileMetaData.getSchema()); + MessageType fileSchema = fileMetaData.getSchema(); + MessageColumnIO messageColumnIO = getColumnIO(fileSchema, fileSchema); ImmutableList.Builder columnFields = ImmutableList.builder(); for (int i = 0; i < types.size(); i++) { columnFields.add(new Column( @@ -113,25 +134,35 @@ public static ParquetReader createParquetReader( lookupColumnByName(messageColumnIO, columnNames.get(i))) .orElseThrow())); } - long nextStart = 0; - ImmutableList.Builder rowGroupInfoBuilder = ImmutableList.builder(); - for (BlockMetaData block : parquetMetadata.getBlocks()) { - rowGroupInfoBuilder.add(new RowGroupInfo(block, nextStart, Optional.empty())); - nextStart += block.getRowCount(); - } + Map, ColumnDescriptor> descriptorsByPath = getDescriptors(fileSchema, fileSchema); + TupleDomain parquetTupleDomain = predicate.transformKeys( + columnName -> descriptorsByPath.get(ImmutableList.of(columnName.toLowerCase(ENGLISH)))); + TupleDomainParquetPredicate parquetPredicate = buildPredicate(fileSchema, parquetTupleDomain, descriptorsByPath, UTC); + ParquetReaderOptions options = new ParquetReaderOptions(); + List rowGroups = getFilteredRowGroups( + 0, + input.getEstimatedSize(), + input, + parquetMetadata.getBlocks(), + ImmutableList.of(parquetTupleDomain), + ImmutableList.of(parquetPredicate), + descriptorsByPath, + UTC, + 1000, + options); return new ParquetReader( Optional.ofNullable(fileMetaData.getCreatedBy()), columnFields.build(), - rowGroupInfoBuilder.build(), + rowGroups, input, UTC, memoryContext, - new ParquetReaderOptions(), + options, exception -> { throwIfUnchecked(exception); return new RuntimeException(exception); }, - Optional.empty(), + Optional.of(parquetPredicate), Optional.empty()); } diff --git a/lib/trino-parquet/src/test/java/io/trino/parquet/reader/TestParquetReader.java b/lib/trino-parquet/src/test/java/io/trino/parquet/reader/TestParquetReader.java index 67bf1a963e60a..3acfd1f0cdf84 100644 --- a/lib/trino-parquet/src/test/java/io/trino/parquet/reader/TestParquetReader.java +++ b/lib/trino-parquet/src/test/java/io/trino/parquet/reader/TestParquetReader.java @@ -14,6 +14,8 @@ package io.trino.parquet.reader; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.io.Resources; import io.airlift.units.DataSize; import io.trino.memory.context.AggregatedMemoryContext; import io.trino.parquet.ParquetDataSource; @@ -21,19 +23,31 @@ import io.trino.parquet.writer.ParquetWriterOptions; import io.trino.spi.Page; import io.trino.spi.block.LazyBlock; +import io.trino.spi.metrics.Count; +import io.trino.spi.metrics.Metric; +import io.trino.spi.predicate.Domain; +import io.trino.spi.predicate.Range; +import io.trino.spi.predicate.TupleDomain; +import io.trino.spi.predicate.ValueSet; import io.trino.spi.type.Type; import org.apache.parquet.hadoop.metadata.ParquetMetadata; import org.junit.jupiter.api.Test; +import java.io.File; import java.io.IOException; +import java.net.URISyntaxException; +import java.time.LocalDate; import java.util.List; +import java.util.Map; import java.util.Optional; import static io.trino.memory.context.AggregatedMemoryContext.newSimpleAggregatedMemoryContext; import static io.trino.parquet.ParquetTestUtils.createParquetReader; import static io.trino.parquet.ParquetTestUtils.generateInputPages; import static io.trino.parquet.ParquetTestUtils.writeParquetFile; +import static io.trino.parquet.reader.ParquetReader.COLUMN_INDEX_ROWS_FILTERED; import static io.trino.spi.type.BigintType.BIGINT; +import static io.trino.spi.type.DateType.DATE; import static io.trino.spi.type.IntegerType.INTEGER; import static org.assertj.core.api.Assertions.assertThat; @@ -98,4 +112,39 @@ public void testColumnReaderMemoryUsage() reader.close(); assertThat(memoryContext.getBytes()).isEqualTo(0); } + + @Test + public void testEmptyRowRangesWithColumnIndex() + throws URISyntaxException, IOException + { + List columnNames = ImmutableList.of("l_shipdate", "l_commitdate"); + List types = ImmutableList.of(DATE, DATE); + + ParquetDataSource dataSource = new FileParquetDataSource( + new File(Resources.getResource("lineitem_sorted_by_shipdate/data.parquet").toURI()), + new ParquetReaderOptions()); + ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, Optional.empty()); + assertThat(parquetMetadata.getBlocks().size()).isEqualTo(2); + // The predicate and the file are prepared so that page indexes will result in non-overlapping row ranges and eliminate the entire first row group + // while the second row group still has to be read + TupleDomain predicate = TupleDomain.withColumnDomains( + ImmutableMap.of( + "l_shipdate", Domain.multipleValues(DATE, ImmutableList.of(LocalDate.of(1993, 1, 1).toEpochDay(), LocalDate.of(1997, 1, 1).toEpochDay())), + "l_commitdate", Domain.create(ValueSet.ofRanges(Range.greaterThan(DATE, LocalDate.of(1995, 1, 1).toEpochDay())), false))); + + try (ParquetReader reader = createParquetReader(dataSource, parquetMetadata, newSimpleAggregatedMemoryContext(), types, columnNames, predicate)) { + Page page = reader.nextPage(); + int rowsRead = 0; + while (page != null) { + rowsRead += page.getPositionCount(); + page = reader.nextPage(); + } + assertThat(rowsRead).isEqualTo(2387); + Map> metrics = reader.getMetrics().getMetrics(); + assertThat(metrics).containsKey(COLUMN_INDEX_ROWS_FILTERED); + // Column index should filter at least the first row group + assertThat(((Count) metrics.get(COLUMN_INDEX_ROWS_FILTERED)).getTotal()) + .isGreaterThanOrEqualTo(parquetMetadata.getBlocks().get(0).getRowCount()); + } + } } diff --git a/lib/trino-parquet/src/test/resources/lineitem_sorted_by_shipdate/README.md b/lib/trino-parquet/src/test/resources/lineitem_sorted_by_shipdate/README.md new file mode 100644 index 0000000000000..d1b04fb3ad9d1 --- /dev/null +++ b/lib/trino-parquet/src/test/resources/lineitem_sorted_by_shipdate/README.md @@ -0,0 +1,11 @@ +This file is prepared to have parquet page indexes and two small row-groups with many small pages per row-group. +It required using release <= 422 because the new Trino parquet writer does not support writing page indexes yet. + +```sql +set session hive.parquet_writer_batch_size=10; +set session hive.parquet_writer_page_size='10Kb'; +set session hive.parquet_writer_block_size='1MB'; +set session hive.parquet_optimized_writer_enabled=false; + +create table lineitem with (format='parquet', sorted_by=array['l_shipdate'], bucketed_by=array['l_shipdate'], bucket_count=1) as select * from tpch.tiny.lineitem; +``` diff --git a/lib/trino-parquet/src/test/resources/lineitem_sorted_by_shipdate/data.parquet b/lib/trino-parquet/src/test/resources/lineitem_sorted_by_shipdate/data.parquet new file mode 100644 index 0000000000000000000000000000000000000000..82fd0166124d80cca873f63b0780a3e709d46c12 GIT binary patch literal 1582678 zcmW)md00~U_s6Gc%I{=pW#*I%Q&x_dGG(~~Q&x_dI_Z>|3YB9{nIdTlf-vTo<;Ikj zl?YQ>PHB>vxv{unsflK;DF}$Tplr$x_viDw|J^_C^SRGC=X1_^pY!_dXvjg!?=1f+ z+F+@yT5I{PT?+THYl6MZXMXuD zJgBH@->vSg$B*s_bp0iAYWssd@jD*~zpdXFb?^CKIlG;%4t^Ko@y!}k`C~q!^`-{j zrz#fByMNR{1tcAVjT!DF99RK>RHLp?t}+kqZU>wlxf}e$b#aAn4JAc7Ce*P9;FeDq zvJ2lzgZBsxQ6UWd*FrffDVPuuk^l^p>y0DYNy_7*FhW3xi*YmCz}zDgz#bH_2^T`l zj1T-9HDP;%Y?}V^x5vT(36Q=z{&b^{~<+1R+ahY%Y|`K?(q9D!0LKq>{ky zUhcFG@LL=rD?ueio&-Q2<3BcrE>Obtdt#bimad*EHV*h}l0QYfEeOUYfU z{{q@8lo3yL3z-g%NI>Twt_`SD6cyzY&d`B|a+;DPhmndb3~%%n_vI_pb3_<|4tUS2 zK)-eyfMFGoKmIkayBh#I>Hux|fw#Hc2OK5FC*?kKT?Rq~*7yrkw{ZW)l}JB($xt69 zB;<-7=#2CRNaZwuuvd5v&>J1@ztyrGg!7Vs?pDx0Mb0Z)eehmk62wt7{?#2?3G;CD zCcG6b7QWRpzXTa?r~~dFdTmHr+^HA(V>HZ=U`dIQN)IF8Do5DdCq+c@9s@I=+j>zF zSi}K-4(c8xcsVu$J0&upi+RW~iEue&cr4%~@J>A_%2n#U3+2bW!J0E%T>;Qp9;uIo zxEB{;)F^H6K4DZa_LcwGv_m!w>_|xU_k=qL3+O{eS2BPM?5-fRZ~@Ayc~VQcDZrQV z1Ce`#`P||!f=)%QajEUzgm1{X6@tXs5oR>Scl8agUHnXSpn#?#O%k@LRn2L(j6%5| z?DRu_Poz#KIo(YKVnf`3EtK~B7oz0Rp=|{ldA<*#@cLBx2!I5XnK9q6Y63?k_H~Z_s`t@q-c$6*x(BLcbIE2$g;Vvq463T1?l0_QZTe!>mOR zpybVnDO$Q*T}pVsZLN#y#14t>0f-vKy{>Dkvc&gkf|w0t9kVJrKLt z5dfNBSu{kjOAYz{8flk9WYkTM+9|yZf{jxwPEXs!Mc9Z~_{+i{kJwj6*CyOa9H1}D_qeLXSMf)@Xe}oK= zj!d9^0Bqrnmq19O^Vvc2jS!bnj4j*J80U}KV$2~TtM4BY83NEy4MAtKG{GyqC&~x9 z{Kqa4)*9+Bn%JT>VhA!(e$mE+;*T+}P#9sKfEi%>O2t<^U^~BwFwR;$#eA0!5RE^z z-_5-q!sySK1Wtz7&6Vvo#{0Z9(IK$LCuU9vhnTGN^v|UtVma-FZ*@M~cA_bRf=uL1 zdxk4zI#@vW0I=IJOrO-3;7v#iVZ_CP05Wi^{QASX2pBSt`w3Q3grNF2cCFO`z9F2q z?MP)mkKnE@KMGr%y7anRSR2vQB zlfm6KcwlUw&^}W0tZZB>G_8)EW%+r{wGeg7{2^Fz5w&{_IewlfXL>qH-ieBXyzW}v zSdF{EdXRcn!F^8VP%Iplm$?X5YAke3J4A1?9BG@*{P*{EZ8uA@V13F zI`vlb*_4Qi)Sg7_aw-}Y_0QP8nD3YD@&`$<7ag8!7kviGiq&+}uJJ_v!DS0rlIO`mueEt-I1*f48hmED=AIr z?1@d$wDpCEr6cEEvnqA<+K}UK_kPE7(hzmnxcM-FGt|>gDGmQ!d$5 z8+R?X0mV>)n(Yu1_oN7;4Bd<*SWEJi!$zwAn43#5fR&6?>w%vgnQBR?GI?sah+)&2 zWVLEbWZl@JJ$byn*jtu#h8}>{{lz`BCAD3&wR!(l_ILPy7rwM6#j@&NK}OI&cY$t; zn>+B^^^Gi8f@Z+nK+ZB}7LPAB`g-WrIlM>|oECwt{2M>B{gkNzemK6c)Y#1GI)Z=O z;~*^27LD=b?7(gxxXnT_wJ6OR0MVpsSYY>i)dT6`1KFegMhF4eDT);sulS5Jij5_O z^1F<<+OIV-_CNq4c?cY({wB5Ad8H+IZ(5muaAa*ZaZ32ZMRf6?(Mj_cu0tVZv=!{s zrte=eJTIwdQGJmUiUmJedVoQhc&G;kmBl_bmZI4qv{c8X|N3`0r6^yY&W*r#E(Xft zKS|)e#r#1BW!+!xFfr5~^yZ4Y?ILul-W-qG48ylc!i+_(ea>n2wwQy9fbE~zN|8OH z8*x&cq(lUDgFViKaz70x`634cd}Z7qT`@l+4mItLx6KWZ%MX=36#34IHR)q5*9C<~m(R&*LkCpVROgMuA zimzZ|E7QGuey^R+ddBB9>jbphf_NVvHpa}G)>62`i2j)HRUf}($UYJBkT*f*KA)X( zYEb@dPL*mNlj(BWA)m@DyMyGCb0JHkBlG5`x(_mX`yg7bH}o<-1KODFo{m5AvoRf$(}*X17-(4p1>D& zf{$qBaPq4P06%5i0p6}L*f*0;*pclTic07y@0Z)Qr{C2F1#=w35|6Oi zE~9>K@_4amjSFe^LzRRbjsF`$>cBWF_1A6Q3d_YggZ1e75Gd`6&HLmVTY`P~yu$|O z5e``X1tGVyj-fE%5~{h2FA3qLFr6h2Rqznm4c2*LPDrf9Jl?&_M#+t{|A8qQK*Qt_j$OO?gT7kemYk zkS&zrb+^n-1mU+Tos8i&dty%NF%H3p$_jInC-0R$OqaMVH`4-qAw+4+GkWOaler2P zV7@=Ncs^op;p8&E_m3na|-l;}m57YhK94+u30AIyD9PYd4Fhg%rPoyFps6|Fw^ETc z9>L3nc&GJq3S><$hzOzLoS0_0lI?5j(>$K$0*jlY9xiRPGWU5U`3GdR8)52Ksekbe z!EAB;hA9QiTA>#3M~)wsW4(JOQ-@E6`B zIP$0OfD2%0ZYoiFal0m2fb~U2UPTmV9DeUVRvs}g_O@rEyFk=A`M1r`G^AZMjwjd# zNpwOe^C_>Sm)a#F{y)Yxk@jXAsDzb14M|N#_`>>(@&02@I$H>7D}^-L<7&i@AY2zV zRF8432v6>OwNfi4b=Tw<>@j&^%F?I6Tk*;|;=8H%3oy}x6N^8bnD~6E+a#r-Z?CYx zuBqQVmA}&bQly^zg%HM*{R+U_2lh5&ZgZU$311jCq=Y+&kS#y_r;W zbEV;)`O@9olti_a%jvO&+wj@Akt6gF%T$`$c$|~60URJ@?15>V8R&mq@t2`1kBUxc zHU1=20p^^nEsIH@$9^&+2&uJFetv1b=A$$Ir*4zL_5PYQVDIIHhj8ul3h*pYi>K7z zfBTv+n9TlZX3c8d>6WmU`9{uN!OwEGAylLTJaZ%Vo>JOO9 z4P(&=F00-VglT2=@IN+6Y)4bAxTLX>_3W7G(6}1AUva-#t0$R`jtYJCMV}|ym`WpT zJgJOgotS>W?8DBAjdavj>Wk6AMj6=Fh@i99_lE>)71czKxNHTJDYMF<$4932i?D^c}C8 zi!@e$K2k5fS^*o>U;8%OFY!tjftAd%wZ$T7CCPt=8H!nZtx(TVl+Vw@l7ICh>Fhj^ zjOrF}t-jI3-9g(dhPd;aoY=rqK?wgp2ZPECW+p`oFxCb)EACK_KV}!$^NJ)`tNBHB z^9!;AM(yzTYj|9?@L8-sknrm7gj>;jF_W%vWNuFydhwG7?QfTr=38j~N&KsF5F&4I z1IYMRa0EKv{=hUX(Ki)3m+hdXgQ4MXxkp3FgOVqIRR4l|^THLrBB@Y<0pWbvM0?G` z`P6RCxc>bnhleZ&6T*D5|5^AfOWzgc>B!9M$#Z$tgN;0oNR0OZkG4ZfZ(MW;7o5-4 z{Ft`<9XA)vyhXH@zS@yuLyI*8tdX3}i#|ne%v^PkDBmMSsB$nU|C_W{yS4>bdi+)d z;(iXT1WtvxtPI`*#mmNTbJTe>pQxf+e#GR-lM~^Y7GdJ0sOBqj0XI?eKM{9KW*Pa4 zPruY#+}e0o)?ks%**_vTwHJ)s4!Eh;*|S5B6tJcGL&H0E63cL+4TJ}?WrJ>x#o`$Y zz>t=ZgGkZzd|!Zj+wO+_vqS$kYP7`tRh#ZXL(E<~Z9f~yI#LN?FUR{E`MZWKcTmlZ zja%79t??+QZV%$kJ%ABH&IfLG99fnQ0EqsTg*zoHUkgf~Po26nr4e7u^drpGo6(P4 zF{zgyPDk~@^^MV#C@+U-NIxC(WGC%{v8zx%AWB#Nb_D7^4%=SweCoF*p{kXsSA5G| z>A1UApy@NeBHt3Doi%|gqd9AX=@09o|24|LcxtaX6JbiACqBgOONF1rQlqN_2$#Ev z;%nbAi`Cd~Oh&~2QH_m%f=lo5Rh}LzFQ9>GSAImiQE5mnlOMgw!Z~W5#EHPjq-@gU zglR<~u$#gGZ4JDH`&#(8R_>L~BnS{(IT+)?Dg1%Pk9GDSwZ=Ko^nKj- zBaO^s3w*o*^>)9BkGjLs_I2Kzx0zNLHbS!S8Kp{GsshR7W}f)v%?laPhLozLGUmKi z_divtmL8lpfu(IF!6)R8a3CaF7MQY`!L%}+*y^!nDs(X(3Y-} z5nQYTyN0|Oy!hT<^8!HfdjGb)5*>BP(-AyW2K%);&Y<3uIy_?u;~~zFqB?T}@F{*K z)6&n|TwX^<{ZgfB!?KfgWp8s*&E~P&f}zuTChnL0Gw#tqR>KNpI8KrKZS$GezuQ)G z{EXk3uVh+Hja)D&mP6JQSk5)xs1hYy1>^r4_!NZ>x$6-{kVxZf6RywFQuXdXFNLCi zz8!pI)1!~|M|d+Pk>r=bCe_7rOh2!>HnVIo)rp*zp|D3?wCCfzW*!^=>wub}eMUeq zvBX~fIgz)T&2mHuLMvF&RTMj034ZS%9y?--KcBy{Q~qD^d~FC2>%q*!IpDZ$L$`1f zNmUX31znFEL|~?P8E!XQVCORU`1xq;lNEsDJCd=|9juPuYwpP+f|hw?j?)`z*b1~K;*Zh6Tj^is z%C1#`Mqm)blcy2;#MpKQb?(!esG))tTH-~E1+UAc#SP^a3r(7ZI^4-1xLc)N*7@vp z*)Sp2RV+G9ah6W~!dbjuaG<3zSVOv0RW@vl@QQ6AhM?)|z)8zrp5ywEPL8RG!gh~y zpG++0$aD|yhZ$8g)iJ}VWTU;$fq}39kMO&i2ug4#RkA$ropA_9R!o>G;lr>IaplI5 z1-V(MCjX^=uuWAv{5a|-WM2Dvv%u-VzVYHKv8Jg0@0Z+V{b!vVV-qX(yYzNIO_bxg zhBM0vk3oNoUUC>dlU2|k3_(9?wE&}&x9CK(${HEB-zr{4u%P7ihQ$vhwKKQRSb|0| z{&v;*?wl_R+W~+@J7SFd*ql5ydlWV!?Aj;xG>N?}k4s7+Jd3`~QP^RAmd)%Nj^URD z2*z`7V%61_y1LO%zmvRj^xxAlYs}gDi#<$6zKAM+%t^~MK`bk5nZNR=+jIv8O|Y03 zW;XD41p?ia6>Fi5JB1v&%vgiG!keNhmyUs7a_Q)Wut@Z-gt%_7xxr}3IlZil=q0}Y z5&;B#A)H3S|3i`yOGBy;L080cCc&FzzMqoTtHmZTrIb^tCx8;79-Dieg#Qj#!Z|nX z8Fq4tGoS8!h;VL>RUiNF z^@f}OAQ#$3Tw%eyY(G?oJMDEh5Eu;*ac9;7zmLU_HN0Q>`dT%i)<;a>mT3Vv*M4Rg z!#y)K_dJ^l5A={OKw`CjoD8#!o^&Nd{W%i9YzhDm5DA@(cjxP;6b>;F&~!ngDf;m( zibPtyS>w0ZEyy;(k)JDu(RwGoYf9t&*rpdcc}>Dblryj8J`?Z9SRb@p;qO^sFo+qO z)jJndR{r5H2!q;7Wy~b0+zbY3>8wUwZ&hHuWY?qmZ$(Xfz6-(K&rjOaL0^j_SQy%& z^)tL2a{U#ny7TqNsq|b^iUxCn)YxX_>7}$b_vuNs<6U11bf3<(G-H${Ezx1nUibO@I@o2rQ2#<9;ief;XaP~ zx6)Po<0p=*cz*Ts)2fYgqp?2eTNdEcQTvw#cQ~Xcv199q;cQfS9GidFSL*J~8910b z6(*pSJ8(@nV5qkg*h@Gq;NBDuc2NA(?oTDL%gd*1RysMzP9PBX^NY1f&pMkv&$m3bRSq2$Cj?Q4`gWw~7F2`h>$60l-X6loP}1z9T6qBDXG%b~OY;x1b`Z|< z@8TVch8KRPw37Q#UoKB>p45HgU|M5#OrN@~UVJ(I(4^)Cp&?>Eqi=N>R;qq*$$v=w z(H{byF1)aOYOVHN*rFm&PJ?DIHn_<_&TeJT^l>K{ss{ov#t)>yKupFV_L_{}hGGPG{p6YXm`6 zg(6t$L23nHz5HtiSrdhZl0h}K<_pdEWyLr6JlVNS&Vf0$W%E}oqEF3b8P^z=cW4wI z%)bzfS(RfX>?6!S5z zL0ffB2Kh6{s#=HgK3GG$2kNcFUOjyZeVn!KkIuPfsRO>~EOQ@#3>O_DrazkDT{(!@ zWhaqS!US&$MVj)$3*wn)4~lv+xSdTdT6>Fy3sT`O<;@y_CwBZe{v4!PE%=3bRyKB= zP_*2>h4TCC7jpwTyLcD{M|4=B8{_a*8QHhzZ{Bt1_?py+++gCn3ixIGS>SXu-O*?1 zl?V>eCSo?JB5GW$?Fwc;x#1gG#l+;JxxmIL*!fdu^L1GsFIj+z_+KmCdEr37^1}Zl z`uTb@gTX#%RN9^*Hov4w7xt^DJgf=y%Jkgr-VCel$Sk`vsL>3oOv$=`Cn3hkFWeJG z!%x7IO9WdPhYZ4taCVwQ0vZu>SQ(KcTPE+u8$S1KOZQRfvKp}8Wi#gLFVA|*mbbf@ zkF}Q_5@A%*#uLGxqM2I^4QH!o>mR0^zKnqAkZDWDn03;@zYSzrM6t2&-Tk-rc+gKU z0nK^^vrG$99vKOCXyS83YrU>pizrjvur!==^_YxUjq+|0koDhU3ZJwL!DmgqoIjac zC93n&^PnyFs@-y&66W3rSC>VkK&D11yQoxgY*GavjudrvoR!*@6@+9H)+SR>NmY? zKR0zN8+X_D)YC4$-p;2mtNw^U>lN62bT!esmOaXLg~#D_)plj){ert_wNyv%Muh=I zs(%r6MDY1xc6>*gtL8o_E$p)48}I8S^=WQ>u;r9l34>d(WL#Av=RUM2(13bgw z)0D5mxo&jiZ+B$p*+3u4p@P80^xO9Fm_g4dy(2iHjBPI8>xmND8(=LaR*8<_AZUk| zeiS6cCMgr$$PtpsE)1h4DAMQ4-wVF7k6*A|iPNj)hb}p$Te=K9@t8WIY|cqS%yQ#W zD1*pU6z7ng$78noA)@gw-yJE6UqoDK&r419%att%sp`m#0U|oyH8eU#Z1{Jx?pFLVQ4ve15b9kYDbDK+^lhAQ# zen8b6pCN*_++E*%e)_)rrvyEZ=sx2&7_nI3ah5djcXNGBU*yF3azTK6zkBlV1DRT` zIl%v}sbL*)9Ayk3m7E$mTvu<6`AwbpjNQb4xR0*ukmRydyZfvyk?d+joFMe3hXz(xp+Nejo`s$wv3iM^Km{wwaJ9_z(>fM-bcX9@kc+lLL zOzz(mZ&W*Sr5gyo97or6JVlI`TnRQo4WFcMmp->t{4bGap)fv0b{=7i|1@XY z$%K9+#Wx}fPgUUZq;lk6Qc*Zc;SA{StC$6bZ;}yK#>m?GZFboyQ7K{B3)bjEJT6xrP^@>3v!hM}h^f(|(1qa!T+4%9&8J)2zb~!b z<_A&Z%-AM+)g?A-ZNr;4O_9t zx3zP{Q-S*XSEvwmRQR44`@Wjoq;(w@j*{>W%+JYx9F3gF$NfgKx&-uY2M`@idsAdt z#Y8mH90(FW-?nf;Gnql>urxlL!RLq#iHdW8O>af2bT}VR&s{mfV%n2Ec6RxtoF9?| z7!kKjRm&G!(JN#APv~}Ywfl|CzwP6dD<7#|HOMIDAG6gpJ}dP`@nA;T8dcHj^SZhC3!x;Y_u)p0BM0x__PyKJJ7`iw! zcazt<^|91r?(HjN{Em4^$aJ)iJ)Klo^*&^E$t&FH>hmUL62YHiMjCJh1bWt316=)f8sl z{5gI_*8wgTH*F+vKBYoCaUj^E|NKlT2h@Wl#tU7TCo^RT^(T+v2T!bUZupWPtkRG| zjbqb)&+K9(jax=+^989Sg}=l0XxLWh)w=@b-)g&_^CaFYx1v-6CdQwlt-{( z35F|SdD_mnGdc_>v}zOM;?yxe+93-OvlHX1b=+G$HgkXVji$zKpirdv?LnMbb^t1J zJK~%b$4Z36aJ=EhJJ_*VK_kkLpPoPe)rz@&t^3sZR>@8jryO0ob)iKtu}Rr@59~el zF;3rDY?Xc~dy~%@nTF+2l*eAr4(oNl9!<_;ZklV!!#B9W*C{86Np3?D=V6C-9#-@C zi1BJm?S5}&G01VYVhxlQ#&w0ebQw;eU?I~bbD4Sg|2};ELulV6Zgxpnsx8j;s9$eB zBOU*T4zFSNcvx+;o9s-(D+$;3#vJT3FVOkU0t=4c~%JRK?_mEU2w2i7V-EY^@m5?(vmK^CE>u- z((QuoTnoF7LTB8YuCNhx!WQFS=C8zM`(2(WvYYgx3Ken%>6SRY+Hx)oN@Ar|Rg?|M zpSCh%=Q1AR0~d%k*!?@Rnr_+5c4>lUdKaOb@Io+>BVPU}T5Q%)7TZ5uwa26}VZIIlk^U zM*G)GBqZ3ZF`8da?pZ?|)q{)8NzG>4J+Wmfp{EAACX zL!elsIeAf^@I9D3)t#v-qF30cJiC42IibY|sXSG*@$0Rw6DiOw#S<+&7jFoQ5s!ke zXH4=X&phV;N9<-*?9ip<2iOvkb4(NEBO)z+BG^K-yOwxW`Yk?f6_(i{3CbVYOK|LJ z+{*s|A@#`X-Dr(@>NCv>6#NJzwcQTO!M<6VDqtS&*pxjgISzwQ0Val&y9Dl(>RWu2 z^L=?Plyuytk3;TYR=iBhsZ@3Enk3Vls`VPqtjV>ssWiKPW2-9gjmX49 zzeOZ%DY0(27s9SM-^J$Y__7Gv=oO}Kit4Ce@-gOj%)y&r?VQtcs;8s7{INBzzaq(R z`FuX*bu>_^EvgeeKn(G&b8rJYpzmg>acQEhORCG0dmz%+QEYi}Jm#OPBnh$JMKcLu zKE50Knap(q|DI(*)$6RNHW0!VmDRoz8xE8!`6U>!?{;9-ed(kwD^cEDabCmpNqy25 zO~!d*C6_bZ{7Nki!F&2qZt|i-nVZxjAwI@W;BHj)eHrt%=~)kyZ|Z^BB`dFsd+e&; z1kBvK)^@ML!~GxV4mmM=daff>Rf8B$vYQ>Ir4aLC;Sj`hk{JppQu9j_nxD<&M_U(qyGkkE&p0T7YRs`^QrmNSLoF#Xv(a?(tvXAvr$l-jH#cXapJjcs^&Yv zM235=4w*0ctCZdfM5Lt4^``3PGgPol{bOHA%U9U)M>9h2WsV&IkPF_5uFQ%-+PZIJ zS;^vGpp`oem-MO{J4z`iUsA8{zXL1hwV_LjDnAf97=@7`Zv03~jkZZ7d7a%*u#zIv zk8(SzVbbMs+UfSRN2MvB#)dXcq~+)j7@)202-eH}$ii$9SO4OQ4<$O?j^s3T_u6#8 zT-}uk#WMshyRwOS8Z=TTt9( ztY4dBYs@mLf&0|`tTY7f%eVjSHDawr7k53WO0j}S0?F3{-FtZfP-WZ+{BPc{*~dHe z7`c8Qtvj&rMxMKzUmxee$y@nV|59KGSgc~0D# zQA@@eVxyT5V_f^3_jmdZ{CKiq9dPw+eHChnr8yy8_^QY4S6bW!2Q9M#NtxE7@25&{ zlGu>I?kwZ={;P`(*`|bv_`(~$g!cl{M&*}b$xT6|nPEdLT)pBe&UFPLEMtSDy~pwV z=d0Ll4f7Tk{suTI4!bN_*YMSip<14|&~aH9f!xAYQBA1<9^#?>8RqhEs~W~5$nzoB zc$+k7imyq^+_T4NE|;T<(Jy}GKJPFUoTr}PE(@7KGx_kb#pC#wrwJ1*$YuUYfREln zS3ji+Cb?URI49wG2_HI|;qq-9J=~}^CB`QH(G@%{cS5pW+Oyx>Not*wRP?vb+xC!Q>Z|wqnp@acbxs>l zcAII!zL)lPy1PC!ndWZ#?Zhj6AU2AOp1{6lE@W8fwY~<4CJwGwpGt*V7t>Q>XTP|o zJlpYdN0e+8CS}D_8C0>(Ki#iJV$C2|%>Lo`cYLm&HeeK)N|;u03pS&@ee` z;#IW`^F6xhUoKS7phA!+JVTS=j{4k!4&;+gC|6Eb5Eg(Mye$T}z?{f}m#=k^^`??W z3Q}{EuSCJN%aG@6`rqMS>{uutn~a-?_`ryj|JAr!pIz?X7?FDdb1biCA9|*^CrIAb zUajcWD|eMvOEdt}HuQFU&C zq|xV9Ci;h2ev~b$wqkcBFn+3=+7RY4Ea*tLp9vj2ND%;V=Vp)m7bi`T9b_agozD_%lZBGF-yjko zvE4zEE$kl{@lwJm+(bX(WdrCD5F=A>gr1{G4ko+9n3f|GJK15fD2}dc1{tD``G#=r zDTvqeH}m!M$e|j$yi!OfrWXYir}at|3vQ0s#7kvxKY=WFb-{P-x}kGDtWFwuu&-p0 z;fEOvnF!lwcD*ZIB2N54_nX?>BREQ~I|$MI1p9}Tie$N(5V7)0$fuz)+>l4?EMPYA z-=H%>{BEZrOTNZ?k)DGc&Fg{o)n7tJ;FtGN{>dr^)sU^o2*JJeHc)x4o$auSPV}gF zXv=MBe#*7&IDr(F(>lPCg9Jyi6TkT&r|&rJ6QxVfk1i=%CtKXk?pq-oQA!r-vzRIt z^ApFERrV7QBB;PLsCEUToP%I5V;zE1wy6dz2T}DZlWxC=jbr$U?PMhb;eU3dl##UL2q2{&8_;`ZV|gCVcim(h0zX{1&^~9KezGz4JXR7-2z|8WE=x}iWA0Kw=6+e#^XM<1-k!c?S33mmrY8Xy4kQK!?G7ZyrQ(s_w-Le?o3+r)@ z$&1Jb9qG()N)v6v%&6_Kw8%dlaF{7~jB@N>Cq)eiM8*x)Zc1rv9dyMnZxP+-NJ~Y< zZ-K{4`w#Tdf5xx&6*&k?UTg5Ii@s*s?{NbtgD&C8A2EaGT5Hf#Ccrt~Nt->tf5IS0 zz6m5zMl*_|+wEw+>Z=Fbh;D+LP$+L5l+uNU6Q~ZwxvRKf)RH}8p>{A6z2_6*#k?-g z=QdiiTb{_O^pzZz@`urq-S`c54^m4N?Wg)weEd(E`yj7$E8Qp`zZI|8;7}=d6P$NH zN=YC5-c=udadm~3CiHxL$9|~R5v=+N7%Pm$SXyyT7$X7v)viiz#T?+3uHGgpQjF|n zR6PPs95-%d9H@Y*^vNB&v=VX<`ZB2Qkx90_yZjC)8Er}o^g%*@?S?SySoAq5@*9vB zKgYpCvo4JAx?NM$)ug1y>S-#{JuRfYCqOCP4WezJ-vWmwbK-k_2M&VKE}tY}dZQmL zG)4j*$Gpe?TUi>4YH`+8^EG%E$AH9V7OJq^vSKZ!2XA)CoF|h1l{xBg5_G-DU4Ckl z;jpDS^fL^WD6wB!TUSuApIb9f;2L#qbL@najhe2T8gFeF(gSMTJ1>ZkB*d%=hi5|t#6=httk#yZmnf-Z+F?#R%uxlztz&`wUimlrcWW(p}4kq zyc-3#LvepKxHQ>CwB8ndw4b4gZ<|1Q(wgtyK)Ra*nmTXA-8g&uT(94PC|IkGz|Xbm zinWRhF#Xg>Z5r-#02Re~gI@~g$~O=T^ztD3$xy(Y7|JiQHj)c8JMd^hRTV* zN%b?nu>?WeOY$0~#1hAJPXThT!SEUcl68_msVkvWyQ>UuEoLrMZ!84J-yQV`Zg^8T1uYmfY57z zFB0ZEcai3-BGg*Dc$*9#jCD1_c(K+gDV;#ULH1t6=K|aon<@;)6rwYYg=^ZoX@j3V z`UH%p9$o)%_UA@B=;M9?2aPRmEqKn4Ohq65hYRaqIHuKBarP>Hu@X12;oi0HSFFeuBmPJ2a#xr!U#4Y??`?ro4?C+nl|Z~mO1%t;RzFmdT^$J5Qxiw^uJ z9#$0kkmf|{y&_BF%|m9f`tUr18pSA5R;M!#_PL{oS+w*mRIni`UokI5jRRJEd=#F1 zh|}dU5|!SJLb~8Q=SQMQz%I_h;P?)DD3 zuK0$S(ZNUcf_Nc@Fm18BjOAR8-K-IGB6 z6l1ce1?bmi1kh~8_c~bXoHJ(>)3(DEJ3mi~#o&F4anO_|MceLGfS8tlP55X>vVCKVFdJx`NG5w<|;NE>c#p zQcwMIs?!W}?#V5ELQkBPSW(E$zf6ywK69D!1O8)M;d3K>yUD|j#X$8#qGF-9zO-}8 z4%C;>Dyye3mX!H}Dn~(mbxmiZd3zQDoQ$_nwbn57`&To4ImDH^aI2_h*R`o^z{ zjzZw?MEi!f+y$;Jw}QQCxtdQeQ+U7z+vH9oS)<675e_ogiT2hD8)+47T!-B91O4$- zc}Luecc<4bR4D706oK83X~thc&&QB!T!dps!*#F@a2%Vy0#MPfuGm%W6kY$&wYWQ& zc9t3$0u3~fCk-8#H{u%whzPO%VfB-olAwd;2dJ`RVUVc-$|E3??cKDvrmuIZ#SBWH zm8d=-+wUO^gn}gE^jYvIdo3YtbMPm1T%fnL_$nJlkog`NUR*5!4GpJ3y|Zo9210>x zp%d~9tVKao+v8n2+*4`_3)Ri!zN1<$H0(CttQbPEw*bweSY!ksNp&=!OaN+hX%b1X zWnt_xXfuwHqnqJ<%*HznY(gY2ZeOCeWcdKjosG|--~r35M-(@*U=+3$wcWd_qcMt~ zVWN$#h>BH`3X=Tzc5aFxtok2U;^Q`xy$$rkMx<7l(`S?cBgy;?lIkI>oo>?Cy{*fH z8~2vzO#sl`QYAe1YajnYQVMl^JdD}>`fU}7*A~2Rb zDlSlL=$zv&_r{A;TP*_7k^O|P<-bBI*&f4LAAKj=b$&SJtxRWQl9S7Vn%3=NBFDvG zK0nbb|ABj4^P@l`24k#HOq`3OO271--*)NL7Blf|ZIk-d4vcSup{H*FW%Ql)U19%4 zNc|!(2^x-uQVKrSl=$G9Gjc&S1-~%f{B7kjhI%+@Cd@v>f2+D00`I~EA2i<`{Qga~ z@L=DAyNZ^nQ~UZ<&m(Tv^xcBTwapdCT&%epahTiLo6_j#ls;-iF0Mg;K7zLVlzD+a zbvloV54sz6QXXVDPh1ZSk`+62?;zAbio_oFgQ`qg`;zt<(wv^B!0}=OIkF$w$spCw z0RX61)mCn`J*f#SB*p>9wK_^WfwV}doIIBzhqq5c&A9fFXcV&zJ zz3!aC5&qXT6BF&Z7kYa-Y4GVrf!*VV`o96QcLYma`)s8TnLSk-N?ObJjK(itw2KMk zUZ3Mq-{F@-DXx-{$Ej>p6_KYN-c0Av3QzqHb7gRqFW;pSwnRwle!_mXPr~ zNpE~A*ybF2CM)P<4f)5JN!cH}mUcYlE*Th0m=iLVDU>+`a>CSj(MCJ8VaBjLgQNcH zg359wq0{qUIk)}1pOq-ERS2=O;g8tBXwliuGTeYdE-c68Ya%h4B{xUHGbo=OC-b#q zKsyX)#P3d+oEsCk8*pwMKjlbPXgKQgo|%Jc_Hnpj5AyIWzZe5luJP)#cg6dnFp$2^ zRoy<7QKuj9j!KU)9HW%dc!j}>xV)qpwT7AoTTtC&tF*eP^renakaRJ(8vh~yBs+0N z|MQ)!S^e8TNb_5O0n8@Uxf9LZXvJBhq#cXRST)mQe9u(IHIl`4Rpm#3T?KlnP*~bs z{ZG^l)bqgF;^h?@GLfPYaBZC72)wNn&O`L_DAHy zirkqR4z%y+)xE+CFf-m25Ewo$x{s$n17kC?#ZJQXz(xN-Ma<%(JKQTZ2$NN$6KsKB z{&|(zRj0L5kY8L+fGl5-LAu+QCuXIXUV3vgbaN_YsU4K#U<($?*1I5V1dNgg^6N zd8lm64)a3prB4hT!3rXHCJ07Y4mRL2!3gsOT%80wHgI@p=srdBSAaQRnIpsHr%&j4w-`+N+ph3N8^d&A{3l!Cy3GWnrrj2bE21<#bK-W?%$*2v z<8|91t2&CqYN~&Q)7F5TpoHHk;k~&ZB0l9W{M5JUHkFcieW&t222ray_J3@>=Rehd z{6DS~l~87ONRm}a_Bro{WG4w(*-7Ep^Bg1DB(fdJo+o6FV{eXZ8HZyWE9V>sXFFe? z&+o$T%I80LU3lFd_vhoj9v!QCu#N4`FHOa}&0gTsE;%nE)_^8}8qDitZJH^mo5Q~2 z!-W#`Hbn)f-h2eE$-TMj^nNmve>0~fKpOePW}tb-+yA?e`g)@)=sA;N`+Aosgd9z+ zIw`^SJUYSE;2*XVV$n>^g_XU*xxp{)Zq<`!s_2mDpy@Wk7*S?(J+3D4E*nF*>BTm` zHyE$pg?&tf)lzSudB6~V`z*J-@J{{1T)ZIu3aazmN?=mZAQ9$*oUSLh*+CKNV=Um) zV!yoJxOA#0FXi@c{N3@(vBxZng*mZ)fJ@=tOBUr|Lug})Uz6jUBq>fzZGmDf?8duK zxzW{%&I#-dcz^Ths25ego@$` zC>$LUkT$1Gcd0nV?&=DE2urs~m~tLP4?A~EJ#HkUWsrZaE{8!TJv^@>Et2P6(FLu| zhsF1YwMpVOW*m9mtNk%nz8hZ5`+?msx8LhuXknP?E3k%jDBG#gPfi7Jo~_nJ^3ehw z)7B^pej8IKyj>_lvyAA$c(laYNN!<-6VT#qt~g9p@Tgk*MDZMEp-(oK8rO(f80sab z8C6E5N{)|A{r*kSJ#DBCQ33S{y8(M?d{9U{C7oy$>n1f}8q}nbfah9We+Af#bu(CB zRLQC6NiYNjrU)%*t=t|*#8R7;|0S}KV~*>+al$s=cUq;}PUKEuf2{Iq4!&CHujd3J zT)Qc*EuvZ?IWrJZIZ^^iUHwlX{rRD0-{925^#n7BQ!r%yX*RrVLKAj`a?BCnBs_}| zQT6T{YO+2gFj7XlaRS6PB^Y4@ICb<#W?V6zl7dp-Y+|;AvmX0dTdLT@%zXh#G{|4Y zSf2`F(-fL)TJ5VFSeZXTc%w;gQ;KNN%TJO*} zB_=p`%n<$(f>!DjS2zP*&iwruCHFh*o2p}ck)-Te z9d-zwnYhqe&yeP&NyzrY^Z_&GmBZ#hL3bYT?rBdswWpF7_pMDu9XANC6l6wkv8?_I*tL{Ibu=sX&!-Ms8a7O~KfFkP#F zDrd4UD>|;bq_+O`vatT66nZJ3aC2q^CV3t#-3nNp&?9X{e4!*voax1M zY~t>2@iHG7C>=!hBNY9oN_ZFdWk(=&-iYVctCA=90kB*Pkw>_4ZH$*b{$~&2{YP_O z z);jd7PV%J12w{|CICrQKP*NE@cp0;d&tnb-#!*$M^nAv>zu%`27X!<=ao=!yB#MGR zTOM+Kxw6e#?Zg6Kl0!cJjlT^QA-ou9R*5TA|1kGRh30h(IFt z18<V{lF6;tyb)i|BsGj z;5TNMa)e@=eqBV}F(bUcPE(0}x+G2{=!~E&6xZ#q@3-+gWYbcZ0G@)e>hQQ|wi1s>01&iZBd0VdQomf?P&W*eZ%U9O5ty!n@eZlgyM}H8=zvgif zdv@(yP|}yy#-n~EtKSnCM{?D8lJJIXzxNCa6cGc|I^<&|lK3gN<{?ilv%CD#{i|F~ z4_*QMIMH|b2-f@bbl|_ffCALl+wCWPP@7(i)TxK;AIi>I2J7|h16Vq5R&U@v-2122 zr99|5+{nW=^FC6whn*JMbQJS%r{fAe6ZJ=08F!6h(aW7aRVZ_|h>I_>#FA_!aDdn;iBnd(o6gk}iRp=m zpbn=&PWu5bkh(o7G9BZc6%Pc-Ki#vA)}-CK9wC6vUZq?2KHhIFg*vyT#1L<$r%L$-oSO^6 zR)%Y_9l9Tz(Ge>xHK)>t{zqOh2kudBojpPR-)EteCP<_RWJQ*1wk9CoCtJ*{4P)Toq z>ey_z$_i}j72%4BF$w%R3@AyQauCBTX>t5he3oe21AmKcl@)Fcwa}$44Ir$V=Hf_S z!r(hS);Y|VlmNXKC43dR8=ZF$D!JK-K6`P6mcRD&DubfE4%Y#M!GbkJzORX$#;?=& z1LT1wUWV!f+yZY-QzCx8zR{tr5len5h>W3XZQncvq$x?`TU)5ZFLMZ7gfe_d3R1|< z>JgMwC?ZYV55+j6Ivw|?27)d&nfD|~fkxx}rN^w(``0q*LMUsMlWAUOjb06j{U9cH zRj|?*dHpuRd3l>@XCCx!HzM~mFsW%#d1dmkVGZCm>M7{yIXODhDK^5T60<>&^*<}X6GH`)Ulh16Q6*x}w=VM0Z#oo`NmFu|NX@-IgnEcEFB z3Roekf19xszZMr!xwoB+Qwn7TfqrXxbLv~&O$XZrw}swxFr3l-LQ?P2j#@cf~9(33c({?c=Uws{s&%?9&PCXSh~fzQ&^y? z@T{@q@lxFjKV)5Fs`3vs-4vAg1?h})nWdy38C$_=u^71>42!SY^(e$|XURnEn7qj? ziWVr*WBbso@yxg`m{U4Z2kuPw%)zh?Z25!rF{x_@W@=Xt|BOEJ_C6)6aVw&K3RtSq)b ztBn$=lvo2`a+f(;|J-glacrSp@1(TelrDCF`=W+H?JF)>?_s#WGM!P$O_fq|mBSI^ zYm@avZv+y`Xu4g=xTFzu`fz+X;kbU^pBw)N73RD8C}{LXQZv#o{!F@`E;csy)>7PB z#C6qygzI1iORdg=9XuT+dGKRRs2Y%mFf{)hu>a2=;h9;8Zwuda9qIde4d<0V<+=j0_>Q| z3CN6Ibf;?IX@)KK{Cisq2x(|jI8eg>Q6hd}esMmy`raL%u|!>4ckgZIeI%~KnVLM1 z2ipg6oSw`3;;}6;ULO;%VMZ5|t6YuyxRHm|(O&uY7JNKtUFSi%d!WD~B+mA+E4|2f zDPusO8^#779U!H7{I~%8QEA8@%l#dhSnh#qq()_;= z1cE*I2%dd`Uh8SekfAvF0We*My?vlV{?s?oCwIRxOhnQVfhXN1XD(ybXxhO`a~O9j z=!qeq7a$d|;|F@Kc;KUC=>MYt&U9AFMY*$`;zv~}lB``U{J4464hJA!KbQ`IU8i)+ zn*dIhM-)Q(gB8_+67d5GBbL@;hIxMT6f?R_cS6hYZN(P5IJhpzfEiwzHp9#AKL3wV zjhyOCj7Rk035S+Ikh?I|Kj!!j*7<@n21WHyfB0iWbp;sfbI$@9M;WrG1&uoQsd(Lo zu-X{(SDcr$pswkwr>8Y8#&F7SXJz^aUS+rG4KSxI^egd_T}w{q4)x?cJgRbmXKcOs z03mcB36y8L0m8li!Y04k7d-0q>@*yT#8Omp&vFC0HS7}Usmt>i2SEYQiZo=GNH4;* z{^T^q)<1kPEDMH~I$Ky0$DmF}i&Ep5K1SN7SK#ghf#8!;4XsmTIA;J^Ax??M!{aOr zL4=i@+ISBc0bGSC*X@?sEp@x-b-{*xm_^ifGg)H~nbWi>Hf|D&f4%nj>_1CF$Ef)Z zcXzw4O@_hGBL6qa+%K|M6DhitdA5eKEiR57C@b}l2W-;dWre)fcy&fWfj+?cVBHP$ zcX%Gk+8qJ>ooazdpzQ86OVM|R_ePJ6#i=(>T>4bq{xugtbo?DTNPsr)dj?7UYv+MS zq(uR06PjJ#zxW#Isc%RX5Q++Z;rm^ev_$=s4D_2{dj(;52lJNR9kT5EIdthhpA_9#D;m#MyMm zsL|3mCEIa$sjwV5Nk?tEGnSxofcz@o8slYR4~Q3b#qU|H&qOc-YoJK4-TnJ<%m#nc zizH(Y`#D?eZqbi#x|2U6rBO@iW{?rOMZb@seETluuG-4#cXstjQ)`NqDe$}fvKn=z zcD)Yl-*2N2CXWb1XJGlcrdW)XS{9L7*&e0Lu0~bdQdy@eoOlZa8l4^vP~3O6)Pg4y zeP%$X^?~f`&@0sXhk4n>EFVxP{uOwHO;A9qQ7;e`>Z2$o0m@R-2OG;rVA4i z-XLU?M)wpq)TO_}|5V}at<9G-Q`Ek7r9f9u?%7}7NAo|T3Le0Rje`d|JV1|=#xtJ- z59jzT+ieH9@lW7Xs?o}?D|xygSPXTcJun!Rol5lOCm?!BC=kSFsiY9V9avu?(z<{0 z76Mo+X!cda^`Gll2|5*OZBGoqW?({LeG?kQVj=K|{Bz=?5gUfnQu_TCPmVA2lWym#|ZGTTh6Cl7;LsUYR~VWex( zeu07K>2YM23y?XgLKQ@dShPp5$)~T4pB4M`wp?wR&Clk6;u#0fESSqcG%u12DW@r6 zH~OiMQ*uvt=~`{T@O4h#$v5af9sVz1!9me9sz*dt3LCt*i{Ooq8FbD zhKU&7haCiI_)e`hs`;uKcT5~#{O2xJ(eGE48{&H+=I#UQFvgL)F&|JAmUWUK1+|S8 z0G~*(ee5R(t*3+upqt?F;}-alEAa9WU`Rh8!_1TwuH)UWqrr>Xw9JVs)ldvLjR`zn zd$;Vbs1A`$w9rdA%$3{CY=jLZ=^Y!&9!U>SHVKCQs2GUS#zo;?6TJr^o`(p;tq1i7%cU>_ulVX~1MuAlAJcN|2DBTvuZGcj|~vTESnCgyM8pf+^_% zeJtwr50mg8pjff-KLo<3rB+coWC)t*;oj@^=swsVclQ8fOCX?D!$9BxE42%6u9)s68wPEdr66PR~FasQth*y2W8#@x^vAc*yqT6@;YYLv#meIvLn|FQf%OdtqO2VO&vciNY;96t&xKsKLTD=Zg~-jqK?9wKrD9YBg# zqAgsDD#Q_`)kY?A12FXCc2%|3H%a=v0r@uSkW0=st7rXCDQC7QTH18Q2kRp!ed#Uc zEcYbP58!S~KBF*to8xN?d+@?L?Q+(o{j_%=YksT0F+RZ{HdTPxa|-SxwYsURj+yFp zNby?z*0dhGEJUQ-o2^5!FQ=FHk^Yi#=756y5EXx)1Q7z5ehW$HM>LQ>3AW6!bo%W^ zoNenEjoecqJZUrO0^4Zzs--s4i_E5s3NMtWytxJ8JaiAxJ>{V>-sEX64 zM4cXxf_9s2Ru-r8kjdzxfs(`fjv(V1YLBI0ODGJ8g{YaJFi@OpU%&7}CNk(N%(YizZ2vTZ21Ts|uAlIr1NbmGr{*If4x8L^_U#_~@*E=SgQnE( z<1@>9kXu?}M!$ygp{^R)dygc5w^sp;aavmtt@+%}Vy$!i_y-5DjgpZxLSFzw4f_o@ zy(45n)81c#$cEZ1NZz+HE;g72(rg6hJznl_zY3xYKy|trA5zDIkm}>iKjOe*?$`Dl z*40<%(J^^EAmLb0LRlS4I z%C^L}+E`9_z3FO=)nWA5KeF$^2(S7Y0Epa%miRrBjphYPL>vC;64GD}FxjI?+Jq?& zh!f4}Nvn_Fgw%SV7W~1zfrVys&j-d6bggKIHS`4J$`B!`%7UB^XW4BH1NLNK5<`@< zlgkh5+;PMwf@f^+7*;oUaezhIm;S;%trFh+aGaI&OIW@2D93j{>8gfS64$(iRl7XV zG);~ePg|{#zj+WDS%ExUSX3r3tsvLpBB(oWEa^{0Gc^LKO3z=yweuFX9q(28@gOua9jjG$E!{kr}$&D9tAhCv#m?hu%3JA3OL6^!dw-^`j1v}N5zKkEu^bt zV(ng8@PV}NCQ`vCnjA<7KBf+IXrY`In}laHm^OZ4F+pGr#cCERY~)lG>v_|_`mMbA zpsRInoIUQ~u@(9`R9#=>obRzH*xT4|jCO};qDA!&&9nD}ZgeQ&PK?@MC^f4<7)h|j zdcI&lP5p>f#1)@za{?a*yucj@u<5PgS%O!L2Q>5{MX1swUDQvoo&4 z*mXUPlCNN>ihB}Fj-;y(wOG@7ng_coQLmt5fo=zn-Ui zWIV9fxInmo*gb?^gjY-;HG*{c#ZL~5EyBQDYUM7}4ctZ5F>;xMkcMGYnw@Y_fberyWgfB>DM=r>j` zxVLH&2J#;S{^3}wM833`ten4z7Fb~ z0U)9AMlX69u4>+b+bwvN`4iaBq1}DF`lJ#sSgwn}CKsztd?JUHj5DM|);y5G&bzd{ z6(lQWgI6CTKY?gPI@h3&lh^Lf}Jz=iq!J3awzj zhf5m3c~kRTr%Qf>c@Qt1uRB`<-<-iODdmXu9pGM|-?!bNr4jT}`BQ!fD+cZ$ zFRfmaK}(zEojdFgBX)#A1bj8VF^|78{YkgagA;=61Lr&Qn8EewUb(1eF+#FlNveRO z)Ibk=!jhBlA*rTiM_PkEWQv?W{!TcaG0q;sRH@}U+T?Un+2&jin;QJR6X-Hy@B-mK zXy)|h#?W?=DR+6o&o?wSOSv@>A*4tt<5B?T>4{-6B9ufePIMRPl7-Il%-!Do*0{qs ziUDB11QB0;p@?_{$CXrb@YcCcs9S8997DAN;kJIx{)brk-99y9NdCo3+#K; zxmJIx(KG$sY*Ptn&U`=7;PBp_-`vKlG`QdVx>7(#T(&GeC@bi!>&8^6GzBdwY}v<{ zy<=(*v=aiPZCqbm(sBiB5e*(w4=NaGzmOuwaanEl((IPm^u{*QNE>9$&jcLk+97db zoTfBB^eHYn2MtyC`5_6SULJSDA~bx1wN{dLEdww<+;3Vx@KGRU^{4+$_<};jc`HDRUNfD7($4P z3PCdXl%GD4_fV8)cEP;`Rt{Gpg1DDsrBuN?uYCD~x?kxjSF9CnSt+8}|j zNY6`gRinLK3vp}@vX05lEkj01>yv3Hq@l;gUuEkJ#<3&UFh6m1$E?6)3= z`Mqb<^18(;>altV1{a+fBFfzCCr;Qd75P2It4=$aGE7*On?m2FNKPr;SYfS{r)`2S zQ$O7nfeqGSNHdITf@SP&!*wTK1}vCQ(OR$-HEl=_=^$X>6xId-5|YX)d~iKAAoaHp za%$ArLZ8pUV_RSd#@)HyLXflxDIs=gk|j5mQd?~K3=H?3)m{KKeVdW<8puJ+dBnd> zVbjCySk4h$q%+LDhTYfbWQCJ7#35`|W;%xJRq|=R+}1u8;%Sc5hFg;bI`iw*0~63} zwcvQAS#08cHJi>jsioXbdPzP4-Eom*CU9VhtMkHy^xCjZ|FQN&#J3d{C0gEs+mnLs zQ*KO2s)umU#vS`}TP^~?UuowzrVC(EOVYCR?TfsljsNg|C|nG+Dl_C2s#lz{aAROl zeGlse+`F@r;(99IY%>U05;-;3&Eq2xm;Cd3Bb>%f3lZAmBTA#DE8af=e~pvw#7x<* zzjqY+LjRZIw?MT(?eeL?Z^ELc&&dC$^6ZDWFbHg@A|(htwE2=sEG8%}eZp24`<{q* zJgh?+&}@Ndn<2Oy@ZYd=_1dP+%A<8HE28haC1-4l#)U}Go++2vk=a%V3t~VoM4p4ax7i*PbSsfIiTRRyGz2RY_K`@3iTmK!x(?5;Z^#Rgt?fy;RqD6q_Z0wJZYx`D)CiRNJ9 z_u%i=t*Au}l{b}IGxIYgRPV3kD0QdX)t@S#5~#Gy4#WZ_6=}1cZ5+Hdrs{N2{(Z`T zIK8ZN_fUJzjt9CTnvHJl`hr#}P2d+fmJ38hlAVx_Y zKXPCASHE+*KjA*b3T2+K5wW8704^JBcGwz(YpeqQ%4-p7YYMYx#KCpk(JPCi8fX>x z=ctfPD(@0d7x*OLuOk`U6L~JAqo;&9`0S*}8{(t}AgFSM5br!40?+EOJ`oo@TnuRo zKBon(Ot*A|gLh&3-mB_Q2hgWPLvdQg#*hz^z9ABVD|J-2AOjC+Z0&P1CcvGlFTBAO zYCfc?@V1j_=PUd6W0-0=KSUAPONAzho60;+DL9%DO_x(QADZGh%r+qwk-TTbMS8Tn zRwQUgwRzyCppI{g*{Zu=tUd-G2x468vW<^7+;RItP$&*&2v+W5aW0G}92$>LhyGo*huF zf#58evF+b4PW8I!-_Kz!G8}al` z>apHZ2o_^{csj1}Wn-mNe0*U{5S+UiEKhYShQ$Pq#L!^N#|I1AHhf^ScLN&4KxWD3 zRJ(4l!P;ME@~7KL#c1ZXkUMVQry4ntH?T04Q$U>6+9FJa-64mCSbszrI84%<)%7&I zdg}2F_Vg1cse$L@03zvfV+T^!}UBlkQxbD-if%NwSt}ZB&(HHu&xq>g^}Tntbz005z@Z! zK54)=t*;pN;DD$)M}%p>UV;C^-n!U++E5<*>Z$HY{U+c(rEa%b{cOW2qnFNR zg*F)^xlKQtjdnfYzoXoH2o>SiV;U3tHQzhF?Bn7!I!1JP`;py@<00;mjUgMsxrC*1 z{K_D*f2W|w?d}3^8s**)E|Y3{6d(^hvwZ_vKdi<;W=XOB=TwchZ&g=lsiSNt8>*j0 zC`Z;0#|^E-uM*9%Ej{g=15cm~QPwXpe+q_Sa zsT0YD=Oj;ER3l%5eb{((D^xM)?J%0W5;P10tXU>NP@_HBw32YP32W*Ki&F5#8}r+2?ao9*)ofGMqNNvJ)YjXj2yV zaL73heU+pok9Pdh>MUdf>6Gt4x1y7peDq;H=yO-hF`aWt`{-G4`28SidWX}#^G&!4 zN0bwBm{ZMSqfHJ7XR$^LZ!a0LD#36+m`}_~k|H(+4s#$LANAR)DqHzY|4@o&H#uj=j90j2EIYqGKC8TCR`n+`h<%Q}eoH6g2Izd) zMK00jh1efr-B%cHnf?BHvpYPF_tLxGulFOKvkEAAydTWBAGjc_5lm&^cs6*A`&sOb zn*8-k-)`xo-P4}z9pdD8dGlGo+LMR`7S2~MGM^O1eixUjes}p>(FG3KOAQ`7sJ=no zd(z!suNB-JxN-Mp{2kDL#qpU6GEP6V3z)`2#a&xMZKs}(vMCuc_=jSd62B{grZc`O zL?y<^`b^F@N0fYK;R>j{_Tr}&0xJ4s3tKZ;5|IZCJ%4%kLFwZgY>zJK-FpFi{vr z>*dI7m_^eAhv6bu(;K4iEpe(-BXjcay4pOhKhnF%W#=6`mTVt?J&$!D@iIf$kC*Ze zLv42~`uEKgoDn}XQ+Ph|*MF$3yk73L$=~qt@9U;;4o8d7OHoZe_!IpbcRbYWA~}0{ zGn+s$gSq9#Y!(t;=!v4b$tSs4KKp;o6^uu7vwbnIs&*JKuhn&N8;>cX@2wkeuqA%C zeVp|AfyFV`e5>CEd&ebG=()*8d11H)jVa{Y7VKFigaMed+S$j?&3$gBdEvIl&8{mK zS%shBoMY0(AgbOsM}K}c>Xy{v`oO{cIsTkO-0A+?j4KuwzfgHjTrLfXvR(E0PlM}H zvca>MvD>dkCtWQ2jK9i!U}(6=`#aA6p3^O@XY2jkQyu@__SY0hst9w3T;O}<(3!mc zBJH6$u1h;rYN*qw`BjdRy^KXQh{Tco^~#W5XS~$Z^I~yZC&1PJ>WVo)6(3h}@^_$Y zjI7?PS4p0_RXL^}yGIPVH|{g31#E>%m6RazECd-$u=)SR{osi*yRPaT|M^^w+q^ir z=w^H4lH9opyfWw$-n4t3^HT#fE?WEspCb zSxYt0rge^aGNR^(*G^iwz;y|UR!6@T*#<_8y6y|5t(okrPg2^0(YS?}pD-jm6vo8! zF=?!~>obem-o>E;t)8gs>3S(Wu3}Wmlg4i46q^~k+S>oXbtOLG8hS8yFW_{E{ozVjKFqrM9YdbY zp8~HZq9N<=x!5o|*|!}E)_3Y(Ui?{|WjJMXuFM3+^FHyGYjBkn?@x2`F|*|_{oa-4 zmnRVmj~V_{{~JE}!=@3DtJm^RRPLaynKg5gwuf?CX%ESxunN|Ss}?>9VR188o>*(i z7X4_xFn1p1=gFm%=(8L>n4g$Flw(rx_3tw?{U=YPtJIbUG3E@}mX(L4H~3_22HRAe zC&$BpQRz{IH@0tQVDk%Jd6|r?{B%tQga|wDvYCuly3rz&_qvBB%0y+SFT-2ub_i?l z#FKyQO;fvahhLO1$gU{f*OQ_##wcU&SED{u_rpzG27>zSQVo84s^X3N`;e9@`P^ME z)Av$A#s4z%z3IYWsqeU;XAQeSm=7dN>sG3;9r#Yy(I-nzmo{)=?KSm4|M!q{qHzQq zdLZ{dIDuv;|7Z(|%!qe6&71h20nET~?@A>h>LSDc%e4tGj9+fH=iuQn|D}Ka;bi(#9r@{u z$(sXTf33V2o;f#a6>%;jC*fOj_*?=qAaWgaY0E4vF?v5ElFy;CtH4R}=zW~#JN>#i z^JJp(tS0b0p9A*^u1?DZ*Y+g(>r;*)+vMFH#xx6Vnq#4)mg;ARr$1LY!d;q$P9hv# zZe>1U`E;lGR?5zw`3)8xWzwxwJyCzwSMNHz#P0Rgdp=HmRsO(Hrt&|&^WW11wFLD| z-VUV)nXW`TrU9JpWTaUz*-t*KC*fm(3?2(b={Dgzmbw)_T2J^CK1&s4*niW0l{vBe zJ~q?&17Dtu-3Ir0CWCyw>v7$ME&|@q{byfEIprwEs2HY-oAH0(e|<|(ly4%(-St&> zv7&tag3tqua<{27;7!^OZ;vLDh>knTbIlla&rc-B<;pr2$o{eei^x!)6gp>|wlTti zj}(ndlkYfA`_$DNE}W}%{JJD}eD#l!^2+j%1XLwH)EIi9Z!G~DwjXoNnD6kiN#4DO z6BlI!7cSbdgFRJ*Vj#sv+&`TEuIv8@f1bSj1M->YuG!ZwS(!5-X%Btw=tu0=iT>2% zc&zlCBMOq}=Wgn4!tWi65@LM!^%t|16GW{*lP?yW`_eN0h6j4P!mUs?{JHbR@hmYA-{f0YSL-`laaE1o{Y-b# z+kPhj8vJc@z8Im(Evp-MS}GJxiPPs{?PRjF-}iF@ALF}q9(|AQ;m}PK!A3I=n||Vs zn*&_n)9pF+{a#bbeg zWlsIc{FRg1N?(6t-fI?L*D?@TPAV^Y{!h65jq9f(46D?%`mY6l*#A)B9nGdaJGeZR~4z;%e&sZ_uB8@IlNq3)Ll+@jmX0R-6i3)EkHBY{O|XXdzG>wT!NCt=SM7!r#+{B^?cReoQJSu zip6@xJ2>x)K^;`4O*vX63=BAd7g_s0cBfYcGkjAs=;N``V|GyuQxbE~GLHfE)47Xq z{fszDoTVxsrLSBtJ(6ueC}@AQS0qA4@3&~rop;4zN$T3)ZrQhD%_Xu04~95mBObs< zR?`o=RmLIJ zBd*_}Do>TP`Sq(Z#D7yBZfUge$JD$Quiavhw0-Kz_s8zGf}LK`dz5XmX8g&I)ZWea zE!SU8yy-5xn;U6+@9Epg46Zu2vSQBAm;7>1)+M54s_cz+`KGeoWdAq);-!H?oa}sl zTuFkREbyn<{U1(ArPrT+7`D6SQ~X>J`)>HYi8yZoHdVU9>eFkp^|YLaMWO~C`!XN? zPFWCVzN=Du0#+ z%D5D#JKy{0*XIxkV|)9d`jzwIHWLHl4#wYyR~!X(Ls(TXcNc#=LWvq?{`CX>_1B|6VLoL4SZXM0lwxwr@i$*;bL43LuE~kJ z`*-OLmrP6Y-P_@1*~>Z4gzZ?j-26w(R*o7<+F3WuW7jO^qJNgamh8pNSY9EorF{L2 zOE3htj!#yjz8Q9XcnK#j{``E4Ybo`z$KD^l@bi+&o|g>_cVA4eW~AkV7If}s*lGO+ zJXG#J>+<+-_wCwm%{!_^TgGoG8Eb3e90gh<4&6UrP|2w1CT7l;R6pMfi+n7@m#F)4h-d1m z*Tn;nXbbNKms8E4wwq__$lryWTDkn!Y;r^Ydi4HU6>8HKUf zK2Q=G%fIBzV;8Fv+MqICGvdkklr1#t0rX{nj&m-{l`M5avrgG%@3r3Ui|si`OA8~bqWh{?@iSz&ar)bXECYgqy>v#sD7RkcYV9sQ_WJSSj^)x z&*bRm)+Ny`AlKg9mUpAt3NA|H-G|p0!Sz0X8F>C`x)hzSt*F1t zck~{AV_vW1^}Pp{M;+oK^Ix?CPokNFVnTJ z;>C1b2C97SPWgYI9W`oKj(??V{Nuy8%KTh(t)9(1sFX5kP0mS*Pt6Avxdk2XqIS7XD4+f6>76qa2w zh(A-BaYNO&HYf%pGv^L`KFk54n@65yRz9i6xZLNo$b66PAjpndhsT)&{4jyqamRQ;JpyTq9OFDxGTwiWsI^HHm- zRfW&&PwV{3$wyslNvb^pX|E4U;Izw^JS9*fc}<=Y9Y+|@$90ZIs$?lWl0Ed*eM<&i zN6~X^U+ph^oL>6CbNe0lnDt6BC+*7n|Hl5?3xz+viTJ^pbD`nmQvtG22-l4^9#Ay| zrYp3flU6-!$qr%pJchmY=B~&6UttihAMV2bo#j`r>2+Nm?`VlSTa$-EuEiUziJ7VF z+|zl^p*M0t%1rR#fc=XWy(%Ufv7=+!6a7W>Yms5Z5RQHH z!SBb>MHj*M)fZXUZhlv{F-eLjTYQ!5>)60{FsAiXAz@JceN~baHsa=mv_ebg+G~?9 zZi#KoUut~T%~8GSeOui&dwLUCT2aPuM#zx+Yw=H)dDo1w>ysOI= zo%nrGk)N&N3n?xx{iCvaT|(tYrQUp7&n7ORSQgit7xEH0Zr>eiclT4z z&}wdRgkz>r^>)^;7$=V9AFBBNZs^tzLlImSI->tXUdFI-d?Me-F6%1@p40POLG_M^k3<1*l34 zI$2w)`C{zK>#mAFJTAR`av!90{;G_P5H~cnav_I#i0Zv4kx^DnaM=Ns}Z zvnD(9Qf=pF_WUy$KNN58XeD;@eo;-kU)g8yu2TBlFWEtfj_GHA8LtAf?hIIiuX=9{PuL2NEFIi5;D?MIK9Y{6d?1Bb?k*) z_KuNm>wkf|+?89D#bxlP(4khQD(hsZclUdj1pn_(3r8n125)pB z3}Y)zSvebAb5KaK8|r)$o%d#g-J`Q`mgX<>5f)uFXSy z;RUbK|9HU;4sFCex8~o`|$YuG~9(?{8W1zcPp$ zah&ybd7|MouGX3=ET6q4TO4EL9lKj3U-SJpgVn!u$UjD>dyz41KUuv%4{mKqj+p65 z`(oYu)wgClm7Xbpu4jgcuXCiRls~vz7cep^K9o^bIc4-OeeBt_mYjtV=eHy8%BFZ8 znmYcr3MqZ{C$+e$SmnisH&fAii+PIQmAVCOuQ|;1Iz6g)Df=|J{WfvKNy(%h`0JcY zB*Rhh{uQFWuH7f!!oA|7PN^o3*gCt8(F}Q(re}^9VgHYgkz)w*^*F~nruRU3r(TQIu{Da7w{a!>f;%6WB4>~h1nbkx= zVu8sK_I!QoC`jQXmQlB%SK$|yRA1mjSU4hu`ZrCrDz1-dZH8I$CFbF&(~6QHiP=3K zzNJZW&N>%yqar>wZmQ>DCP`%A5dmG3h(Dfk#2LgtCci>CCqc|mRG{*2f1KZ;4L3m; z1;rxO#gm?yiecG~39KX?BcneYdek_fRF zw*pd;J)t3gAd!WN>MLCulL4s?rj}`A87v5{oRpxvb)&I zdz`yW{SIF`uk0odho!t|;k5zuZ;usJ*Gv>2-U{4bc`r;QFOH?Hb-9AD zO+QNwzIhYXvCdRSIxNXw2A9N&*frd( z8>xPDu<=LvqgQuzqqfoKfb18c?!`6}`mq!;I0% zWo{IaA>a9n#*kWFqGbfVPB7JSCabKOA>x6Nxy14w(o}4b!yZL_(*So3N}O?dxd?;h zGESk>zs_4-BPIZIr7B`=>U=rd>~XK2g>7vEA{Dyc!-$r>a-=m8HACL8X0LOSJJQ{m z>E#bm24c**x;-tkoFA+PVM*yf6Nom%$_88FI78$M3|I>arzolTHTj>=;RpNS`{;py;QX-B`YzuPRxM_?bMkU zKGd2@15Mc)%VhVPlUl#l;4aP7Qtm;_`CHKDuE6gb*Ku4a`y7{tg0S*q?+X=&{fv77 z>8l*MIPJ%(Hsph}tuvs-@QH%~7qgp8Ca+I-%K`y5V4K zOnTEqBhr8#Z}~vA%XB$xt?F7Xrz#~9#BI1q^rF-PPynn}Gqzkx&u6|>p!BOY0%T)W z_1sQi`*&#+gN59+er8-Y8C|4g^?zMdrJMRq+YlG3h5o9v!6M6lPz>u@%EC8Kq+YgN zIU6(L@@TvO2s@~=oW=O`d+b8b8fbL$xE;Ro1(?2PTX!^!^;}MSr-ahhU>-sm5(dbW;IJ~oAy!yb;uqZx_g8yAoDFp%M<^R%qbrkL{~}g zE%?ENajcXibsxjOk>~o3KPhp;_75zIRiuD_>&Ghh;8THp;$@2#4$0ieXS=<-Z0J6H zdP*N7zkv=s{(`GBR*#nVd6KMWADMJrIK}=ALEf6SZ(QT3?!iuS9*=_b-^Mb1$ z4e{>s83@PJnVlc!D`4@b*skdP!(dZ5Nck||HIhH0g9&ryV2HZ+f}87*Ou0Up`Js!+ za=rEd1MR%);m!_TqgzsSR(rFlV3A{w`T;rp>zU~__Pw9y0*U4Fu+~+(eayf++doJF zF(nv{DP*E^a|-SwA{&-}0SnaRei^zyNm2@M8S1`2uITvVT(Vf=HaMy|40@)tp4Kj_G#;-U;2lNKx_Pg9#IwSZV(kMuqrT zbNE=Dt#<*)gjjF?3r6uUR{sa1xBtLs`0O@su}wq5g=;1*%P}4pIxX;$G8X!foA9H3 z2T>ZmKk>uLj?dDZabhkup7A!IyG+Jx;6C5By)dACRrjRjen!xdrVuUSP-b#=B2D*N zCX-kEw{_aA?*4hU&a2Wdh^ARVkfWLGiMjlzdc&`Gb{h`(6W5@Yq4)DjM-TcF#d!B* zUPdSHP4g3Bh?hBLDrO?Z%#*oq8uJ9&_rEnWr_|V1LQ~(ZoP~er8;d398lKz4a-CVw zFSzAZ!W8QYyyP5<%UD_0sN(?xrl}Q*q>b0JSoMW*6~6 zwG2K;Rv1imi?+y|R%Lj9W@wd{_KBR)Lo=5Ql%Hxbsm{_r5#%FjpGV4q9L&%V=j2-! zme~0@YZ|RqPMnn{v;=`Qp=;_44$A$N{p6)0E5eJ6YRlt+$2E$hX9X6`%MjL;z|2S; zhR6ATlUdAlr{P=px;U8!RL6{Y48Rrgw3g5;&fHh2^EBO*uzBR-`c}{LtP|HcA%Qx+ z>P+Bhp~fmwA${>VzXT&8bz#(z@I1Yg@iC$Sj9{ z{O7~%YHgpL@;l+W`(|>9VBbkmwZ)|e*K;3ef~`X|Y|PD`FS?M>vE#no#{2QL@&)by`_3?ALBBj=!eqQ;IlpVP!4}_?2V(H zeiL|`jbwf3Pr<+KzFr$u5GJRoPLQlyU)K}ZZfj)H2=ZohVY10k?`}xmh9kCAgpAsE zFKk>=12*+iWF9zX*F`kIfC8c4C0)WFwo#ctKFauZ!HZ3eRoK*W-7Wk3X-Yd7MXvq* zmO%7i`zdUaGVR+)3xZ46CLABe0XD%^n*r5bpy*RY<`1J2R>xh!B2E+Ntcxg*x7~~T zduQQtEnyPJ4$^G1bP+-6*yxNsS`ORcSHPtG=~InQK8KI2w|&BG-PBB?^$3to398I< zpZ@Yk@~y;Ha)W0DqtDeS7or0)EcfoesORvnV=t6ht{J1m=do&;TWtS?vV_1p31#iu zDudMn@?aC8V;K$R&9$`r2glaHaEE9;w(7+M^9hDl>CDcIBKS?7NskRl(N~@{sr$M~ zAJE}4#QO{eQj2b&?)e-7B?-CDzb2Y5{na*JY zXDXp=4axH)7LL7FX4x)Da%RC*5O-l?+o9>r;+g3t_sT5jLDR?ioao>&0%PMKhoF7Q z%I0-^9a$5F?)}`BsW<>2aIy9*)d*%S;TBy zuAJ(4lW7JxkG;0d8B!l}KzkqTg}sfXCNntEz5kK2%o?EGLL+bCZiRr8G6t z>XR>j(^eW@7l7?#LbnTi>2t!9&N|oL4na74i=fMgQx3WH{!jdejDGhsW1p{XW8J!1 z*q2DvPOZlo?}`r&`nG8LKXcS=4;pIk-pdvqdS6Q_`4mF%`2R92hzP5X5bJDrl=pv6 z3W>UrY?Xq+!M2`6^gr&jWqn)RIZeQwSDXMe`kAJS1N@*ez$E@p!@tkl?Sr2}m)*P9 zPaZ-)o0cyimMkjl;9VP5%B6`X7Wyl|-B%{Ezb&a%v_cy+l%F<3@BtcIy*9&=l9KLT zj4ea5feywkZS;}D@kLmf$%4O^qATu2dw`Z4p>=G61D&ze7av0OGgX;HT2MTgfcAYDCt%`;Y4)Z4%~ z#%~skN$Uh13zwNtmp_MaeyjqGw(P1O;ZfTWe0>t1<%JI@7}v+{gvquh7j$l8K6!Kb zy6X>56)Ko((C3))-@?mDn4V8=mr5mznDWv)Zu+lsN5iN(UhLZ-(4&CMjk#@Z5+bOHc6m-dgv*RteX^y(UXSgzG>OPY-<) z)kkXdk)Q7$#wW2etgyZOJHrW&Tm3gYonWUMk^moHkJXkH15SctmJJ-sSgY6JGw(o% zGjc#FsJ^*dJPka6U`^1lI80y@xY9xIO1#TUl9&WbydP;lV6ByErqm}udW%E6LLnRL2VX{r6&F$r#^ zi^>?*^Ni>my=lrIERXQ&tzAdrgz~{IuF(GH2R`FG%K~sndD~|7wNZU6GV}_;;2q#w z05H-Tb@0?6EsZhtoS9aH>F1{Sug~R!Y$6fjE1lNu zA&b=Uz@I)c&(Y*)mf!&7Wz<9u9HQm)h@*HAxWw%*#+!_}wbPd$ zkXz#&VWMU2YS6f=k51&9l-aA?2(}d!&S(zZ+EuGws}On&mobIM{$2r(m7ehxm7gV% z@wssqLvu{lrDRRi1ipLpmmy@>r&ZEvVx~alo%}tA{a3IIMUO66(y`JgWeBx_L;tV^Rr9R z`R+lR8Yd{&~808TTUG`YVAF%g2rK^gY=&=H^X%Im*YL*j_h zJ=VUrYa1`q2m}VVnycUS;d4o7c z$FrV}t@=y0f05`bg22Y!TSt|8i&v<=V}5$wk29qhC>j8RVF80=;bZHQQTy+gtQY~)Mu8m^rxdCajO~h_#^|I?Om_ zDec?#A$mI5W2n&?7>c#BXk#}q3i|fY)@*`H!(dZ5Nyb-v-*icFnm7jQp zfjrJ#w>Z)BS<$tG*}&61J|245)|J`o9IH1S&JSIx5AHy1vjNk#_+l9nGd_= zqbh^D_Xtanq&!wTdOeT`br|8?(CR9LSt%w(aAYuCmS)3tDE-Ld&yz*;e(%|L#*p)X zae>IL0(P%v))#l6WBLjdw$3X-UgZJ2H5yPB#b?8C=N4VvA<6RQ_dlyy$UB39&(NNW zeLSwxVglQpD)u&LVZB%}z42%hRzDoN1I>CvLP-_Z*!1OTd56%X&sP$cZO|VP*#FzI zB@tlV;A1s5ws(Fb#Pa7# zTm%s6lD*zO{Pje1bTU26-^r8f-MjRwI93S_wJwvv>Vl)!(l@W=7cPK^Mzpajj!(I8 zL_QAsXneL>IL}&tJL=b$E&OfTjwahfHWr+8{a?8v5ee1vgywP{f3gciyipMRQ1^i9 z;unk{O0n`D;@=!*PC^BDz;PN!t`3zFfp<# z3A4vPTE&9>?KjL_SqDh^&@KDZ~Nj&&yI|J=o+|qDZr)g+5{#zPkcYSH((qn2UPOB{&TpC;we}ppEx9THkSpT=dNJ_@CO21P-btEBf%vJ`ZfikEx7%E1ikqFlw01ZeWL>kTW!?^uKaBVq$TB{ctXkrjU@ z0YY_=EN_qZ>jxbpLH02P9w|poVT0Z|f3B;FDzzM`$sU4lJP=Ub*K92Vq9X9iSI zrFJxpbcJFK#JilgS{xz&P-6kJAN$DYLf&@Yzu zCVz_joVX2+^h7w+LwFh&qbrC%R$Cyip`RvN098n4$5O965-Vl>Ydq6Dv|ol$+5dAO zSP!E(fn@|sLTwTnzvMQ4yDnslbq#|c^SgcV661UhT0pyUXu)3HUu#bGaiEt?0pC|VU!w>a^V%ypV%rdla$FpRHntyz-*B)qjwRS{>$9jeXjh3x zKOmn37RVg#f2LMky3O z^Zw7HIUsGc#YazuaXX%j%uSQnLnMP&^SndJQ8%W57qg@6?_n(=d)iY=kePRs$ zXt4EBbtSafJr8k9DvCbuTM}`)8Qxl+(|Sp)i{)NP7ueVX2?z2-0)1LqCkHdG5zHly zS8MZ-f7&jzvF8G@mj=Ez4We}-S@dX34(E~%L$Ccv?6~;43K9|b31k2+H*WwRl zk95!CKNt*VL0J|AEkm`roN!Bi{GeZSnup&w-N$i-hI^F1J!S`lXjpwh*#@zOxFXeW zg>QX=AuSqFj{a8-UJzn+e7W5h182UBBQgKOI0D^`;HbP04yI2$$O#|z9YqpV!AQs6 zuLhEP1;Aq^n$v8XfRsSZ%o)X>rZC8_EZ)NgoTcQFOlOZye?4@&ZK*t%ejqeg7Um>j z|ADnfHf$_c@iSf~#*VtE`B&p*IuCG#eRb_vK8u%5Jh>F2aWVje=lG-KVBBCbE-#Cw zmaI2&D#D=Hob`@_u^jG8GLx9S{q5dRf31~z@CMimgL1x`m0?(+Ark0f!XaL7nsRFQM8z;0P%$8v)vEg%LKB96*O`;9VR zL6Ca$L+beSxa}da?42OIwGMNXax6pPQN2z#5!U)yn+S4d9Vcw8j3WV#7ts<|LN`H1 zk>Z*5Ec$v>05RPe@mrsZ6H(>$gZ%BfOCJkf#p-r7S5KduxfSeOgYw>5c}RO2*F~8N zFz6buG(|fQK%Hp=u}kQ*cpsLN&Uv=eK2Ev-h@k(sN-!rE5+z{GaseY?3W(YyV!zy8 z89>O&K($oH`;1}Hm%X?&tHQDVq%idY4{fULra7ug=6r*9EtSzH@qCFdjL+ru0ZKn1 z6X__JonnxzGKMn^FX;|FduQLjoU|netlQjXVqpBH*m?Q2zmAekT-0*Gc(#^(&igNl zWXDg%$V}La=9B`4-#aF>lVwhg57VY%wlT>oSj9QwMhMbv6U~o8<+pa&aT>t&%Lj2{AVNZnSI0t#TyEj>gD2SWN zCe?JZ`u!`rDE?u9+U`Pc+=OkQ=k#9ckKBzBacCafjww)l->QOXVrhlW`PtC#yEr)& zVzWTVasKC_C9hItbk8X(F57_0IS$IM2;Z6F`(jpWF?afrDX=J>v1EFwe)3=)8!WwoVNjf4y&Ubz6Zr?-E!Q=d zJ?b}K<>A#%5sR=a>PI5oP5BLqKOUP$2z82O%?z`agZ-TkUx?FS390&27F3D3=bN!5a^XclRZ;KkUvMN&xR;~sG(-@8e(XSKk$dV zGGU1%&49!qR<+>g;IEf9tw$HoSN6%J*gqx-NbCGYtm78ZKqI_QQ@j%g;H5Rg2H9Py zLlm0+cu%1w)*Vra5bPBi_yW-2K)B)K?`u)sw5ftrK)@j8^h(Y~xYRqYXM8~Z!x8mty^k?frn`8+Sn3tY%agrSSPXQr|Hj9AqQZUZk5 zxW^a=aGE$mN03s2SZ~p>E+}G~xmq2&F(pnft=O2vNsXdBx;yY&!YToJa~9rn_T zKl#1Nz=uM5nu=Dm(e7oqS*b+1{7{~RtQg}Tx?_ZyO0T%!vV1jH?zFOm z!@uYh=YekikXBkOojxLiR-E+(s`puFI^L_12R?pUBtxtgDL ziG+UcPxoohcez&dlsu8{g`3g+6d&5Jlr=|%UEJ=P8lom{yd_1K@kIGcoX zeJY0A(I{O5iOU0*Ahh`zTNNNNTxfxQ)&`NWt|(+()X8-dt@1oivIQ%2oUkph%e;t2 z@X|R&Wa4cg+crdv2CPu6@q&6FsPz%4L~D@1&6MH%n66fc^QZxmOdfbt&RvIbb!MR5 z&$)Ty;avh|2~)iu^N@Hk^+QBrAOO^f)1rK65lGm+DB=;>;PrNeSAFU`;I4qqn22mc zODZJM%QW0W(mjFIwl#-7cdYi8KD)V)Fb03(|{dgy8Pdw?V5<+n2 zCx1IJ?zCq&;$4p9;YWciJzP0Su5PcUzkt05{b>lt4Ykzh0RvJimG6>(EgSznyoqB+ zCDkDvkCYieds&3#gJowV3%bO#YAsoqhQ5jZnO_M1igC!%6b$;t?^E2<+BNi6KFtag zR6o)jBGCBmkTu=(i5LunHGyJ0RBwK(#~y3Vp6j=M$`Nbt@~PBy$MZQONg5}{C_J0w z!2BYEuSHHMSjkdbi>9mD1pCiDPz7hxb*RDyQ`t+#mTI>mMv=jmNr63EL1EK{*NHR; zzK4%v4mRQ;L~t!uz(+h%T?S>$G&man2c+3*eI0@gN|Ii%Zq zxJ{;@D#k$B(5bSY<(hw9m?kBz*1lWng&y%MK+W)F?NXrou7S>u$J~)+XKScWjUkUA zuS+y}=Ryr)TFb)VEzVla5Rh5L18_24w(7B|wda^P;)G0r7{-vJ8f?^6&H4vzK{*OR z;BWg?^(tnnrS-p)c*d^~S;$s0OeKOOz>FRQ8{$a5dgluRraR%Q(jb!1#N}sS6qm>k zVFtl$U?L)JV|&-9CD~u_&(FCKn-m1HIB<@>hqgH9`EMS1DQnjmt~lcT5(*NTHkgA! zQ^z|L%iHBg^@pnV@l_+kEi0+N6kUg8V8PM{!1h!Bu@S>-ooKWf72nf##V2kH>|>l_ z3Dv@(chIP*W=as`TlwVVt?OKE>*TwVWOUZXOKHYw1wTHl2DkeY@*_ML&jd9Dc692I zEL@N)A7{YEUJLtb6W!ffoQFtm+A%~?=jn|O&a|} z4!v|tsS)}~B*hI17n>dLbDX3U{h!$Gm0|O-q@9G){+lj39_kd^%Iy> zefYAu(=ps()KaN^q60Mz;{s{2{9`bA+B*!)SBO2W-$LVHR|r&x2)zCxU`@2(Am z(ryLxh?&bua7vG7+&{xBa%xhS4A|Y0B_C>rlMl2#G3tNz)5rH=BX~{X#ABfAU_hFk`z#RLKOWRrOu%YseIcT6R);6fL@_O?Gv-gyk1o|kpSCYan zTr(spDk~QiS%gs*t2nh&^cN(J6?h{de@$%k6ro+`%{73Q+_x z{LR$a9|eQ#{@fi(7V6#?09Cn8^RFyLVK!4YndgnP+X`>}b~y3H->?K}V>V?WlS@xP zKSt}iBAydO{u=s9K%as9#bCMpoiY499pf53fwy?V-9C(ucA{YJb(riE0S`ogfT)p? zokvK;om8RN29_d@k!$ao#E$^<+Ua=08maBlA0(p2IRg)SD;4Ow%G*aZEz`0N)#LL? zq1bO)reD`LHVMt@G6kCf;^O+N6z!Wi#F;}FLLOb7MRttfhzOcWyzy2yOUb{*8^uB& zMZ19xc|~g?t3^}J%`U;;FdRA1}EFWq8*X86k%ER)JWPQqv2x-+2<@kLWo zlJ>va8nuh@DrXC-&_rZtdmG{unnyHJvdY(?(;&*2t&$ZG5&SG~CPDMc^Qhow*JI=5 zTTM+($Kgl;(PfD_r1^`u{7}eqi8_C9b&Ld}x9&WbjfJCmH*YRtM+^-~5(Etz6PxMs zuK?LB1~fKrj;l>dKzB$kX7PRMcMRNfk-?tK z{LES{6Z4riAQE?I9T|Dj6kCypo3e`T)Ln{Tku)#is$DPy<&c(|*S_9Bf>a=IpDKEr-nC_|?Q7=U? zR^geE|GI@5WiO=~PD=EshXu$P--MM*a`FSNFnHeZ#UpyW2EjqbBTtCH zl_wZf-P8`R>^N}tLS7@Ik`teZt-#`+Zz|S)R&-(`1##q2z%WL*;fBQ<3Ial#{4|-9 zzJlKx3!Ab_$rb0YMcTnb%*c+yn>>>0HCzw5!JhP8|qV215 zOd}w@VeY*}ZLw5y$yd<`A2^-QgAV~nJul6A1G034mu{<`xqIhEt9yAqf`-cMyJ-{8 zN84PS&`#B~Xnl`pQa-~AyTQ1obg-I2m^)$e8gq`E5{ic$^azeRcP~^|j%LjqBQX*1 zGO&Woa=N6hbMO-AY^R;@XX%N<$U8U!j}UV2qu3kNu^sF{t4fE-4j&#>w|Z`$M1ekN zeH-HbC}8pm3y(J33(FYG1vekKp-zMlxm-xqJhW;(I&;FSiy+g0h~ROLiH1)qRs-_!(0pfi%l(V5U17$Bu7Q%SR5y zhp=J~eq(L<_Iu1|morm}6W6`w$ax?p5G9FvFmayW+l}AX#I$_EN>#Bv#Tb=?0jTfr zjjK@NZHW7>oLdDXCz0PFkB>pp1Mi}q+nxqkuwtZPc)aRE=Vj5nd&h|GWV$EbM@Oi; zK#Qu-KAZDJRt@!uc5?d`>QK-pUyODsY#nHq-}<|Drps^TUl^r&8b;Ys7c!5t7SZ<~ zab@Fw!g-rRYeYFtKLPI8@FV4Dc%B?-wZnBEh44?D4HVm^>N{JA*jq^WZ4t19Y z))V5SD4o7J3g)5weVHQ|B8_|eUc^AIFDP4(p!Fw~>Wsmv-lbZ81F4(b}A+Ao` zq+}MJ;MlC4-YcK_sWcB$yjj?iaYRi-7^b zIVwR>)*C!L$7R;E#JTK!{Ud@#Wr_OP{Tw5~b( z!YITRHLd)ACFlbI)(ipGKxp9*0wGq*|5``Rzq)*_BfIS4@9&ymYK{*KITZ+erWkb7 zfX*PAt_jU^mRZ!I*?G<`xp{LPh;v7WaQH=#HPDdMVs)OuFT>2M4wg~y2KiAJ1Ky?y z3J&Tbd~hFaOILd_*b+M9v*$5*@)1C%R-U4%)61vKhe@EKd2TBs-!FHG9G#*)uAs5t z%*7BL#UYEVZU9Z}op#x7RYT-~FT{b_*3;Rn6rmS*QRn_flt`5Y(V5?4BSD`{ir@v) z7L-94yZli5Aq3gDIj4z&Z%2W}0gmzRH2cxAgy)7h^CV<&O`RfNazGy^q|B^Ik2eVFc&F9!0ayDeLzt&+}aMe0Xe;3u9+ig5YHU5 zAbofisYyQn9jK2#T*e7*2xYsgOwS2f}Gz8};Li+fl-eXk*S8*b&t)X7ox#!%@ z<-_9yg>V(p@?5zG1yB8w3Z(T7c_M<`1rF%zu;`y+haB6B?Q>B4>mrKtoFhPZy`yhw z?AUD5?}G&AGf(QZe%X&blTQ~qMl|4|mbC$YeFK}in?uEY$g{k|58H+aI(rzXc*Mg< zl}dSndS@YAHTdn|g&naDi~nw{N*)>+ioDn9IdL)BzAq9mkg`iRKg2r1!#kRS9mg)!0>um5X0 z93>e(&uC^T`^kB@hayuxClqd*AJ*-5fEgt}DWs1hw1C*bOFZ&;@8j}YJeOUvvoP;?^0C>l6c5msRShB~{-<)#8@ zR;Yo8qkT>~R`H~Du~H2uK!^T4wi&2id4W}A6xueyzhh|!1O-17lHAo&Yf6;D8Ftyq zPi1H@c|ei^U#$Q$TWhQ_G6YIOzm%hhWzMD71ClR#L_*xscLHp;A=Gw*DUcCEbZ3h?I#%iej$E%%7XEb2}V*zEL$ zsUZ`zSkIm6#-zG0r3?4Dq(HJ%J*io~gk$lIB}HEzg{%z)5_j-OpN&Ki_$+bHDz7gQ zSAM4(F7GMQ+&9Bhm|XUR4a6ds(wK6tb@)J|g1gau$B^*oD>ivJHoDaq6B2*a2_R;z zeA==Qzl)AAQL&3)OuwH0duIV~6Q`Mg9S@1URS&PS;F^he#wn=Z|KLvgWvnc>7-0fz zKQ)W9k+7Dw%cC;YB(Z=Ugh1?jg`qQdc~T1qSg!x+3@gemT`R6|@Hl@pZLyWY{$|Vq zhK5j?9?uYTruKbRvS?t2dtsD?$*rpXfnaeMVec zAB|I;$a%c5tjny^(y*H%TXJ2zOIbRwFG_C(Ii{TpR1IPL-TRJpM3Y9sS(S)Bk89Z~Wa%GFzvl>nhqf zmCHDl))Pp}QAn3^AE^B8jM-$0>F#_3Zvk8=jy8s>ImRTpsX z9!1fvu05qVpZfmxb*L+NWCuzk13jp4wViG0P4Wk{Min_CHY)vR+mtsZTrQ*^6K3DK zyR+P0Gm0zrzTcye zu9koczIN!jG0vPbhl1Q>6Bhb?oLI4tV*-87Ywoc|!k@{y;7tmxrlx#31@Db!CR`cm z!Vu!ncV!0I|1$Yl&rr(R(Tw+SNUb3S*3Sb6UiwBr`)KoR)#w`d$X0u>ZlrvryK$Ih zQw|=aXDIBi_65vZe{78htdX;SbH^P_5bzVxEX@A|TZ1nYmjx{AC?g7-~UjO;OnsDPyb12`G*0HC~2_T9sF;G(^n55KX0g(%3u>^n=J|EvP!!#+~qT3 z{Fzg;Xt80P%F@hOc4M6HUhGnPh3(cZsbV9CZR}xug)q6+Mf7~_y~zV-=xV)c)$Dnq zm8zvbXyOVPWtaSgjx@FM! zPV;1lpG_^3VmO`4=5M$BO}@=foglC#Lf#>(7c$w~NU`CwhhyqXtxr~8oy#y2u|FJJ zq~dTNxY6I3__C-)w%_w-AKun!7h+fnMb}q1Fu<<)>P}&}2Ouxb^SQ$u(-LIgnutUn zjn{h|F6nAd_nJgc93AsT4_-VFw9>gqUX9+air<@|GNK{#`9hcNdO2 z@oN@rnE0h7h+)+;-mcp24Rdy~UMOvaJaKZ*=F+$U()E6|*0@}hZ7oZ_uhSjdwFXpn z*|mcf_e_ZDYfW@3+cH}Q9`4)L)TM#?lS@1eRXAG^nM?6*J}SWF=CD_pB9whgRfyy}-Z22cs-aK6e}W z_2wpbwcGM9*MyB7%DMngGwjXZjLcb^;)~f(0WYyrD4hTTckl{OW=m@DUI0;Gy2pX+ zCoFQN zz`+rOJ@c|L?(ZBuemTsp^ze8`z%SzAJ2|o4I-0cmq#1qdPR4W<*l6Oi2er8+VjmD) zH_x9;vpb`m0B}N-Z_?yea9RDfS%=}gRIv!;seV`_&ahb zW)Or+KEd4-LU4+lyne15gncI0>uYu4mA#P_98rIU*#IJ#Z%^C++j9n|)YkVgfKDI~ zN4wqQ)`qUgtXng~t|%8M_<=cF_|z#?{#HN`0CUKzEwt*%1S)Ihnc%vn)jhoRgk`K= zZkY_84av$7u`ofZu4kiJ9E;EiXtg|L9NA}^8a(LGJvz0`wYiJ!cq%-CBeGkyE10~~ z5#Z)JVJ8&l^Fn3n&iVS$7k5B}r304y{f~y6!mT*n4uBWu*0Fs7q~ho@zs>*ef<*)zYGV^1PXC)#VjTr@YsT?y^af zzaaT_NE=Au)e_xz-ocG?Ou4GM>jedR9B-(FIc41}lC}?x%6{&sYB(+@tbX+(ww?j+ z)O?(g(eb6T;$nKjYWWj<60oV7=ZGop@?2_H!<8f-<)d)e6U*7LH%{@dA(j`qKqd6^P^ zQU7!i24skpxFEzYpX=nDf^;zzjv$r--CYkt34je16_;d^3xOX`KeoE+&}z| z_rs&5+<(Tt|7xc`|CqvAh1xvi@c-;^@t602`jcr+F=2pf>?e>g9qjUo{y3uJbj>G% zzAtz?aHNR?K94A$seiy&{EN}gn)g*e{%70c*7}Kq=*yXdR$R|@=Vvm(0|^BxU$4r;KX4EV3{}lF;V_GQSZ-s08!l+ufC^; z&yNHe6uvOZ{h5O8lD~f#q|bXh*G`B2K&N>V-+5Vg)Ubob;dvQ;lQw|o3~af8F8cAU z^)g2X;m!$X=5M?#V z`tpI_=FOKN43u3M5&rO(i(E8LzYaS%kT8oBG@WiKg;RDd9Kp}OQaf?qFedF$S(}kX zMAM=7h;vkfdX$C>JJewxh;~-R0$?2Aqj)S1cL7^Z#z6?b_OJQ*q}B)wFjIUKuYUe+ zr=dN6A;gLa%2q@u6Fin$-_Xa<@qe*L0Em1&dXOyWcXpDPZU_E!G>75X0e|cOW!|Ui zgZ6+a&$Au~B4a@m~rs7c@I#Jp`Cj@m_@TlTlUJqHK7Uam|D zavYt}7NDyIG*vGa@{;1go|B%+xhN>1aQE>^!{(CQ68i4fpp5sytvoDgtv3eqU83xY zAY{Pae@o8}^hSH(6A@a0{abMJ8XR!-*SAG~1X@7{lhQrVp&7_8A`O$g`3>5WISwsJ zfN&Z5nb<4XZ~T1lCpV`C9I~Mg#3x+&kxZ15_q~SbMJ@!2br{Ga(FTW4w(sq=`Rau^S@#V;{(K~O*;@Avfzy+cS07Jh~#!`KrV4P8=HreZsYcxT@G$PNS0 z-*Nbmr9+hy=$|u#QP#(1aU!WR+!^wQ^4n1KIUW9#2n)-VSP7F`v<>#9L1&oICoT@k zN(38sTRb;gE}-2WGe}6$1C}Chp1~M=y5Xz3>Vu zs(*z3+bQ@WWJB}?b;^;^mp=c)M7 zP<{=M#|UC#8-K4~$3y*8w203?V z;2*jk(C&rvjh_EzU~l^MG(Pznk`M*;I@|$;%feyhNY^IHw;?*xOawj1+(hjfa0~~Ph(S>x3q1&i$ zl^s+TNQ4Fu4o_rM#5qwk4em zB*cVpzxL*F`uIEA69B~GA8$K|cY<9P9X@G^MYRx$A@NxKeF3ykFuSJ$E?2b()FR;4=dZg}6cOkb%OBeFjK`&1k+ZFCZRiyWaAifhgNH|AHIMFeVod@OfAg)dvtOAVRT z+=%Kayawajprgq?bp@ZrZ6*pgq(h4sNxOvL)IlQ*(&3>?6wwkzVzb@E3MNIIntjVU z^&hz!vmJa7|07!@^F5NHmcpxFO*|K`-s;uq|NaEiwpfV_IbmyQ+2ZQ!u=LT#7- z%&pUbs-x4S?KbAy@oR{d(zAkKh&mdwKD0^Xzgj9NvGgYlru=>H2v)~cU!m>R`x+7s z&R{kb>W7X@3pDfH9@PM`1Y$G6ELvo1a`vXk!T4`peG}=I;BTK7e0dyE0AkD|hl=Dc zry+u9G#LFk8u0A}iW`v#O5b|ArB)t6q)H}%A=5ZG0{Y7s-bJuyu0_9Sz(4bkU;>Nq zz6Y*dpn}@Xa35UAOHcjo9VjG4tuwBJidxqFP0^ku+S|>aYzmc$qB_xV-`m3?zD5?I zF@`PG(h^AEdAuoZshoL8v(XU z1Qz?hIHB%8oDix++cWSu^3I>%2Z@loCVdelZ$ku1${AsYbMl_o6Jn(K$?W76VEcGIG^agxusKQ=#cvd43V)Vz3e=F_Hp*hAN(`1bH{Kfdt z&vS(8`+JijIK=@+s~f`uM>8K@KFkY^2}hhT1832Q?k<$xk*Pr<#R6x&NZZE+itS8r zsP*U*KABEc;8!IwfN&!&Zfw4eO@wnjU6goK_~Dw-oN<6vB6I(bxAJq|qUOeM*NY3+ z?t~TS&Y}`XwA(fF)(~3+Zlb=Pd{hSOU#c$iQH1r}?D^;n2bP{t@O$&8l7kpnn4=^*Jn(V9Vcq$fqJ9*P+s7|##xj#J_sSe)LsM|r&)>{bT>KsO zH^mcpuru`^jl-Rr2cU3r$$JGahcc5kNdNy$-%cQel+fxu{btEWGv}e_)OGT`kYHE4CV) zr>JY=3D32dqsgx`kteJVa)_?9iwYh7%8IZw$h<-*W2|u(+slRqHNuF86>RMAC-nB$ zHDLQ*jdZUdNKokiM*;BtVxX3Wq?7zfHO@Q{El%o2gJf#Y6De4m zjaNueF@j{i*h?p#z~Q}q35HH{6#(5d(WfB?M`uDV{ES@?QqYE`)4L{qLe7M z>+d)}Kt~Vs`aU^2Z_!JvyZeHCD+rf*N@pQ> z_19BX*}6b|aI3cuJg6k7=X zn<)G;@(=D?vq1n>LNO}ujl?bP^VV-9N`Jrl!&IRro^S9bbc~S;%DhqibK!WA#(|R| zhb*6NL{e~HkW6qGW4SMXfJC7AK6@GlaV8+#JTpn3)1r6p88b#9HHh$w!meVFQil8q zUPHrn>%XT(YcBdm6o?KG#gG~de;PIj2?^p;x@zz0=OK&z?V96NohS;DmhnI(-mz#( zz-@S2l{$(b-EIO)y5GiIGU@ z_dVX1MaD>q3ZJsoC_*is(}}`tq`Hj~mm--cSRsj@e{477^ZstKd>jj!tmGKq6X9nA z_p`Xj+3#9I&a_nbMbJ5MssOsU{sU<$c#}&=5~&Is_-DNZRrR0Lt|Se1xp!%%b}E`Z zxIam901Qw+RRnrx2}nG8t&W!b2aD(H)m}Puvf&)-LoW&~^}rr$NhJFXOj&yU&&>iJ#Tx1>+(3e#c{qt6R} zu(0C&926R4q@-awO|l0gT7D_?$nfGr?U}@;B+ob~N{-p^!q$`JRwZMw^CI*gOMe+D z$wf-Cnd5!YoQ-zyB8+++lp60V&3F^B{9OO&~?q zpX7fkm<_!`#}uT7>37K|S4srp+!k1mqy;fa+U&|&W1rd(VW=o<{DRqO%Eo-*$S;fd zC{se`lrdywJ%~~FnR0wruBjV_zgm8DFrdX#!dO-4Wtly2aZPE6l4lcCZX|eDXbp>t zk)q<0m;s{Fd{Em(k!lf%POMha0km`!dnF|)qrosx7u-NtE31>fMz^;NGR%$sCU zrl_E+zKouGFq|3&`GjO9ph6AJ!eOgY)lkMY)ngpU>UyI5BqN=S+;&uN5Yu}mL@kPk zj=*bBSM0YsVp!d+?F#8*NBAmdXi2e8X+oLg?Wlnu(=Z(gj+ znU;I4VfS;p?bXWAP%V}}QF45k))G&DCs#a;!;ua&h>8baSP+hs8a>A(JEnbvPez_5 zpEiy`!7OLyNg%hwniDbwkkXPs^&rUIS4>HmnR{!Ouy}@@GHyxwgo{CzkIfIF#uw^6dWfqV%$jkdVkxJrEn(D0j!_X&9{CV;m z6aOm*zG>;>wr^C%t%H$UH`+mRdX==xB@ngP-*ZauVvd>EBUJ5_>H7`;GUa$ZsAl#B zIa3J|@zX{Wo}l>tpyfot%+l$N!CBzk7kGY;d_dPNo46gN7ceR|2_ANms*!f}L|xvm zG)d3Fpbr}5swAc{>>LbQaQJZ|vtR)oOF{oV(LZ!n0VJbaNv2}x%$T0mfB+U(x5PRw z=)6nxbHMA7TZ3Z}GtH2XxlUq-8cETB?jUG$(lr;6`Ne`-+YmvhrZrz?q_Ot%8)UraEL!Ss^$LdUE(=)cG*S&|3u z^}eMas^~z;3v1F$ne^R7L#e9U^xcDO9m_3 zACs659yg?)sBB}!FEbj9OQ<*Oky3)kpQk9GOhcLfg3-??gG~2lfH88?5tF=M4UdGZ z>>ZMb#%5$CyG+YQY8sxC@!N@;en?ZA&7Q2uCL{1#T5mLJ3{k*@Jtk@Xnz}_K8vomT zK`&A|ZnA!ooJ9mamtp8&#M8iEe*yPa`De>=VYTL(PSL-!xD*==bE}b8u*v zMPo=2G?=Dw!zq>Q>Sk^7>BTpyezIx!SfNJBRVhr#+_Dp8Z4Btf5%TFH*CcjwUZWJ} zMqrmr>T6R1(H0Mc&(yjL;Sp$N!;`(V9Ol%;A@NW3V*CT@W8_pN>4V`LbKi%>#WSh7 zsHRfVW&pHW1EtcahI^6X$*LF%ienb-(9tE|?t-{fnb+iT+v$Z(WNczK6DhbQDkI38 z$x~cn+irN*CbGxKj}6IV;za?dK|=wS_{NlaNHM$|YI)L5($ur1k&~IQ4P+;ixvO}d z)U0QjLbv8n3Tdlo@f3_HNg}EiYZ8=g#Nh|O7Sg*C42MMyju;}#bQ!ZGlRtniy^;!> za;__0#u=SnAxwstW`AxWOn#nZi41cT+?qmzXA&@~nT+z_us>1oheIbc?%c@sQ8j#+k~-$e zr&NB+wm^3X8mQ19$TU;SiMawW97wEuif!_+63wzQ<` z-Ic(KCP>!77#gtT*ehWY&&gqT#E{6U;Ker);EZ=@)Q0-6R=|HAxUXn|1OHv~|9>Ka zy#K8o@P9-EdH-n#L?=6frDs=oZFd1KIi=U4d4nVHe6N)JJ_$hpRs!i|Jj_e8KR!80 z&Q3`~74RIhl!@Xo`{*EmBIPqqB}G|PPTwyly=iAn1~wXt$sCtkn~rhnOfuIJsfC>E zMp1U@0bZ)dGFJJ-m}F-+1*%Hs#r^uh#4!~MH^muGp2K~IkNw6@vQfjkZH&T)anQL| z$>!9<(V5(&CP#mgfdCh~>DfqjbJJb%Y^OvQ{?Qu^{>7r>uGAb=&a7zzcx;BfW6o?m zCq65Mg-m|^vS_tjNU|&a@X|5=@&2hzyb6X)|Ma#7ej!yfWePdF8?}7GDEfBEnNK9D zvzy!K@CuowIk+nA#nH%YVhx@#x9Hp_vlY|5Xp#}0*qKHgE2rc3N09h;J`Sbbi;aj1 z6*(iRtxD!V)62Wu-eiq_*&R=O>w{SN?9Vs)2%|fiRH0Pb8@c_eVkk_dwfLkG&k~jN-CbrGtFb)o2kb(a;D}zmqw@XNp*MCb<%gyR1~Gk@X7McBby@E z*r<&Z^DQwMN`l0*ZR$M{6nir5^r z==9_>BS$pr6u~B`ZR}dZV^0xO#D@=ziP!a(VN=g=sO^VRXSw3_-f;?-ngYEvPe}Qs z&YEem*Pf)2WnGl1C^!1=<2~>k+EmvI2~JKvHG`g+JdG4P3$Y!^K6+9?nnF|Mf#35* zlNrm#g`Nc`OlvrEg)xi~WbQJBG|OLf@~>6s>;|tYaf3}ZiE{Zcgc{^9$Aren<8!hD zF;rTV=8uIg{{+hC!Eh+{C_TXoH_}ALQ@(CN3w~5EI zsWQyX=#Al$m3N89ZnB#aOnlGeIW7Mil2?4=2blJb6ecx$r;N5o7w(Wlq@ypTt~QC! z!_))F{gn|ur|zcAp~O&>9kFW{{!*-w6JL$4^sY!bCTx4q?h^A*?~gO4axCn;|L-l^ zUlb#NK&FY7?n}id;sUdz02EoBNS435_lyN!7i3739V37A8e4N_!m9-`|4JYGv2Ol_ zDlytwi(h#*g4MpP$E9Gp!S8(t7M^RQEWjk$3*Aq48WzkaWAcS74gc16Y0655p3DMj zDC1Y6s^l*LO$Mq8U`d)1rZ{-$d6bETQX!|{(Gl<>qZG-eNg4f?TA1YeJ}L_@Xnt5e zm_L~u1X4!y5|N;><}5&Mk|0;!j1=Lp(8fwS!q65~LO$V@+xxm-K%DYpO#yX{ukaw* zC+0_^z=d=hKAy?}Y-%1gMo5_Qp9=mjWpG@H6e%HTzZUR0*7p+rq*A9ihmB!IiT*P| z02W5fJ9sI70yD52BK*y9z2R*573ayTeN?^%kvH4I=VS2LPobvoJ0fgp(GvU4=>|OD|LcrDn zfh&6NHf;CTMJ7jLxt>WH--1*B4E`#glu0618o!$CN2X3YgF7wiex9J3fvi+I9lNhZ z>RVywykwf}H(9FY{YIpsIn@Xh74kLSPBV0_qT)sc5|vR+p4Wpgq6}#)0nsT+1a^rM zjM7x?dWjyz32BCOT1?Y41*|}R6(X%4_!eF{DXq?d52J|Vbm)}7pmSqyBSmLv$LUKe zVsnbjlcPYhe?0_yiwm)`&gY}_X4gFk&x^a?V&5>My_78585@uMy3({D3K__u2GE8 z;&-QZBE4EIFy_n%Xk!&~f?uxVHaP3jyU1Qs; zz}?KPOx7Nsb)=)3DDTZFcDXa3Rup}foGIR^D!y{`Seq5l;<~1-nJkU~#B2TPZkKwt zAnh)MN-{l>Z90}BrJN$lX1R-pn)^X74SP&LRT8?8KmriA62P>~Rx43k0AWl?j7uw( zH`$ahs`4jI9UMzzR&4l`@HBh?4t(Sf_(PoYyuJn-Nq{%JH2NO`(+s{ZfTSeZP(B&D zczU+QEEpaJRQcBVkfM1g9 zh(Ncv%*7~gPR5ElmhzD@*c$x1^FYHA!J{l^b}B$d&PVPNHMf+fu00 zqBJGB)hV@MuKbxU-|W<;^bHWUrYLwUi(5uboGP#oqcyrpC+vO1@He-v(cygx9YL;~ z1~Im@!6j`DAo;@WXwKA~-n?K;Ag8^?W_iHWozupd?xa}ZDN9#Qy;$0rl1?JWbu5l) z$#P%X%V|UXiyzFZTGCgQ(1)d5Ghsk4A5XihlcD5dhcf>?Qxf)Ig|cjR&aPwPdzMfk zEnIFXe?taH8dxc%TlzUP&--JA7}7U4SoBO>_! z4F~==bK2+sGaM-S4-Is<_%W-RS;=~uGuoDU&1LqFam!NYTOynn{c7@(R5h>U?LJyx z$en94RaCVC%Mw?TP8A5vOMq12=p{1CxVp>y*yGCPdLtF;kHRO*EN^4LGoR_XLQ5jl zvWyq>Ze-D9g_|u;Op+t+bY5m{`6)aPn&y9P~laVL#uk`O@S-V#LOvyy18*kyOrQ{(UwKt0^e z$(h9{K9O6X~{ zStQt*ergH)EMoM(u`1J4i_9ipBiW?rWkOS_fH}J=K9&H(qxr!Kh2+?{U+1e*5+`WL zc|s|9FTdRA*}E-x1uJP?!_wwcy!frle>Wx414Ay9MA`vTv~rp*-86Lz zX>^U+EozmS0A2d8zLlF;WjK?%n(W5lJ@KN`6z7{6`6wP?I(WhS3M`$XNm+VT^nLl# z%GRWqr3Iulxl4;>=z)V|U4rIvC_ zxAJ541;igzB&=v35Ul^>{C@F|y9DsEPe-q`lfKdy5BqgFbv6l#vn8cVm7?WNoH z$@gEsi(8WM=I5e2Qo_!|z^?cTEaX>HEmNifS8|lSEHJHokG3SS&D7@1F#52|tC`YV zFlIumsBFATsZpwl`q6oE>644-><3UY?oj#+UNgy!n1|@i)@HFaQfXjk zS#VY%ED~D@^p-c8fM?hPp;YP-WQ)N(y2V*)If}pPJ47ojUMcfUxSX>V8mvHsrbI(I z{FpK9YKrtM?#G%E5KCaS*L+OYk5qd8p>%7YuhX1oyKt-}RdcG1Yf;C7Ii0jQ_2yYV zEYRs?zU2gTeY|>BnWh%g18n10*aPc?mLxQl=}omGs2!iMbeFL;WiJ7XEY-#66}ig9 zQY*HAUGNImR(i4RGjKknXrkJO>)9NJynQ@;hB69x}w& zRWnUF`b0hbglQ^;MlIn*E%`O;Mr$lC@NDMhr z3tv*O_X4$_;FlH!1CycD`elKOW`;v0_KS6E;{#C|m=Ghbvgid@5RiRbLaOXor;6+@ z{6)o%j{mg`d$TyExgI=cOV96Yw%`MfyAMrM!J!AD9!pjivfrg`%_^-)DIFW17bm#- zXsMxTu~RUpGeO<}4YD8$1a<#|d-{d>Jtwa{<%BRM!fjUtg^>c@9mPAh_6# zLQ|7jnt}WHFcpFKAIlr2ykv}NIkB@ocGTG|5shn3nW>oO`f=(xO;3OoLl4EKat*OP z2^LWKIigQ!s|#s?*=|b_R($@ID6R!hjX9z!tS6cbV72qSLS;s zFy#uc{3V^GZiPZqQ`XY_8X)nQtGlS8*}ZHj*;DkooWHvY?CMqnTB7GxJ?6EoHgGQQ zR~m>?WitQdT77qGMm@a@=Dej-4XvK#uwq}@$JZ{U*KsWnXt*~kQuE}~wzOWkE6r0) z51LEVY?VVg8mmH-^M5u7edEA!t+8RjK79fXx?WcLsaYj36v? zgt@H~ZLx|MpDYK?)_2gX=8unWd2Y7Nd_j%c4uMW~*$P(8NJ==0deH|dp9)N64lTBQ zU0U+e!L`tlsEKO!vr_!f!gHBB?U#+awtFcZbO1^%ovSOSQ}djb4rO)Z7+icSW`P~k z)5kAr*EHWtYkINrYi3Te6cniMa&k^FbJNsF31l+CuH7{&sYQOk!FB-w3pbctdd>o? zRtEFxf}mz+D8PjmFDi|bLcV%)Cm7@omcTWaO)V4Su#r_P%#!XtHNq}bz3W_> zuO$HfwD!QRrM!=bQ@`mDn}9%Z74cyQE6%&pOE*_|4&^DS)i^ERdWp<302OLGoujPO zTH0lPR(L7oou+oDEqkr$Z9)WcYc8>q#}}1Plz*SXo#*7M2LInifhw_C@Ba)3F8;sz z!lwVNQSg8Cg-!oKgU@#Ia&*0OY#gGzvpm4vc*lumtTt=Kytf<`Dq3=xNj|~2cT`>%}Y!}$CIEg8I zzwaHMAFCADqYntp;pgf@(5UoMhqW=20K=xn=cyH)h``$fGF5(U=_ApA7T^?ws8-i# zas6ES)yk)YeHH|)d!2V;Uy*td$g~Jf&5LNRMw$7|uiy>|b}bib%v}RrT1p=t2IrO~ zWA`J>!Xb)LSV*SUVW#y5O?PD~k6_P88X@2hJbNF=?x3aKA*Oko zoH0u?uu*D1t-}`U$eSa|lmbswco-dsA&n^mNz26as`V%EQ^mw?7ToifYhv)06t3m} zn(X=2hpc{tGZ-`%{$3ccnw`-S7Peu4saJ6E3tQom1)Y~6W7kuF+LBhNz})&4_jjeO zSY2r=PnAMb;K=(j^d|Q_4H_@;S31CEY0`XHk;dD&rs~<6Ns+y3q~a3Px|U336juqn z+tAkJ@VF73xk6|y(^4D4OhuGDUzNLSeeUg0kb0o3uE>dKZl(KTOW529zP90+mT%4> z*YLagE1zODBnCT=gS$^?Wkc53+6LHMKi3%iFheN7O5x$4uF&lVteM;3JtSZ zRXu9>);vtsZVJ_}nBHEhRD__vr%5n{bg5$xfG{g96?!7u2BL=uJvEwW*?dM=V(@rV z@5_`^l+YLhHF32vcf6z~oq%bjP3lxF2xml3omxP-vWf~=b$>T*q{L{g(rl524W@Ce zK+t5;R!OYF=Bj3H4qTgOf*7>hq?>}DxV-b0H{kg_!Rb@wq`)CRP1b1LfvSw562qm4 zxK+Uuxnm28<~oy>NhlzPx7qT6(cc_v%&9U>I4%9!imc5dlBJxE6>M+7UwS{?Mt`A& zWcE4o@ekusjBTS=Qvv*R>GPV6S_@LFN{~p{k8NfZFB-9 zENRYLnUh*ZELNm7^=)awA}U>&Y(b#?rTVi(;TrH)%ZDo;2!F0#+A!6a zi2(A}vOumD5p0S8<)+Q5oan*ZeiJOc#w7*w)8#`)mQF)(ws=VQuRxu-k;RL=i11Y^hV5wq56vK(d_*#9)g3YE95o z+7h0&CDS=tD%iNWzYIiyC^B;d=UVEoHmIm6D(d4Q;DZn;X-X2xdRaYQZxUOMsi*BDVm&^rp7w)vk5+4^vBPfM{!HrW)eXj+VTO zH5G0BQkvTHW@hf6OB(vwMjXerFfC4Qi(BfwQR4U3byJlNi%lr1%m`vKj99>?BAi4o zmE$Ob(DS;*8j|K23~jhIGZD)u;z}{D<%3zHSM%6&17LM8*AVrbDuT(gm7QRV)7_NY zJPV;LI(>0<#VGe`zp5l65VxwK8iIM(Ilh8Hs8W9>y;xJQIIpMmyTc~)94fV9YN^N# zXl`x#eA+nFw1>Thvuq)!{u|Pi+d_riZKr7k(STGpcejM$tS4#E5`w+VNEdVvKY*p7P246@8SDl}> zgSLpLZ5wv0kjqBWx|6lt3=?r_ad%U-=&}NXrh;XDXtChR2ia;5jcXe-N-DITq>VPH zLwDLGGYVFhmAa;z%?Sudl|nj9ThEt1+*=;r^3~NW3RTyjEqe7cEsH}YjHz;nyhA&7V!4i$h7edG|s#d z@R5hgXf;&>)4E;+U_S-e(vfks!QytWM_`XqoVpdrtj$E(QvWBw~DF+I?5U*|#ac>UO z{>tDIei1=cGM09GQZ>F}iDsO7A=Vq5*>YS()_Wmh#D*J`x^=hhhT4DV{y z$(an;4d<5IH6Igfm}%=DGpXKJ2()wnfctB}p^GuwN*LfGXgNGudxUwFxWjb$ED7ZA zm{zoemw2atzb}AXZr5jO4Yn8s^bDNYz15Tw%NPaCjq?? z-4X3J1l!nK3z&f(5I~}}Lvuqh@Pd1@UC#ZqM%%T+?eU^R=RL|j*V@6fy)U7tgnctcRc51%X)R%U)EVaGpdcV5sGKSu_2Q0S}tbP}; z`uUNq(nZR>jkEC6+6tbfYO}NMVE|`Kc->XiCW@(n;GyRPI3Qq$%CaliQkn`mSr~+P ze75b7sBq%hH`7Y)^!NfcH0@#r<*qFV5n7I2DB5;qFI~%S7Ha`WYX!KPJ6rx}sTUx6 zA~MXHc68s5p{(a;*-+H}32oPMy%|@ho^bsFN%gHuX}!_Xi}LowWI3>{v~=xc28B8Fk9%&`WS@6cFaF$G+2Pl7tm(V&dRRPT-}FOI{afi@$G2kYM@`F(l%GM2d8$Sw@fCfpMkwFPuH3|ARiy%t=|Oh z==7y9ZPm;7JzH_nmWK9crd|G0?$t}cVr?r!#>2mDnM>sj+R=nYzpLA(x}4ow#fHnznh;x5<{7H0)^xDbG996HE#t*hw-Qd>$<`*s z#?$`tzFP_$I|1ic31WmZKR>?P44Yan_fOfTg zjL$i2yEK=cI+pg`O1ff*tdZGoY;jIbR4%jzX1F`pC^n=84GzsOI#`Fzet@c`sC5=cJ;K@+ z^CD`qF8<-DW@+lx_h+sDe)lE!o1Nwvy~dR76n3wo``fPiOO%B^%QI6i#>2Ls=B>8b zBeVTtS47Kep8jF9AEE7wLeGi&U!(1NYiACYSLD`n+)irEDX^^+ILNhk=<#pAzU--A z;bmcItu*)h?$+LfTW9SXOxK9p()jkXod2o)B3IMl(tFUx>$=`DLpXAM!t&nk_GY%P zseM##<+J*~3;p+X|Jlo*_q|Lw%}yE}-ah|WKB4rrKUBvpwGw#4IA|011%U4Jv>)jG z;p+goH69Pcl3TCihRe1VOnstm`+52=Rsq&FzwawxJw{v9p>qOqt>1us*jz7Z+cW%r z{%25%%}7rJ+$-Jd08)F^>NaGLDp}3dlYs786W_j894LtPbhZV>m9B0>-_k8W9sbOc zQ?#!f9Mdo2J%yI0-&KonCCQ2fZ)p_`e1RI()Hf7s)srH$d0X z|HqVf-?jpRBh)B>m=xOP5%)*B&D3a+S2_K&afIouUHah4Z0(T5D)gFODLi-@tu1=b zV7iio~CAh~)dbE?a zQmkFE{T?8MtcuE(hePU#B!bZ348c6sI;z&^wFA1wem3sFs$hXsh=^W*uz0wsf zc~$?_XW|Mk?C|A z-7=4j=fr6US3gQ?bN_G22`A(8IjR~*4U`$fM!E3QI&f;g8{Rziwg98YwF3!rqnE}b z%1+O^x50!RXf;3WHG|=;eBs6A}NcDclBip3t||A&8Mwc9dm}0eRpr*C(TW=T4$G^lrQ=9!U-Q!w ziYC!_ckG(s(m}4Yk+oea8ym6O4PXa~=e$ zd&Ufw)wLAN!%i&(+Qt|@vGz6BfBSVjc8&IIs$vt*8aIdiH=YoJcTqQD?OAa^ja$6; zQJQ?GA-!xbrX4l7L?7`eUA}a++rYlq$2*dBwMmrGh9AX`E%g7@hRRMCY&BA&>$^0r0t3~XW?t=p{mHXJ`fu;6O`Iv)50ue_M^gJeP9z7In?cuhWDk_Xy}&9j+v?B5J?6^` zVY=>J)?;k7Tt9<7u$;BuJ3}g&MZ!A%y6pnif@0kFE_QSKrtpZ$jR|ZbXsSh)hB7ta zYmN31!!s{MsE6VB$Zl6RAqluQ_)8`I>gqBPxq&0z%A?lc{QV9mso1A)WF-;^PjL^x23ilrB8nZ{du!a968567({p^ ze_hXo1f8<=2@bJjynfyC+6Z8~VV)jk_4ri#`)S?QwC3!*m4FY&-XyLn4Ax2NVTeQ% z5A{#{hQq$&C35fL@?+2+V?qnfbC{lCEIt_R^>K(}rlT~@7>qk*xPFpOl9*%bet(Vo zQJ&!uBtpT7wXA>afinM(LUfWYn>hC%J03;+MiV5M?Gm!tJT_8;FVU*%0d2kI*qJn7BWpHhxlGh*lKz%t$iC`3%BD$e{(HeXsnl_8zn%XH zdd4JcBUbyWEVhShmHOY>mq{;sb~?RY=2c46U6T)i){Vf@5Rp0!g)tRUw0lGA=IHll zl#xGqleWTjN^#9Tt+4-g^N~_aZIyAxTX_*7`aU`Cg!Qs!-Ke;v=Ef%0$-WoL+}+~O zZ^K4OI?k|;i!dz`t0&-XPn(o@_QU4ZkQK`9o%2_xbQXZ(xUAMXY=|^?}jGobs=hy?x ztF%h{`q4JE*T<#fz*Tx-V}5xn?#c^#OTSJ%qlQhhY8EV(8%IPTs+O~inB^Sb&fn}<{6 zG1fufJ&YxDgTx~|zej@DR2XJ%CN15lw~Ac+ zqW#bboM%nlXp|4z0?jP-+FYM28W@fEo!UHj_QfSIAoVTX9oA~_r5X*%|6h=;(a|mR z_rH+6Lq(iKO^iC)3B;l#j{9Gb?T8uje<3^WKag$b$~LvpVwKnol7S4vUFe5bzr%ga zIQ1ayuJacGy5A{%Q)~1P!*N&Ft0tylfE&MU$?35ONNB#>W8tD=CPBXL}u zCJ`f1WX~A&P}m8H)g}ys_)m#eQ-8VSdmEzJRz-o28h~N2-2|&4WW=7I%6M}G9LH0; z3xsewbJ=gkw8jh7idgrAUNb7`>+B_q*;5vDWNALrUm5pE80Fx{05AXMj@@&S)u#&w4fWUPeiO?xSfoU1SUMy_~D*9bj=!`V5Uo<2Xk{hLi?L8e@v zGe+x@IlPxD7sCQi+C7I`$XH>3IFEn8Q(!_ft?Ln|wn$j2y}j%~gH+Q-#9C@usxy~> zdiU-e@!C=rIMvl2#L(a0_^Er-&C1R56!b7+&Cl+=63y(YPzjJ4EG6g1Us(n}_32Xu z^bRNDVsEXKfy%4sGe#&HcOL+&!U3UNt5jo7u7^1WYiX&@$F|h{p7Wd_gRIiv=vo`a zG110jQIn5lX}L#udSlKSgn=8tk+eWt`+}a3Oa7_li`}$LE(i6F67p+11d7E+f%61+ zKxv+JYqiwFN9jmzwtrIKV8^sgICb~fTCWe=RZwFYe3xomjkq6}CZlB5+bN4Jt##BJ zVYIU!$#Uh2v(i^!5AU&o@Le5(OEQu5~&a)jza^XO;W0v&MW0!|s z3Kt^ia*})oTBUwM-^RQ!*H$|`5_#e(1>Gj+c?9HW2RupaQp3B+BFLrBVBh-_n}DJH zc_PcrxwOAa_T(7dLdjT2Xu~32_+NepJ*qp*Znjltb&52hRz?8iH>K$*jmjxchi~AYzsOt zG}(mAz+oQ6T-WvEn`Ea(dJI9(+Rg4OE0W=Gz+Na`E6;-p}nd;&}a zG)uOktVaIUABVCHb&yK+qqwc&+D?}I`&53PFzAt||7GU*25Bd9mC|1MZk0h*{k>v@ zCP|E9(?>qnvM$_faB}`b;wlAIlx-c6(@2aaS)pOto^wBle{5n*L;BF<1f1*rg;J^q zCBSxzEYNW`=8Bvh$!L_mMXBP1de<%AL+^hxNdwGT68CMwd53G}NfwRYQ|0+P5Mzuv zl8NDO|J_S+uW`D7NTgtqZns+Hb#rMQj6KZL(U?cdXAB-mOHCdmb2DzMmSwAcIoqGe z;}>u?q{j`cuhFv}ICl9lW%hh8u=fZTFsUF&1_i9b6AA2|IZbXfzhhoyT=HzMkqnsp z>&&leN*xxUae@;u+2TSXx7{wB|D)Ei4@%(jLUo%7=x56XYM4}a!`3%vNzW&*iHaEi zp()3VIQNLDtc{gw*KT3NG!tRT;i=NtoYZWRb2qoX%Lj7ozMK+U;B!Y#3VU_e#gRUPMjbg&o@~kxYoS$fs^|DFhPOaZ63D z%zZ92cf1Rq;WmbJS-$&liptdSQQ}Mu7y$|rx0+|%`MPr`L;9?*732)taSCniEe-}h z+iEF7w>K>RZ9h52!#hoix!b$oIQKf^hrX^kvUUS&tHKoy+=kxiIUeLTQ+ST4jk{Er z%L13#x^Q}aYIVj%rz4qS;+n{4;q@5+GU_w`nr^)rUMNxnxR`317h^Ushv(y;VM3-hzNSZ$o(W>~`KEmB2Mof)r0cS%r)f0#2rcfzyndV-WQ+h(n9)pSaq{d>!r7)Z z$w(&nWOl9Gm$PQR3k#r`$S<-EyuUjE!c(ilAo(dc&du%FH`~;G%2J2VIC) zlTs)ufimlPy<|lnbF%W9(PUbTagF`~6qGDgX<8#6*LqCV&~4bH{GV#R{~qUzSm}=c zMd=J`;t^`%eMBhisnxIJf1z|Frp^CF>5l)P^eLZkpecMLT%(%aX*yDnP26IWvUq~y zvc4NBz%IA*V<}yiGaoR~)vR)=A9xp(%}vTMO140hY9PlNZd(0!bHF);2YW{@yg1yB zmZANg|HISDQ_95Z>H@^HfGe;44*h=WQ|I4BrvaI=`_83~ml#iW!H`EX9{$QjStdPK z5#G#!%a2{-a-A$JZeNm_+y&v-oqcdBoqcY=K)x0xRA2QzO^L@$YDUWSXZwbKT1h*H zjpB5xrd+87p+}qsPbp+nZh;;XRSr*R=aN?(CLm{UUsI=w(mgNj>r3F5nlihP&%{2? zKIo5J;9N8(y$^G(r3;u@`Td%2`*(Ocmc38H46&Q~p@A$g694Q6WLi-`c4|ip7})Rd z3ozi5LQ4bCR)_TpP`@{#g<-q-bBsEf`R)v4a^mN+_I+| zUofyI@H=$lNI1#A{x@4+x1bE+ZkbMqv3;!d?sbz;%S!BIH!o)n8X6%-?a6Id& z9mCj1K}~xT)AWSTPjApHsF{yz7tofKdtAxZ8}d$SZ8M&w#IZ_YbHGqbXw6U%UbxE{ zeld!e*A%nt!YL=La~7SP(9)v$lPX+p_@%yLR(4J;)A$px-1aZH&TTEm5{_ zB7v=?aK-N00V^2mYW4cnwXjl+`>mp(=NwY7?FEU%KR0RFyVb5FlPbw%s(%O+qT8#f z@dm+~y7(IetQ9}LH0q5uxac-+sf<=EI;v6k9=!X}k8%z#SaJNsJAS+%JiPlvK)la^ z6|lp3yoc2>rDqbKNnF56Z}b<4+lDm7s%4=+FNDVW1p_^M&09G+Io70@szJulk5qvjcLUMF2KO}a>exyJykvaGFw z%%Q*2rP%ovUd_e(cXn>q)tDKjf1jCo)>1G@Da6@m$)Q^bJhMKSqfQlAw6JR{ezYsy z9Mrou-ZdY?shO#@%NTvS2{fiOZ7Fni61=njlBELouvqiI)C?Sq=DV5{n#SnXC|nGB zP3FHdZC_iH4?)^`L#8kmRBkuo%4;+l90BueTAYl#e<+KC9fmI^*ltbP&6DmckUVO5 zvr}49`WDlawb~aJePgduz6P3zQ>A3rkBrUky8%wOPrwnKwYut)+^GPHos`icX#Hpa zROE(N%z~<|;NG4iVvge)u(r3NQ*6&GJ3G&o%r3O)*jDR$Vfge`w3s<>hlb~4*FhiV z5#)c2Es<$e<=lD&!{4<3JX$`cmD<;5KTL^he2*v3}~Q6e!iW+dP#eEw^DhVW+GQ!yzR7Dpua5-!lIkD z^bV-4IhA+o0G@FRgzjypRft_SNez|XC#J03av}ZT7-CX`X+%CCY9EBy7_Ij2*6S~h2kPmSGPcy4;nFLzreAD z3b}8A)of>s47ze^Y)#;{_B!p6jQlgY(TMyne*b@Ra^U~JoE-Q+Ik{|2Cnd8`)&{J` zQqi$EiXTvGbcI$+zR;(~d*ftg25-yOeCncsdTTo$T6c(KyI!s%7IWqe5_b@7DTL1U z4Pyds!ilI3k$N89jfZJdLH^qVq(ZhY-;o~FTfpJg3`_;~ch>Kl=}vVJvJjvuv%Cn2 z7>-GGAwBHC$q#oJGR1caAWeG3N^OxI zLR?6H7uB|u%TvDuK*YCbR9Zd4oAI4bMMuU`U7W41z~y|7Q=1deMFR?}p8X=ai$3aw z@m3W-6skO(me0j$=3L873ACfz)_7r2FzEiybZxIqKGwJX$=D=n(R+O2Uv0&l{S`CZ z*QREm&9Igq4Hp=QcP%!Rrc=1(+b4ez2!d4i)Q!^ex<}6C0|esM-48VE(YVV4!9%Vt z@?rithBu4Vj&IJarGhYl@zuE6%e3xVDg- zpG7EZPOtsl#ax@v&swX3@VKWh0p9G|EiiRsSqI{N0SuU0piofS#qiAC;H!pAd`r=p zdWt+XY+V8r;b*O0%%&W=6)zj^A;|By_NkV~A{C`h2Ytvt6t-KSdtfQBeo$%-K;f~w zdq3LV=KQEQdUNe^3(~o753j#@nE9fm)Czib0NFUZLo8ir+adRpJ?p<*cHupx1Y|O| z8-NgkGfTYX_yte*?WX(s340g3d6^qM>`fF)hvoSe_3~!w_ioDx`F$iO;87PvoVTW*)0GrZWqw$6()T0#ya$f zz`AyZ8*2-C+-liq4N>TnbD6P*tuLsj+T43~DK{-DuCOiGF>kHhXBc0GT2C_x>+Nn& zl;vnN^#@Em2fMi+@KQBsUKLadg=A7N_{f-vzYi(WdyccV=0}0Iq z+q!|g_+qm?DOQ(9BnREz{BQ7A`E`Q24`>y8Xr1>lrqC$GNo}#I+ryh)iLJ_j9i^>f z(|h8ZE~qEv;G&_{4Az;d=Wp5(OYS0}2OIDPZ%0Ds)qn2hhceV)y&c+^1j}=sd{#GSjd?0m~yGIm9JQRN;@so+4gX$Cb5C^aW(kXh3<$H zuW#^T{sE2<3=`X>jHp|wC!<*tO$#$S<`k_Ny&?}pgx>!4v*E6&h`#DpTpQQc&)vcq ztIl>hvtC@)4u>5vLMpH8n;VeIZsE1ZBjuHg{N52O)*hbi$E>*hGtQKWbDFJI#@3Fk z{bEf^v(R!uYvW^hTRxW4NyeIkj-!%wqi(-`L&yVSqs0=s_h8jOSU-NTB2HnCX^jGQ z!cGyh@WWdN7>G?2qI*G=IOhU(R6P=Y=&o&|My^q6S&JV~i8lV=rcoM~S򗳍H zc1}=-JL%}%9#cRXA6?L&`mo_9{kvbOi-XU4N!I=D!gKe^mU6IMY5CTfv{e&r$J%P$ zuV@pITJKC~EXi*h-6C4z4D8%~P;_6le+7G3pt<>D40?_?Lg;i>u?iAS(y+~79bvxY zfX?)Qlf>)@tVh1_7JJRHfCFgkm8@$_c{q_z*OOejA5=af(0~hqo9O&SI|k_v5%ZBc zBV86WQ+F^I%j7FZWRa*r$&2fU0*ZK zShsq1?>vq|oUh*;u7DPGw0F2GM*Df6e)R)1jAj3yoLta*;)eIc=`*L0ryZ(&|BEM} zzq-Brk3`V-AD*yCsoB3x=2e}3K8CRf2d83N80ywEH}FAMkUHvSc3i9tBCdIQa2(1s z_Y)hozTRv8!ufR_U;Kk<;IO1CF?61{H4PKE=h-vZEFiXZ{LU`%IDNlHx8ar85|lE% z(B2W!Gc8s+CmoZ-5)US@3b91V+ zqE{SCcbq@i_;zQ=Y`fSt-rq=IwmJ*f<8mn9@1|%G_eQb8qb#k1##-nEMG{eMw;=uF zK#>U^0ZFq}Kx+=6wJC6dT~G>&YSYODoq65&A2&5iEl{@Qw%QC)H8eNu>1|Qp=lY#x zjrOuT$>48)@XTwqim7OndrA#1{m1s*0_uh$|_CanSDdodL)hAsSN+vrpY+MsEwDbpKyvxPwLAGrg| zu?h1Vda-RDk(^jyD&A!Ng~H`ZN~`VKSz5)EW5bQ&T(yoq*OB!@xAV;+WK%>aq20pP zYoOHSzG0Iq-A2heBcn;g75{O85L{@sZ=pC6&c~>M5*0CjboeRS>Ff5>8*Zs%DQ7Lr zHvO5OgJwrU#BLW@jN2INtjxDFC7kAbk_+hw6vUI>lA#JyI?IEMF<-7CP_mv;?2a@U z1NAOH64#j0id%OJ@S4nh(z@Zmg65Zi%vru4hwc2?>)?b32ZZ5jk=dR3AYt%=bRDa zp1kj+mj_!Dc8?>hd%|5_Jud5tRPLTJQrCoEFg^`qX;)UBwTVKXyD9`OdEA#m_aGJh z%#HewySjHN?x4m9@blZLv00!Jd!*ukOLFSSXMJeG$iemRu6pw8$fJ(p7-sGVL zdc!KV_gXhW^)M!Ug23HKV=pwl`t+>tV2(!U+-a}y&Mec`8gHJi)x>+1)3e6eVTP*v zv%6{J?)6-EvwM`_Pr#k95bhn5hIrZjIh0P_u54Jx`QrF7ZdND4|k>Z7e5d! z1OgKn3+Zh|{kB*K^S({;&Ku@!MIc*+iVCWRUt1^{-AZY@JT{0szYP9DW}A=S#W>r- z;q@QJ{mwLVXS?8vmPcnK+df5GVXJ+h2uwjpI&yE5G9%Mz*K@$ONl|*gn=+!%skeNg z{>CD6y#ZV(eZei-lgn?(xaImJNjcf#P3YcQM$So5HMraN;5BYstC6lh-p2M{bRN_U zyNn!c)=)V@ofhz>CR}h2k@ZxCi%+^j_2(-f&9cgE%-RLKVo{&}2XY_W)|ly-jn}r( zqIbE^eBbDI{LUHzb;H3tu!d)0ULr-hXd7$}cZhT{x{p>A;i0Xzk^5Wj`LMkq$cU$# zOyh>lT@&5qdIj9LQ603dx|BaQst5Y>&YTZ$Gr}PK*4}a7I^zmft$|AB>5#LeERR=8P|R zwop`Q7L~T;x-IoW-=$X`v=c#>h_NO276Ip-CC^Cr%fqpO<13xS;xx00&@ z{-FV7DEXda-n>4UTj^(L;!}xZ-a^?xOK8We`)R7p~HWe;3o3&(9*dsN68{ zQAh4o%#QgF!!tr(%+^ah+sqs3aqew-ch?9HZCdpl3tMk0)#Qdd& z_u-KU*z)j-4XJs31wj2=OvZG`iRgB}eRFu=xKyunxh9I-Zq)#7pSPuyI^R-v9SY(1 z9~uj{W82?DV1%y9m)qYw+r3Z!RSfS6dAGxXR8<=GiU&*j&Wbb({P)eJPTtgWrwv+| zt2&+$jAy$Dq4b$INDXb-T;~hDRM*1{yi&!3R5vA<;6D=&q*Im%lWzm~FrIdLn@@4jSTJjsZvg;LbH?!h>bGYvcBP zMb8v5mu=jM?aT2^L?66P=wv;7sN2;l@QC0ck?w5KddOVJvk|F8`M1{Yr8P{c`5$bX z5Rv|P08m^pbls^qA`$KIv)yM!vI_@fu9NFE`JH~MSkb%<$Xts_?RIwQK3Ji>!gxG- zFWTvR@P>U=S~wVrq}Ovuy;Aa8-MD#$zuG4fntMg0Kfrh;hg{_sNyT?csaS2j34yM$ z=q18z|7#6AduG)vk3fI*>>2Sh;wEZh2sJS*s3skUl6d@om;<=e*#B7``^+HSEKm~< zuvaS-(VWs!4KO$D2K8*3!!%TDJCP94IHOeNrlkkD&UWcev{qd)DE`X|* zNlQyoPvD8!7uN`s#nVu6b|QL%+GM4qCgrI{dijbd1~N+FDhx`wEwfK%zdgV_%h1ll?%7uHs0`Ei>QVl- z9s5IaY3BHZnATn+j)_`U)Pip**8F+;>Ts4ZyUk;ObGnhDoM_L04o$$RruDZ=asLF+(k{)#=4K$1;pPxHKx!uT<7jAp?jp3K8~ zZ|rFbbGy_*!~B~iLbcvN;s&7O;eR-9+WNDb$55;hJD7>4x{c=b#k13NNnkmI=~gw? zAQof&wZKam$>!)z`~tn2Gs|@^Hpe=NC>ez zj|}@K51N&-c7^-YCL8LewlLC!h;k7Ve?t{sdv{g4(!=Ge>4~ z=28men`=6D)x>!=7~+q-R9kn6RCc$Sh)+vg_>es_mAyx^38H6<3&AhEw@ii-cUAsc z?u2}A-o4sMEbqcCOIG=jK2+EjX^EzA!Pxp4W^y1KyefCrx%4vNi|Hk@`V#f8`$~f% zl1nxr=9c`z0qZWe>20WT{nrEORZhkJEZO_0{S_G=DgC}zZ8v%Dk=rA z_FIdb3)@xccr>_P1--}UGZUM})fMqJ=WLtQm7D#!8D;0dhqs)@aT$$$Q!1x;$9Aza z@AVC4fZ}j7T>Zjs{NhT98O*6=_hWafN9Yi>))6S>p(V%p)zR}kEe^;_mme~Fw31&f zC>&N(>l{fp+Gm&5HrrlI7*5Ga=TN*38N$Sz5UJbt?A2;+#BfO`(EsMA$dE)5Da@n7 z!t_&dBShOVBlRiB>&1d5CT|{$M>s!8_M`Im-kT**xJMLho5X23^Hx$LJBq1NL-i3C zz4&wg#y6>w=9-2eh|8jj|N4{YTqU8DY|JSuw5KE63>z9&=w2|vB-2z%wp!Pipn7}n zG|g3m~OZMO!8W zZ%z|paGPAk5GwEAya96b7JVbFRx}D4{g!nlrSq&Fe*^PEg;w-sRaU$4_ZyUx0TZ*M z)}{cndhzPp7euk2-o(9-XL!rB?UJHGhi!5E?j5Y%h=uh!iHhY##oA808g0lbAi=&^ zUhOUN$Cvg5~96y(gwB=iAFe!n)Qcc!0Inb^lNbskDp*i8mGlXs=zOPhR0< zW-D2RWsbjBbHyj<)a;s)xALJHHNqukti`5%!J|*WAk!+Qzv&v?ea9kAPSy{++YjE-@hu5Nh7l^LBC!za+rez;)E-YDR2vIs5@pKU7O|~NZuxV zo1hy}3@tKtpf2zcVB;)k?u=ZMmi^c-o8}yb*BIsPJe@+6J7MgqL29U#h_-VqG7{3?IZc5SV|k@lh_3Y;MI zueyDnA1?)GbQrutH#Fm~UVXl)-X;*LlR7QSiNt6QUb<+eBuIP~n^!KVPY#(OSk6ad zR9(UW5qX=o8w3c95c8OfR1}WZ=7#sYr=Oz5M3GeY67qs9Z7ybI5K}>(eN*_~E zfBBf0IBWa_FZ?@yAFt%*(UA?y4%HYVOv`BVGkqH*n(M#qku~}wk_8Kle}fYOT_5TDOc-nt-%B6+VBqK}`&6*- zDJc75`q^N((I#oV*)t0LO4=U2!J9?I9-&}i_(>uRYP3rlBNG*NYuA{j!-)Nac zKR)-PF8h-9UsoXFJ@F6?ar)nfn5XPr*8c?V@TUOP^JhmN-~8P#Sk*9rz$Y5(qaj8x zYqlEP+ba7BK_T5&3{tS&}I-Hh$ zNsRXW;4S+1XWh~=Y*_!S+Qn8A-@gsE3hg;9Hm<`uHeD0YA)K^k9rX1EQP<`S$|t(9 zv`V1O7W)Yr#TwOaY`>QsFWm6F?%6t1-1_ypz{&8!fGVR;M;gN-B=n?&veQq3bbVUb z)koY$=p^a=+iPa4_DU0qyKt7ci1AtA$xlW|Y&-K+Bw_Pce9mI=vx-6lLQ4FtZXQ-T z%T3tfm%x-NZyO<&ZpR+x18Mg^#a`_8p}0H*#UT`kpV-x}Z$jn&(BU0BT?ZBZ>6j@b z;S8UGgZ_=NeITAIS>HKx>+^j@y^6PFK~VMwJIdG6%`9dq5cvM^NhjU4?%;d@;x~s0US5(6`XG6O`m4=pf>D{K{UT!wl=^wpgI}TDdajoreSfzlYytVZ9fku`MgRjqJGg`XianiSJ;v^7(kkv*Pl;$`D&c?PE!;H_7msw_to z?BB3n`_QODQ3GnTs(Zt1Ct^ozW-#0}(Syi=5%&NKhkEa5&qOl=MFI-2@J__VD zv8@fCfsXsB#1q3uK~o<(j9ScMmXLcktWPLgsd0L&Hq|sf8O>+=G#KD zk4MRCr8=kAwexW=!Az0)6tzD6L%$OYsmwHbFQOD2N_WiV+i`Jfp!P*tmOpu86lTA} z()?JP2t7Wb6O*0O4`>!R{30PT%G}F#OZMP3qQ4~*TMd7+=IV*~iPeHe!VTe#GemEY z)3BSQA7Ip2Z$& zK`mviCY>y8nVGgN4mUy}ny$jgCIaDH9I-6OmKhIpF--Hw*io3ulj^3HK`-bLOSt34 zL#j`$632v0Kk#y%Zo7xCuF;?RRisTIr)$xUm!X_3po=on*3e{>twRC43*p<7+ZF zOWwULU%@xh_rXhUB#v*Q5%Aof6J3s${T0gF8Zw=8|A!5%K__VW=6PdU+ZPjpyS6n+ zjAJZ^d4cTQ82IW&uF5I{%tr$GiE2y#^DUAk-QeSBjw6b|CW<(oaNDM<3I$~8&?S&K z@AE%kwmWV_eb`ryzcRj$;r>nMm5EwrBWY5~8HP_9uSNzb`geYaAQ2mBEjp|07Y?!C zyULg(KAe)Fl>0bS~o*hrxb3do>00uX{)J2z5SKJP2v+^^>__JdKw zJueQYW!d40^R{Lb|DFHSC`W?FN6v0PNb%8YPni2~PA1(;%XR_5-g|Ziwx;flH?LHa zLoh6$>_As4+rWc*xJ~d172j$coW&ESKrEqM?VrZ)Dol7=e_@8pK1(6J{M@t)XhnD_ z@rpY~@R=&T@Ha0@_ktf+M1vSEno3H=Lk!yFLwrp{r=^6?PAW(4Y@UexH*K@vwqRj2 z5}Z~Kmlxx!Ny$AV9f{HN2HRTu_z3hiLm#CbFM%L$1ME+0MzOeB{P3v=*4pp|$-*i} z^ETP+iKZR~>!hP)nu7d2k`KS1|18h=eqL%tn9xiFf2RD>UMOrA_<4!_!?bLQDx*a7 z>j08Iq5?a;@t$|K*NXzELVWnKdg zgqz@xE&t-Z6oI~v`*E^GOv@6TCIQ=QE@RC1VV`m^(YLh&>do=jAErt_>SYeyQh&O^G0x68e*Rf|)ct_sky(B#~K#9givfys^0>JRd)Y>^Vq}{1E%O zHqIW=%c)>lms_?jG=-=mzFZRclYLvIWjDpWt`)rt;?Xr>t3vRq5)!Hod2te35)bG9 z_(hk?fv&?Kb|*_R+7GqAG>P$wC4Vv7;~hb()hIG63orEL(JMZl;BotAjkx2fq#>@WErVn1<=`%gm57lCNs5m?W$uv`Fq zI{roRc%rgB{o$X;>K78Uf)}37NiIZ`yQ!ZnSY5>b)K0gUc@G@yBi(UvtYehWuCJeB zun7#o4+7Ft5EJIVm!PHfwYj)Gy+mH7g5HMq3O;GuVb&TBfs-+FL_xh94}dnpD~S8L z>Qr4p%TKrHdcqgY-NME+HUiu&SeoKO*v&k%F3;9Q6*^_E94rr0+50fd%P1)=Q&!z| ztm~)7Hmi&ID;Cw>x{($KLfw=5``X5%?2F*C59(u0L-9P=)}vEk=FVnaM5O$*xRF`0S^ zIcB|r*%)@)(^VMqFGvIqN;cYGIo)yOPDG!<@{VYW}W#bm6kRDQ__@Q0k?2FMqbran?M;Gote*q2*6R|0S`9 z^)zRfxlDCNDGu(I_{*oIDH5b<%gzKhPk&)=Ja|3Zwyc9Z@^By`iw;(QOBoPnixJki z%r=%QzesJ9p%4z{KMd%YE|XY-+Q@ z_}5iRdju$?!>Y7>^3?B2d3rn!-e^}_kuI|L z9QQjEOP6&pFVhXrV}7A$iBl4krt~w#aKfHIF-?bEXk#qXDIc54W55w`cM?ysffM2O ze;NK--#x_$tX#64t?8n~A`B^Coa?ii;#esB-t0NvKJALROh_o139 znQ)Ve0&goS=Kkhydg7~WWA^OAw9NpLPl-45mW_$aM0) zjF9R+9X!+}FnhG<*?bo%%T(>RT!TT21p84R;q8t}vNfy!{04ZLSUklv_~?U(vY|>Q zWT}jF_`PvcTn~*P(Dp5I{zEj6+mD^;Wjs_!=nr+Dp#SObwC8zZG3CvtRK_Tzi#WI< zIk9Y#=c;~jjXnY^$n1L9w7yh4%hkx*I2R_|M*Y|jm6V~&3O*job{*pKCv+}re9_qw zCi`o9_Rq&wD{p+g5;rHT+gHD{M_U3;;<6apIy{&;h=_S&M33A~^FCOD%_LcXj1?+? zfJ{$|>*spgk!c^zm^7|y9cG0&W-j;MdZir;WRhqD#2;Uo=`tjKX`eOKp$}cEEM}}_ z#m**wH=%H1{HGp(UcEjwvNBkqZqw4aNKSiz%rqQis4ZjsQy%Xv<9OM`TZV;INY|1)lXJZ00mD#+FkBv9mXPw zbYHvCasGMot9(a({}P9Al8A@DR*tMOW~9sWXA~trj4a-vTSygZruo9S=M%qB%BY)? zMiF-s;|>+piz!}7j&l*XXV9XhB~?Ci={ChqSlUr#WmGweW!n|pSnN2Y zRuU|9MJ`#RsHITujC7QR5dET2g51}wyi~d zfPwf)R4+E_-merWCQeF4$6RD#Q?3pn^Uj=|9b%4x`Ut7zeAJYIP60INUJl9n>4eR; z3dz>I-eaGbSqPq4Pcd*Tzk--1mX`3UTQM%ZQ0OxoC7s%}(B z=&Rtx$sn-;Uh`L>u;L>CC^7`|NBk56b$bk&=?xU{my$H?)52dSW4IJIUm#-bE&?1 zV(CKqj3Nb0AW^heK4e)p9DnGa*+@Xtrk;T?LDHlV4lhXZ?jlzi-n5b!-1j~{=fd?j z^8YUvX`m$@dQUvrHGlLZMf|TVKO#?DgedTNpDXn+MFi|M+&B?t_W~kiH}r>-V*;D$ zd_P#VX-RQ8Bp{%4_2oJBE#hV7kFyXAYnTtFavA>wb}sJ>o?i9BXNB{>xbS@munG+; z8GqED>r$i`(i`bGgTt4~y~QFGEtno?S~^$(%xL z+|QIf&m@cHYEk|hO^?WP^Qme26=lzp^%b6npyeE=-N1lD&bQPmkwRQywa9URxEseQ zuV-(&2*)`O6*encS0JkNuL{kVZy7$KlV2O`VFn&@bz>5WvAJVg!uljWn)6MzhJL($8oCwsdj{zSLU7 zP3WIE=GOF#3rUPgS+cr@u5^Ur!^_uTLe8&CsxY4kwKLC^#^(mC#!d-IrgS}T4f)<^ zj={fP?$%&zKh9Qs1t1G)gf4g2!+YdrsEuXmqRYIXXw%WeW4Fo^Y9oi9QHq_hQ)3!0 z4S5>8Utx}f&>EGN%oBbtyz%^um=**CkMF6kbK#xo^6?v8GrEn((=KECMB*1DbrGZQ>MyP{B|pmrf&5K=&MV41@C4cU! zey`L~jKJ1WqYVdOOubkXr=(=CH-N?Y>djOTpvwIEN`M7-e&%Im(&kz)JfDdi3Q10PH2XK`V|Xx{9B}1Rm;|Are2Q6 zU-3$0>A_YiWY;8~yt$Nh{v8G!kV&tS*-czsAbz)#AJ?D>zZI+$Ip>`7E?28YI@tbv zu$%wAqFxfh*2KuFSbWLqSlpD&F{yl%G+Q;U$~I@dl*RP%Rqwl}U1ijUYR|yER;NHG zX7yU7telRnx$(K&!fnnw789qDa>H1TxAJh=!QStz)0PUo2|Ic~Lr{r4-i(D&>?7&7 zc!|8=xQFnsbyuG`2#mjZHy_N%-hJI#KaW$+-!3zUi>|m0Qw&M(bW*H3U zF_j)Oui#c#1gUj4nE7un(Mkd@+AL-%uK#Ko%HV%3oU`~OO4gcMH_*!ss7i zoW1{ELTzn^?U5!_mhyGEzNJUSuCIOexpvt*bIh!yD)gT$9NxkEDzN4n7^k)e79PEV z&M&1U|H^9PF+ntI8|aL2<$HUb(Q16iV<_8WOVf0fH|~3UyAB} zlJ+n79Fb|QfKHhL=L!GRh?@QL#1!r8OPy1jE9EjkJ67QjSl7voF?{PI3uLMuh|Mw_ z>M>GMWN5`Gy=43QxzCmIch!Y1*)uy+oy4k%Tn=UX2T6l7)*X(In0AO|kuk!rLv~O3 z++R|tPr9^yJ9bk7aM#H+ZHry~tmv}<0jxpIuTpTE1k1IPW==-3fe9O9)675D> zDa_&EKd2~dow4hCUMO7lKy^jc89?S)Tg#H9Jw_}m*anVS_{%!M==_@|mp~?C&<%HK zBPhKfBEl1(SP;BlrA|{HJ0*~+bA$%Sy%MOZs_C*BNNXG`=~uLz3YfPzw&7@vPPLnT z_L*VKWB{$JZ(iTX(N~yIHFO5;ny?4W4^2j9>vbC5M^MmxybDnfWq9HSY>H+C;U!_9Pgf?gjC$=^BV4n2AzVTg`L51No|B_@7nskyz!MK>pZ& z3tVC;S*o$EkPMk9X#9El(6#`!yL$sK9!}o`2%S9+dIZO@=EhW~o^T!x{Nq`{rggc~hX>*)F5j}@!QkM-Z)v=tlvVjgje61e%~ zq?f4HUtmPVHgEBeY^eIicYAj{u_egyj%q70VsBBRAu4~gbd}(>11 z2m=AE&ghH#;i!#zBZ^mN$JIu;(oZ`v0?olGWSePM)G_H$ZbO^*khsHG$P1Fgo=TZG z(iCIG(5)9WfExXs3AR3)Opr0huL$q;$L6m*Bi{4`H@)15&ralPQZDt6G0KGE?149R z&AGD{+rl3xw!}zo{wlLyUCtz;T~B!w-k`l#&C9?JBbl*C2|*`WaTwTHw#Av5bWq!5f3mr6X|1BuN@h-Y*NRGBRw5xA7#*xTS)2mwCy= zB@?8o#Js&UbIQVhRHzU8)j;-Tz_hE282GyJeAJ!Gi^Nw*Uz)!GgQHLvWYI8h3YxK!9L@ zKyY_=cc*ceZfNM6``)GsS7L#Z}XN-DtPDW!~`>kjo|O)n97 z>hTUrVK~M0V(75k;NmP$BRnxOoQN$$GNk)#j*K~=;aZhe7-9S^>@`^ zQ2J;uUYWIt;hno?4u8l&^_4B0jARVTy>$|OCY?7UMEJX}!SsCatm=udQR3@hJ5)e~ zNQgQ8+r=Q4y8hOla!lc;GOv_cRA8e-_=1E#k$xGqbzMVrpM>u#z6D1rkMtfI9gkh+ zF32({My_|64&U+fG|>;czI=>$YCU#*Y}_RR;;H63F3#^RrG12Y`kd0Y-OU^VHk3HS zP%C6>-&58i;eH6-o4*udYes+htXu9K2KbN8L*5Ej%^iY$g?U$>ZeR&Wp>WbrRD6>_ zrHGjmJ;=)*(1V0ZBr&c}Ec}?uPin6b(}-9HSAL;bNNU**oqkETrR@nsL%3t4*yhz z^e2f${3^Z9FaXVGTt6c5C&2uzvNX!*2*bh&D)3Be-Dhk+x}yJ)B!r+Jq8^s za9)%tEN#F&ye=TqCJim+CVZ3US^3hpycA=GoNxjE7bm*!r85TUUeMqBqvoH0J-SL_ zudCeW&xW&08k`AIEcX+1y0R>639Z5U7S?r)=unIq&4zi6ZP`J1on;~Lc6z%0OM8px zAGet|5+Cx((IR%}p{i@z@$Jpfm^K2uSQ5LlwHsK@JtpI8_`bR}&x^~cpq@2t+>NK@ zEE^X?U>FnI<9ZpXx;~ylbXIqLd<`fuf>mf0VfMr5#^jp>*zxDYP{Qpt-CwVzFqt5z zW*h1TV+CQy!cXZh;aG~gvG~>W3Cgc26FVzKC!U&O3XZ`|zKfCh+ZKHAroH2kZK(y6 z)h{#bwH?z=IQjJ{;_}g(G2hXerTNVlF&ox*$hQ`#T-QBF*->%gyVe6InbtY-QpXb3eFT`DFU5wWs#q*tk#Br5$z zjYsU+uS%6AXW!wM#@79+x17snl%lO@-dJ5pQWH6`ip)WmjI-vZlGn*7qq2l(y}|8h)W$bYh)i z6Ck&m_0nlojX+msVIrW!Kl$}JfX9!bbX84w=TUC`pQlt^`mNbv!DLn$M7u=WrFH*(^W*MG<)SIT+Q!dCQIpC)c}CbZyr z*)|@^n+0LW-IN@vtwZ}Q0^WIQ1M9R^2ppvM6=}{tIQDJ&8hIb(p_(nSDqBylRP%~1 z4^ZR^LU)t?LhyVpaqIQKd&he=f-cFs%4nVNK+XJ#nH>#_NjAJgV!ep1)$hJJNql~A zU4v~T=JGexR?>)1c6kgs&AzucCoWbjtvqk+2uQyqp*&KN0N#PZ><4lRI3Y8m)sxmS zn1a6J(?OI_kYJ5W7#~D|fBMyD!f+NO_bdiOdUfP)CHOz|pQ9$oq9*vU)mec^MiBGg z1{tZ2s(9WAk+=pVJwyo&pNopm-+s_Z_z-+B{*dxj)(7td1H~}1VD&TRy(QUI1akg- zMJK~c%I3Kf6H<+)S98t`t8y!0@6Ey-bG%_qAJ1e0=a4ycqXo$7L64;a@AtV&$@Tbfx2TL0t{DU2cDS ziC&2tE{UjBo-d_A@9R0?&Sn%wENZPsPf$ghh_KBTXAH*T?ko|=zO#JyIOvs338TCH zSKX%Vep1|;e2nERP6UUh%<26tc(IZ$ahHw){w?KD1O3=C*2B+75lOMuHcRdsEbCa} zGm+w88fUmEmom{<&Dp>1F|D;jeaSOm92bzXqP58#OF4Ol1c}s8rbdY=s2>a%Y;ajE z6`$n?gqdV~1B&59TQMJFn*THV@2yby@hW6(B(PD4VD+>K+*xkE(4KeAD7Ork%%bc~ z&vAuLLBD7cOIVHgvNewZf{8l0Sp|YTWVw+yFPQu11_IeW0M>K$!y7gOi>K0A$Rxg!#(3ETQGiE<*l`UF1)q>r zHdov4ox4MPE_o-^guuEKnrAl`L6$<5V_Zq4&MqG9pUR^Y$9v&KCHDR?@-D;ZBm>*m zGxe8Yi`BSsKi-7RH&4mm7+k*kZQqor|DZF2PKYma2{!rjX&t_6uywUssnf|h* z+w^VuM(v}^4oR$zI>lg|xP)&1tUkEjWa-iuamzWCd1JVR-{FkmsXS^xXGvB*99WWL zxvX?2ia)ej9^#tCQ6LaBkzi+$RfQM|KlsV^u>0Pcf`)|+*YR+40?P%NU8kLo>ZYv5 zt&E1>Hm#g%(R9|txMv~6rNwZv1KBxyi?~=}(qPRkWC|HaQAb6MGl*10KaVf8yf53@ z4bg3=ED6t9VwzV+Dj62?+7!d`bQM-R8*{giDBBlukF!Z5e0%gcrMsC8QrM!nvcIUC zFJmGe_I0qhfdC`?Zulg#r*2?I+?sZ-YXghVE+L7y(oo)sF2L_YG)gIhBqE`dRBl%! zz$}oXTG>sHJ-{n;F_=ob!aT-*63S(#npCrA`IS;KFUSdHVZUW5<$&jYKmDl7B|2RZ zCt>xDp-!iFsc8?l46)ka3r6t+VWN?!q)vY@!DiT9kkz_wFP(crqK0Fx`5VS1$(QBq zEAWNUC^;B;x0;=jrb}w8QO65M3##+|5CadrES;I!dV%a#dkHtPH-UGO`Ux9nIhRLK z77TlKzv58|wJBa42V0vBS4Oyz6Hv;>shy8+Or01F*YOoyy|(7ASnfLEjnM1=z)!Ca z5-?Mao#tHVaFMRj9L#1QQqbZ$fO<#Dq|8`(p1V&9|iSvu6McPxlkZbIb?J}Bz!f&x62*eHK@d@`h zYT;PZ8N=erzCB(->rK}C0{IKi$G=+vX%oURX}=@M-ib`7IQ0rtHq;MGt2hs)or%xP z{=6{VMh^t>X{j-?8?3979h5;DQpV7#kyJ~V)qJ{>KG%rI_8wiJtGIpHIGO#x`ot}-O3VN$x|youbc|ow=&Wi|G7+)sNH1F#+hk_@yDt+(ZM&b1LY#; zx8QUE5^?mRw*N;59WBoGtFNwQr?O?6d96A6Ver)RKVuoE;RhuG8HJDwc+#8pTUsV{ zI^%$SonQ1t#0P?7Q_ig(DJ7P4Aw{{_)9P9ts-C7s9!hC8hAq|A(N71lxJDIN=XKYi zB84<%NtE&QXa$@{&2bl4t;PrLIZDgf*0_pQ0?QmdYxY#r1W6}VYN?`3+FG+JZ+Hh4 zrYI%B7P+y-!d2HI77wmRwHY1ZIW*RIDVv7EXDpFkzn69nh*W;bgIZalq>84pxI zm46ZkaIG8>+%f%W)cAZ!LO>3U<4){P6`Fo|8_ord%gS@LAt7CBXissvNxFcUyLiK2&ZJPP12cy3z2ItU?{r6nJs7kx@vAzL|2s$9EwqU7o3I6^pc4BX zg@qnUKArV*Wu!0@lN0Y`8xWt9+1@7VRl?Qd5!DktjqVciKu?b@!1cTEtF6wPr^*j1 z=XD-w5ge!A_`JSzW}d}d2YPrG?WY;S&D4|W^%lP$A#XheOv@I#PR^6!w?yL?bsY7% z7*n!mq?^eV*}AL6XN(7x_@8|f42C-B?H8UiQQ{VFHbqp8-CuEu5Z>^%QWsuDfb+UO zVflE^sx6TLM??9>NDGdVTg{+SPhiiPwkIt<%jxe0?>!*TS%5UPf5!65T=7T&pi{B? zDB4fJPto}}0a)wNSp55iKso6+MtGj^Q&~|~Aocd&=0EK`K4|RUPXaj>lXQ9d7`kje zt_vwU!Ds6{GPLjY?8Tg-HogvpUvF9bMt2#geGyYb=EAW3zb@mST5^JtU}NZ}=iiz` z-hUw(Ff(2?@F4J!>31`x(uphxkN(tpbib&glBGzfIv6@Y@f}|gR*e!|;y5Q{jWmDd zMIw)`AGPNiwfgdnI<1-78~8~wPQ-gIw8Hzkv3oZTH1VV8*`1Sb%?QZfAB<66Pxv1} zptl`=%=#m+Q+OF3y9;Uu=xCNJ=@X{N;J&?S$_F;lV(1{b4{sb^iwAg8wCvP=V0DEP zcSa#~3erJ$%XQ;S;CTUPu{u%b*xT!poyIye@+O2x9`m)8lsxuVQ>~f0i5QMxunzaq z_8RyO);EZ$D^a#1YMW+v9E*+|S~jJ4<<)84x;-2EWsy9CKbt}tcXNhK2Xta93WU&C zYBdhL^US?6jkym~Ts%`N38=#5%`3V5xb8;*s|uhLU}DGY9+P;gdF8u3MA*`33V{xD zZj!#J={m+@ECKQ~VxWa$#^tEln|sXyAC&&CRapd;rR7_7OWzFx&_veB#VF@r8Zw!1 z6sqH?Lpxcgh#Ezm%{MP|h=LPRc^t)`mKK1lZklB?`dmr4_r|14S0hcoR)5V@<(;7z zct8S@le>3UJ^X}q^;k79ymXCty!|^}E?Se{z=V@_F8jJ+NDV|SMPA(}GEDqMyL!;% zJq6)+K@lL%_k%ID@`y##9M|$;805?3hfg$XE>mgyiRg^QGD5Z6R)47{75uaGvS#ja z?i4kuTN1yAi^|Pn;YO}Znmd*w?k1czx)x;xh^X;{eE(jzNcyE7n-)5$Aw~7aQU4_G z2hx8skH?B6<{C=dAFzFVuu~RpN!rR=`|Nb}t;$g!k$+Tkp7Fz1t)JPy^QmuNu|1SL zf0IM=%^@Vm<_7o00uof8ZZvNSIJ3UB+4fjPeV(xbOsRE8PZ`i~?;b2by+G2k1gyje zJb|s(S=bwqle&uv#gx8(rHNE3Uu!&mqvc7u4+wt9f;j5hHVX;eKCRLqsYMn1BuyO^ z_W(%HBIr<~d)=dTp|44u)sMhiTgdVqK zwPJ7!TeX^3F)3gDx8JGfBLYk{T$W2H+Aefn5$e#mR@Uv_YaQ(jjc(Yabk(asuGxpn zB^;k_yWAJ78^Zv72^P*sMg90!-O!=OB5ILEYXvbcUHfH(Dy+H+QR@?@)&=devkVq8 z=IA4DmfzAZvaUf-^Jk%-_;x0Y*Mcw`(AiQROp&uWJ!+#nn8fEWKq?1jWS^_BGI;5!x`$uV$vW{QUS-4tRWZx8AdIFeZ5H%VW|Df}agPCZZ|PdNZ*zU3 z`k3>=kwe6xxf8HqYwX_f^aW;XZ4p8gv5mtw-AYkx6I!fsP<~vsnA5q1_BYi@dAQX> zPxf&m5IqKaX;8pA`%=}t;7Kwtx^YNcbkD8$%tFT&~SQzaVf{x#3_o6Gn|uKs@-OJnEAZJf$WSJA_iFh-OP6LDY}9r4x~ z8K*v%it@Ek7(=8texJml5qcT=Y?jW)zK`p=cFp^Fj`_p@e!?Yzli%a^;kgjw58+PAW{S@Q^nMd`k%w?0!dc(ok}^g z1EXz0uM`FfRXo4K9v%DkRHBVxX7*?&P;#2TOKhRWFsn}HTy8h^)EalMo}7|Knp2kM z+~29H>qK~m1l*@JkLeSq!~YU(SRAU{Us2`IGNxLLU37|l=@;Y3W=a2Qa=g4*&IXv% zK6Q0J*t{s zq190sO8UN|IK>~%q`^+T>6-gW)5(M6YU4(ytoXaVu{&&*Xt9NGxPoMkxy8nJ?zRA4 zrYMZa_b%65)fxTrF2ehH8#E752*|FE+@3W1^_WvM@beP=>f9DwqNoNg@Z;FM;|sjn zVFZPJ0TWevK5@e18Q;t{=BFutrrYE1v=<{;3~&QaIyeg6zMR#cv{z_<&s~%VA?{t* z^`r^NREX=l)70CE>H6UDzWm zXSct0Qd=fHf4mLDMSH~s{jt6BB}ItjI;nq^3=U~7fPY=; z-k~8x)U4P%&PEK=$XzOkK81T_wX>=9s>fmyTF+joCf4XRSf4N3^O5_7so6QK7=_O;9eL~1lhj<1D(V)AA4%$v7X+K1Owg+Q z7}}v68Q?7{4Sd`#s0j|~n2+3TMqt8P6IL0po7~Z9$@uK%@Pkn(=%EdJhfq~alHR94 z;?Cw6IXnJvZ=n%ssXW&Lvj88}<`;7dVT~|+Q@__2Q8?|l2b!6hyeXyEPC^w|Gj7#I zj9@F~Rc$_uqhbYT65KJO0EDr1I9cKTOuoU6c2yj{>b%S049c`;an&_D&QnT<-q@hP z%+w%&YW7ZB;>{WY4|qs&=vwhK+12qpol3;opEdzSJQ6CLmbyD98xm31QeLWYnj%n# zOC)EWaH@vkW!{%wq&bA~9$a%@U28d7D%)A)!n0oW%i@J*VtM6H&kx1ro@|uM7<$#> z^TjmcEU^(RKSU78r!I0@Z6?Wk$J?3foiqMgZpR%b;`J{K=taWJb{aI^t}^JWNqwf^ zLfLt4Fw^t4vs=UlahBV{{mf0+0@1K~%jm+c4`$y3Euo(ta&ybRVi8L!sp{0%gO&z; zx%y6gSL^XJElbJ<^rc}|=T8iDn_Zmb(C_EkI$|D%ca~M79!Vb_zLQ{J&=G`V2y&oh z_5|w*E?UsZg>Ry1{AjcF66X%>oOeh`X(S*Ij*5dcp6Na8Spb) z2+Obk;NxA*0Vc6_UilwQ)wMHzf&c;KsXtdcK<(UR0WX>wl7`f?OcphsVL&q9j~Fv2 zj)#_`bl;uSR~E;s9X-7Toc`Ykg9O}(?zHmcYiWZ{{RtADYJms{3fqBCGF-Ul*hSJZo-+6q#R|GVpbx zH>{kaUG#3{X1Ec&m{z+o{O;#vt$OOXT}0EP(|2}+y*OsLkqU;lLfk%N*SG}yvdWH? zYHxfH6Mq`| z@Z0FN!4kmL%tumdf7{*%uFRAw*Pzh(BJ3fkPM%Y*cwMSaF}29mUJ~sZ%I%{t8fkoa zBmhaN`I*>VqWO#M#2?#@5U0grZpDuF=mY^0ij#wZMdN7MsCd!ytq%nS4Eu@v*NP`p zO`8%u*zMI)StfsGTvo| z9fmz_dN=kX8(@N-wOfBK+002_TQzdnCiQTZJPQZPwUTiZl*{a5tL-Q4`wb@YPF;&J z4O4&4ex(T1Ul+fB7p5?K792>z>lBxB2MeOl5?qeqQkW2%x-Go})J{If~9j2}S8 z-kXhb+oY<7plf)&P7;wvw8Jp7HGDp7Rxc`HTgv=Wr2q};Fm`O47L!;rcLZ!a99QVJ zFFxel64leKcSf`z5Q8ADV8xi=;M1N@h4y#7?^;>iGQ`%-pnhrQbWTSIj(4}?a zl=xm_O=0Z6Y1}1mJQa7wZ-%?RJCBV9Ouy?p#GNWxDAwXPsyVPX8MUaDE|h%Yt6xth z{)JiI)i2YSjx30Z_fw8AFva?>FONp++V``pnRp;c6(vy$E{*lYzQ`QtMpC2C+ecKd zWaVR=Op6uW4v^=besTzMAGPTcP~uO$3zU+#1N z3e0ScoQ5P&P3(_E2GhoG{4#W?TB8pPc@>GXjuyJ{`pYw-_b2s+_7FNGU5h;JRT!tP zy8Njz{g;L(L2RmQMVY;E9kof3(Sx~TPA;1(x90$+34p6GbP?Rfw&!jO!|*M%4db#Lu-VJXO<`P$nn#I#8f_Abq1-1Aj3wBEq5Dgt)hNre2&va8G&c>d*$=5Tn178@f1oVA-i8-B8NEYR9_pns^owTx+cLi&S$l_2_Z~( zuOn5NGH=Ko7=uVjs)*QJzMyn%cR2^j%gZ~QbkI9q^=-qu%*=-`gg+g$i3OFVvyaOP zd0=gG?fu>@>worp%}w5?4qaBm*c$kk+RV2@`lJ3~0mEBE%i3cTUWFbrU_fX_C0LAu z8b)P=P*)KwcJszo9py%)hObPI`$t5Tp$3_W)zYSmx9U|X1IN! zQ1-12MK%B2&Ls(Qqzj+@PgAl1qmCyO8Hr{ఊTB}L+>gQ9HrzJ5wuk{S*m!Z(d zQ!47`yi1?j?`OLQ z7DK@a$2DL=0b^7O3{%<#@mb6)hEdL+se1IhWFcbX+&-h`+tc4u!+i^}S(I|%75(_P zQbg-m@0#l0)14))|B|CS{ZT6!uVSQ7x*j}40kWtu<%`yY_h2#2`KUW>qLfkN(M_I@ zmRlC>-CFC?i7dvZFX<=n#NYz#aLZ^LePy(jFI}ecmSAYLm#Swq@VjY%Qft}gNbX#k z*N{kHg{Lb)7Q~VW$w;XydkcJMWf5_5s`X@Q6Sr7bTDJn=TdN&NTmz@KGb_qmWbV5r z@?t55uK2RMxzzj53m za+0RPn;+)NsbZ@|haq_Q+IlgDpw@SGCnfepsS^2XKf)POG4aVmcY?$(WA3_wq%R=e zWs(tCbIV00eX>bdQo~`z{=N~Q)C}U{JRa*4hEdmT5xsS$i^I3v4Q^%55OO$P@#WYC z9{}~MLEZM2s;sXMV$hKQg+%Ii(%C{*k=~p#+xX9U4vJ6y*nvzR-N-670@}NBo{`0G zE@%SY*&*fN-JjJ3IU?mh1@!)8Snk9N?l0xJFW*SavTA|VKYFy>P#9r|#a@*gwtkA! zdhpr8X-|1MI<}e7%KoxKDWiVCAix76Rnz~XOqf&NWc{hVS6`gxi%Wh3nrHukgfxp8 z=3IHNjGPboJTGQ4lGW;jCr6O;xLMBMJG>=pkV)xX2umU)445L7@#&=p>D&G=#xyL+ zvuHM*lY`3Zzz;v0=aDes)@>=tl{J)xbL3BvU*Neq+<3{Hh0!DX;RNCU@zCc906)8$ zUCE_Y;Kw>`9>D{1?eLt{k;rtRSf*^ScW=(O_^&b->vJ2ns``CDZFs+;FE4)Kp=qv@ zaW>N)oPc{~NS%3Ok^JYeU!m(#-I@t_hhE<%deovxFA2&6?L@MW_Ac~iYAw><>!h%m zf7rZowNBp}@0jpQO_#j9`n!lwT#ojsw0uFfCW7B;o-MnSAHNQhyL}LFE7jyP{a{_vAr$N8B>B)-Y`hHGA{e#MPmw@R<}; z8t&FN+T$jk1+_QTV^fw_QYKt=+dM8yS?kfqj&i?f{-jtCv%kcl#@denoaM4q%Ht0! zkD6Wr{1GH`D@xZMvYa+zaaun-4N!Q4RNk*Oh~y@V>gk2Nkv;xs@_01am@_+@aBo*z zZA%G|7LPP|U!el!gQHt9DKc1?hue4VT}G4*DTw8uap2XzH1Tp!;ms}M8OMm1Qe97I z0@aZwe+FFe-`G}jZejU31s_&+0;<2|EL(gWVl39#TTC4|U1<)|{z+la30DI-M>`q*^7CJdl-W9PK*3&oxLy`U)6-V+;qD`yRd+wb>2+Z&pP!EDyOYpnGz`uHRU zi&%YXZZ7<%cz>mbWH6SqQx z&~K%3MZ+pTVx=^T?Rdi5j^J(q1=YL4aLaA;AY4enNFJ+t@$La9X=~f zzIeH4lf)JQ9i<(NcxxNfg*}eWI+gTk55CP|L66D0_M-*nUM&wsKG^YqL?41*czEU- zMyn!8z8k?+j_Yr<`DN58=JjLH;Q>D<>XJCZjx!qh{-ylo^w(jWK*H8guXwD#;PUV> z+s9AlSlK)u6}{C~3=RZGYCmgF*3}47&3%blK2slF1FaDK<$kD8vayB65Rnc`1fFH%`Ut76w9H`*Uu$2$G5bsJ~6|?pe zz^$t-IRiHi*Wvm~J2)0wA@j+0|FU~X>ldq9<55=Z#VTzaXs2f2yKGMG26cbK_8C>? z&-gnbNCe@D_lkci`^>b^w-q0gD&Z#p7)BXm6+E%u8|)G7Yz{XOwswB(I*zPlRUZ|a zHnDB=(RiDN!Ntx3)&6mvaR1Od=oy~irwl<-YV!|tVS*w`f_MKvm}{*8z!~$)NZ0Do z81eEF^0rHK$rWjzsd7}KWYut$-Ydz=)2my)pN;!K|HCp4e^gZ}C54L{ulvmRrCQ7w z#%6!j7hIM2zB~}0A~gEc1e@NgVy&ZjDbKO z-q+gOiIj2@Kb4jISHCtZA6J0R_!GIY+P3HM<=@K>*I3WIV8>H`hD!lYmQx_+tF=#^ zU~T`BUb%n%;;Tv*SKZ%^<-U`9f*t?1l+z9t7%GJG0I2(hxD)>jligy57!EiF#jeEm zRvo|W%k*}gV$2Owf(bhsTRxnZtnld%i{C=0x@zAdPWHvxM$gG>N>E?FoEL%udIH4@ zBu}5zfJnyQwyqY%6bh;!RXeFW^Vh@>?D=7Cz?*ep(Ukp(M%Ne^>$!d56;YYaA07xV zwkcRv0mJdrYE!wEyH&R6hNDarEAWAchWOofE`#iJ6*eFqiTWnkZ&-R+U{M$P7oj?0 ze?d2Z)6(=SSg~ykBGf~2&VDDdpyCMOg~E8>V?2r}oaF9?9tucqiC@FHmK`4p%vC{bO7=`(O&MWNNpOZtOI4?tf!0lC99+S zhH`5df$OzJl&Rv(~aX?Tw8V9HH^L6ogGN) zCmq0P1Cx90+C$%Uj_NijJUiK*pI*LwjWc=AiPs$s0|3@Wfgcu8auNa&H_M?{x0&Ojc%11^Ujn}uoIL|{w-+C7;EeEG?;aXOr-`2}4l8Q6I#(H^ zYN@sfpBv9>{~f6t-YL^Lu&WTv@6#k$bzO3pTXY!<;GBJkEF5h{5Umc>dv;p-B3W?g zV|8;i|DbV22AJwgJfs|Z6N1Y4djqgTVi#a~8>FjZcYH_NOo63=WAn}FKmyo@$KeLp z_=|jjvzsWcH?fiPTcFRwJ^Iu1wt>GvTjo7v$(X4wsavy41>C#3J?5R(lD;<;T-cj= z{1k5g;GOF)HhxF&8gkzcV-4hjnZ5$Eo)Y~v0kB+kZt-uFK}!{?0G890eNR8SWo{yM^c7>Ps{q zU)oMJ&C}+-O1fWuv=N5AQdoNV`vj|3p}+Y~;4TrDPrLJVg>oJy1bO?jJk)&h;&|iT z@BGjh)a5uOaQ!#0eCNX>wwYQ0)2ggZ&aM06((oA%GfIahovp_Qn0Ze^5s`=*6OweP{f(fbGf< zfIhoT{Wd<~x?}lhyr;9b4h9Ch@5;Vnl)@xBH%}+WMVUbk&mt}{fZy-FOpPmtbm=b7#=ieeI}gydnples0weL*&UEhorq3Jz5Mn7Io9Re|sF?r9Zzq zwZVRa11j8@^kE8^^tDD^<`Bz^)(|ze+D!3@YXQ=;;@^Ta6u>iUPUJsNcgh& zoR$BozeNhQncL2MZS8R9hE0ev-J#v+jI1kdVnfz{xwF4iPRQ*bK(Nz5-#nWRUy2I- zZiPy#&`iqhnWh0gW`MO=p|pVa{_nOpV1oWR9abwsrwts?oZq~GWSivY`Ku~pt-S7& z6SWWX!dHxM5$A}5PtzbyAm0tj?$gFXJ#ZGN0I;)E#&G^A77XKdZNBZsK&e+rch)F4J0BnDRuJR3!h=*r2_?K? zJtv+B9{Nof!*A1GQUHG630&;Jl7Uo#!OvO?0u->E%K)(L5iNw=sxa*-v_<;f{vlg@ zSL2xYo@kab^G@gufvrIFKtSlq8xR=0c_$J|dq7FwWjm#|3OvY5gOs;utoTJ2=0DND zq z%7llbg>qp6&pK`i6C#NI8if}(BtF{{hle#GVlv)k{vx~I+CV1^d%4#`U`%k#MBxLl zP5A7=K%7#><}J6?;PTI{{~;&CmENjO9jxHxt7qGxu_TlwptG=*@XaQ$`5OC1%l)ml zdmz`EBlDk~_<|~|Y9q7`mLX6otaJF!$1pB0bG5ddv>#}6$=U~roaeJ~&UwOm8O&xq z5*Rej3lt9Q_K!a?u>yX8mog_#_Pd>hJiLyU5J& zyW5i-K5^G1{blc8Uo+k}zgn0AWD?cSH}hc*s2=T12XjN7@UG<^DfT$5fSFGt}cI0*eq`td<@pc&(=J7@@b>t2%nh;riIlhz9&3ui7b}a@d`X}b$ zt?2B_jIGOTV02WqtRFK(vrcTsbSho8b7Px|VRogyG$1@@g4F!W+d#dMtH9J|L_d+j zbt4yQXi^|YfJH#!;fAOn*puB(iiQn$5k0iv2iixX}|;W-tve}WT09gBXoP_ zYAo-IyY!y$wTcU4Et`pcKz6}_8Qz(etQ`jFKX-El_JyKtFKJZWGf9}8j=0fMU zyn%>x_^MEV1el6?8F}Vde!9wDpA+kWQNF+hI1Vz#wCm-M*Kbc7s^z=ic9ye_iWYPg zbp-6P-!I=?O>GbZneM&-Do^en^Ehi7+%6k#LLe_a_Gb2H0LiZXYdxQQ;im^RsaHFw z(e>CWiq0Dbm8SEApY{NvyCwC3h0DqJRuJd$*Y=iEh|w>Y zKmh%N;7;bj%U3>?jeu3onb&BsEO)oZyW38h-(RPM_AZx}zd+ifgt**&pE=$FYmfY< zOt_{eCuDXs0nJB#3~$R&tbz2=j)G3Z0)the-bxQj-JwZv#|HLjGlM%2pgaT{BC?@O z4v-KPgPK1N-SUk*p^6O%Qq0Q)ye0TtbfDZcK`|HdQ(xgX8_#)e+atneDv?86y|v`1&weL+~kL}RlT4>CqROq zuNw0d>J!KBM4mEk(}|j+$HqUss_D_duhm zS@0)Mn3c({yY3jURe0QuGUb&9wwn#I*&g3kD+u6WTw0FEvpBZ7#qMq2vp5jbdo2)9 zIc{(`TD&%Cy8_$HEbn#en>?Hpn2WtDROx1TQhJ8c0qz^sHwteF=AWKUIJfsVBztO2 z4P1FnyvN&Bj$}Pvu1%~&q#x!#U+FzQKWE#JKh&EfH}i=~8g}QsEVAajtU+G~0N?EO%m%44Z$|%O>@;{UM*`H{S~o;B12>+x#A=EtWXn zN#Dh`sgyl#Ly`NG1YMpr$-YbLKy=$45TgRoLj?ChJKa_R&#RShc+aOPPuzYc&(zRC zH)DeBoIo@`zDG_TzHvj`rg35IExsEsaO%=s%!46|iAal4C-U=)JhU7tw5#4m0GI}e zGT*Ii@Z2T#*_^r;39m7`&}^W}9OPjLos_f*PtRv*>E^4q{_BAFw`0#K7i_C8vu{P=urr-iMH45sZ@bK#vs$|a7kAb= zh>j~qh2jE#bg3MeWVTWnHJ&rBY?pM00b^TeO{qQkY=9NhwSz!OZ-hY7>XO2#+o$Am zzoX>6g0Q=t<<_SOk#6t}7%RXdbh)Zod4mifA5|5#T;f~^!ZDUJ1Qd>MLtpPU(#nBL z$#)bm;`7vs3GJKrg5Rg7n`;hN_G6;H zU8q?gg#(81d;t+TG+qPorUKu+%XovyUrlb=rd|-8Q$cEv>CgI%CgU5d zmGco!M4Lww)VHGM-KsHD?8Ify)5bDl)%F9Tde4U<8bqFVml6U4WWqc?P?R2auw^F53o8Vi#K ztGr>|yWe%`G$(jYM|_!+d4y#URQvub1 zD%O+=z~6!;hk990nJqclM3w6r&|t8(D-3b+aPl}v^glM(Lmvt5KN3_tX4YYo5j6c5 zeF!`)@&@F8voo6T;NJt`-cj|yy~8){fk&k)>OsKxyz>W z?{K9*y1x4$$MokNo)Z7fJA5@+E+hgiT~{PRz3+dJh>U#x^b(ndQ4f8zO#2Z;Y*UmR zL1JGw8$s&agBwZaKBE>%&a@F6`4MpZ2Zb^Ke1k$2jM_Iq6^3sj0b_U5cWF9DRmih#W(XL!TUDfj@#gQ*oSxJabuIh&)SWb)P)z?+ye7w)zPP1@@+m z5Csn7<9c*XwEMkr&YpL)7@zuaG~&31lR^}^$JqK6d8ULBl|Gxu@?d^h)OE+?UHv{k z%)8;SKgqWfMmxoSkY+I@kW-YJAb3_cpCEMAgEvinJK>HZ^04BLEegT-F)a#pm`fCc z!G%eGMZmmA6h|WVz>z>>{);1tDR7S?g)BukE5)L!nJi6YmX`dD)O8_QhB6pGMV2;M zbM7-keweyE`7fz>c2=$)bp?*jcN&V^la?4tUsgTvlm$-y;;D#0?(tN`(Z1lTNfUeG ztIIJr;%g`gJm6~*OIyW}t1EdTXldzr5@>(2EM3%|c3Dir>ePDDluHHd(3)ST>3T&rbm_yCF?1awa3gaYWo{;O zpA>i^^O%+6BlrBFWwYt&X_mRE=2}y>>9yer+BCojKr!~&iw6+_4zs-r-XBM&7Wkeq z4;lMiK6sA#h4v6p!rxBHm;@-SdYfp156ia$p%0*KARIE&PS88T?>oV$46+%d?>U%0 zhGGhWslssO`KiMXwZ5AL5SeHFh#+-aF@vKBL^Y41NsuLrrqA)AiD9lB>5E|t#1W6> z=t3>w=NelXj}w^DEs5vb`#Ac^+_G0SL9o7wHc{jmY(e}LnfV||icl{rS%xP2XNm$# zuVt#TD4JE8o}2)EdZ{LHS~`+ZwADXPoJRUgBkSG6Ow$j)q_fNuF3}bzUvTVCH=JFiR z(+bAIkQ~Whj?scF;<2MMQaV2cqm5kqq1vZk%L-!bDl6AoD`N%(cu1;h+MZde|BS62 zRsUU*53SLVa$_y1-VwAlYBevdLTEQep>$KdUaP9@Mj)-Oqr>zg{WD17cP>2goLOBz z&qhkuFexas(J&)VXZLqbork?~QMoCpG0iyEv1v`PjlFr(msH1mn*q(KML0@`Bl6%` z^LHyKBh;zw^fxO@`$gLeN5{<=>ljYJ1ZnxNL=VZUFuDc%-eCrZ^0-|je|5Owvo&|*j&t^tHB9il zq@z!MS$9~O*HyK>_>{@fb;iYDQ|yWVvJ&2YQ77fS zECcp>r^a_Z2*h>Rh5yz}!E;Mx=tK7fNbOwbyVtw$Ie;aANM@yc6q<55-qwU26YGDv zB-K?o0_C4EgmMsCxiJ`>_W+zJQwaXd0v^W}5>jhf=)p@7 z0PO^%^Y$u)L=3M?G=p*@8rgxI+!#vEOSJM5F@#o~D4_K*xfRV`_aNvP#%f&Tr~QGC z(XR-~c%lOrqz-}=Z4}PBlhMMK+?*1S|6Mxr0wgUF)=XNAJ4YENGNmj%le)CML&X;|rCR;O z|D;7mqmxB@()pA|J^)K=LN#qPo16|(TSh0~1>|6Ln_lAzM*o;BLviVx(OW9dkaj48 zNGT-M_({eXLIt|pC73y^i=Y{)CA*KgOoGBo&YV&;W5%4KC1_<{mwGy5;SQX=5LM1f z2OuJnPsv_wJ!4B*!=Ym?F(X-lWtq5 zuc(j|8sg-$#mhcC-zx-1oL5W1r#M7(mv#hjCne_#K@+;ttjpgKQWh;|nWnR#XoJrj z^6@EB@cX2qTYNuWect5Vm5Fze6F|YZ(fx^}+-mT2YUrWmSLaLfBJ83Wv`S z1{Ss`Lw0qM4NR^G0nFl>7@Cf7B4v>?MD{$FY4LSoWs$vQ6X8`&@nveL7(?`;s`-j> z^74hXv{169Jz65JS=RT_k`iw8Xvsp=fCP;vd*Tvhsb_PVB!#yUT3l-^sohTdABw(Y z?SWG3oDnIFLe|tJ!ZPX~NsF+-_AqF1jrAd!P_Ya8q9H-6{y00hMtqK|{XL zc!dCKs$3{#+I>*!hPA~3l+6lvA;48~D+vO-?uAg6mQWmFBa#4I8l)&0$#$+6EegO* zP<(fWJAMknuo<*!P_kM%V``mtZ}*s37Y?kzU=wiZ-HlqAb*4Zi@~ zj#3_7`!&kFHJ_Bu?m%5zSa-J7o^ipdVGOvjkiBi5OQ6H~{|NPk1*pUZ+>CcLg5d&o z{>N1uupdP5e-X9wA4Ca&D~e?wP^_*wr8$ydfOeertj1fgS{d8fq%~u%4-F$UoX+pV zEu(F$*{L+8-P(GkD{ave1+=kUDA?jGZy&AEz;lD}vSe+yT&Q~h%gxZsuZ+5m*>v=} zo%MV7^kktu=j*@JNl_fc+5Mn*@<`6(jf=KzW>+{oCj#dL>rih&y~{aZ+P8{Rz@JaV zc8;zlMaKH*Uf^0`Ow0oQAQ5YNoKkg7fgjwpf-+#yYoH{U*(>*K>YiuE0Tpl#Kqu== zoeNKxDFxfRmhtFasjqjhkKS?gXbS5;hQ$m!k`sP?jqjRi^jz)SwVSqNHr_Dx?A+cx z3!C7zd##i2gER1h0`$d+h7+(+?v1w@_Y(G!dQZ#5O|3|3+51O(@rtE1i!yd$1I3(P zU_&%EUVAulJ4h~eal~ATa*)bywCicDXdoYpUdmMv)WxU-*^2eKl$F!2%^FZZAMj(Q z(&!FbZG3iRHr|Ia93H{bxQQtD*P5OX{|tgSicXL3ak};^mNsyQ5e2ifNi{UvA_pQV z?gusw_rvne_&w@iP@ZZ0a@$}C`29PFan{U-AKGClyp7Pkf94nE<$UQF3P;wN17GkL zLm=`}P!})DfL}`KGZ?SosJxUF*xd!g1ZFc3hGrv#_)gH_5(=1zI$m9?YH-yx2Kezk z8ea88FxE^ge5(yKSl?kyVO7Q*TW1CWIz|nW2Xw_7FykHORR|HT5yAtlEt!_=YA4y@ z)PiA3WHw{iU#%<{9ZollIoXd1fRePYQIlyl8C2-RX0Q{UeFX-%*n?CCM>@;PMc0A8iCZ zTY^XUJOA&-MDv+t7}J~*A@cni04G#7Z%7n^P1a;2Ae+P~O3pS0)kis*ats7YMOQZ< z2wsi4Tx?Q(<_P`xEsMLf62{vX%yIwUZOi-yVUN z0wa!1u!2S{IL>WUfhYZZY1|zIcM`u$&kV)eAji%O>fsd)E=>Ca?<^E)zJ$7RfZk zq}14!Pin6zIY(EM{9PGH{>ntvN>n%sjp$ToYU$ z<`8ww!Lw9r<~?!CgBha;?aULBi$zs+w!lb`#D^G4-k2|ex{yfObSrfhPxs0MY3EF% zC5-uOZhRs5^NjPf7{lVu`Y^bx+~}%U_Hgx?7Ne4!n@)>S8xWQFgYbkcgjvq+O)>8W zY#^AY%jyAc-U(*Ah>Bs->V{5T-6I6P*{j)^d`~>*w91@H^0YxtqoE%52lA9}^4l4+ z1$TtTh?{?X-ep@6&oP%3o& zq$Q@1;5*;#%WkyuWMk3^|*Lq3~p9Vktb4 zUXtXp_!4ERbkHl06@CpDEnE?6!Hx1Ir7ukF-?;p;V-KslL4*Y*)e`NLGPa{pT?cYoIHI~v zvPN9Hl}j&2k8xPlS)$c+(wB7fL2+qeREj>V@R(51++nJBm4Bm*Zd>`rkkWtF0x20^ z_q%7vjwThT!lauF=p{Mq0%XXm7k%@`Znk5@_w8xH#hkQ(Q#*)?Y-1iLFx9U2X5VZ%1o zL4bf(JnUBe{*P8m*nmJBz-9CK^*>r={x6TN3PSzAw8}*NN2~pmHfjqx9In|Qc}-{G z8cV*Ik3v>utd=IGC|w&%Vu@@a*A+D^F9t^?(y^;43Ufs;ARm4qUq_lXQ3zBLjdJ$9 zc8C*uE+-p?$PnFDL7)@$Mt-1X*7BL5wn&E<0{+5l97s!oXL?~Knj8kNK(1QO!{{WH zv5kt}$yY_mQeojsY_Xm<6GU+7lIaKT0Io(UjZ zki*Q(OvqpPmV|S_H+tYY*hBeHrV+cQAGuWMClhm&QMX}mu|*q9iG!2s!@N<}Pfc2X ziklLG<+bo~G^mj^Rs4afO3k1*&kv4rW>`b4(-L`Awl$i2hGVQLdCC&?-OG+8lTM;w zC$&+aTi~4rP(W%exn0R>{Rt2A!p2e~uo%2jb(qKFkKQ^779n545(0wVT=dVn635O0>tkf#gxjiOKZLc=KUG zO)=`S7kLL1kj}PN;jLJEDC@2)-GRa~w+Yb;DYHyD!F+pk%Ry_^ng+aLyd0n zpi^0=S7LQIWfjq~1t!_pz;t#ZBb_V-SL62Gl<4}5Z`4=dmQ0Dy8_HT1O}1ON>Z$qW zGvR-Cx!@~o_Z_@#S!~Pa4|T$K*b|J5NrRuLl1c$lRM5^8>y>ms6Gb{Gl>Acrfa=^n z&3PD7$)t^40yDYol@8O{nbkB{F6FUzmY9sjfN<2B)`J?xAK!Uk!xev2txA}Rw>wAJ ziq4l4`32#c4S`AEEzX2F2?yqi8Ub!&_s&2Hk<0jnLHlu~wH>8AI4aalD?%p* zfH*W25Ezzstod9oU=~{~yj6J{ePd6&-|D*Q%P}CEWyoZxhHf3)ksJhKp-}0_cOUIB zZP_`YCmN)swd*Dh)+=Z*^?O;Wc(T`I+sM;9V=r%ZK)GIbTFlp#)UJ?N+!W^0d)g`K z;)~WcPwv1*F&e|4)WfpHM1UJPr#%f_ydKf`{+;?RUBoO&SzKq3s_PJ=VgZ1fN-IoAheAP?JMzHp2hmE z2|QT#p0hxD9&&KWAV$*t$#;2pu-h$FO4?d`x~`{jEhywe&v@)IVg}O^Z_=g z;(%JFT0-}w663Y3kMp|{nSFY&WjsBrtE=t2K8$WCx@T*?X;hF65+!sqPR~MO1uNU8 z%SiH5FAV${vA}0+nhen(jgBK7K8Pyg9}PG-dIpX8G*w;a^1F+}n`5biMGDUr6x_2! z>4E01vOi=tH4kN!N=+&@@)TYO@$+u3YeoT+;A)TJ{qUmUIio_J4O}%4t{jy>%JHi& zBTkst@G>6eHs(W+xHQkwrxQ@<+X){uvfE1^5j@Ht&I?_7wB+z8{L1w55}0 z5t*CP;jX7Exn${$Q@=hh3gJT0CHvMgRKq*1f>dP$jh_oJV0ISeL93n8xf+=8-Yf}R zX=msq%48aunGy9S6ZN27ku`unc{L;L@#p@{htWWh<|{r2(^e&mGNT!I7%xbD;Vy$g z()RfkKE+cGkkikMUXj$g#U0wB-h+&;N0;@qi_r85j>k+ZJ`zP|m(joo7@`ao{$j~#stdbIG990F5z!^J;(xw7&YWwA0$u0&j5KT zRb$-v=Vbg`8rE-6EXrB0a2+PA>VkAf-YvF8XF5IIfoM-~ebiC}htc>ZfSjg4L?n71 zH?o8AR7Yc9Wa#SU;X2Tso<>UAp2M$uG>uxYg^F>_scELUSf7%4ns8sUM~z3?=m)&0 zduh$xF}CV@@qFxxaoDyi&3+rJql`A zHymCEGU$8sFjZ^zT;1b4hQj2o!z!o~hAR8CAM3ie?+$Mkec3gym$I9^cw*%L_VBF= z>ZC1qbx!L{n?i;;eY#1rxiDA#jWn|Z2tgSu$?y1OgF$tyot9*(jR))4Aq`ylg#4HF zS-cnerLbpJZF40?PvcBfwfjYE4&58v)wG)?!8BkpU6nXKEpFE7(xN~KVQ5Pj9~l(T za`FRSGefSW8`^=!LbV>3BE!oLPYrIbf4Z>lyRZn=gesb|2eihil^>FW7+%rAT}A}S43 z?qf-$3bdanYE$PceP{;xp%%6?H$p+3;aW2qo*qw<+itF)s5Nx{cp8~-*Lie;!DE^N zS_lX`f}(Cxf2~cZHdf?2H#1G*P1#3wZ2@dOtLg6kBN?y^H!}Js&nKeyzAm|G^FXOPG(7k#SsR%K7>} zSYh_+(ixE!VOff;gC<)UQyy*hUYzCbcQ-r#B-Je+Ra4BVwP|az{yebX?sJVOY&9Id zK$%3{7+9KMvb5FB@#8I=M(3T*WHb?xx^o4yv~#sq`rqu&A=ID{}EFPMD>qI z81YiwR6`$p9yT{U=;V7&e1fxvdB=BMf-lsv3bd840pPrdQ64<)B>iVw7%qQBQUaG} zP)YQ-IoI|LcrDeE+O_S3KnO)NoNtzYy=bKtAW!K%u2*58vkChQj1An^QfFBe$Z?!k#t zSWdQ;srzECa^X|_&*JDpjn@Ig-r*5&fh>`HJwydQI2G6mBsA-h_9pfrZ+Dm3A_o2d zan?NdJzhgV@HMSbD_x-q$1F;pnvX~GOz@L;b%<*8(-4xLLkgBF>%pJanlg~x_T)!a z(hR`xM!R_lV^r$zAaanoYvxc?1%K9}sIq+V61+kQ17yv#UM{r_WvJu&_y={GIZ-)b zOi91ixU6aD$-6cdKOfv1J6BXWN+EPgXENt&X5`44;8>4pqgzoHChnq+BGG1Opl&1N zxxxOtgJXXiRi>xHn8jL~YMtv`hN_<5K_f~JT$HFgpMQRsj9 zHuEb>VS)R>fpR`Jk#zqLM&;Om6>Pw-o|jMrT)_ALf>8>D?SEnP{T~b%<+o!`Ws<>YF_-Nj%fQ2$R?(5JjNjckAKeVt$R6xyN zzQhQ^;$%uvoFc66;8qxFIL%u_wuLiA+gWrkx!G|g7^)+C7lC&s`5kjQmtG%CioJaCcoGou!oj*c$cyb~;jc;N(ohOoR@8!8nc7m`x4CxG~ z*~~pN?v5l=54-8I$8quA1v>Wg3{*gm8Zgh;Wfh)-S+qbPi_*~veN>C&RNBW>1evw@ zWhQ#-jHoqCL<~=#E*xbA#dgk8_#;e>^xcLzec9*i4f;SvGV@~UhHSkB^%!1xkCu#H z%PDqg?`3T?9JdKo%wODDhtM#OD%Uo%#IX=Crv)OmTojZKDwZ zqrvqVkgfBe9F`%4SKVq6OhrpdTuMbji8OuY3nSCcE<9xtOIaFWqekr%^cl?LR3G+$ zGa|E|HZErl^Ic6%4awu3Zl^Sf&CI~f+w$pL?4V=FR{j7-) zNV@^njSO6W@E#<3O&bptRh9wc3;5JH$e{8AqxGKMc=Pm^#50UymgKxB?MS2u_SSEv zFynmC17HtL$nF@!wG2&IuOv%2lGN|L!#9tkWnc!47C%e0Mf8Q zU}o!9e>$Ov0J-|}n)vO`ro5lyapMFVjfK(Yx%KS<91WKz-%^U0z9QOf5%FjwTU~S` zLb(oWa}D{Zk|*F3@fYm!bChz!MCyd{d`Ci~lU)KZDInkB5cpOiAvn)5{M3aEnv0YR z9#sA&Cw}q6aT6%DsXkp`URgr`CSyk_IvoFjPLGIq{g=9eBqclAD;yL_0e$f1A%Yu; z^&g8+%=~Ff0;@p2=(OROCMktQio?%9PpdN{0+`s4-_^|e&)u6ihUag(Z14T-(lDbg z9CbFrLDnnQOQ(0{M{9jDR-2Qu-6{clIysw{Qxl*So7&PIN&ame%os=XMI`2_Rl^;! z+IiGZwJgA;dsJq56gFMvgPVN!6}Hw@QGbs#L9jjhC7b%`0R`$M-L)L^?7%GYxcpO^w$NF=j)Ii@uSb){aFgjR30u#_Yb zM;QAc1>(H&5CzLzEjh=v=+QRa-GE~ocOZH_$7W3>=k}n3BH}eG52@RyX=c$UPZ{yIZD>3m6DQo#MoUxFTtLdrxQD1>H>{SLm zsnLzkK05ime8Di7y8}umm1u`G`hr9g?7?uJ$PrV^CZW%5e{Iv-BNOfiV0|Lt{Ai%u zsYBF1ZKz5n|Ki^->}4G9+GUFlPi`O|6nd@g2-cl4ds=3x*~3jwahJw!2AyjFjli9U z7D#Q$zhpj)Vw@Cb16EjLGekQozwlUlmfvJOSE;p5rN9AQU4PEjd#v*rKXA?L1=R|k zVdnT^jrlbb(*jE)kj2q`c7P?Sne)Egjhw}{1g6035!LRaIqu}5Kzzz!x-l6Hd{YWS z=Jx*Q-ut-FUMe=wd5XfiM^jXQ6B)xI(B!GSs%o3*1i6BACixw`Q?FW%_$aBalYLzudbw*Xe-#se=+H~IjY8TX>1 z)uxNJcDnFXX94|H-yecUZb|AojC<-cx-DA;Z6Yz;gHBJmp0jFBm_>4NAQLM=7FYFN zklYeZVKQxwyMxxIBJmP*ZHT5na1$X-lRyve@9ccAuq#Ma!ly zn`RR3s@RUPt1GB5&v1=u>ljpVn%9MEEOb`k%A{REw3F^?%ZyPlCI)lItlQ`tw8`f` zL8Wd2pO^Cg%PLF~J##c2jS<%25hNxosUc@AI(X8nO2)KNJo}9*|;YDAFX^ zC-$&y2$*1@xWqQ1G=5s5a#%UkgRztMGsmO%rVU<{z7!^XDzlispzLI>bviT`WzSO<0-WVQC#m`|*mZ^AKRB7!KG4%$1(nfB7oIftyERJg-Fe zK2^wF<&rFk^#&#Cve%@RrD_?pQe5{JOPv{Oy* zlT>@c3C5HSFck-bB8Zw5wshRkDa4I*X4n5B?7G&}Xo1u9W~)06-QL*-U(SA=bShK5 z$Jhb`zi6BHtt?|f2u55+5rQVqiL=ztS!GfYMsYwsNUe#ls%;Gol~l>d)sQ5Q0D}pm zDuoZIV>73cYnAC0A29UEbWgj?TQ8lhp1hA=meK|-~l^DSCdLO_78ow*JUz( z!GjOg26n;>44kjBJEoMAu2tTgby_w@Sm%A!`qYv#ueXu6-@oR+(Ha zxIB|^Z&IHpW2J>T(eoKTZPHZWZQzAUZ+FecZrKWIFMwds_Y)Ig{2naJE}xrCVpMDK zR+VeJbK~#u`~%@g4gC4e=*wP?3Xnq_YZluAM6d;mTXR#$<*mill@}e5f5-OCBgYnN&b)?zREOqi{UI9v^u_;lmrb2gN zm=QHGza|c`!4Aq}WP?0`|CP^5%1Sx3i|y?GYyEWmO=>uhl2$251OI&Y@zDK!7DA=T%R=$3S$%k1R4nSSn;bn|XVkE`1>r0EPSmD9 zj{11v##C&~0y!$+tpbD>FI*cz=wP&)Fx8XYKI7Bu|7Y%B!Z|EquPfrEtH0a6L~@?$ z-02n}bg=F&sF~ypf_6Ik<9UNy)=0<09+PIwL+ptAyxswHkS7^;#w)IXYcH_E3w{A{ zGd?g>!kv$xVA00vkdxFWK$zel;SGH`+tBw{Wo&XAhQ!kZ?;AXi?cmIjH@VS(UIO9s zCEy7_EMkIMm@x2IBR835WzQ4O98T>ZG57G@N4t#7Gu}wX7A{Gc*Uov4R8XBykrWY! z0k3ngj!Qc^GjFR1t58<0oJr;y&Fv23WY7(ITIOzOjG5+BRKL(+>r!ybq$|-m7Kx|n z6ayEWvp+wCEdZ;CWoq~NcqIt@Ch2cEL;zHIFU0SdRqNfY=A{N=Z9uJNKO|Uy5UF+v z-Vo*QHSYB)IJwhY0*ijNgeMA?Xdu%R$`}J&QJ!^fjrnJ8;okkO0}w)V$_!aDi-d*c zJ04n56VyMBwzOibyj!O^2t(WACx9}c$#3?N&};6AmZHN7}nZS}H5a*ht9qjBN2^|=t3YT&B0 zl=9=b_!?V2aFCuMNBI!OaKeRhETBioqA1Uv)Sbv4^hj}8i7D_(@|6Pn6KI>UXtS%6nhah``!$QO zpdC6*l1yF9{NVEwF&neIlEV9HZ*#>(Cp8hK(W-@m!ub^iPj#RNO9hKhD)6O+TYe9! ziR>%^Pn#7K@0>%r00=Ja`xv;Y7s3FK*$!+?~(9Ggt0rIemRKC zc4cG0r*iNL(6v0p>RSK#z*Z<~6;g$gMVzBW;n!EE`bK70Sy!~^w%2W|KJq+h{qd=v ziP4XhX0-)^R11ysLZ$&G->Bwp6(Qy!;FhLViAQYR!c?9mfcdXgNJ=VQ^#59gZwGaiVM|EjaV|EM#rHUnm*Q{XG_1F_15mX_*S(_6pR8Y_|oUoY&-0KV?x~KRaypFs)Nd22{9=*+cC0AbhITGp z7-yQRSaKVsoEb`P(oz9$NW=W5CruOn2O3?xpue&OTZIvoZgC}I0QNq|`-1oe`yL&_ zx!R%D(G8~k4bEPj%Uv&)xY>&ciTG&K!+O;8n z+2UaD>Exd}=A~=gXkw{aZTV>KYPe<#EFgK^(s94&EeT^ElD5FWm@ZYp`+fG6eqtI$ zry4R?V-dCEMzfl~d}(hvNW)@D{_;u87>M=B&5gX*M&+E7CB42I%R80PB=&AUc-3_% zTCD?=u0OV0`PUx0HPhBD|3Gz^QB0tMJ}QI$tR^cPI*I@IoIGb zbn#=Fbu!GC1lHgyw&t;8x4v z*O$@-UixCNFwZ;j+{k=(7I)XMIl+VlTm3e#50tH_;qr5zBTkY0w<{3ei^{r=whuU3^&w156$FFA?g|Wz3*E6#Sg}3|l%p;-YF1gMI!-rDyuhuhA*m z!uZpq5t)6vgEY2Bo)SS4+PC2nMN`**8e{^IZ?60Jw-x&5 zStP4|te+Ze3}d}=_G7~?uZ_g|LMV?k!uC~mu1G)W05ZO)J*V|FjA~guxXrTLNuz!*w!id^acNBYSE-MeMN-UO(JnGkku6iTDD(cuO%}Y|o z>_*$$U?JVb%aL(D3!3AtpU*0>qmg|EP18<2oQ*qFu1{vs22KsPUkvdJd7y4vw^?j% zqhj+#Ge>s$X)0n;pHfd`1XIRbQ4{@vnUZ9-T&Rk{c~3&(bx1%B^?ibt3gAm<2&2K- zM^`FxXja_WNi`Q*CL()xMXCjmnq*9a5&gP;yDU+wskzMNG6}8cmR(!O4X)gY-&{Ox z{7PY`jX@LIiOeDFQn!2x6$O1oD$3%@e*58erPr*1(SD|_Yg>oUs|FYq;n`-u z+Bsb7NiBi0ZxBtcG5{+gwo0rYU1jw*x`mLJ<$3)la91}jC# zk5UeiL?k`D{cxA_A4Cf8Q=zNlZDgZZlk{H$rhNOass8`%Lj z^EJ@mr`?r^<{+HsiZ9~g6fyNSgfxXLD@m--nI-go&~J9wT==ARLy=eGpQw*PoD6p% z&lmP)H6{Cuk5iUFc1&$ho5fOGFYozLeEV&DZE+ylaWH```^>j)5-+c>Ug#5%Ob<;j z2`vQx6z)`?!BEUzV>Vq|B$QJ=S@3}(xwiVhI_GR+u_d$g)PM8Xu*lX3i6;}V1Yw_B z84&UOpNi_m23%nSf;a9){~bjo{;$AAK7`SKS(Esmz(p5Eu-F^a&2B2=@@I46dJcAy znX#uYq)-*~-_rbW&~r^H_yGc=rANTC0Nrvth@^sR8hrKOJk@M{kmp`51g?z^t#3`H9YH|S~HC;P_ z`_Uzqn_+Opn?HHl+tbPnIJ$bx|avS?MhJV#otNr_&)Y=*Q~$nCi8(st-z#@@Fq7|5>LJ3^&SHN(mvmgY6djWpy> z^P^c`z&ww0TY?ukyft+`9C^BQf4A3ibk(?g%+*<9Y7tYS?~#I?=%Lu;%l0Dn0q7h& zl5|-=xx}97FY6RQ5}dm}t%d5#^PRt!1rB)ZXCs_~H9EJKhv$;yM0Zc&S;^=eo zrcH74RK7z$C@gAs^zrc2imk)YWK*FdP2sYR-1z(3+FdMyiCcFv0j@er6Yd>EJH5Mm z+T&+)u4Y6eHsk>~dpAae$NO%MN+IZRycFkIzGcsK!LRO}5+0?^jLFLrM$N@|)d?UjDw2G`HVK zvs5$Tr;n^`?{qCU4e}sggl=8 zo!#DOK!|a?^ci*S{1z^K@MyJA5#)pJLC(ZS$x@&Zc-r!Yz!CHU??2!8{OOLZpgQil zG`iLw?l5Z~>B_;;=Zboq#-oH}1r^v~oSCjnX~eB{e)^VF(PL98TW~me1%3UHW@r`t zI_-7HDcEt4PNXix|B502EY$y4J7Tt9!?=rj*#1~!=7pDrW!G@e{*UN$hfL{hEMmcyH=xAmqI)A#I< z1QQzi=OP@KO^#6*Q4i5Cqw3s^dP#`MHz!XI^*()zf<$c1`_bkZwQD9C+idO~IoFg3 z+DOFW`lkg%*sz%!xmhzu@!w_2W@un;_3rlU&NcF{4DZ5=n1dg)EU~(tDaZxdJQkm@?1ENShIMN)bml} zVMthwppZh@k=gN&f%TtRlTB_06arB^cYo`-cB5A9a0!W<%#TupSx__08T>PgZ9fPb z6$5wMLRcEyOoO0O$LB>?P&kZc>ZXk{1tTZJp;C@X?-|VZg$7IC`z+@&pjAgWh|0Qx zZJ%hyb&KxdHy@#i&Fv=EkGqy0QAUHzJ%L&A+1+yoh3?mcft#d-&zyg;)|6R{w3MO9 z@4gcmsljn)89R6+&rN_Lp@~!UEt2|OeO-?U+qyXHH!Bj2qkW1o?k*Q1qjI(bNQ2BD zjewQWVlnCGl!2P$q*b7EQ0(iR{-#Sre$qMOb-F>|ii>jRg_+}cPdCn0U{}v0dNW>2 z9Lrh7-C0KqLom49g|&)p@^oOYNcV!6Kvf8iESd+R*5Y7#{JJYH3N2dS8?)$bV3|!U z+*2c(I9HQy#_ohhNn11yYi>R zchHMX`3HxDQkX1n%$5qz`H;eD9Y~mbcPix}vPwiJgoE`G=gM3m{@O9DjRj*&&8b>av*r6Y{5{aS_C&hGZPlIKF~=*ksf6*L?EhtVZt6Ka<0ZN!P6nrg4~#?LZ~V=Ai^`H29?tXSNJlKd>>n zQzX9--M?J<(iWl-cinqdG1Ol2)k~#f?+C8&=OI0Tks6&*5rp$qb)P(Y$xbKMxA`-^d4Zlw(h#TCOf(=BqP!+_5uAkGk$$zo zcj^$I_x1_R^zIY_Y_k67@imM%j~41rp2)i^ukyuSV~H3oc))L@-_OLK@{mDuPn9e! z#4SzQWM|gr8bcOh+)-~aMh-^e5SW^;M*H;lT+H`faA0?-T=kKyQbnTG;fi{|glLT6 zHBQSw@IQ>k1$h0lOs6!9)#D~lVx};mA1*p~Ps>E7PIo8M>D5h2A{FR!7ipxLW^i@r z_uG?gP3(LEf4IFy%-pKB_8a_nJ!p4PCZUv*aQYIDlD=T!kXPv0Gu`Te8&ug&F2g5o zug_*u#oJh*TKio)T<&A6sp?_G%sMquz1K{Ru$vfo;`vXiz31=mB7 zlrT-Q0~DRBMeuj~TyS{{0QBj8Gd6faEvS)J*1uhVSp#z1POr9)-^3pTZEu z3l3en6&Hyvk7#IrXb+Hwzo14kr)5bE<%BL7R-YE!3m$t z42eL6$a5C!eQ3ZlAuTH%`o??}Pf(wzgimdO%d^WPZZS*(QFI=C;kwAMEWBde{BDao zw-^bp-k=C&60+%|kvD%2sEj^&Z%%jO>7!wcca8yF6rRTbLD)WZ>3s2IZkhXqs8x+P zwyx?JzN1$~p9z|QWU-f=z%>awOBVP>gtCP=h(v+6rVa0K+gYmXiBLQ_FUnSp>m-d1 z@=00my0@+Vs&^@l!5fm2TS3#f25-?R;gmdB8~g8r1uS4{>WGe2-Hf$?_s!qUy^`+b zsOq<%`?#RgVwz)Z$wqu?BcTx4&<*WsYZY8wMS|D5oYW7gL%R~9=NJt~rx+q$pqo5s zy)$I%4ZK;soke=bM_Jva@B4pWL~M zm-D^P5&qrsa9l*smDh@9&NgoR1fDFBoIO+x9c#Oq+d|EL`nk%L zxvW}{oqc;~aqb{$VeONL@H;4)={Zrjlr@mn=}5#KeXc(NRr^m*2+KxiPj)TK6T)W> z|ME*pfaej%JhWl`An2by$|83X%7q0I>va3 z4s`wEd=HNW(Qs>)n?m0kDLeCuajmrzB{oP)kHd^*Nb+#-dxkGhqz~|)pC+-qaRx{c z!re>VZ=7qQ2C_*)A+a+eb|&mpH~VYWSHor6D}>B{)miC(a27)6igd>Z&nlY7Y$S*+k7-(iG|rBI`a|u`LyB*y z|6JHX08rET42N#wM8D>%jT#4<`w3F($P6YH06}j75OKWndfsbbkI?9MzSXMqjSXvd zXXy=>PgdM~dJFoErDW`>0yvUgEXivsC#rF5( zVuu@PDf3Se2pJct0vu=2%vUA^Rhvu&Mc9x~@>!%iIZN3_Nj>?Bi2KJ&RY9>BhBcnc z=m)yCf;6LaSIfH=oUth5{M3sR-r5QJX(bA*eNe zI`e`P8aDWaK8gRlM?yIhz2{Y3Z>>As1Dpg)l5CvR&1T!V@bA=7bC&ve&m9z~&NJ=_ zBvuzZ*G)^zHVa?NX{E09tr=<%*4}durJ9FD2hG`?t_ zxK&y~9sB+`$8C7Ds~baDRO|2|bt~gjZWuw4SZ z)aNr;Q;PzkgigLaysC+qA@-bhdaks0+V2EYXk})Vl!Whl7)kXEZfA{NOk@w$l;l~_ z=7DlvlY3-XMk&f6M}k)`VZ~NU8yvb5NERKlc=?cfz_wfk)1->y|$b`>bA}w z;90q{3e;~E%*9T$t&n157ro@D(kkksQT=>AAX?@8d(NGyB|g&4enCtq(jrnG6@Yg) z@&+j_!)UWPZ_jNNPjpq>m)HD$1sfZZZ|;$9j?%)#-MrjFo(cy73lyoCH76K29x%Nhz`0Oq3rb!^p-|QAa`X zYiB@fxKi@3W*X*;6av=wRPtaHY}6mAHq2q=(Nw6h+!5!Z=o+3fjGH2>q_$(kLxjYZO~9Kaf7mf`Ggj*HsFl^>!3p+U2le*G z$kKD7g&@uJQXNu;=D}c|Xf6!7RAV#NVWc@&lP|T}-X;m@mHggJYof7r#^&3n=zDDCAJ$%Ac zf9pFO4M{-KOrDaIh$#Di2zv)6LAr!#yKURHZQHhO+qP}n?w+=7yL;NUJ@4Gl?tT#) zu@PVWhpMbRSDs8#elBvrIemf6X4F6U?{W?rjYkyr%?DA3uF6mG&*yk-Mi}%rCp&=Y zx(}x>jui$kc`~U)n&!0EMo;jj9*_D?Yj^nAL-cEu6GzsUlYu}`aWvkVZ=v~qcHT?i z(6np{4g_Dixj(;#E05zII3gX|!m2OYxpE0Ifq||DjxL9neI8PXXKJKBg$oS3mOEV8 z36G5(Z?a8K*z$q5E6Lxo;=axpW8Q_(DAe98FB>8m&u3aVr0%_;PnML-w+OLhYF4r* z%N`zz5CBoaOW|7ck%-4bR~mhElnpaClQh_llZ@#+z-%D94fG7x`Y4hq?uF{6I7T}O zEa?8FZ~y@11gd^5|MG0Jn5bWvs9U8;w|_;h^MBzkbFhs66S>ZRa2IwyNSMHOg*lJH zo`Pb;9!mvgwc{g^`fw~-ahHhDFYHnAEv;DXJT6H9_=Qt5d$lfgvtdG{-8#kmiq!+i zYFZ=tiF>7)l0f7f98ii^kWDN;b@_?A%dziZRDT2D)r(<`8hzu>NSX+U(#6c&*9&)q!kNJ^k8c`qPFu>pllTye1PV`;A zcA}JY{brcnxRpEU5xU$`z`6#4h$6-ykrqBCv0`TGki^NvHLV)sI|hNvGPkMcv0U7y zXv?`?@t795LFs45jS$MxG`^w8nT;F!loNPFnw$-<`%``Ax`5?0SA0!2w zZNb}=Xt9l~SV#QmOZuX!fgY~~@M$7(ek&hX6IXz^ECly0C zsXL8<(jQ87KG_lVJzB3GYR(0_#w0&Hja4P!*@mP#ofq@mB{YVkN~`jnm%3_z;kQcc zx@5bHtgbgCm$UNex@2|b*PW!6$ym^9NV8ykAe@qM@zrWOGe5(OSTONB{J3~#+0aDTk@)?3(R3Kt_4N+%#ST@z4k6Gib1|$V`9kr?WdMD`M z0OEuUFgt*4obt$#-L?@6C2S_b#G2qjrb>c?W#C;F$qZT5yMJH@lXCPkyd*I|XylqTV{qV7Q%2><6XIM6bo$TQ~p27|L1-5?ExkzhQ;- zTihvevdGur$*V4GRgo`M!dwmp7cOBj){JN2_HPLSkyLoYn4*|7i0&S&7A}TBPoDFE zpb9QB8x$Pf1tv3-xYOy$%M}AqOGy)}`a+HGc08KVGGbW>g+abIh%waSW@#k_LFw+kmX17T%%2w2Loyvj+k>$ACTIDm+0?eE$8$>}V z5VY9#8v^rw3&C7>0R&lbKq+>pW777)7{9f&@v`_+4=MG9GWzpIM8!Rn3bqs1rZ-3M z-BP*|Y?-;W)HRq0ez7|A8Zj&b!mmbKcmp$XIOwEgz6Q-BzED}Dc`$X3-bDwnM|<@7 zpvAdKV1ojlr#Hj$J?8eE0V=9~qyM#_fp5YHt=VPA!l#mwXO7rg4J7^xd*m8ZBh8G* zT6XD7(7(29tJt)%0w5ZQsKPH~FeePw*R028s1$* zU0mS`?Q7)Fc)TB%kYnVmZ+t8UvmX4YuD1H53Vmwii^A&fx{l-r21v+D!tr(`hTjHl zFs;qRB+H{^V6cg^3(<{WdeXvu-VCbH?zo%G9}RLa_ln#}!B6DQfHDXC70Us@x?+HU z1M=PhS-g@p2^!mV>An&AooitElZ2ln+m8rQB-3$s7eR645k6sZ)yDZz#Fv#L^o}QV z3G8S4) zK?#KuTBx5DU#tQk7cUI3-aVgOuzhRki|m05)$OF7Ktl92hhrc}i+D&dLK)|$>v*BN zxVR=_L1H;4F0tA18kDOg(@dHUS+Wws4`S6lm)=LB*Mvb({Y(y5CnfpR!Zo8uCMoho z*ziWsRr&@WhFzSfF}m|b4gvvwbnhiKPt*3lyerE*pv0cCAeTMAJJaaOD?|+9S9dq= zVzQIIoI!M8A=*Iixx24$5+xLBF3g~zS$9--5cby%3Y3nPM?RS1jvtGp5bw9||u=?Uo;Uh-+rU-H7(d@jUe9Y4Cj zlRhO)jHo3mU^-A+(sGrmtRd>Q9^GldTwE$H0&HRE1a4d1>m-Vy=1d^gDO>Wq*b~>H z+wV)5`X6mdValyC{~rOvg8yZ{{&N?(;Q!DVoQ~RI(UfR1!@hVmRg?kQI<$ganYj0y zi9b_yH|8UNhi_uDQDaMdQ~YAXIgbEaR7LJXqC>m{WWm|=p#q{h+nB(z`1<|R`(*ME zdHs#~f>JSVPOhUmSq{MXIX0+Tfym0>0KO&2Vd;S^EEb2XKA#@+wP?@6530S|Qwc;S zOiJG;msX`Q#B$7At9#QM%n6thRGXl8@QUqE$F3S)_h^9q#H(NxydzJ|H<;Rm$D^K= zHZNxS6i|zc0fGzVFcy|!0{kkzfFId2p#ODOdI@>-#SUoF{v>`r=FpP2$Goq^zGveW2qwPb_wC`GYr^MEU=p23wE zZ zl!82dU+=Fn@Lqn$tq3wYusZ z{OC9fNL~jH;aL@s2?dcLGCS zO{j&5ZoiWp2u6t)J#JtrxzB+KoqA;SQQZz_%-PMl;xv@y;NpX4%bTPAen(vL=qn|( zYkndhLfMF~D~jeH#gT)k8Ay3k=}5E8VBS4p)2h_X| zl^7TKStM7RC7x84kG4x4kGCJ?k6ZE zk!0Y?Q=+&f5Z!NKxiSv3m%SIrnwZ1&9v4&6iGl|f`#e9?bs&q_E|gzaxi=k}2QpdF z_Jt5ne16!mQmG2&HSQxLr3PvO^?Ko13BHgUDmX6|ewpbBcj!kcsgB{3z~@WSsI?Si z%qqGvK<+Bc4PPc&Rx`<&C8NVL!t-eLVE3@?I3s_d078;59NZU{ZWxJeSGD=B_xG@( zH8#)!x+f9FL_3nzyo7&Xc}!bGEbxfYJ@~!`Vy2aWfdsPTw^_48=OGYY^56^^u-~?B zGhIK0`-mg!paXsGWoB`IJ{55A;Z$Z1Q+pPy8<+2bMR5ENDTYN4j*rDL>M=6_ej=*;7 z4E+ce;)wDv@-OFea0`pWTOP2e-3**AVwU)d&GF)(N_U1h02D2hzvFLPU?nXDf-Im6 zGN;;yG-~1XFe))1yqyDqU#?AufYck&>QSH}=^)^NUQ2z#2uz&U+qi$#34$wKs@8N@ zyJsV+25N87t!|ssZldyBS}fI-IZDsQp`wGKR9_mT>j~TbAY1@)65)>#d2K3XJ5co~ ze-}^~F)8v!f8!)oV#qY%n$mAvFSJ#x&t=dRfmV?}_m9)00ynmQ1=5l|yZ?Ac*~0=O zYCA!&7?^z#BEy*qi-PAWXF_d@tLcfEvuXDOCToG^BdBIXY_<0>n4g619EtU;iSH~C zbE3mzdm1}fS{08Dg6f&{a|mFA7E>`=AU#Uu*M~u{Bn@_-HOx}c2Oiu$G=vwxn@2U4 zf3k|0bgzi189IfUL(o^rIK2umtvMZ7L->l;Y6z#)%63W$V8=QDWPGOPEpd3yZ$>J= zLZ_C|-xho2MIkoQ$}aD=MzTsG+#JkYs=~g{UEd-^ts?m+=9&0pJRggz!M%x{11d1; z$O})-Y^n=jR;}b&{|*@bV4>DvqfQM3?)(jXbp9`8>jM+`m*e_J*&t*nJ$yPWGE2uI zMJHTrDH8XoU{b}MtLk08j;bzWK4d8m6G}tCT#8Hqhqzh2ZyZLX4~gyJC0FRW$lFU< zYPN6}+F8q?dI+&PDcENO+^-h%0<}F8ok4T;URrVs05eyIlwnFQj(lC0r?q0NS8@PvT**7TgN{W>Di1;s?JK;5?uMu5t6wDxj&8HhZDAmvbZG`c1_oJbB5Q)gd8TAb_kLo@9f#NZG zm6s*a;3Jyo8SJi692E+ik_tYfow& zY27^pRJbbH}GPtm|cfWpf~HwI^iBdW!HmTKibL?o4th)Tq~DefVvR%E9wUd_S7gMA#&R6 zM-#XzguMODE%|}4oAaW4NbK2sZ&`Vyw=_?hg<@BYlafl_*1}_JbkUNL1{o`1Do^12 zxNVt9pbRT5XrQxAaB@eQ+$Aq7{*^;Oc`kse`YmBMjG8gS+hq zGde!?JiNJ{p_CwhH=Z3Jaju-D9>@a_&J}Z3MEWQ=;-GE>7kdDn0QE008)*vf$NJT} zvhrFfD;S0}A#}qNagO-oDg#|joD3ULWa*#%Zj?elJ0}6!B3;M=Z z-Rx_s@n%OYeRvwp-yHrL#+Ls8K2l{s_1lEosuTL?&Xsgaxg z);j%kWe|1-k%w4ql+R4vvetEOkyKE$glrceHG>*cp0CT$*{N_|cOt`&yK zRbHu}Tms_cVFP$A}9jdTVWPuI$v(|ObNdgE8OcL9INumyH-$~ zNoABY$ofr%$A(f!?x%r74dZvA`Do!@d*fz#+)%#1?97_rktXxH`nIdcN@$pVGy@TkQp<;Xf!h8h9qSJfuh3oZS1Z zNwT&<7My2E#k&k^u=Nr5)Y6Owsfo?R0;DPWYe*ON=7#UCRCzcUc z?(udL(ADm*pP-aOnOkxU(&Vb>^o+|rl9A%^_tZZJf$xk7=yk@q3g6iY`i6Mv6M9T;sex~02WF}5}(fX682 z`(5qPO?~TgT@qZnqlr$u*4Em(t49t4VB`TKevdNX%KN62-nJD;L~;bw#u%3Pwl2-T zfFm#$pN|XnfX*SW$Xj~Z@ala39F2y@Ua}uLkMDd#%wWf^U8~0-PDt6_)ya*0cC5Nb z>u^a(8pz%iRDn~tojEbsYV6dkbgo`#IbyOlDtrn}8WKQdUil_4SaAMA=g|c$>{!A4 zljZMA4R{yn^LbnwmY8Eeb}fC(;QUykuyr z%p4LUGDLMdx{c|Umt0~r9%It@994C+Rp={HkN*vVW81uw?xP6#kCThy(9-Vvmy`Q1 zjMnhK^zJ_xt>GWN1Hy5Md3S9{=U5Bj=wJCQNJ62(S`zXnf_ElZ>B=9WW?}Sh!_oSx zqc?p+{-2PZ)pYGoMByD7nGYW=o#(1%xsK3-E6rh#oaA*2W%b6lR zA0J;sGdtd2pPiM!G)Setn%Dk@4&tylnsLPr?Jl9TmdKOFL~b08)11f~MO|Md&zXEl zfbs`?khGwjCW#^Zuqc=<8HKR3?#!gSM=JFaQCiUy%M@N#fg`;5`oxR(z+|>75Ev*B zT?IP%0ArQE)x5AJnjb_w7Wo9#yp7uwj*`7cD=jtz&gnUl@K*R?^PI#lLkISDU$B6R zvjXG6AUiR|-ki~uGm+RJN}PXZlJ=p8xMengQ_=$7)0f{LpA2sfW5#JOXjC{x!PDPy z)*N-xHD^WuaVwS3wTdJkCx>uOE)o0E#LmoWvuij`=$$5n;}$oK z1#=8_N->cZqzqyl$%GqqJVl~(6s#a-UrszWdR)$fSf{NKDW4U15TRoaq-^vC#$929 zsNS<)6#a$C1rmSzxN?sZhZk=(qB2gYUe@0==loU>HJSPV=-5Q1XfVql(jCF_T&nBR z!MPIV*-O&l-O|ZBkdT$d!hm{J>F0p+P_tg>Y-93G0Ju@|s-q2&;8!zjb_rV=)(-_z zxmUo0>p;*#1Pv1EDmu+d?Ubp(Y#aPi2Dn zP)-XUiw+Rv-9{oVVUG#^gEfR`D#-puBrLhe$6##kWAZr*Zi)Y4Sogb|{XGM)eCQs9rYRsl^VZnX&UHV0ZSY&<`Bg1b$2FAyP?p__kqmU9d`jugxGwu(P-n%+N?i zeAzn!vO)%d8y1Z_^9f2aiOn#h@t77gahL@LInQ5z_fLuMLU@lkuv%ypESqQHztN=q zT2jaqOd<{Zd_vnt$X?O*MVFC%C4k9Z=pDtKF)lu}6pGuHcAuIE~To z&`$nqcRMaMm24+lerXi2+nUL?X*Q4VcQUHR63w=&IvVe~SWNsh-4Up=i>xCQ-5JG2YUqljNIfhW=WCDn9_lE z7*PuKLx$-bCl!pOzOJNELHK5PIcE@F;B4JDu|SVTosdT&t`R=3tZ z)^9~230UoUTCSbAEJBGI=TG@09fTFlkv9HrS-};O5_tBtmj@_D@X{m&h~z+Q&fJ_I z_AL}V+>k9=IyV%^EN4MPK=1NsH`pGg?klG5$HZ_}dO+1(Fcj~o;v)%~2gIMy`A5Wp zT8X4#;{$tK(Q07z#KDq#b`Wu>rY|3$^wPf1KJtsZO#*Mffr4grHW;~U z@Ur^S*z6tAOIwJ%U1ItMz)xuB#QZ`0>|ToEmvp{Su;S3ER8Xy(T+huxAC%5_f#2Rk z%qv{i*;nV*9P|3RkE*HYYP!qsG!_)LnZK)+CU2a0OjIDT*+QS>vc}p;_T6loOt2Qm zc=6~4Hn#dmh#zc+3?Cg|8#mh{e#hFAF3rAZngmira#N(oTH^YEXKAoRZV^1vW-cj6D)wy;0vP&n^xJDHxcBT}Y zt100IGdk$tk`4XtWI^yEa6!Z!(4M!yDWq!J! zVJ0adUn59y*mP!PSPYFBQQ?Y*@~0FDz19Ssf~)eCMosk(3w|?w6HpcI@g~`QyuH1GYI$6C2thi|E4_;He7<;c4ky zjaWb}|4jd%r0ds4U= zMU!oj%66R)nN_pb&66`;rRGn{fr2Cmw*6wlOm<0TvZd05UvRr};1m+~7fQ>0`3koa zfj&^^s?0I*s23t{6!m|JF(;qmMt83)I(I^2W!~7ziB)ZY{f(}p#w~{^wfseJ1(mg? z&f9Alukv&RZwxUr; z&JR*bDr)DiYSQ$gtrX4bz$H+#i^bXgrwFb5KIc}o(k2gy*DonMC%phL-j4FJVLv-$ zdq?P;t~i)tnp1@G`M5NemU0TD|UO82k6)=<@H<%Pq@StVq8+Y*6pW^`OL(hcCQ3n%#z;`7X(t8)SsvF$=cI7#wHAcHP+$ z8=di&!NN+GJ!;V;a+!xylLO6MPj@tRL>M_$ObmUiIT`BX}i~hT@=i~ogRDW z<9pn+i1?WeAHnb1pA{8p!6hrwrHwDUmYQURmL_5y2Vmp06F)dJFe>7GsqJ!1;Df+l z4T8hbF$#kUgOf`)$;hs_e9H<$xO$vqsni}GWtNFfJX(Jjh0aNi+*Dvi|d{l6G_-DeY;}7^PzVV$}vPC@5vDuc-zIC z37BVKHS+1x+ZQ-_nr!u8Dvop>QFA!c9^s|n;Xt*%Ad+g5>gWREfrP_KuO2Cp4}87+ zCNe(Et7EKFbF)Do?Tbs0xqHO?bV{4_v8;_l*P_i@El4u2pO~`0dNx!!~ z0kbc0b2AoA+9gd{1&w--0jwl@lDg6TyQouVUeD4^PE zcREQ*bmVg!CtCzOTk-myPV1#LNkN8>E&|476H}{+T|?>X>5<9|(Fh)jxe7{vdCma$ zAhYqnaw{;eO5IlD+4SAY+EaQn8-wBSA!5l78{KVwZHh@?MxjH090?yd2?`0nQEub1 zFm?`mUcCyX<`j!Y{9ZX)aq$EPa(*z1DWlzbm&>Gq`cy!D^Od#+izSbr>@ICz-I*UK z9oA=a9wuNQ5)}l@ikOb?!p~mWTt|fRT}V`Yex#Tbb|d6IER)0m-c`J*7)mq3~T#e{z>v4e~^#g;XO_LoGj07E9ey?f zNV{CmRU^x@hpRb2lTs>xX1$fqu#WPn!$CaN9VR0)keR) z(~yyeHB-izOw!jkAO+z4*#r0Q>ZCWSVnvB!UX1o5$+^!mTv;srwMt#R0#5`9(&8%f zgNo^#l008hXGM_j&?lq))6ssS3La?Q7H14T+*0BM+WL}X{t=~gDNP3;9fN9 zPZ1V1VJm<07vR?D%2lVWl%f>_@g^=t*e>&%AH%wB}&z({r>a(GwT;da@St%Ht7kT!&q0uM2__KTr=r{T+V9TaerqhZel*&(y$zdA0r+&{iz@WlgJNV;)3FB@768R5q-b4p70wO0C z1X4Z*JZYGvZTb)!B_45 z>aJ_5^}T~^xumdS)`1R=L^A{p{;gooL8WP3iEvRfWX=dsNPir8KE&NjHkDiJiwIS- zX>SE(a)Dr4nl~A_>wfCqt}GNKc z<{WZ74pGxK5J@Z6?Ub(8$0QfQ$~y(hgi!eM>A^NROxYw&)92aa3&Qb1F1#lS39_^e zXAGe=PB5Dp0;A2{RT}5;bd*}(ETj>$popqa(mpn~*MLXIqGHIi4Wfy?zfq`VW+59= zPljEC?PV>mrs8X=1<>q|S*^a1-SDVgzx%NQE!*GQ=l}pmGB93PB6B#)_45go7H7-3 z+%))FcJOQ+W79a%cFI)gjC2(FWXs0p{w$MmqsZcteHODL`l@FQmL9UlKiYz<-tSLx zIvU1=!+imW#NC?`W`S`Xl0n&k37VR5&b5Zwxqz(rB-5gw717+qqEeI}l3`3=4m&$8 zGS1p{p|1N%65tSIJ(Zr1j76Xt|8wv*?xL0&fT6`}kgGH0dT;TO*%S|*lMW<@U-ywJ zgL@`V0tseQeTmu;P4erNjWLF(DEmf*QmZwjJ}ZhCCOKkv`bjxf?GhA4-Wl9rj@M;a z7dQ-m)jcPAI1+ma`8(j}{$A}Vui9j}f-RKe7hJ+aVz)_;P8R*@UPo~Y#?4xCr5sTH zFesAj$7^c9-br-@0eRcZx1>L(<6Fd2gx!G4a#xE(CkPJ+*QN)f#7&+zlp`Ek)IpWA zk@@aH>0xv)^IMdLr~|Zs=gbJGkerlXv!0)(VvN9-F}pEog4(y zM%*$0mpJ3)?&y&d^GRsVPP5r_&G9`}WDKb03ES2Og8Qp%gFAIjo(69MSiZ}MlKNu- z4y7%+kUL7s*=1BBx_1;R*9{%0kZ?)#0;}te9`vs zZ)=i~Ws;`vb4O*Q zCQpqtgFm~4B5J_oqZ&dLb>T3XQ~l0Pdo9%oS@7!wFKa_Yzr^y&T!XUij41ZZuGksu zb<2I=9I?XyqAuW+xz=M7c00N4gRuG>sScc$k31d*!VyQSgIYL2`9`-Z<)dRyGI5Ml z7-U4G&EKEJTh2E4f2M2H7#~Tt1VS8=N}$w1#2sd^$75sq7LdkV8p*>zE`R z@saCbSii_r@;^jCWApT*Bg-C4E_rR04t?SV8!#og~lqDgVe9*^7uAPuJmDJ?{!@|K&`!dy13v2WxH$a)bA|T z94&Qw4>kSAHmsjb%q79Sx6o!|IN%{_cnu1>jOGJ!UNd_$AJ@6}=zGeIPTVcgYa@Sk zDTjWBAD6pB{`VH*YOl+7HD~L(m6p{vSON)J!Z3J&uVmm-AwDSM3rbfXMG(JK$Hz z*m(sD7lBT&o&|ItFC`C39S{~O_-T-ERY|=*54%(9kggk%&wBh_r?7vWY+o5OmI_a8 z3H9}~(rnx&T?9rQhj(+-U2jU1Z6}lQ`qV+v&)?oD>qcgyB_dT?WUwzH^D7pm2xAlg zZ~Oz+sdVdL#-jbgvRgD9wcHb_=6WtH>0Cn_i8!ptCEa%@>SN4T-7a`sP=t|;8o16Q-&xQhdW zot+{yYQ_D)2o8EJ2)lt#el}x{r-8N+KEGFu_%;OQyg2JXSUdqP9-m;(suQ-4b~J&s zM#&Z<`DEh}1|6%`6E#KDtAB5B{5<(nb2@kEnCgo>;%su*z1`G~y7IE@gJ{J1hD@sn zvK#9&oa%R@e2+%G?m1LDBIlhKtSkQT-N4I1%fy{`@Z*(Z%6qD`fT_zov7(xGFVW2o zpkI8vO?@L~No|W-Py2Pf9@Vd^O>F~B3cQ6aeIz1xw(Y=H_%`ib%EyXQ3;Boh5(``{ z@mg;?DA}qYZUIa%c3;Y}0?=!p>F+gg9+w%vwaCeyz?`gOKenl&2CO8I!W%2JTN%ci zYvX|-#r!*Anp)aC)!>Ov>^oFdY6$;=w}Taxy$Eokg)HeHJX1~Z0UD33$}2?|k8@95 z;2xxbuA{m(S(zY0vf3&07MiR#=b3t$hUEJx-2zP4SCXfOw2{30t?Fq$aU7X2<>$GtfNSmk=xMwS4|%=`o+k9&)>D)qc~z)6sL-j8$UfY^jj z?FYbGjTi3HoAYM47vEKyJUEQ2dceks<@}r%g`TK=6 z89&SSphi{OJ;^YVGR?~RG6^jQ@gLy@8?sI$J-9&|!iRrWdAv!QRXT+$K~#KIpvrNf znUmFLCNM6S{zS3ffINdp2-amz5sRQ-G!)dYkUmTN9zj(mWxNM{*`1aJ#l8{ z)VqzaDR6S^1YfwNLbD?Pdji&l2c&?z;~pBifVvo^KC^@V zerCXRC6?altj_6=gh!ri(LKQ3O13U$3qgg-(}?N!zx8^n+5HS;%j-omn8?Hup$5N$ zEuFnEv8$(EkwZ1c@NSqvkH&Ec++hmmCB^z;HK!|GusC$(PEkTPM$w_6aU5s8B7#oh>O0yX*ihc|IpeNfS>3Ri`!?6amMa zpRlgkyWv`)8_aQnrP6+mY|op0$jmAX2sx?%^Z`}wm~oACNM=|#gC%{^PXw1()lQ8- z1P05VkW3@hQo%)3lk9%6(nA4vM22#Lw7LN2aEGZ!Lct1HQ4<(>Ym@gJ85!!od`FND zaLJ$)ck4(BSxzo}9fEfK8r)X%lM_eT9*_h31Q4FIuRYe*P?Sa?p*Yl$XsSQkJLFHf zH|fRMvo5-i0egcr*);f`k__2r{IR04I6Z&yyspfqU<_r72xmusF{2r)@(rAEXqyHK zNuA@P7-dWN?I4^&(Nm8Im=vfdG-`yZj<{WHWYSU zw|utf6W;OFN}O>k8FjDC_NnkT>r(qj~&DP7KezjkXkh6v3%fs8O_n*rzwW-{7}JfbNJ{(U*nvcbD)Clmi-cHlQz?sPxM<$_Euf#>)Nyn4(0)r z+j(WCPbu;))T&1LWrkplx&z2lbX#p6iQ}=+23g=9#EYaDA*FZQ{EQJA?Y9;h{?_Z* zt@1?`yud-HMmP3jLK={d&E97Pn`z}a`wzs6`%`AV!nRnHw(U=|l4? z0=3f1Vq;nLU=uvo>JL{l-T*zWP#4TgTpbXGEtf;I@vC1K3SSYo3}(`1Z}RE?THp|z zQ&p3>(ATZ^iH>V}Hk2CZ!$x_wgNb1oUn;q0tR=fZ*x#9| zg~(`w^n?)XbH=yJKM;iIB{(KTva$aRwQhr%Y@RNJTD_Q&5B;V#Yc5Tos+S3WYA2< z+LmUX5MaMQkFK>8HC;y;$TUiV7c9I2o9E>jl1#fl!NGH1aU~tOh;V8ptY*HX7An)nl1e-+LG(?h-p)u^E~)Hb8CZGySxcvUnb@}d`v zLD@`!N-(o5gzbHJ9ed^0*AU^5No59al%C`Gr&ib9-1erSCoLJ%JLz521R?wx^xKKQ z(q7>)y&qAemn6SJK~`3;Y>yh&FaT7t^keqmmOmzs@eDNB4F z;g2p&O$%jZQIj2=IcH0eUMkp4DRx>26pHBI9d6c!Pk)>Au`h*-FN!w(WYU%pUc7xe1M4-t>0T-Wwe$QBB33kwk9Xz<6-{_Q$8C-;%#CWhqQ zezj<+K#jjIWynDEQ4thTwGrRqyoUI48D7__s7F0=sddS{8OB2*?(W!ZCp-vn8AlVi z2OFR!KK%cVD^Z;1=l-25Nx(ww#X_xL&fNpWK|T0i$SnfQ{r_^52mc^9xo;tlrJ1)< zkIr{8_M+588E}s`v^D!4@TJO`Yv%#l6X^ThpGBqAAxU{`CiZ!1L37OkvVa&3yTK9^kut1Uz=l3nA92U(D9T0`jvwQM@V3bPeJ#9r-U4|W!FaZ2OD6>C;CwXvxM z-~yi>Pw@$n3jxNIb5!p6d9oI4H?WcP=Wi#dmd}DE?7ZaO1Ab2p)w7Myi02z5jK$l* zc;a-R_LCbcrT(lM?08JwqXrMAi#8lBrsYd;`{YJW1 zU}N_~zPn5|D`+iWH^EMp@&|vOPP}&Yl7A=`gEwcN26J9o5v;*tkKSld{wD(`waSL# zaDPj(=t&p>RJx(O=#9L!TdNXTd6Aor)R?Jz}ow^>NfEC;ZCPa4SF{G@ z`k@X;BLl8eRpzHeE2|b$ufqU0s5_^VuiyI$6aYWV&S5EZeBYY&2Yi(=QTOt_+l-Rj z)12F{l4w&88-q3b#641UzvpXiJT_hP6A%)e{|Z{Lmg~iqM3P9tlrv43os$7TN>)A1{EG&*!?os@?o!~ zbQC!^PQ*H{7HNnu`37=Gbo8jV+E(<;Qm~DY`}}}eW1i-NppcQAToKp)mDcq(*uqn)lv z89dISRrPcr$$quyR_EWA!RcfTH-0ro1O~4%V?l&Xo zk(e{16ayVRayOh%^Dc-U<-(my+%f+g_Pvngac#+a`>q`mc!PHGl-#~Lj!743Ek6Aa zRTi9LI!GJLW~-#W>6M`vnkp$p0E6 zEBar;^`9}aqJM{p;H=UvdAAT;F`AD2Qovm4)W!!4-0cSJ0HS+0@~g zF)fg2aEY-%W7TBT$~a`5;N+k%D%ytJDu-;w(6BM3`>JJ5QVP;^P=eAk%+53V(2~iL z^&Pjgc4)rP&lv-AMSp9ckrvq1+^`cs)d%i(%Q?ijP$S>F6v%gMnF&@7OAfnMC0sW! zY_RexVwA5T*xW6qLzU@o-K#=+tp8reNP+k_z_NPA~ocL>|a1lN2wg?}PGWqqJNBQ+{o z{$xg-CW&BY5O%mTtg7|MbAWSqZHYlS=;$7Vgyk4*^6$K3cd|gQRUCrPZXPSz`ZPY_ zbS%=pi}g4$F=51hCJTcr?YOR zy}`sihQ@TAze;K;?{-}%xz-)G&#JB_!B0A%GTX>{oE-8+i_LpTdl|!i{I+L61~$i~nl8##Z+K zDeT>c?xDKFM3;Dj34O&EgWsJl|MBjj)q{C7Ee6{E^~u2tBRU`upIbtu2RFDoyD#Iu zr@y63aQV3AhZzK_TTW4wBN2~6&we={Km`7k1z6SGq@wA=oUVg> z!}IG55FXtU?N;tVeBw^4%IX)1AL^0&_HJPGqRYpENbq#>gk#qDI~1XBZ2a%PRQPsW7Qv@tB-||1<(7)r^zl4 zjPeIYB(^*a{|652gz7?a-4iXZY#~9FLZ4V0bfjTz@_=Q)Ns8{)rCq?*g%k3ufLY_I z{V=xXz@LTbL-J6r+GmDLw0=mza_O#Ap`A#8DAKt9$JseHX~LyJx@_CFZQHhO+kDEl zZQJOwZQC}ws=K$}*_jU$F|i-^N1Vtwd1qc34wyN#^(A|I6cS=t4Xu*sO46tnY-r!* zT;K|Calp8C5X~G*T?RTveGMxW_f?EjjE-;)j%!HeRwQ6%q>6M6`BO6%M>_=~oQ9|x zdWckg7MYC@ZzJnSp4)vhi41?&8a8=E8v!c-)$Ht7NE5Z_}GD9}6=y32j@+kU48blU_B;!w>b zUZc3FdoR1baefIbB33SXiF zsWQO;NNYg$b4T+9gI&B+&gEusX@Qm6o@5jt7X_CRn-KJzNZU6$!;pv|x(fifd>kL+0k0suYLC5mEsnSM$8m zRXbX(6dL)&TfH*{9yB}Rq6*~{flSV6U4-}fj|0RqOWAVk>-;*~FsmtOk;l3K_uxjq|^5!Kv39AgVr^adZ*qy zq7%cuPrJzqCV4U5Penz}KqH3Xwi%~ByG%BuZh+btJ}lm6jK0K2!8@&7ei~{_3TX1Q zu(~EvKcy z7s^-16-f;FAJY&>rHJqO-(zIO*ysz`=w*{(+keN%Qva72DMQr%>lc;!j~H>8Tx5Bp zu<`9BQ$%cs{zh_h&8VWb96FvMqfOln!os1h_%lnvW?zd)zarY@$`U{bt4&s~css2* zpDH3lUJzWi&t$?+OJNBeGd7nE#9X$R04z_r8DKMvXk2XA?Q{G#Vk3T#$RGNrfq4_7 z10opOD(FTSR%Jy3cJ7>kd(VO$x{~`(?g%5Rc}y+%N55&OJARQ_CioHioQ7=>`8}6A zYBQ;wrh1kj268uI-XoKu7SIiK3zMNg)lm6B>@1a6b4Qp3*OU5R17Yc%gj3y6NVs@APJ5IO;df( zF!ffgDjl20uF!EEKZ}ECJxD{Cp^dyfMn(b^#Fn;0Oh?al#P*fB7WTELzT$P^=x>uA zl|k1>5Dn|!LJ?I~lmYH7={hQj7|YKjI|rN|5Kiu2E6k4oDp~?lns(AVmop^du~x0a z9xX1;pq190lgu|Qh)#T2M?HBOhct7@N@EOmC5Y~(aUO(+jq4Bz5@}OMgm`b(N>4aR zrf5&Anvag5LmnEcXeWiT!-^Go3d*!u^kmS14vf#n{VV2>t^k0kG!`{?#BuN?gvq0H zTr^ndqQsfmDeozEyiw^C=fIf3%}2l!Qc`$GAC3@DiOYdfTlV2u6 zP8iFY)u0yqEa(-r{RvQPk$Kt40QV^MTjE8XFB^!&GGIOFao80Sy9F z&OKT_7Qt=^ZQg)xi?zbqC1#H=>7zs%P~pKEG8rLJ{r%Nd6ij9`Yk8Dru7HZwm@!N3 zJW&LLm?R4D+}|aSA8Zcefpmoo^sA_v2I+ zDl8~AYAM+1ek)J{cdT$-KR~jc(>on%N&ma_gTo}%C2NDe;$?lNv1fAt{S3cNxLONu zWy)W!ktIMsN^5HUeOdS7b>R>OsJkKVpjN|O@WP|QetmGnRyK)-XPgGw7-x10IO%K7 z#X+hF9yZG|p~uwGh$OpzqU<^HXqDzHiHcx8vo(-nrnaj{dQ3^*u1ZfCya9{0IOcl#~6AWM-eUHk4waMCD#(B}7 z>Dq5joZ6i{azxkNNbBM9^bv-je@rbL$S6TxL^b>>>-~uNm-$gVM-M?i0YaL%ncxSi zgpo*eH?IhgC@Q%aw_8$H{8x4yGdXqx*b8EH`YUzPuTojT6WcE5`Xwv~I$;){T|uU8 zJ6|$x-RbM==*{95Y!;k>**=>A+#$JV3hTIu-TwBo-wQZx-Uo)L`+*@tuuh-UhKy%D zK-AOQygu-~NVD^fr0mb1X-7k4=Ex}ij7PB6Ei3wga%;HNj{Oa-Cka}uPJ}Bin4>ky zyvGc}ct*B4b+{YeMZOm%HxbNWGC?dks$4wi)`ipL7*D=Ukdae z0@c7j3KVw<+?L@vECB-n%&TQ1%d-QeQH)&njFZ|QxL&BWrs7wE$FS0xDVjR{>gQK| z>mm|5k4F=9&8vGfaMRNCiVsgYj~|!FL@xhQ)Ny%EXCu=$Ls&L3PxH!g+-%+s89CyO zjLpFNE)@ylqwgTlGBq)4A#cL`^Xa;KH>nYp+80ne>lt%!GZTF7l4pz43@J>Pp|H|j$q7fk=q?tJ_|TZd@{Ju{Tea+)q4A?338|Ym&P45o~-5`=9I`I z(*-N2DX`N^Xru+0++g8xWis{ISd(Sd&hmu$xP8AC%hxO_2sqNvvNmekKKoF~hxsXz zxy9vy#MVEXAA@kUpLP>yi#oKX@xrid`7Y-Q#jE^o(#fm@#$@C+!=OWI^PuvFm#FI= z!%=R5u(tbhlu?3&^e6RAm#sHRkkdt*evWc(60CSsBTU-fnbbK8WgGiqq&LhZa*$9A zA3c&=Vn`ujfY{y)j0h9^aHqNOs>6$zt={I4W$`+n#2s3}+hqL26Jc!di!-_|R&}Bu zj!3_szy^DnLUoqS(gqpMEOoZeJV1X5gdLi_n47_HT+?&$h5X#g-Pzv{eOq?#8xX$b z0<*u-gwXn~?6y|Oq>$4Il8t!-736)CN9DNEZ`_(#Wy0&9U+q@J7jj-JUi(R|DMbMw z&e}EAdXwO?KYy_62_}V;PCL(o3wCy9kHOR~uyND|G3f!VS5h+L?h~!`C z?yu`bx{R{8u(w(G?SArc7)ssH!|!ULOX(Rg-Kv_r;zIfR8`)?ypUsA?G82{ln3A(uEAK@Xp;AUvobcQO5?6JsjaqtBmuKIV_j(W0J*lNPE ze~;3e0?E#I4jiMz^?hUd=MuINPJYMq1C5yUn!4s0rngxMVWj}oJ6#-S(r(VMg!|< z_2Eura056#x(3aYh5etgAS$=o_rDx#fgVnUd|3l!y(cZRzIwOfulVSdSiDS-w8(>P z_%rLlQqK&}`0^o^pTu7u?zvw+f$v0J=IzV8Lt3_phg@h3{>XaMY*~O#l4qD+-ZtK= z=$Fxm-Ek@JE|t3TR`OLOkEGo`w{Vjy@89-Y1kGU**1S4Qb@!}2(3{{@Kx{~d`zLz% z?O?p`Wu58X%%tpg6QT-h>HOWjL$}xaU?=4V2|0acNPL9J(I5|e-A{UaQgdLFI9taC ze$qf+oHVEtR5Y04DCVlbEQ(>6#)KLin$UtZMa!dDi&e$=SrwurojlxY&MDbCTSdWJ zKf4?-oIsjUX#qG7K3v}qW6w?h``)4{&T!uYKL1Ui`ma1i@Bb@L(fd!HLNn>XTf*bk z85xrp3(wLvyzfav1gb>qaduZlZvt?|5kftwH&j^To?ZeT+7S4mCt6puIK<4-#MeD9 zl}yq#eY7n@av!JPHj;|c-i@;j!~Ay=plQAQqC#1d!bka$8-KBise+~ERe?1Jryaxs z_to~#dW{F3Nt~cY#2{}|&Ff)J7HbBV0%*|Nz?;o!;h*Pci=ATJ=@O6~$ z7xkMf=EkHjCMCska!IkJ_V67=d6cP41{Y}Sthj_j;ra?;TAOM+y&`6M5PkPn^7@*^ ze-E8^i-~Qlc>#G&N0hk;Rd*h`gJ_NM-c+ank?xi!NXmdD7ev*&>s??+Ow=f7C$gMD zQ{InzkRJk{OevmGNJ`r{8d;XRn<_tUym?z)GnhV2Eec|Mest=olG{PjN#bdhd-~af ziULNFBz(Z8bfB~`{%8~W{mXH~LIH0jkTj;L9;|-yU zO25UB>djyaNoN_w6&)Aml%jl>0iwsk8k>F3fl?s`dQ#L0cy}=(Fno1Vwa4D9P;sSB z!F7KP6;Yqyw=yLlD=;Mg&Anz>#d|WS$#*AYP2D*}LwL_b~ z8#qg7;REaDfH^7iKi=#+ao>cY@aN?CDk~eF>rXXhgCblSlgdgft;h~~$1r~G;Egj~ z9@v0l=ZGt!{rV+`WkP-bj3&Q6)O;4pIKO9-|4h!VG_B37Y+7(pBvV7Kw&Lse)Fv^Z zZdn~kTF9uha0wqs=>)>l^V_jYpbwVg!|vU=7LU-+5Xv~!Vtm@WBBVww%%mdnsQLE8 zwu7l?Z-v>@v0gLK6PFY!$j~RD?}pX3ACBp%m)aFl!JElJ6wYsZ2nQ)z;u`sN|DusyN7?8mR@t6)Y|K;vKta8S_|-|yS)V^)ws zKm)UcutW4I_Cp5bA|=h$_&V;+?XQu+JGe6^ZfA^^IMVPz)3wOdl22?ZVe~xpL3x47 z->x!9+0+LK!!q`eC@vce4Hj1B_DxhS&IrOPd}Ne&Y0vG4*OjWLhTinHcuh5?AbM0F z9ivgU4q-M-MKvur^ttnY_vDo8q}J;|fA#2|;(Xx#@Iqvv1VVEihZp*l8lIBCdfbs> zeyxXqc5qqtNpRW;Gbx5Jc|TL1sQr-|DZ*PhREk#h!USomWM^lC!?|Eg8!O;hvDRIT z(^a=+etR4 z{JUQVv<0fZ0eWRZ)l3XL<3;Mgb%2?VL;<@*&g6VOI{(v{(y)@r=K-z^r!d$y z(a!AMD8m8EU>N9aAh9W4Vx#UvzwHkkL}VC_w^)aOfCa%IVe7^Q!J>WKGm&8^+d+Oa zbB#57xeXyI$T=KX+uoFjvW0zveWVo?{k0uL5p|;!dsKr&Ma64Y#1J0Ijwq z){nBzts1o_HD~!HD}28tAV;JqPj86J;SjKLZxBAmJw!9(Y7Y}mEqSrteZFUWW658_>oTvOeDo ze#ayQN=yWg_!(I%2j>$ZT%bH>ku_j3sidM5+|6UyKdsj{Xb3gE4+lUbY_ zK^iO}eC!E3j70B72#GI`cjVCkz{@_TZ1{;?^`fHyj_bY?c`7h`BwWAr4h|Fu2+$tF z`0qSLDlYmEE_%es6;|LEPoJ5N;X`91RIgs?`tgT~ zC1ZcC3$T6{g;R=5n5Aw52Toio1NEjp87=Iv32YYhLuQ$cP6nm8@e>q}u2q^SAVfqtp!53-SO`fkRa@*68H98su5nG`VAcvKDAKo&fh! zpP+)8XPtZN>a$ph;Mtf{4JiN04n_OL=p{Tqsy4}ckZk(Ho0Bk&!{cvEENPcI6>&Cm zfl_y+K6WVS%XpP0I2GaxB(t3_2@H$OpSGNJv~f(j*xRr*omR#_Rw6$+E_>ylO4LP) z-y2v-`z1WiI5dv_gc=37S;@v%gY^^An`S&6P55Do6PMNRWSO3mZ4_DwcH;QM9rNQS z05p3FyI*zLf%jL-;lODc`i#E!u2^ALEh2XKMhkIyxh+4E5X5*HtW2*}4Ydtmn_MMb zhUD1%Dz<)0Bd4<8h{-Aq!NtXrn8rYU7OIxzSv~eP--50iCMspRXIF@o;q4TD&6nMS zCB z#Tbgpe>@IgmtFDdyw1_!g|aS8WISA15AuK4@|uOTP^0#xM%c8G%X$yQ{{`D71?DI9 zRI~&sM53+qQbx^hUNraIphK;ObADe76ui9TyVwSh>PNWPOw>^?>)yUDQH72f=A9pc z0=1pNBpk(50vT09yo-f!&d5to3a%V;vl_T09Oh=JKha0K8XhDcbJ{}`D8T$5sJ+&b1$(~EQH zV9z&OK!blK6K8bQP3FV8JG^c^8vZCdhQ0S#d^5-eU2uwQ9y{R0>+XV@@$q%UM8JJ) zenoJK1p^mjfaxY(%3KuJpeq4Ah{xYjI)ntdjL(Em(wgusUn?yPY1;|N;!;V{P#kqw z(k+D^!VVk4zqf;m5_y8;#g1G<0jq17Gk&F$F%xsL$igk+hH}xEb(i7tyI#HCpq(%v zlqQEw37y~Poa|BNCxKN3IRgrGyVA8**u$DyYz|W^eao^dX>R(n!pxUC8?wIMEzsMQ zxs*?dl3oZF77|O?0x$!nY>bvUlpCqvFdh1nu2iLKppb~@SDLL@y(&UM8?Z>Fw*^Q( zB`_*qM(~ri{8opqz@$dq(=Gg63J>ja+6T0*jE@nQAU;c0?K|aKIaTTFW`rit;yk81 z4Ahiqf;E!OxYbDj<$!h1--qLL+o0(`#u_?*SLpflr&Qn#taobg+w+6*)VcctG+!zr zPiRwAh{aJDn=-m_cXv&h!KP7szY&6|=c&tRE~pFVqgX<=^%j_vUNcWWOCV0ER8Y;! zXwcGnum~H?5^cHL@1(ygwdmFwA~^@c9~?uRx3Ouf(w+EaY1HR5%{b4%(;l1-w_Q5` zX!!b`(Ku1ekAzdUBK!FQTZeCBJ5$w|-II}T(3YT1W-|PF+xnWa;49f}VQ3z_a+7Ch ze>P_}^^&YG5&aGjY1S+FF+*U?f8NVyNR;Tc6$&*^{bPOw5H)cd4uVR_ZM>BKx?i~- z-A&)94w*nw$5J1LfBb?37}Fm56Y5Isu(HxBwwp8oD?_K|HVIVIZZlu;`O^-vD}36u zZ^uW8?~9Xi(WF)C^1wmDlAWbk6CdXhjyclVaCVO(t5FdT=ckW+y`e4;FOPg4#T886 z>Mn?!h}?X*Fd`#8a@mf^fmV`K-RI-cOE99P*;G-io9ZN@RP1&fL&xc%lzQ91<)bqz z@n;uE0gg1}G70IV-^6=0oIxF1kO zoF@~`Zy)lx%+x;>5iJx@AXsyyEQ-BrS&6u$u$DaBRJ#7eO5Xn4gACS)q?!7^7UT&w zdNK}r)c!Rl9{R%nav)tG;{K(43;#Hfw7KJd+F_UUWoV;g>ry){33M|tUGDx2I9$A~ zy`$O;(N#}?tJt6|mBW|Lv~h=kP*)AUZ5UGyBTy!Rq1WIBZ}cgbx@71dRqy(u@c_FA z4gcCysU8C1TXkOeK@LpBj80F(i0&dnLQ)q}*bDNFGE5u->j_O*gBv)+nJG~I)h(Wlz-sb~i?r4E=W&U&r zr1*Le#-X!9uoYD%^6Cs5ED15ZXxQ?lVv|AANYdcoUOS6VBS;EjYv^S5GH?$-L!c>0 zgS4y@K#)5w{s|gVf;)?guB1$8zQl^&yCdA9TQWE&p`JkP&R2PQL`N$qU~^g(l5<(a zzyL<}LBEZlZws-2{9B4Pw9Zs`Q-WUxA(b4r0`2v z?NctWJNs!E4jMDpopNwK$vKlS7Tr!6&3at+0fd{W4{z|G-nzXz6lM9N95Z}h#)L~q zT5Yp*sLj&?reoF+mA%DWE0_#L_!N>|wryeW|PkS@Qz&>5v$JIMD#)Z?Sc8yBR4AQD=xbHMGhl0|3nGf*caqXVFVqA~px`x>8hmrr zy+V6BZ3U>rYr>?*h*8RjY^H|Gd$1mQeworGEY8i!xM1Je%H{2L@LQy|N3Ky6Jks>> zlsVZIj`c7LcnUCWSXT;(zjbVHRU3-p+5kIU_co%U{{UqxlTW>qQ|W?onw~|+)~SPT z(4JO5qpGMjp*6MfPNiRe@{3)y!~{d<--U*>-`0budS;@SV{{DeDL?9K7fMBP1Poo# z%>0(P%{0w^1{u=1**%RbFDK{X(rukg@ z%Y=UD((LmEq&uta#vu}qWPS6E!SIkVuxS-65f1M9nZ(-BI~x4r+6`s6LffM}UI<-Mv9Glc!>w>ywVenH(^+X5tum5M0S z=OLwhA0DdG!IC^cS`rs30G+0eQBTtt#I1Ep~m(f|J@EAVjT0S54b;x3n z_-PRg?Vn2v<RdKs{>Lnsvg*fhaB=z0TdEo2})!!$LIA(_>b8hH>42rph|kM5BMB zzM*l69iIN$KbCkr$wNKXq zTddfz9OzgN$mdTx=cW+`S^$+xr1hMGQOnuUE^{D5Qr) ztA$p=v8<@nVL8Agi&U(Stsz(MEU{pq=~8U0eai+kwE%2=JBx>n36tgQ%&EBdSAG>i zzT?x3^Jj-nbTOKIds%~;y;21?H|r2Wj!L$uctu&=J?>NdSFZIrOv=Ke82f%%^_9l- zNIbT+Y5BU#|5d2}ifI)7FNyk3Or!80iMl6cvlPZVrpn3Ua0q0a*Jvc2O6T4j^l^8l zC%+{KoK`fVu|AkfAWOh2RRWAoW;0=eMG{*Ikv#g?3; zd^w$R5l~0Bi5v0xpi0sAV`e|<+Tu8)3G*Et4Ly8ht&B=$EFbfX`Ob{qEjq{-1d9sS z=XjjRJ0E-ULrq&j=hQvVHnc|0?ZaWU_VCBrFaqN@Ba*vUCLsI)bxpaAu8AJi%PM}< z9HYayjH~T)$1Q|^+@&|s9zaDfI*A{g{H}m!Nz3IsUPa-~X+(tercVpC*i1q1k!iz- zl^3Hm;6sm&G;8A7aoz-d!FeZJ177`1vmgL!|`F&@cD@`P+o1A z5~(T({yGOy^*W)Rr)TA}05zX#T3o!jPpKw#i1>Zm^)L1phlD<# z^^f1q-6M*sr~9pdb+WNwi+)q^iQ&%4)ARg-PDP7 zBwuH#Z`sAt17b+dg(~8;?JiUMWXVdWcaJ}w#G~$u6_~d#~I5-H6=i7 z0~)mWN3mZec(^eo_#<1v$NJcRRsG)4Rxj}y5s&^nFo-~?KOC>FKec!XB-+=MZzBK? z=(yuoYEc3rmxw%yj#C~e3cu^<&c4Ufp+wtiB~b5<3^$0bZ$}#?V{J8DL7GOw{9QFV z12!wISQ8;E%RrB^PHMdYTjH*8rH;>z4E^rSs%fEj-IXjCac*>YzFHY4udr3j1eAi3 z0P8BJc;|13qmHZMS#L0E8R2}!&!vokw}M_-B?P&akF3*_}J+^ zfar|XyHe{w>h__TeugSUV2pknd?%kenOgCppoof$uk9X90L;YHY^z(|T{$Pc+xD4) zf-sDERC0q*;cv_e6GGFGGn!!Ohu6ZIi%Ts-M3q0r7+`U4fnyLPH1;yTuz=b1mUwbJ zRj>#$3b5sTm^U65CI7*h=yqO*Gu(vANPrKaab9(?xn1nc`f$ z6SVx`&NLi@7em3iQk+MUo^L7avv1n$Tx;1_=ZzTKA8Bf~2cvEzXSEeDET(73v}<1oDeajl_SG8x&! zq5!h|v0QP?z zVkieZyt&9<4yYR~Ohv_*J?MZINxOf;%NQD>y_7N}5=^ zhYBKvBPzPZauO%ZGiy}`y^X<(NHin@nj+?p;5|P4gYn{q?H+^Tm6qc*v$Vm|5mQG zO+l+bGH_Mg<5Y>N(4WL9WVUWi;h-aBkMrm{b!5oYIX8UZGR-!s^jXT)9CmP`l$k+*p!SBNJ4sDEkEU7g>k!oHlDHzH{K)^H z6-hm~F*O;Pl5Xa@Q%`||KR)x3cn-0(={Oo@y-@`ZO*d}KLXjL$M}^TnUA9Y#N6leY zF^@=m0G0TB=J(f6zjgKhe!01NHR)0!LKnP;VBUd7Z|i7FA|ylTA-Q82O!@s(Xj3@0 zNtg+C<*c>ICA~_Nyg`8uQ1p%`YF>RX%ALP;EEACe)f47|s`sMs$mc{*uqB>|0$;5+ zoacI|ObSw($WVo!3FSt26?Y|mX5+HK_Ex#MAKi=9)J@OBl|MoFS3?S|jmMa*ij#`L0Bf{kSU zu6_a^R8~g1!y*vau1l;Sd~vOYXF#4g;;FeZ_b!oUruihVN&$6jxW)UK9M9ap!qE2y&gc5@# zC0OHgp|O*-y#X|~tat^QJ(XrUk=n!m%YIGY?b=DB_R&R=lw*~0M5`9X38x-e`Kqj? zO3AkC~j<@kWzz2w)WlP<|F`Qjc1SuK*)^%gI^t!DNP3CY`%>pT3A(X^6;_0i`Ag(zTG#wl5j4x4E6e!TU=Q!rXY7YdT;>n%bYdZ*wG>>)v)7emBT2o zbep&!-@0JjEO>Re`>M?kM7schz;802RU09Ukg{kH8Wxb^B=Ruq3 zWg&<#*zi+B2b8+Db6c_=IoauFDjb`Tne1vz4;`*WWRM?4^R4@~)DE(E+C!Gk99w$$ z6GF_d1KO;A%&&i{CIB$ovJsKc<$>qJ#Uj>nc(u=QEktekWw|MU0=6BN!-g4x8CDD!(TRbUDtzIRw@T;RZ8;Q5x^w(@~NA= zl{{|53y$F~T*A^;nq?A({VT2crn zpqpjbi3yP=c=}#J(FhuWc(l)rBlGG6RF`iw5}#uz|KkPvN*xmX|6zPlbdTm)1PnN) zK|&tZj&u^W-3kBfw&(~-Q9<0fG>n3>D!T%o7xm{@n%tpo=m>ulvdW2VZ-2@GrdgnF;Ao5qbb_MW>2-Fxv7^uq`oU=F*uF09qKYI^qjSQi%4gdHl1GR=(^(*{nf&n>;k9%zO@sabFPu)qo_?Ck&pN964 z>-vsLSqGUPW^3PBo*G0th`w0x$@{NUx!)pCN~(okZW}GAtj?1Wu+1>;(LtZtk%&!K zNZY?HFUz$OBsNfUDC$ZmRgub0+Othnk-S7R{KfK*ERsEW4%D*x8D4z5sNVt}nKew* zPInj45^NXK$t1q-`ppqLF*o&*+snN84DnW+1O6D?ef0-z6rnZXw(YJfDZYhZ_n{-6XFs&K^B;CG z;Yb!TPJc>0vO*9mP?0Vs7q|;0 z4X_;a#!ssj%+hAJ$ag0SSn58q{cTYYKv48k_Q*?@x$g?c0%=#^D(#3*&mtW8`vnBK zw{C9W%IiL}t=L(#Li_-G`?QDEJs*&D?qeS02mBNg(2OO?ds|fzMX@YqX?u;A+cImA zxsUUJz#kcG=Z9xzw*|KXawEHo#tbg~emJiCehT~f@Pxy>GR3Clo`IsJuMHM)>Q`D5 zutN2m=G@%7vwSUdca)Bqz^LOl?=+gF1`taEvNj7F#Gu|HxuvO9OHvn9oH;!F6 zIrnWMBEc==sEUERU{~@}$BtDw=4TTAJ;-kr(2pbrZrySvc}<+S4u_~sN`?NPN^*!{ zL|icZ9)r+->aaPh{M^#l6P5;&#mX`Vi>1LIy4$PES9a6;Nt4^6t1wHyXU#=0=Un1- zX{w}thCDD4l3iSoDWBZDCUa0#;i}x?2xhRDIvw6ky3GQ=XbkozTB(TN+I1dpFbeB~ zzg>A57AI^yfa2H4cpO3I2YdS8tT*#Z5+u+@`zr(mZd5>Q=`on!6gaNB*^~fWw+MV{ z+KN!vOBi1@n(_kT1kuK;Zr4TSZw@_hnFq*+^VzveD|y~cLp$;O(&-X;<(`hZh$Cd& z2DO&*K8)Wz;VCN0Fl5u^6X0(+V!x5gfopQ#Y1U8vI(EaC@5|Ueq2{U2% zbCA~%n!nMhJPDj%55Bn^i;gW4DxZ_fnpO%3?L$D!B95*v7%`6|glzKevFmsd{*$M7 zK)+G_H^%qB>cy4+%PRa&y}0rps}Pxb8R_u*fP-hjx)X2Me#VE;&PuFQ&1LPdvKb3YU30@b^q$03VEMlt!Uj?`n6@7^ePM_`vTYC_OW^UD zMo0vtP;#w*_cf4E+l|u=mSJ~OBfenb17uf~6rR(;$DgilENUr|;zf=oP(OzN?hCi+ zzvOD^dgysVx*nk_BK7%xob2pEdP~6XHJWnG|ALwa7T(N4SaD7W*_h}mbb?Y{!tWjG zNaB!zsXSY#=5hb9dSzQXsz%!6aW0)CsyqBUWo+AEI!s7Eo@)7Xkoh5f?rJ$a!oH)S z)BJ$***>0tU)H_vd{IvvWZ3$xn0v|ueZJ<6!=e+SEt8kjv8h4?ewuL{m|f`oxZ+rHKxG>9ovKj?U1!rrKwiII&c|Ik~5 zEp4QOrD3#=Sq~PCQ7CGDHx5$}%6CUCD|Rm;5N)~ib62{QsSgWL^_|C01n+QHsI>bt zc@^j&J&UzRb27ftdIgJu^jFSU@2*5`aEF@lOh%^HX~keCOp_4qd;3ILSZ6PM)GliQ zXSMI91$;NMaL7r*k@?i)hZHB0Ei@&VURillgu%xZe`hW&*RAW03mbwUfBlLT&unCO zk;({SIUOaS3zIKZzh3MxLz)IhEWIOQqo0E{rZi~&RM3ciRTf$ zKl0=V5<%0W%)n6e@5_qWlTvk3;tUX7}<}o-+eUQSD!0G&&b}WY`KWDAl zSuWq;=Z&=O`dpD4*-@IQ<1}%To@PJmH$9e-e5w&GeW2VJ1Kq%Ufqr_N_ zt(#`$*16%M-Kd7ShMo*$k+Qtv6oK7_gU%_;te4L2P`ai!rrKp;2%II|enS;<>{V5% zdiQ7~kIy^EX*GlYF+Q(~uSsw*kIS!u0jmk%e-EV@&#i|?9$VAr#(?#Pk-QnB#b`F* z8xIrP9hF6dvM1uO?xRAfLQ{TgVr1|WyT-o%-g^JXl>laM7aShW(g?f9IM0>P=V$c-z!4L>k^xXlNT?(y?XDptSrIBw|07E)iI?X5ayv>(XwD+JxT2d7gp3foM9wBf{w3rQx#4P zrskB8l<1Td#qDhG+vJ6^pd|B55x-&1WQ9lSQ@aqOgG&eKOy~C#9l7 zI}kDaTcM6zrN^jAa=#y04`?aTmFv3dPU8ct0UfIe7aSZXURp46i*PaNeWFVl+NIaH zkEVkQa^D)4dxV_PKJqbsg$1fTuQtkt-A$2W7SwbfCz1u5V7a?0j2P12Gv9_CYs2#~ zbs`iqJ=D=o=ai+cw$yLdsG-L5WUTZnCAfd-(eiyK(!cA)y*TJsIOri8ccXvDE~EdK zYnTsV{I6(5^gpg)r-7o7VC)@?u#cYro{gSw4~{kcoypc{XL@Ugiv^H)rMY9ZVrl@!9-094SwpA3;w7w zchqlp8 zm-=?~cw`{tT$DPJ=mf1VcQ{;X%OleW`6cMcnz^5I|oV+s$xC?RR8ZvH-r)>h=DE1mwE z-)|qJAPf6(xwiTxkjZDDJ+xUo72O3tcXwp9KoF<`q4likk(H^~ehEPc2;kt?smwo< z3N>QSjt@dNA*B4ykMxlb2!*GQsq0K6?d-$ z4*vPK2&N*3j1zjv;wRL?_ZMK^eQEu$sys_x6v6jN?{uj~!eQZciHz-^M#mid<@~a{ ztee(ogWLIkID4zGw)gJa7ecULK>{I2k>J5yix+|gcN&7bYoQJ9?(Xhdpp@bc#jVid zMO&cMySh%-`~LTH&VHVC_QlS{?<&`yImeu1j<29N*PI^l%<6;sANW<=vpX5-`)aeC z#5GmqLP_n+)Q9*`vjCje-|DZQiswNKIyC$~+qX!0`Xih+!n`Cj(!hI>{>LGO+EkCJ zHlC52r|#KT{p!4gH?B1+Mv`%jx&pYrrI5)zens-Q^;=_AYsaEuORV;W?#4B=R%ZKw zj37@zkgQc%NJI8cA{+doYhf=KZ~8~`9G(s%ZNm~#=&Bxvj7=CnEYVO~Ysf%)fs4Pg zs(&x*;uJ4ngW4=F^H2wMKxX~;@dHqfoVfJZx3xzj>OZ3CE|L0Us~>uc-q5g_U2YRm z6f>=_MZIr06HFJSy&uc}%0kL&k=;Y^7FH_WsQ&nC12mG*PF~XE`(=}|u}es#&x?AD z<_6TR-!a&^OTa0i50%wY`{3Z6WIg=2E9zxk>5jU{;A|D>!seOmVGmCCC(NI==1L%6 zAty&tV_%;_Xs;PC7}YnBpb&oS9xU>c-+4UB+j! zBVhO%TGFKLb8f@8);bIQc`J`+7*FFwoPqGOgpa?f0VlAc3Ta;f=%{N+8^w^lZ@X;* zQ3701cVDbn&!1J{$XtNy@`xLdb44X)aG@N1}G(#iEe>Ht1O?alYrb1EJDgKP8=H*>p7qoU&spUC!KAsM#vJ|UL-?tZ78pWYZ@-wHo8nrJ5a)*{j+ zHOp8LRolS^%GV;*eT*%MveIaWZBFLMR*R%fS&n_W)lp`i-K5iFe)ip7@tQZ7^OZ>; z4epdr;myxw>LnclG8N+|!9wA6kax7x(49{ewg4AsK8(kCa@U8VUFWq|xTD~q#1RL+ zI(rm#8LfieNb0Re9s=j@A8V0T#X&$*Y6z#U@2vQDLf!;}E(lz3nu{5=jOH@$wJVMGE84|Pm(OAyN1_9hB2rg@ zH+Yl{L)%wA$E^Z&dnt=D6Zzjcg*BaAT0hC23o(eyT*@3Tex*7z|K3@q%kTAAX}K$d z%##!KzELz4pa9J3hUxI?kOqE~$%&_abiH+0{pHi6;wh>}4#v22NvT@&e?r1m$ljV2ddsfU;|CyElvG(pQAne|VJ1RM=!**0 zPjmLOpCj2dQk3$rfO{&GoFFKZJdqz3J;4GXx`N5m>zS+>2fLzQxpYWuE z%+hzA^$;i#ENs<)jQl#8m#ja7y6ad2~^Q{iMilIV`MhN{dV)Z zN!WlMj#rUrV6q&|eouO|xx1r~@u{2hCf z`e_g#>XBX!*c#)2G~I$A;n~G+Ea;R=m@PKQ^=`g3V8}8umW$ z8EyEzMZOLXaM;jZ>M-#nw_KR=z??n5rT(q-Yx0TDNtfRLYpJppk zXC4}vn}G{PQ(6xPwh=KabQhmK+sNkZ=E6>%a*}Lw9Z3WG@neqte>V)?JZ8qr<#3HDb?Guy?J*r+qZ|Bs$Y(B#rD?@38Vi!mYe@< zZf6R!4WPLb@e=dP(Kq(D!nKFI?#}WwHvLQU@%!H%nIS|SA<0*A6Nz@&(65$+Qv0OIT$A$A?#Tqy*vGx%5r$P!n;EH3 zvd`tr(nJCDVVui}@lTre4!-4Cl^yYdug%rvvu z3=jJ{9zPdaQ&E5VU1}AGA1a+4h>dX_SzV@wK4a55_&)y9q|-GLN0B59@+I-1glIo1 z8O^Ih(U)(Z{&u1G;l9VNcY?bVd+Ac2P=j~panf=ecCBybH#UkYsN6NzOUp6*P>Jxl z{~fy8ph)^>jh6E*M>-Mi%F?%^=rfj$6%=3XxHbAbSzG(mXRJ~5`III>YBwJOaRHKMroO{gY>&lik`!CcDWCV?=@a8} zUjUc+g1!1CJEMXVU`;qf35vk%&ZgRQsfG3|WX0qUKA8_$4&IUKrB8{>F~qIKg*_MY zYFyV!+K;S9E#-89hg=YwTT|?$XTqO^=&6g(ub>y+-+88FHoMF&hHK%`or~U_o@hA zmhj--;sVW>=1yW^`;KL!Q(bRt)!#QB4zmJG8MX1nZZ-4tm+!T^jdwkx(9#(A9oiFVur(43p% zrM^v{56xDcWCMezsTJmHl=CG<3L70!g`7NP`qnEVo>7X9Hk$9RpFF>-v(RPRl=(7o#6LdNH$ci&pZbEUjj8FRX9v}pP;+hc4&D{^G6#&cNs zM<4mU+`^YXNio@29}*`}RUKvg-WRx@5hgfp_~%M%?y2qJ|JX48x2xpNe-l*yE9JfO zA3?QEf;36yrF{?@)|NsOFFCluLONHq;&s%!_^8WV1FyJO=0F*2x+F`mO80rUGuM)w z?bFb^0~2K<(&47B%RYpWQ1_M9A_doxbIZ-qE2`3(^PAxEelc zKtG1v+$$POPyZkrnm?ubh4~Ec`@7e4i-mAfC*v8CRFg3DU~nUEL^<%GLZ_OiF!E@_ zy(T}y^?_6^#3PQGD!t3m6-c2HGJm&REcS|<<@|nNF?e(f&oGu;PgMMAFUR{_S}+BK zB>svQRp=XH7Sz%Hg2A|UlN+0BiuA2@_}i&gkdmGEmZ*=AQc&%(WlkzRJs>3tiMlx6^R5D1*&0nAvpM?H*UM@)zq6q{}1o zlKAc4D8pXN6N7+NrfKcKxLW-OIfFd*#^@!FHukjsjA#S14wF1zieJ{=PNY)0zQb&o zB<(kh6@8JUyQwJo8s7Gv4p#(!X7W5pkUHi);D~domSBx4i6J zvBQolw0hLD82zSML}&h-Ntm0c=alauW}Sdhzh!*tJWImf{8IU)re^0Lc{e0XZmmX) zfsY(N58E8LpNp;K*RmUXObM?z$e@mT1|s0pq%R9vo{f@ zRpH$A`2&-gFE$Sz7mJ#%y*XA*C9C?HQK!50ROf)`CYDISybS2v-S+vfQ~Q97W~KpVr@YBw{mj4O_o?1{@rlznQ6vjCmb90b!s5QS>5|b z9D5%aj4f6OE;+Z-;ec|S)i_uTxj+4)`1bc;^0Rn(t>qNxa{a@EK&;&1{ zzj+>mJSc{MnZ!e{J^H;r5=+>>;@3D^QX_bO38wNV&Mjql0OII3v%yeA>2b_Xcv^Va zxBbQu=ty|az$?aBAH~yEDFfPjAPi zHVip_s_=iEp|j;a$|qsy+(-xNOw7$l%c7CB`R-1#NGBYRT4st_jiHLBD*6no2$hDV0Pf`& zc$D-z($$gIDe`(VQM`lM^B)P)5)zk6jEkVB)heabFTeKYuf{VRiih`We~tR(=p|Rb zD$=wgg#9yj2c(Ra+rR3JNnEKkA@infY5~5GP2jgQb=v9&c^@gK=z7PD#Y8V}GF_^K zwZC^Fk`>9Q`77_;{a@!UBSC%Z=f6+6d4vq!Y1?ggoBP~ujS3DT+e{qpFTY|BvlHYW zb=aDHV-$^BQw4{vyyMt=25*|^)hT|m&gQm3>``H`c-kkoy?6}mLsDnzckk;%5Qs&) zo$;Ggyz@=9E(FP<{gKJjw`0lA+B-j_1j|f2WOp{>FYTuU`;`f(lssSwmZw_pU-H{! zC-oNw$hXO6ZaN(G&RCnIW*2td6ew#-Nwj%4re9hrLPl3vMAN$Z*@a6he`AP?rXE7k z#Wh!fI0_e#wbO-{3z1tCd0a17aDT(ge34PVPF^^OoFWM_j&ywsl?~_Y+BpV`1rGv$ z+mxKqCawxgi$(BsF17q5@v}>ab(=I$#JLR4QSbA3De~J{lHgbbwDYnnm{8JyG*p=5 zdk-}6XBqGAxbB$w&O`5k8d>WM-Ja$-S8v9=L$eY%r*desVi>O;p(nn-S2tdTbxD=w zK+%ap`gl^uUD6k2yOYv|B*V6%9`e)tK@wsB$XbL{C;H#j}dZ42?0U~AVfGg50d_T{`pbk3K6{i&j9g^ z@=v|XA>WJ7^K{&bK8U@}`mlVD|NWtF9kYiF@IvzAli8fK_qOm%{uS~cpptH`qi(PIt_CmdPF(7v&|t9(lb1F47Ykr=!b>zs+$)!J+jpdz}W^ zUytbmCxb$tM9W(+k;Rz%lUWdOt#j4mTPlw8-2B$~U_IM;qTnjXRNzn=)1OCI1{%*M z&EWu;UXV&$`Y6C0?wWnh2?d_x2nb<=i&RgxUFyX+n_Z=i+cbiqobE4cu1c4)7baiu zb#URFKp9sxe@1#7P9<@?aSx#Sz+Ld5J#nq8+Hb3rUZYwUn{KuAmj)6>d%!V;h(6Xm zvp?9Sw1_Y(H2L~Hp@TbLymHUnqt6)4`XLJMtRlH?=J>lz%umCIK9+bsD1EQ%U!R|L zRnXs~pq`ku;x02G!aDD!Xe&k4%$5PBNA@HPiixjsV$2N0YyCsprKD1J2ohxXY@ZYi zH8BBqnBm9^i-{0)+(}AJALZT3o-gopovHk?QYLNg5ro}O0sbr5`>;yeiyrKrO}_ZL z6O=6H!<+xxT3D6$*EQP{$vcur182}AdA*=ECK#$Ltm<;?IW_ey&@?ij9!@$SZMm-x z%(s0dWqt}w7VZDw_X>|N^o#Hkjx$24uUj%Yr|?TFRChZVqNXT7i$}iqSVgaNk0bgsyzlxV|pgnK&0vUq}H23qF=No`EZ@ z<}j+2AwdW6Cu>Ds}f^)>2hxbqTKBcipJnh9h-g~3eqZH?? z%p4F5dM6;G+vIxp+6(KK@oP+IyE)a-*G+pGT_tR6F>37Ac7h9zQz@>79(y_R{xuQp1mW&5W%qD$4UtAt)t3)Dk1kx= zOc8D5WNa_c)1rRtrYU0FS5EvW7Ya&nk<_Q!r2ArY-e1`C##A;`0suF!iv?I6S-e;t^O8I2rEApF9H(=#zd=c!6zNzoAD~Jr)!)2gFqLjdR7m4{zpH8*rM!OC2+*&kfGATT**o z8|^aut`%!P7HB>HC`KSx#Xs92S%7#Z*G;;FT9UDPnjF31n{YAN%vp+RTLPQX;pN}9 zsv{OqY3yN34WiF9kDDt3Ur6+o`jCg&MBD*WC?75M#e~UQNal5gUDoap-m;kixP@ij zyF+gIZQ)0B2>}AllX#l<*Df!%7)sn6u|w;bLFMIucVm8-&2X>C?H4&eS03rchQuk; zJi4{zvVu|JmMf%XEP(9Y8#eWXy$l~eV;~G3y1VzI^G)Z#tny0FU%jQz901q6 zrBD{8K`AuXpa`?^$F2pZs`&W-CbFYCJs$}DGtvV>hDjk`0FdVU%x-cy$TThl`2Q^~ z{1cmOfIn46kv${YJ)`+5Gl!ohBSXKJ=GpMU;~ z(@Tv@_za6hD{@1DgsRFNXdzfCA)Jj~n;SBRkDZYcV@Qo2I}esA!{dfWCaYFwo&zt+ z$;na39tC#s~OQqS5C&;A2WJPTc)Xv^21x6K z*3Ea~Vw}V59mShfObp_erWAiU1mc!#v3bhyYX4F3 z`fE^|uzK#ec(;~;-8dS%znH4Yi`zve2xNKevFv|AN&Y8{fUU&;a+}nKW?QE=p1zuB($+88JxBH1)swtG4 zx&ZHKD5&<2JE+j*6oB`g-V#VbgNhd&j1A8gtgynw49;=tzT8IigH}bsVJ)sQCPp_6 zMH#&cHmD5BaJ%OP&Zpa8KAamsJN0<{a~FhYco+}RKE{n0NIU~<)(A+zZz^@=)>ozP zOXS|S)fF?IdA?%dz*wu9yj8tovA&!{@-tmviNdbujYryw<}r12cqzA<(34%o=oS1$ z&QDMfuR*#W8INTb?u7w}G`y4u^D8vc8Kwpm@$Ugj*gz~2ts*URL-5VU~$;e$uqJRpJ0Q;7k3X_-oYoX z#4;Cur;+Qg$4XPYJ7_Y5!X9Ckiihf1^`HETUl5%6>JSE7pV=f4dTX`N!2Wp!LP>un z_$h_lnD^v5SVf$di1{kX-E*`wM`(n>?a=}8%$AHJ^P+7Ip42&uY`PoQARt6cU&xIc zG@Sn-n8A$MO4UalpM{#MH?UaI(Sp)2`|OHD2vvId z!ut9d{I)?TTP5{IdE%7G!S}|5R3enXXfqn8GCDv^Oxc|?>C$4MkeMLmiJOqiOOIYtwcWcxlJHgI{w4 z5~saq3`A9)oAnXq4U62MdQUs!NHVSyyFvC>JMzSJ{v4_~(IzAZ49s5`$6;vV;jwXw zX%{AFc!VRq?0S|vsK-G^ePuI-o}S&Ol^dH7YqwCYV{oYqxWjR@U>g7Q#2gh@4F0DL zm-b%d(ffnWKl~X4LoUFOzmLY!2>*Z56qtbr|KZP{GzG$E1_W1>@*gD1IlSq9806By z{3SLhB`2Wu_y@5%RUiRs{3m7IE6K;7`MVU!PJX_e(Ir;W>llV79qnTizzI;fT~dpv z(1)Aj9B@)O)T~lv!6!YyP5Y!)fYiUg6{&D|tD$9hD*E3HlXbv^WgQtjAST1Yx8?uXVI!35m(Hl{U@dDs&{ zE|+5l2Wm}dqYbS^k%&-WkxPSwOAa@~Zk;dkY3Ys)D;o9*GwBSqx{?=u2Pubheip-4 z5a2KB%c#fdLWBl}Ca|&61A(CDp3rK3A2{s#D1WK;=|tJ(gt0T{%Z2Ezd?=MNu|r)$ zMxdy&Q$r#ZU+fZb9wb(TEJYaP;OfJ#sK@=CzCNb@0ziU8%He1$A?)d-53`NxmmncI z*;H%@Wah5qRQ7jHngNZC&EiNgTi$-0q$hJ!sr?GKV#P_e*QF|FmASg zFN`zCqldM=xQP?BQ%vMraY3j2QoKct&v1W$G-Zd1FUE{uFh5S1O<|$6VW*dc>4Dvf zLw)8iru=99*?j&bP?=c@Y@G}-6H3>K3M@}RC>Sg!$8S=_%!_a(a!xd$Ax@FTC&D(Y zUnUms)#ddaMFM*cJgIrpvk&LIM2p%x_leEK?^r9d8lzVgP2h80c#%7DhMbYhUkBub zH(3*geTm}mDQwec+iPVOR;I^Bsx^~ahWm&GW6ku5l$y)M670uzLkvNi-KYC~$^8>z zYyQ%<&kONw)6K9<8-~8q9g(z<$mlD9q=S0aFTYk4Ue{JgzdRLD_lYwbISXy^VT~}n zlE{T4jpwzAMYG#S*&;}J&2k^DDA1%tV+-zFH3Thh6wl1+qP*%jOIf$cO|5 z>&!?lu@9;4IesdXj*xhWZ0&}(%0;x#L|)iiz)CH7$EGRGF=sqzG;Rm>DqP-DiI zWe&~b&5<3kM8|2gv9=rUfSE?h(h*yW_FKW6OCeod>lD1V!p%-fOpuqTZY~c}adIJw z$YO%VT!RFe3(~4Wg1{aG1~Xd;U*5)LVV>erGy0PV<;>xjY+^^U>=m?+3fgh6H%#}n zw^S)mzvdzq!d`{7< z34H6zv9&^;fcv3+7R*glBX|~VMAzM@*E=3VTi)KrijfFxo^`Z@Y}74;CxifR`qbVQ?dMP4vw$}(OM_`wdmAm3L|ilc8z?_o80d^ z=E;2408SHDj}m%Rz4v(XWcllHMg^SCLxG&|%GmUA(gQIdzYZ#h{b_8q_4()FU$JUK z4b`+5A!vgBWe3yCr(76>xC)E>g;GhRr(1xXWzTJ7zZ%XFNW9L)Vrk)`3q!85#Ra94BowF4r%xi|kYf z@j;E4Rx5k!TRqX(ehZjI!F!gCw-1K{IkHZmYMWWvV|NXQng;eP_==)3-~il@Gv9e) zgo#CgFp=7a*UP3ISU1B)&APkwWCNpPZw5ihTryBylagnQ7Yb$Q!7qE7^kkk>X$$I= z%ZMffQjN2nshQ4?@pSm#0@-x$gbg$dY+ML#uNXt?P1bsk!wE=OjV;-*jc@RL#XF@n zIfx9ZBUrMsuzjo-2Pn?ldYp=IZ;_Ahe`X6jNg)`arXpstFmh%k(J$0@<_63NsgGDX zML?buCoHGYH6;WKucyTpMp>vCAB=BM=Yk(ncHTL-<7~Q8f_Wevmb{LV{h(fOnrh#7 z2mB>t$5m21{XR{OD&>8|V<-2CmRkRI9F){IPjtscqE<#lWen7~oDJGgaM29>&hnT? zC{TaoJKB=11G>!SKZqwZH%wcMvo)Eg6G$+?@0Dd3`h3wMj*$4TfnBY>rgI+c0;iTk zEEM;ZGb!$Bww3aUmYo==3%i(-bqK(DD<9^{@n5P$QbO7(A$`*~Lqv>_{C^WkO2DE2 zP$mB#ktCbi?z$!-myh+*BC}3B+VE7$kOQvZ3Cy|Ns56~n7)yO9N|M1?b1W0}{?dWc z;3Tk^!~j7_(fKYLOY(pLK)MCm=qCs0S9M?9<3LnDN0HoBsz$0Q-eLPh>Bxg`nAL6D zSo^8`NT*E}kyg@Ge8ri@PdJM-ZH2hViF%U?c`U^*QgOfp`3FQg?+&#N9I~s+;ft_u z!UN5@joTR#VOa>tI~2+xfBfs%m=RZA_3URCRBAd|H$=O#a5h=*i-Q$8nr;$5lP5O^ z9TdH-AK#`=?1~d-n~@O3wwi($nQU?)y{MiDi`;fr#UX6muun)F#Vx95j~3Fi)#&6L zUJNXO={J%@3bb1UeZeTFYFZ1={(y#{42`)Hlyqcyw>J90dRtKqA>u660|5KVX87 zk{6a?x~~1UT4Es%SM#tv-Zn?1&%%9%(-#ircyVFjr zZ9d?qr7*nhR$W!pOSOjV`Zo(Lk`vFzggk#Xp+Z+v;Zg~;rZc%f=n$#x3k#WX=2f@@ zVCsReL}E+;q*9Eo9KH>XwjFmToDuTWVkgvJ`H~7O+=#`sY1!yV)QDn^2AWq@1tZA8 z`<{kSkkg{^p122Y(y?ycskK9h&TKt6G&q-P z3oPA<%SnW}%yfI-sL_xVaGPOV*c90dY!H>Mu2&^Q%5$Ow?iK#x=NXl%)2f+Hu9fAX zME$o3w-}d!#F%By1s7y_n^Ttgnao>CQ0D++Qil#T1YcH`XUv4a`!j77e8i1P5}CUg z6HLq)h=w0J;5k~-+~^)$f@KIZg6^HGc}43{c75P7P9bFQ_-70q^ZUpIBD)BLXg#$_ zo(@Crx_J+Lk%h841Ex7Rrgwq=CK9k|KPd7vLY;REkKK`81lTEzOHG^$SEA1jsgX7? zU3V*1YE2&vSW`BmqVkys^Z5+oZTSd?P{Oo3RJcFHc$FOS!x|K&AEGYx*ZG|bp>`JR zwqAl8d@LV8xKfJfz)>`}WIXja(_tt8sQajee|(GVizCd8IcDEu=SLzJakW2>y|RqK z`IZEjMk=Anr2mW=$kD!Rr)I+#{W|dLcC~#p*L&hX8CtQKxA)G3P#D^ZmZG z9dm+=^MwA&ncQBSr$5;#sa6RedsFLR|CX!141c!%wD&(=Xqsmt@09*aoN_Q^5e$ji zEw20Hgl_$}J_Zc@_#fi5{?o^#GdNI2YdqAe*5~M3h!kVR6?vir=r?T7B2yErw?zyl zTD$ipeh9AgVP=kf{lm82QJC6LYen1+Iih?f#OqN;dQB4}LmZ_)&bUFN_wo{?k#d%TF*T9qUd4;;)nB*qG)+36m7 zcXqfg>CqgVm-;gDLJSzG_0nSQMSx;4e-2goZbBUGG9A8YIri60`zdDS4mB6dgjOp9 zCp?xe|04E!<9ND0PNsqo_ErO((W<(3_Q^4&>@xQiR`LrNmE<4u=>rZ{uU=q7llnhO z(3;uAgQ7pwCkhKUw*!NzIovuHYe$=uVl8il#*>1RnV~fSpNxA4DyoIP#5nIple{<~ z<5J5^uHx{KADIc5?%`D!Dy~&YuC>p$i8A0<%QGc6=FAR!XrSuOD=%M!Mv8F~!TT9g zGe>44b;vWIyv40@B5{BxATsN*bEDnH^XiJL1EjlJUPaA{DU+%qrfr&+eg>)AFg#b~ z;60AS9Jhob70z2^dm%9vqqkc|{+iOr{q4RjQb>ZSl?P^!jpM)<*V5!!|BeO%emk_& zK|VixQCO`u4FrVu&92~VyLmpq$v3>5lM)$x{V|t#dMJ%~y+QS=m7_xJ$(a&tF56$q z?X+ISlhm!jtU|||Gnm1tVa8cU>Inc;o*C4hMB34WbQ2oWG(#HCKw^ujumxkWl!Udi z@A;;}000;k{o=<(`MYUo23f6{w8!Ph0&t9PU8VaffK(Y*Xe$6W1)gycB~vRS=prT# zmeojR8Qqp=F&uWeGEsE}M?`0x7bd%CuFtN*?)pkJo6c~IbF~Ye<173EiG4s*JA-f0yM{slkCmJYyjZ2Qc2OEQ7rS)$t`A zE7nu{2paW$rpY%hLCZ@3=FtzL1kySHJsdS#k9$sLZjDX1I8oc=y?92X~3yc%@+5@k!`xaok^0O+S7AKBnWxfx z9!4LCYko9U2L3NVPXDMNN=Qu1z9u8&*T3;Y6gdAMe*F4}AA5T)mHlQawxmA64#XCd z%{rpS_E|l)koOS|gEnnPojb?c`HzAy^2!oYxG}02WSMkfM&hs>yKK6%kRw~^@U>zP zrWK(=xPAd;DSognY|)2=QRiI+W9*+0lpZ2c*L};C98*Xt5w4ofCJS1+WL3Uu;|k7+ zQ#BDaV12WSu_1pn4cV~%mX5}-n8sIn^0-mn18fMI*SgA|rNv6$-k3nN(QVs;2aK(SgK+yaS+TZT?y+&_TTJY} zkOvOK89FV`GK&JbUq(2lF%nf$n_7y@=45jaWzw~Y#l9KmK17HD*rU7HxhcARVLn1r zQlryvn@h`8NPok8g{rAfr&-NPNE+s#DkRJf6JqsPYdkIoX@YUZn6mdY`p(8YYgGM_ z`YDLhS(K7gV-g==aL~k_^LDPn1>bAPXfi017Bq|XtKmb&ajfSpU)glxEh$pP4!UwR zQrBEaC)1I80d1#i_p|UhA!=UhSi@5@7MczWo%3f(Ydz-(ArKCrUB698+#r(0`D+(X zOvl5b2;~kY-#Re*t~&T3c_AKPOqslA0?6rVJotJPo@|UXFEb}4c=;6HPG4=@5N7p` ziq%g93Z-c8EJ}hpRBk$1ETw3*5kiJL6VO6Qf?<2V>hnfCFO11Rvp0v+Et70po`yTO zL|sSqUL3f=uhvZ76Gl~*bJfOITBBoV4^)XL!zz!MK64CkSX#_f6)c!I(M+%cO4NG< zY*eSm#luz}kU~%7b-J=Pf}l*Qt;LovkL`54x~806F0g;M2D1*oXBMC zBAS_ieHsFND6y4)i^I8}E=0jKFRU85)*>!zr5Ph0Spc%&KErOUwJ8tuL1*|Fv8ZVgOUe+Va=)M-1t8ZQz+CZ-4p z7Y(5oGdz=9ArKNJ6tW>JR5r0fq_Vs(8+tNr%(vt^O0U_x{R7@Qr`@*>?ibq?#GX0w zBz5GxNXU^ma7!50AtP^XG1+iavXaWHY*dNtg>1}51Y=PwRwM=>+mvY{z0)`3^33CO zw-Y!xczy(iqW@Q_$n2($jnQAkb|OOkl8l<2EXI|Q#qKz#6+0Xtoh!GGx}XcWA3oy(K?eBe7#{K!oQ;z4lE8X{*j^nng#1Fg~tvw^Ci_?rC-p!(rAfM{l~b2=@FcyC&a z?W!-m;Zn|4T}!J_e;J-0uXtuZ~Td=~?~JmlYzu~lmvP>{xq)H5;S4l`sQIhR=@>ey90 z#X&d}ruLMbyVrky8xu7q)MP+LA=&bww}3gJ2x5c@ka{Nbxc>YY-(QuzlMj^$ji)6e z7&0U1bo34j4RM#D)}E(gRb-ear$oCx<(ys%ePi5JsAkf4jVsWe*LQ$X0Olk4SnrIP zl#Qd5qYAqD+g)+=mlAuj3n~Q2f1edH zf}qEqW)mK>P0=qojbheYNz0EezsvNup0GQA->#j5e~=tma8+Sg62B&QDMe*l%){*m zs~Yb3t%aIhYNP4DQB5su@tt{0rP&IustKvX)Nj912BK9gY5E$*F>*$B+e}w91D5)3 z=Mb-GzA`6fFrJ&x?Frbq9E$2CEXTCc^WISqx4qkqbnPz%juYSqUo7nQHET0mU~TjvSY z6-=TXDD$WnH8f8|A_~G9<+2nb=JkVi`Az2&PgJ>blBrC{l*Ksv`kr{x3Bo@8WAgG4 zcHsFRRrG}dQVoXGy^d@6vn;~?TUYZF5BTe!Wi|Gnt_IqH%IfCe5UR}-k>_|~rmt+y z8y~<}f9U3eA`loI{wiN;f|GH1xf;+Z&D-UcwtZ!oNX#t| zx$@jn8(h!BHI)DmkDDY0^$jP`QBV^E$}ZPO3QuyVXbxrnx#yM}MHdQ2$6V%jF6MWA z?zcx)LL`yx=JFZ*GmVKj=!n>8WJB>l?8nh07jt1@eG&Hcr~Lc1{A>HfHHRu-@K$Yo z$|=i;4Ox0KvV8W35euYRFs%#5*h~PJ9POv=k zR|UCcX~exzZPo5|^DICl;1f_+XYyOnK2Na~B~b@vFA8F#Al{Su?FQn!6x6p})QI1T z9lKl8$0CztmXJPv1O81K(;#KsW`#->VsbocU|Yq#6meJNnms2f8S}8HEedg=uw+S` zbrezXY>+rp5wsUci&Nz{LMy@NsWZTKK2RkPSDSOi%V2`_3-_ttr|-xIXbM3u-MU=0 zyQMt!@$x^&b))Opf^|B0nLbM6kI}%`1x#I*pQg=^t(;&@bgSo^7`sA?2+)+G9K(mB z6vS`zWqK>>-)onrC$H5?VKOJJ_5wBhaUPpys#U15VtTNnAz3?>Q+4kT1B`kKN(I@D zSOWO$b5#DkN6$;R3?7F`zoq`bH58M_u_EA5k&alN_vtoTVKc3@Yd;r@1xyW_ve-Nq zEpN|p4P_#5tB>AgBu)1)y01`@-io5#Y>BNX&LaBR?@XNP=B;|k@%#Z!gm=G)WP+lb z%mHI*0eB8=pc+_b#@Xm16BEO3NFU5wEw1k_Q@D+Z=QG^yNUf<$c$A(apxW;sQfQAR zf03*-$8HVlRdB=2Dnzu5@GW!#Qh>b7I!^O_r z!tUc$v=yV?yn)4{-zthP5EuUqVX59`ouo{i;#Wm?4&JN(&^qa3A^F{(YEb?HsJ?g#4yOqN!kxJm#x zwiEBDXqc2Q@j2`Pxhi%Y)A*D?Io96GU^GD1$-u=7M2FUM{Jcz6jW~(yN4fZ_K|k^C zxD2culd?o3M|oWW&JpR#ajAw76iW{#mX%7VL8>vurP9y3H^tm_cLXM}VC+;KGU)5L zq>%TIs`;~6z6V1pDItkX*~x!ck^XNpTN&WQe-u#qKQr5i*R;|pL%uuX4N0@uSl(=h z&2kvC{9^%Z)c4``QAt&@7@s&Qk1HcH%-)#;n7zaK4gMHWv7BAybmaaenQ#i8TTZ#O zh@f7LZ04v;1-2{O87s<5`#j^DHKj}=s_90dFuMhbZMSZ1z|qyOI&t1)oaG}H)#})a z$0bK(?UXpY==z~+i=n$Po0Y+{|LVre5`nejtiG%kN&l!CmLRTBdd_2iKh$?{An^u8 z%r^#+LAKr>f%@a`z97BjY zA#p>st}NZOI;~o5OG8}0IhktGt<`Su094wg%3Cv9qkj_ZhtLqg#sAhvjQ8N3epelv z4t%aLfV66j#R{EK4_|gyGk>jAe4ek=k$gsW1kk;W_#vvU%6gJ`R&513u}a!;Flh}eD5oUpeW3-;OHb+n)#vwk(4wn{ z!pr)3$#%%k_YN&tr(W;bHKCw4!BWev+IVPiPdu`&YcjLy#kPf6lCF2fLHq>wnkbH< z%4&SnQlkVDfC8sO3%8LiJ;!aqTGu45rM`hihN9vwII`K(LTjGRA>nzWRVPOXrw#pU zPZxa8konhv@Ma-AT-+MfH-}h;rbkEmk zTGYyzD~2+C9TeYW^~NlrlrIId6J;P7$C$2sn|QiK9&QFqjb!?l$_!OYW*mVk@%Yd@ zlOvULkj;!CFQ>eKQPW}E_^+1(weCr#OE^kB0<4#NVLAs(Dgk1&L+}X6Be(~}5jTDz z-Z5Ag8&_%--xNgT2h(S?ZuI_>Rwh?XLbQG47+>vE0{=*26F@DH_ZQ`X}Oq!EeZhN9nu>l2hS zbL}YTzJMwUc)Pk@>F>`rl9!|9?VF{6DDa6*AFG;K59caM({v`07R&+}(%E$JTv-fM>L06u+d62Qqdx zzx(5a6nNy(!0B7w5%n(La})4Rwg!Xj%cPUqfJl8=tt2OH51monQoe2t3HP2HnqPZZ z0f`QWrdf;ZM+}0LiB=s}8=K>qjBQh>I)i&x`?3b7Pt<7j{flF|r`EJb?(eY!Mzw+v zE(?C6Ns~g)Xg?cf+y$*4N&5PRSK^#oEQV7>cyqtV;j_56h1L#e)~GB#S@KEsGf=-l zXp`V52*McI*2pY@+DbZS)=_UTTc{RRrt9E>h8B5rhG zk4N~+VeB8j3X%Lxgm4p>OktMg-JaaRH%eB*LJ8IBA0)OIlTd4#yoxaWG`dE#(_G(G zB_L&)!QK?v3eDL!2s$=$@6yohuoHdnVYR#gc4ZW=%h#6JoEM8 zww21Wv^Zi}2RTC!OvqR{oB~lGJi}ItHa>MGLQK#^xm?cxm~ z2?;d`geG9<9YXKy(0d7lUJRjE0Rd4_AwcLoAYD3y-bHNm4pNn-n=VyB(QP~LT=sd- znYr)Go_q8E@1Oj!=J|e}wbo;x$R!+IYN97LWk}`>^Dy)RvIcC$?$bAYrO9CNuSQ9ynSo8jW9eiRo_5-Lj;S zcY2|50z9!s-?{PH@7yBTdy^X6&$XZlx5;*B(J3}(?T{~dpg=XaG|tfl)yBu?ZiL`H zSdSPbA181Z=1X%MN^^L!Se>s5z9B;_P7EmOBWEq*u>v&MqONG zYAAd_XB)C@4T&%LPN70ZC&8SVT@d7<0=-?ESq%_yzaG#e^h=;{tlRrh*s>vW*IdL* zb-11iE<0}JLPAb*U9Wn|Zh$wP?K5Z=_$GBfRv}v6ji`vxL{o-Z_qK#MYifpW{+Ktu z{zi5ZUZv~nG==9{6|SFxwfuS))xgy-NFS{&LnE7e*8_}`?$E_gz96%V8!n}8Da)D1 ziHuR0F$wuU5QhP-`-vI_YkV6>ONM#7QWYVTwTcwu+1`w18m^;RWy~U2 ziByn(?lCU>sm#t!3SYE~I}H66;gV_5wI@->xDh08#w^F>t`kd0H~;g16F%pyUnnf& zm3HdvVj+h}X0FSS9k`>Y{zS9#gVSkfM*jLlU&(!lOJ##e9G3z#UWpSd8))Iz4MW~2 zaU5kEOj(XGXRUBvi(z~1rGUF9#m~n^cQC7_LL2r)K?go+C+h&%UoLlC>8oyL)O~Gf zEdu;rs>PK}+Yq6an9qh}c%i*%jHg9GYAeoOUDAPSg;9GrW;wtfNOWZ54pV zkKj6YLMsxIlbOY(r6rQ?BT}Ercb~*O3;iEh z{WxyRrdU_F9lw%(E=}Vib(|BCAq<(CTfqf=ALD#vTaV+t_N+8$OcixQ^Yzqj4KP)C zIiU}?(%LpU0s9QFqOw#RPsO>`W}nc6OT3aTS@04Eb&UsI$&xGd{(%%33mUj!)g6)8La7Mg_C*DyEONyr2lLv9>J3ykCzJ({c5@jM z!}fd1lrgALsf%Giv#!LHZHkWt7ctX8owJVhnCfjB%P5%>KK#{BOESo?hao7=aesKL z%7Dei!9jcof-K*IQHCIhCmf`cl`zD+fuL>>(8>@n%boyyF4~?qW@>L~JJE1UgI=C> zaCD42nVBkDl@v}?s}Kw?7`V#quvB)E+<0EIkC3N|)v>!`^bI$z@^t-mO{JW6IWk??b8 z?0^~5If%)t={~7ARg1|6^_=HH+-ZAN zCvnmJ)QPA-^Gc14=4yz( zPf@mf+0IrKRvDjwZ+lj0ylz!6abho<|R21$`5o)jdB1yd7(6WTBuwh!` zUNpSsYu1+@71N&F@W|V`OI4oz5kkBRD3aaZ5D-CMnE95sR)C9H+Bps@k`e*oFTF)e z0Ex~5t-!XH?UI&xLpJ3S8LsBh`d*_wBz&G2qTKEE{#QN6Fb|Qu!aLr72<4yXg=6jt zo_I-&jr3a|jtgNCHaNChJHLWRN^+G+w>;?eL1y_t46XZlTv2-rOvPs6-i&+&8N>oo zIBhjALsM@d{;-Cy6X}y+*Gix>Vq4raCp6bR?h0>&zZ8&E9QH3eoNfX`mhXJa*q=W@@LBgPZ z*K%m|{i*pw9oxv5*NdmLb!*Ke38DIZRne!!!FjIq+hi%plEvRu{k$SNSej@s<)%a{M6I=Suxr~Z|^2o1hLs9SL)#{Ybp5?ER-M(7;u1-LWEZ*c3S z@`@VLI@UFs*&e8;Fx%$hg;BY#%3MNDGt9SH8W~;m=&L2|OqS)PZu(*cTx~SjR(#LR*n*ZOq9NHIQ_6!uNZ z-Ad_QtRc1AFqQ>YlKEw?56z=f+1R5@sW&dBlfYF~1NW44+e@^Spb(x~E}2AKa^=_>vnU_^~Z zM2Yea`hnea>slh$3b7|C9TjoqTTaHhFJ|!L+_?6rE-4PU`m*`zpSuhA9HRElao4%w z188{#RmU7|?KN?JM(}s{y2XynYI!_Y@H04MY-iTOKX1oJqZEK=yd(9NXR;$OB8Kb; zIB0TGMH9MQXrGme?;eexvbUpyM8BlWigAm)apH{Wo7rSjZ7`N2=G^AlN_}h^iu?fk zAE?7jQpLWd25~~QMh|0K^`YJ~EauJh=f#oa^WiS+dynK-DW=j*FF2VLCB(~ z)MK;6t$RQ%?Ll9?X4u^>I~(h_EmV>+M(6vFM)IE(lx!x941|Y{)tQ*hMr^{AVhk8q z4Q<0T_D5?bM;Ai-*_7pX(|%oF=0iz zr_}@EMf-Qo5QT0;`AMut=OMn^v-Z?jfkbIak;=0uHUI;LzR5D6=U9zfu_4=Vou+-A zK^ax5%VJ8K2CpcWM_`Ci*5eugfi&V^idRz zb0NRMyrOPv=jY?t5Y7R(kFb_bGe_iG5R0fwEekk<`KIZt>|%xX9qHx8;~4spCMS7e zWw$7`#Wr_|HLbpv zx}2`IIbx&>S4dx)=dR;@Xfh&sLR___#yP(+@vG7c+a-nRM7UHj!wIe@Y~<0nuaTp+ zXEIj{x6EwnwC92vtv}eBZ7JD_tkD?o79iGJ!!^G1!6LkxH{ZWxApNEqn}wiAe!jbw zq?BTe5Gx@kT+{M>=nh+@fSC=(n&sL}TLV^j<;jem@tJnO3w@ZFS|&f^St zxNK7sp4)J@$Go?D5-GTqCZZ~!^q8fMC!Jusf^N=qOvHmge8_q%p=TRa z>%0?YxN6`Ul=!ErG9LyhMP@o3fzr<<8`lcYDkc`N$41~_LLfjTk&S&#$Ih?oCTN8iX1LlO)TRa@7XN_hRi!`@;*{8QIr8N>v_ly}fL|*YWrEK_fx9KwF z8o1`ZfEUov-oM$_5(X1XHK;9H)Q7Q|8w!k(sh41L4hlyYzYkHgdd=b*Wvo19y22k31txY~IH)E6*dTbnkOx(`P!tZ`o5q zqjXF3P1#c^Q4EulD&L(Y%*V!)ccmPQhr_Ev@AL?_G$)1D^l?$-eiA>EZp5{b4?WU{ z2nin=RQ=zx4|M}#@LQfONdtCw_~5DQ=QO?}xa_1Q-p(`xt|l1-+Bw#zps{QocxOX0h%lyf@ba1tdor6anr4|3VE(4DI5u7CHhOt$&9Zuq#W zMmk8vfX0wuYbQxzSIKgl;jZiv8HkUg-{2Y?&oqdswNsHC^HfzscbBk#QjAt~(%MQG z7dtELfBgDto5Gzw;KuSL;00EGV@Rw>Lh<_X4l9<~yUStur0M(t)D^5-`ndqG8sRGG zD2B!amL7F5@*;wNqlHc1Ps^b+g3D#S7pILi$5YaxAkP4D%FfttJgq(wgHo~2JBEba z213DxC4gZ3g@;|*?2skmkEmmLEJ+ydI{k%boVzL4?dH z8Xy;H9;$jkT6C0DY+B6~P1fN$68Fo6(DeJ`P~~i<3~=}2%%Y0@^t(^81=ydhcrZ_t zbEIyK7IaGkx7aFL6Qy3GVF%2{&0?;86N-c9vxxS)3K(58P;_~Lq#ZJxr$)eX1vXE^ z$Yr(S5ITRKIy3P3d?2!>X`R2R%Lav+l=N&SCH>LJ?s>=g_QpkB6H3?bVRg>%7gayn z2}-u@ghYwV5T|g{puLL>2;z*x_x*W$T_~nwdNdDtbO)2QDD>Q8z)J_3ZM@cr7;nnMFOW|J4_EDk!S8YL zE2bpDcl&}r``kp7Rz9FB_JBH{Sb_vC)7VT2jEq0~)yOT`XI|N?!uxjIP3Y%Axn2#W z7|YG$jy)_9nUunAiIC4{&d2YpkLc9%JEmw#2)DAPPRn&QN-X$smRcralzZlhk!i)5 zWbjNBqHbr$SPfspepKhQ);Iz5(Aha~Rwj)}^4#Oi;|5AHBk{aUx1LJHjdzj3kHoP# zM3X+JGlT*yA^wL_IDhJrh&5u0C+$!OEhwX&x!GA<6W2N6VQshmB1Cg5Z#aW$T9a$a z#!e{IES~qh&u+q~FDM*aVEEVyrz-TF4@3tB>QNbo17)fQVX^KWC>h%ENp!~pqN@RBlg+J+wxY{f=0*&NnEiQrQR}BxQuHK2oblC+-`v+1Hk?fd z@b-HNubxV#KjTUP;D(JlwIC0R{K;L5y!Z2R!h3n{Ubl0%Ie{;UNb&hi!#JMo6nwdg zP=q26!qbZBC6(wQ8^!J~ABki~RdV}vB<=WY#AEz>gY@7h*#UBP#c7ahJ$IbcS;Gi=ZkuIw50f%=;Yu%=x0x7$iG@R0Jwz?lw+!&?mW?N$v1wT-pirp|N zU=_|L{DDn~H=>UiPnZoc%mub6NE6dH4njG~3iL^UyPRf;;R*cMY>?=+dw$6&7h9@~ z?q1AXK+k!DI_{;<&|v$v&g$r#F7Y$Ti+(AtOny$MNg?nfdG(^mLDQRVoj3keJ`fVu;*+hoFIG9 z-A&PIp2>DxXA5~;0Pn-zvga9GdrNvMejf#S4H!bx6#Jz3ZDS-&$#Kx1#z2XJ?Gb1? zZ5j>BHmz_u(=UF{acqkx$8a-nDb?eHKVh$AfF8uEtY7zzU9M}r9xOXRtZNqJN%h`sAGyjmSax`Rw7(x zS?=2K+B7EehdQr9X3LdxCt}K$b2E+dIBk^jn#F=>WJVR7TM2;cvW!N=Ehopwr(7Su zwTMg6@*77@-zk&XIPu}qh*Vl50xXAnXKMM^jvM(U}0=WDGz<4&0h*^g0U4IbN&2Iw9_KF zB!y!6W_^o*s+inGOuEThf?(Bqy`$27wyl7i`Y%zs4D-FK5vi7s593V-gWR`-|4oHI z+9m#@1-Sg5W}E-gobtbCn>809Abl#IC*h-+S4{(J+Z31aQ~kWzTos6s=8Vsig!|1E zrS<}}9;zu*^;2H-f_>Yep5Hrw&${M?MK&A-5c^}JFFeW$7Zt?p`iKiRYdj?H+fi6V z0;Jl55_Dj{R^>T?qvR+jAlGCatAQ{VdeO9Ntv#X1eW}4jY{cF&H+DmUolC>P=x*XR z`<6eA#GM6Bo_8TR(>d}W8P+@BVK-6^_qy(LR=}@YB@QiMb&gx`;riX?$J@8>-IpJq z=sB`gRF4f(O!m8DF%}g0nckM6mHz8AZ;P>AfY83}X{ZaTgR3qy6EX|H(56>3=QlWC zvo{u8)I~!+hue;2v9@U4ZE_k;2uNqv@C!4C+8e&S_Ltn=QI+#j6X)DFku@Cy6O5eV zz-~2B&zb!A)6Ub>v3`5c7LggTr<=|}TYCa9HXexKaYsLudaRS8Xsy^3$RF0rn9Af( zm*;m)`J*woSm{;V%u))j#rzF2h+{S)b(RnYwa=+C0Z7N#0Me^Wz3HJWT5S?EiEBD$ zFfrA3ZblJS(ec{sg77s)U_mj@OQRVN1s9?CVz=7s-dO)2lo}vsH^= zX>f|iv)!BMzZma(bDuZV$2N?#;tn8BQG=v_9lzw*foJEJUNF(R_QVe$E~il|;fz&X zhyzlzceyZVw0c`&0lrD9fkwFur^w<6wy|3}rfGp7vq1VD%A07YbI;vLS9?E!brErk z$`2r?M+p~82Jm-0fMBBuJR%Bguov~L4o&74bp*s{s2xMpsUmhtSyvaAKC9-)@_$+9 z^3_PvyqL2+;VZS%F98Y_8{)N{(NQ0b_gO_ee`wPpK-Ayi>*wvvCk`{>rgt z6)*{Qf^m(!!706JvC!qef9O0CxBUwl32+VIH0s+1IwTz^;vdBI<<8yPGHq*{Jz6Zh zBZYQJ2RMa16W0G4ct1|Px~e%sy_~&v#er@G2ZT=eu@M}(QyZaBp&vv8Y$&$6z!~B2 zW?PlwGq-B#v#j2FBlHoib(ONnO=C6FNs!Kj!P7wy5F^|Ea}`X_{zh;H|B_jW)2xx< zaAeWH9q|)em2ZFZo6-(NAmq`8yp1^s)$8BzV1(oj<=*;;-}>ZJzu9M9n<`DX+>Wql zhbAAc0zqpZyh;;(oS-y*lv$<^IWItKjL zj#m5CM*270ZV7Yy(oz^w*@h`$5uR_136adX#-L3{AI$hGmq>=QtoF%sO^AThwIAGr z%!*G1k~lU9U0l0hr1vAQTxgGDV?T^6 zh;A2pj5#7o`^hF7d{mObiTAf-zS=~3bQSIz{u}|WR=^?w(S~m}uyCCiWarVylbD|H z4rw|gQ^R7?2y;X*?gYtgaX~tv${(-j+>_l`2HNo9;QpP?s^O$p zKPX+|?2tp1aH`bwGJ9ekAy~#-Ii;zj>yj_*SEZd%OZV0jJ%E?S1>A$uYd*r*^fYD| zvBaC@Qo0M@z!x+&MbOk*<9Yf}-gy*EtwycKCNcD_xfIt>-r{!}B|-*HK?mljD#uPa zpsc9^LT@iE1gnq=3cr5Wrdr6#m~$TS*`2||2R%ZScKVFJS7C&_F-`Y)`8<83s*)fUo8z!co| z99gpc(yZl0Q#U=#^9>>w%I-;$F)9)0z?>XFY6V9%64crrl4bJ?obJ*Hn+Bb3w`#&f ztzoXlRH@@6#nRInbiBWM)g49pIHeICOA$Fhl;jJwC)IT}6zOBgRc-p#cCyKYPJ_cW z+gpjS+lM(k8LJ$%S4VzRlI-ob1O%#VU0NH6Oa!b!rL(b|VXeTnJuL`5bzQpB=?Gio z@$n8Zl0%xG(|BqCdo2t+v26}0w!%cm=R$ZwbLBvz5eFCYb=JVn2 z4o*ydy+#UDeLLF-JM4_dxSU(C8$!e)t{f+yXq)${SQ>M z`$o_sstsU)&GVU(7eBd-UuDnk>7eOOydyOtnXph&q5S~B_ieC-H~Ay&KMo`Dl;sY(hs;J7!vP{F!KPpAOKodm*v_S zY@67)TiM2eE^NYI|9lawB=3*Dx?mLS!t;}97*2&bW zU{V~)P%IiRR)DDoz-s+~Gm&@_k@C8hTLhyx9M0e^23R%`pAN7zn>`18Ta;2AY*Vp| z{@Vdj&~phRW7vmlhkax>EsTVV)dsd%^(ZAjDilA89_>FsvzY8FcB+-yPb2EL4T&1I zE%nRE?y>%M0+mcRAARENUy&N)%*h+Ngk@KGg1{rC5{tzEw(UqYjICJSdO=jvRo&m! zuS9HD@ISAq|AUdc|7q0uFGlYEd(?T{Q3LhYeONZlW}=XzDQrtK(zxeWbQoyT7cMC^ z-Vy8M!BcLm+U$n8I~X5FH;3Zzyxyb6_*Ii`s!DXbfrmTjFTWB^(kL8u63&}fdtKp; zVmE}t7kYO`?&cynWOXrG9q%}|*MsVPQd)3N3L@EMIVO(pZ7pr&{2?5Jtd^7dd?B9y zvc7Pv>iy8HjW1{4BDyUta+9`11Se4u#;KiUX=^xw%Hny;e>sY_%hiAY6^1-gHmc|T znyqA?8CN=I9WKMZ0KcAuwc={`tgH!fUtGHjIkk zF(v?G(~KRogo3-Soioy0W9MGr$rYB|QE|r9DcO#ApaQtJAPL=a=W3{gvN>u~MPMpo*Dd8qsm z5N1ef7U>t}S8=8a-BT7F%-gCD7q1!4?E~|PyZm7#wPllAJlQZ~t8kn^vE}ECu$1ld z^lz*p-qz;8Mhyp7*o=IK&|BWPVWcbWnl9&IleH#i5Z~m15Gys55pY;B=z=_Njy?N< z;YqL3qx&(%V`Z|mqM9?&*S(K0*;VEks0=6d^aIKkmp(9Y$E_AgK+fcybHffJH687A z8lI3{xzz1NjPs{`?#)bpxum_RVjdzeq@UQnMq)6ymEmWUAhcvMQGk@p|L_8)%6C{T zsHmeT*l=bz%lt%Un)ve;xj>by8`IxMsx8KEay@O{_GWN`i+3^3q(mvm=V2xrqo8r9 z%^OYE_G0WfoSRp_f(={^;d}mT?WpCCoRdEw;+xGX&6`++KaH`}yMlCk{!uOF`(1%n z1}iHA))$8wFTQ3QK?F_yrT#!`iA$#R-g}!j{$A!h={K(nIr_kyQW?YxT$*VrS=et@ zp*R8xY#YL)g(chJ_R*2=Kvo~|)6?|y++Dbd9kzS^MUa$qXThdE&e8s!&SBs7cfWTOHl}=kMRyfjo|W zYz!=(3lv|+P;)U?GmnU739r|0yh}>O%7lI1aMuZQn7jIS?t;u1a{`rQoYKa{&%egT zinmYH{Mgh^8aH=XY`#>@_&dY|0095BGV>pdyb4E=uks9srwvzmhMj-HIZ~jde;aUi z{>C}ac%?SBF`?@S|D1Ps>|OwSZ{^5Tt4ZG0K-^45w#x-XpxUcSZ!H?nrZUa?Z4`w?7woyxeC0Z6h`}2jf{iCDy8v+<_C^ishn& zvT?(Z#e;++0OZv?Jbkb$nuCRB4t!6hYkRXDXzeQTCy2(s6x<=11Z6a^yWtdT>rAjS z!?a`$4l55kwSp_x2^C{moa(cy63=NDYrg8>jDs#E&Mx8P6OR#9zqYk{Iw zZNx(9=Kh>ZQ&YZ~upKgGpc~u5{$W`1H?priJh1$r5;P} zjO|XviTkB!DoYJdTr5;*#(Mj56&WXPwX;Qo9#RiS{azXo4*? zY}stw!<4@470t<0FG_i*hyJpLdCB6EIB7dXFc$-vcY+C90Rpj+iPhLZqrf`>o`j=YioiIr8>w$t z-CLMdNj9E2V>PjcLtBS6={&Y7CM|d-;fC&TYp)*5vV-;Q#_)26`9a`zc@?eXeq{nM z=^Z09StQS4x<#U@RNdT7JbqWruhtj^H0g=Xf1|&^smadjcV4ev_ngO7jvVW+Qs%Ds zBeDhEzUY74F5D~70tRnEJMeA3C_b%Qh{pZTXfpjDHXr2F0BO#Y)kkJO-eR*6h?3QM2A)0Hx=6;e1+DVmhvUW#<}^tE7PCG%PX4hPNfCu z-e0}g4?7Wmj>u;{(mHGvbnd82WxFO98!~@pBFBiW9|s$!xmj&8?C9O$UOBXAc2@g|@IBjPM?6))aj}k#_XDsB{zu#EWu@s8 zSQ)xtc#7Mwyd?CLizHpP?s?$146KnA5;_XhFPgM@{qyV+DrIS!?R4eTXfVvNlU6RX z7w~?nhF2!(F4Kku8J+j3=Rt(vh@)TB-}=h#gIygKL)J^QQo4mhP`l;Cs~6ZXeoKz8 zsnsU%LkgETJwUQ;C_}wXp(Utifs|iZ&<)k&_Rii5O($q|=?5-wOdSfP^IkCmtX$S( zrS0cFDt|ioeABDMzzB;QE9cILW87bY%^^aMTScA)v9obmsTW4W=MST!V99!`09=p+ z!a&*^SokHBYS2ov)fsm^ z3t&!KB3V3aK)mr3z~X|Q{~i3yD!?8_9#ntE`Kj$*4ZsCG&)TDU@W2@E zGY}DH2xolzIn~lkws7B1vuv!EkHgF0cjM-$v|mhB1=bSrMTq^TZteH3>6uuiO{&Q0 zyYNn{_W2#n)E7e4Dtek|Lm=WS6H1?ZMM{eAlpy;?*)UO)26OoRXnUocxqPP~9;P6| z2v7mtk>MvrdsD?QwAbqRfn4?m-oW}>UMTys;iDSuiuxwjn7x>!{Dt%|fM^OJwQ9=G z0&KW7VEAs&Ug5e-7hF4eHcw1ABKxP;l0XdX z6|C!4Xpf=mtOs<7>|`iwQV=0b?upM{cH=7=n9Fuh`W^Fh981@gnQNW+7Pu4f@TxW0 z`o2&uhmM%QyLbAf%V?9u2X-X^%Do$B4^nJaFVo2Kn1-FVf@4GYtugE`#j@r69p$rS znJs(8b69%5cJ_isN}`Fy{|X2Y#NGR!x|v)!Y5(?&$d9G%YfoR~9~zE%C@w~y+vaPo|dqoLd+ zn-mxM4Lll)ON^9T>SZtw39^j+!q50XjYm3hYB5-@1(U4a-jZ&}^fe;C25$sFl9iB_ zeU7>(?USHQVRVkCG!c1qGo=fXr33^_s-9&MIq;h9>G2Yw2OH**M#Q*10o(scCo?VR z{|;`Tv4TS&{FwQhv$axJCWFc@1L^DmJHB}{w3hL7aOP4AO-t9LWIDZ~Jt)-y>&0() z!*HqH)i~u5Aa!bkTR6$Vb90TydO)~)>G$1-GF;hS^sY}9i$8QggC2>sn4i=0J>dne z)J%#Sy5(RM6b0&xaF42Dl4ak2h}7d6$L0}%Q_Yq=uhz|-cfoj+>7lSzBWSm|&L8ST zS9-JeXbB-wG{wS9_zH^WbPHctwW!CWR{kzg@m4k|nc4rCPQGQF$uE#|Jzz^{ZEMUHz2-Asps9mf)Bb>!o0vX5iVdk>pfTt9d zpJPIn@Q71v|G{{($gqwfS@q6)dj}v=AsuZ$(@_{x%yojPTV-an3iHHRDyP=6+n|1a zcg$(BNTi816of9+cBT3~2Ntv!%6BwF>xPT0U#1L)rFwMK>Ef)hU(DNTDNQ4KtIf^% zla31D9$xpaDorqJL)+aa`^7KgMZZh%obbsFPI5A2jlBdFUQ&}8;P#-}2nz}9>y|HT z($MFrW0xam^eQe6+#$UISWA^(rhNrToXd*wBJEjXZ9ku(t|_Ds+kj2@D~eM`i&RIu zn`Q$l_&u*JMl|ixk2s%lYW9RXRWlGMUNZ4LpFXALM_hw&zg|6|z;%NpbItVI>pJ8j z+?-?wY-^4*j(}Pl0SpVlwQD0Exw%;tt4odGBzY|gQ(LlzVQ5AUl5-woOf`6m*tcM( zq}SgYLG-Q0(F|<62WAS--rL|-SVQk$pok_y?sMIHd+4|=`@MIL<~EN$6%il7?vLO<1C$!=s`TGBHaO=BstPMtxvv;` zmatk$z?NwzYlbA=nuVk~`c=n5rk!|jcSwxmDk6%WjTmO2@Q}!iYiO^lDm>xFUN!YO zA(dBWK=o2e@6%#+IIpZ$X8bx+USlHtzop~#0{;Q*AA$t~|D+uM4HgXitsI7;pXYbU zEC3c2y=o@)S+i;%8wxWt^m|pY(rw{NX=E$ofK=G|4nDXdg#*3teZBvg>FcTnA2Y62 z>3N%|?xK`Fz?2{8rIGYSbdYA2N2&zZ?@~(#f4SpD5t|Nknt_J96?6wFiG#77)Szvs zjZVsIgc&_LMC7c=Uee2R+^Of<1iRG~OYau}8SfjAqexD|gWdA10$PVB3UxM>Id$>_ z)j@?&vFAo}%wPSTb;Mdm+cRX;WeZtWRGFFZ{1^THJig6GnVqz+Mnprm4)zQb7u_N| ziqC6gva{|my``s6WIN@9TiA~!m5kl(|0tdBIgL$uUuaRZ>yfSXzJ-69KPkz3dAbIk znNcHVi}eAuLq3I8a`SPL5>D<;1(^+9QY(E>h8$=}Y6hbm&%&iKFJzhC0pPV~Z{%eB z$nSmZ3SH1XVLPH0r~e@tjVd4W$Xn8Obems|3d~1}EFKG~LB-)AG?MdkvDXsk^jyF( z^8Bx(+Sf4ibD|1%X`Uz%MJ|kk ze1Kz9q(6!NX3Ze!%+p8q)x~1jp7?dk5b9m!Pf8tG#&Im5^zwFgzvwO?f!)P93s?-n za2&#wFKiGjl~O--al@c^>jv2SDkgz#Yh0B&`}^nxx8oqDgE*Bb8G$)$h38a&q>JOI zj;V(rlVFH%_X=5%tBt=uoR2!jezG(blt;Si2Uj$BM$mhEH=K4S&{yd9)4F6q-F0|= zP`_X78k>Fm+yL4zCD^M0r-1PYFqOkM(}JfZZdjkVD0VO_3??e~&hcEGP7Vt+XQtRz z`Tz2FR4-M4>*+wLZ(E_gj^3Q&Q-4j}$#c(QP&s#^t>h8Xhx!vUa~r*9xRqSzeD%w+ zfg6N@;_OWlE zHiDYZCjUz>(;mjgdN+;qpg!1GtVH$2fTJZ=%g==B!oL^^VeHvt5fDR_+z1KNSq>}J zZc`0gKBdPd?3XN9SKxJNcd;pmilyoLpcHlw|0nzwle&6>sxHW(=-qm&V-uo|R3R>~ zh-AP#2UcuO5xpMJQQ z{WqALu`LuAzfDXm);#?LTajTGgMK$^kh(45c0@fjuYN>2=hrV*Q8-xEW9lT5>HbYEVc^>{-TPfA;)b`HeM?|V)g>7FerK>yO~7ev z=gnZp^OrP;D42wf%&f8Kg5)t_>5zeoKZD>RQlP8uHMD>pxJC(0OXIe3{3=}oN9N&gjHWG$ckNcvRhPU2Lem@uL;z2+ zX#IfA4iBQ-*4Pkx&ept_oEwAFie<;5#MWFN!WKGBpNz2SNwr>(Xb^3Xh$AgSsfBsy~|io`FS)0 z9|Uv1?Np4B+nxSymm0}D=2cLkAuju_;_ZfTcQJVu{sAC&$EdRKC-1$~YWK)Q+Mjqg zBv`iupIN#eAa20SJGDddk&D!#YZY`zC}&SULV?X$0&Ad4UOOC!k~E7QDyi zE%QOv4n6Az+H0OU){T6qRfpP?G_Xi3l=<60#aNwP*1knBSOTn&_%HG}88jQ$MI3f< z86xlSL&(0}k7muy-4l&=_by{--EZAF8jFc#Vvihl)Lsv2IhZ|aFDI`EOz@Yy47?!E z)8F4LmLWFp!#RSt>UgAHG?u^33AVY7#do69eQb4e7mG~M%00v#mw%0-6_~12&RvnB z0U0)wDy&IsL_(u^j!VPdo%MA}%1W(7k+V;db#*_1(pxBNwbLhL?C2F$bmk9XUS z8`}1i8+TQc&6iM7qn76uYuD0xG`WvY>u_M)K`Cqm9^_LuB_>^OUE890W&_*%e5&w$XDOYE%36M!DTiB5FW7 zTEh3EZjK{ZRnAJs8_9*GC@#bk%5g2&x6erZu|ByvtU*HDwbC{RN;V^n7wMl zqf5D&i!91w3f6^a&5cS_Lps~O8=eh0tae#q{wS+dOd_!MT zJ;?2tJ(qg+;VnqSZFp9o;@5>RbFF7b{Vzw?no;x5us?V-=dh2qHVZPjy$Hu)dg}Yf zomjs{Je@%Hs!&)(j+Y!#UG=%GhwwhP<}gIDDQHps&vH-acWQLZrfB(l%)&~`S?qRo zz14@1H^CHrdOQEGtn@F5_}cLo+Hw;U6LJ~;?g(h`@eA0sf>WhrF@YCU55=19olfI4 zWx-~&+D=W#)H^X(wu%;IKDoL&5({As5xre65f~OFx?r@Jq#o*@ws^$O7?VaK#r?G? zUWoJzbzCuQch*UR^`DN->d^MnFyi>fn~hTbHKdppznaT{XP!U14ON67p zY953ph@C`dvC;PsXC=TPAOZJ$R9*`@sGw_-i!(U7miHN#)!?g z2G)3=3Q678ZFq3A&qBAi|6fGqN2gpbbO&P#GF1ZMua@F$76U@)bNqw9#Qy7LDu*8p z(%uB}anj#nUAISWVyAzsThkqwJ|pKdO)`<+`j#MQGIvB!=DsQ?*Ej^Lq_-Fj^!*@J zZVg16dKuY-ZeP|=LY=2Fzf*Og{pNeH|A(_Tk4idk-^bY`Kr}!wL@Pu@a0?aJY92r| zz$H{%QftI5Ff}#XYeHO7TtZyJr9xbCPfg97n#ToAP0O|1YRs&$ZJx54nP&U_oaggA z=XXBm%=w-3{qo2APyV~^>%Q*SeP5SYV)50F`II6Puae8_E+-ayLxO)cb(_tD54R}) zIN`Q=yv|DXJ*1Ex(MI6!c!+XC>sL%>iVWupkte$+Nt8c&srTshSOCFCh z#~+pq1G5u@4FC>y9&0v{$fmqNyJ$klFM^w#&%NPrvqww&_hoGQWrRLT{?!HST9c$) zsx05t{+YV+pxpSC$7p&8A?rt~78%4k$fiisefRC=x6{4gg>6%PQG067`DGzX@IluO z7NHE~@A<(y*2fN>1LX%~OWXZ}tyX+mC35?<(C2lwK;U@}6jKLoIaud>pT72cR!iM3xq+o@@Sh7mJ(v0S9(slB zEpuy?dcA(-R|h#zp7^Q%01s4oe1R?h3jBNmub&=waK&N6*cWN1Pj>gpB6h$pVbf;Etebk|J)|9J2P$`N#W%3KT;ul}bq2{y7%=Q-*_4z3IN>dqoifQ_7&$Vqg6395wZt)KYFj z2LxFkp+m3LYFF4B-ff4TJhoV|t_mEWOJ5x#v7j>~#KA$G`}Waxv3;s^YVP8*PN&e! z^4)th6@t0O-%Z1$+ac3;z9<@KojCYm*9)4a6$k%FHwYXsS!iB$?M2Dwp=3<;(8Ze* zJy|s+^66iRd-GOy@{qxQV>GcJFlc|IjB3vBYS4FMHzhv+)Vvx*zWMrHwT! zzmClYMnbOuQyo6q1%Sc(=Ky7S09A36xm3U;Rj-b`-%)w}eoQJ#G3Nn6m35cb%R9!# za!$aBR0e6I$7~TyKquL@?r7z=t&|TPkw^dpl^vjPKOYiS^d*@9v_KM*Hb(;ulR3tlgcKzIP;k~rJ z-#y30@1qcaJdZW7`ld=Z!J9a;=V9#n#vNtI z9p3Y@)x6zRF*6!@P^f`4`Pk&AwjuoI4iWy2fgR@YN^kk|@^%CMo~qVq=mwf9-rCl) zG7;WBQwGIygK{#j2{*uQNu1G(avu~w&>wJ0mg#-};>9KKa31o_#zH=c(mqWeAC-9z zm055Y_Qo^1h`isN&Ppj$&NAxPxgP+B2gejZ!L0wfiR@r(%+Kbbc8a;na%05aUBhLT zTM7$YLi!6OSPp7+=G3T9_SyXKM#RBhUsg7gXks(~Mpsdf_{4O44|o{@7e(?#x9F!` z51Mz08wMz8X-BI*MHUx+AJxx{Eo+nLh-n>K|6H9M!0z(SU~<(bEYNB`A^!IQjxYD# z%f6rv&$hIfXE36{BWUkZ;{IoR6tBxjF0J=65wkAM%mCBdBF~2ZuyK6$xT5myDuE_J zk3U)KeuHf|>jHq7UCp{#thi&G> zd|2WAXZb?It9oo$6X!DXch>Y(ovAj|X}%X0NEEl>yL_6r4F>i&?UZpJpMOS~5Zv#4 z5uRuLvg@~iy$zP2^;w6&?T|Gn6nRj*ETI|Oc~@6m>xt_5)Vb|P^UcF#z=1jsLmX0M z{W!*f=(5ENYj{&T_Z`Z8T4c@c0*FC%E<|p4S^Nv@PD5Z`gzM)!whztB9IS$(O?GX} zI~=I!4T$3A$C8TPuBpm14v)~OKX}d7aHxj!X97JY2Mq^6`b}Y;ozHzv(-*}K#)juG zyNWnBVKQSxEblNYRU9CUs`T3Poy*9HeN-fU`?%jU0M2SjoYQUAFX|6jvnR}+vcrQ< zT^ZICef{AUnHu5XU*B)T)r$1(* z`GMgEHr|Js;qm;W1}?cgJzC`R>Cj?dam*Dzqt3P*(zCXRf5wtd7W;hHpJ3G#A|vl# z*FO6aQ&C`SnY~x~4?^<3-dsauWQaI;rm_dI;9MDi@mPe@9s$Hd~4g;W)pGUiDzTRc2^?JqGn~8(u)iHv+?SJ>Nv9|pY z=vtYRlC7d&4*29W$Ztw0Fhk`A zv|w>m+4$=>$6PVqmq%T5bgqnTKq5kD1N)Mjc?Uc0ynoU0>Qu#y=v7l1H|$cY%E0*SY?7d7C7x<+PR}Uxd~z0&Eznt&}1(;NWQZa-C1yBAx1eJg~Z)6@p|FTcO4kZ z`4#gid}hpU9$}o|3)2%X5Ex@Y^yue^_6Mcjzl3_+X_rH2^MQx=n`_S^0j&ouZ(M9E z6f1pV7J4(BNBw5sq!afDo8l^T-Mi*xTBzxGMT&mEM?EJK>8zA%ZrfXM!L(7Bj(?B+5jKQ1Y}T{RV~24 zCU(bjo1!xrN`2Z7MrzgOwtzVtIu^IX?&#THfgY_@pVu(2v=ezpJ5plRK8)7#9ICd=T(2P7lM{M!VvFIteORx!g|`H(t3k zn=zH=d^bpvZpo58JE-=qweqOiU}*Hj}|xh<|(rvb1?h2)v-fE7PRs? zqHySu-^VU8X8%gF!H(RbM;^l!0bjJRrQ;=4?@aWvnDqo4Qh{;wbNlQ~lefjWtk*5J zG@Xs_#4E4IZ}}tK3cU29v>xADyWQsY+A@M$%PZC(Dw?w`iwi&VTiy-thx4bL z>|SmQwBPIW!0%kVelhV-O5M4juk=G+Pl9(_?)LlP;;;f*-v4kdDbiOF@Gc)rd_|${ zg5}bA=Jy*~@8^=d#wyzH3!RGE@=EJWwmDVix88KWC+?3n5$$sM&4yq)`}idV zfv^g@_AJLZU!R$hE&gqPQU(8P~>&_`L}Hh z$M0)jzgy(q4o>}<47Ul2m~qCr#<}p>-M}!!ws%?5EBUNUL;s;qTIRLLkYCJ>*G?Q@ z8>ATCx!Xj2`7sU$9u?hg++N-Ib!*+G2AUxj>HS}wJwlW~_K>&#pjT75Hw7TNm!M|N z*CYJNcn?^7=z*ZFC-VicjKH6}PIYA`M0l86XS-`-W4ip4Y1$O>nF-Py%_q?7kP}$r zwXs4?abGicuQ*_y%fd)$U%rWazp0!u)2c?C$`C(WnjPp~{r$9Y&(4#{VLoP}!H{(W zojUyn96NGB!8~MOp_EDS<2>CIBfHw9;nkrpS-{Krb2X~o+tsM`H8;PW_?)34>+6Du zQ>w73zLCu0*=NMOz_eY)OXmUy5TlrcKNuC%mi{TiZsXu&;un7_^~z@>?rHwMYmvjr zrHWjIW(4ky^3LCqj61iz z+^3*t{wY!f0i<~A#gM($&aVH?vGuI~iA(Ry{Jtbok&W35qdqYB73#t2oJf6KSQ_)J z29XY0ex>jtoGr*-{zcYAvmdjl!yErA0=@Dt^?OPEKL_l)G7}<1yMlwpbrMU28=T3^ z(8i2i0qbZ^)#zc+pDrmaeqQb_wU9$PBQGJ_OD~i36z5HosL_*@mnwVZ`{Ldg;$wFn zet2o)Vss%#{h-gTm63})8HxyTrL1aj@&4ERr*FP#&)|Ohq;=rPqSKRWrv0)Ng-I>0 ze)Ry@rh4c-p|ju??iPXhPT^B= zZdOz{$6M#YsCj&NIOwW!Xb{^K@z3&OO+E}_M^J_z+VpNzx#;JK0fg5$wQ6}ntREFm z#@&_PeU~ugWjT~HZMiXeqQJia{oIWJ41Q(VV7Ux(EZQhuJYzt*s1I$VRVZ4f7pH$T zxm}JhOCA4iz+y0qZ{2d)6c}^9B>X0GfcVF>?Zpq;Ho!l`TfctRV1 z=aJZyi}0NuW!wKB``t}IB4sho3^N#!YI^T_ti7e8WKY^SPcvxCGO7g}Z@^#Ts1grP z?J$%|XGj`OxyiH1slNR3&8P_G9oH>}fg6+Uct&M9F?%;ntgjta)Px{Sa`&CY1vZ8? z!AjdQB4?zJ?)|c+z&z__Mb&u*Ne4ksP5qvNcD-$!65hRzi`g3!n&m$wTKyxM@-L|c zvickM@kVFXpHSiz@3;2-;jAIWF^KZ2Z)svrp7~+N$C=buA=GnR@v$cNzCGKP!1M&D z9VHWKbpMRvc3*!k_!k(pDf8R=)a@Z#Ch<$uod*qHr&3*tKvt@sI|YG?)d!(^!Nx?? z9C-Yy(t=iE@gdayidBhOSWx$pJG{p$LS>TIa;E5QdDzxI4U#ZK4lWpDqk9n?ofE$^ zdkYV1$j`-GueBOef%zacu(a@wpDE>C@DS)<)XTN^QqIZ!Grgu6ajBMtEE#$C+u}pk z|9-Rnh=+LnC;8-mt;^N_ohth8x?KHVs;J)k6O?eTgIkSm-s_pbJsQ6;5{Uo|Pq_6V zV)K45TkHBG(((YrYrwqwiMi}@V;=6oqyx}icudD){`#}?T78){hTzy5_8wlMw72yS z-ipn&^F$ZC8({pAMVHPzn|3SOBVK`yrR9$Wwqt(EuBT+BI9SMy@Mm(LPfP3BKM zS981q6BFM~`1b_Z#pU!eoQ)LR^<}lc9D3ZKlFgs{no@GpHJ~g>2dfCm%NXwXW$sVU zpvk#`+LzRwKj??+J-`<3h|%f&F(eeHeeo{H4OFxi;T{5Uq_mKq`$^E3WYG>Gr<}4N zRrc~041PelXqj>4&AVcFsvlnL(|In#b@CbP`FT*b?;B>jnP0=^yM}7SX?K*@L+AER z=*qsWIgC%OeU?RI`NHSA~T$E3CQ+T6)f=I9aKa;#kHJ!P1O?ora?)Cjim4Jc84E^GmNE-9AO4&}^ z z52jna31+tVW&6xp{Yp9o0vuF|TN+NOVqZ~Td_yvz9-h|m(ruo7a3H}h_i+U^3kHey zV!fPUxs~+}ZQ&0@xPA})u%?Okav$m}wDt`a)c_=~Rx~yKFH}nqA|OB*u+h>?Gzk2C zeD4lU*<4`2SKYnqfM~RHtyMzAmoeETJJfVVhU>TA20w^?aTlc<;0RxL6V@i>wZF!k zM^)5UVT^kl{7I+YQ%$ynkV&9_eRFw@bP6VF0i@k|cNo;bXo27Qi9l|+=pVJP zfq);s+Fs^#M_ea1EzNI)Os85*HC@j?L$E3!z3VM@W30BF`Ze`5g`2qT%sH`nFPoF6 zdm`3G4_uFsH7%Q)60FOve(ipoYz^AExauG9F^Trfh~j^QD%!q2Oo~%#OW#JJfBN&x z*6=3sm-CVT5bXcC-PXZ#`)8R?x2E2z`gyqayE5z7MYBH4#+QKf+_pQ~VXaX_s2g)y zSG5Ay3yuR`ZufxrU#a@5f?SLAU;fX!-2ZY6m;aqO`foUa{4a6T_#}p34!j*y(4kq; zi3u7w<+UeGC(YE;@@|)4>4*9QZnPlaB{lhl=ptJJtn!jqWxg9d&{E&&6aVhFF<_DF zX49}Yx=PR5vN=v$f5@Y%mrLx=Pu^;rb*A9?)?DqN46c()o0+`>)lYxasr}AZf z8I?@Bp%<-*-{$QpkC+L>?a$eh)v>~cQNmXIcYk8wINAM-(o7wjxVX^0e|lb7Qj%m?)-%ZoTk`X_BBbvVCExCa`GGys7RC9gQaxHQ!=K=M14J6OaiRRH zklJ~5nCH{?8iJ%DmznLWEU{`P#O=#_rx|lyO5YT8IO;=)o%h~|xmX1ecnJ zbJGnh*U?F7Did{1;D`>@4>S8rG3M9yr~Buftqv~g8{doT?t5Hj8gC!`Y-n-}kR|y3 zr&I4taV9I5qR(SV7lOh*Qu=Te4Qs?lj?b9a0?<`CLb>l)UC0xdxC{I7R(#jh?^i~; z-hzTW?4m+HV6HErf5`MU%^XN*D#CSB-EcXmzE_Q7VJoKu`UO`T3-DS{->zctO==9W zc3kvpV83X`4uUkwDr03;(tO_ z)C2b4S%mw-c}PQDQ>)_T4Aq|IYP#=n&kA&7R22lt&f5hsE38o#m=3c$+rCRXb0kl}+@zynmb4F=uy?<>fF;SGG}5#N?7mWam&9pxM;Gx$$~(-76GFk*4dRxf8mL zW>++_cxE-@z7B**A1*62&eQ37`trqO+@GEk6*FJlw~hkhBY0*D@! ztPP;-!>~~*UMF8}GPte-01#0G-hUj!|JMY!^6z?;|4wi#|LRq!Yv$?mMqkbP|MXjG zYJ}Qu19ns4)>+EfCe*2Q9D*m+{Hit$%f07tJk&9NzQ0CiTjh4y#}~+B;g&BQ%LLae zlB$!oBXkUA{h|Vz0ZQ;@8g+c4kI&%kvCkmagqy-IOk&iH7r_L=6L3=pFUXAtAz%oCe(#Ah%znGQok=(W^=e_D-g|H>!eS5rn@WPD9ikq2_F#%$Aqd~ z1~UPvI`wAx!|z#nbK{4L4DtrOd=rw~!-F&SJ5KA6i~G`FziNlpkYB$wt;vxW5oLWy zhd!(Md$A+s;VqYeYrKqH6mtkYPP`j2>r&a2c34-plQYp%7lnLC10<*3dMYt}lhEC} zcDZ_Gtu$CJWw_A>F7Tmpn0`Q1#r~H+8yJJXyX><<6H0{pvZCiEnD7Op&gd?s4 z08e|XZCSnB)5_K6#3tP7+gq=%eB7`OPn*63T(kcMs;4dPPnHSe=*T7$>eZdc&Yte- zpj#CmODnk4`r-${l@EYtwI7eYw;33spFG_B6DaiFgmH9MgGXElcA(kAS_fxFpgi`O#mQZ#}<%CCD0I7N=w{ww<4 z!lqX_)7~N$t+v)$yVx4sU{=Z~ubjzUMb~{*{mG(X18R18sK>a)(7Iaq>D{@z-{%bB z{By|Hr%tRqKc%-T@<LYFdI@$f zOUBooklfNmv*_9-Gp*%Nz1HmEuFjjDo90_nHE!&-GP<=OCIe0D3Dc9=eMMUV@8cX) zR0_?dkLqputN<@*$Ddkztfq`qp}sY*o=ovG+NNph=d8lEgs8jTbM=ivG3sCDv4&2& zPCVXjXyg&gah0Dgq$#~Wjo?ZG-cZZzHl#s1jj=9q%{au^OZK7sF_im8^#qj<@N@YH z@!4Nyn~rW_*KD1lwvEqAq#juz7ZU+q&85*syC7B7+Hq2QuIHun45q(V#4ZWo7I&iIq`EEe z&i$a(jHYuvHUpA-5ijd&B*7GzV`S?+NTaZr5lk>l?>SX)uTN|bbQ!rU z$uP&Hb%ICVw})&^kf1dVG1Vn_-Np);NHr*ZzvAjM*W`z$`?}{D_%*-o!gq&1w|IuN z3L5BMU-3PrlbDtfWnFba&%^~Xla?;a&gNFtl|Gzl@<-F9z3Y*cI&OUT9nble$m^E` zF^*<0g?gkeJaOUN<&nb_{EZ}tVWIKox$f;xbn^l*Fx_jXZu)(;gqb;%4Z^@C@+~>* z88rvrkKC(92YFxzI?@jIV6J*4Kg%M4Z{$DI6kK|O^B7uRK6iJUinnIlUs&Vu$$&#* zJ#BDxtjfOVU-0Z(+tZUEj535_5&3%m8yZ|yX5$zBKCPf=tJ>(}SJsD={l`LNtg2=W z(-nViXd-yL=4Fb$XZ?n8vJsPf@u!M>+o>j9rMT78jW{yKkHd34T681qy)IJ?e(NpW zGpAnvwi47ti8=x^nY#mC%}fd$hJ76t)u;SPN#3DyYab}+#})_Fb{FTyzx{A-yZLu7 z|J6ADg4>fttFL!NJ&m_kA>|=mJF{*Zh<1<9u{Zhk@8_er-_&vjfK+A=h1(h9$rv3o;ba1&>_MI76(RcT*^ z^{w#Vm4zitC|OF2s&_WBN$q@>9q>S1;l)P#Z?f{9-iQRSO4}>!nB$aSA}Nt%5ZJao zErYgxChjKg@$OWS?F3zt;=A{Q{k1r#&{SInBHpJNc;RiG)AS@&wh_(KclP8L9bX@FJDfabRR!T(R~z340fS1N9_jB%c~YNA zn@t57LD^f$y z*`fH+7Bj{>7PsGco+mx9Yk%KcG%(Hz+qlh{%_BTc@P?FT4{KhGM6I6rPE;&~ZQLeI z0aK*(KYDj=8prvrTm!@l`*SHa{8s zGUsRKSvk48vC$Wc{srD@+Hv1%jmD3z%NpM3`z1){DX5U`a@E>v(LpnGYUgg%Mq!$r zS{=ZFz)~{kPa7Q*GD!i@peu%&37?&scJ$yV(_=lkH8QQu1X&Wum=kW&ieDx1m zc48FspCitL4z5vzJ}?M;_YSfGY*_Upvq$H~nvI9jPB%Ly+ofn9ta`}deNSOJ-l-VQ zxwPM1tXRM-UL9L6I&(LN^Ff_c-uL)%K;3S;THV*dFJR9t!`|K^W=&}ai9eJ!0}eyq zM%`QZ#D&MIblw1txej#?i6QxFRs#VAWo`rC*tmNx)MX)Km}L~M+x~g>IEh+p?X2X^ zhpNA>zG-4YO>Y$r*}n!j(%?zQ(SC>Ahg4Vo7?weGJB_cl1@Wi*svnd$<=cao(En^O ztSPPdoYdExU~GJ(eEcy2+xa%ho=wkL%M{W^?zJCCfyj@_n_P}J?6yjA73~eG6kWf5 z`c&)A;`dgsCP*2Oli1-Bm`X6^)c`+iaiE8GWURMg{E_Z3_0ViAYfBGJgj9oPht4T` zL*5-YB}TT=0#4Z<`iCd$@K4@9ekVJ0jIwTDQoLXEi+gNM-4<-+uRme#+O&cLT=a5H zUIHsuC|jwQBUL#kfmT}I z5~5QZdQ@$4*y7T4G~yHQ_Fn3<=}!lSGa6@2wRXzF<#FeT-^qwt`)Hc<&{rhr5gt-c z7ZV6vdeGDQY6Z^k(%-R=^-0a(dihujax2vjZsaozAQ2bW4!@_hnRs1*4a?U|36CeU40gv# ztrUDC7>Jpk9Wz8>_Gcl?T;<^UWCLY< zd8f}o=d?utc5oJ+Vj5Eo^srQkZ3ZX(ks( zAb6T$S=`Xml}6TC<_;7yJU-GxkFiPq&)airn6-46frq-dF9GR-xDX};ftGj}9a@w( z7myQEAqNyf?8wgSVpu{Az+crpHAGNnI_3LU>6G=M)pbD3hy>=0r);mFzg zWrmLsV~!1Hn=A7JM$#%E(eNsmDz8~Z-ESPq^1^w5GkgvDBm8@KeX0=^EOD@oIG?R{ zYD2`iq3DHZc$GXeY&{%s`lyLBzshe~pZ# z?*e;I(N=c{LP4G?Dgf_;#?Sb zMsHuBP@|P8YFdrjY^RtB)W)3Ix`cI1vjL@4k!LEjJYoRKo|N+a0+`7lN5_iFoOA(v zAylW67^i0Vb#^6@I4R&QfyU%)H6>{}Te8AIAP@kOd)8}q!i?KtO(Uv;SWq1_?%svY z9%hj&^cD#y0w71uftP30638MDg%a{@%pgn}zt0qn($P2}^IRqX=JC0nER98x)u5i| z_Mdj0{tM#(+)F)Nu|96o;Oh~5^SJo$ka`T9_iv;Y|Ao{ms(3{=JX9x7)00qS%9cY? zRge83qi9Z2^a+(@+I?dG z&1lGEjolNd8k?DnH5cQH;6-FW*o(XjPgnL4<^VmPi8bYuZF2_Tm8CjM^$71Q+JL7w z#JnUSQxDvfA5}a0h>`%T)fX_TN&&dy4tNeZ;q`C`%25WZ@q0z+_xF=Bx1ia^qET{L zoeLU7N#3aQG07kfOu|S)uJI%d)CgRcM~q?9l-oH=T~T@~K-v=ko;F0-jZl;NAX zVUF3)YFcGlJ>JWF^k}}!9+K`7cCa+f#Gi-n%~JLxSsjjRBzo)8X-C}k4IDuCUnUK3 zFv~Fe456R@9qs~!9TEXu>Gd)6BAU!5fG&axVHA*o4aY5IHk?ld04|ioJ0ANVj4JeT z-|6F?)LxrHZpJN}{|%dFgHQe&qYCp)jLr}-gQbnrNI_RA;#rSgbx5A^x6mtU_$;8^ zfyNWw)8h`bOJn=|UGev>?)q%@3c8Ze%g zVXEdLMr~RTcb4lZWK^vi-Wbp5L^w+L(){iA@za_>)QFGC#prZRd?7K@q{e5;r7CnE zb+W#mV<&mbv=JWZ;tkP^&5<+Q!bq|N#)`g~QPAwfEzLM%Jd}`OpK3|dp^b3wXD8@6 zd0$YfYl*XPqXXP&ibQQ#E>-w`;&P`mS}>Mdv^`ZX`0+3Y=8Q`riUOL2@gZIMOEP#V zb}&mO&g5vy>P+D(SgxV;ewNR0P)r4brHTrz=!E)M@*AlqD%p_jGX1PO_5Ea3)h#{X zlGuxmP!+R?T40fUsC?Mm4DTOCO;pwNa;Wgdk@y<2AHctb)QOoT$A?pNi?NZ>(V%o# zL^%Ga?4{35Jd+(Qcv45iShS&P`iRbDNd>^{n40`!ZD@beo&%8 zC#~8temhBPb~6ED05n)+D~OW0)r975rlWiR6jWk8TBJHWPs#)?BcX)~7*Hn>=$+6f7s+9%3rsO4 zh<#y>$U>nBwb(MR5IwAJT1_cj*kcXe7jI*(DN-R(!nF2!nv@Ec1LJ115^9}O(>*gM z$mMg8LQs7n0^%*HF9wjx0;oKgSv$ZE(C-v6$;4FGiX&hNq-g$3o1tzUf4c(O*RARE zPYdg|bLf;zQ@Aov%ZMrvB$XJO^D4+<6~pLtO;Qm+Vz4Ej>v)vSZl0$j=e78Lf{$Yu zAj~soo}6uL0b2r`=K3MS84VJq>w+S~WF#XjggKqrJ$D?(n}&*vHmR7Hu-Vg@S+J%< z46*Jsu!{ienu8!WRFjVr?Dg%_U~Eqtf-VLW{*hFh08aR3@}{**jT>SP+)m_N!%#<;cv zUQ47$5<^T=ssIVqP*IpF@rq09_DY`;ibP1xCXks_vrX?v?hz~mP+hGL~rRnm15qCyg3mIeX$ zj%niq-H0nFpTanP1qJy^%?gFude!99&GOlOOsYeG&g5}0HHF$^Icv|WWU~BC5#>-c z!7Ts4zAm+{v|>f9gh02~_%#9Sr^tR}3$EsN-g*tsOsgX`t4dJ}4T_m+K*S)~d!FJc z$~Y0j!r(1p1ko8gx(WyJi3X6v#1Y^tdmlMD9xY>)2|b<-h=FzSMbKn{r=*-3gmI2> zzX?ybMPwdx8YPezVWP$b4qG+{4VQR^MS;HehtffhOGFpP9ktiYDW z?Mq17N2}gcIuMS#y^@@Bnou)wz&_zBES(s^Dk8(7sGd9ylh}@cdNM1A)27-2%S&xh z79t~0$utDSxGcfY{RXm}`~bC;435m%(f}ZoA%axEw5S?h*`9b4C+|?4Cjr|9tBP2% z7F9qarxgk2Emyi? z?`PJKKT(&G-o{h7p+I^HcBikg=s-%R-z)@HL4)2W%ME%nr?P;>p3JqLEN2^p%x9EpH_FA%sl!Wb)dT2hfYp0 zRb;(|HasQzjx#LT=1E{wL6~72piU5=5lPcxEEzeIEMF;NmtMd|pFMbG15 zv2h)xlBsU`q_vnFQxFGb$SZefMcHZ5R!m{Lc^Q^FBZeqDril0QuM8xj zUGgh9Nye!CNTdW4qL#X38o79vap9ckXoKngbQkY+PuOO)K>^?){XStewF@By;JWeZ zZvEy!{*f)dRa!0~lXt;{%9l}FKoQIer&cnt{Ig6Bc6OP*lbKD-qomQ49H4F%NCpx& zx_p~c^WX*Nh$R{2WFA=D$vVVAo8}1#_v7Unj~QVh^$0@gor)o;bNZyrh&V)N0>&O_ zToxBeOaR>G>$T-OY>z7;sf!Rj(dBBm1>KMCjbU=jIKG5v{~0X$b$3qWd_EJJ6S*y9 z$@2IB26NTaw!$fyS=s<138k7%k7L?$icnXv37mycP2Q*Q( zuTjjgbUS0SERrx0=mCE^98sBJC+i1_sQ@{VX>6JtDd<|p`$pZ&E@ZHbUrk(uS7QSReO)i7K| zKEmGzmyspMmP95{y|5`t+A5I)%Y>ybGl@J{fEEyT8>Lp8KQokH#hliZ`o5rG(12D9 zov#3)@S4EqotYz$)Ole%hRvHHN-D-6LTb_zHO|l)7)TbBk0iSjQwGLx;1UU`hJ*L; z1rCYWo;)U_!JsXt5D>{}6A-+NK)LjQT2nS@RH-F@BzS>@fZv&%)c+9eFM7B-ecW#s zGuJnXcI5Af_8d6<-$Xm|7tvzsZqIvKqoD!)6iHe)@X%U|l6xF;i`7TwN1FOx?Y4^x zF6;DU3dsBgig-|y12puphK4W!toM+BX#g$Eo{!!ah#JQdaS`(xj8&aClP(~2m%%NQ z1L{VbIr+%DygcaP6d*~XlC*J{LMQbyzYt`IC?<>hT%hv|J#!E}S{9w|6sJ<7&3ffC zc?~mky+vih^!(<4h$~!dkwVd$4p^Ea*4=z`gkqyt!v=Jh^!4cb-!8e3O^lSQn0T!( zC?{pRW5x{DeE|XpX>4Eaov3m_lT>0P8)iz(RAlS)0VEJ3eL=fvT3n%mV)?40FUwOu zU0TpV48+$pJVANWi8KRM@(f*qG*2r&49qpn6ARHD!|mkNGxdtZ3T=WpA{vtI1EBG% z`_j_;j%REJxr0Op8qnjDfalCw6X%w4C+2gfoT54u0WL%oBf}VpSOyX@`(Tlj7uH9^ zK>3ByrZ*!bgf^ci{ubL=br&F3fG{f&=dN7elUXBB3h=(4Toh3{yfV4Xq-JVhHqk4S z+gH!R6oU*V5Pl3=BNRB@qnEj2>fPrdFl}FiAfE$tQN`x@P7+gDTzO>V!OFrsbHofW z(`-W z-2x4;+@(FB+>kQ%Hrz&?rvdG#NhUm8$qKOmRn~ZGv6PiNVhkiEd6Hp6Bg>r99Nay^ zUFvf)d_m?d)qRCUbP1N~gWuAEsP+g&ZO!QP@w$(500Y*JUQ0UnKP=m-k6YHqwLg9` zxB2$8{C6yS87%%cSeO4|SslBa7Err@C|;LN3Hm(JAtXU9Tf+&oA899Su5EWnIAjQD zBsp{1sF*x9KP+tQ&QtPxrDL(pj!;No8YFLM(#B=}ZyE!TSQAs<%KltLjUfw%6w$PLo3UmVn&@P7=_~1XFc(eYSoI#d`^g z1Z?rGplyky`tO|(_R$@cYff!|CIaB0)rXwqYm9!DViVA5i7 zWLlv^NtK;1CPOowC4o8FKIcmXWXaSz0)0s1=wkr&9#im^ zq}+E>(}U!kJX9nJk9bZ7f;bTbPjPv@IjR#`&Z;7qJuo&CF}o5iYwh@#*#q{}RYtNf zi$tY1TO6pVgV`CNcU?JOBleTN1j znOl0e7im4{Tl7-zO~f!g{ZUe%bd}6NBUuGO21jW5k8)7yftb2>rmJ37H%nH7ehabI zkS5=ko?65NjY8}O7}Oj>3yBw(3FA%4eU~@`GfcmTdR4kLg$m(msc=YSYL30BtS*(L zO|YjkD~0UPZ@q_@QLEhevK|irMohAj5NM+40|&;OU}l{@jgO=2oZ8c+$FGpZSaNrp4FGRTwDy?9wk$L=*K`QGWJNy^y#ssER=_l!&O z|G$6%0RhDj!4$O+acennrUi%uBqoUFEHyPXH8oA^TZtQ7;mnmOskt(Dw%sc=cjjoQ zZ5w8L-v0jM!T)}JU-yIiAs0O2b)D-vpL5RpeH?&sWymRFH}xrDgG9&G%VtkwH?yzB zFd?zsPb-n4N&pUyI}H=;$5Er*I%7{QopQ?pr7A&E?I;QmC7Z@6fZj{&v=J3Kgm~mhfmlRe3@yaOF8WRxvs2ORLXFGCN_8y;4=~O0t(IQ_Z#oD%!Gz@xd3g zu43{js7jytZdpQYt+3Szr|q<8sfT)%g~(?l;A!i(sKj2sBCzV50^ehAKn~4NL{C=U zid25nq-3Pp`y8h!TQ-1@Olxo92^0x&6rCKQ}(j6mAA(VgJM7C z!|F6SDlZ`s9TY~jdNCL+yOpX+gk=-r6evQ4HhYu~{H?E00YXP<1lwJs7yw)rSQIfy z-^e`to`+VyOPX{ZyZ~V&VG{bTmDvIf;PyKtf^7}(tcqF&NKJxjD$J$XDK%<8nHQq* zcs(7YjJ#gLR%VWRErH=xA|BNO$cUi+^MgRSo5BSeUg0uuiU_Z7jd()lAka~BqKO%Y zR3N-4yOz;|(VxTn6^}rQj9llTC+4TMy$w=B6#JRO;E)1(cr9}@aUgKCTa|bNC8(Cc z3Bp=HxE%GSr;SnId_G#8?S={Endze?wfN}{vNx$K8u%$vjJtcegA~IphgkvxO#mpl z?Y^4?KLL{`kr)6fV96c`!C`87c&(mlS)Y_PG+p5Ch@R}AfX$)DYk;^`X?4=3@6d+; zIh3I?H~rbw_=T{n|3+}W47OJWTbj4xs*ElBHvp$eME)0m%l-pk-eu+@EiTw5Fqdct zZw&yq1!7ELa$>x8nL(C(eVnt=0@0zcIb*&cQf$t2g|*mo=ykPtPrX2oBWx+GpbV;w zi>oQQ!YunN!O;ZNOUCn~VRU_yKq3{+)@!?Pyk zwbTbIMFw77{m!`=vItUDy-P!cLR(iy^GYuTl-#J5+iWr`dIoaMEXx@>B`#-dQFg z{8k-!e0D)KbFvwvbQvf&KwfRc99FZ}e#%^_2bR#{Y?=%u)L^c7#T9euVgeLLJOKoR zC#-V7v_Q1L8v_dx6^2&;nu?+76BS8_taQR$Ghj2A0}fUs+M@gP9rJ@yNe!55oPv1x z6LXFaROXVjw~zYa_)dQORSFsj04SoI6r4(DyBNZCu?UnVWw}awg@Bg3;;Ot)$L?0v)5Z`y z-wtaCS14y2%Q~TFkwv7+;=BO86z}R(H$+#G7B`n0{WCg8w88Lo(J%m;*Wmcg@glta$C<(7= zk2VNI$75u6?MOxYPb2nwX+#vys^*crlsS<}-N=0;khk-ZR02bR+KEdKLmLpgOuc6T z?GG_d3Au1QBd?OGZzF+Z)c+id)c0Wd8FjH0{5k`+q#4(&t`2V|F!Y~dI^=?W&uHWb?=D<|N%$!VZ@obh<(CBA+VT)IS0)!{NdyDkKu1u3$C zUD8Puq3rRyT66f2`DWEpBmI6bJJphr*O*R%mS^L8m%)lgs;3U4?UXCv3PjmSZt5~A z07UdjFlQH=GVvtdQ(6K+i;+&El@A1_5qzfbK?04agiRiVWh5>8`6Sf?b%38bY^OU_ zhvngyoL$wJl!T5jZWGd-;ZHS##8#A1rlvE?NDyFhu3XQRqE?0&;93VoI*^cfGmxVp z636b|x5-dQHn=2jRU%tzk)7T#l&YEpP=)!BC=ysgD!8sX9H3s#AZd$1I!@Q6F_LlG1g3jefmCHCLNf3uKgP{4F9&MRZ{;AJ_`GrE zIEYn-R}HTjov9K$=QZU}s^=IXbM=WMgMsW83a%!9IFaFICQZYJ#%pr2s{>*M=F>V= z$RLWG28GE>ZLTAt6I$)%@bpf6m&2q=z$EQ;1sL9q5T42n(cw|lm zO6UMoCP88g*ry3#!{0Rw)2;DB6 zA4g?%XssFOlp8dl88-MXMoX)7H`)6V&96>WFwaFxbm7%nknse4u$3*!S;N=ql8CM) zHO^chLiW)$Tosr~^mOgYQ3I1GF`gl2^aza!HdhNcUM2aa>j@R>GKY@{AAmL{_qlFS9&kgu{~^=&#a(1xI4_S z3KYm*>QRU|Cpigyo_|cz4N)q|H<)P00cRkx5G>!I zDM8BCXc-`U87LWffym}nsM{uM#{mEjasvMPK z$=pXonGKELI#d!4A5ta2PNCpMnn4(-INdjN3S2>y3)>Yk7CsdmbV<*9*x9a|7~%q9Om5~9 z&O=nnV&tkrdZElB1ud|2c&|9*0w#$k3v}A+S7A)EA?}zbI02lDjV!#S1sPluLNOSy z$sy4sxO5Rn+zu#g6-mhoW7PnWLKlhwGAsv{zsy3YyBcDw;xWE~jSRq1^J9=4=2HVN zIRn|wYOD+?OU}HKY2hHpVa$^mDi1p&sCjZta0LyE_;%@4fofiFE!7}!f!_tl3oF(` z*qMc(yPVVw81NM2ayz3oH<^l#OwfR5Re z7+7jTD!au$Buz)Uq}$#o*-PhgW+DbJD|t-G>?EK~J2#Q`a6GP*Znv$MBS6^|5d)S< z0z3te5mvveK#xPybJ`5!%PjKgb`BB_({>>RjNnYV7oU`Y3q&};0y^1cHyxkBvTB$q zt6N;ytrdZW9a!G9djjZQ@*%N$@F>~59k~X~EK)E~lG9qn^i`VCd0A6HqA7Dib?C?K z+P&t8@Psn9_C0l|yWOg9{3Rl`#@UgQXt=mN##+d@PYAp-HDAxaC#MoRU11UOe78HJ7u0Ost2tZFYwTFLzK9Vd4=3 z21AqCQZ`ghGDPs9!#ppwYK8;ZDZ3XWB|ELA)qz&hGxiISqa~mbPTcl5cP)5jH=_s? zfFlNDfdjm9b~q5wg8@bVk0+EWgYA&PR@`3ufKbMM`!~EMNu2sGlKkyIcwOOU9EPcv zHHz)aL**z$`2lkY>Pb5GBe@}$JOY4w(_;m# zFh{K<8^hz%EWd>D9pxst6{8)?1VRPOL?qC2gk)i_YxtN(OINK&_Ez=qOjlJ4h^oHqmgxW zd=+YF00ng(N{>O8lbDM@Kh_!S%5R2pI=WXhk57579nI z4(XolKb~!rHb8bW^d?!e$Z9pJ#c)tFM`bXIQ{Lu4+c#Rx77GG5?{$#|3f+w0kUMPs zGA0p_kL{`s@WS3kJHKmAIgLKK;EaPX@~8a3EF&_mPg)IzPe5fmT9hY%eb8+bDJ(^+ z+M7g@b~8EJ56<+s1kP>`24l*VroC&O?W7=(_zD)$Ko15RNB|$6TsBmwV4LtzICZ@1 zl6=&3eBc}U%w!|wo0FdPoS`W=oS#u=j&uYZFC~VE=)ia~H^-nFdChxVoTo4h zRLbUfP#Cdpn)>6!{qu=Hz5%4d2T=uv)2f3QaK-wvau7E&ce9eI>Pn7_zEefU+DHc9 zOc%yy4dT)F#98Q8W?@A*Zbm!aNZm~biLjxL;?doLAoNktFv+`xs*bZ@$ROE_YZT>G z;DipdN7_qC)i6F;1fr#@NdkIEWDL>HbI^&vgMg|L80ENXFkz~nUCiAqTn0L5GsE*U zr$}tj-c4Yc?SCo zt}iDGV2S8O%eX#MWa>VK3N@~T9-M*Elfc+bI`E5|{pYkO#bDK{GCMb35T=b;oEw#8 z%Ff2K#Dh*n*CL*o`xbX_%1O;~W)y|qMkOOk9f8sv6tD@~aax#rqCdJ-C{e|XQp_sE zhst5nag}g14auzHuNVZ9VXiCmyj=O(M4!xg9%W>_%pr-N-fyocKUAgHgw&*f;+lG& zivx1C7RvHsyFfwNcJgUu8B=(Baha%~h2jj(o(*mz4h2g<;^+H$S-oK}A~C1hD=9f{ zu)`Of1950is>Nq{IeAT?QWGpAa|o^K^16;Dd5#P(mA%?hZGehKEpP!7(rbcVq<|sJ zqWe?W6yIg`SUn^q{lU9sD6wC#<5rc8pvmDJ_XH5B?i^RaFtrQeK)CsJR?QB>` zlHCVmCN>64I=0MCrl%Bxy(n+ znSnM4)5B8L@BvC4<=v)iszzp$MktHCqe}YZW*yzO3bL(YL$_&EF%&34vJILRI=KZD zmO-SNVVC0A*`|Z|3cty57TJt45AqhY3*H!nV2B}TEu$>NQAhPK{3ZQq=;q$=a$nyt z6LMTF3AFB z8sEPY#Bnsxj{S(2zQ7BE>FakU7vQcB5%ay|aL9ZsY!YS*j>Ji9GjVg)rW&F79pNFS zEF#d-ArJV;0rt`eP=hy390zWQnI6{FrKV&MQK;yhP?Q|?q^xq{2B%O*SZ!e-(H=)^ z6R)Cd4_AP^@KAJZ>YxD@1P+|)ClFMCQJs)p660tE9!1Q^H`OVy4D#(yN`-|!h3ajh zu9wTmXw9tRhZhiO3>wt|LcxOp61G`W(;a!v*kWpvn%In=YjW zDzAbkwglpt$Oub#x)Ix}vRzK>U7%%m*fPse)9L6HuL2Ud#~@e%6N&~_pv-D=jrO{< zcbAE}IL*GJ7<(lU+HJtEnio~v-Xvd$^05Oh$J3{)42wt*4;L4fb{9uSTh5%*nP@wn z3xBE=c6h+6q=N zWkBT=YCCgSo~0-bVQB_@k`Jz(g`oQb;YpcK+7$)^Hr+05_t8LU8=)f>Plnolpw$HO z4j0i`Ab23CH6)83w^SE*J$#wZJ5beO*-JrvX6{z)-UK|22bd{YQ_QB75sp$ElFXoV^@?^F%^t z0NN_V>AZB7l?Z1D%NB=cvAn|@1Gtx+3IQ0UUU)GneVJD%MF#UaHNw6Gleu$ zA30gDfJC>c^Nt0!g=D*c@tRm&KBt=Dw8*7^05P)h(sOv2S`X0XdZY^8IY$SZhjvA& zE0)n16Rontoi4I1+^3i;07JV1QFaL_4pHU*agLj4pflAjvqC~sb7u<*vo|$|TJX5j z9E@)oQAmapP)zsoTn8Q4h!^NJf$WX9U1i8*HDOTs4)>~Ra;vULNKP)GVF|fLIE|p` z0*Y=TtT|T}QXNGc>ZaO;T}gzzNn|IjS_E&_==35nLgo&Nf2_8lpohWbN~Shh`XS6x zJxCgp@-a_ZVT&kTLpiUt_%>(VRFR1}xSEYtJEbIMm!KnTZUs4j3|um_Idu-j72Z*j z-eN5dpXVo9j`%Rkn1r!#E+Klf2Fh)jA|lf5G85z<1kbVs;thwDg-cp6wRS#7O*~iT zRBFxTg7;4dAPQ3OE?Q`bX^=qCXChyZTZ~16lZ-9y^(1o~lk{Doh7-=6{Z_!w^7NSi ztyDWARm{xlg`@j;u^btXBvNS+P|)!!5*~B%eN;o zdJITxRNB^Voi)R4lZ zNlzJhNq`zkNboEuqKtJ?kv^Q(>Qzn3Ajr`G-B`4lDz8PH1q-2eGKV`tPzJ0L2kZ*Z z)n!h|g^x_)sc75yE{%GH8u&Ou&$WM)YD!%olDm!2PXtV^M7Z}9fkk+kMXll&(64}I z;Gu&j?!hcl%c>>1D)3cdMU{Qard)|CbUe)uZwhLxK%z|jU4$7*hBaDl#9~t~GyuaP zAW=4aYOz9AB1Oj_2q1Z$I7Kbcp;aBq7SO%N%BfA);jII7R1ZPTw|JOmc z`acMMSO$vA-@IQ4t}uWPBLJf5gzI;L+@=+!M|Lv|MCf3LZ2f|MW>K82heaG$HtZ!P z(G7p%XuF_55~2i7nm1reH<@5*3~_Bbl;W@eKoZ-yK(Yb*L0lPFHqJ69!?IdziWKpZ z=ljV*65F>j6UyQVRA+;(9?GFiu9$khZnCoh0q$=aMUsCTbIQOierNnUA&K8|}M7D?Lf zAY@#1NC7{7nHXBo>xYKMp;NN*v}9#9Lkn7&S-j&gMTk(K$g1!jPDY>&S<5IVoiw?hKL!9=MmT?T1| zyC{*;d_YyO60x92X$p%-grzXZ#Gv(h{Tv@HbR%j$8RiZl7cxOo#6Hm8zGei^6_4^# zrOr<#$L8pE_t0P}1Uf9YS#O5=2F#s7A=FHddTPl!CNqnrk$}w@mI)Fk0VWza#mC!E z7ldSkvYF0-;@(lNhW9H2GDy3no_4MPuTlO|R*)zyGSCp=f?KshCp)whP059p@HKqW zq`6-%o&s*q1)1vCqu|9=1Qm>sN)WM&$9iWQ34C4L%&#Miaj3f9bc5x1n3y{Hx0GtH8~cqu4^6j5SwY_<8mq2kT8 z*ChB9I$KG@>ChE*AD~gVag9CECKPGl9hYy%NX7UI1nr8nj)`3v(LRUe?|z=6Ln%ZK1Nz#&XyBp<^XnpZFvbA&Iyg7<5~b6VmKP& zP)MgJSK1a-IC#;MKyCWfNf40)Pmbo5*m$ZSHBeVGbd?Btod_fVVAEAzWHY;|vc_T> zMv>VhSLOW>9D;7lP*sWd8b>S}u%B0L&MA7wW~WmI0gRdoW-l+pBZ=)QD@NNe*D>mK zM7IP0Bijlw%gZCk*`;HE^TQZoqC*@Jy3m`X7jX6}$RyP0ie54W?^@O^3H z=@rrrIm3R%&~T=dAZKq<{NQG;gjd|i&+a!7z0}|50cMfKben!24jAZSdZGjr i7 z#2e~%3v*fQY{CvP&A%vFrl`!P0L6%J^n2UbHVMTR^oF=Q0Xr1}fJb0z&Pc8aV}a;W z)I2F?$^=;iV9O~SOb~)u~}2iWju^S6L8!2#DjVpO!*! zYD%1R9$TbgpVt@;)w&wTlM^J#PL(mUQGjd|eP(mYpn8x$NCRcD38W#X&ds$qWnx(P z2M0l%GbfFt zT#alH!C|^uC=!I?BnNHi0cuc=gggrqpP^>K+=rG{Gq4+>F(tO@^Xo&3Ou16G?G!kM zO4TsOR96Ad$3vbb@|V7_XeU_Zf7XIA3|V(Mn_H|mXPswrfY^L>?80*UJW?4uqOf`C z|F!kRuShjA_W&x%rX@FN676LRA%JA#1cG`2jYj9;QL?gwFkdhzj=cF1uSSr?Vr8+i z46qSMQC2oC;Om)dFU}U5y#Mv$?avoW*Nu-o8u@keMsst-`p*l0`}2NYeX(*(^6;_6vw_lw;eaoW%(s%RUhT9K3j=J?TWcQzsCwKq(6}Wr%xAFBq-%p?ZG;sR#k2^n& z{(O3I`q$@YF0m%t-+OF@8x@{x3|CTOygQ>!o!w*k>fq6>Hr;8cZc{bjwX zeBjWxBc-yY+^4yM)~#coIK-|x&7>`Q+g3!LynBYzONU4ULF{5*^)A0n+1e_Grla8P5z#j zlLkh0OCJPpS&O%IN!Z>$Y$LC4Ao7T#=8*!7O`z$Mj^FzqdW~m>| z(A`hLX^lJYuZ<}VWLj-w z^87ycAbjB(v=n}9ZCk8Ty^l(V?as)jrNeM_nOk-87Awarw?O9(LyL`8PUFLm-F8Dg z>@E-oh>?nr39kbR$t0`oBl-P5dcn34_5Q`(=#sE^B6vham+?=ut=tyPK56?3!yfmr zZ9iqM_S_KycHZ1M_awv9MfkZ%`-I&u<~ zFs^R8R7)DSJQUwk^0m!Kq_FG5O5pYV5B+X_Q%NbL$c$(nR{WtM_swNnXAjQ70=oQm z{o2$=i34+lYm*g6mwOKMovS26_6&LMiHi39(SP6itm_%?g*&5nKJ+NkidMEgg1G5* zyL*MrW-Xj#*z2ieum5uSTlK!%Lf*H3b6*wYO;5T>M1H|8PLtHg`p|vPR~s+&AV=rl z%&9BB=WR%Mhh8+i_%r&h(No31mmMA|iMQ3=5BseZOqsQ==xOe|tMzdezP0aGiM5ZT z|7po*VC}E&9XoT^lQYf?J+AX57~glX;`r3fiMH?Gt+gktTlyahBZIpoPcPYi>`}Y& zcivOH)Oy3*hp6z;*>5`fk=Ceie`uoSLqLih+nb2jSn6$fR zaL3r2f_35G?H2dj5#p#~Vvh1r?u6Dcp!-MWg{}0+-mBaf+prStfZ`XU=m>Pk>`k@z zH45NA(|@Zr*4B$xc+i=HnAPAJAO#U)(b3SdQ#y2kp6rHoo)KaWn*T%{(1Qe+>!!`VK$$}0cG zw;myhTS4unrX<#8jC>GF^EJB03>+AekLx8Bh&&bC9`DiGUt zzbcVTd>IhyzjLmWUNZFwdBSpK0l9mEps+jlwjX>bjh9Efz_YBx19(-VBskx{&X_hEPJ)mChb$&B0Scy^-pQ{tK_GV?I}CHgnWfAU5s>k_eUxJ z<4udpi$@n_&YwDO#aa(ho$Ao?U*_NJD#?u*Y&>vBMLIa9Wo_4E#74jR3BcjN82H=K|1=aGi~px@?W%OmD;b^9EY*ikevTQh zY_k^+FgI(3W2U|v+&#p5vj6W}n>I#Bvuon|?Kp#|x>Kwg*|D^TCVDr2dR#t%|62UW zKksE+BTFExYc0yn8(?3SRQ~1~r=g`UlB#Z>^WDQhh>6 z%=?1<+wO4sC)w6-4!(-izPK*fd)@8DQj+p>*CP>*9?^DHKWZ5t_5B-%;h(==@Wo8C zd1(jo_Sc6U=(QI4MyJj+*!*t0r;_k;S7i8UK*G)8hG?|v3^MGmX?UPBaqQjV?Ntlw z)WGWz&V{j|!OnZX&Q2b8-U$=5=YaL!UEL?wKA!LT*7+=CQM!&h$KS^ZeZ_zqYUcf#l_RYt57dXgDXRIw9YWsH@DZPKdr7$dO}h$9YJmz3*eeT7-?*jK7O8e6F<@A})?@xO^dRfRg{dwR`#Tc9uE%dpWcyk3){Z))cmDmeF=Bg zRz1h>98_X7NM1ZN40YT2_!E5B<(n(338iZbKvn7EEl=X&p5;V*{#F0J+ql~N;j!>@ z{pknqn7oVE4C_lx>TT#nD~xYtfBzj26|jB>Jpm&Qo;kHRC5k_Y{~FT0?M0*9tFqCX zD+e1>>K9*m?tiXt@Il^m`~F|N>t6R%ke(lQD`Z}I=Pdv0$=#gCTNRVl$DV$ffQ-19 zN4#D7W~3Y;^CCgt?OsEtgyye)Q}W>P$W{2|zR(ACkY5Rido(qNu7iZ0iRD8pms1bt z+^yZoZDf9XN($y*cnPv6Jv$vkPJNqlQ|gcRKfNx*`0W?Fi|Zi_bW+FHH$JdGs@CDb zSL#BHVrK72kF3ZxX52AI*E^lBX#g;NloV^Q0Y2NPsL)@UC1W3Ik#&6X@f_MY>*mjc zBlyIj-i_N0xB4;~HP>I)f1@bfysh&w=FQpb&pg+NCwA4)8FB7kR<2u7rhIdH8ruK* zCcgQT_fN00oaE}>0N~%uh|H0}jBEC_YSlhHlPa!@G0We72C=1)(U%Y}BGe0**^{sm*evGa*^afh|9MSIVc zOMa)l=dX=hHvMkay)$&o9PG6GB_`7(;%H4)Xw;t*pKW(6F7K-Gepg%b)?oKp^Fwl* zPwR)zDIlG(l-LhFTL#a_%*M~Iy#W6r{MzzreV{32ZtUjJ&K3P%X;`a~|7uvV>}a~;0hJNxlVhE@2NIOGw@2bE2RIs~UfEed2st1)yfZVw`Bgda4}8yDUV(3*;Nv7a%xJsW;}j_(x<=6#(U$-dAu^0 zd;ILnLnn3DmatcjI|n`A@BwVuwH_Lc6#5_N8ydErmm2SDU$vBLUGxenUa;krJaLoR z!aA>7`u>5=5Jt&Q`ilsCXxGcYSM#B@QN9S>pEnT)l-gik-Gic`iqqVh1HMrB% zjc+>PKBB41$vItmm3GWM;#X*0vtV1mYRBt2P|g$D)#-kl9cC|;x^BeFJDa6RHYkU9 zshOVOwJzEQN3`7VN^xmEw*9up#^tHUa;Br;#UtzB;KsyvAKw2!@zzo#S1tQLJ}_wE ztiHRM^KeFc?&}5hkOMA|;KuE{A~VBAFOJ0UJQoiT%%Yx$e@$4t?ehA|wMp9z`JXCx z)z1#)5a#?vWYBA04;8*z1>zAhIR+PooiyG$^^Iojpy#2ah@TAk*$!8fGO&=nQ zG_5tyh<=3n2-I{6CP)(_*U1kPUleZ5_&L1lyBPd>Lshr%YyG0z&B(Tv@}Hr8>X+-C z2j*(4%o?V+d#ZP*NFH!__slhCJ+IFL7_Xx;;3QiFY;9lqGd^)FZ@=in;!?^b$@~&m zc}3*dwVUy#*lGLl=8ySD!V6`uI7xlQH#}b6J1kQZazfY4(AyyWS;cNE*WR3%Z?`Qq zE*K=wPjk+lH#_iOSjUJ z%mIB5vtLqwKXb9vN#5-PxR|-qpxzSuS)lYq=kS>NW!* z&0w{7jIQppMc8_C{y4Y4W3>0i%Ac_{+0t{hSu6YR1fJa~QT8!-s>SD?&k>yu7v8N_ zjdwq`+<=5E+TXdbDJ9es+C~E{)(;RL9Qy$Ov$}ZBmU@fzx7o9`uu}J9PY~ftpZH*V z&NOkl|ETqz{Nvo!GWMF~Re53J*XDEjnuaGzasnG)XPhqN?n(04Z{jG_F8;ae!tD`` z^&>+)Rk^zf;;65$GyA_3rh3Yqi$xx97k}=m-xV2aJNoDTGu1B|#L_z&f8I+i&39J$^b5}NVP{jFc3`%12gQ)~+m$LB8$QEZ&6YX0f#**4z) zMZ*u85tHT~wpV2F)C>K-r{$OW^u$6=%65OY(#-VZZ)C_nr5+Y5VO44e{SUo((R9!n-w7>%T@X)SP+zcjBr07wM_LIZsNm3$vGrmzORc@wr|2<*L);-pb-X zCdU;>q+xW70Pt>|?kn0YM9yfoMe@l_D{;`b+3s!r)fg2i{v#?17FP3qn6@ut%E zb+R5eUU)U^`t`E&f%~P*OQ#G;YQ+j_p`UvSZyhe-*;?Bc1*ugu&Gv+$Q6z1>{ZnOE8Ea5{x!!nLt1$5H6B8uLu~< zThVd#WV{YaHcRjyDJ)J9@c8sUSAO5_FZn!s{p;A;cWdM0k7nB1+U}mc`|mJ%fM{{?H~MOt5-kw{I{~kd&@A?h%P$9sEzM-$O*M#RO6J|ZSu0!1%~`KE zmwUq3zV+n%fm*z8B<#d|JM;NqWsiF!?o>bF+FxbG=rr&StE8=^EhilwHbD-?ZRwO6 zcG3J1OM0{L8ffK}yt8@kT6waD&0Fyy@2#bDrJk)@5~Z9Ij0V@;e}VT=a?S(;&s1B` z1u|prw)%ssrP)u-|N4tU?&j~CP9Tm~St*UMY4E`;JbPW3KD{E3bjBA}Ohl`Z)5QTf*q487|<3aznz7kD<|z z?_^YFbM6oKS*DEn>8qQa)kYia`rBK#zi;1qrrt~W;W??qm-!yOyALAVzBP9zeSI1q ziBCD)y58zWeKEIn*d?QbLCQFAcyrp-lyg-X8~inusJQ=NY6|fL{o*&YBGmGEv2?84 zZ}aV^B;S2!71`NY@5o+#+c$3U=34#w>)(GbK{O{lXH~Ze^1sbK1XzA&y)Zd+CG4bA z;>%gZ&WCd+3$*%&zijW=PIUn}qaA(&9mlUWb*Uh% zY{efub9Wug*r~en{Y~8dXKMG;FRSEjEy%$h;`P+m+wpg6a8;HU(7`AxAc1gF(KNfmN@^kmD>6M8=zJg=HM+A*>~XV9+UoqP0ysQbyPly z9M2s;y&ikeACdJW&u;2@ig6eL4lfOU)jy#e6{q`e!|EKHI5r1ln zS9F74>tqH$kuh}r>bY6dx~lvsJ)gV2ji-4!&vNx2b=zxRPXbQ?FF4eU^X?}YN8u;V zXjoS#^wY-}?~rmUD;qIJZq$W`7CWd`2G$P_Mk}8g%GffjzGUh7%Er=3R_ zl3Qo(IJrILQ0jx#p{CdKMsL2&&AedTc(rAob5^iL`p|=mFEbdCW{ADZJ9giFWVVze zHnSQ&!+KeQ)vtasHpY5TpPqWL3>ewH`+%;JKRfX;P<`}<_BN= z+4?K<&_$D8`x_spSh;uho9uBWUwpDZ#Hw`rvFnG5);6$z04BdgzvtGOSqpb=<;eXl zD_Gif*_o3`P37^&A8IG_v~N=Tf^XZ<3GNW^rH5gxME%Q|4 zX9X8VQfBs7W}cdzF^U%6hhxFtBxetEg|2XpQ2yIrNdgv0igNAm7Xl}?K` zFuIyQk=v}gHprypzZG>V%rj?YDW!5%xQasxkMVUK5sa;|k5r;x;_uh<^}<~|D-pPd z*3s0BcERPc(r2e?0Rt~r*){4fwA3uNbR6#9KKbh6`OG@U`&vGSX8xHQJI;w*uE{js zAEf`;`Ca#u<=s25_paS3EUEdmX%Yl~+z<^T+Uq1r==OPYIDW>m$+?ef|AX#LiM z=%lLrS6`p(y?;mQL!{YM={dDmoa-qoVO=>>>qE-DjEyN&IlRhq-w)a0d}hDG;` z^y?ccik>|Kh}TP}53Ohid^+Jme5HQzeDh_wC+oKcvr63pu3C1z@o>C;kacF4#dx9O zuc5zob-8<{?6;#p~|9p z{p^zB*j(zL;~(pBD>WN=KTlcdsPvvR$#)C*w6gH!|6%Ja!p%SA;iKNmnx)Lgn z`~H0IuUkfT@o4va$y-7R&?yWo6>J|e_7--4mQdwzk>t^M|puIF>OrCXdY zgr1Jh{|1g-Zm#qdEgO|6JUwKcxE$=Y1!E-yq8TTekE{$aK`ese@%3p4z{T(m5BkL$Xv)McBxC*V{*ajhhR+{IW}KD~l5M z^neCPK5=*$e`T|eTl5hwvo-8C1omgZ!0Fqh8F?}E8{EQ{K(>}3qK~xx=kT$8vp=`* z4a20Sk~#DnJsbUA%^>;}M`_0OtiToOB`Oxfd6=aTzo`X#+T9;wx5cROO%8G zMwChSSIgSi&MFNV66w3KBzC^{DM}9lCvEVCbA^m51XvgDv0|Ir7n3IvWv1nX5t(Z?V`N!0w5ie{e>Ph_c4jNJo2BG&YMx1RD zCRVc>RDptW~+@=f8Izr~5e zy9Oa6fN74#-*+bP7n^<6rYxR={!*bdwd=F^M~A~9a=qkUdohZkktyew5^!P7Zd2?q zs1*CyP?58AKfVqc*@d9+pau7lR$9LRGAMm`%zVf+$LvVsde%(*V}iO!;WWYjf?#q!{44L)lTMfRrM8`eE^7CbgBI>%-+a4F?2`R}LLH1#y<+ zCM9V%k~;Bm#F(hfPryMH+!Nn%d+8#W#!GY2iBQvIL7)K`fzt0Qenzo)0GTPN=XZPB zRzc6O05lDv6`c?0`sbn*k-g#I-~L<=YqoawGcxvm+pFF&Mu(hx5q-&g&TKmwtjq}c z=0L;T-oF_&9%mT(DWyz`WM5N@X7dy`2f{+#EUCGgGNcVz9y@Pl9wRpaFJQ?if zbIJHC#8~9S*cXmb^YfoVe-%A8Vl^C6MSYmS;2LFm6>q=`;+9TWmTVSomL}fQ)N4KY z;JB>N$zhlE&+Umuq4LAIF)QzMuk!;sB=y(d-yGZ(4)&!evin;Bo6A$x>jdo=B;fAv zDA}bm43yl@QoT*&3V*5TMxEDFS0`!5W03ZVOn4xpX%4+7d3ffQP zmFiL(kM;5twaYiDjo@cqyP;HJm3VpKG|ybi`3X<%oBfmPpbM=6NEdSRuZ zPEJ=7^a;6cS0kOvIn~V{wd&gsw#}KYZFmu_8VxODS#0Srw}b3Hi97}Se)ccL-m+7AG40_Z(*XlT*(!K8vjUuQ<@?Or5=c zK%f(*kvUYUPUlIE+hpr0D&ooBDkTC@(Ku@lU(V=i02jW&~((`Di4 z-3e(@J5`M@&S+P3+|1Q8i@4E|Fzpg?Xg+HBx|oT*CSnhC)%wFT4Wk(h4P^APzlHq1<58!|s*dWcfqN zOy|0UC(4VA)4{rpQ6UYUNdGs-T)d-E6)rJjdb!79^Hb7zPMJ*+A zzQRcfK`bdLhJIS2b`r@f$BH0Ji0-;U?r0P(Gi~hTxXuV_Ga_vWwpd9vv_#X?x$y$@ z$_8OTfCSzK1;82wht)sX>>672T2r*h8yJ!T1*ow@&~S9@HSP(2kxOI zDkb~pjWk4S;HF#6n|1R=E#_o(lB;7-=C!pDdZ~Q1QLFAq+KCYrWZf zG}IgrtT!Y_}gIR8Oz_Mqp|x|AFSq`%cwOO zI?$%*-Ini?iCb=Z&pbc0G@$nxk6VT88~~pSl1y;J&;Zgw9$Jw!1gO{RUm-kQ#p7b$ zD@?|PY&91kvrN6Z?*>cB^?_sKh9Ir)rj&*~OM`~)`_{Y^Xy9V>uggQ2c1&=$MX zBvlQ8JQF~J&wxr(5s3AlPtryrW)snn-c+Wi#eBK0bQ=&@QK@fcGc^Jf8zYYmbu);Gz5GMCZ5b?i_oK$}Gw{0>&RY-ZHy!MWe;D82louOt7>VEJr{QyF zO&9yUe9p`dU>5mq+7lVqR`lrX9dW21YP*rSLWlPObDrz{WbZfb0F&&NdZA64?nU~> zivGNJCV>TCZqG+U%7D24&)RBE!=p{(q1|r%n{%hVs{d#D$nHdIK&edgpoLYR ztun1F>YIWR8)f2M~#MVL@Fq~`490$`Jfx!>RN*C zzqQ((8OlU2G~iD6&=-=aDJeJC_7}H%<43D**T?;C*M8FgNO{k7!^G+I`i}RpSu@f! ztGuCIeP$|o=W^RU;I_*2Va>2J$_03>Cjc=$ecHcVxEE4Z-sCO<2ARQC?$s~HMWje7n?zTXXk78TacjE23 zIo>0t_OJE5rUAkpx5lE`GbZm51m1_8yP9PRTT4lH0oYQAS>&?nI`~CYC}Jpt0vs=( zPuw*dSMDPeMJE@Dce3`m)yb#D-ZP5HH_GJ6J!w(qAe)VyUlRkCo8m(?^T`0~KNLis z;cCb~?W*u56HV&2Puw4fBZRvgks2o^Ka4h3Owk=gzMUOizg+OM-`$?bQPHOJE0ao0 zTT`H8lR3xDV9CabYD(N&*?My7!8CHKv4E={j~8KTZsy`1>`A!pCw&H8+O07ua}_(g zzWACROmZ$j8iFyOaej%T%m<-KSbLjZQ=14cv7Qzu7HDA|b4k^M4Ko61%-Y08REXEc z_NLs?Ay3r+OGIqRkEPv)mc*KKd>-Tv?TK!TxjQ+y5GWiUOx6?N$Fa`#!ra@Sf_^Y&26PH_x5FbkjZhW67t5_IHbN2g)UW^tV@~Ab2uJNse+zaZ`N@Y)Ca5rshkXYfyD+3OSpG*F) zcxDq;JXZu}1g>~qe^;gjuc*~I8SYg|rfZyYTDLh$^XAAl1^r3JyvX@~fRtwINeBfn7KB(cwZ-KO2NhZubq zZUO=uVu32e9G;;yALjlO*R7MCQMtD|-%#DigA9I?%KSBNU6$%-ib3}CRd(xctL4{@ zT>Foc_dK{(o`Y(XI1P=nnp^O!M}iSr)%kX>^KwXSyZLZ%~ht~II`Q>RIGA0TU}ga zFhVN<3(Mu!-(F(zyf~DsQ}%{*`rKZ&Z!Hf9F2Jk_E_Q6n2cA4pfR*dfK7WCDqd6-M z7Q(&sJNc%Ff^LK!i>724*O{KpSI(p-N%am2XZx04vT~KGTOnUC?yH<9%V|{7OCwcw z9KykJ^xvMW;B(>rG{+HRnpYHN5s@$dku?==X9tgfFr7wA}^@gEv zv$m(hT9<4WVNVVslbX7l6O48Jn>$P9?-ON@!AJ_Ir376@-Ivk}j&wdJ+ad?6GrqVQ zK9mbPl=2icSSnF2W#2oRoaj^+JBw!J>i@vl=KX!^SQ7V*#Wla4w}JI!panU$I4UT` z&x`HjHMINSy+9RIH0OUtP6y7Eu-DwY!Z5421tFO8IfEL2ZG_Sy`?j>dhrE1irr){)ZqS2Y?hu){7B2ZdH__-`i z*PCo-wA!kGkKLkH^5%fK-alcCfoz&AZLbXI{z%l0c=*}SUqL=;@%S~d)*C|4@)MN*Zp#!+^)w=bwR>^wK6K(c_%+~m`rT1KB!7tb z$3KHeceYmr>rpl*4MgX-LsIcsb7$|A?FnJt{fYU_3LTi^2IE9yybv@R*nju~J1;4k z1?h8c%YuPiU1pFlN4B{O1nsBYwxoHq;}6&jH-WDZAIdzT7qCmbROX|VkCWS-P=6}P z7m*fIA)gwg{^1SKXU4~|V8O6VU^ZD!rzX`>)j4e^UDM8IfH&@!OyOwe(N4y24gDzh zl{jAXVp6AmbJNV!TH8*=&><$`DBba6hG7e5V9=0=;2`@Gg>8mLR02$P4dJ%!JM8tb zEj?j>Mi9+g;+Q;TdE^M*nIH1V*TjwSr&wva8;2^zwcpz!1a&tLs=~M=bRb^jM7(T> za`hffKCvdh%C~VY%Zc7^YN%#iAuB?a{i?;lG1!9-@;+Z9$gNu2U&t?u6G;FwhuFep zm9-|nK6bU6XHVMYUiO^4_HeP)Wr6QHa=mpk?jLTl4|1DDlRhxxH>mbK z@4Cn*Szd}NnV3A-^_4BrQZ58;eW4-CNcXfzh!LP@jy_B+`oo+RQ@ps{UuLhV1^t+0 z|9VZj-vYcEK-xHCB6me$)yXo?{NDs}?_O5Nxa z$Y|3tSHURsyt%|-skpqnT(V($dYb&k_%^^e6Aq4RXA$gOz@)A(@eVq#zgtGaVo|Iq z>yDU|^^FUcR@bq*q!PJ{&4n~CQ6&7yXhX_6t0}rx_u@xR|{>sUYi z8=VghRjG)W3b|ELQ5;bd=U`~V21|K43Hv)f7Ar{`wZVkY7qQ%rC@V`Pl`ZxG*Lh(wg(I-ngR6tKL_zs+5R*jb!QT9! zqX&m!$B<(#)zJPcWH6wRfJ@a(;uU-%DyhVcW*kAHT`Y=jI9Nj*hGPW8KBaCBUeqahp!lMN}o zs{5=6YQ4&+ZH@&>g6&}IFYMH56iIB6Qx(Oshw5Zxn00#>q4s_-^7$MmrkEK+S=E1A zQ|?0dIUdjuLg=3m=pN%KohH;h`=PFK#xt5OTU0olxMpJ1j@ht-Y_vZ=BnfIf(1Cnq zzEWG#iuoH%m{196ojIVQ3iH4QnQu|HSrR$PbS1|??eNu8K~j}(-3FS`T<}ZKi4NtvB=eZ#Jb(;L!`C*o0&=~)<0)SBfOuw zXdjGy6LPU)rlUP_h<#FW{6QYD+I?1cR``=Llm=P<^~J@tbi5doKre>C&w4+p6;a)9G83asXE1_O!lll zJXBE1w`l(jKDY;x+jEssALN5}Yex@Qlhb{s(6lGTj(sl`hxwqYX}&sL8Z5NhY4BlEuu(Z+OB+ zvUeR-4HSx5Lpf%G=gBP#|gJN3X>uEON+ z2$|=iqq9SQq}I%K#4IP#NHpTQ>cj{(?Xhg6&HKU*q`6u$pu~9-Of$oW?x>vhBFnyf z>{`*YB>l2rONW(cOMB+uV~Q4Pf!5e_t+0TaG}ly6SOT&$T6Bkz5-{Toy9Nj)6`41( z$F*hvzU}^JO6Ht-iQHF9=*~2O(xJk8eCIzGwcSDN$qObTg%a-{Db(f}W^VT%B=XZB z3kGBaM|3h#7J%X9-Q{SszcR#auLB(8la>)8b-={LQ*D~@MY)2jk_h^ zU>jnwqr=y`T_L&P9Z<~mCUYt_&jUQk;j`>WBhR=QDgxO!w=ipY02c96r1bpVEIYF2QL#yw$in|W+y{EiKeXf}l3G+J};BYa#_rnxt) zZHH@{9QH+eqj0p;`|kDflzl03;Q9m0kj6&!G?h;|f3+3_5Mi_W)%O+Ve7o8AeLm!+ zCJAV#T-U+Btcwn_39TWB9DT31323~uLN8QIHcVLbNxoXLcAYQQ+P3SJT3UDCOqxh~ zd!u{Ua-3bUx|_DDvW4ntFyit1p39uwLxLXbJ|3Lk*hI7-3%=t?fBHg1PE`>lg!6H9E~CbuKAd{B#tyLYAsJk`Km~vnM%X?1MaUh zwGWq_{B~3-#rbrrxeThOokvdqxNn0PzVJFeySr_HwAwor>Nq0(*ZpWr%`?9S(|tNgUEkit|viCN*oDy1b!|D2erum=eyz*I1*^7 z?xfWJO*b;!q4>3~@bEII zU2))<#EmhW%c2ysBWvo><>*I+E%~@w>z3Ape~61&SMlut7N%^nftF|##xVje^p-WQ zBw~Hv@;yv98h87{|Cv^AQ62lS0(SKnjS^_*T!>h!IXX{RExqJ;IGd2eyRQFv>;CP!F=nV0xY#R!tb5N*8j=R){Y-Jf?R*^rKo`x`2EhL2Yo)3y@K z1vx9Gs;|6NU=`>{=lxj4s*yvXvVnMWSaMg}Fo1@g=)7=$|FDYlkW^zM2wd ziST;gGk|5pqkIJi@t<_L!n5laXz_cue@r)uP@GLtW#>y~s8s%o{UBpA z;Mhv)yAoa7*Dqt(HTEN^m%KSH?k}-y-fO^B(6Z`?f&VAK3w-gj_1HN4jTlEVJ0%j_ zSHQD%fjIk@uMH3Ibbl#XzQXgXdOo@+w5VP4mO6l?ZzUVP4joQlz?=^E`McrumpVe2 zvz6!*`u`o&KlvV%6T{^xcf@I)Yt>$zvvahRVVp;nu<%c@DKO`=BEVx_jI+xj|LD-v+dD0(-QksEVeT1fH~9$R=} z@@a`y$Rvojd$gSvrJvs7QYMpo|3M?VbMHVAlnG4bq+-*e0(RaT)wF8fE<@0Ej zV9{%>5~0N`v0+j>tp09EYOaR?QzaV!(S6NkoFHM6SDKdL%BIMoNRO1^RgiaIF%%!p zYMEGu%laM9yT%A#R>NjN`NW$#4~};`-&XhPUUX7XbZ3}l@;Q|FK>0f+nk4HcK0goS z^_^J%?Hxp)j;CX=`U6M*QknFurMVA$*6f=oecgAnrlyXVnUkA}#ukw1E}^$q)M6AL zk@k2*iRu||78K&-eAe!!CZ(VfjpJdtn|1HzwklVzRd~oUI{CH3qZzIP-8gp# zUPqiqNXFM7#|=SGQ+N?NEvKyn{a)3*TS@iQ4*0f)hr@y)$TB(TqXVnPKkCoe@jR_x zS%&AehHMXQI`~<9rnMub9A19*L7C(dKxHw()rAaA+=YGxd&c}v%UP1N>TxOBJNnrv z)vi2-H88cUH@27P9ulP_S?Tj&p8UU(dgT;31b$c9*E=%uDWuO3Sv5OplaWr-OT=Se z<=NqtsQ?GNdzSCe-wR6fj=XebT9t+Vev(b@%6v=Si5@(_TNY;{yGg%wLYdeqyOQNf zU3irlV$XW{fw4+f9&1VI=5}=EOrr9Sx86`QtT9gPq8SSKUoLN*sR^sgC7w604+4+# zp1lgH*&i}qbLCctv`)$Dv$tJ&Z4#TGXyl8)c9J#UvimE`kCZikfZq%Ry3cC4S1}p< z9Ws8`$NO6MwCUFgbhxS0@1^wxGg^_aG+^<~D z@dM@3=-_=dU>BmgTbzIAZqcj+zgIP)^v6WV`{#b`Mf8)<20x>!HE2S)0j(yFyW)53 zsB-{-v2o<#jnC9eJ)BWvS%VKGBuPKE_h`GCVoQ{I z{uCQ2%PZP2Kw{lGyy0P!1s#2f&23faTBg^Jj2MjyY}d+3Y>9M}q0bNelKyNY zN&(KqY>U#|0T{|!t9pFY3yLc&RVoLFE-R(K^nVsUNNXqx!`fEK5FMALTt!*>Q*NF^qk;6d}$TYj%*7ifn<3AiXu)QWd zz6MbKPKD3klYPO4p#-MA#^?r*tFsOE-zJayEl0elh#D+oR&UeKD9Vm?H??;*_{WuZ zYFO=Vrawh1Z3Ptzj2O5<_JwAeFGk!A?Ji)|A4pW~OjN{kvsV(a4J_*Gb*vMOhgMH#;tQQmr(WLF$;V0t zXs%zYOn6@7Ufx{8$I*T)Vm_ZzXO{NPtY+9QZgSTz=Vs40v8NVK9es;xr+O!_sFs|r z-tvu$%MZ&IL>EIpS9=^(xPGmk#+skD9%tJ7y)k-m`mJq08VrH(5todeO z!e{>g=Y#od#6q@S`F!x3jrIa|Ry3IExWWwLCFkg`4MIZeC^Dpr+--b5c`$Uz%f%iV zRCStc9dhb%rhA120|_8lCS zedJEY4DKamE`S{gr2vqdGD3f4$+KYg*)a91upxkBG^72H`IcAD8_u4Vl^Dw_a&jou z%g}%N!%y85Ba%Z5A1SUp`4U>PzsVd+hAF?wqm>PFx#7aktC#z<#Ffypitj}nucUhfa??CFo3aRB0e@hMhA)alik`-5c!?%Zh)e_7S@pg>E78ji8(Mg1(+@L23tdeD`OC-U1d2wE4O4f6 z65+MR@En}&nlu?L)3atf*Mg(2LI|*rPdb?5YkC$HX<7+-JE1x9WUCB5VG$G#v?6Ke}u;&9~4|bm98WzruIl!fnHuC^}p?lmS^s$n`7v_$x}fN$9MFNd5aoylZITY*=!n zn5B(MU7t)xl+rwJSCrVb%-;_CpFc7kO11PP-stzD5xzqmf-hg9?i}^i8ZD9*DfDI@ z6f*QRxB20aeDVfVO>TaQ9va*w5uaK zIeaSe>Nsv?g^`l*fOK@*)^;T8``|DRn%wn~l1QZ2H7$Irzfq#4M_SDRY3=8+p+ZyB zC(iuk&I%^jRBg3)KrQ~GQ!{!h#5zr~+&9h8(m~e`l~Msp^#H#g;y?HlJ<4OaDK}rH zY@*d_0i!CL{bNmhq9E4<6}5b+PUK@^YZlGqo5eRJz3?GBDLKpTTq`SRv`J(}@Yetk zsU|2Rq}Q#r3^0qc{IUwNtS0)((50{0hIaFJ&pRq|jze6!3AkaLOgX_>V(M&b04HUBZfW0Qq30>B@?G}sAMoI3Z>oLq zR=4h1{dNp(`S4vh{|9LqI2uhR!d3ThYVE_aiyi~KXt#v56d zR+s7PIeTC6d%8R(t|_E3S!c89$i1+yv@6Q3mM%H{>O#f2^1Q+d1}sGKCS4gaYUlIO zH4Uz6vhlYO`F7x`SB}w29;=c@=zeI{4pmknNL$?my2Ofj$iyonL>okqdKdFivPdki8<2Aw|2@^jQ5tCx0>uB<1mNu?t@ zl~rcsnodpV)moGI)I4+59J+E`Mb(s-pg&oYLwv1cMii*)Sjd4>@h%YpZ5lhHLm&M` z-$=g;Xi=>& zNze1i+mT>`5S3I7xXJ38n-`P^L z5?yVm8}loiTq0m-#3ML%kG$H*OY3;nngAbGPit+8v2ML+0$8L1-j!-i*MdqF9)9tf zDK&ePf#WJ@&s$hdHj|PN65W~Qr*I=v8#$=14@?6LMWW<|DtJ3Ps(c$h#_f6z?hN`t z3?FuLqmbovcRPD8s_Q|F49Cj0vzf)_H9k$LYfmMX?*7Sl#kS?XuAQxXy_6V`_ey1e zY)Loi|C6&@JBT(MfUoi)=Zfn~YkR>e!f!hH4FuZLM$zn2&=R22CkWC=S=;wgg7 zJy|7-a=Z8Prm>UY#R;eOKg)?r%TCOzcBB_(YmSOHXIVlH3w2AU%*SkjBR3@bL&SO; zax$gL{l`M*C&!Gm#{~p3ZnU%rjzp1A}=-8*Ar* z`UeE;HC}2U#Gm>IY}O8>B#E(tXaYpE>cun(OI_8^)ZDTeN|_Fi#J_&%zM2w}&fB%Q z^&zyiO7)>H8kD1J(8*A&ZcmmXIMmoh@{E>iPsiU9jOU!J5&u^t75~7a-Q}X)i1_`3 zh>3Rof2bt)4F3PLIE1VH-1%yf|0wrPA>FlL{l2Tlx;yURssD(hy9)|z33riP-<^?k z=QUZ%B+{xX5pv|p;&6alVu;<@;+%E%DRyRl&mFzi+1Y6tuy_dpH=cem#K>h$M=X3o z;c9PynbUyLUQC&7+wFPn&B<3o*4DJ?xz*a6o6pb-85lao{WH|`-^kLSE!fxCE*evC z+7Mz3o_*3jQ{cX%eZJav^XvQ$hgVw;8*Ce$@W#1+#-7?heF0^I??Sc)?Fcu0Do&Z7*a%F2%ic9zcyMo!dFBE{ zGwHSFyiW2QU(w9{rSif)5`xXs{Z*NdKuuh(tS4C497Wt@axlR|MCB?yKMa(5GHXc( z*tdUQTo$N5gc?ZaGV?o{KSzcgEw#ZSIKQ!bd1k$<$aMP(a`HM=DLHHRzJpWm+8L|m8-EU3Q8xJ$ zcc!kcNUQfmvEAueFePn|4idjeE*mOe!P<7?xNhPX-{PDi$>7n@b-@NuJlkz z{M@Lz=FMX3jubx*Aa3k+6Vaknzh4l*o>H#VB<^nDG3X(zV`95x_gj*`^;JcQ*0{`MnXiQpLOfd0 zzh#aXmkIDdnDS-c3h)lQv?M-yq_YPh8&)!`qcJ!oXsucD&LQ?CcXbudcmDRmBUY>v zWdCBve$tQ4NSMor*`qg^SAR94b+4n@;8W%P=Hq+e=dKuh_BICb5`+p%>r8!S3R5oj#J1)epmb1L-^3@_Ll z-%wJ^MC(>{w(wjL#`o7?E1p*d(AY#3P-$) zm{Rg~ln)Q1B52YP5(b#Br)x$0G{Q;}{$2FUiDKu5CQ)ySsv3M);E`sj`RXSfG9!_Z z_qrkuSwHpdW`3VHtD77*NI0Jw3k`ONLJ_Ro`z*!NL;j=k+rRHP4xOsU(`rn}Qk`<} z7^yV)uz-`1#j`A6ahn{=CSd&c0hk)Je7-ch_8Wx|Xj`xaeF;T%lf+%r>^_kDNfnT$ z+c}w3Blm}-s3D%8;z3O|4n8HOq`bfPfN2m-&ks@GnQl+jv=i$$BP|%6FB{q^jW7Pi ztGf7L`Ni5jIlB0dQz&ETRkf}TCu;9)SlmcoAI0|(9?!=TZ)-p32BKRx?S`9XCj@NP zVv7=HLJzzk7Ky#%ezVMXIHQb{B)%sPz$`5-Ku>VnhKqTEnua{=UP`h%bz7vZQkjr4 z*6`nX{ewp8!7t`5q6d@u?CrRUNi)@4lZU3mlbRClaHr`$CdVdYCZq8jj2@}5r94f1 z+fkJE0RGq?R4nV3X%^@@&qnGBres;9#r8-eg6S5ud?{^eZ6)ZFXq;uq0l;1xe` z9&-{O>~~vZE%}8ru&%P7ICV_EFpTgtAWIxb>YqFVB{K=8E0f8qkr@j*EC!$49C>)G ztod`%&Xh8hCGMDIEsDBV%-X=a20($lxut?>w01yuxnOdAudSrZ3$-0U^BZd2x9Mu+ zMp6|)fhB-Xu#02ed>03c-VdEg+*%rkBEHX^Po2dF=t-uO`!ML|AMCF*{mN}&vh32z z$GQP!PH_Z}Mtc*wk$=b!yP0JTaXu=d`&Q@Dk%fnuMoEU-tv_UyH`JRef8`EnpO3+) zM+4SX<5*m%-DPQZQ8U1WG9U4=k2f5 z?oV2#h)zbipoI=RdM%M}X9^siF85r*?QIg(60iOq_b(nM@5Z&QJ!Ey*&9u2KMK#?o z8rSk>31tq7RfCI{E&|uDmA0ZSK9f}))sftOE@~>Qgj}E-?T-mZlZelsLe=EJe5L`z z>iJa@;-(V4E$K*mtM4yLFRZt!%A0lb4;+f$p7OxlxOvIjk08XoG64%mlK6jJ#gnlf z{@SJioF4Vc_F8-j<8>tV9Msb2iv(@mSGBWkMVyjjBFoDG0?F;GuMvq6%RSwG_P%K` z^`MI!sAeo5q)D{#my*l5_6Dx9~&6fXIjtliR~@rC-m zG-j5rbs<;Bw+gB(?R=lLBwUVKjiCVJu&sA%4K9diV_R#{Avci8MpJAdcRDlx(~n&h zGyh#Ibo_64+%0~qqAQ(%%+lBg*K*+7uxZ=WPH<_}q_rXGonBC)5#ZmPguZ zsnPzIOOLy2#VX*Ty(5~337BX*{wJ4ym(AUA=>!ket+3*(V)pEJOw(P#!kvne{yzj= ziNmm7fsNzt!;wpwv=FeIE|~y9bQcLjy5B+V_F31_Acc^N*vp$ z{Pp01De~k!&2wV!19=vaS+S5&9~OpJY&iLzkrufya$LifAIawFdeBDZN6#yBHZOfm zUw~?G$FG?sm5-JiDVZ%HTmGO=_{BC?)3u(j+3zMaBNl`GdF`X`1`m&RXrCvq}n{3L9Oa-0LJt`C`=Zg`<<`S6iT{K0wcFLVgZ0yo}~Pe%!qxAf?66 zTzxDk5qSF=&;PGnbLAsf>dVBn+-(HyWtfpaC$k=*_x61+FVf4tU`XY>W+k`HntfF{ zB>6sI18CD_M2XeSn5If0JA$e-V<#&qYdZvh%2E1#&RQJzk2={^g9#*GBdP=&_8-z* zNV%{-Pl=CH=%*sP>#&!4_}MZd&Y6gNq&v!L6S`~`j6^)zspP;$SiLZBIe7(Ok>!to zt}gwQ+)g6>9$%2(1?~>&lX}j(nAz$y=Y!~Ox+kE3$6rSuNaP&uw@rgs;i9rKyFXH; zfGW!Rs7PWNK?CCunpc-&_IQGMaz$C8KwuY{G?9zf@SdmmfbKhw1(WF}O=$<0i0N2( zi_~3wH!26=Zl@Gzh@$q7rLefz|HNK(l_aOUUiCZVqb4Dpqbdo0y^KTm>%UY#s_?9h zD8(ZBg|gqVegQK&g&!Y;ct}0OpLKN;83~j>q#F)q7wwe3a$<2nMA6E=PH*^02(xUSt8ojC1^6Y87{A0) zw6De*m4lS_4+Krj;x&7gg~Vro`<&0EI@sH*)KS`&!~qx-(j0sk{p)Q|FM$<(AwcIYp7`;eY8u_P77RH{s zbwq8W?AULl{ddwg7md4yH_k+iQrXWBB^EcIMFej6q53h&Dym2Yq2Uvbwf)s(Kb zqK|?&I=!I%zHge^Dp}{iD}_g7%%7I5o9x1Z1s>TutE4=;ZeywrdHlWyned=!HO0^E zhiV;I21|7rCD?=U9ByQ`@6P=g3r*EJxFf+;M9Q* zS!j{3u|G9x@vMajMLaciiSUg;(hb9(2o55Ze*W_w`hZ{leMjzesoQ!gd%t%ad0xID zq!xu-y=c~|!jx{9P%J{NCz3Hg_-$k|E$1umP@}f9JCjUh<0RzhUU5-Zvc?c?`>F(( z8oXM{boBVTU|ya^zUJs*{TbD7gt?8K=GyEwiS!Ynv!|SxIy8(TwBf;Ef9P+^u9uS0Y_NoE5rh5uy^WvdM+gU2VyJ`$vlM z-StK3j}TbtJS2a{XO7v%s<>w98atVXLk}Pj5=7+Kr>fp^_dv6Z)}9&jx=DiPX*);d z?vHkePaQ;yF5#2EBQ~dNNPi>bnVPAsB9_bH3Ha@2N`ez3hm#Y+Qnj(4&*aNr%gU4CTPHs-2o5Z9*ot-gHX9@AUp-!F;u~6s z#eca<3~L$$D_VpdHS$vbbtWdJC@R&V^+awzOcN8A+w&4aRb8FT6KJ8B-*fB=YGlJt zQu2K@FbAi6sPuak$q{!f8HSDCV~s)8uGm(qV0k9y*+dLGXFckn;{Y8Ggw#nf);SD$nt+c<=pHUIfwQKQ6wqLsKhUs@GEE^lypW5Qke=;(YP z&1=vj|JxGmtbteaz0)F*`Ald%!)Smgz|TNte+7qTHh%791+#rJ=3WAkmNT<`$DHp{ z)fCKAzajX%ZgWr0eK}| zIuizZA|N?Cs@n#k0MZ#?Ik5!+W83-V5FN?8C4_$yM|exSlkotTbq#PqyHYJ7bis{s zuPwFF{EO_WItWnlc$$+sJ<07dL<=W$74p&dJ+C%;BQEx9=$y?Hq~cHF-a z95Q|gVG01fPMQ3z^WL{<_V?t@e!UxJ&J!}r>yI!vlV|mY=!d^k z4RFIKoJYPW34+su;|~qGTFFf+3vrE^NIM76Gx{Tq$^*P!neJq%s1pPP$eyH9Y0l#+ zDVNuTw$jTk7?Wh^XV`nB4O7787ij%dI5uYa#wEkVwrtkw=tcSW=R{lSXW4GH zc8ogPWnV)*X(sP!yM&n53*Aul!ioy^bS~gOjQV?8FXPyQ#|y_p@)(x0@~h*}{h<>D zaNhr8>ny{Xe&6>mjnW_>jnt%(kS?V}0THCThJbXZfQWQS3Ijnvx+O+;xzSx?^k8G; zkoxcQ|33Xaz2A>^+{dxwywB^rUMIkF4h$Dz=CI|_WlEpCuZTH!BJTsU=O09Lss#*r zxy&27fBW@D9#x6j$rsE)c@*th-Lc_bw{EvPVy;Q{?2%7z^rl@6EDlA)Vyh}pZ76s0 zllk{U-Tesx0T(XQbeqrqh1p@~&rAbg>i>(UgEDbwPq=9(7gH0kNvf>>Q?31vVfwRO zoa-;e$LD!5N-FyL|9M^5R65oR`_BmDp!^1)#2(nG*r{MmFRWVo7XM#R{o4h|?QbQ8 z2yiVo2&oWMbK8Y7H8C>szueutNAylDH7bNkR8jbroM%F({~gz$ug@Fr{&||9suIty z8x;Z-s=7jQ=F%JL8WgL#!m{VC$EN>zdzoZPmiVB{-Q=a&5NnMJvdHY~^|sTYIOTvB zpiabe`YQT*{e7g@o}w`WiZ#O&;zg|hWK#{33vri`x-ZTB+&86M{D5X{bA0*|uXwD^ z%iL~*tiKzGMm?Pm1_d?qT|sbwCBio z|MNr7@F&4S?YxY8RIbXbb{iRfJ(MhUr|xThb#z>@gwQbx**#+ZZC?TWoHz8-u|f;V zYA#glJuAV)@d+HSix@fBQ4}T9^#Y-zI|y75Z)WAZ*f*=h zgTAP&FIrb=?*Xv>jrBGXpu)3L!ZR-*unT|H*!Q}2UDyJXrxG15y@5rG&lEL?vB8~h$PDvV8n6~ z!76u3_Z`cV_8=wtK-u>qIVBD9ZiA+V*+9!hMXe>aWM`XBoMX(@P5HuZ*Sc7xl z7B)SW?)6;Cow}+nLg`rFx3)N3u5E)$qgPPwmkujsQA2 z0M;afVpwLV*M;~9rlg0gqus|k_EuYG>GraAq#m*ejAoONczk%E8N)hy?vIo6Qtg5h zEGN!z8mCPjfOht!j{Cjn5HePmatpp1n~PiP;d(q6Frf_!G&OEn^*;8vCTaWThoXDF zi}XjlcI#1J7_ylpNA399zxwS#aye*fIC{FuHzq1W(%r>xBrxrp9_VQRG508bVT5Fm z`ciS+u83UXGpr*T!1wj1U)z|Y8gC!?GvroDerKMi<~&yXKoJ{@>&bxdnY%w(>R~nd z86QUVJb5x8x3qw){P=79A0w_P8WDVM@x8!@`Kpg@s7PUtm~(4VLl-~!QE^! zf85XP?|n+%#LGW(U9_A$0uFFx$O+(d9UmRrF8k8vCBmJ{(2v(9*o|PkjxXf$4e63S z4$RJn!I5-+H?NY=Ry zCl2b_S6$M*6a7cf?tr^X-Vh6Oc7O|!E}&!(cPE+p$9R}p#M2*f}}(9g8&LA5XdqKw3g)Wbad z15i!8w$~)r((L1_o>dvfvyQ#IkV~7DZz>}OmZKC4)Zu6Y!&XNkaxISPb z5B5@x*3PrsNH#S?v?-3}YUzq!k7xQ4-oj8D`Q;8x?LSEjy@#2gtIzI7h|6_0QRVK( zi8IiObjh^r9z}!bcu=2zc{0+$fLV#Q(&p)Wv%QA{(R0i=YV73j1}XE1RW8n}^Fj7o z*WGQv?798bQ`}#fGruHf3}VxrtHxUhA^k)XZ|M52tGVFwCxfAZ?mMJ4qYou}DVz_& z@vMU^s+B}<6W!s0z+m9%40JanZY51oGDkm% zJlRPlugQb(m4AzAF<^{2eaW0x4(;yydH2lnbPD~L!zZq7tUo_#`$fpK>LpCNV?>wR zQv!ti81`7CR`CvZ&;5SCMV$-u+|g;(99x&})L~E44P&4=iQ)bJuhr zCY=?UuOJ_{$NW&Jn6OOQ3`nqOT|-8gG27!4Ee+&Uy!EvuWim%o{)sH3H>*bosNAjj zac!6m?xMOARLCso*ZW&zau`0ZD1Ev^9g%>v?iu7p1dD1VSwg<MiQvw}eZ1*1ao3$3mZQv}k%bcJn6WC8a&+itj$wJxO z65+|6A0Y8Zrjn}BcJ^%SM~CDK5vM1O8P%FBHhehtSHWS@LK0y^?ax^}&tIx=^O&96 zoeZd6skgm7_WdXHXY^{Jfuq%n%(GaLam7Y(18R;eU-0;}k5pjwHmbfMBNl8S8lNY7 zoluGaTOWQZ_=GTjsXMjY&T{X-tr>?kfxn8QKbq#JF7UU!Ez~-?! z04|6YC>#6$ll>~S^b5}VeOj8_2|4dMuPTlH&YQRNEfn;quncpzF~mZ4edimgokH)D z@Izv=rZ_t;;+GSl;R#5e>C@%RO8TAdY*ZKVwQ2mRVR@5e@s3omq`H(zNp#4f#Cdzk z`?`Cb!jf42h9dNGUXh>}*0ySTc9ri#ylW4#GnGPvCAv z*@vGpwJSs3xu-u#dNT7~!kk|688EN}C2=KmT$DQaQh~UlXfxZTy4BL@$@!98)XS2T zgVmPRd)UZL*k12?7Z$0qUHHCRa2Vj(DcoDFAt+nV#VO)7Z8o5HYggzD`!hRhL7IQZ zrfpNuxK9i}oeS+`?4oU+SDpRv#g@g_CM^2??TEsKpvaa5@GYl<>-6NI4kB{;{=qxK zN4C)aIcP|1dk=<-_BQq`8e32}@qbJtKjl0%$}pyk($x(!pI7B}5uqf-j|^`KcNC!} z#*c(is6?bFi98|3FY0k*cNL+0K}r1I2^S^l*Hu*KZUBVDwH11AhRh4v8DD7?JM{os zTf@$vruPXX&61_)q^=S)8X^t})rJ?Xehj{_W(>oox#$zYcaSx4A=bo>{Gssy@wN48 z@hv%vHeMk5uegGA*5JD3n>+CHQ=i;Wi|pnuE%?;y%ZQ_qq~~2mT+J4VSBBG=jZfT) z7$74mBzrKL4Sb2b)&%=V+_f#G>oju#i-_4QeB}XaidEtZE!Q2?$-aKAChNw$Vhv5z z>bPQi1H83om`YOizg;Oehf$5!77i0*7kMFk)Q|{kspzn*@USd=F-B`%IoQ4~w=F^) z%71@XDXPrGq7%yicIN;M#vIxcUb!c#E#xq?yi{-8kX*>vnI-sf!W7#5;669bf?b9F zkQ`5=Z+TJ@)INDG!d$}?geGc#HhjrX)Oc*=imJ+dzVJ%+i4_@m0=GtqQQ`CMM)BTQ zLN~I20I=pfQX5O#ohx-*#R)!C-Fw0h9V7hbDU;Rn)nbABPUJ~n+SL)28xi2*gja-+ zwQAopvHjXn(p)QhJ5f*gvBhyM^$X^pqhhPU#F(1~8ir3sEzu?A(!LZJPRZI$czWYNwp&lXCB0p+ z8$nbPN0d03LYZ+R`#f0nx4%wOY>zN;x$>sFgN8k?nR`u>TKDy|5tAq6yo^^9WN3aIXEO~oSFJS}S< zD-eF&a;qo7+%n2+E^gMrRa*xvLX(c8F}wEAlbA$+d+H5P<)Iwcq)sJursw>HY(C!( z_Q7X~Ep^HrX2jTO`P-tL7BxU!1`w8_RH_aEPj|IMIWg(ucK>8LEuuFno|N@Gp)GoA z)RNk9>p`$^ubAIPT;Q*nn})d{znkCui->1rpq0GUlgP#yQQ$TXLY6Tymtj7#^L%7w z>&-Y9MS!Ji9<-K~)R8LVPkA1vm~Ue=Jg7FIu|GSUEFWez_r%LH5>{-qF5$v4ZY|Nv zQC)j1KT-Hm^d1H<3O^s=@NFX`iHJTgGbZQHUpJDYlNQd`P_pDmm#{bd_h2;4iW(sT zH|fqJ3YB&9U>%dY;P926_vA@4Enx0$;!Ewp3pJ1tm3_%U!&=MepZmJ)y;_N-5O56r zY2dYMG=JtwKXi30i!>TC?qHvDHRHVwrXIOTH!nj=$|NxhL^N-VjB`N*GUX39orGo= z5AO3*1xW7Zqs7ln-{+NuZ5t6$hjNmA*(Xk*(Hr9jMDtq+O)$RpZ0~@;6+tmQUAfJI05gJ?)bD?E{Q|MOA$GP~uWrgwD3 z*2BK$TUVZHY~*mV?lIernXd<%7pmH|T1UsND9m1%xfk2!WQbF3Hc|rM>Db6m?`IH5 ze>1H-PWYp zs{WP&U+c8Q5s_8bQWL-Fkv1ya4mf{_>wIXtA6JeX6zX}QjSdPANpY)Dl08gcaqA$y zb!wp*3!J~R$aZpd{L);i#>CnAS402GN0`4Z)c;z)j2RJk#g&&gJ}2U%V@Kxs*_sC$kAw73xrU2v{tH+P>SMt zV*Kn9=h|9D;8_$euYECwXBw_G{^YiIggAHYsrqw7_7sB2q|rvn64q|VF>ZiYm_-Hy zp7pHrvn{^zPv*9sft?Cw8Ipx`yFBzC;-f|wTIKTD3|w0z{wpugb04`$2E8L)22xW#x=sUj_PEP^jl!nkYlUO!#ZN z1(rZE#Kv-^hiiUkkLzxT&SJVF^rl_ZcJUUeHQ-^8o2Yu{o9hB=Pt6M|)iygauD)LR z>G>Ao*LXYn*5WxLX8~QPp)_F8Wi#@!`8;TNBrez}_X1~bSuql(FISVEKb}wjH0+!| z+hTfXvmbhU{^G92g@J2CfA(nHkLbp(*ubmy6u)nHi7{Bat6t*d{$v6guX6{*Zgd0w-u?c=x0WlV%R z`Hy;gDJD_BwL#*1i>wB%l+ zqkVA__dsl4fVXH^@_XGHLyj}erGMT>I3P4`^^M=a_q}omXnpjii}Jke-*1hJDnEII ztJ1$le$`%0!@0ZI@51k1ArB3bAS-;LX!A*YZ`(FG_u2 zbB$~MidBBl5+CGW%)z|WeTa!$CkT5>|9+$;(QCr)oS?6+7EiGhQcn{$kPc#67+xJ$ zcefk_Wcz3M+k;i6`LZo6sO)>b>$%$8z!*}|40+e!ck2{dEPJz4Ez9)k!VSKb_7R(a zRrraQN?}2U@1VM|6E=|16+taKVW&zmJRrE|Xk8j+9;`d+m-vP)logrjsZGAN6Gq2< z!4`}ZWOoVPU@L>+{5yskzWe4)Qhc+gCmr6~QG+)dE|P(#xWio5L)p4Pq=laCEweLw zrnpC9ZiN>d6t$z)6~%+NQHEOetq?%xNF;4&HDZroRJqwzv>kgM4L~Krf5$X`JAVuK z!&Y@QzW7g6xDti}3v1HAu?QFvqMz`f z#tS_e-A@ewi!n@&G^e|`7jWl``3n?oK<#ibQm&LvHDW( zyY11Kyqhqb>&7;1PMFyNSo1iZ)Z_u#@lfaleHga`50A^YB@gH;GFEEZp#YazU77d% zp1l-uk+siZ7yXSBB6%-2Yx!bSUHhwU22{3j8&{hoxqM(lSHGl%YktQ&$Ur&`aqEToiu1x7)*j1ww) zm7Q^E6K8@~7ma@5To^0BjdncaA6Rg>wlW@zGFjy~#o3YUBqn+IIsTikeJWN(fyQgK zY8afbGh%36^?CIMa|cBgEN11K0Q(?$?oc%(mSVk^_QmIB9s?uc=Fv~>*T2@6_`XT! z4q;2@3`!m00w33LP6CO@YLMyF;sr5d5?#M!#ZO)*KMM!w`@CJ(FyROcMDnu{#c&Loz;Cf zBnyDinOniw37hw#mSMSK6Ll9=%I)JmvP@HjXIYx{(SSC^y1cvBb{vwVi;1MR3Q3<+KS$Hs$IuD5x$>_F>VVY

e2+aT(eN{CqVPXM3+b2d?;OCsplV57h}M09SFE z&mQA-*j4}3G9*^vqU_%JNnx1VTw(*W%cfY3@iiF56OBkM#sAZcnKb78OySQ#0GJ+l z^Jt6PX#f-xi;#yVAgOFozpj<{ z3@PN0JESAxq|CA8qpB|zHa$r2$C;^p{d*Cp9C@m75KB_6LFw9#zPj-VS7y96WZmYzfE~yuwu9TPl${ z60;Ko8uVwxvh&ZEKSF=W*j$L7{fJovPi^_N!B;^#lDAd$Kv%Ud0i5_3qk`|Hy3&+u zaSTH)6rdjpVgw2p6=q441Y63M!o8Lio24{44ISQg$Y5{gJf6SMJ3^oHI?Oi0Ark83<}J_s37O$3lXJeA z+sZbLx0uS0vsDcCo_|nS- zrNVv>H)y5l?Z-7`2UXtF@M5GCt`%+Kaf~3!JI*DZ1)AKN>5t5EVfzAvw7k^TYSOE& zwtGD=?)@w$v^LP!ZhTb&$T;HG8N;BRg{T7LXT04U^>aDJkH56#fS= z65#hvK+|qI7&=pkj_bOLPK0x{z2Y=7bc#;{UAAur8pa)klZsbes{Cl;7(a*|)1~L- zKxu{z#fsN#)FcXml3VaZCAfS9{$9^Jl}S6hbDAUqQm1LrhgF~_uh!GEfWmWbZ6`&c z_BtaguWoUFw>iIYQGE6%JNhtF66$m_nCUkXZhC+NJN)f>twS5A%aBGjqicf7j$xrS z6;G8^2<k@@>t*4%Vz|vPsLNGP zpBm?|)6Kp~O4}3CTP&)^Zbbc=2|Arq(`K04Yz=RC`W_4#x%Uz2c?a0Mw?0l7!2q}C zyv)}T^cIFcTm4Gc9j319gJnv0o3tCg^&h3bhmds2&7g2WEZoA=EX+55pEd^}k2EEo zdo@S_MS~a8LO(_+e7pKEHk(%rISd4w4ZNBi8URIo3fwYGpUIv_S=Nng7alAQY;LnD zUFz7<{7)n0WZ=-QanmM^m*-)jq7nc1IM6*q{qcW&If79yO{cT}TNw<;w&YNy)BYFK z!Xg$#S@1cqXV?iKAC{}|Uv}%;e_44Il}1zuWIP}>E6?p>wL@|D4vF-`KCZX(6>Dx3 ztIfjd;uYD1;lhY9e>hz9?oU&^AFBXdCw~`FVSj_Z6|KGl$9HuZ51OuLcpO=WHF$0> zikh`5imhj&qjknGh_#8uL!nSh+1#3>4D`kPomy{&dU5T)`NI%HC-<$(+4)!2K8*{O zgFlkSNIITmtE;z|Ntbm>Dten;i9S0z;}iwVd@+kC%x(~D)6^F_@Lq+|5;puTH<$pYGKnr8BW)7-VC5&aZ@?K0Rua(V zO{iWIWUR!$_nP#0=QJEcp#(^2vgBHfYlc4^0hC5AFEH|NFje*-TvvrG#iKZ-D)SN= z)5y9N6|PF0sk@S{nXnK79YWwQ0-9e@t(*Bnb)s7<~TBT`oHwVbWFu`0#<3i=A)yg< z`4UGqg3uz|1tMn^=hAN;ns4xOSN!Iw=pns)kd()xrC}GDN;~1_iCdXs6eitq=4I?) z)z9WekA&X2OW1J-jTBenhc9T)GLQU>U+<-gvn|7Pzux_Mh&5&Hesimj(A2(pHU>T> zx|CC?2ztsH|7HI*Io~V3;gEfa=ZL-29R1j;gVa%3sSU$^5;2vI2C}&==fd}_aTJ__ zsc>D$X}|f&`0x&rKn`sb+93G^X~5Ug(y8?H#qGNU=2Ca>nuy$u&tDeg#a_}nEpRuv zxr77jcUehvHmgo8j5B4Z0!T) zV@XfIc(04I))z=tviy*mJd3>caztRO?K8J>562vcSqFoyjg!vlsjP7x=-oNT?frE% zAgFtw^RXSdEREZ`pu+eDo@0vM=M(pXYa?v#E%j{Nl)lVfEsx2ww31Sf+r*vBilU`(wrWd8bMY1H5&q%Z=I!j3X5{(kj*Y z&#%VsMIkO z>5NT!Di+iuNX2!JE$$BCc!{WUs9j~tp=cYs8Y*?Tmd+}5%1@`~pCqX$W-aTxj1h1| zln~^U{r2U#a2c?uZChDFt|Ihl@Jau4U6$1UD(#UV6Kp&dS>r26fEz z-hP_2QphQHvWj9aaY^>tRKDc+fkJzdTy{#qbz7NM%CM0N5RU2Z`3QxTt5H&CGx}h{ zx2wj224yp)v#S;`fltAk&6a$n&Kb_?^)m7lm-=luN+7ANCWOtRZs2khu6YEKW7tq3{>`QVVH>T+49CoA{0QnTvcXNQj8-t_Ki8I6 z_3!0hd@qSbnX+E9%SDriMi^3-Rua9sWE!^}(l5k%0NOmEs@t=#8CH~b%p~j0osced z?Whf(AFX~}wNZSy#ub#+{`H6Di^u?v^ z$A1hzqV3cg#{}z~I#A}u`-V^F0Jd^zjm>Xp);VLn2o#}#9&)QT{4F;S6U-buT0J@xB~zd|Jsl9Vk*yA zN+-~7xPSjULYPk=hjDJ(a5M=m!-|sJLVsoh+tFV0pMN2zsUU zE@iRMiZ;AIm;jl=w37%?@Xg7lZ1`Bc^{p2ms_4XL2oZ}HrC)t(le!jE*e!3drZwZU z|)w|_)G>mcL4HwCM`VVP-FsIjdt3+eN- zS(v;x8h(RZ8>gf~$>jeFUCFk>v<_nu25ay_!%r9}J5s~*y|uKTM}jPmCJDZuxE7yE z@SBtUm8Z4dJUdx6T$7rMQ!9sgoS)pkR_Y6h@+#tl+;#;qtsieGsMp}_H(3bUT4Z+W zr?zXh{~qrY99dq+bT=K{Klp-XLnZug2snpTVjJAFMezeC#GeooMuuy!Q~%esV`3I>}pkF z%&=j%DsJ&~@O1IC2m9~pRT&`W%W`%Lw`|zWORu}zJq!r@)UKE#pV6j#0f*kVF00SA<`A94{egv?jL)t$8_NQ_%Hsy_Ap>jfuC6jTZr)tfyLoV?7p1#uZZiD(3IBe zPK`B_D!Bq8iLG5p?}^7j%4{nearQlrg6in+Vf>D@^jB!IiLaa}w0C+&r~* zSD_O$GrD^)=9cgJZu8-vJz|1T__7MXaT8!fBrauqSno4If0oSrB?aH|rL3Hsu4@r7 zCAbid`Bbw*MjWY_$_@~1NHzL0ckvp-5%WBpJXl%y1q=3Le{pHuOM*cb$ zQOc&{XqDA)CeQ<(0>>g|I)OhL!B}5jSwUaVH~gnOopeAF z3Pw@zr4gO>^l7N`s5=qAx1APb$ykDbVVC~QGtr9dq=7_+{k7RPzk)+PK!F|4;t}KC zHw#E)qM%$Ff+!D>!X|_s4}V*@eN1^MQ5u&VbD1cKX&3Wx?-bzKBVkm5FE}ryL%b`K zT&@Qe*C2)h5xy}#>tkan_%ax2x}=*vo#U%5slpy@i3m~g7*@#4LX^2(U2a+UFMgaB zPOn15hsizw4UHYAHxYw)R74RQKl*tkVbhF=Vuk8SRC&(U_rFv9B@^-XNnZ{;&~H^3 z17V6O>Qs0#8@aLe6+tPn0B@rtBAIK9040B~UxnqQ3~Q62mY(U|p@jj@63S3`uVG%| zsdpuTnWstAkr}PlhY}5mAwMIw{MRlFtZ{w>X>mwgnB6RWIzvM3<1BVOR2o}!fb8kx1H@(Z_e7o+ycZ9DD8}h6#HfBCAbkG+kUvcU?}z|6nJuM?;d7()cxolKP-8OL)sp%Zf0ck zPUUBAU|@1{s#X(a&@@;}Q!Lo9!pG%T`+gfqj$8=qB=N`9^8#Ti0bQG8O+oLrZvZ+E z6b)*%5;@WyWF7;xYZKj{f18%bPa8bVJ0_X@WjYgE;AlyM1>Z%dr$s^e0s>)2Q#eq} zT0^x8=dm??eK@HC9E8ght_YSA2K^Y=^$|*zmN-DY|3*8^nL||^n0fg!-(^!To(^Os zH5zWm^T4z?QbOJ`!Y%M7tLq`YeWDwM1*jn6Ggx{neoNh)=6*(B zM64R_v}Ky~cA@qffqWE*0`nun=-G6}Z3s&C4x&~KH}A7o%N#V27N+0j^i;I?QY{Jn zAh_SqV3cFng4J31H;Tl+-_;FBFYW=Jy)H2 zNZ~{1+UNGN9+GVNgnpTgkwCjQ+E8Mf0*;>}C8weLsy(ZtABImV zk+VGgT~I13uq+?2JC?lbpRM5RVvHS9xyQgI0GQ_fQ+G)~#98E^$xckER zycGe}pD91;LkBfM5=P=Q5vHNntzMe0AALHp+c}S%I)0bH-?6+De?|TKidAr`eEHlQ z#82ma7SbWhD3EH}3lH5sY$175VoNBY{fw@8T-q6No5yHzo)km_4KM?=_zl)C2>HE* zj>*qH2EC#{to+jM@p^jL>o%v9@B}$ZbE7BL&P(q99aW$9X-LTZYp9ICU-@>z@3~1F zfVAeCSxBDA-I&KaE*2dr)I7i?TY)7#VmEfbM^UKS?-u@%FhwNW&?lnQBD+JR%!og4 ze8tS3B?G+Mwvk-6V^REfK5kUnuckxK2}7FUS(AC=sauvTLQD}bWjt)?Wq4kXs0(sP zcN9O|d`o}zffm@NS|eVW9AC7Cb0+u7>&xq?28%Bt(?mSgkETk4EzGiG_FS)caEH|< zbSO?L#DBUeUXp#|FMR7VlYT9qF@Se!N&a_gRrOz@zpDXnT;y&fJPb!k?KQCCe^TrK z+q8qt$QNC#{%4A{{ohjT-HQ5?!Pj1|HBMERvLNi*D%cun4HY|8dsQq10_%#gb7DjC z->{MSdJfTPY=j>BU56dp#8ZenxpoK2-t|)-hy1Yhl52X*e z?ZQtfPSN{I!R_}R3fc;Phk(oGr^i}dYax?m9SkdkGxb4nKe@jh9*Ais_`@b3Ayn&K_qb#1yf)455AS)sOrB zWX$sSepYJ-fxp8kuOedu?4k&J9(r~bvOH7ld={0iDBCAAsGbk@+g@G>l6HXb13L*% z8_n3meuW=}AM95K)e#*Rar!{JA#m%I-O*8e(wYaw$P*f;k0SvhwAJ} zpAg6*5-&Rj4*oqQhCEMHEtxB2RtjIRYFAKkpYhG0YtPq?9^ScLV-3Q^2sVDzzzJ(% z-%aqyYB*2o?wQe~6ZDa~;Q$`}yH}Cy$|M?uqm`t8_D_UDmw#9bmJGI9b=yq3RPOys z{i1Hi+9e-aB#VjgTc`T%@d(^aAt8VhMqN`J6rETr<51hEg5wb@glre~6d=o)u$HGL ze9B)`bj#6?DCBf`m>_23X2kyZxDg#nzq$9j6b`12nQJ4FTVWAVB9eADI6SyWklnNJ z>aqCM-@2CL=h>nvi0&Hf-O|v8JCJMyey(&a{aRWn*^eIFo(-mix5vrj4fH~ooXU@GS7z(v4ll{v z8VouMK1*Qlh341eb!;WYaq92erqc_-I*>$yI~S5QTbL! zbY4v58tp^Q$2JMO1B3e*hGp9CdAjUm$)$9k!LJQ_Ij1&M)UJaEqnOiOE>!E#>E}*CGvtTKDwG1NeY|>*4yiX1THo3*!!m^ftZdcwziAEa*U#ow{ z6e~-!?!S>1?bU#{ednAV1362-8j>(6uR0&439*JPq)L2Cc)sLtxJ}JaEOFSQu=pZr zT@5z_oZgq?eD6GWdA5Q*3%zH4t4G56g)~MmYN- zNwVs;ww5K$^PfCFOE=kRu{;`Kv3!yz(L=RsV{0TarTw-2gGKlLa!OlJ^Y2O$bzkqE zRx9mQkI9Ctm{7@4L!0~qdSs6C`|E&fdBX@REBVKwphsP4*eEr$XANq6=x`~oDYt|wf$#!F!@tr{;L&Un1G7hi7czFffHJ+e0`PSTStQVK4)3; z1Cz?fodw;2*Y|s>p@+U<6s#<4=JLff^_HajS!9olmi$hv2gs#xU!622$xBMC?kA@= zvVDmpxp{zq39)3Qh5E=NveNHh+Pf*A_%6fc0$17xu3ukiL5iNFFqIxMI+YnG%9Oa> zlxwWm*Z(b`^1H$O)%;!j;%6aOSX%LOxX|;UyoXlO?!Z>*H1!|Ikp)HY(6&Xjn9*65 zjJ@F$r=%Y3#~-`iwLJt2X8AH|z?#4x$4{7KkBk$~V|Xge_d!3;vl6G34PPqOeb9wo ze8q2ZzBn45&|Plxm3QWCt&h8WlBxq~SU%sAX%!FDACKs~=G%`B+Y5ZxV|&uq0O9;} zU2P}a*E`*peZI2AH^03!>(@q82o`&Ne56z$|B=Xww@xZ_LnZJ zR83@&i%k^#w}e}KtXvRQ*l>tLyIz~=XyO!35a**<*O6!1hydj zRG#Iw>DMG0E{h;YR7#xNfK2{fC3OFHJ3Q5rrwDF10u#=U)c^VS!#qY}K1VnGyWe;P zq))irG_WjdPIhOmT{y&m2OeS?bLICB3h!^ftTJQ_W%68S9bx~(l}r!wpz}>9pQ`1k zlv$-X54cH1WJ;~0F=?w1Suwi~L86;K@{p#G2!=-_D8^9{c;46D*O@WRU9SWDw6?(6 zhJ+uA-d$(`#iK|Zi~mJCqM zQurFkbL0EM9`)(y*X9B*Dkf$WapJ&-+BI~b{KqEohkNciWYoCvo}&8`f%J>`N~8UG z>mBp7ublQjkXvASB0}*e(6U6$HkpH2yKSHC4Zctqfm4Slnxyi!!Sdo~_M!gj!>;(2 zAWBxiSi68-m#uJ{yX$^vNrPjpHJ~zyqKh|F{r&vmzQB=Bg5~?0nqNjGCu;Yy#GiHq zP4B|bLlO8oYaR~kLouqYLs^=zLJSQM;C{=7C;VMh;(vRKo!E?gIXCS}L&F$0%r*Rf zhq?Yg0lu$@o8Zw;tlYv*AhBgSR4M-R?jEsJR5RUnH{YmW_7+jq_p##^it({E zkwj`E+{QlM29#d62i;?r81r-xHTi^573do$lYBEBgs@xN+A51UwB#I(Ubq7D1O1&p zg-zT`aRpCWvMFH6Ut!t^oqK~nO~UCH`_X8`jz2;_HP@h59aTP-R#ff2BK5?q(}!&| zK!G2ZLLpOO2|XU@7NUVs_fgRXDU+T}kSA(HHDW8RV^u-p(H5wvsm#0B@de@^^$m9& zh@<&!^1vj^gB#;~WehBeYh5^GO_jBUQ}9K{dHc8U7ooji$5<^Gu6m1=<=%#BZnP|a zJIIxcAyJb^$BKyU8!d#N;$quy0(13i1@=u%BkCym>E00?6f>oU?Xapax63j#X^EuB zs!JzOX{Rx?15NUU*?e}2g9Mbt$Pq*T+E!#V2Co;~~d zSq>BIx@qkbP0mWpCdVfL? z8T}zd%qToKS&r{krFI%wAR$`70$mYJjyFsKX9qv8p=19{du+7%hjTqGenlVr& zc?zZYMg;PmEzwvniQwk!Xj117f2tIm4x=72L?4yVNJwTn&HA@3)U^ znCF(-=Z7=NGz^Ci6Z0VhJ3mYV^qm}I^FQP5N!JV@*WdIwO3@eil!J}Al|!Gsdg`G= z%Xodbkv5`d6pZ&}j$90QwN9{||nMJL$0%%7h?zIt3zf`2JKe-XN zGo_oKrt}=!j1p_YdroOE*;%drfsj43Z>@j1BNByfY@_StxN&mZjW6;(7__#peAqN| zW4h``hDU(Rl+DzK)PU>ZvRO;;U%_5KAfw#IwtFwVHwA-*(Xqo~hu_Xq0IU+))Z8oA z$F5GFt(Dsez*fePWjE;WTXtVe^oiFNKJZKv zva|;_Qfxx|O7!N_Pyo}PcFfkhwGw#~Ch3qn`DX^{oDEdeX_rg*2>mf}`U<1ygm+pN z9L45IT-j{Yz6WDpHFg%W$?Ss)ew4wv*SYE|db^#6hQg+b!xhk|PB^xChil|-hd}v4 z$~DNSN5VLh*ej(^z{EvkgfU2Rd z8xIqw@+8#hU#RM|!c};zgI>>7z0A(sgtBnC?Tvc@tToy;Ld^4@o{vy9 zdMb(xVGH(Vs3|WhRYI0daC?EKM`OlgmWAJ}8$7T+1T6&_BUc3E_@5AwD)cc+krqQ4^7%3Y8r-HY;`d=+`f(y(9in0Glwqvz6H zB{_k*i80}EntS{Fo}LK3zTYR&Ji06$wK>kUb`^CIe^4KH@<$brnsUoqq*D$um(_dW zOf&B`E;tu!%M`6BU`4P=C0%)%rDOL+_V3@$Z_J-VY-QUer=?dd+lgIL zCd>h{`;xPk3x@LY+PN#dLo+`4nnd`u!s3&8mh#2QeizDXb&}~&psqrtZ_JIikY!w| z{9=fA$={kFg*oF4*=NR={^zxxCk1WPhL+h6D{P`O5vFfIBMTN(V=lp9w^rLxGm|b) zesrVvC)ru?>0#BYO+kp%g~i+%LEQTWHLHQwVXMQH4hA35b>nJ-p$7ry5{WN2J{Ow> z4NdQH?58SZ@R%(0P0N4Es6RFFTtsUfiGI+lRP#K$_27dvI48Y*NGOc_=Jni>3&1B9 zI10PcSu;5!b^9q1FeK3qr_I``h;|tE=R`%cm(Ep}t;hm3Yc1`l>pnOauU*fuK&@88 z2f;$4V|tD{W|OOR9G0U(qXV=}T{^{M%M4~~D=RvQPuk#bV&04u_Jp?Gb&T%ttv0L& zODOD}Y9rp->dg$gnvWmDI8)5FDvcdk>fjwZO9mGp>%`z)E0 zzgU_lT$`4129)#Ki|_tRM|uCdi=yLSC;mS6c0EmKtDo)<@pUeGhTwZH(QWY*7Bbd; zx7inpOg!ke^866eI^cMq$+_C&YZxq`Cmfbj%5StM{u5mFm@$?g3 ztqj2)t#9wwv!}?@Js)P+;T}}bDl=z!tmyke%(!t%V;3f7Q*q8lcGilt^e@i5olCI%}y9T0un); zjq4YJ&|Wc!H~$Zkx^f2-xLG(e1vpsL95J* zgBQ{d56{_8?)DR__qEoifQVQXdmpeN+v3vFlgEv8f2mR3%k|E_n0_rbl;DJ|LgqoG zUGz~k<;D^KCd>UIz#m*&Hn-G_WjMTjgV2^E!{%A^@$ z&4)Gg3TTYm8e>NG>^A)zDewDIyP$jan`Bm+Q*U{R$f!L^cTL8Vo4TEaQR@$0t)^Nh0F;aDZo_rGhZv4FPnEwR=4DZSgK{lFb=HAhB`9p5#rY5z%l=B) zi`kH`W)4O>yVsyA! zc(ts|bwqGWE)ILOj{|21Bdng3-hAc!`-|az15;sv>qo$g(p>ePY?pgGCE=Pvomih4 z&zX?QKcV)GS$rR1-%B$pSMJrsFby2Gl*`D^-<85&8pD3aOZ9$1YxnncXb#)8p$w{3 z4)oPosfwjaEHV8nzfGR8)jY7QRzk71@a5@28mlW$JxawjkbMUR9A@VBh_us<*( z0`uA$p(jIsi?Az}WWq1ffVc`)Gs$3OgSwZ>Bd`swX8uzozUdT)C=a5gV{hcSO)pns zl~L@v%0X`QrNufFckou|M4YHlZ8XeX+ggXvwg?$Cq*lCawe#Wa2&E}?+(5Dc*TWmx z=vGRX>YoMMQ#F*GPH{DTw{B2Fj`4iuzhQ%A*HrNpZp?aFMJis_wHl|GK3=F*t$#5t z_Fqf1Sw?rv=MsIvE8kbyX*YTgSED?W-%I6~JtzF+X##5DAoQwfcS03Da&!Hz9P0@_ zN@am4@+7+4toB3JKdo$VJ-aTfnY*?SwVvUP=Xd#P{ume(fCX@awK<;zx`)=r6I`k= zfF1Tm-|)F-uiRO5uUu$yaVcDzh|&{PEICkBg5)!AoF4etd@S?qGC%!#bT~`m`l$P( zK(I^FHAXqS32&Z-OgLrDkV)0sA)E!bAyiDi_f!YTv+3s6kj210*Yr-6TBqkqpwPOi zaZIYoozTwV&%hmLnd-D{($BjmSzjaHKF%s!U%zpl5 z_bqsB_qg=k_zk0I_kG4Y+Rk+@HaA-Wu9RZ4nsaZXlfUxO_Sk2CW^7G&)d``+RF8Wk zm$m%r>OW2&+B^;(Jq^-BRiV>n(vRdv+o-_x;JZAh&0Jh9!dFcfZA`Tq>zRs9@;&NUEJ>=bODaWF=>S6bTlQZW>|VEq1Q*U;O3qMCrDVjr!h zM#WBj;>p*P;HhX?^pn-OkdLQ_x{vY#TH+S5*tG^UPLfD8ChfZ9yzemTN7DisFr5^5 zO|n(RDd^9xndAyu{oBW=^C8J=cl6}Q`0bdLaQzu1&IktNTs*#&Vz*dY4r@Q0YbhRC zjBhB;$p#wuW3EkA z*rk>n*937+dpVx<%K-oo{$Iylg)9I3Tg9JdcEcdX;3fxHdfHTZ@i*;CpPQadH`33Q zy@zekw;BzE3IfhPcxff-Yxw;2zXj|+m_-=vc;yeTrfbj+BpbS63(W~b&i`;^Vgak? zi@B}FT3!r6()}W(TiS2i)AM&@#rU`Mv4rSvj0Kr?p55in-VPV6Yd|G!gwX`pllUqk zP1pHqj3;MoSJ%i1+H1Z%l`EG)%2FCanGSqb$UGrT*?B@v-&;>^rD{sfzg}L1zHL7b z?Knk&tE_OOYPQ?S_OE1DE86^#g{S$Hro^YZ#v9#%QO0q$-RIg$WmXG{gpHE;+Ff>0 z&2|zZ`R-4s2xn(FHTTaVr4(j5F84G6DI7CJ=ha5oc)@!!!#qJoH|f1`M=tR($+7Yr z{F`~%Zfc_OGpw3j zK4bO}qoevGqFo>8K4YT#oB02S{1EbHmKoV(SLi?4t`0D8wgD<)b?Uv^I`%Advx^|k zi}^A?^RY6EW%l(iO}V{P{Al4z>ysVJcXV_j(sq3~$}yU9voa&{^%aP0KZ!ie`O(tK zW$EKOw;BASrAcg88?8Tj>h=r)e(=yIv;!+{Mb0H|TsQpis#F`gY3Qpc*dZfC?Wa2x z>bjZOSohHyI@@W%UJad7zLB_qB|ATu^K9Fb){|ATo$8KoQ%3`@K$=9 zp}mw3A6xlf8I`l#^^VUJvwGb7d1JidYnAj% z%=)F@yBPasy9p;^o*42qv^6VOgdH2YAHpMkUV9mf$zAz|o{El34ujM-8iFdiOtA|W zyK{lu{ucY#LD;5}VJQ1s|2Jx^pA-Do7%JB6Yv)Q|yAOJhI)rzL+omHlX}Y&-+Ea;S zot<+V957_O6ha_!v>=WuxKG~(atO^=_IyVzuGh$pU0hFT_vr%O+!I)r0X;V>$uegf zQuKPZIOv!oo2jlseOu)x6@Ll{AYYHrA-cgx$PV|z|I{m-fiz?st|4bbQb|u=NgzM2 z0o$@2X<@CCJC~<>QWadL@KhZ~*33i+Z{2hWAN`h zaeeJ?z+L`}WaX*a$c48M#YjvijRBXCB$Jhd3DNuIa)s&S(68s2iHR|}yZzGU{qRbp zkOrbtLLG$xf8VpYaJy^hRZ6+<)(3#g1JTiws zE8FI~bn*U;O;g9j-ET9$9AEm11fcEi`aMzqksJ?ywzz(X*}M8}=0(1)JAKnxJqzXX zYJIQI;88uYapa9G4ra7rZcr_1Mph>gwI3%Jlrn~Q@s1+3*+%Yjd$TU#*S89i53y%v zIDOCHg3jJM-iwWg60Rmvh5OZ3Rufd3_lKLdNQ*UcqQO&d85RC2ecNxlNnhy_^$7ZHTb=>Uipbs3x2qex~%<&g=A=nOV z4@{g~i@=zk=MT%{Zs*%&=Rog2A_2%|)d?5C=u8be`z2+Wr)(WRY9dA3RD8l5Bn#zW zO$Sg`cf~xiLDr|td_^|0(?%`_sW|iLie^sDGrEl;Vj>$|7SQW z*H_X^+f}cUquS*P-r1pm^+mEv8jEbTZ3xEeyN*>WY~$KAK-8%`Q)&pcV>qM`aW!QM44hd?b-^-zY3oSmHqft>V%=lUz5}rXX zl9%(%lh<0?kVV0Hz^_C9t8$e;A}n^- zW6-fG5gG0LEFy&@)U8OBFiBxt(&)Vv6wXj3f|cHi+2o6C=kA2=&v)axtlzw--g1Ly zBo}x9E#-%G=GKR=&TKX(_JAs@qH|?HW(QjPk$9$ZS0%QmIGpk&YhnKUC3y|-u;eC7vLL0(eXagVukd+*zF^hiNDhu{iuHE(M}8N|;hKzr z1tQnDciweJ>dBb3Ri8k4^GAp871JI6vCR>{&AP)#oCHv~U>i%_-78?hmTw*bmzY7H zD$(|D6RDqG99jDVFMlcv<)bOo=QDEyV4xM7;ZTrfOC_4IvsE`w1sBJ-gMl2iAial_ zjB~8*OSaAz0Zz!xk??{&X4Rjq&4cDwgjp%e&!s?#{)uI{$gGgHpq>B`Ewa~C%JlJA zc&Jb-oZWsHgN0b~JFr$M`m)8EpVPmr=E*#uo}Q99!wI_BN(+}*9D_tZa-BD(ZG@bH z1aIgKK)+1{?+Bzj>Vk~k{FPVcyxnx_U?lUAJBre3{uC@da{68S=QTW%Y$fztUj8?_ zm7o>MgK|C7S=TB~(}VDG{77>KVi8rwo*E#?a62mdVcQ4XDUm*v-(M2N#Mwx^RZ2E2 zPBw>ed>#f0W^VIgjP<`Dv>8?0qDe==5{-i$7{Q>s zQ7|=(Y0>O~l>N!d#dg_BnUs-yICc6NF~_^RY!h{UdR!WRac(Uv(uSX#vYMz(gP;4ZbXxLZyp((M~DwPo;`uPSauyGBNf)O+(st-m8iNykyG zFTJtJ`4>^Mfmqx=fC52Y=WM?%cBp;!+hu}>|h=zk%S8L;#Tepal5*h8hx72u8P6Kel^(9p`U zXTi&Bp7i?NAOB5pJtIh`3AFfo1;=&oNbn2)M~cfLmBm3*`(t2y(ByY|C5(J7d_-0B zf98agj7$iYDf<5vM63EC+$<_aM#RD+Is$b{$%*JnL|9#niw*5OHEqd%x$L1LpqEmF z>Vnw+`p1O z{x&=VNtpv9CKk6AB}X@z!?9c?_DsYfaG`o)dAnA6qV@N?-@nGyu}KHTJ4n^L4a~!# zhM+c2*1_mvDt`OrLhAIMwgGHtgR)G;T+QmK{PE`%Y{s{^5=i&Abk6Hh!q(mN4G9@iRErTwjv&O0CX1%=Kx zFa^F#z?C$JeNXt~`he0Kk;+I7{k-=|;Uy~4uLhseh=VKqKewC(FN4Qc_&~OYX;H0i??BrP4PS$A0~)AVMHpl+!_qxh z_M+&jTUmH2H?g{s3ml}#gvC_7W47}Hu-n31qeimcfqN8tnPi`D2j$4vnpIEM!{0YW zKO|yCrhAWnOpV=WWHKoC#3BoX$pvmx5Jm-1`~3B4R_8toGf%n#R)PHR4c61r>oik3 zKOkCqS}TQ;{9m^PkGN6MSK+Y{3EoNqJ2%GfNTJ7 zRs-?~_%tquhFFDkA>wnpIHEwQcPc}7lXDVxCYEl&{1Rnv9$vba>_B{~^mN^*2hsbQ zvFvOz0MYp*Wa`+Sws_j8=Y&y7QF0_h?U%~LCyWR)l|Bxt2cXuYB3)M%oeIsqPo|p* z7lJjTS5%fo%52^x|2(;8hx=|jqhNKpsn*Vu6rnoV{&_!u*S^RmI*-+ZE5DlXT1A|= zY=O?x|0m7 znnrt|eF$_9RJ<08v<+|%I=wjD@%h3B;KN$`s{R-Q0wbxP#0>X+f@5k{7WvH}efjd2 z*?sGwZ(4bWq)va8T{PGx*uLaA2ncnYo6s*@= zbJHu(^tIlpVl*|?{QGFScw-)##!V-wV}b3UZYP@JrrA)7b!RI7b$-bn_CR`@eG3P8 z?l!cM+A9;(-?h!XpxK~SpH$7b+H0q%u3*I=%>SB@_mL=5i-tci3;NVbk zsDCih_K|KUGiw`bZbkQSH3#-vX8T{T+VrNVWncA5L6k(fpSb4%R$I9t6#5Mqrf*I#!Wi89Ybuvbd`KJz#H5uI;{2h zqBac`TmdfpJFTKdf|Q%$Zj`E?yv{ExaQB_nbdRNu(dyT|zIX$xhvW_|ZKzL)YxDdFqUm6;#3RjH3wL#=?7sZe$CqBPgOOOkFQ zaxYm<0S%v#9ye|my0|ExLU*kXr)mx8oC*zB7C0emuy?Iii9PXeK=Q}N82u@E#qvB~ zfunOJdlSR)`ukR?9^)4W>X%AyPGY72_PCs;!{zF= za3Srre{{Yp?7MKCYXK^cm{MC?>9)=vu7fu!VbL|3hbgTYd#qOZH&uHgD8LtMvHn?T z&0M>Mv5^;k3SLU$gBxF4tE&7YLm_Q2(4!S@=oRp!5!9cTbTB&Z_#&Yhe*kxRQ5kE~WjXIX(b@m8xpDWa`Yt_Ujswl@`&+z? z8z%jH%8R9|mK=lXlRixr&Z4Q098<{dMCTO9&oP9j*8Q00J+y1X%1kP|Tvkuu0T~f0 zw%3@sqh9t9t4j2i^0A(*7<~ubs5`;OV!>wny2On*{)@cn^HgUIrIWWA zIDkx~=60Rb7u)M12=5-x*wf?+(|NAs^mZ6&f43q!Po(3_30wQ|ZxdMfGg%RGk^P@Yk`l}X-rY_rhD6Qd&vmwU;603kyFJo%?-LqQ*Wga;kSe7CoRnnHi3{( zMb#i7GOXQ!UD4hZb=r-7=^4=M3!y_|#xiK);vi_3`jhUxn(Oq;Ko%Gx=g?sYn?Cox zkOtu*&coOv!|^jvtKuu}4Y(8&SNZ5rk{oBFJ0%)H#wNf`u1(|wpXO#a%H?~FZ~mtz z120&{3);x(Q0>eA7qv$?W3x2)m(TI52=wXw|8T|-BJKaS9vzJ>Vf@cotL%PX|9)lv z5Q2(eow9$nWB(k2O4(_QfS*deo-n5Vj2`qBHOEhm9rrymfypc=15|d?B*%YLegU>W z1-)dn-t-?14GoR?+LbE$z9)Aij%W7S^Kge77p4F=bd?KYJ z;Qr_IB8^jxd}HmB)e3JbxkA1XI@-ZvGv{Ob?g&KS@mIaMtwovpA+jq-EF`f+?~bV| zRJv|fEnGrovNMwb_F(Ow=6^5@ktBS!RCO>RIpnzkoS*&>oKyBbO{s%|%6UgxJ>06WRXBU?cJ8@5&cBa_CNS#;=@AFS|N_f=P_}lxD zBOi!EG1TzN9BKh^6@B`*VuDE6sZxTntZ}%fgR10V={$yvh*BRn+PHI8=VQ9>TSa+M zfYn5-(@n7j@`O-^485htU98Qhbv~RwFCC;$LTD10(bX#_DHM)I?;(KPWH+_!a1U2zR2z zDR|t97`@w$w_g#*UN_0S*rq8op~}eCcO6Hj>kNu8SK!88lq*QXVtu5o9;dq_7kP-P zF=*q%qw3bKLUH2{1$0C-7^{0x=$hs{+!D9{8&rP~_6WjYvht$Kml1@tN!Cg%xL~Pm zLx~pUj&1Sj5V*HE`4BCujGpS!6n)R}9vO#-Jul|Cfvb8U>lxJuY`S1yCZ4n3)cA|i zP5j~6c)4aMTyc7y@c6mjXqK;X(>2j2TqNyF$l|!Y|2!xLYX&y(5H;l4!7Wc>tf+ z43DsujG-)uu@?LjH*`{##Fb9>skHEG6bhgur?scc{Yu85o9~&~Xp?&0z>&r<2&GEs z^j^7V7Ctt$Sv5Tn!fP-_5qpAFx2zPr1JytH%le?jxZz$U{)Kc@-r|jzBhDHbcj)rI zt9h;lC=9-wR&&_XM0(#tKt!?Sh^kT}K?cMI}UWmedhQ{@P+n2VQ>@GoAzS9~`fQrJx>i!U3tGJ)eZv~Gt zk$*qaOaJh{Hr z)^euOu~T#9!@=&hVf}+>1FA^5P8n+o{M)Q(Y)LdQvqdo_a}&2YUIsve2Wf`fDIH)7@5+()UM*E0R$ zQ*v&^;~b1TBOJGPXf7ViXgAMjmz*JQX!(k{1s0l6vWvlj?=6! zzpJrx3|%zDL}{eL8HnY(h-@8-Yx@E5aQGD}MxkBYUzX7c>kBY|Va0IVIaL5tqO7O} z#kLq!-`;2pq$P4M80vP#(_bdBN;c?tyc>NV%k5pl*f2dqr3X(xwt=GZm`1LZ16woi z-OD~sq6jlem@D$-K8LuOkVod(39~4TA(6-)d^(5-O5B=}EV;&{KF7otV%(zfEU_70 zHZv52J>wogM4~g@HaC70?8Rif^yu>r4aXB^dZuMAyYcXQjWv8Urmt+0pwB*M`Nb{fEL4yyN=j?et1x(g2!fGHAq6XrRmOAp?x)!TAxHDeW zRw=sDnEWzew|t_u^NOYJjp)`h*1`BcrV{7MYw=Zn@`z@D-9&F_PEnPf)$;lB36ccwHy1D_Kitr%&a1FCaPcR zgwTBh&RTZ|#e|B`>W48R7x)aMtrlzC0q>aie-63nC{2M#5}Zz37b;v5(@+xgAUp9RQISI=%Y}>XqQeA4p66AfD+U53cHb84pp2T& zH8o~_w6gpzY54HvHhu>GA?2K5%0nWN_%QiLL5;Ow1y1PvO3E8wW$7v{23#Bsg1z3; zikY(^?aw9(7QN1!+16b0R4?!C*CpxS**x-6RlSyXRfEHeHInLhB9qsBkfI_3Z?G z^myNqV^4nu``@He!ZTojhpRUJC+FgKp1<^XAkVssw5`Rd6)y#~y7bdik9VZj_nyIs z-Nb568sk}bu37=!0$beWdGpPaMUR27Hg)E9Nx$X6dR5Bmi(daEIrxf&e=%6(NYbnx z2iFAWL%u(0%OEBZ&}7BPv|l}vTS%Q;fv%pKab6V66P+-B@9EBJ z;yb*YmU?DwoOV^zj! z1>FekOqRa2o8}{E-^eoT8}x3=8v3%T@AKU9SsR;RYcych<*M7X<=om;&GAROLMR^9}6@G&|%iCFF((zo|mR8g@UE$BTP?%(`Vp8jIjW;gbjtcC<+ z54c!#yjguJXfRsKtndZ2gB=8;q0v z#5#PHM6lOis<9#;@M(uDl9SzgWpML=R(9zwS@Ewo7C^ z<`}rS#}|?9|JwVF`gJSCC6V3sr^^)g+f1E{+IBMrr`Nprk>J)UQe8BSO0tt`Sg(xo zH_{6@w$ZRkE?zis7^UnT=233hD_sYIbC+8Gc%63%U7FNgMAjhgH(bDN5VuqY`IA)J zJ{j&Flb3^{pCME#vrJ$dOId3Z++oXvrK0S-yBvO;CE|GDIwwhR4$OEZeXS6?;(%W< zfKHYh&~w7{d4o4yoYHu`!6Ncxsc&>}+*1|C0TeQjK+w@3jVO<20vWC~bkx3Xvv`$k z{BQN3V?^8=m2L>K^ImOjfFB`LJ__FGNp;hj(cL-{K(f$@dGYCJ%Z9S97Q7Zaty&Zw zv^>{JCCiQXCrcXp7e;774aTpHPv&U2FjKbs_iMoFM@BE5YSUhd!jS%{AJ=E~Qf?N+ zNGuoDLaEVvimaEdwZy+TXFz}eZgC7>X0~13F+(c`IlRzrrtQ~c`hRfxc=!0uO`UTk zwX29)#7m8}k5q3e)=P|9_NJo-7hI|{cWC^`aJ)#vcWU$Di#^@fWQ~I%2bUv zJx>$0EhVXr8(`GdQ9@#Ee?emn*$Qf)N_j{R(# zP2|uyD$vsFNh?%Q2cMsL8*0MW-BIP7>u(|JHST24^dx&@j7*SAHRG%Xn!V+7%Rt%1 zb;%h?d2>AqKEHZ<%c=I!Kbx-)IuBvDiWco>EO)u*?;f1kA*PWtg}8pJw+M7?^q9tM zPYOkCP!89iJ^9s$oUHe&7$H`E${hP5kA(1Sx*~$UsFYFp)~WS zw3a)@C}bBO^1$j)Xeh@FYYD&+H!_0#(pZcjPZFEb6qsM*UJO5QJRaQL=xRyE6hNtL zELs@Vj0o~t$&+SmbJbJrxYQGAn;!#xnVY}ZR*lE<=yl!=<`}xt%>EIj+goN@3t15` zY4xhtTC#D>Id!p$Na?BWhVql6pKOW6dKRxV0FZsC|H5xMu}SAh`cp!+*XRNJGQb}$ zs7ZqCHURt6QDOd*lC94AZx}}_bKkfV(3A<32_>AbREy-6b=g>=P|6t^!57^sbHANl zr>?K6dSEzd{Ue556uZq5k8gR38rLhbhGe$?lVs#o(9U(GJlYmryy!_%4Dg3h~8<@5|R?tcV zPDktAD($WkRRuz;=;T|}wMX6SED|zalyPVwx6AI03CjM0mKl4XpJB;Q;Gg@F3@(=q z3ID>abyA%g2q&4lgV`i!$~m zeNR=>T8EL1HsI9(3Kc7#$|fJhmR089sqST%;?E^;#_4LClktcAO!k2lY!&T2^*KnN zmA6Mhan1?|XdaPaz=RJyTTuxxblB{-o!h@t+tVV&x2da1XgMe>h)no5tV{Vgwlw=h zdBgO+zA4wnQH6@L?qU^TiUF+f(jH{~XhO zE3eqyAsg`rtKRlWKJh*q9h?WL2vk(}WW)f1NEtVmXu|X6Xf&N}!=@8Y6xF>Y z`!lS2>DPzX+x3lSO4nUOErys9+ptG!@6h0#Af|ETGd4xLEZ^Mhp;?0P4mN^pzDd&OSF#=@Li24TSS9`wD5H@aNhlS z@_|4z6v52(Y_w0Zu4Xg*l^_-x^srOxSvi<&NRakxh^-iYH#WRC{Y?UPyaMm_i6hjm z8!4~~e~$3ECqAi<_Y>&#hg6YpH)8z}aHOV|s>Mwg{!^KvCH|K!*DYuN#jF@=r z2mBxRfdiAEn-Kx*7MJ|(k?|+LmbEA2rdig0t!-G8H)VBWtmxmEC8qpnk%oeTf6X&V zM~?#F4i$ikm%NDRvVyid{Wp+55n4rFo7QT9=qGoNP)X1G!eDT8{s}>oNQO|7+_+#m zv!1()trO&;bVJd6&!^Ey;DRmcAK$6BmGlt^bB^IGC1v{}Y!`;=5WZ&quM41V+hdP* z*WROh3KTgK6kM1nfNs~?w+{PspE|;r?nyhnE0=-uODk*FLT09y4$RHP)`7t$1h_Y? zFz+Tt70X*ilBsXpyliovd3>w4@=bd~e0A(R3;>3!Uz;33B`_Mjh8JJm2Lb?p>E^|M zg;*cL4jZDuFRvXqM9eHN|EC?sH`MwB%oK4(HyH-;b{h;GJ;x^@i9S-{0udcSD2fYYv_p{}mL!`|e}*3eFTuiMF$ z>EqV0slij@=Zco}3SzFBn$K)tmcF`hghs35Ds6wxY5(W6HDTKF9K@o12>F(UBHL(Q)(9nYw>3MeUtvta2e5 za?o#QZ!*UTWNp2l>>R%eogv}T8w=5A;Q+H4Y;>@9lk2~YHeU$UdjbRz-~JA1G8FXc z4cEc0ut`hAEY0%W@q{;dx6_;GK75|cW^4EL)q&}dq>dxe=$+QdtxcjG7NU@^khnO95~-;oKUjs5xb zc+ES4JIM+Es_ZV%qd0GavQM-2Ep`kUjCFH+*RsC)Ad7h6{`_9Qg1{ri;zg32-tuxe z{2}rZ=M*FZVRH3gkidVMhr4!>w@SF{d$bm&U$f2%y-?fJr*Q;~br2}wIZBdeZreZu zb9zHEzmmgl@tZgZWfm7)%`J1K$-IOX6xIGQI0nn~b!r()F*=xObIzP8`9QQ=j~mw0 zD+JZ~d*iToy4R7F`2De;f6J6wNul-=$@!xiIr09|mdLb_5;3SgY9xUxTD_&`2Uf8S zP)Kra_}n!s%lWKUM7c5Od_k@sI8fpJ{35SGrUB7$s58(>QI}+b6FwphQ;IFK&`~9s zviVqqvtyH1+ju>=aQJa@=60pDZex~WclGhoov)ntCi4e;B{SUb^5Q04^7H+frjp3G zsHC_tGj3FUT#Dv&;WD!)Cq>}3q{}L>tvV+edA0D6ohm^N8}GcA3)iOhjK=@^d9Yjh ziW;7Hf0!L7iFPthO6*sRQ=?QB;)D^$pUJ<}$V&?{N#0H>o2*qK(%kOR&qDSn8f;a# zBe8X|bYv{U$8>fk-O@Dx>Vds6HcCs&z99x=5f zhYX~6F1y>F0IW~$h3oq5>S)yrE~o-ZP>!)g=UMAE2YwT<%zUAeWc1sJY9&deK=rP4 z0Y|6^?%l#rz56NR)~l{%)-b^e;I;X{hMu9*@Glt=)W>0Kh6xx6>RhEs63Qd3L#Kt* zL&w-zVQuv|hON%s&Dw^?ftHMY_R@dd*J?~VTY)z#%M19P99?`oY1z0+-elsD6~^J| zQ4HCiHKua-PYm}sd?ZH+**LQCQurl!**hU#KR>jW|Lh>#9wqQzp>ynwx>~?^-GoBo zs>stO_7RJbZnbM}9)4?OIFI6`);jk7lT>)Aw&g&iXWMT6MYWexFllP}SiGm^KN=s+ z-~iqMV=77v!@=&c;_y@S+{+?!_FmnhilHVIt1$*ob`+lyGL!iRpY9LEczR(rsiqQ$W76u6E6ZChx?Qs zE&aLlYsXF7ccF^(^%&c`Ml(|b{CtR0d6xg(g^vsQpe_!#*BaU{O9MpN3K|=?J1Al&wi_Y@dhlej*XA(!#dCMT^15b)qgITl8kOK7Z z?5b&ttaMB^(=qEX$`t0^52m}0WXVWRJ(RRkTM|AVsnC7R8nr*YMDqz82MUU{z1d-EY3Jil|RMBq)o#HTiM#EoX+`Qv=Vq`mnf9Xhx=rq`o(IC00V zXs=5eu*{?qo*}f_!=Q@23+uj}w#@THq4+Jv2G%ObLsn|>Z|zad-P8b7Ji^K!0LaVh zruNPao%nJ;EnPqX9j{kKhDlR!P=5#O8`Nc%ri$HpK>MuvB|x%zyuUn{6X=g@ zG{uodc)WQ-nrSgl1D>n{`sLVvUEd<(6zwx~%^NEvvn*cQ*jyrqq@Ot9rL-2oEtLeR zWy2@vl?CSqgP-B7M&}7f+^Q~4Nqb-mJf%Ex{_84SI z`9RYB?M39bxH*y)n+`Ol$~ku)HE(=QbFrUs;zBN4(L)i5-u6Q_(^@wY0*0-cZ^2KS zh(7m3d*1`eY7ivhWG3q4<0rwU6Scf{1|H_4DB0qj&JRbBk`(0wj~&I<0{!1L^hX;R zZ7I!xFs^Nsv)o0T(DDR>_B-i8x>Qdb#18AfhqvqTjLvl*m#2nDacXahsf^SKzzP0l zbQx#$a<~jWosPVH{xCw^g7IMgF{8)Pink6oLV;@tL?~xFn8TpIxgzichzr(|r4Cm9 zpp}d)IH;Y2ks`1HJ>AJr4qqib``RwBE3%`|-?KjMcuC_~3QD%QUG6Hx%lK%rXVM>Ca1^&H1)D5UEYB332e9sDSk}TM^rr|q7DED0_|U~STssZicISfTU7xBY9#PQar^-t`5pM5$wpKAaSK7b6 z<$pRavd#ludB%rN4_2)_iM`V&uM zkYcWTYt8PM0Lo|v1(6Mf+K1+Lubc-I6G=J=?pTQb+GzKpKk2wS6Mu5MFMeG zMj5G=cT0!MWo2fmFb1+Z>-ZA)JH=Jr=vPXB(qDrKAZDjVq*Y7~$b`&z51ptTZ$u=} zE@M&&3p?qi=fim<@K~sR)C#i%8KaO^O4RgOJ$YL6pv8s|Zd;)Kltaomd*yxIh3$3D zv_^pny+t9()^+{Du+I@q-nr4u;wWl4j;<%kdW_lh^=u<4wuPlc30J1aZ z#x9?%py?=K3RKS9Qkt-9D-iFZ5zu#YEy^E{L>wNp^`M4#H662z@s>S+){wA|B z$6}Luh;|sz*!o=(I>RkKQyAJW7gU9AA=}8%Kia*K?#H|y2~l5>MK>}NbE%zA( zS!9bJ#~=oJ$5MLK{#G9_Ji39n5LUt6e9wiNQiA>dY*E)YBDdn8S%QLYE}b122n*-< zAEr`thU}++41byc#zlki;UB(AB*efwW!akJoy)W9a$MbEFG-}~POGglVv$}BIxWXp zo_ZOtQJn7C5SoA>do)jxxO=S8lL{jwfU0P;<2N4t& zetU6O^FM7h;XO;@cxmE-*AnsS9hl1FC{~Vwli~|-lE;N%KB@vUdpANjX6rgB{C?^j znyt`&zf!3#1*-CSSB5 z>vzN*<+#Co3Vtce-@CKp`pKL#>}4&LJCDke=TGE|7J%dEM|U&2c<4QG{eMUHa@W#7 zd;fAK6S;>ev^;$$v*jvna30J9J%C5vF+ zkHiyf^;Vu8uLXLwFtsVR7CWB@R;N9N$WqYsWlC}006`-|T^z>o}R-#HkDmslI|A(7g+XY{e@oLQn}IE8EG6&drzGQ zMCuZ>P5f0cAc~zbOc&Ct6WF6nt}&XUnCNnswHD0yY_i)eV1+q)@6$;({+i!7n4-H0 z3KZbGn$G&|LPJ1P?|TBee;-fzmi8F9yPk&OiRx>^CBVCqr~uDOWpXd_QEGEuuwOff zVzhq8LTx5?92)9prVVVW;?ujG<{4fPc^KFJ*wbq5C1!oAQpcmv#`0h%H&ntUlj19ykxq@BDln4nz|L4_=fz&!K=9 zZ@@F`i+C4mk_FPF56<=pF9p}CWrrR;&*8}^8+DWc1=LBftIOL|qp0oZa-PXGN%eTh zGzs}+-tvdI;&)E!Ev%&NHv9^Xd?Seo0@3{@t=oNJ#6FHr&q;F~Y0rmV_|Si^>f{<$ zT8FOy#kq(x>c)QXwByh5YoBG+OZ{l09t3=7$+-x^i`H#`Qwig5T;NjR%nwc?F zaNiJA4}jAUk+^Ux+44+mIZgb2S+?f-murZ&cHGPM2#zY>{ubz>PWbQi9{Ln)y;o>O z_<8F){qKXWKI)V1s@}EM+ShWX*aGQJb>nKI%L)@L*nuv59S}i(bA+N# z@%-+HQW;quuHBT4UEX53+r|`to=VzS9ic@=0Vl!Z2n3t!!DKux;@BNI$vO{6b-A4_ zrxI>t{OTVX368l0R#FJQ#wf{ef&WeA)vjVka7pDT%+7Qa_~zFM@LkFzem+n>Apo0$ z)W3&G*x@X#Oe>ac>(xTkRnVsxYxTSQc2(c2^^@a=04|S0#M_yEZAvi$=a>7*#AEkO z>;#IXA}d0{_U&-eE);Ft@;GksMZL6zKV)NBH!a7S1RXE%C$`z(lI@WB!5T0fHTUMh`!lCJJ5wgVAPG~I(WK2Z`O%?sbi|8JcL2`~e~KV>ba${-ucvxPbQ=n&As&U4h9?R`gY-yP2%mFmN=8zqdp1K?OwfHolS!uhiCDb z|E)3adp2Buyn2u1GWTt0tYg+2&EBd)Lq^sQ3ME5{pFlCyeZRLFE1W3 za(6QYBr+n{*(*jN9~_5E)FNwkim>5>YtWlPqVM`eeHiSsQ>mZQbyQa!3VysvvvWPn zN04HaJFdbJb3-zuDH2{^j0sw;YXnQ8-?SO|q}3gBM)u{QGIXh)<2l1Q|zUs1>z=Irti! z``1g|#$ibNPr#>q<=uXm7sV^gp6pzT8XbB=x%|Bh83NXG`}f78aE&4Y906pqIoM`W5OgE4}mJMm#a-bZRd8?)MquBab`z_lCaqMP^Uh>NOK<)I~Y- zvF#>p){!|FUr0n1zeX~$3Ri?q8je^La>*A?L@bL z)3B!!FfYRCIvuMHqeX519PUTNx{hAsr5N(J91gJao2!9;pd2{ko}W!*Rr=0mXyG}> zM@Y7w?q|&R^~{*9ZZooLo3u<%;e1t8(9{cCl5!U8UHi@boz6`$*?X15Gn89lW3jbj zd%;Gro$?+AYkzp_9AyPCovmO_*Rfl59{z!Km_Az8*I zqW@1hocT39pYcW6#d7&~M7<7*?F8(LAJ&(#EmM(DaZ>8MZzNT=@Y9Ed`7f4-h5eli zu?xyN;(LTVvwi|5nz2O3XJfqh@V{*0n#y!s3ZzpwB`Tpu4=C{wiA*DDd(Ql*t!vzc zmZGvZ_=deJ!<{V{Q@?bDE;9uXQH)BM`jTnnml_ z=F?y6-PM%ZDxBu|Lwwz$HG*jmB`X$nmk}3uQyLIHBfRX2XQ#Q!rU~Ob8*T+Y5B5!& zsreyV$Io8^Ke@lV|M$q*^79arC&1RD8AQR@bkzYyxhYee8iVn^vfewj1O2tPv4jwh z<6!-xZ#i#cLG`Vp#y=WVwV#gM0B}}7{@%*DwKHRFj63;AakeVMZ%FU^Qh-`y+zil( zdgGk*J;v6jq9=VAWvi1_qJS>ye99Zeu2B{W#xG)Pw(8q1qp=-JnE&oP^9L)08t4)% zej)u%wi}J3fc512Z|rwR(fHKIif1EQ|0x3w&sm@W{}6Nrc8XaTnuM~g)NsJa#NLN;70ZlJSWns%vkq>o|eJLFhSMqf6maAWdScWgt>|3>JE;qr3 zJm4qzgXnYT2|@7`QfEl!ZLk3z5-awFE;-MUhe5tq`{2#gpaDdQ0T$jkAdMskZf9~6 zuoO7SZRZw7{qPCZvpdbomzgSh_zZ`~rjCkO44vp6NoCn>NgTEom`zmA#f;r&zn@nA zdbA7dYGY3#8uK>F(hTgdNVutR!UC1}UJ6Y+8BFP$2>*nx*yxZ#t6ee>32!mh9FfkI z+zegV-IuouF`#3A+H1U=zgcD+eIQ{Fa6Q^#++46EpcE>kZBxI`Kk3Z|CZ0lWpJVy9 z^?a8jk4I{%x#u#7JyaOU(#UQnNld#sUSwm=pxh!ZM?_4hhTff*U-J?+?EEhM*P;{R*u9_2%5 zd+67n%A0~)k$yf6TO`!j9EJy_&>dS-Kl1=J$G8r)bW6I2HVN#yzempx4r1%5hri{sw@5)%)kF<}uTPwbZ`oftl}KQr0SPht6+X!l&t*rcCi$ErD%`@+R;+B61QnNs zEh6YojclbEY?@#kU5wS4?OE?ysi-|M-RHGap`frkrDOj?Z|BN9tp-JoJh)TwkQuE% zwR7il(2@%C%`v?!GGXZ_llfaaZ%Nzmf?o#-%vAhaTzkx(a7}m*Di3b!aQ)@F9x8kC zDHD?l5Xx_RX*XG4QJI5oT%`2Vu>Ip4xS4a>wKb`oPc)1x+$_AGbWGFTORj3MAczkE zdf53W*;MTBrwxirZ4=u{K4xJVeivHA4NOp5e2^GvC^!qsV2c_!$r3< zKuv4+i`Ocy2L|=8;KtUQ_)LcmzCbSG_8!)XrPoO64(q{ewNSjpeiB|Ac=PKN=ksH) z6-Q=wMeRV!PJ8F`EV|o5=j@ZcyBk(IHc-c$*4cV6rJ}%XCF&PxpExOQebib?wzm(c zsi6-p$HK9k+rrX0?62EZY101V&Y$GWuRaym+?fYuchyeYoNv2%QTE#r8KTVYrRSVG zjRqz2k7f3{-p<|Kj5PeYDP-==n`V2uDuDi`$cFK5nl)r-b%zK9j? zxuG8zI~{F3fkoxKbh)p5t4|nza*%c~{|~9U!;{BQ)V&rd#1hE29UxUlQ{ zHxan|uSW~N))Dyov>jdA{<}1&N$EH{+n;<~yr2f{KEFA#`(2s832j9$SKI%i&rJ=! z$>?!(XV-r-WA}UaQ}_TsB|;1pIYtfuhRuSy_# zuF@_tQDZL#@0WT!0j;j*X$Z_f3cvN+YT}3C=#%R~V@eg(B;oaFp2<2Uh-%&uGgG+vqmBs#3DprqbAXJg^KL7GMIHxD|H8+%-%gn$cgEqr`mgAnUM_|4^UXf6T z|6yDhL0Ik4loCH$;o!-yL*54~sY-L38Ov}7s48}=~=ksYHk zk+^d;#DArdd%9j=tg4V?Iv|ho$GHdFh|x~QQZ2s@#9tt3Aa-UhR&S&ZMIqZTD8;$s zY^kwkj_@Zp2dR_2Qs2qZ^Cn^4cQu=}SBP`sY7Q~h4yNq}TnCGxJY@?T&_N%S7zYS& zUg0D4cF9;gs-n!wvNm%7#`*C594hcY5ANQ>26d(lZ895dhCAr6VrwpmPHk5XTiwKY zrt>GxM5TBZ)UeDiwE8w&zbyVfa4{eco{ag6Nh)=wHB5Vj6qE(cZs|fb+2#Y})ZU1A z#39QbVVY#T|2^A6ghiR*^9PV^MuUIP>`YJ5x1^7``C$H!2{>UjRpc${+TKC3g6qG5 zlPxMvc-HaKeeH&oxEcq~ZmuF*+3*OHDu#shJHe~iiqy^K3_{;LpfWrvCbgsu+#wa& z6Ke@FNU$f>tlPO*UGKcp8xi#7vDnR>sK(u4Rb;zA4+(=a86NUH4cq6X&p0zpzvXm1 zF|oY0Q*d8i8^#K-M~B}!gwGhg(CSWZx2t*V&1=oNuOFWj>8KrpVRJ)_VX)Qz z0EzeUb>}$sn$SB&4hTycMflyv#1Mk^-k_JT7dA1);KT2FDNQa+cE8iPgOuXTI#XA) z?bu17z3%}zFmvR>7aQ*s(`Vka*Z(ImZyNCSv{sW68{j+sd|{%gc{%C4+&j9X7?u$W z=?s+6tLcv$xVKHz8IGkodcu!k86Ds+-@y*8R%e;FX9ew*-)Mrm?_W))C7qF@KvP>8 z4Sb2Z9`x$TvZmP zUl>yf&#%mA>JnOS4UPCo^nS;Pc0Au1nZ9}S&(D6d@4DVa5xTKETIE}O+%>nEK7YhqMJYcW=h?)laTif zt8Tn&YEyvVqTONn7dD>3nG@HLbwr4F7uiOi@B`k>jDc>nW*YNzc zf7pWr{iNUO4SdEQdvm`##Zw!S(+>YJt^lg%HVPupamm3H$ ze(eS8cMcQ>?|%<1XV2zqcW=S|@J|B?17n*piVyGa6NUMzRf{pj_V1E2VZ0p^>aX|r zBf%v_yCpZ~%dUF933Ar#OSF{RWHcRn7w#Mfo|M8poyD&1e0Ka;bERh#z#aNr=|x|T zGgcFM(4=YO*}xyY&77TSH&-%B#~qg?{EnItrW1U%l2U%}P6~j6`tn;jGKM6(sMBeC zOHd>+?zx}B3n&CQ8kUe$+Ny9e)u8vJIYi(6Z28LTTA8hrqH>xPlrYU#R)i5XTJAAl z_RjK%jj?R+&Mx?Iy2vq2LY7&C^5kww_5PUo+T@}r-*mq9(c}tPd*gUsTYo$4zQjLP z_;o#%I6*%bRqACWdZ(S;dfnIlaGzmK@fVNytMa)a zl5q~;;=>jsP_wN82S0Q)@sqNY*!gf>4;;L#F5Ve|_F{i|9QzA*_>IT#=w^3jc)ovU z-=k|eH&t-lAIt|Eg2d;fdYM@k7m4@K#0K-~dVvcn;ikbM*_l zs>qzO^z62?4D7OuEen;J^s?jBzVhtjTa|vG8;7@Nkhi0tz%=i}byv@WFM9f@X+1V6 z4Rn&eHchR-xodzci8m})uovQOy)NYpD2x_Pf<%C(wqsb*a5d2qI?3z>ARsJ%?Ni|9S_*|&o9n4{I^lPy(VVRCg#~+Rp*c#lXY&(fTv-G3OF6Ww3?~^}r9kUV?zE%L5_8 z=ouS@@?g00EbwUC12@d1;Zp4ABhZb#=ECG>1Gbd6XG}oa7yh}zU012zj6s{ z{+-{p&~9*{>rSrsKDTpH?g@B+Dd$3t)(Xdy4i+i)1bez|#|n}oypCSi6G8JYH=!&D zKF3D47Nc;yEokq|k(GQ&N=4RKR|K=sg|B==OHG%5)7h!S2=8=8(x+{y*ZeGS5g_RG zOTi^8F-Y!SLwMNGjK*sHN{r6XIEI_P7%d2_iiO^H`x5ZaAUwibgf78m1gA8XfhN>? zH0p{iHQlDS<9=Oc&kd)zsWPDA7m-c1rwOr5o6+Dk=m5F`O>56@D*A4_jd{j=)0@SD zUDs)W3G27k5SVq?$ivMGs%|a@hsOk-pGNXpVXF~6!0kl?@f;X5Y4)@e{xWE^cB4hQ zHKZ9k-T(BBq;l%<2$R+SlqA5$V2x3c`UFXvg@XCjq z_`Sg%_Fm}+8%Whu%mRvrIiEDi95Xl4dmWZ;EF#@}9o0>aa*F+9caoY>90F0G_q*7u zxHeCZzd#l3)Fj1Bw-P|WR#2WLNM{)5yZ`Sc7bIixo4y@j+8DHk;T7HHdopo3mnz*c4C)v#-L#?Qh?67_oa0(CO}=T4#DV-+KJnv zS&eSn4x~o%NO1hpfQx?J{WD6VadVTI#UIhmwEC;{SWO+{@qeh&7v?qLgmhC)%eelz zlLHU)@Rb*9$m};e&8xlm9-P>WoCaYfG4@!wD`lOlJr%l!Mx!4inh7Pn-Pq^q_p6q` z><}d64*#7j;`Kylp+Cy3$70 zAtaM_{Aa$2^4lIF+X9EnsT8Cz?hL#1c4@u!L94dY>iR9^3SF++GB5Tg(+sD}l*e)z zb(}`^6}0cfNR)sLQgrz+WzA2wT+KdbBF-^OVi_!+y%4-BGj(Q1oq-n#%1Dd-A2!2G zW@#B@m@TnK^api+yc*jSV(6nyW!NFCimxN*b5Gj?`46RK;dHkp|F9REJc!0Y_X3nN z!o)8(Y-I*dRC`CR>X8;N^as?hj2dI?xtvBQ7mN`LZg!@`4bg}fJQ>#t{Vw-`1U9TH%!Y!r*Q zGyDg@&6=e+SuVGI79@4H9QlkUE&01wBAwVP$kQS2IB>DWw9DT+I7ariUY=uZ&lg2w@} zS9X-}JB<&V1@uu~Brr)BD;)<0W6{@io;BLb=askgW~V zF*aP+eMQ|lqRt+t?$dv6PY6o=B5lB@<_zF8yZ!2Jf_*G@KiRpxhlsIxCya6)?xk&L_8%|j(#^&#g)5tu|?umy-$dBPSe*x-&h2oS}o}M?x+azvv$2>AtPz3!g08!84>DK_-WCz-;VW z@M(Bfu-EuQ0lBjQRp`B9K3CO)VU8@r=_!Om=Fnc|awJ!Tv&e^BUhMEz_s*|Tye7E9 z(pApO?1IJl)zgjHh1M%{c}7As7^pY(mlU0_a#6eX*JGH=&AE;bQ5#9um>TvXP!LL> zXF%@I5rfm&Tg>fRKfjy8Tf5Knw_uNJLAwbqOG~mYRr8ePH;4Tl!2o%G*6$(K>7zMT zyTHt4Gx1>)NCoWVWu`nahe)mRE(eq_-9A|IvQV?Vx$fr*K-JCB zm?VZe$NF|!jLi|)(rf9@UO`)CWHQJ_$6Z2jc{{(#I)iB^7DH{U>SqC0!Q#^E&*)od zMF}DtP&g`@(Jn3B{s@R_D$wCk7Qp*W(soVfTB@*KWzT5;c3mxOHEsLaRM;Y)3L_t6n zN4W~^&o1t=@|_t?sYVy=t37*wCR)nik_(Q-fDk}8$0_s_*ptG;^TO8~Z|FvF5ghW5 z%W0Vt?}-0Aa5{AHTih#ZL*2`B0dz%4p5Y|gBR_R`be6tR^|$PaO5Je->0jrlmNrlYPs*^=Q$!#@w{D;G~Q%pPR)0F(I(qJ?tPOX>}TMg z0qwjKqnKV@Kkf*pblu1dgWf#4UHN6y&Cz?mRS4IWdtlwKFS9JJ)P$Na88{v})&7I` zl}Y(==-3|J0=+r%0LDr04qHPSF-4zMQ<-&3D@`uFgnTo~;pbwR8rkrpq5=HE(W=86 zirtt*Efx&sOL3jMoVd(G&C430uQ0DJ=cq<2zH6f)ON*Zgaatn}HbV5sihh!cLX z^$zsS8}9@#b~8!nboURgb=jg5N9GRLCM>8$=EpP3sZJwrfoKwHH)_S*bAv=ETFU!^ zrvdE8;F&syA9JZx*iO3}{ZuUN``;P9o0B{&Lj;@gXsDR3v3W*w+94Nz>E?5d%%d7= zlRbV`EjG{3%Ecx`MSGZMg;Or<4^n86Uxp@=N}PUsgDU4KfX=db4ZCx`pAE`abhi2? z1g*%%I64yk(<9{poty4Lb;n4JvrQFT{8I5@I7JMOi(#M5SV+-2bV`e4YGcXd^fv5% z`sGLe5qo+sDD895V3_zEYtqgxuU6!m%?b z;6bh4?g*`nyplc9eNt*>NXMG7EWVN7ocOkq(K{_EAs)R^TzP5`-#-ax<{w@rM2~k= zN)1d$7htU2ClRR93H!`Ua*bsE+p#}{TVhVP6>s!CmI2J4FeeqWM~LeUHuFvU1^6iz zP*GPYL*K~qXwvbOofU&J>SPj#!yhuT}hWJf265y4@&Pxz~UCD>E7&mLbf(j>Na@E^1e1 z2H}rK>)qDWwzQ4Q)QX^OfQ+89H4T&N)2nfmBoqYssD+~Q&paOs0ti4eqG$RiI8TgC zU!IBV*TRfNk9hsu+N0bb z`cD2PPNgsE^$=ua^T+6vn~meVSg7NbR=1VoD;lqS-yy|4R1H_(P)U-wfjK^2LQduw z{Vz#y^tv^DNvU!y7LwiPA0W3Zvr^R==>?d{~Oc@|6uD|udT#h0} z3v9J>#w79jEOJ%!9g5FM%C5u;X}rYz;AOz=JBB-mZ}xU=*Lse17;n( zH@FhI7~=fw`s^)5?xlYB?5|Y?3CiOpCSN&o|3(Os(VH zZU*e+phwmhSNpF$s?f@RV&%ZLSd50(5EAhZ60g!}ZG~ZzeWO{3u9{>+iW#U6*8p%Y zW^1ex!%Au09^z0>lkx)f#GGA*(mtX&sA6S6q+qWiXqt8&W@>xzg&sCk1y8^ExBmLyxl<^I0keJZcaONF2ks$gZ$)b%aylGu0 z;734Qi;<5d$-QT?#n8(4FCeL4^D8mY6ZR~kPe%7nRVV<>)>NRlAY#!@<-1=^os${6 zMc1xps}*umRwoHr#RHowRV6u_e$BjRlW1*SKAA5>f2@0cYkz(3POz4hpo&G|Z%wW_W&3Q5UU zXgkre)RGI?-xU%8?)UYqM{ZE$MNe!TTEr4WyfAuDq}o z@XF*LvL1v96%^2J2jBGh^$n%aH736qo|YbbW4P%ay9FB$6VTdFZNPf;hAit=$zQMZkQk0#iKC3k8Q{(oks6 z$Z?uI<}G&JoD}k%sAbK^pj-I-P@p|xRikiwy74kwN&366nAJ&MXEZdNMl{4?NA{tICja530l?-PLN*zz8OH29wJhkm2JlklY zVM00wGw-WwWywKyw}Ue9Y1zDrL3C$`9Z}@ zzp5*{;(v1(6qFng&75jiS3_fCH2iFYVGa>K0R4e+IXEdP>NffQ{SR_Y!&aKP<(QfV zq#7;NKThz7f9BtaJj{3!!LYndD5$Ij;ut7Qzz|)C)nL9~aDup{JoPmTO>(8; zH|+xw@a?$pg;vYjz)jWA{G0X^{g2%&cO%78GUsk>IIk2(vdJ=+;rD21m)Ay1v9FJP zanS2l;r{Ou3@+2f%ls`^7Ekae_VtQ}5p{8kNCnYsjiUE=ZK4mo*%EMnqmxw(-I8CdFraTdoA9S z>v=unrvNG9AM#Oos=V?`QU&f(DD90i7`kyZG|Akyt1C;b!S^rMS6lVc4Vh zk?3;G`dAtgJCN`oFxKP|a3fe=Ur#9R+Wm9bk_Xj-;(~|u2`nDvVr71(6rtRXxgy{b zhx2NX{91>0Fm*g8KMu)xl{>MYOu9+mI%6szqS@#5CcNGiy$;(TMevtU;H1qcl04XH z@^?+@t)w}iCZ5@4(s&5}2@PX-#u{^?mKl1J8Ks${FZd>3rkb~zli&ZfH zF{*i;!&~`FliVZt_PJm_^0%QpGUp6gceylpVSOeCzJ#rHh>c!i{xW@LaJ-2>fr}kC zCGw@-s7-j+6r;|WUnI%)m({7YlB3#mm~wr;FBkq1XHCR&=l@$=E&J^M3MYy)pa3hz zfcdu33nXH|;QuFbDVXW*p|-g6&w|D*QTnp*LE4_(GKoAx`Z6Xs9*9H^^%FV|8?lRC zR~+g)#nl|95>(MQ@gE|}eejyS^y*(wDE{W>yMHKjPx~(=fhKrgD(|Y@yR0vKjcJlb z6+Fj6VxCs1DrX;TjnK&SaGFhxDKvra>MIHB%_tQsW3HOZB_T~chhJ@mfnG+~%@!zU zTu-&Nm@lNr^!8@}uHz2!evM`%N_q`_z|z1TrxB&UK4T>~ZaaALO^Mm&u35oov8@NC zqNDV{mmp-R*~vG|x(|%;XB= zEKQkb)b29BQ&if|W4&{|ezn9?qpHx=z z(a8IsA7AQzE>X;Q_+;zC?s?8Ga);!!Ad}Hp1bOyGdem~pYaf0UAa*-o^0XI{;kuYY z;irfvK0J(Hz@Oz7_EjQRTeZQRJqYdX6A$CAh-{IVB;pF27~h ziq$&_+t97dWen!B5-x50tZ!O&WdDOtt3fsnHV*fcG0#cjn=F%m^4uX4j5%O?c;jb9();)_Suq-w#_SOoj-#0o5ReSWNG_VL^EgKRgifcjUk z45UXCfXX+v9+OMd2Vq&vo4q>6H(;@!>kTN(afLW*Ak}Sv$^skSIm^raJv-@lq9Ps}06Eh7 zY#J+(uyjMM3Fld~?^3DEji2#QM4lIH@iDuU+B#&P8`-`V2$^zFr8K0?1$*`61sN8L z7E)2eo2|*@tGH4xzLV4(NE||QVIl9!u*xm>h0B|kRKf2mwU%>O3Ah$QV?-+m8&kM{ z+~cjpR!6o?Upd5TRpKmkmR^J4KWRh9dT)(loFPS4=80$#N^BA`QDkZ*HVoBzx*vjuP7A48t(D-}N*_wjuCBOFp=nwJMP(>HlRn z+%jRc^4kWERTT~Kv7_(eHbLM_O?+!nvn(yo@$bbAmahmw1TNY=g5YG9Rp;DR+FMU? zh*>mL-N1&)%eZ_kVNk9}*z9_+f&Y;nLO9Yl{O3>gkxbh zXWG>mr_eebH`CP(_i#%Y?X56BuiqiQYbDCFQg4vg&+N8qTg9d&35b;ntWmL z;h79*EBA4e$uBt@HQ%`!0zM0uYmM=uBO5wVqpUKt%5_Un&Sv1aPsiN*Q*}?NS%?nK zs`CP_RdKAmnA%G&;1<7@d3(KyNLb;gG}_%NA;-sANE*UklYXYRZE9p7zO5=f4i4KMkAEc zM%BrfWw=0(&R8~6GN>>IxVnxS3bz|?u}!|{^13xS{qyyVj$$!2Bz}LAa&^JT^A*_M z+(AFTJ9le@jIcp|Q5D0YXipOewhJei{fGmkL{lBEbOf~z-N?F}_APS@?gO3M9M~q< zxp`zpmtP#PWhD^9rgaX*o!cOG${`YICm@H?Vhru*`(ITO)8`<+h!U*s^A7!x%QX1m zl^OH&?T90HIm6SZ>->lQ(7Gt_Pn$O{vLQo(!XfvGUD=u+(2ADXO7=ar33h!7@Tk#* zSow>awQZESi`ph6g3dagRoipZ^m_E33irZu<@d5J4%a{|s@H^QjJ3vJfNO;|<6g9| zXGAuJum!|Ai_H0_4YW0rS8N-{Eso5JP?!{v8E`PG|MklYVb?SW_nEBNsA;TDt_Mjm zTu%_|0LJ`gtKH)oYt-88l&YNJn!x;JY;oeFFRuC84G%~Cif%qIKcS@Ze&C*ho(Ag4 z;&v~9iG3K@+OL@ybq(#6s*}tC6a6=C8b1~U(7(kC!DerBVjo%1M*xkudw6?rtumAk z;WE>_Obv_lTA+KWXig(9u9wD+k(BCAQP%G`=5FC-j0?JzD32T`RK`CMY_@SAeO%b( zSHrOXX+dZwnUgyoSi9oo&y2jEv`~qm2Ps^by*H;?`Aea538z_dRm0eyp?11YST20e z%RrLx4?lIwlqNac%7_fBK5kep|T}o_$ts z-{H%6oLP&?>M#eETRi$40Ns1vMZ!q3)F6b&z3iAuM@?<(J)#ya;aCgIM7Q>eb2hB^ zm2&9jc$2AMW#IYu$j?XRXN6nAt!mppjO9vd+(*+x{h32b+Cy_w9Q}d)-@X|xp{zF>)yvR}Q7CS-jFy(S&C}g^qf8pPw zPTtv*PIkZ;GmG~z6WQ&|8Ton&e_P-gqg8CzzE*DME~u^jcDlI#^GTn@9LGUA7Ydxl zPP^wKnD7qc70dY`d3fmG!s*yQFot?bxb|f+wV^~s(wNB`6JDdLEI^Ty%Es6(=FCTh zVFy0_q+_j0Ij1i_PXE`lJJ3s<{(>o_rNwG#4GcpvM-X9_5UnoB5eeZiIl0^_D-DR* zwlu~|~P zv0d^$A39iGiDx4S_x|4$;i>Atx1L252jOeL%GR9Bx4^G^)TN7MSxyHO> zvw(@7j4YNsz@58PNdvzj_}c>HWc6;6YoE?qEo8W)BvR^csy?zjh>}*Nwa=j!XM4O* zh<@j5o~A03m}?&h_7i@%PReB^bU?-)3Di%lCGp$@27OevpopB8_7&fw9xG)^nEJQB zo`zIt(^g($I*uAyKKRiD_og^_Ytgf(e<(fsO>|7Wj?(?3xiGeBGf#0TuPrgq(xF~U z{|$8ppAw)Y#_5Xr8qCkHUmQ5cQ0Tr1&w44fvek_CzOEOjcM8gwD0qiKxUHK|P~JS? z=!g(9_KNcs=zwgupf^Rk{5+g^mJ{HQd3)h8RVk*29_^H`UAu6{PX`Qx*N;MI*-kTC zI|)zvkH-7B1n_|^a{G*36&b{#lf4z4QD1TOWrR5eLNThyWJ@@Ddb=W}#eV&@n8Zw5 zq2{fGA4tT}n1Aa#VCa!ci{8Khk?^<9t4BB#!rNkU6~HIM3;xaG-`=LRIbpP-a}&IA zD@F&Flt9$L_7lOX!awJu(@LBEa5-7jP#CUHm*wQWv25%jsnZ~MWL%0o7d-U;=XAyT z@+{3illt0lE}AukWiqYZIK>m1!D~*h*K5wa$o-q*cvVN1IQFoUA}>x4C`$V9K%;_! z)-k$Mskp17 z+~BT%9nJk}jI@d}oOqvS@?B@tZ&`?9?uG42VFPv=TYXUIMrS396I|q7jdtts zU&f@vwfJ5aGU0}1o(KjA6{8C`=;tZ;;`T6C7T4h8+QJu2fbEPnt#_Kwg8Yo$6ld_@ zb3~G8a}LUOPzJRe()pcxSg&m07nKF#u*s(h72t_M(p}P$w64|k)VP*un2p+*de|ee zALcepuk?2D-!AE0Lw#*| zg|5a@?IAt7(qr=Fj;3=Cc7L&ti=>RQ^0k@#nY(jNIfMv~3ySdXv;n|jvV#SGS^(iA zjPGg>w4XWMN(7aH@Dx$cp&08iqG~Bv@{0!b-&0Y4*j#jbB}O3$g~hDaZFh{TwJdwn zCWiP?1BH9}goR%}2t22WCN==AM2xOP0P;$0Gz+s|^zWp;$>5z0q@xf`!=q1308T63 zcIZWk^|D^R`Eb?OELG$-iyfQhGYfM%egr!1m%_%1$!WYLtpPDYHw@n?qVcg^Bsta4 z0S5*vZu5iN|-zRH+bxm*6|JPlVMb1)2X*i z!s_5`c+io$KoKztG$sA6;;@ypx`r(0*;tOzs)Iy_%o>Tn5bvMqZ&TrclD#pvcegn{ zFl%5n?_-53!R@3U7iQ}5} zIUR`BE#%z&wh3%B(6~G}m}!D;Ji~~058_2dLhk?g0skXIhg_Zg*V-;ux0D#~_Ma?O z0$=jAK_75}hwe(Ii(6E;x6x0q=#;4!h&^LM-pDNOcl)0k!fRkpj;9x`vq-X zoWm*tyIy^v;9vAZGM+Rko*<@>3`Ywop$^hz{bH|6@o%EYVyPk7hKf}_q$EnHX3Xpk z=r&2B)vD{t0@K1X@gl?*BMb_2n<_N6pG*V@;y*n=23krka?w7IQeK!v;DSI z+(R}qrg_;?RtJbAa}JVum?G&jaRtagQ)*$7$K{uA5)!odGtgeZ8 zahHL{H9w@Qgi@8WmvgG193W-$ylbl14` zK;HuY$BIF{X|;e;P|Mgd^QEFBJ$B<8=IZ0L_WY{}4TU%ZJ?eQ?Iq;8SGumc3L&8+G zlek_-R$I>{>2K6*^R$hjumiOv&DKC2QC?6>H8aZ%zWjx_C@0ZLi|8 zX+66MJbtl)1ihNQvTw@3m4WiVGTAHU3FRgk4VJq%^2%M*&cu(br#_F!2f}X23r1C? z!VuL~=K$)hUvV_c2{$~g578Eca{gphz*jLwBdNpUZxE^@fXj+AJF>l{k2&`(}F3|(!(2vA@s^=hP%L{>Cb=?~L|H16?I>sZOlhdmTN{byHR zHp6s|=p<44MR_eWY)-&U;Z&0D1gHJ5yw;CqMN2&-3~ieY@B0R?tfN+62Lp3InJwFD zoQ}BoBR^|cvfi+0l*wFLvgS6Fk^7+aGpQ0IJuT&u<_QCKC#2A`r%|;zfOd&rIk8Xt zRgW#A6Yp0(=tj6mqFuO;EGh@Gsw~0~!590URFlX#eN$<(MVI3YEU1R*PU)Tyf`U%OiyWXSgWyzs>mjEVRj&5a8i=(5cYPPN!dLwn5W%Q_A}ypexj~;kL>Y z4wyVV-*nSGh+v^fLvKJlA%-dHrvDVwp3aCbAbDrx?d|I zae*VDFSt)dScc-V;9p-DJUpv*{rs=jl0mafJdXk}x^IKz|6}W{qT*<~ru`6tTX2Wq zE78G?Uj^ zZI>TLm~fm#lZU`j+0I)Y%c&H0R=)n{B8Tbe8!Vy3)zGFrI-qGMED`ZXK}=jWRjUq) zJZ>#%!OVDs6XlN4Y;8SAV|xrF=Aau_Mao$aP$o@uS65Ykf!ZY;GMtLLLH*~Ef})3M^%z9GG7wD z=4`&@{V62K_p?kzGCQuPl>TCNmV!dLjj8FR4@+ir79GEhl?a;!YpX6nH^_1dU%f{s zM)(q=Q29_rfP@dFuU&k`gz;lR(N~^y7@|yjy>;WGc0vT}S(n!`Q=eTrY=L@=--S)i z%%c=D(9);6JnH$XUEsbR3hLY9+ZR96zZvv%z>YjFz@jQtp6^C6Fy9l?A>w|r(;fSn z+e+nql4$x%4DeFw4f#1WC9w{R{;kRsV}SQjX?F39Oj`T2+rs@;&h4bgRNmHpvx^kx z`FB^hAS%KUFo8{S~_}Q)hZX_O2WYghdN+lBF$&7EwDnh{Edu38=@dH^ zOKM(i*zGC`V9_%|l1{?tCrQ0VLj8*!b+6z5lcp6uRsLT>aXIdzj!PF8vh|V(PlB8G zf5sDBHB=Vn2SRf>>vPdFf%Ma!gsCt=FqjO}#=pyQ@^pUmg=WuEu#nI*B)| zc^d2Z>SOW?B;tCe)I&WIhVq^F?isHkF5s>9rKNNaAvAH%zj4DAJfPr9X|1_t^>Alj zR}JG`-0gdvz6mAZVouk`b?F1Lt#{^WF8*qWY3^F1KZQ)ofEx*25z`-d7DE~h1qEN- zkG1V(>@0iNB{!v64)=s${(={}D4zLvgKduf;mkA8M$)fd>L*!>JH>>cq#MBqF*L

H1WXa1L^(8vG?``-{ zOtkYIbFWaccaVq^_9*E!Doy_uMRV)bP0{)8de)0Ub65PHR7|H7GdjWIT{pqgwtHIQ zb_oNSg05UNP}!!7)rTQ`lJWb87bAo5K4)KlgXbv7^v6BD>m2(N#mr91e%ciaPIBaG zoP77)+l&#guob|uB;((1~469ZiDA)H=G^w`JdwS z;5_H>y@_!B2| z|2rgB=VlZP{`=-Vt7QAC_m-x?ou!A-+$Wn^W@ol2=(T`UH0Cb<62mS2-U2$)tsx#; zBf2Z<0?0z$t_N2;gx-mpT=K*MnU~JCzd+w~v7NdT++p<6e)r59o`ljbr7FKaKIO)e zjBm2bG+6ss_;v^{Kc&BPB|Ffj$|Z;KT$L%IPK0i-L(#|JofIht8DhSUB*vtAab&%_!iN^(ej7}4-)T}k9dxCks_fIZ9bN6OA9T9qT=lkv zE82ek6h0Z*_^Fh_klyCh_Q*&Wk#xw}8C|o`(RDmd*jnG6wBPae3p_d6-al~*^Pf4* zhz@~d>(%)@QEPkY;@`SzyLKW^$%`UlFs@4{FOPtm#l&xk|5_{tE^Nzyt6Brc>4?QG}qF|COwCwSdEUhKVdtz(Dz%!QVfBp4hL3 zj_FT%%HLA-sK=28U`~sYUj73Ux82tdUHSi%vips(O|gPAhq;iHJ^2S_?YN@{x~cw8 z8LZ!!cG}-^)bZW*{(;#)i(Y}oLw`!+{KhO&{l;jBTqOMiGZDHO0$n}+l+XB$*-!la zZfKa33je@-Paj@`zT2N7``;L!zupZKtP=VU%xvv#I+TMy#kYUSNw$E>Io-=T`Uhs{ zwioBm;oV{=Qrl=ufIp>sjsf zOwce?4(K+n_#c?go~8}xjQEq+`x|4M{F}D#1D)9az;wNj>4M7nlLPo0qxY8_zoACo ze_$XbzjmNk@BGOH`fEItzd7)+7lHW)CNK817@Bzb)7thM;{*Jy23yAhi~m0`qyQ;E zQOWoTG!y-&RpGas-^Rp8E_(w52!z?m_u9tzKW0b((3sxa4ONDJXT6~G-DPwt{LlOr z0Q7|AE<&~Y--&~c%Teh&(mz0u^r=y(s{T961qIDtQZ@Vo^z|q!9cqC7o!x?htYwG! z{sBtNyNQEF+5gUJK|%e@>h%8rg(c*yL#@fbvsqA3p)Iz-Um!R@raj;z2r$tyMQf76&&7_l#491{*BfHbM^>`LNy+NdDre6lny|!g&gwVX zWiNndvc!(j@6$+c%mvh10&4X84Hg>n(H$*ubC&va+8Xo7jx7m3EcKh*G#0!h-NuUG z?^BO=%f-#z#;f4(*PnFDM>X2U$>`|Qe&?1)va*f;zN6px#H|2PXd64hw@)+wJeQzr z8&L1tZ@6}zkLkaSTX5f}`|&)F;%S@U(|!M|XKXA{%I|OCsM*Ak6xc#=LlPm>TxLk) zZy`V->_|BpYTU3&DWCv*q&$;3ZbbV4Py|^{jv6NgQc45lrjmXh* zq=eO40tNX;Wwn!G>5m=S_qRYj?nTnht)Y+2nsHavUoQ~bRS!YBJzz;VR%4FNVoXFe53R# z9^uWoTS5fwBOn%!uo|N+0p9jeW@nFx4=Yr&iK>ZVz zA_e*mv?SC1^33_WfMv!VZeNUk#CODsYt1q zv6(w6dw1N``%6}iP7w<^i3tpA2@hgVv&7Qcpwc9S$Y)h~GkPkE!M7TbsjJFXB1mkJ z9MCNd42W?$ni-m{d>d*B#VAfW3R%^{+%J0=c$_kB`cWVJRTJN;Kq;|>;3{u0nKpSTC* zIQ0)4zvzOI{Hs!xO7jAVGLbGxB$+OSJXIh~ES)VwER!iqDw``uB$q8uB%iiGs8Fhi zt(dKZrc|hmr(CUqsgkBjsG6pRww9%i?;T6MM1w#hM-y$cNQ+o2Z5zvbu6DT&woaiB zd|h1KbRRiB3H0#v^7QfavkdSI$_(KRvyCu}vW>BfQ%(qcR+?n^LN~=RO+CXhD=^10 z&$YnyRbY{8iDg;g8`Fx|s?eIiy2y9F4S`LjEw*i@9jRTOJ&t|S14|EU=otP|s1-N> zw080-C^Zcl&m@BWw?e`9A_Dux2!l3GwQFwjwXvNBqC6&}aMiV?ufN{s+^k*tqENe= zlo7s)?7;H?JsU9a7CR5FM19QI7k?_*2RDAf`4s7!wwoh$n(QC>oVu@Qziu8_`pkHv zYThe86HK4zW&T0#gSv30NfUKjg14wWj9?@YHDNiw1fK#&IPtKuTK<4?QTa3{q&aoB z;c_460Je_9JBf}`zb|6n$`RhUjND9zjeUc`z-+-4*_*6Jo;MOb5ZA`eF-h>kX$G%5 z^y1FgGkm>&uQI?t?h57=Sa5(_E`d!s#G)9tSknmAOJ-`9;uraG5X;{wtt+_Sy%nqH zXW`I*a_tX)KYZ_?{eT!u(8kY$s@oY!OeDhO6O`HkLE=ZC+aPwE9_>lt`a zQqORs^jQ#n$kINkQivXl046U*J&d|s3(|W+ZHZz$^>8Lx-9kooZJ&8P?nr}8ugW*M z2$nzX1Kb29x1^<#6Ykaff=s~rZelgNMGOtj60S?pf<*>P<6(v02hZzO)01o*oG@{ z=jiQh7Wej12S6u)i|jLmU-*yl-^nk79wVYmvN&x4?CRHp#S0TIIU}-zA-_*go#V z$XCPX0Cz;gP>*owF_KrTM~H`lkIX#(4g4DD8L+{SZsm3XJnbk9DzrV2y3PQ3Od8`IFlEo#^5xt!FbSDPUVbu zT+`R5d4e^~)YQ~NgutOlfW@_5_wAbC>38X_xZw+af(ku4yug0w(ZZ6<&-W}p!HSVG zA+(O1W~kL^4d|;X9WTE{(OGm(kq7cF%ibg+SavTaR$}5NU0Ikjyo3ue#iCc2^g-cQ zsy&xgL-xjVQnS#DAqT6R)4Bq#Xjmjwu>~*esU680yQO6|csQ)o z21+e(#?kcsjIz3v4D8yO_U7FC22TP{Z!BM`0Gpx;i9Uq+FqX%KczYr?;Z95I@Kz&E z#~eytrx5nYQz5uTj8MqLVZ|mBO2%8zs-@7BW6f3)nKxU0!Oj_GO8gM1*Jn6nBK{7i z)@%WpC(>opX~P-!@FJca>{C(6+2?Tue*(HdIVIjh2#EOzyd=L5 z`YCq@c|+^d+l3PVyjQsszrg(&_Ncs#c(hJ2st|Ps{+l0YhspL!K+Nv>F*L2*{s(^G zd*|&3)L^8iCq(9}lvnf8)g+KIuq4_cW+N693J1eC;KP%X6BdV2x7>pyqqPF6Oi&gu z8*JEPNMlso68m1FGx}4wF?hAd83+3MTR*UVB>ynGiNYilr4j5S#m&RY!^y)gNHIz@ zGOw4bZ{!_#zbH84w~2TRJjcBux|BEvoW`ByO=E_r7PB7nE@5u4u2BpnFQA#(tlF0Yi;UMiVHCSc`+=EyCgD?xTM-?EHcd&muk!B&@NjESPBBXtLEw&$?vD`Am==xmv2QJX3ba~*!p=QVXBQN9G zJ}GC*LSR{{hJk|>+wur-FV|zT8UI5ADPsdH>k7yygz}DQVV;-E4E2tBCf0=t#jTD8 zyDiQ>)>#(yW2jn^Cv~<0L8Ex*D3dy7WB(NntsI2_=y2F7^F|G=82=fERTh*H&p=6=yJEa5o>*zPY09ld3 zo5#y*wNn9;)X^fZ4{G6n~Lzya3E2RfkrBH@3ayoYMY2@B@zDG)!V?6Rpz8@rfLJ49EZ7)?LEma=j{bxSL$a;ToF zR(ihZh#kFB#uB%aXf?zRWZ&OXB1m^E<%NApdnkTO{fP8GoVCO;b!e>gS;#E*@8fh**v3Rd=aqCV2}QqtT}jbV5!fsiFNZ}{;}N|;e{(bNmxoh0+&v0amxVu=T=-sX&6n&jVbgs-r#4vz74_0V5VgLU zgJaK3k&8#)KIrO1y5rtaXKpOwIDR~G%CM?w_9_MOA_7JBVz@e&P9RNG^WLZxbxG05 zguWyFgSt24>xw&*UGamGFkhHiRa-5aW3Rac5PhGi7@ud+n%Ny7vjg_Gop zX2j;?Y|Trurb3(s_mG`3z3s{FRK|=mos}@%SlbdR3*ug^P?{le#**~LB?IW<)Jca` zztbXXX4*w_E>e#&O=5j9qi*u`k2Nk#Rx=qeXaZHoHz;jJLC=|))2V{1c9_%H^m=}Q zb=;pw{Wl}w@PJO}9GX~8{+;jrBO|&BFU6rAj@&bEFK#6Ssw*65{Y^}6#@w{TABNWa zPC++o7N_09euS!<$o^N*tLenPv*sodO*|faQ3x3J2u!g%pt^Jm=z?95(85;1Q;&_L z+FAtdYplRUarEu%qgaB9QA&ulLtn)hMidOx4m*f(7iVLA&ScJnIy@8Ju^1~Vxh%EL zvE*8;otR4r#A0ZhXxJn}L{wbSgg}n?{GLo2E@>ZPKLqX={s=);#}`S|eo4omSgR`( zb3jM+(MPc$+O8M0=*0$-AIQgk}KwC zvJ3LZ7pF-p;98q2)Jet*^j~B@#rJW~2So)-(G4X3hqfU&Ku}#<2sF3;C%XgR`&FS| z4NBX>bH@cB8lqvr2f@OiN>$TRVC}k$^G7~ywO{o+6+VIzC~Uua`RGrc08T$pz6F$R z>hn`XJ%v~nX~ssHrJH7KrWXxOk7ENM42{*U)$I1J4(vx~UoE&ON!W8-b4DGgN&wyH zHF>6pGT;tlsm?|}tnRzVsfTF?G7Ve0|?8syZDnlLl`G4ljO@H(=Ur7?h+c}lfV!a_DQub*aV9i zk&Lkp1L%{?y@C^dSBS^5eX6$d&<;1e4T5dPVBVJ3+lb51<^0%Xg5vm9tZ1ohV|R2` zImCZV)NLt#{$J!ly=tiAlhe^K04+N`{6Xi)dUbh!>)iC`DAgF^ZrNJyAMq1-3_I@L-zee0&^v-;OFib`yz`T#m?`n};Y+HRsnN}& zst1`$$VO4*V+}w`SPpus?eVJWB<&1Kv}WOp)U{Oa8wHh4l26QT8xij$kFhTZ4Ttj* zb2q=7?F|q>GAbFbxNpMm@ckk_%JQgTX3AdSJ`UFd-CqN5*2-s%PtcYI%` zKqPqa?q5Fi)L)KIX@A~#C@)?A)^hKNUIX6HQK6d=hO{`<%D5_c+{&H9VaJ}`HIIL(U0IQrVGVU&>2<$L_{VZE4O%3ZEuml z@)eG;nNf)|)_a;$*OO6SF`-Q%GZD?RklX0<@C=zxKoKMnw@-=dl@wR33lzUNP=q0L ze@{ASeA*5zHQgTn-p*I~M}GdYRS4hX&VMmC4y~D^SrhfKK@Ap@8M0YY*M77^f9^*Rr7<|l{=@LmX@+wjk z4!;{q`$Znx4+(`SS_^5s!DIR&7Apn`{UJwwD|ltlPlhr3YPwb8Yy$)@t!qybI+6ok zb)M$;VZ~|1aan|`-n5%sWLn72FOQii;{FxGLbVFo%AI?-ng)$w6aO&V;Cn|o{<83u zT0aTE1UrSK=CKRQwsmm}GExw@M%G4XrYPq6cu3M0$K^Y^|0{tsg#bkE)&r%V>o)B7 zb&u80ch-(UPQBrl22pV0BnIZo(9>XE5+$7u*jo~;O-;qMERrk5o|Jt@dK`IUB&r@l zB4(Wwx=(74lroH-f`8s!<#iY=lyce{hV{w0#sVB`VJX1}$~4n33eT`W znIml#V`gVOCVm_GZPR;S_(IHeK(y$~V)A~-M~*J`c3hEt-FD*30Y9qseasTp?r_mm ztg82(zjHBgfJH|@AsEo{YbX~(2#`+le^pfHG!sFFFw(t^|0Re#ve*{$H4(H;P`V?cN9B5F@)_r>j9A8Yto3ii`3 zktk9wrC5;cL!M~Y)=LWCQPdD$FwBc6F-qtb5D7X#uKFL85Bi}FYR^j>+yY5RvPR}O z=~xgN;;1NbLLBU*?Ex@;qr99eZ|XpK$hY5hYWY62eiYa*i;&2{S%L!eP%{jvOK;D|A%-&kEK}`PfX> z%$JK32AVzq@}&EkN74qH^*@ix|nr6KB@Mrc{jTr`4zQoPMQh>UaxPPbz~pQS0x zB6pk1ro9K??3N*zyGi=qx5txcqzw156SIEf&LLu|5<-K2 zjwA!+E_Vb1qD79B6u`j^rGW$Csz<6AX<$WPZBqj5 zXP&SEe9(v964#j4dTfg8w>ZqA;u;Hd+@l*_`a(bHmZoTA%1OdM?xf-#nYUuen&2}c z%;FJrNxF^ln!n#G|15iULIxnGDcc@Zhag@cpxDL-tkT?)i8 z9VM-1j#Y3bMTx>LuOx(%l#NzOBRnc63CocT+1{pMt{PE7X-WzUu>{jgk0=VbC#D%$ z786R3DhD!X=G)^{z<}tarBPB6nzRbhv?~?(8ImG~xJqflD#h*WHS)^m%0b4JvVz3Z zx%gLw7+wcT4sT{4Jf6kO#&%y|1ZIMq=ZXdE2xUYcreYtSWL*TNV${)!MPCw4NDZ+i zyD6D7ZDsW-S+l3XquFps5{@BRbB2DLu;TL28<5y-%$-*<1(9S;a-}!La@U%o#w_)i zv^A$_JDMXq@Q?T}H0O}jT43fZ4eDTc1OXklgogPCkue$zP>psGzVi3;v$!RHBi$l@ z?K8q1@1CBMyDOsWJIO3^mOXsDh3<0S2OxLN02^(C5_|^z^3U_?{kP#yky005i@$V0 z5?NA3wu;!UCzE-5gxU8^{LTt-_PZ1%rTG1UNKiN}PUGD!OZ5Ks1GO4Rcfo? zwS}$wCRW%84K=#FoU&5W`S2qV_lwaHC4tS9XfzrSZ)KRG-eyEvI-BJYqxk{PYKs`=_tBCfxQi6iq=qXLIb}su4BRs8Fg{Q z{iZohxYm+Fc5xEJ+dZ?@ag#5CuU}rJAqLgR4E@7WkCk&%*5$F4Fe3l(i+J}KMWJmT zf{r1(lZ$9gp+qs!r}lJWEUl8goBemuuaV`FE2yQksS3@3Is3FtC%#YL9R-RD62-4$)l69G%U+t zs_5m-n$#mvv?>_*=_I+wG(z5Jl?!p(iAh&ar4px>@W46O4=%7%1kHp##-W(5RKr`a-=Rz z6pC69_4+(;#PBm&v&`!E3x#v0S;d+WA7%AOq_f43PT0_r=zWIOX-rjiv>+H?>NoIi zPG)wrp~*3J`oyY`UZ-;KUrb5HpCw8P$;{7`r3ae_N?pMx;1`VcGXze7)P zKkNeQ1Ezd+-1DIS7OU^MQ2e?qe0B-~=3|ePq@%_SwjH1$pN>+XF;7WOw4`})F0Z0D zlM>5R3S?gzQ80iu54B5CamtTM5}GGP9$QkwBS*=JL?=VE@JhM(N5yi+247HgP*gFdx4Z1L=#=9G1WeXeh26MB^_nM#Phu+efxcEpwv zG-vk1$*`xuE0u^|W(^}Pu!Vt`EI=_?lioUPq4JJa^xyQpXn8kh8BbV~)hvyu=rjf^ zJDM@0@(*I)cqH!RZo$OxO%mO>=LO_KIpzOicH^2fOS%O$%)J;aZoxS#-x)A{dw?cq z(HTM;xTkz$0Z>Q8MrarK-e+9+-Fa}mKUj=o{bj%yW+3+5D%npNy~m1YU<+Bkn9ysC zs`_irPaVoguTQU0H;$3usHl6RB((alqJjpfse+><{X?*G+NHqgN%AUm<~Rv7rL-@i z<1IcUlq4y8o(koplb2(chmHrO1S9LzLEwJFJN5#k&+`IlH^!( z#jM|>k?wpb-MfdWAdkr;I|#Dy>Tq~?=OWe91B4c{nQe+W@0POL9NN5o9XC6Dsy_qb zAVGeiJkkkEP>3CalSW^rB2s!K-Xx0jm^OtkY7t9Oxkb{joML}U5D?;c$TD>#4P=7r zC`NvOlX{%pE7?qvA8k+M8pKw3K_qm(U*u1m#6Sat8PkbRU@v1h=s{B~)#;Z?l`sFI zV&6_3t;CH!)-p>>O@dD^gN)beeCNTT$jUW>M+je7xztXWOkawtp=V|3KE)nv+Y`AT z>)9IUlL$}sQ<8z0&lZOSlxy}*ALgZJ;ycN-g1EC$oS3tuK*H~oyZk_TOZZZnM)C53 zAAYq%UP0kNs$v*xT$BR6`&~Eo=p}JE)i8AP7mgPEBRA~e+U(T*M=zFe$ph~wD-Vp> z)V0v-QD2kr{IS_aGPoXIU|%aktCrPc>?XLV6ikacfHq>=8-%x6vzw>HFNhcKx@3=e zEsj^tmS!Ks!wg$GM~@mN<6RU(vt+23OcshYGPRt- zM#wm6;aD7`bZ+Tq&|Hz3Y1Gq-Ti~wCW@aCrrypG83LPgQ)f7H%jL^qJ*q_JH4@(C> znX$UjSRw$i<%&7Ov=pNwOd!?ig4j*UPA_*Fv3N)ql16Mzoo9$~Cl)_5d=ewz3_)w1 zfQQ#mv_PA3F59u@!?TT#2aAi;C31}6lBV@s+9*U1SybYW$0n6(qJR@iQB0dURWf#%a{cr*HQ|GtVcevF zuI)zy<SjH=&2!y(r{g2||V@IexN7`n~p%a30+#-pCaP7c* zX$5{$o5uN?YZ>|RQrn!SX~c4OgUxh^0bP{cs#t4a-P?SZuUL3FVzwHFXU^q#{FtDT z@Zo4D#}~YF92g^AD=*&fW32iIV>-YwIud>xwi;FfLSL_2~U#bWyetx(Wz3tZp3;(z=e3ZKwrJA8CT zZE>nbxa^ArEQ~VE7vJiTS}grh$LVpgbT+-oCrkeF`fa3+ z4D3C5Z8WQD9&@xDoYuruP!T(DsV$pG;gu0@?q^xX3u>MFZkmmj$dV#LdOa~T#KUzVM_{=;zXU98o#p8Dr{fDVkX ztmK&!Zu!+C<02KvpUMuyE00l9a~t)AfX$I9)w*VT$>{Ls<~?SsbP|$-B!4e7>$W3F z?IM+9+E@4ydf6}ZZ(#SQG3$9a%(@UiqiHKWAKrv<;=CcgeW*haeLC(awnD2zq%_X@ z!ete4jB`>}!0VkS?C(~Z{Y~cE^AKu6K)j8f#&))B*lZY#%25!~sGn;%CMuxViI$X6 zM)=#aI`4dkP8Dtt2Rv;>#gp*??(FAm?%U-4kaJLoT8H<& zPYtcJ%nL3nB9bhcM$Q~*4?iP&xXJkL54GEVs0>8xg}=8Onasa!a5u;f@y5$upxr z9M%cdqvNs#DEXmBT`P3Mkt*gbBO>M+6Syh0{i}s9$-8>PGRGLEOq3Cq>$`iDMjBtH z*1jgb>m}|>Q<})Puh)ZAJO*aQXEWO`GAyRcg*;U8k!PlNm0X*Xh?`t0*Sp)c1RtAf zx2m;2n#-6S`!zQoJG3d?f(7Mn4y>uTw^yL&p~u*yf{vtixwC{1%6XU9*H-zl_P$kq z#re4ruz)#p#q$Wm+wn$wxGy7)g5Ev&gM>P{^YSFhlk>;QPih(GpTY(Dj>~RW!z)&v zOavsy+WV1Q0&U)t)x-oa9+b_d-|vE-coNiV`*2xT`a}zt!+-q(uD!QGOuxByW66H6 zsTr~Ea(UbNfz}E4G4X~d!jdndw}pV2KsP&QX5A#oT;Su$Aiu_2zw?i6LZ6x*Zi@-S zoHXA5+!OmcPAT#UV=;3v|9Q3Ul*!}cOi&B&Mac#}X4?sU z`-buZA43&y`&Drc?>o%=iaoziRB0Mxoc*G&{R?5}g7^TFU-?4` z!cIb_&WOaVL+ZB|TZ5xDe#iTRrv)c2X??sD1kY9KNEr((j8$yNx2j z%59##b;<39rk*eQikHf{=JkixdrQ#UX@;L}K|jSMjl*0gu;dFEjegNv!?v#0^nGwl z-b&vsI7!QB3#5L+gg{JknUVkUVxH*$we*0b@gGG{FXoR;Qlcn5*S}ASfHp}XzqL6& zGnNm7k!8gC>yi&OEzQK2+UgG4GoAyrL{7AuV!`$4ZL3>6QcXZVJ`Ln=a( zgrWSxayew+^E>{cGpXc(FqIN*iBy{6b7~C$>PJfDEUxYO1W^sHI3j6U^*o+jJX35B zq~JupPm5|A8r88eoK*{R{MQWgAn6+O+Ivq)4Tz?z|1rH<7$!YcK2z3Nk z!0)c@t5qrsw$xLq8?`$6Ou*~zez!CupMp!|sJ2=}1<|2l59a_eUiW=r(J2a9$=s9) z3es{S=YNnI!OUe-7Z|K+kzAtIGgJ9&wq2?PkXFVU#ZDZVX;gAE2NiQ{4W{yC^@%0* zAI{V2_;NH7Y13xr60+`31-aXxu<&`|bsIEKOp>Y@O|off+vrk>s|C6ma9QJm<^4}m zlV=fiICaAO$vA-Wy%eHlOl4-k!;~t$GO3@Dv+}7FLV-6^cjpIqO@zu4()`;4(9>%h zCYTO`5wS`1TNg;1;dV!3_@WMbsANJ$F@+$@{`R;aj55AlzTUBOl%yKww+Mt}*>JQO z#au?Yt#1(GQYm=Kmah&ki|5NF9eu=-q`;~`arU5qv4Mf9Ebpz>62VN9DKwNpn6$ST4G?%fyby{)jux6@4fmmx_vjbAIb{bY( z7M_D1rF=87+fqx^T&hqy9Cq|eR6radq;LZ#+&&}Et_~&dO{)WHp?-mPSanIXW*vXS z@xTGFR2ih!?%ivrS&dhuZ3=l(PaeT#=daqO%i;phS13(6%9~3^=vHwB(3*)$)Twey zC<}$5a2}4AaGTNOEkUBwC|(894W6~4VC1Vblxvw=e#UFCbc&BFE`elLa|g=#NSIe8 zRdT$B+lk_?!Yvl=@p(}*U^9j)8}~dyJzJH^BI}&bCC60>Xy&mwOl5%I9U~&n!9b4L zfzBC+-Up@gKFjGM6$p(o>>Q&elYvnS!Wa&yH@nN}RFmjO>TcD=53{AV|-s+>lK zPRpNFo^*4Gg6jU~?t?>ybuxk8b<{0(da0^oGQA4i3IH7O1-5xu%~q3-8rpL6jm9BJts!3f$TixY9H7TGdQD2#cOeZ5FNd)SHQQ|ci>{AU8cqAC zxucsIRmfE$cLz|~_9jcL-DU30fXXRtu2Dw%I2jdMWdc#Kt;C4K*i=I;p-NN0#8Q zV_c=$1fJVNb96$vSlrZxeYulDzQvkz*cd=Iw?a+5SRciT*|@+s>6wefvwDimsJxcG z*OLy5Pd_iH`=NAnk-#SPf=k28DxOm$L;`v^`%#*Je1=qB=Vh4|$aOApdO3Vgoke4> z6yV3>Ag`&YF+dZVDkgo8B-87l$o?UuK4Xhd!xiizRn8{#~D^hoEahP7L zndYi)@by?HU)kJN`NES9y}_&=?bJ}YFKeBCD|gVl<3fSn=gIhbbRwXsWEhK?w?Uu`cxydHs}pT4tQq#% zq)mL7!7ZE9?sJ=5cN6;7?yd=^LZ!8DN-COunZj0aVf1Te9a3F4d-~`tN9{&~ua#8U zZ4(L9zNVgT+2x9ubSifzrPb137K>~G?`RZT7{1oHni5+HL{z7eI({ice!SV&tx`{U z(`es%lwN5z>%McT#gtj;`7lN`OK_OMD>v`$)dyd<0+On}F;*NrCH6B(G>ny3iFbX@JF~Bm^jn)%1S0w~UEO2&iOWp3&pZK|i1O1wm##U`<)w{Lr%w~P zcCn=LmkKT$F?a~LnpnVK{rPEArk<3UEuVPj2w%}SU*GP^wnR5JiQTH6A@7j@J-a73 zj%HfGJ5DnX<8M0Zm;jR6GuX(T-QdX=*_0j07-F7bH8EAZ*eKT1LayB z>QpO#jqPs!RZT%W}_S5Xb^cv_cwDe++T02{u5Z&659^uWFryD_R;ulXK2ftXG?7Dw@(EfK#_ ziM{#QPZ_e%(8+8n-DQfbLMV_$iFD{I0&{_Q7C^k#T>dYmMaypRo&PRILe~+%O09$3}?vbeA`yr{Y{3Rpf?IJAn9nH34ica zd59g*r%u1YdVNNFC5^$9UHC25%-A(s29rUwNm#aY)Mka$YTwaO*z+;^GhxonH9~(Hn(Txof@4t@fBo6!yztNawO8#AXh?sah8{KQaYb;>j=7 zn5_4SAH8N<<8<1bAd<34%Hq*%T3l7I#?oPUjrgS(7O56nOlMSzy>ysSpGoH(?J{S} zKI-7@=*EHcSOZ~$I<3i)aJLO)ylxjV#Z)jqi;qE($H@;X>lxLJWwa@ z@$W9r`U8O?b*uIJ$wtOZ_Tq7UR7DnCK2;-5t9P+g zA8wC99UeE~>RY$*C5o;W0FUXKp(efeu~US_hj&gSRc86H7e~5DIq&1p z4qng@ajpw%9-BN6-=4e@H%2$~9*3JLVbu9$4d1mly$OnZiiQ_L)ZA)L>B4_!_!XUo zJu6>~w&g~3yUuo4g3JQpoQ(JFl53gJT|G66^E&J#+(Kf(fbAI5n4^9&ExSbw7l?Gk zWxFfp5WyQw54WTzYg)zG(bjo)$g647taLxRsMGY|E~y_-OU~m>NyIZp!%)~@4eu19 zS%sA+qD>ZKXW@IY-QbVfeU1_tW5Vy&ZX^&pbO>nm-0|t&N;@&xc)<&`?3q9tSZUPBndd%D~l z`bjKAk>`}M^%Zw;oaSqnRk;A*eL;wi@`A6G zP2OALTRg|uCekXs8N zdmbx~*-^1Wjal8MS6yrNM)}HaVdF~j*_>Q|jR7gitoIc<`o}pWjZNl+k%=<2_jcdf z{e{(tj+BrgVvg;4Wa6e)Ofi1RdQ@ zfCqA(V2OdFGo<`B6#CU93ytX__G>8lk_1i0%?>!YA-Fd<1*4Heup*j!_1j$qxCAP2 z(QK7M8SvmhTmg0(sQ|WFs#?AYk}-R{!MIR;cIt7r9~wL><)`GW$O@_LHlUx;wNVcF(@ zWM|87XPm!MbKkr$7VNAu8%Z3;=O7)y-Bht9$V-EipKeK*fR^r2iN4?ux>Aka&z5l( zjf4=05~D?UQZ&TVlC&v$X{?Lc4z|&g+%Q(nWOk-D*51XfjJdJ+nfqDK7X9J``%;EJD@^cRLwZr?b&dhs zDDXUUI|;VEt^}W~@H#JzyA5y78YgHkO4JMRMx%r}P7b6Kd4~CO5S0(P9AC1VYx%@b zkCndgoF(dQTN!q`D7!7+=h-RS>vE=!>l(u9M0`R37zLfU6 z(ywX{MklQazda&tQ)*AG(uh>!mJkZRFy!^x-G1R@Gi#=v!lsxGjKHuMtCnGu$ko@_ zmIyncO~1UEB~z?H(bsU4~;@MRQnmRnbewzqC(W=ZZOL!AAekYAIa)Gn_k(sD4+v zNZ3zej_(I%R zbF8Z~@n0K>2Xn+I7&EHwFNF&3B^le}lngg?Zskb@{_U;~(F3mN0f{{WQ_x=VKW!p> zVS$qSm%F<5J3WfOI_UWR%J!45c}mk~ByXAph+sgraMlG#H6CLyztz6U$}pchrrgu; zqO3ihmI&p?UAnnaJKq*EDamW-O;8|QjN`k>(&=;>WCllF&3A_7xI_=l3hcW|G3Ph} zWQyFj!k&09sgLJEOfJK>#u4kTF$1rJ``k7r~i_M*m#KU zt6e0OZT(X@pgjZpDocCyTAM$ac0G>D*BUbVl|>_o*-N7?tA+Ygfv9ReO=l8VzsP#y zkjc)Ir8`ranfO|t-F~Lz&G@@KqSpt88TdaZ!}MM96k`=}I~m8=icLz`J}g}2Tu_d* zs*y=p-Rsus6)Lla*VzFmrz+SGzOKA>+xo=Ie>+={rODDpgNKx=J30=RYX*_Ov5?pWV7MuS35!-ofFWsT^N$00N z4y{-&V_9}9BQK*y7vD64Nz^;*=?=#|aV`D@dxZie0t=%zb5{DD^m__jQ5@DX=}<$* zt;o4yUMw^lJ#U|Grr+QyX^#Gc&$v~|?N;5&z*E0e3`GCpDVSoplo?f3O+%!6_AM(S z%=z;p7g!_qtzjSN+hU1EJeO4nnSCPN+Iv%8=O@1Ty}?-KE&DY7w#LD9TDNTY`}9ry zVGS#hbdp=kxde(Zg`N9EQ<)J`c$=rp);)tbx^;KFgK4sv3K@@lZDeex#Su>CW#w#l zuwo{kYWZNf-Tcqw`W`Rb?X6LYa)and}7%<7h(wQ0i29^T||;0Z<-mmcos0Pmu8WCKUSF zOn$JZpXwUh(a2vDcBjo%3>%>q<;D&X&qWH=B(p6^bIqIE+C#t}*h-d454*`Cot0@->gQ7BHeqt@%k{NJ z`DT~3k1GR4yLg-xB#^xEU*&P2MA zs?NKUXiM`?SmibMRtYRl~kCAxD(_L(^mj@7#zLBz+j040Z=NOyL8 z&?%)ynJErZ_fHxny3L6{Q*OSSji-`vPD@I#z8a^7^Gv17>D#qx zfvdI_>{v7Hbg9moQq1_WcT7xn2 z!(pJQLDfee0q>VB!wA-2(@=GAVyQz@0;R3+0&wOO68le#s=cCB2l ze+OuiN_*HZPh8Sd)zm6e&I#NSqn%2=v>X@IE&^L$tr?FKi!yrDY^K?$i^x%n*-Wz9 zZmAQ6njTlCvL3eEs)5VIpk6BTRM)f1Xup!JS3mRelfiJ+AFu5_2gTZ;S{m>d&>TPJ z{Ae&JcNmy=wRyOjtz_;D@}v2vQ>$>{r%JV6zFrtZYO ztC{k=(9F>J8n)5iV4E4hcSQ>G(P%NK;+SEwT~jul$!SIPw#8<55^K=^7(a^Cwv!qW zge;1Ef22}uGH}N4?&_#eDAexNlhbi4zwV)O4=%}GwY=(5`daUqbhWY@)74$D<$P>V zA8W~B^Q}y3HSd=27T=m|u9X?pToA*9T68pM%H3#~(r&LXo5*~@D&cvmGn{b}E;5zI zw6hx(nIrmwUSK#|QEX2v#zX_?FNQ4LcTKFlY>xCM4KGToGtP7-ZDEy+XWAMxraBun z^DNq2W^sj1li;{8ZT58VC+6a!(Qj3l=p(yHx3O4tr=rV);(3uN`h4zP9ztwBi@(v4If37%K zBlR~XvwmvPF3=9wwwuJTU;gWSF}!sC#4Tv6MJ zGz`tcd|a=SL`_D$@SxG#mKDiIjcs$)5Tj1t660-Q-1_I>TMdTI$@oas;)!l$6YEW; zuS0ai#+r@QV5qrP@0bRo&3deMp<}Ddo5HFyof5mmdfDP~D%yX;mz%rB__)T|-c>7$ zcCA30+!<`!$8vu}`@rkFs?*)F(=cga>5`}Y;U8*&PamqA>GU-4!zj1X%Wk(ysHC@s zr=`wjd0~OhY(vY@zNGoY?Rb)n`n0=z0Z^LD#dHs7p?nCtWM|RaOsst#(&p~atZw}H zZXRE=bL=cOM{o6cz27}wo*_fOdT}xAa)r0fjp1OmS`edG5A#`LT<%$-rv z#by_~YQNs8VTzEydS7-~8Sq71NttwG(&=x&BtGUzV_sS*<4+6K@?hL7Hi89D_02Fp zU+e4a5c6GcGA_swW)Gu9V>M_k%%E$Te1Ep+Y<}a%5NIyTyaMfSA`f3L#ce^*Duh2cTxpWL!H`>vx*`HnwG5?rtu{~-G)YBvT zNvqtc{)*~eyD|PxlTOp2KGUMU#bng|V-NfxKt$+OZhD|E}Ui$~n3lBXmm&>V;<@Ut7H4ZQ=1ipY$KIB+X>v}9RP^JU} zLXwsB0K^zDWbCkaI;>d44gN08W2x^kcjN`1d!lQFOWf;lS2f@Add&#?FDG-$$g=gI z0+RR4E%)MkEzyKiIoenWOUx}%haMJnZVnSBI!f$p{4|&ZjoX4)*c9&H5%N%^ zM^cra&lrZOD|{<@nzrY~n@m#bRpY(y*!ehN=ljJ|I#U{2C?IS{_wQ-nVTerR$m*L)12xPl)7r0R@$W z6Bo*NuzxSjg6a$g#|pYY8r zVyHj*X4i7E4L|kJ8&GvS{Uk&k^i7)>awNkc{Mhmu;)wBi-PcXj9<(NEJuR+>BBz0C zyW6WDjIB1EieVFtt!m1m2XM$N%-woXV7U&S54qH0byjui{QC=?wnI|#T5l=0=;=eK z3mNAoSdl~$Zj&r4nwnAf0~;9az2q&RyRwxH4B}f`&||NY2mHc|OE`Z%@3*1#M8nj^ zDyBh*?&pikcE6h%AtmW}a=5}K8hHwx6JZJ7Ew`g#} z9(I6~GB}77_v@$1pVEj`iC?<(3TR_!uC;T;_y9?_yYXje-LQq_CK_dAJChjoo9f#h zOxTjPTos7;O7Q5R=oj!mCJt5WLf6?*2b6_lkQ}P71ol81yqU1bm=8x{+|JMq(Z9*h zr_> z-L@SXJ#8~E^B>l;Q<0cr%soZmG!rwT>n{1ku*{tvq?Zf(HBWO8eU~1R&-=1O^?uj5USv_94&%{>yTDn`FeU4f(eGL_(< zQL*W<-ZIFtj-GvwtNRxMP-@yzh`@PYOGtJYx@B+W%dw(Xrpxw7hWzWyhSjBSWe6%~ zNAkl%n$^;UhOcvs(xRDa5yxDIDwE|c6iC`HLu`4&8MW9>NFI%#YOkG!$usG_tooT{i zesYhw!b)tg{H2o#ShsMWHW+gUXgO>BD({@XIQXzd8^KS;6mND6=;&*o+oq0Qd|@b; zX@4uK)Gx;czoX53ix0|@8AFIt-9zZ!u#6)w29}10W4#l$R7IC{mIqHmkth@U4{nVH zdaX6(hQxT2wmoajxW*KF1?WVLW>70@mX5gF*2P(+uvefho$u6xJ))meB#{Vw>Oh7X zY6Ty}o7h`np0@`qYjtlzl3|;5D;0ygEbL5b$RSY}vg+TFvBIQ+S@oLzZ>`;bqFLnorQ@7A54)o5UL z`quCJnRyfVOu>BcOh`=*T(?8!LzXmRW0APn8mbS{6Fq(0kUH0rQ9xthVjhqX1v54ERquGJ;JHg z=lnPRYKzUjE;IvU0~wK8d-U4kCj8$4tBaV`JXwnVVV;c2>{icp<1tk9qCEf&`u%XZ zN}Y3I`)<9#yA@Bxph{rzY(UQ2irgvRYTk66U7lh4Wa|=8)_Ivf+dLQC4{P@R2Qlbf z>`5~$X@@*Vv7w?%Ug-5!PUA1$9qM%AhARMQbjM?o#(H)u^bCZMCurrAby+mZo$@Jd zTFWUhylS{?QouB+FWfBNhg;m{8^_GzWY)?7ePvqf9>&nUWstPjJ~(h@p2JSzeV#o+ zSWNF=UK6C6R5iPNZ%kB2W<*kQFCY@tbS5TRIyU(${M`;FLdy1SSwoqEEbT3Cspkq-y* zpU&|{Qos$PcqHrMj$f4gQ>slvo$WuhDkg{(&}-#$#D3f4z-V7p{6R+cETK5gtA2k> zEoIO6;;(REH51MXk@krGW4PuaPqn)^J%gERa--Oh;IrrDDrkIikQiz!P*%!<<#Il3)we>#oqYn4GbgPvME?1o$hpS`txGF z*)9&)QO-4O`oR<}hK;kerCh`3d(ufIZX>V9!+LBjZK)voq3XS?*Fc=s(2u8kTLS++swX^>Sib@j%SO*H-oKU!~>L?|^i_$Ec-{%{+Da` zrbV5mMO_JdUi-&T`=3Ct#Rz3(@ZSNBrl>-I#ZuJYPKN}0p{fA2K!sT}XfW=Q&=!QI z_~3Yi4kl36u3vkjclSt%@gSpt@$tdVW;vSfV8P56GFxcBm2<4h5X!@F5PolHmC#DA zqP2c}!S=p=H`wp+nZNs)oV2Q+)CE;lNl~LL8F)bbr2jS=d=0o@o_=R2%k2H8zomTa z9`5_4J3RZR@4-D*I}>sUMz@zZu$#70)R_889^_Z|NqP3e;d?J_O4~(GBccArz|JEj zm5}9Wc&UhC*xR}@{mfJ+JHxg5-pg+mR6p(&nD?Ju4bT^2pu9_Jp}z>}D%lFOw5wUt zsq}CvWUXNS!s@KuG#D>^mEg{_xAim#D_=8Il-0EOLbLs!pe)@V6Z{-xbCt@-W0A_Z zeet*0uyeopC^E_s`8;<$#-lXLU4nObB^aK__~1&)(fDBJJK<1?PFK23R?PR+?b?yv z{uoI$n~0|>4E(tHg4eqJ!%ragAiJ6$Mxg_zd{5V!?JH;PP`K`|Kg%myvcok#v7mB! z4X->~1rIEsdhNaLtuIgk^b;SWg*uNBOp5R1WP#l{_{SuB-2@?`S?F;@yhUy-N2HE%pXOOZMO<*1Zd(3k1u<=rRtA%X4eb>Y;&xoygNdqFKT7tsV(vY#U>x@G)v= z@eRE(8S^1GvRP@7@2h$~%|VMoAa>E(=yo;q(m$oyS^QWi8gG8y1O@K-SpC~3+0={J zHptN};#|!mgk;qGHpVsLb#U>!CdR)%(-`?u4QRFIFw9GSqsBdR&50_rfojV$ z#@hCRL`JuGES6~QxCfu@1S9Csmb`|&)MJ3EA8p#OAngajYsfN+6GnB|)G`xi3*+zj z07DyJ>e&JBQ-)`!LU$;04p64L_FvKW+=?x9pWZa)L2|#+` ziu|05ZXT?@{xIWscFXO!sdC!s3f}W$$Uchkt1#$$TAdE)3~T2$O#c0JP^^Q*oMyO4 z>@xbGQ?E-~-lG4CbWs2Wm#c)iZ1EI*P&g&G2D)og^)9%Sds5AarorjZET2;ugT;IT?%OrjctG1_|4 zr7&sSRMPs)6g}6y2aU+GniHbBcLUmwHD+ZqNb7HkRxAmBqNx71dR4L)3jlM`ddaXw zFRDJ*E0J%-hf;&yo)r@V1y3Za9&Zz+QuI*~l~RA7K&t;cy}#jXM)|>Zn${kJ$#KMv z6x@u=oW%_Gwcgzh_@Ll&mr$R`8iaJKXV#dzD2Gsv6kA&GhQdZr+bRhq&1Ui$=u58g zq_oBjw=c5oe)R62StJip)!18u18BPVpzW;`-~BCGkE*s9?t5M#Oigc0WF}k9GZ~D6 z%K;A0s@YsA(~T0Io@F^v#mQ=+kYXd6+JUQ;odEFCRMF+pBWfuAoNY`fVN$%lU2PVyJ@{d^SBD)dl;b zU6s=m2Z1cytHR0${9y;MnPXC?V$q`6bCka=QvATyvP&oCe6*>bOxj6}W_<&1cu5V^R4 zn>FK{L+ra)zNGQqJF#0Ro3L_3Y-PY=vzbQCM6h;C>I^rrx#crb$<+7cg)rE>uoT3Z^e5SyrbHFNL?+qa&SE z7~8&Ao#E8v z#h>6(h}|*Ab%l5p+rmXRIxzwVSy73lj`1zS$*KX36iS)T0~K#tj(UU^q4$}{?1Sy7 zi!fj486;hdKq(TP5h|SZk?A838WYpRe2DKRq|xE`zmsV0dQ|SqC6Hv-1HLJMmnUUW zeXu^ikLqgiR~YgLqsB>MF8J5ShTzOxIW!WP-P3M~5#WE3oSLz;bXr8=qxj&Q++w@M zPJ{0vak>jDDPUQvFBj_Ayk03mb+*E=+T)qAOzwgq$S~=pHFMjg`DV-h9gw6J?5u-n zw^7E&5Yb*7X&gJV5+fwt^0-_M0VmS~7)j|{lGMX=>NvRU#ygnF_>e60o$}U$#$_7l zT!+!rBOa#fSEfHw$f=F4s!5IT6tuEtMk9WC8LjQ0_ntImIiM~sWmS{Sk`I{3A%bfu zoBw(mGs;+s*A*O#S50qoGroh1Fy?@}v+b5#$hWOSHEnvrr>56bE+t67Z15da6Nuov zdbE&^fPP|6fkp;GY|WTVto$R%0l9~nvJm@nXYJjFkU`)FPgJ{Bt{mao!F&>*#CtZR zj3Mi^4I-}U85M~;W}awyZ^gM%`hyVp1n&!CyM9S4v#~`HiH+%+oZfXB`~t`}T>V=i`2=pV`35J?o{;eWPk=YjqZZPl&Rzz6{uAK;i#!XPdLfSu9@OWYtx}B(c}Apry>I1= z3eO*wyhKE+ovoeO;I6~#(n#Ji0b1sW9W?TsURSX+yGl~LhlIwxx(n_XeRb!&ebApS zzE2p_qLT(9p4Mqvt8?l0CJq4YJZ<$_0AAWeo3`giH zyKU}&ew7j|VsP&P3H3=$PiRaopp++Y-gjKynLJ_^ZC%WCJC|yS5U938-i2@Pa+%)? z67@QDcc-MRFsvNom?e`W^GT%sNm4prf%Uf)UpjLO40p-pK^#<;%5uXWTwvs=z~!;w zwK&1b-!gdwK`^wF|L(y)bZ0|JD0czJb9?WqncPvwDm)Rw3N;&zGKO^Hj9h9bjvUU- z9ELQ4Z{DIkOTuCpU#-q=#tZW+oGvSfz*#EkhvC&B{1QxrVl4`~g5b0)qI{T$aaY}A zEcQda8zS}lq!@;P1x~fG)A=8E>rpvlkmDhao|3S}<|mL4tQA1uE?fbK_Ft9WtM7IF zl@frY$U5EDo(L7#a*1j_*kUKq+WK-sCWqUK^eP?~GfRs$W)B{O=gqX*>IHh6r{n$V z@~)#E-v5wv{oQZUDQ89CH;hWXTGl;uzc|C)3|!-P2ea4xH}o-qpqdBYrnboP$9$HN zl#%zB!QOW@c@Vb9Pfx~ob#-v093CwKC2sM$tZ%)h?Q`5HNMQZA1(LtLj8;_Z(elpy zBz_>WVq%NZc5lY)^)L;vP*PMCy-LMPXK2~ob2oWRL?CR+_J&Dk=T%oS^|@~u2sb8= zYjLpAOu472QNe}^Y-hMMAYC4ug84^@FTQ9ugy1C?TMSHzzt=u-gLk9UTv^z-E0Bk^ zPv1eW)xm-FR@59w3>LP}?sv;!%AH=X(#_3rT#1f)s%*#>DLVFQ32#zHTZ?6icc*;q zfwJS>7f>8twNb^K7oTWjPVzh^R%Lvn`R90~xf(TLsgCh_@%MCPxdsCv#%~?>7ni%(R9a{O6+#+7hxFp_8hWNcj-hkADFUv z#wk5W;u#!uY~t9M_}t5aDAPrCbE7)?Pl@q2VRZ#O*JrJ~`B_C}EN;cZR@}EVqtJJQ zTB&05GglslC@t#ryAn5W{Hx!G)6#yhm(Z4$6s5}*0KwrT@`yj8Mw!Cn@L9i43={?5 z65vwus30s)YFEf&=HSV%*@D)vk(M)*4T`;M^JXR^y3wOnanfSMrqHa6$SSq@7xY`4<`c+yM( z=Z6ReDI-Ew1!zAwUr`yMv4E_`lh0c??>AWHfT6c<=))Xe;=P`Y62RM8+SfhE!d2e^ zaB3nEOG5=wgH^kNq<@dB;+Q;pHCZiOGG=1Mf(ompKAL-jU!Ae}b>6i#$otBNIUuCI zvG=kg*Xc#^2lja`v)WWR1LFC`>Pa2ThXW}rimr9lNkJdyHxCgU~JHXl^J#^n=E)0;H;o6IoZmI%RxYBiKzZQZLVp?Tlu8dP?EF7$q*Y zV=6;Tczq_Y>1M0E(3)-f-2Ck`F+rmrYrs} zJ_2p~T=6jU-cTuT9_RvKzE8)$iz-rJWQasGf5e=$bxVe0z7Ov*;U04-8+@x zJM4RBke}(h=Fq>x+7m;eA6YYNhU9tS1am9f7ExSUw%7)ZCD)Yc96&Zp!VFli9`Que z&q9*PHudHNx%h4Q6~u&$zur&hwSaQlI|`~u4_uY3==>g04+frrK;3$Z3+5PeAZ9S2 z$ufIX-+`oaC!Vm|7*(`obJ1c2x0yMr;I|ckNzBkb>MvtPz-J<|Y=uI7+Q=2bj>A1E z9CA>KF(2Vu5+iw=RsRCp1jqb=X*$Y^h+%>koudOs`81|}{f+HUdx?NS#7uYe@tQij z`wY}Pngf>ZQCh1pmzQJVEiw%L9GP+%LtHz$s;Vg)2Ww?5Tug3WA{53& z%dkQ%B@59~tvHB0z{bqTDMwwFkPSfVF4v|vw+yFA`cmjtdWJva(n4Bs8KElYPYdtX z$P(+z<|!elpnb(`CYwDH_0tTVnF>-B?u&GRQT&;xSbI%tA`V9P;)aRf&H|uQ7UhjF z*K`QciVDd76C3fek)lyhaAN5{!Y;G z)U?@~j*DsNx9%!N)or5>Mq}dwHI0tXH67a-hy(j#+s@R4*(C_c{jJUPkgZiEa_l6XDb!Qac%r$n_tcZKxG(ZO6|45w$zHN!L$*Yg?pxTnKQ{XpKs}-9 zVFBHTAPyR#M@xl~UvaAGMLi2CyG^L9H%R&|mfCk~hf=gB>i$2B)2yuM^%sk%RUKBf z1@BR&w;AQKgPdtw-qbOveZl2-lrYZ(F>|8xC1P&f%qjlZy zSZK%)MecEemu-Zxp#&i8)FQ*-(eF6vjq#nNT6o$Tx5}|l{zYD+ecrCjF%e!sYY{vR zD^n(Qgc%#XDN4oFG()WK!Yx@Hh`p-`)7`TD%U^v$H@dGUKUD1yJ$#k+G2bibYwl(S z9JYa-eUrhz$|6ctnaDj0prlW`qpeGzC={5;%1>Gpys$*HQR*fQdk*sG@#XY5h9gmeO5K#^*yKyXM7XPm21v@>Ch1e+Kv`>*{fLx{tI~nf{j7@-*C=$0{)i zXSYIVo|ROOSWDPw9@;QIAK$NLh2%tkLeEu9utStLZAEmx?#1DyNuwvErQTD;L~XJX z=ZtQ7>N~?WZn3()hZr?=K3Ly5q$$a@avL@D{yySm72f)g9x$Xj2Hs&|bl0;w9W=`d z)_Xty=5o0?SL<~=0-^;v<+nrAqoFwpIT?$B({_D6>Pe1aJXimH=Z_7Da)e03<9htj zMeJX$ytt!H}*y zJ;QX>bX{v;J@aLB4p4ZGtGViZ2uR_<6eyg%3tA^6mu{!|Mo3U$*rBI?Zr44}*h9!| zu1J%GVp2Y;w%$F7JVDF9p|H+-$A9l(zsAU_ytX5mVXLVlkksi zod60)!QW54`C@|4G>Z1GzlH}HgbVKrFWvv{B5|0!ij~)QD(92;@}5IB(mP86cpX$> z4c)(&%l9v9+KY26u2WOB(O)&F(>H-(=BPuJxO3H=;6j_$8U(+A9NR#v@Sp~IfODO2VWwK%gF`o_E4`=(PN^djzOqL-UTF>_b zx}|Ap$ocL+QwNq~p;-PHl9jo`y|4raOP87QJlA(*5a&T~nYP%2`~{X1g0vdDh^jut zbq@4_%v=e%nq1(F8er2T*L(WjBB(YoOUj;PmC~{i4j!@g*ugQC65^&CLBRl2jrILt z$6DS_#0ZUIQChTc>hnfPXjYMq2zKK!THvQS316C*4NA9EvPUC4>&phYrCo3_99~(~ zOW~@2WF_l=uFB`ZP7e_b&F;Xh956sgY42G^b<8hqgAn%2fS2$@&p>XZFFwhwUU601 z;kxC>30=pZ>Eq;X7*%ebR41L*JLvjejE0ceg**QikouC3gLLfl7>V>ALsE?9fce|F z&41?~2+_bD#(@0$L25TFW+2_kd*tP|g~Vp9zz+aBJ=m)1((}Xor9$5AVFZq{7H4EA#8n&>8ZN$V(a- z=X}#_$ZC5i3CF8#Ml-ILi{R8+*~#KnKXlYWxqrKYieYd1gtgs{CW~MzUUEO>BP*QD zo6t--3iAeW&26BJ()D5Ifz!&0*snsTAibf{_Y$)|z8nzMT(;wMw9hopbf+T1Z%y`^ zFfg!?ahB5!J1^xfz=b>5Z-IfL_1j0 zeorlBpp$K_`f8M`werf~YtZ7>Ktiv1ju^OQBoK|>)x7zp^ zsMDv}2t@uIC1RMRjME>heiG;RE270cxbY%kHqR^Wn_nHpsnEqIeaivYT%q~+`i*LZ zxya3yOnL{Q8wA*IseLl+U4{Dg=4rH=Q}*>!r&6GOTIe>^wv2h7^_&meKxFgdu^ZCg6*=*~v@25fJS4py7w< zK3x3gz}&GsNU(t{W+UPlSH)vC5pwEbIU2tu<88wmPP5qE-n#H~G%bszujjd;(Uk&& ziJHg9P(`k<9SiAW&EASryI5cO`elYCCxx8EdOSJ|8caxi$D$T2 z^&3NGsZAMYbUmKkY5t4z6RvWc>tDG2wA34h=ry#Ax(^c)#;?0+&5{@`i_Ry46(I!T)1 zq{mh$CW_9$$6jl+(xEF7o>op~3_y)H{_*?6z_blZ@u%PqSP^a~>N!x(59<~kAad|K z*f8oIY4T$XFJ@{BZ09h*Af#RuV8NomQ&)%TDtXh*S-crR;w}PSED13dkm@p0b*8|S zq`DpS1t9clc1Ih+g}z#AkISVY>0SxQLIthlMSpH`oGJhIpR(msxB%MbQ&?O|iQv=8 zY1Gctu=TZ=Ip$5NPac+J@&115?a1-xn_$120qw0H_JGZ6lSzoDHdX7;Wb=E&z7*ttP zx!2?GQ}QD6ay0X<{$5}4j9_N>Hn^3L<5dt>7tMH|bD9|{SUF6SN7dhTYolCJ_LF2} zXcWnCwtTc#t$&ng)jaaG&r-U7QOKT-*il+HUN(UB<J(iYaA#J#w3@A=ek{nUj|K8$d+sPVwog43L#9%HDd zBn)t61Ayjku;vfyt-DD)L?no7O2i%t#$^5(Q$xLHl|<6+K;z}kxB>;H%{Wh8?L&M0 zDay^N*J&jzXedr*!n=17UYN94<#VAseybxp@Q@-qF+CrV$73?hr?aEaz&MPJ#^5TF z_P4+Il!$@KB8CXKZbNRp8zKd$Je=V1Sw7-4;t(4v)B zuHfWAmcW(IpZwOfy(+jLiD1)u#~A<1&u=CdXC~BKccIR`r{06Q!JqaxieOJ{B#K1R z(GR`nct_~Z-K{`o1)VIXg{pVg6ZT-dQ*Ko`jL(vzG9TJGZ|x9SU${8$bwy34c|7BV zHwp0P6I=6Y#_Jl4o`5b`#Jwmx2U{3#}7R#T`kjS@`685+{87;+s z7o>!|_#6qsriyhk9k+P_{#t>bZn43jpSinr)?o(x1&lu-mJ?jacD#~dMpWn0F(FW$ z^o07_@^gc~m3DzN^q=tFr$Zg3Lwy+8s{sGs%{8zKKic!*g?)EixlN$wnzrjg&gNIA zwgMFTUv^n#RUtO@PUIymI_pXzJrU*x*;6Nr&ZUdpBM+0_pz`J;TYdYyDyUWI`tO!(~lV;)H)PlZhP*Q=hr z9Q*Y-V{uE%Hge;CVa6*jC?bwjsq*zELutcNNeOJ6GJP2-_|Y*qQsZMJ60}*H?s6|4 zH$ITX%KHh5Z*o#dP^1O%eL$U`_(lM;a4I3Eu`(rNDxIGyY3{m&Vv(&q6~Xr2Chlmd z=|?Oiutm@FILgO(H{nN?u28WRjy~P82|`FPAdOOP=arsPsF&#{ti%6aAe+8?z5RHq zrPq|&gyNppA!7NFAOYt_5DZFc0SJeY_MAdL%I9rcl%Mi>lNO z1r0G;U7n7cq_p{5)6-?FSWvE=ipaZe@g=qLpF z!LlIOA_@)9+qDWT`eV!aO0@2KU;~HeN2gBw*8$vj4Mk0=ZU{Fd-zjt2HO$m(HXEaCyqtSmsHtGEf3i_IgoQA5`OKhYZT9HX6Xrtj@(To80&GP7h-}l4C&t2{?(3C;WkFM(uo8 zG&wkIA?(w(G{!`~JpMyno>JQrh4N`8gynP_qH)~Nf)6%T6Ox$!y(6Wo0=%_#eoPJN z-QQ=P*I&ZI=fTV>fj1M*$Lwg8nE~?Dh=|tf5iFaxZ{dr35&LdhhmhMqClaxmU|i6eijB$J7iEX z;!}fy;%d0`8+&uh3zN-9BE3gCLev*)nn6$xO*O-`ITA!KQMR)n6vDn;(CftyzPfVY zS9es8Dn7pM3O#NGfxNEumBPObZoEy8jYF(j$QqN0mdz8lC^oiVq__gK&?Rl&a>K+N zo`eXu98?vzOgP+nh5xPL)JQU-Ow_>1YO0=#SX?KrJSI>f5pGKc^hFO!PfdD zMor=8P)Ax!QmhP#P?oS({4FkZu%UH$8A)o9`DgOAHxkv&n|_&&^vp5umpB>ZIX`v3 zU{0ymxo4GGQAH8o!oR^-+Vs&M&RY4)Q)6`PqhPIf9&eOR{58+bGT{;aoQ@TDxcnvB zyn)qX+D#_gO~aHZVyre&XG_>uw6%&OgE#1NXU>-)EuBiMe%%95L)^!u!ZEniry*nj z+H|*A22N3rWo&VUL|oa6w&N+6`inLyX?XPR-^mUciLj?k8|ff;3-} zX$(TNx8`{?U3!;YNWM=%rUD0CP`|Tz zI^@9*%KI=NCfs-L#ffU)t#n=#_b>P@9ObX+%6J@=h}@bT>irZ*$)~@?^Q9G$F9{9| zFkM*T-eU4K&O^87M0!EucBVYXRLLoB#|OrDg}$Xz5@K|_mabPg@2R69Dd&&##%!u9 zTKw}Z6Kdt03C1YhoQL2>%;uL#N9MS!<{FaQn;IX&*jq*c5m(0ZxQE_z8SI2}6TZ9W zsliUUaoQD}Jf)}g#FUBvSuFN(6+Jl0Q>KjJ=SNIpkPc5*1`12rT|-FTo3>~~rT5|x zoqh;DAl*up?=fJ1SM71j1};+jsjEWKDBK7lLk$_wT(wy#ZL_RvkO|=&!cndPOaH4V ztZ3|a)j#+QtM#`ZeX{L12|)?9BP8%+M3M}`3lVh+mXpu?%7DvHu*{h2e8<8-qk1Kh zX-K7eHUnnRvuvu?Dcv>%LJc!o*r)xu1bJyzZwO^3O z$TQ+>5^bP@YKZ3VRez+?zmlab9e+x?JU}K_5L?k38?{WYRWK~E4;}Mmy!A|rHeE$e zgtX;eh`?+`b$4$)O!YHAhp?~p(C zV<&WFrKm>qr0^^h{q>$S?0y*f$z5}o@5Iz1bnv95Ww%Ugds-VGD!+04#Yg`hDcz@1 z5?M69X_Om!haQNCphbQ0X2o(}l#UMetN(E!FDkxu8OICa9h+97R5QdZo@HUHalP&U% zNT;;|gb?fu|Gud0%n?;*TEKKJT)PFQNc3V*JIPj5jrd>0E%9l!+^^p$S^fY|f$cf`#^8govr5n_QSl?F^txR*JE9CXC z*%|pao_An3DRb^(wk*{jH9AapC)7h&Y;aY;;mrOvFm=d%Z)i#lDGy!eI3DqBI0`Qw z_Im{(*$}|LXN=9Q98m=-#JOs7F%(}X-Atrx@yyGS+&C*hw+>>lrS~|qgG=eh^p-(^ zrrWVDj#-st2(99sSwM zX^)c=^?NORp+oze(F6R|h}w6&EL#B$o?vyg%iiN=V(hwA_kXXY9wLn(rsp`49FRtx z>{L3i&Z(YCavx{ckQ3UOp#50OiA7sD`NWuh`vFd*Wvu^Cz2`HdMl++1KK1ASKjp5Z zsS(=$Z1ZqaTi1^CCOGYi|6}W%wnR~)C0n*_yLQ>OZQHhO+qP}nwr$()(-{49^C3SW z$BK+tF-i1mxaz5as`v$w>f?j}abc@lCIyiy7wu2T(8I!4shEHgB&~vRa`znv=b9YC}<GQ?42*0V-WyDzprhA;G}{i$D6r_PxFH+!=Zh}+yQ0& zK>;?Zr_4X%rju?7FVmHJY57LiB>VwSalu2Ok*fnT;>tdV>EX{Cp zlb2(9*={_H90}m6jK~$j2|#yvu{% zwl^?W(veqPaBcd?@tf*Drqtwy!_m)G+KYlLB(gs(!Dr2U`3kB4E!cOS6@7xfrGk~| z;CksItZpmg?Rt!ki!qn48kA~Ev_^fR_M)$u>T<^%5k^KsT@{Fw>k@3YLdTFFWFwvS zY#m?qfm`zBWPe&lTWHS>_3dU0&)Ag4pxpvCpPRJM+u$D)@iN!a;bDo;XZ&+1vxHEZ z?4S^&^&5cCyH4<13$Dt>+Y0be2c6^oqwx6ko6q#>wSTP;BM3hU15)6WH6R##*69>H z4lwim2(JfuVO3SeCDI!~EUQvjzr;7WP|*pWApRzCGor47Thom(GlTiu z6VW=1vF{^3-}qAS&D`m?HD1bUiC5t*B7u(3LgGCB{(v)#Q3Qq zH<3J$AMXccqyU_N+|4tIm*TG%O8}!ssc7{&)Tm)wd?DiEE*9-iLW15{ZeGb&;AM`KP2oXz^1AzAD+i>ccCTCDy9DO0srX{0E zGPD*dv@ZYw7ulTtemHgA(&na7Ag13NiALWhChxt0#p)c&@HF^tx>w7}XhTY@l3BcN2PlRz~Z*_XUAO>2*4)7nqg6OKy|pI@zTv8pAXx_U;Knt7?v#nv!Hbr-+VbuVwUGi*p$X4Plm5Db;MVfe z=!YvQl@j5uAM^+`2Z{}!%*L$7B0p|V9$tL7u_-@AIg5gWemM5$+eR!t9AiBZEon8J z11i{-YM#Pz%ER+T&M9qBot4L~eq5DgpL*L5*43>z5~gcJB}waCEIA(uooCbFn_&Vt zOHX>N1dxE^n%mxB8b(eg)P;LLN3kJLdqHRakt%G+lrmWD1XrvR=}(>*BJxDMj)_d} z#vN+30oORSS;@~1wK3vxH_+m4WAbu>AF*{tRPj2bdvGVerQeukY!;pO_U~)Ak2!bP~s)N6ZSkTwK74IFgWhpo@^N^p$-FpKGzlxC>D^gBe<-oOg_=*2!Z>@QN z&zI{Cr1P*>Jv9(QDW5w5)cI7Ulo$2=dPgm}ATj)qGtbV8A1T^T7={cm4qm^X!ELU*h&tz|Il+LkOAtv#o|3?ZBaE;ARtsBlXEvn|p1+ zm9%S%m(pOdeqfwi=afu+fF6<6E5ifu4PcR<%pM;Z_a6^Q3+(19P+S_u(@#X5OO7Im z%K2s`j>vTaZLl-U5-O=!WOB4eGbce?DjHJCL;Ppu$J#3M0UM=GWu_>N9wMeTyXbV)I&6oJCPE<#q3)4O$yC)-87ilPW5zIe6QZ~Ck0Bic5lE1$R*r1gYp4$z_ki?(FS<@d&}oxl}8Th^eBQBwV`FiD_};Az*LK9cikL0 z+7(Ms%g+_Nc?Dwa@zG-*9tO^LE6UEO75F$Y7zHU4F6d z6ZKV8w&J^h*pUo5H~W~}t3wH7xi|Eb`Md9`9j;R}@>C_VC)mV4oW;rR!|1%{2z^&R z@rGuV?EjyoK&dmf-%)eaJFDb__@upa@^AxkRgiAy@nb%+G^@H?8hVv}-BY z=GFr>+v}$YRq;a{T4`>rkiS>h&-M5ccE3$2w>bKZyKP82ob8~whW5=6UmY+&N#5I= zxc=Kp^}cAXKRKQE1rSe`pT?6o{hJ(%?KFS^Yc_pX0_TiR7I5Sl{LQ0Uu6YReWV}@~ zHJoGPB;koAoRH}ly%~}lzio+q;YNmE7B+x61yShXkG_J~Bw7NmS+nO}6vk&cu2F!X ziVw|#jTSDtk1n;=2^eG2<}c$SWlA>+SStaRciCE-6|(+T+p?`X+2)k@#3B?4=F2nN zoig~hs6Mv8=5@2B41~CP<*1vX)VQdFvfzP{TZDv|U$(0k7z5eu3gpO9Lt!XSc&4*< z{dM*uGt;T+w{iIG`b2aw^yDi`Z#IUp@obx<+(oQx68MlcCTNsLaHdX8>M40)aph=ta-#_-cIecM?P;sj zc!=X=YeYWj=iE*uT56iVA|<2T$5U|D_Tk&R$pra|(2SlMZ02dWuw2CUo|J~j#6V2+ zD-cfC-9r0$i?DgLgN^V_RzLFUsh#zUk>{@UtlCD*Cd-xMyb<=`Ds|KXhGCx6r#$b; zR~sdna$h;1IK}dg)x>Y`q*mbJ^1+|t$RIp#uX_#{AHBk_U;hmzGp!d%v<96-yJJMQ zp}?}R^d2@Xk|*I#ezXWo9m36m`s8H~dCOE%=v}_qy+R42JWd>FGyOXQVpkcvZj;36U{ z3w>onG5C>Fo=J(toT_AQzf*BL%j1eW&+t6Eh^ZyM{TT^l;YTx|NKPMZm#$uy^E}uM zIjkxaU7-SKm7k2k8ttNsvHLSSS2MW?+ZvWe?r(-jgDSEYZpb4E-xYr&ZJOp&} zlMtPCj82?Oa+#zl?-G60*{wI?HCCC=KMIPrd~g;}^N71K@#Xy78ZB~RZ5HMEJ;)#b zq~#x0bs9QS7I-TWm~V;Q^UZlZw-LCOUuP9$JV3##xdk~*UzJg`JjgRhduTR?P(3PZ z#_0%WYN`q<2fc!LCxIbrtWFv{GM#@rk!W%yojKWbfiUWnMwZOnlsw7+ouUOfT|`FQ z)8Wo-3T}*YFrQELy4M6JiAskK&*SJwv2Y9pqx)Fp5#?z?zz1}ZvbG+h?#O*LvJU0tBi0ak$3NIrm00*qdROkFS%zgJ zy6W|r*WE0MRF?5ElYOYu|14bnuFO3^l~f0nGEvxt)J-S4_8puQP1OHz%_P?%8cvLz z*N5AeIP63h7j5G1OQ;1tcoy1LbOAxG<^r`}UG*`!usI=8P1ax^3n=BUm2)FM8{?`- z-6hHWh!%W<#s}Oc>kYXyc1!s?I0~chU-cMML4h#dZsM%J2$KpoW_8*bmaR?bD5@`> zGW@b5lYXET*u6asFTgaJ6|*&zb*Dg2U=PG*!4|~uNPGiEf?s;4^Si^|G0`jB=>e6F z=WlH|mZE*qXvPssh-F-cAx_{>t-e7di+DquBswJ%R#gRrg$_35VamADyNGa+wnRD5 zHBTnb>*dsu-R@r3<@_B@yPN(2NT4rNvr} z@*jFeOz<=ncaR`vXU_M2i-8_Vbb9~peX}%%f@TV-xm+BF)v|rG!U0N z+oJ1{Z|&JyR_b%3ebmwvjhu2d5}l~_N{A2lcJzsN!INJd-Xfk(VxLQM{ZZv~=F+cn z_B&ryXzpLxn>Ix6-sgB@UQjlC+PLAJw#135hh}|Bh+x8{Vp|B~9Ip`FxNz%XA5enn z!pTRsCBibvf7iz`^>F0ceQH4uFy0P3Pl^aKL*h<~0d0bSm_#o{n2}RQf)hAi@hNrv zQ=>n-8+O9|&3A>xO?BXxi;vF-fOOUvai}8BdfOqM-ZWMSwK`di%OvGLvyBAIEo`M2 ze)7%8<`YX=!!B2tZ4ij=MfSfyk-iiisQ1P^>h$6SGT!5d?4Dh#UOr$sq0*!uf*+I` z!Ok67AnB4Hz2g}Qi!(Hz*^!tUTjV?w0}1IT?ftc_e4qNO$iSD4SK;$SD6Imy&b1(+ zaG)5xvH&}AAYPQMPU7bsj~r#*Ug0cvf6mYJ8H=%Mx@sRk4f3bT>8*+7iEXn)t_Sdh zVz>5t-~Wn4QX@qt3pLd7RYkw!k+%VtjS8KCM_J=3XT*<)VD7D^cNc+>9n2VjVh(k1X%5&g2A4l>s}ZxBSnkpCyHt14Ks>-Mz)Xg2OzW_16V z+-`U+op=DmY<+s4RQQam1BY2YoH<(w{m07$>GVJXV}V!#5No&T6Gy+EsX-DYNmYKaF?L<5$AQJ7^8+#Ca6#q&* z6FiZ3DCy%VU$y&2BH9pdu?xlDi9C4LDA9c2-f>b$JNe@={~;5Ng2BUo*@-r$d`5t~ zV+JWH#xgT-Y?iGM&etcbecsIiCt)5jzs1eT$2uvreHXKiR6@1m>ou6juoY`Agktx! zTe<;xh4Vqxgu$BH?Hi5NbF&RFB_*+1%TH{k#S{t8E3EleidX+3DxOmaya_M#K39+~ z$Wm;&(JeYx~)T@MFW#5r(Vs%6b zXe*Q=B|P4?;1fbUG#Zju$>=3Y5b8%&-L;<&R8vn5z^kKv_XQA4Ivm3UU)eO`3=_Va zVA^eR^3{EfJ^7M2Csm@PmMt;5YYF1~uW~Ymqrb9|ciXn?zd|BNlNTNFv(Hd$GT6mk zp}htS4;K7&dpKuq#{HsrkN3x5A5R#--c>!roG2jdd4o_OfdXQ34-Tr}{rM1mjD2IX zv<>qcqzDt%&y3H7x&5p?l+JB1gjm8IeJRn6H6#BJRD_ELx&C+Yv&uxx&c09Iie|5? zV6x(ptE(r$*ggU24(G1%fV& zJ01h0Wf3bWCg>TWMxz(7FuOyHJ)DY$sp-3ohnvNZBCglM>fc~V0tJj%Q)GN1vx4N`SjoUQxZ6!ax{k9K8;(I>sU{%7E@DNh2Hy~Ou;La6VwT*&0z;Fd} zKpz5BFP+-ZtIHxCJ19&A@{Y~ujoXAidi3aftadx!urUsShi+{6SLEycQ#-bI8?nxW z$0E~_S<+B&@ilx}Zs_Mps?EuRt-cDs)06^Q$OkzFaVlb-c8xTg-ylKh)u^58m5^7X zvTe~UVr!;ahJxKK{Qy?mT{$))ar#I86>T%B#lx7^VjpuhN zEhe2BPE=ypdNCA{n;ej||0=l5@W%i_f$(^9cjo^Kz~ho7_lAkv51KM~e#)C#Yc?}! zsiVytbfCeAacU|{FH7{dqOHcPyQbeXxgohavtt>>8Qk!480cUk$9yV?Yu(hg8efp+ zzFtOAX2}lCk9oDWk2bLV0N1+(A+}B@(dNC@?KJD)hY)ES`RwT0{9MU%ecEn|ylLxC z2wp>Zi%@NiWjHHRbk=kmqkCVye>TV4Khzw|H!Wl^pwNUE z$oOCcAQD1o7$8X?!{G68Q;1ccjG**MV$eLm&qP9~K@O8i>6z7qTl zU%_CwHX1(78$a@Aw41Vhpx@bD>{PS5*gP9BSUhY;8c=9rx;FO7CNBr{Jmsyv7C^tn zmqCf_$hf9=?1OufxfDc+&1##D9s9Qz0XqqKaQMW8y(a3CtL@Ft<;itYuxq1P^e)o{ z*|}}=gOb(Eyh5_Q^vS#0?`#j(DWJ`TxE)|#Wk<5iK;U0(U-)KYbenH~-#| zK#6_1EWm%zz#6HPI(Z(q^dM#+(Ppc>elt!OnM{at2v-o{toh(v7c(v@C+`k8`+*JQ z?d46j%Vv{BDq`?`n<}UP$DTONBuJxTmjE~0FBJ#FFFtj~cgJBmp?()btNxZn=QXi; zbz}vzD0y2YIibYVw}6jCCw6j=JFf{d)M}AdetzMuy~#V+BT6|u4wy$cqz0jk0*h>X z20X*5!RB=KHWZ+3hV*3ougD<{n-b-l^eEqtrPuZc(a>hqXF7cCnFfRI9RnKue&}jt za{Aewk70phDvBiYtt*vAgL!7FIH84m=yh-i#+Z>Wn$M~Dd>30zE4m@q>D&JdnF<)=baJe~&U2`Fu9F6G?fv+A6QMx#p{mP$jDVBioYT0jqbdp1Cg2`Z6tnnd@h< zR_jDI-dljg%m+E(Sn!LrdBUbN#p}7`(uqj}VOBj1kkr$^JT8kmv{DXw3wHB{I~@&z z@BO4xg74deBc}%MB}ApLBhR&Tc)s`7U5=PBtxBO;T{~dFw^>saRH!HK*gk!NZ%aq$Rde0(M7<2ednIJOa-wtM z$6x09w>LDUI$}QAUfA4ydEuD652LK_u}9-mpgA_t(~J7j0%|{zIFV<;tCwgZgL`9#BsQA=NWh=erY+0cn1W{ z(|nOa4_NS~LoR=|myH0eKW1SVY zIqvwnsdo}zK&ox3-)fSbi%*i!Gh zmgvpIyIKpO%g@!L-;mA5A*ji|cG6tpS0}CFf)^L!L?X1nJ^K;DpS$!)9>FXYt0;WX zh0{v!Z4^%QScI8o2lG>mi-Xu(51P&(7F33^W-B+u(M!_?Vog)=XA^oiL70W09*)Ao zaN#+|5A6)H;;Be~MKFkPT!my|yIDQ=ed{}hoxzwc5eK^r+kHc&dUR`;;~C>KOHp54 z*)t;1Ik~xIR7ZR&`|-;u8j`?x($C4f@5Bha+qzyq%5)9)f*-Hb7J> zyxQ-N0pG)5fz6_bnn4PHTdR+N)3pK-Fk?sO@&58Mp7~U+ID&En|G&@$TN>g(H3};ZmSYn{J=AJ`AIL506QjA zLywgm)4oZHcGe0MmlP)N5e45i}wYHn&!(5_pVd63Q=GOe$`cQACtORTisM`Sjc_UJVfOPX9FJZdt zs&FJ~hmHT-e)SDcLMF2rVDDN#(U=>x1U?AbV8->2DkfTh80g8o(N?%&rayRA2Gd=v zPNGk<14xUk{(R9tm4RUC?J(dmv7 zLG1`^4KnwB+bH3mQxsAO8uThQ<~eJXc>3s5y(-KY4u*;{e;J`|0X$JH63eGX_8g>< zpf+q7?forEXX^C*S94eI2`%33qc!`zTkP|<9{vMAo7zl#fJlYA5m+wgXe9AQ$^q{B z{*IMUzSEANh`ZN5G9%NkLn*7|JQf0mr(f2Q;Za$~Qm{zl26-q&7m5uuyG^klKvYM6 zQSVA9p`w<9Ph~3jNPWgNpx05Y3HEVZ!|#0x5+WB)m`tu&2S_<~5Wa!;V6FxN+kXsQ zC>ve5=!xhtVG(y+{GXjl(sd~sH0bvNw^8$nOp%GRj?eja9MN=XQWc(@S1*MGpe~4B zg4IXq~jgW`rqH{i~&?UR%Nm6Lfap>3iTJFM1&X}uReS@wt zXBHgpkhc$VuXK*hl5zBI{+arV^)1&IvnCo|Z6F%=e5ESkef^(XW)KOQ1R=BUIL!0o z#l_D`_@tQ5x!7KtG`#3hPB@djy;l5xmZ%UwSnCshM~E$;DeT{S8>c@o3M+|G87 zLC;(9&^9kNy?vcb^W_Q_H$ZcC@LPjOG0USmwtKx>{nPFoiuht3NUt#i-A`<>s(Dke z4FeU&HARtGDNFjd7+gh=g~`f~JlHVPI@1ad3})W!Q#={>s13BA%h1Zne?$8V<53$Z zfWB?X0dVuo6ixNvIHJ}AuJUI-DIgaxnm ziFdL7A%ge6G%|P7>F1(uF)boEp@&KOu#V&1Eqeckhmq`-4H_3kE|t~eVNSptzmk$d zIiP&V{Mr|rD=(0wzZ&hZk(auyVIy$U)dsuhCl^n&SA+lS+~bqiES${}5o^8-jqeLk z=^kYL^V&e8M=hkPTKfW*npwU<@Hi$0ykE1XGOUoA>T?;{n#+URYNkpFsqwe6o@xb- zdUWdGMxn_Y*-sHqma{?hOT=Gwm905{D~!? zgvR5d`qZ6|<&tf!&qGGVhXe^{0pgM@Rq$m*>`uD+0|?ekT$JwiR|`I_=8k!6PQ80> ziJed<)DfSw-0=Xnh1w0K=%wG#cd#1bdaDY#;P;v53i5DUSS(rohxeE%0 zqt`Sr9YJm(YjZz{J!T#eSEK1ji}n{kE&c|ymely1&|~jviCIxgfCG(Mn`yYng5D|P zjwAL7u)^c`V2(BGw%5(!o>We)W&6~V_eW5OT|65A74KCG@7SxI+_34f>xZSdgy+wU z34!;0#^KuSp^yr9Y{c+8)Me+!0+rtc$cLaF=Ao|x=Q}zkG`Y>ITS={94jYd~k7TgN z>WzsD1S7-xl-6nDhIjD;_#~8!Ri(cTW zu=7Nk1|2ZIv>3%@f@JCjqNHo+?}=|438sDI|0O@w*wyT>p|ueJqbf)?LuOB$^q%$NCCkzdjelC!;OG=$re)N+I?C& z{n*Xq6Hh`yTw41(7IARhgri@iN{HZW*6&{dM)5!|QNJG?ekjMfOW*7{- zXNCv<9AxEPbyPGbs{xVC@wEB8%KqUvL;#X}z?-fcx&rzKQ<1Gu|*8>`RA6l7z zI$N(F`|g{rR5~@7CZd)}Vg5pgU?D2_QBQC-QezRWE(}9z(f|r~zNoqNmuq;+CW{rL zns|t4eU?De$buxekhaPaEi7j(!ba5$AWGdq#TogZjUZ&t26*AP(24e)M0|d7>$rx= z*g+c+bW-4tV6|&)=P>ru!k(7mA;$r+uw3rx)6!uSb2A)jy`3h5!tb=vB^QQ+o&&hO z*U*8Yr@UL z7}f5J76C&SXt(ytgvxtTY4D1!v3E#*jc6l$0`$CC+hJcKHWlr;%LkU{L~q%EIhar* zb%fQCDzL==jYSxP-q*)=541NBDCm%qf8(qra@eKmWg3b22j`70gvWbOE6)f z4whZd8^KT)ArSGVjB2`hwNPe~rEplPTk#_Kd(CZ*0I6dCmd#C1)CDDnmd%F8`QLC1 zHH!`@2?cvefCT}g<^;-w?x^%{D@Vml~6-~NjdzK}Tz=9R=N z3|nM@g%%+sZx^aS6aSehqx}Y?jSwBGbk~vI_UUWsUMFW{y;L*o#_UpTLJf0_f4{4u zls>9fP1}#{i6cU<8(y_QL%5m)JYCHC-W;R#;dV5nXERaN>{<`nfKg8$``E0T&)%m@ z@=$^h<}GC+Y0+)1PhD6cN6LQZQ$D8SP)wj73yHqN{kMiuva2Bc%mm0^hwM3F^>g_p23)Ywfz5V-bCoe%_-i6IED&oZmjJhmh8$bwuf?pCl0zLFd4b2q+c1;}A=$FcuQvRni%bw~K6<4t zUhEccIyMQzsOm@^DTrBQu@VJ)Fa>CjJ+8z$k&ejsrWv<;5+&9^p4 z6a}!quGm}JgD-F&rvzVn+-457pTH+$wjZphat1BLy##+O^Ed*p!?}I>*N39FMQ0>R zi@v_9SHh;UmLl9L9pCr|r7(+nzWP&83b3rv^4?=(#F7&1B7OzTA>P1A?DJ3jLW_Uj z4uKI$!1`&T6G_;Me{}$*bFm|}sWmXG!=qj6OGi^e8G1mQffDUkGXlcFedZni9o9u> zoP>tIG*mxpcSeNlPAN!*S!{`~x-%VrjOinHBURT~qMFzL8Fg@N8*y6F-d>Cf%zlz9 zU`ZG#O?iF0#__Zs5xc!q2(M_nX+J{i7}R1xu|`3)@xv{N((aU=|Y0?bwyNZi4&%Y-N>)N%X-wpX=dh*5LHNM0*V<`XoMRY1w9r zjNA)*Bu7*e7j06Zuz3jfe}ct6271o*!+!_s~YezX0Pes2^q7ynEK#G)Y;OyK^=y% zs(G7qimM0p^SC%6NG0PCZJh$Nf3}^&q+h8A>%UDaT8yCazazsUH^&+kLC-7lt)fFq z$kfUs`IuVh2rD62eAye@@4A$T4XS(`xB%AJqK)@0+~vN$CrVgh9^Ozu*4`lWiv2U|XU2qB#yL??-Z=HcZr3wzi%XuBbr8C9-@!RFp29Vcd zw)#NzS|Xf^4nd-*dd8WC*id0f=HEsQP5xFplLJ~>&A{k!?|J6R0e^%Gms& z;LPR}w`kL($M&kq)D)eHr{XS`0BR%UoyAiJA!_9TqG*QeFvPkZ<+{6NPo7!rqMSF} z^so;2D}QwZG$ho`?mDS#KI6;?L{&|^7&SowHjo^!wS2rcZlDSDS$QRrW9f3ZNm~<;rz`FD(Y=tq_;RzH8?GdnMugA@>la^$7J7icQWoVui~V6@KIua*I{a@H42MQq{;y;==#I?z z?qPrVj*3UiP(aF8JC5~x|D-){dKuTjDW+BuzIRRTaKRm^XDa@YuJ7imPqkHgMh3A> zuO2rR&xS*1bmIHmVf7>BTQx2U_Ya01ed(Phb+UZglds~rMff*eyOSndpYA5Pb@2Gc zfmj>ro<_$P50-qld{<$4!mb5G7E!Huopc>91D7>t6OUezFu)xDt)sPH2@g+?vn*X# ze@vTU>s9mokv2UyJi84=I9k#-PY5YZuil3Nz~=S18Ccd|)wfN{RCj8`!s1%c_bD!F1ABRj)$GX8FrT1>JL*Zi<~9$b2WPpT!{S8L~@NX+yO9)lb$rfMDzY`#GzOsJm0) z$Nyn{KmP|q2X^iM&>Fz(M;axQT}cOOC^|JmNd9RafRg$o&;co2n}@`gmGd7W zid09YxHl()ki>s^!xP!UIv14eyqK#=0!iqeN49kRSGVzBt^akuXZ2q{+tUo&Q`QIC zbQYVP?aucI?32dCN9PG%09w@_VD>r|4IS^!)e?*g=}$<&gLt9*O?Ni`WRTa8Sy6V) zlez>Zi_9&w+>*@tLxlBWy4T*h(@EvaD~+Fxp(@XqW|!{|i_Q0M52r@Xrjsqv)Y@1I z*+>gr^{ClT*No3SOl^tXPVbExlh0$2%bKdFnr$pv=VcG?PM>e)IQm-?yvPU2-?g}* z>mV_B0((;iCPns*6LsR^OH(<;kp@f=q3lMk??&bx+E6t|2-BaPiQ}#vNZWi&l3b&XQQ`x%s^$DL=r2Foc?u*sHWbZW!b;7F1itkO|G0^B3$imKRe+Y zFOFUd7rr~|#HHik3dSLXLv8a{%Oa;?E!GL1o6)Su5GwCanTK#@C^N5!$o0L~4ibq- zt2RLPt%YIQ7~tru%bMg!={E+en$&A3V~YJlo~lxocOZz>KoVIwt?hm|NDz*N#DE#6b>j~BXCp!eRc zg4Wqeo;2a4Q0~IJLrwaI)xb!Bxms8@7yMz(r4P$d!VaaDKtt+yyt13VMBD!+juaL8 zGmD;DWx5As*0{G0IRYdteToWx{#W{PbtVM879QloTszsNSv-UUt4wWl@SK_rrj(8d z$ZH+C!`d30x=G4@7<|d{F4mXf%oqPr-ab_ejhRM3axk4d%be0LN$#63uU)VM=P$P} z;Visim^}O(R^H_P>@9|CA;-Ljt)#Wj$r5DzB2!ZUSDZG@(U)4olgZx&o96OtCp) zHBq;lOj(AJgdKuw_j7f?B1}!UOPFje&{zl6W+0|Vm?4aVC^k6fM|nFuI}gt${!j7! zh>*lL&>Do6csYLB;aEr|1lv0@MT+RV__Wj;kBK$%(PT;WLcZt zCthV+L?OuicW*smax!k;j*R`>)Ht;^4wktL)Dk;ByVRQf&%MMQ_PyDJoJY)Hs)8i4 zd{=@sO(!O;Iql5cjuusmzStxv$~b0u>HAhk8bp_n8}MqW70wAd12JC4LoEjKJaZ~3 zKA49UCf}>!B<(h;*OfP*EJghB&*!=DnklxK6xsSBQ=!Aymz%81aZYz3p;mSu&vzt; z(098>-ru!NGhU1t@sS5Aen-myzs8G}(Q1vWc3yZ5cD0z;AMI_k_meQ_RKeNQn$>{g z4d^N13KC3~UzbR^ql4{QMa80Posg}Pv%F9NpTok%2vVZ(Hs=do)AU~s*jMa2fK7@K zpg7@m+GO)PzGI{7>}<|}fGv7HHd9vq>5)p3OPQnlr>cbJqvg)KPDj%41@H6Q&A%~9 zFwV%KR|F9pEU0+~1ooK|zEaK2iKRpDR(evKc{mWq%3&O^J-k11SMi0O1#&K;1zN=X zt=DuyB1I0Nd$a=lc#OsEgKK^2NQT_DkC-hNsUns1HwrLvEyRWRzC@{`Rd&`$)cL=RCXi&*oDVc0i!rK5bIz=L| zbg!UJ(gyccG=I73*eYicI-;lyKy$)O?-D%tv)C;4SjL&Ixb%@xxufu;e&HMJ-bhHpu7mti07*SeIg0kLx_$g*QjRV1Rh$I3*YW( ztie-F&gfwKPH?Pr1S<*$n{MGWJgH~mj7O28uoJRU;eYN|8yR*lv}C8=*p0)iI^K?J z#KAutQJ>@>+HTSNGdWPl(_VvE>GZXnGUSUT?X6B+O1}Z!qb~8@rUepjW^#_V{iYkA zZTmaZ$Lm*Fyn@+p?j8A`6Pw*vov~H0Gs(P1pC#RfItQ>c7#-x|I;^G zxl~-?wCS2`qe>3|OOc=Ux-c7Nq?OO>!qoEc0b+ge=d7#XkA)}QRm%C)Cxii??r zQZHE)$fP+2ZyOZ-i99A}e|6J;9c(-AhI%RsIz)X$7p0svV#`o6)6@=xX7<7{`K#Xn$Aj`Pa|FQ7Pe?;r7XP(# zs#svnqMg;1F==h*+jxQIzJ>0mYAaljry@*V66~Qvk{EGUB=tGR?>Gt@uJ1#cjm2}o z29IFooE0zIuM4h>x z@&R=2-3C93V6-Y$ElclPhrEQJS!;J2v_jAENdM(WE7UKDAAJ|{-O(x%{ zFk__ydahANkcxqMlOO_8Pza)%C$Tp+;+s%YnE@El*h^;^z3XfIiQ|jFs|H;*5&jkMZ5Y&WwUB6465j) z_s90cC0cL#NPoUp%fDv6q3>NKV~Pr7+RBRzI&VkJM)3J_!MuDwA%16Mlp}TrHLZ2= zBW;68^?M7XVIE#4e&6R;b`>gP%As;Bm6!jYiGKH4-Ymf`7X*v&5q4ggTiJF9^cDEP z6r_IIabu2KLU^uLC?{`pNc$T@dG>8HB6#H^-{^gvN+Fsy9D^G9)0Hh?XwpMcv^-3? z)0;Cie&nMrVvnBeGX}Qu;i0I;LqV!Ix`}DyeGf<8lM9IMA$-b^v4B~fu-1Mih|{U4 zij@Of>vQnNityhVX8TpuZLf&PIY&-h=Elik4dHrK6^eSpf@x#HvNl<4m-b0D`GlH~0|$624k<}Z$nu`03@9Twn|pp?3OUOz{y|IXbD2@^1+ zJ~9czaPejZv>XH{dH-x9iVCV3L4Z$gEpsVZpJk@vS1;eVuZS?NQ+bdUd=UOT z50y>v*U&m}7c1-|ilBg7vQ&2s8xZqWW@5ZBLmxocZ7^(~?A+=efLFO=(O)iuuQI^39MBH@<6w8=#ODtj|m4AJfk z)RT2=Lc8pOSk6} zIrtR|*hRi#F<_+?@diB#7BZ;LP%6&z7QzWZ#R%m7{(f~Nn0rn8ls3k5tHqu`GeqEBt%0u}E3b)aZ<4r-%K~@= zx86eFT4{i-Br<2Tf`R*nqm8x+mPpNne6MU=HFs)d^u3p7{ln3;lkAONR7+>^99`#& z5(446->-esUwi?$XW9SHvN>cz?PWwAo(PoUx`rii`bv5cg zD(tI8e`3COb9yjZ2iy0C-$>(wIa%M1*~!fB67fs*9A! zn!It1OggqO2x1?$DX=-ZcH=smVQyI!`{YK{NJ z);k7Q)^Wp`Z2xo`{)%to#Lzl9l z4|(e_?2*}O8Z^h(VAK!f46jPsVg}1bf~DF~JuF^RdxM`{B+v`)O?hF$>jyso7WI=K zOCko|4Wd*p@^~zc%j>5{1>j$eU$k{8)gr499&w|4S8Mtj8N&?Cl%dD9HH3x1A$`nG z#!=4u!|t;Eg(cq^G4c;6>oq>4&^L$36{IVKu{=1`h(?DwYhlti#pN!Y%4?>XVGqbC zYz#VVG)pBO@B7)cQCMC+bbegC8vYI(+Q;9hu6Qnlg1=){;c^o72=I3wt@m9w2}7!# zbSSR#Ig$s6;ib8*68R1#_kg^>FGmb7352?KqDK*|8~K zs0(Z0uWc{b9I4UV2stVRQ$F|W(?>}=xZ3aJ7csZ|B;|*_b8x$is`}^-oUh{7ap#v+ z*eZITBtH7|yU7WW zCy#vF{Vd<$tD*AzTnLJ(wzXILL5H{TUK6>EOfXli*Z>Ks+Ih)jLZ5B?a+Rv&ZYgQ< z+@po8iIXm*BTXh&gRnY?o@3OJE5hA6);$zR?E|r%7L3q5|M$wRzeBSK1D&|A$2A0? zQaF-(99x@|aL{Z;^5NNz#p%!=Kcb2KZFRf0wD!G(0F#%dqN~DU8xSscwQOgTiA+9{ zDtBfTC6!@#9LQKnb?H*>SV7=ZXEB(DkK=!m=27oS2wrzV`~9}12S}Jw73Wozq9i=dOQ9yw3Ico7jIJ_}Ev(qHR=Bgb$cpwT3( z-;RzzKm*U#*IfYhf87h2rP!Sh!?R zx-zeXv%`}Y96c4=G(_C*>JZ(-Jcv<5P5fXT_bd`i#gf(AjJQxzOr2+O7c*Sg0TvRN zf_v3++#ts+{Y@cPGHta7Zk440P%1S)|=L#!MX!3)Hm(Y3X;YyI1YY%(t8bm(DCk^ z4gAxNtpBqgfxj4HmJg#s$&{WKP^_;jyPJ$#(a|Zk?nI*)Xg3R9j^>i;czkN^QaJ1W zdwLV*^ddv%IgH4ZCPiW-C!Mk?tZ9Xq1|~3JjIyNDB*A(aX0JXu<*48RPvIWA9!@z< zK?jULIRs(2swn6-Y76njOV($;xxxdod}0{2v?b_KrdQB6zloW@UCKaR12bs@u^H!c z<`<-Q1)#q|Th@>M*+}N<@DSd&qZfvYLlW^GBoM2E0 z8>R2m=vDEhYMy#tGqc8ROF7?qa5qhe?<@PcWc&Mcg-LBRSKhcFHVkK*QxB`YbyA_V zDxLmPA>BP>irTyM>EeRL{G?Vpq^4U@3jbHgsmaqLbn@1&Yv|?Hc(!v};MivQReJ;p zLAYB`GP(=f&6kj`?R=C_Cx{*wOqSojeyRhDh> znpFOIJWNHV4*Dj!?3wfQncN`7m17x0+d@Y~-Y8f=_{p)~M1|5Rgn~c(*S10WQznMj zZ337TbO`#bgatQ!_(ocGgl>`Jb;Ug=SM9Shn_-+{NWt1--E>v+G_ZhtxwL>_!PFKrzDdw zShd*}P%#vFYt~LgXrGae-#w_#gQa>*rv5#dKa7xtFGre^=FZQiKY0nP*PfagG&`mgc5Dxrq#a_)(jH`_=WdayTbj zLLaANuo(SCANJ&}SeI&@q-0F5nQMT&tzQ7p`ALmAJ)&AxFWY`qIRO0U*I9mNU&0Z2 zMfB69w3&vXIT`uM6{hWilAc3T#$*tRdaV(`yJ>^Tp4&%bmE*|#mX#{4q{jCwCM{@5B(ZkLk2u$#7ef+qh;wlQ#|HorLT<$w-0b!BG&qFN8Rx zNc98mCSh_{5(|eLr#gL6gfE<1nl{yY0apVzk{4gM>T?CU-Hl_oxk6Qi_dxm#qskK# z;G3q%?<=2$XvX4Q--;xfwk){6hTi~JK{G!KyS#)DK3Zw-@DURd7Hno~e?W5E(B#+F zrj#UCjHRBIIK^OGCU5x95NIj6q6Y3G3f_zkeuEuemx{1WgJt;EEqM&DesvgKb?@SrgcAcwmZ)vdFbSB%7#rD6L~!aGj&vRJ=x~B zyB_y~&OTFdeenZS=2u#v?aS7WqPR`%U=3(#F(;1P#~ccfEsb7?L?k8aNppF9WVSb| z2g{}+^!R>M`B{R=m%wURd2v8t4%h*d&$$BoU^1;DT&Z||=LAD)uMJ3#J>{n$G5Jx7 zzl%&e9IVEv&M!kJC?BN3sY=lNg zMud5kut#}Q!X-Kksa=Gw_o!5SxC1{8#7aS2 zhMdVf=K6GgF`D zC^9#HhD<%cQPBrU$K-t%ptHYhQJfqq$-ggOA@~N5l`DtgBrZ@Zq(&J4+K4o^F-o`L zX{?)3sSUhm=4Q=?5O?^9yluh$gr9xYwuvyu)OP1OKYcwtF0%4JJEj2`W1cQoKMRvj z8R=jD@9GPP9XpgA`)*~v^lSCy*Z(PTN%z)Bfh``{O#JhscaTgqiLWNaje+`ywSq?Y zOxDa$etM;}FR2I@xGUoGpc#$~M0BvcM8XScD_!f-4zIb4*pGe0XBiotvG5Vg@drz3 zk#VNv!uc1k$IsW5_dMUrKbloq@9f_>i;b3dEkP2~vlj%EdD8(4+whcE@(yFzMn3{5 z*RFKTQ|}LprnC`nci_!u#lu5ImvUeeiumk-lU2QWq=aOi@Kg&kC?d z+P!OD{Kf5J7aDML)V-G7P)lpQ88g`m6tnNn_6ZIf z%J~t|3N0|U{`qx>;Mf5VNUz6H$c8?Mjg?spYh@1ZQG47a9;jM3!VxwCO<_oorez0c z;mrjJpR*I`7bjUa3y3TeA;Xh*Ha|LJ)OlP=T23=G=Hb5sRX>SSH|R6+kHgEWR|ZKR z6MyPFvbt7AQ|;XzFG<5Mmsy*h)Uct49I(zF5~Q)SADnr|o8Or9YUK31I$(@hbQi41 zD>E`L^>9{fLI8@;)o+=5o)5Gv>e6Y1o&_z46#GRH?a+nPZSZT<-FtS5=ncu$3c=SO zNymIWu9M6gYBjBBbw|$Kc|g`IH1f zjqyZ7lj3W;>>(~YDe$>|3b`}2p@?I8fSZK3E@f_NOp`d(BAX)4w_K#zfZmQ-07VU4 z6da6#1esu{)%>MwYSW3}+y}h89g>BsV`ELqV9W#R?d<;Q?B#wFZUs%CX*C41jG zB3&Ke>o*F>ntR)+5j#r?Z81ePHlaja($sMJ?!{QmF?Fa+UT@byR^4-bT($_=QGe(OU z3)-Uk90@35oC{rwUzpQW4JhiBD7s5tP^R(OIYJ)&+CmEA4J8#G2%dZ~PM_3`W9fdw z)_2;MRl3A;8wB}HL(OVW$%jy2>bRsC0;8lj>&(fPT;%!s=Tf#zZwIu!-A#NO-zVr9 zcnFwlF~s6s(Ba}P6@@)9TlrD{4!bmE5{seXSV32=4%s|tR=Syq&cCn1YP|Pz_k4ac z=E@YSd)p^^t_b73VoRHO!zPFJEOr-UbT`rF1vg57?XnqQ=Mu6s8w6uK1T)ZOvlJ&s zkmF-0YCXDPq7W`pv}J&vdB1bx@MpnHxKb3U#D_SC%#&)QL9ue4|rJ zlO0llJ4CscQ3w)hG27J_$7`L$X0{SQ!-l%p@hNvSd%$Yp!5Ovv4zyjmtuB_m0ipsa zOPE{<4h-q#=5Z4$E^(6u@N=!?dGfA*Zl)u@|H@zWV~!hc!0S|jbdA3I;|gKxO@p$) z7?s0x_$O~X@-pfu!y@Ce62NpCK$1LOL38+re@#SS#*O#=peqrhXUt4e(OxydliKKU-Rp4A`-mZ zOWr$hW2{RS{{mQv8I;$gko<%G_5CYZT5OVuDcFrLyLhJmDLA%Wk;!-KlBl`o%2O+; z;X$Gj)rdhegmC3piG}y5V|=Nrg-isY!J&TzK!_zSHvX_W^D3mP zj$Cr*a%kkk(0P4{mG57=d>-Z&8Bh^K(Dx>rBm-gXr1d!3j3%M1#F*2@)vX1Fk2Lgw zi7Ywu7SXpd@RQqxdT^S0#dU;IA^=U)>9co2c1kZ1IX0&OOK#+Y1|eWb#1O~Mp|!UB z^6%{o?uUXTt<~r@Z`U?V!fOZ4{ofOADbqq7o#caMSANd?^&4sle}wB4v)ujj)R$V= zJBmto^X6{aY~hMSg9?(-u_Sy9+%yewYX&VA_ak*3Kie=n@>dBx#epGBmY>pHA-{OKnQ%tp?Z<47&$+)_&D!cXqV`= z+lwq!Ls9m{mb${U8l}qF!XBbKmNqNR2ygm}{0`w5yd8J}to40|{W`o{xBq9m4)-&_ zxMIDBU8Rkx)~grsx2h5IGp{50shqP-p!|eDL?aGg14LXA(q9zc=r(1FmU+6D=|&Va zLU~?{0+Q~D6PH@!u0WO|*-7WeRqnk@v2|sbxqK8_phT34^ zXcyP+s-Lv3LQFT}{nG7FJq;K|4d3$I>KyW%6~y>uG;1-ZK4$Xi4C7cYn9>!d-wi*z zz;rm*!OhDYtmye_4K!M^22NCYRx^ys(Ub#?Lwh&5@4N)Os62Am+4`3qSY+CN@vKlw zTevvqeJ&tvw>ZUf*6+EjDCQ-eiC-+f_!V`|b4VZK8D$~QIT0OnE)%VB<2{~Kb%Jj8 zZXc#gj9V87q$O$0g#WrvM6g%>BwbiI>M1!!|2!w@r5Gbekna1V>zXCWD3!*98i*_t zG`d+jy}o@{76(O8;hM_nQ{tnqeo$swbo;K`s#gb-y%+%W^MbP5UU=XrbCFkXY<}p$=42;)q_J20gBu~1o6fo}HF(1UleqPqE%UM{xra7b;2nY0_V4pP z5Rgnvd-%V(%@r$lJ{xw-&-OGJLhQuXPBo^ zN*G{-F2nXQ)6;}ND{Ba&fH|an`ydweRTP=R_Wjms&&s*8FF~~-Sjl70N7#)wd-e8p zczYjYe4`(0t~q(i>gdMwRah93>2pBapqf;j$HA)+EM(?@f|N^uq{9~eE%yZE7E8)G zzvYQX=HP5tT{LT~*%6a>-xeoJqOOAlW{BsI05wa@t*`lQxM%MyG@DCB*I(qEm3|GF zD_3ri3G5hdFUvA>G^~IbO3vfRwB%JJyTM=lN1bj6Vy5_n^}ook?PJJNtiI3+h8GK7 zL+@pmJ|$CoukkCxgJ~?zx39Zz<#>}gTrLS=4Em>fUSlLqPH8oM#@LW+BVgq&F+KX` zA76*vZtiO%Q5m-A=^qEsX6ik#LAw;FPvfgC%(0dVhVR)rT@1B1zEJ|^_aB}UOuek} zunybkBv1V$%TD1#^2SIo~hIM?xQr)Ib- z+Jo1M&WT~w2hTHJ<6C*Iu~SofVEm7R%2q7C!VJb|POB5^W!G`)xjWx5b$Vz&m_vdX zc91EaM~G&e1cQcDQB)C2n@`QP*#T7%bgS29E?zYwsm~I*GCE)SWvq?cdCa2-AcO5h;DU8 zRc5XtZ*z<><#qdR%bWwc&MaU5;SJ(1vnC#`8#5>H^plr6$5rKQ6YgmblU!US9ITtb zhh%i49J)r{zgoOaYWHy&HT>AI|9fSZ(a#N2A4FzgLYUz>wxW6Eh0L&zEv?;6{=pjA z2Qo!t=NS#e3cm_Fh%tm4YU^KBbPg$c+q)sw?G=H!4MW~AhhO0VvDRJ(2RWF(Bdkuy zGu4OYt(zb}a`#>#H>Nfq?IN4&jNm>1LmB2>tp$mwun)^Muep5Gcp9sNHV;74;br%Mg*)d5p_r4aGHFvtIh9|J3uc~EU^V= zrpW6nBM4T^34mC0EavCorTdH6zSC`VH=XfEK^K2mhrcv?kK=G2S7B?fJLI2F^Zz62skd`%nGIq1-fGd<=2Vc0%oZ6x?eX2hQ~OVVT|ETE+Ah=5^d}e*lUly zzB@%o8XVBjXZ7jx+W%Mo!4;gX3aOwly%pNzm?L4ya3%e>1~q zdm8+MeG}>yYr>I`F}{9(5ju2cMvmJtehpn0jxMnC`r_EEce;!G`Zy}RbsfBD z@E6*eXB>?Q7?bw(x3;G~<&P`>`6**SmG`i9`gQg^$c)oEYl39iIjREj;9Rf0llkx} zE6!n`{nKmMBK|t%V0{Pdv~3J&lr@I`&76`ybKGc@DIWFBf>PhBAGy^z23x_BD^Pp* z+XquNk-z~?6Ye1H#dQ**p9SEOyPxI4HDalH3t2pKoQ0Sz0@lw8d+%}h#*sZfXv30v zNW1qFx;~P~VDEERX9V`*Cf@gK53_5nADNmZ>9os~u5l@3+5A{*%ggdzG-Xn-ojiLVC=5RD#J$z1Vxv&Q2lw(Z zp}=cW(aAN^@@ zP4yn6vd$1Nz9FWFVVCkJ<0mThPz$1GHPu34ST|B=xgB3fPX}gZGB;K6zur#noXw zoW{H$KWqF5oe8{rw?s_`OE%A^apvBui0fQ)u)ftn&(&)g$ieco{u9M=?C26NUb zogp!$+dP^Lb4KONQ8b)}#3iI%Ol95~H=~AF*<5Qu+^iuks|Nr3m|bM$wSha$rkInM zUB0^ZQFtd{*u23$*w3|L5**hoYXNJjMV%R{kA|$U*nP@R?mpjemsG~?eXzQuUB#@|Q8M8XA z`cdy|U2TEKoCOl(a!t)aGDU;+9#Chh_n*i$fhGaWSzfS&A|f3lGk5fS7+(J%>$XDV zUIlHxH>7Un9N-`VCU2l`(pS38NvC*6`IhU#ZuQO3O4>(xmw^%D8wYH@ymQ}MU9;Q_ zOs`N@r?8#s3!}RCkqLB%m=A&BoAegUD}YJ6!>fX>TnkEZoqm|`n?ia86Ab&OVHP3Q z^cV(H7U8Tph;(2!`-YW34DXzx&~?yQtQ96dPapX*TZ(`NO-F;e#~GOHbZw9 zvz9&bmBEB}2yc{H=r#=d%o^?Wss9nVp>WLLm=;wRBtqO6f^A@d5P~-Z_Hh~0bGA=K zpf^j%R3EmtVR{eD0^QWOWgx%r;vBWlY4l!ZOdA+;@97Q+^tz^T=kCLotW9gQvSi*@ zn~;32O`Be@r1iueQd4A(VTA)DdKfHTI(23doUV)b{dQ?j@n!?r(pi%W-i{E4yaw4l zSfj5&W^~FOLqMgb9QxP;CK|wG=*(3-x}g~oh4-}T18d9@k~#VI>ZDhIYoYkr0g<-O zoM!n=x+uc|BqHyC2n>5%pMmAK2*5NS7kdUKlIhb;)`-TIOH{+#wn$*dlqK<1>QS{R zxiHT>>qleUd-azSbw&wmS>r}xP3fw5#-dSfvu<;2#E!BiY`CryKS5^j{9f~TR?ayo z8zwAAI@4SMH|cCil0ImU02eTfOx%zW)oq3u=rycS-jIm>eoW5&JkNN+nobmJCIg!_ zh0fHF@(DVW`TCqRwQe!O47NhdYar(3+pGkE145dtIidW!gzIh#8h!vs4uvDXDt8~5 z888*mdmDKUI>ti8A5CSt4sUz25gJ;ZOoeGm{|PeXQRW%N>b=UO@w0^1_8If9ZHgl^ z*#GAH+$(0p9%uSyAxEM&sv6Fgr;cO=XnGn{$>+!}sWwBVz@Iazb<1mFI6z!kn{s|& zjUI?KqnUb|cT0B;-BGaNoaLP$YIRR!COY6|ejdt%X^NirJ5nIQpEjgsjWcGjMYC@o z&0gk6r|7m8VCS1<*>W#+W!PbN%buonYD|Q6u%P>@@ZkR7n)Iw-jV-=5ZYR`~*XL)2 zx{Wu@-&+5(M!;I)5^r4O5SWe~w~su8Hz)FNU3BlaD=(}w3x2_#JIin+f2un}L46nB zlzYH=`7}yY>mFUvZNqP#HEIoW7m7}JNP&qzDo%ZydFimDjEny}IeC%h2*DeXCT<9ILppv#!5ejKZOF=CFvoR! z9+dx{4$Mi(wWc-B8UbG1repgZGrf9`%7(M1U_0)T<9h!l`ndI<_OoWdTw64=y2;^m zv|^6%UNF;Q%UWdkN9Uri1GgI`Eiqm+hc)Y4H%cOYOIK2u;FKa{_RuVn8WFJEYih00jLG$o(t@Y zmG~@pK3wM@ZXVJG0!GD{ntqXN9-#B{3|l(6Byw*Y69w{)X)19PK^vOkJU&mwDzRpn z5}LDX>y0Kn+?9`P9H|8A&d5r+#}p_W5>oKZ#i2AO>@XZkD?cx|G2KNpfGpJ>b?0=q z*puiD&4_1pN20k{67$ZirDijygq474MFQr~>m5_kOm0Q$jJBFnt20tYoQbar`&|4y zplTua>@Luug!Id(jMaT+1=7Ay%*(tqbyHlA!#^5&pGAMG=EQ?H8|<95u_L^ATd)63 zRBkY^3o)@n#>>*b7GY+P|Npzkz&oI%UjIu}W{|%U6(3??t~AJ$jv4;CQS63z(~cVGF05 zwqfK3fC@*OGbjWOnWwU5nLL`RPLXVoyVhpZKd#fI1#S61*T%D`-O4r<4meN&(+P)7 znKU&!;?f-proHYt1p)`6$}jW3Jlyj2bFHwByq4@AnsY|{4v5yhLEJcQ#YF-Zs5#z~ z3Zsq1eP_qa6fcvaTP%5qaXWbHnWGLUO&O>)7KF}v^Fdl}h1f)Uvc9?#rd%8u+X{PB z&V1wiqfKS=Z%6PXyd(b9z?=>PD_r~R3GZ;W2;ba8mM$LTI%LtB~|A=W%J=)DP3bHL;o_Z zsMTDgC}>G~#QU4B*FD~w;aJ$Sb3!KlK7B{Q3LEofQhTc*MmYEDRX;Bz7y&~g-VW75 zI){@#*niP0o=`IbmMnX(t4kb?X(9C%a?3etVqy>F`#P5i1Kd+ddaOkX^d=3`o3ggw z{xOR`j{~_HD)w?Msk+vtBzxHkSsnf%Eb7iG&NPK0$C>Gu{F$z_YW}62w<{s;GZj(p z{wpWfQo|T97o+8t_Q+uO>hG|H8d0#lf6gzjMB#*DYrUG!H5R@KSZee8fD*Y{e(4k1C{J}Rt6cz#H61M}rdDSom#<4)K^C;kdNYQ3Tb!ld zj`p~2S@UxFO&M7V7D|sV^FMGn@^l>!6jNTN)IA#O)(uXHaI+_h9vVt94Xv3UbwNc= zx0N)4#~Mw~E4u0KznJ5!74_Ff^&c8@mK-cdeevhKxPY0qdFJ2jU*`I`ZsYfI4|wbT zEN6MN)aWywFg@bWNfY12J{s6-)~!vr^?olaanzVdU{Wfg>zJL3tEXdXRQ<-y8Rju zIFb7VtjTLNq$Lp9YOQ!rYvKTF#orH@wE^=^z1Jn2=T_=0_urcWfSc_$3Mkdm*qm7409o*-^5Ss}D075Dia zh~tqP!Ei?F{0Eei&t5k~Xl)tMx%lgg4QOO5I!pjqoSg7+2h>+xdMNCAd+Zk~?9QIJ zEm#5UGkol?tN+g)S>)e~7t7AB{$D?1VfS-fkVgV8sR;j{xoD&2GVT&Q zDH5rTs^;&6A2I@Y>HbZzW`qjtivpiLnG@xg0I zyGW3dCq*h?6{Dx&NzhO?L@E;QVkA6Kl9ENqN?;@>rS6xKQdP((;x8n}-&s&%W=)8} zG$zNL?UT}Uj4Kk~Bo%aH6_SS0ikU{Mrea|Iq%5OV1Wc-?-)IzKq*cgZSE{DrYy6~c z`6W;EPd(ueM=5xmjRI!&LM#r)PpTRlMS?%`=@0Iuka@=nxSb1e7k7oYf+vb3FAE7D zNG7Bqcp#NoQ_{LF-=YjxV_p`k#K@Gofu4+uRTm}{QeJR zVLnQh!3-^jXPAokTK21_TT264b0&bPDEt>RA&=R_qzPAZy|#oqlqh#>_GXQ^1t&fSguCHs=?@ zr)&LETn$-_4weM#YXeFi4LO2mmL#vd{h}zYnG}QvUkK00D!GO?RPbC?vTAxip-;et>~QuHWU zF}38RT&z-ZWI9W??9QnWBSG1X|*T#VFG zq%t~5-AUDa1dUR*v|nPHb}A|VG)iz3ZqhgYz`Q2&o% z){rk#;J9T@7#ueI2xBl4iqV~9ORvwxuC~Cg(jC%UZV0;nnwZU58`NrTNcmzI_bVO~ ze=!V6xJw$rJE$7(8oZagOKHtBtvlnIA7k(rBco$bv(`0_U}KlMtz*dW!nFWa;E*iA zYfwG^HrV=$;jd8z!Paejg5NG=!SkSY?{yx@+b&Jt^N=wF+`a;!!vHL|U& zv=3Wykb(s@;@Y^J2QWEiW1ke&dqO$@K_yWPs~k>}PU#EaV71h8BuhGF=t;?Ocr6 zx+|jbKT#HjVf!(FWCkY=7q7JnB;n&rG>}*Jg!KfRT`dhj7xo zqpnIW$&0zWNY=cw&@--4QwIC^+8qPfPOf=58;4Au?PLDL*NI~Sf8l4nX4H+YqBFV= z8J)exV1;g?9{mmxjh|;wd#_^(-VT`qpQL{bzZ?lepl6aoQQ-~1i^_y!TQKAZF=>JJV=vqdE93v&Pl#+t%OA_DIQo@@{DFsi)MXcD8qpB?^_;trc zm4Hb(-TP!D&vYW92&$RnSYDs!66N?y6{3oEsyRNtf1-T1 zm4U>c&n)LCfuFXOMbujiZ*49XJU^DP=9^DraxeT(63jk}DMEK;B=3NJ5g69g0VHcY z3cNuXDc0=&B!P)H`@fQ~#AnZ(VX$IP;RTs9-Rh2$PC2pj-r z|5p;8|4HI${436dGy5G87(b6F2i%7Az3q^IC)*VKfFfu4vII*o=9eYNk&58uc?L0q zA|=S#RHMW=%+S6pL5}ie3BTRsL?}vfR#_Q2w&av#jZ#uhTN!D+g_IzW1v!Q8xEvQS zF)w$Y9JOOY`9DhpV->SV(n_gDtA7FfpCzQdC)I+YHHs0^z5uRN3(5Xn%=P`3yg&Y2 zq7O$gouZ8rJN{zua&s}&mnHc5=8_)V%P|d)rL=wK@{I0EQF~70gm7JCPz@h^fM<=s|oK{o721XldLxOxvwrP)9){@a`9G0y@VD;#6R*?^}zS05i+tBwtKROp=?i%6bLb6-t+Qs2K3eYSJz z4R@XDYCuf2P7c6~!|3;ZS&t-;>9M2e?7;ik|C8Wo8#lfW;{ej)ny4!oTB=G6lUi^gc z+V$Uf*Z!nVdU$A+MbKcNdqyy0IKg=g=c;eL{0Y(_$6>gpd`ns6ybhP4#C3@B@FI+| zIC9bNp2c;7>*$IIo-S#c_8;lBF@F`IUSGo`q@x_HnMUqVAj75)l`As4_440SA>ZIv zIg_XK3%a{9sN4`xm(eIDN_$gbM5{u9`sdFX_$pDuidd%HcEt)ThV;>nNNy_(_GfUt{b3G zoma!`uSPYOO`KG}4XX^{x2j3%rBipcp27I%J6l&Ni2OkAj}%~X4Y{3-POBWTsK;Phx?XjJm2lvPH#g|A8L zr`ju6n)C?_mQ>%M{e2$Y2A_7nU87B`T;IX&L_Y?cUaDTdwZ!UU!} zY%@xYVhQ38tAwSx`7TC$G$g${l#B^1wHz^}xmqrtYE%B>`7TIw=V3EX``@ZRiCNa) z$=pZ@fx6&L+t55UKZfd>p3Sc7*Y<%SeHS+z+I;iZTY7f-MBDCqi6;X{O! zm$AR*McX$oBiq?8ju)dg4`FX-{N}J_pT$IGP+3nrDalLXy- zsL-m8i^8Rc`6c-Ypdmbo+2Ems`S*%ZM|hVUtIMD;g_9$Zt|YYCqf`m znK?OFh9ycZui1_!`Bl;#>0lA<`ZsQw-Ms;E6(=o+Cc_cJniPehGFG8kSfvHxnpVeB zCDt#A5>XCWKFvZzN_8tBOVnRvwwzhYYc@6WN){!Rnx8aHj?|hIRlMz?`?eU)Lg*E8 zMGF;eh!s{jWh}8Zq~{D;b{KR}r|#bX5cXP09#No{n-;k?t;%~9KEyq|+zB+9CIycP zh~%*bv#uzLO%G`C6NA3Y(^UO?CdE{Jn{=Dze8OKq_A#zk%Tk$p+xAFRmW7jfjfijN zAxygGX%RL7Kiaari&0nHzGKkOPvfXL)~9)NY@a4?+Tl9ZBdJ|#5KtRu}Fl{`TDel{stUcIxl*BodgX5U$}SU{(7`yy=nMvsF6nesQ^`-^qEApdXhb75F=U zE+pr4t8FX-eyOY%$w&L~uJnDi+xv|#z{Z1rs4pVVxAIZB%XjUA%_h*)XB#iO^UTlx z>2=*^<8sU2^5djlJ0xOfKfR~RjQ@RRWgz2cbjig*z2r73Nq9|!yw$(YLJaj+X(5n`wQAI2g?s%sv_2skxH?+YXgeT&-~b^h9#mTau7s;Z)vXjn8@ z=vb2A(9qb>G{d~qtm3+||F(>(&V`hZcFz1Ct{$0~yJT@ZUg+h!L-hK>J|c zM^F@1>0kXL|LYmSbXgm(b4=;u;~Qq!1?Gtr{bF*NHFYX9R;ygctPIKWSDFP1c`T9t zH&Jg_tkldb6SGrrtw?`_fiwj%sXONPu>Nsmj{wZ;-TJSY56l%@8@O9(a1yYsC#O0w zIIKqzQSI2sJ>mkAekWH{;+~s!=UP{e_|On^a?!sp1QL`qUfphdvZ0^PkhGA688!DL2~ue~NS zX3Tmu*d!0Y5{HAal&o$jpoui2|C$1K(^WZ9$g6gi5;Cd)$S6M?5%r#(PESV zCv`mlk9v;|Vua16EVSQ%ZkAjj`sXhld`mmad~7HtDl8qQoY?DUa%vNeV$gL5UdWRD z1peg6q(pqFoJ1T5qvG3G$@ioK1T*BST?o}m7VmuX$z&DmMkM217qNXc40#z%En18Q zFM)!(3_!5uqI*<%R;5AJ{tcZT7Eppsb{Qd6M@EgaSF5BWx*yG~Vf-h2NB&f~4E1!N zWd)yZo%O9>e06dfUxAZGg_$@A$AUNKn|6#rr;%NwS?eDLwmA11qeDPea!eWDG#hdm zNJc|#SDie?PUCm%A1vtXX&}P%H#0>A>|=$F^45QLQl<1QW<6Q8EqZA{@x*p=O|x4- z^n~?@L7B?VYNLT!rJB)1!+mRUtD6FyqGg&?OgddO^ZrdSwVjwG=;vvw+CujCUPCMd z1V#-6H6`aX$&O{A@M(Gt{c|k+=F}1_SqHZosUX`JDnl!Po;GC>)3FcPc4`{oPmO5u z9OmEd;g@pfcIu}ix0#xn#nh~( zy81LIE6Y)9YM-NWO6&tXta4RJgwDsLg{&%J>ql3>94Z%7JT+={ZLmxvub0`?xeJt* z88fYAZpKX154bwk&76hvHEJk;)s@#n=CNUlO^ORFPWqHOz1jy^gzc9s9GdOLD_)xA zl}{>(x`&`UNMTvZRV*kp6w4|b7#S*~U)HFhqxD;ykwQru1{FQGh7R(G_r`0QWSx_6<?&AIo>z5z=vB;~R+$+3Qf~5pL>*DZmd|bWY-jEcrRMWNOc->O89bbTdh`354CJWCU2+5cm%dvSFxJd zc49`CoY?JP&2qU+W>AK3;A5uL)N(#98!)cz*jDW?VaZO9&*W6QMAq=y)P6vFTV{05 z7+A+l#qg}O9G*|D(Ym}08qjO5x7gG$cW-LXR_VBM-l$8CYDK3&U)r}{S%VW9E_deg z@?N7C_UO~HagDB2zu%cHtHoxIJ21I;be;}B*c(KVzRt9HE?K61o;Pvn-kxY*4!gJL z;-1!bUdg!B4xWDx7W``g-MqGIX3e?&uIcsg+3M}#iOueJ`&zmEwe)S`+PWdztTwzI zSuydp-K<@_f6;GR=l#mENwHmgp4^NBy!NTX zjI9R${4qBB)v`KW(f?xPgDK@>yDVJ|9emL@kT)ltW-r+?4)PK9Tcgv|AQ9M0dJ^d%CGx>dI=j8s{+Lp~hYaDInQSbJ- zDG4ndyxqRtEj)c~{p|k!`gw3y>zc}1-}>qP_CtM`-ceP{)ayi&@JTrTIUv+_@Hc>F%stDkgW z@;h99|LY^jDKdnd4&UFY?95Ty9A3ZImG4V%I_)0s6QBxgwOXAHzn_I^M|8U#F4y~J|lEJIF>+TwNkK!Tn~{*}}2^}78)T_U3+;PJk@ndfr5 z+vN}Xdiq0LCYRgocXvj?j4_MX=?i>%UHBS{+wTsIhy`+HtJ@v?j&>l#KDRsI4h`7T z*&3I_;|7e5MC^8l&*%G!uyUDdr^_1*h-+9`GPRbV-vbJpklM|bpvNB!l5tD*R#zYZ@Rp^>brz4;`{M_<2|PZJ-yiz^x==M% zSJ)2}@rB7uHdojS=@%DeZFZl}3jrRF`%N}q!0YP@(h1{Uk2la40T`=mHI{%c_yYmY zD^2b|C;$ro;B(bBzdQ7U@W#^ZZhs)?Ul^^_+8kbAFu?CMQ`zj!Q1~z5hi96tE^i17 z=EvqK{U6>4Aj}J1ega=JSRkP^jQwFyssb{UK;X zt;p^3`Xk{me~|wo5{^V7a5*@U-yQVDK=UYUoyiyShXJuMM!ne;_WOZh5(=x$8;%45 z(YO@3%@_1XLP4??k2J zh=szo08yX+BNPCJY2u_df7llW0f}^^CciHnh%CT46=qK)5Qo4dxO8@3Fc62$q_PBY zPaqbJ4jlmFj#wZXL4e5{)mBd^8UqK>=B0xC&i4Mt+|6qq*DkPik!5oBZrIPdd>LLnqXf>G>< zM?;Zh$}UL|3xpHV@B$fH9E`>iVO019Q)UnQBS{F-3RQm>^e1A`m1~O9kPL^TsQ;w{ zVl)s=Mv~BpZjm<>{EZ}Mv1r0@Ad-y7pr}b&doU7?EL4FL zeqSJ-fXSqjuWSb)=)|L^j!xt@VbJv3O3|m?RFzQ?V3ZCifF7cVy$K6fXCHCG*CUi3~F3 z5g2jD6RAW-@s}aYg_5ZRHtCkA^Tw0$9A3$oE_38knRHU;gQoO^Bk2@+)s-mshT`#L zHt!xOip78LVAoujI#(>6NN07t7;NdB$p@ko zTrm=FBA-lQ55#kJe>#y*qq07shGI68&lXl+*y3n3old1w#p1zoB$3KxQGbAeN+O@h z5)MUkUSB+$$rN&JX$oIDo6i?jpBbV+E|<#UxBmd)d_0*e5;P|Len%pcE~L}NQjTaW zkMZ41Y2DP&`#g<8@@j?LaY^&Ed}E z%ocwlolobnztHk%ESJe-wtop~PokL3leXNT&R`~2C=_$O(9(D$pDPuLgfp&iDwECR z@m&zH)itX@Yh zQ_STEWDT}xB3sJmDWufya5Pse5paeR(MT*;DChSUyf$AdTgs6QWrV$8rd-V8cKtBg zU^qXsen_%VqL3@$i>91jXR=tz=k%A*-gqRJ&ryh# zk%nqImB|*#{$cJ=B9qS(4~4p>NVbqGQiwhIqIfo6E)~gs`C3=Bkj)bdf7s$^HeV&Zf)rqfr#%LnQ|1l9-z>IntvX>!r}Kg-$9{~)kn=PZ?wLV)`2 zZ#Xw79J8zU9aqj>RLM}K5z{knnzAh(<*7!o8A=Nv%*w-TZ>wvpc^xbvQwpeNiz(;; z0TKHd`h8h6tHm12am$zRQc z!JYjgME@Gm0jKQa{&jg9MG4Z=-zZpq9eX=}!T%krr2}>C+msKmh2?Nf*@K^t@Y-kW z3oz?_14+h&Cy;R=2ocg^iN;-o8A5=PGr7$=Fy870(GEe{r+(9>E`OBSI~v5_6#Dyl zB?ZD|_|~<_>SUe~Y|)#9S*w%t1x?inLcpI+hi8Cg5BT%aXHJ{t&I?My9X~a81#eH2 zC~Jl{uvSdWhY0;|creXk1*lS@cK#|K{kb$`Zi05Zp{8e87ZaPHQ}`+vsP|RWl7X^~ z-j^PO^2cyr2N7`GKPDJB0Du@4|4-fuxCI0!piF`xMt$yEc;#i`>dch2U9~EiR#=y=H`ujEkh2a}F_iqAF~;YD9hsXF zGW(!ZE|Rw}SwQr%Y6p-A6`%7SYJ2p&9{~?~^=P|##F_?P&ZZ`!P)mWZnn)$}2Aq;B zQ8CMlcZMDa^k`K~az(nu&oL$rR|f;a$OJB0e32lMMwEw^5G^vOxd;YBD6Va5pSv7n zXf+ko0wjINS1=YoL_~OsebDO|zmzH`_GyR<$V`N|&Nn%6_ zYK#tC`itbqi$OYz1a4s~+}@{Y3+nu_kZY>uW%8C~Y9ZAPv49PyAttl30GOkLE~XvrugoNNVCZ}o%g6)c;()f2 zFei$R^+AnN)vH(4gj5tgA5M57 z*IhKE6CA2Lfu)ZLNRra+>w-;~ASPE|w>L@}oEN&>IkYJoN}g1tofDo23*>e5G(bti ztjX=~%i^`E;6%ism9o^Ji=mnLvzRC?^i>kmx<-#i0@kFyNkVplt-)8Br+}`~#lL>%37jHC^D{#c@wPBXh5GIax8H?o$t**tP;!a+)^p5WKj1!l1-+75rk z3;_3-Tz$UYSgT6iXEMJpzmF&N)VkAo+TNtAsx@g@`4;nq1=8va!Ylz~Ju_hc3CWST z-Cq0VWfjtO1{mVOn0J1cX8<6qUDjcM(FVt{L&zA=;q7T|L*{o1=~7~4pLX$Oe{HCm zxZYzRJb)^xJ!0zK{7I-mF(33edjED^T}WDmGaEnF?e#A2^5`?if95dv(DE6Agnb#8 z_0?#^b^S?a2l+hOc!@!cuG7Y&wt|hzw0OkRfF^8dTo1|=8N9M zCR+rrbRpB*U3@qJvw7n@zb$>#8Mk%Ahfz;&DTZZhyH-#3aIBBW)b*cA$)P$YLIt16 zHV${M74_lp*`j`i9HsQx^`wtEp4N49+H}R8+BoarR=9v4c>dE?oAe8a<0QKU^X*wQ zaGf_OI6fJdj2CYIW_y$g=DVHUg^Y}v!YcL13o*C_%LcCy%gLh|XX}GkeZO_sEC?-$ zT?G%F&?@!FDC*1Q@T5{_WG&+k{3Y-*MfTfJu<`g->YR|6KFFdMN3Mp{GbbBd}9aHv0K>d!9Ow?-!)vy0zR|tL3b{J--hVn#?3%U{b|nR z=V0Y~o@`~j$o5lbI-EDW&sP|uaOpMBIC3LmOtH(>tu5E$h8JxmaCZuxDI;{C(lf&b zzYB;PGxWE{g!hvs>EK2Xjh5Fuyj!p4(fhTvfrL4q+>O-t{v3ENSV+a_Z9sTB+-P)S zL^9*mB8oxi_04~p+ zrQT`#E9qXj@o;jX?#lFf56sbBT-FUk3OB$l;yqviF$lBN2RmxlX;j*O$`Z zxe^EB`nQDg>)^Cu%{2WZ3QfMh{X}XP^8;PU<$GNDBmRrY3<6p=pN}=-qT|dd7S&j&pnuNd(m8# zftRuid2v#1wFgA^4eg=`LI{3P1blnJfUd0Fj2tCZ0W?$2uTM;PqeRE_ESKN4Ju$JF zytJ5uh%@r`@F`8{<4i{!Xo|gxgHV>K)P6#Uzi4c#7;4YTRwsXW(JPe0$mYI+8jpjw z`Gf3V0Yr1;Chm<4Ejf@aS^_gr{tZe?Xep(8pTEXxg=3NxCyPQiV7*}^*4X|Oj89Ni zigPU}$}F3;cJhlzgTtUn0pc1iNYcnr7c=QsV;wt`n!*eaTQMqpC!{3oY$xhm@ z6T+q$7Z^Mn?cDPYCSeRGlH)no1ZtGVKM0XEfy8RCqKXaw%DqlBwdZ9DN=k#k3NHz( zDEHf6EM`p{YGnLc-2e--IT{Vhs?A_%n$9|FE$RWyCL%fxxh5&OLZWi_$eQepIO9d8 zWfy8aE{GsR@x+aZS-4W12C96Dr{JyTC+{PZ#WiD+AW#Hz9cx^0#^P#7PJ;5Ybg?e)fYUjFjr za!wik7mUu(qFzy>-eliagW{k@{15IFRmEIN@f-^c1!~GK5L$#|d&;aoNW2YFMLJ6u zxts}-RHi_r(||FMaNE;E<~5pjX9$01h$5thZT?`(?Jxvf9tW_P9x4ou2h^j(PmTXy zw=)IQj^nS_d&;Z(&Z~FM>u9Ubc8BxtR=ToYTYN4xvNrY~;>{K9aK` zNi`BNq6N=CIoUR({OU8Z)&1c*56-=IRS312omMN@BA3)8EW5Tw9)0T+IZZc4>`;9U3FX(Ri!%gom^@ zuZ$B2s{2X0)KQO^d1&B7jo8p?fCfH^TmkO`mGCtK;m{7pvR{DjTP}5yV4}fmdL&7mR`}+;Cuk6!E;f zJ!?0E)Kt9^1N{|z>+*|@K#X~tXGJ5xn&@{vFEg?~BB~@o6yFsOk(_Twra|Vb6>EfWN+tEC)I;j1D8N(il#Xl2&Y2S0*` z#!NGyMn3ra!KB*m(1a20=e-OA$1XokARYn_Mt`0gs!sfF_U9nwRhgZk?v&o23YP?t z{*HkHatuG)$)T^`(5>ZEK}9$QtpI@8EenoZv|^D~-IoR#;P+=7a<7JAJ7GsKi5Q#O zC7~#P0Dv@HiJx}(QDO`<`rE>l%ClVx2B+NCPT-HSA93`xVO#%?HFXU$wUk)WAoyb1 zK|1r93O5@lhn}1nnYdUic$-`q+QEdFv$<6GwNrx8Y!$f}tCmRLQQ&BN*g<>To_xp* za@*b7iXSviIqAa@U6oU-l|wFWU?I4A8skv0;X<#pIY5Lm!}deogp5M8mN4NQd+JR{ zXt5L$w?H7_-4F>G>7Wu|jRDF~J#sPxI@~vue#D(|SY=v;vuIM`Ge|497d56oX(d7e zr9;+Dh#cw#SZOhYDhD-eTmH~67!tR+AmH%jBLZn89!%Vx{TCdTtB}?kDmn?ppZ=;|DrU9!>7biHSXhGbRex~Et&zvp-TIn)YNk$<16+9|9W7U9E+lse%NSj z7`MLiMj5;vd4Ry(J+HaSM<+&RI6|A_BbKDf9F7sfaxq&*hUYcZQt+(W=ES9t^GMh> z($#RAvuf1i@2g?3N!l57jIk-Vhb z8P(DvsaP9GVeX7IV;~V8BBOZaNc6iyGPVG7MPQHy8U$NsxzS6Z>f@2I{QH+&02yM#=EG+wOu-+ZE*LDFPfJ| z!L`x1YTQdO#M15!F|%cOioL5%^zdvsykGWWDZ^ee`%k|((r)SY4y|sdmC7`Za(4Q7 z>9yfX!E7<*#9)x&%HhsC$W_&Es-|iO=Gz@U(sXO)>D^I<`}SbWDb7HB%5D%cMIbz1 zn>hrlL)ttPt;JZT%w+;jm}h4 zjKMpo5OK~zbDF9sm0S7Iz0q9N)n=355N)eI@fTle*}Gf~O^p?gvN)qx#dSy;TZh2w&|o;e?fmgC zy^Y2s#p`*a3-xD|h^2Us`!^hSks0G(mLe)nkZG!9CNJ*_pqC6sLE(yaJ*4}K2p{t3 z%SKhKu9Gg7t>CIFpWJpm(0fd_1GViBA^P1fV^iqmgq#wxDUrys*}vptkD01)t#_HK z@J++i>y|JT?vGpry1g8$`%FeO4CO1)ju68Qm}K-;WssxP!I>9G9nIdjypOG=7Y8E& zS8%$IAwu6uj`ugl{`F^bI=SwXFK|8f%^l1(q%|amU;~5_<}*8qr%W}!`%${R&mcH# zjjc6#I*vExc2(}%v+h8KGv{y>_RTL*yHQ+pbw^(+WM;G+7h!CvCadpOmEf^+QJOv@ zOy27$sX9ZZS*L8JlzgW>q0{THiAvSv)VTi$7R%qswydHCxFy&EN+39*{^2 zrsC_Y60VC`Y+J62)x}%Ta+~#3H4EDF>VqwI|5Of!K7^)xnCSURc0Xg|Nj2LxCg-%E zWPD!Y23^MlePerv?#l3B$*f6QUYNaEM()0B&mRjruE&a9;O<^iZtGLL74z)_*dbbr zocsiNPu>u@(fm3L(O`?{_p->UXxy%KVV9_fec~>i+*;#8ey)6*B2Dk&Qu1Bru?#dl z4qq%Rwj_+eZM@$$-qU^j0(h#Z?Uq9Shur<4K`o&|ZMoiA{pXYz`yb>ks)D2();ZSn zbhSjd@~Y;cG|bcyXT5v4xKsfoa0Z6q5RLLZrMh_o3L>&XF1YMK_06_p8in1+Q` zT}{-~nt^KK%>qk)qvn{HPNNGb-xwoq(`dm9l@T1rVbo}8ic#c_OYK3Y7|@s&pW^Sc z%~jaG3(q%k(4iuCphnL+a|2iT=ycq54lb*87rzy7@^cK zM+yuw*vggpVL*8xbd%F6tCABhBhD|2SRu596_n9Bs>nnAAuk5OK2e$2R3q@ESw6`1 zI_lLD7?jh5mtZx)si&6T;GVumJm$Qjmnija!rVt4kXeD5R>$9B%5$iJQZ&4onZAL| zYZz7OhW#}0>~w| zJOLMxINHW#&uCt*wLnE{DGxI5{)C*>c=uNVYd5?qwfa<=>Z=MQUv%(n7lu>?1%ZM_ zj{#!suZQ4Q124$?E~SZI(CMr&9%xnlOo#CF<*}@WTu5Fi5uwqm=AfXgb_FJplGG-x zXd znk2Ks0X(T83%Wc~yBEAyxeM25cgTv@(2@nAG($xeK)uysWU^6{RTS_N@{O`*#)Ajn zFHIGyZu+erc8*|u@F(-Fh&X7ex|s1LWmbgLAy-I|bCYsGeR8=`c*X}Dq2ek*+InqA zEvzywWuY5k!$Z$RV+cD`VDr0IL}E0$R-VM+R+Iaxt6y9p*1r(J!*X$|X}1!k5ux8i z!o0g1aaMyQ88YSYzJlzS2u|31mOv;ppz=-44M;pGK^8!P^s89)~ zAw1^rPf>NqsDsOcGqrG=vQh`949K4(T|`9=tjq-UQZvOq2No41j?$wo9SHN2TjOf@ zo~XvHXx#(K(?SLE%Qi*Isdg+rvBsAg2P|m!I`1*3pD$ycBP&iSrFKntIH4KOW5E>d z0aIbCfkXS^>dvGT#~x2NJrj;h_y~P5Y+t(58_v0ttpZ0dl8-X~{uhW=MG4hem-t{8 zhamma#c!dKPklC;D3`x^aguK=8ICQAOM;NBr?OeAzIo@Xgs_{QmU&s9J|y`LVU~+ho{!?pNAXGG%V&d$ zW;4smlqR0S#@s2FHPquyt;tMEFP_o^S*Jelh?DNMf};Z;lxCDlT8Tyl(Z>slxZ_!s z>LrnF9#}68IX}RTYkz1?%d=eHxRZsHDizyvB+O9wxtb$Ci_fzj<+IhUeSpr^ZvL5d zySl{&Cb_`a<%YJWLLXv6i42kt#nlNwUZM4@GcaluFz(0!+SM{`| zuFq)0?=B1pyL=bU2x=cmP-rxZk8n9`W73)}r5(4qvO2`}0 zU+5CMjy&dhfF!&^g_pCx5OKFqt(^tvC=LU3;(?Uwn7f;ay zh7vd^mn|bLGL+k?#4ax4yG00C1CSE#$~KF7DnlsngB|ch)Ll(umxYNYS?~E*-z`f% z>Vz0uf=kG=P~x;x$h`27Hjt**>G_G2SL@3#em^NzjA)xnC~oi#EZontgiGZHhFHcA zlUZi;Qv3Ju<}}jN%79I2M>6JH;!KFUckeha;mlEWj?M%l<1$x27 zo}L_}o3b!1E13rQecj)3{X1H5KWiKbeGIMh-vka*grC8 zCd-NE+`NE~Xm)mq{r$SOSWrEy$n^ofOHN6z(KYgZwJEWL{<;i!7;m^cz>vzKRrj&g z|Eg<*bYXFJCq>HkQt>hV9z2T%3~Z((4Am}K+{Ig)xyrnP0)2k~#XjnP`OQsrZ$+8k z$tR{EV{8*J`_HrN*XZ_&nt4ItAAj0n&kbC?p-Wua(%1X_`XC&7y*VBxL1PlxKc6y> z@`)l9-NGiv`QeG*%>1Op2RIgvir)?5t69AcY7+N9f8Sz*%zQV?Qu(+!6KdZ{*Wl2A z=va}o4Q&zt&S0;S7yeSyXy0A--no3=dUXW58~W8>E>_{%^LD8xB{1uj|h=CCE%K zrEM74PD+Z!sZJFni=%I{_0^0l4vzizEz|k-d+$qPMtixTsPo>``8(hC(R07=f5C18 z9qJV&>h)%C5fBdQ*8d=nVHG8%HqX#*jwSxGAVH)3|55}Uj;vP4ZxQ!z$snlL^IuXj z@OS#tB6O>K+qBqm+f-bAUPM_8FNGKLSoi(ga>xxa+$1+|jsa7w$SG2>so!5f40jy> zy3mHyq4V=G`#$r^ewsTPgZueV>r1dYL;3n+-H6T=idEgn*5rBo zcCDQin?qG|zaO4Gv3dr2S(N*^&V3BOtKD%5pS$S*b>;lcKAW?6b1hi~7H8S4AEuZcT`A+~Dx zGZtLf`~Eg8TH6=gatj%c9o8Qjh&!coG_WtI`P8M1X?QL(h4|YH9yLrJv~hTY_rr|V z+YsGKLbv3((H03lI*H@#2OSQ5a?j}bYi3O+=z8`;ybKX^=6x8%=TI1GFKpxW&79IT zC*E2Q@A8XhNVjrkI4>#2^m9g6Mry~_D)VO8i<8}VnEIL4J>xxA|wVa_@n`^ARa_N5Z`6(O-V+36=8U9xl8vi zZ)GsO<*g8JqJO8(=V26oIm592lU)0RcI6^wDVny^x+B=dj?EL+zni7#F?_W9M2Gtf zFb0FA?EY$6KsLQtlYxPCUJt5#a~coa6P5{QJQRa;#1@92K>GmIpw+7^u4BDC)et^Z zp3iePG-AATP%YitiT>~}_w$?&EAv3o`dry4ekUu-71by{at@ShcRCW%pCJ?j)eJzq z-1C9EC+z{+u7%>w@l~~-r)SZktPGCA@cVE;Z7FUpOZEdch>>&%=LJCv6Ns?#&hD2{ zsaTRCQwwwM@3wVK8PICbKko=ESzc$5onv z_f4Qwr8mxMa6fnj9BX7@v<>hYu-v@!77+KLcpI3pStK~&WZh~U0&Mg%Pt|~hSzaq) z?27Oo+=RZaNeJ)fXpK|5_<%n{+i-Fxf;=fi?=a01E_7wYxv=Ytg1jiFWSU+;uX*b! zcXo2FB_&*te6F@*r+;VR2u!{JO(0spA#yydPLX4?>JL3OAcvlun4pmM(W3!$4l6y7J$1VqVuFMW5gI1E?%#St`M#RLu@GYqSXREJY1>>W>ctEFb1pE z#*Qp(7mt5;@RcZ!9}mRb3=sre--&Kh+BGRe9;uu5I zm>2|dDp7|q5WNTjy`&|n6u7*U9A<7LUDiDShhqrPdAs!_B>&vd`w|K;@fbiK-{c>; zDNsy;SsO+*4K)Tpx*+ngQ=ty>X;fH(Fl4Yl!I8X3Lw>0U4x1rk=n~{f56l+_gQpb# zg8JR)LBX5}Ok5G9!mTVy@*Fq;PuT~U1Hp?I%M4~`2J29vkIZKB8M?qP%f%n|S10N#uY@imPj420_ zQvDuFtXbEFD(Gk3Divm4B)ocz(h;uAl*Nm!FB?cu0=<^}kU9%HgQn072Y~b{=-V&t zAs;*7kt56#n^YHHYz6Aek6T$%y`-}U{;E-9E=Rp>D<(fi!6*8MWH?x$|0`;h3!V5E zfLnZ*X@W|-@&%yq$>8(Wg0BB|nLD3g&nF#ir*3$uGiyzpX1^y{Tt|G1WDSa;^tSS$wmB zr2KhX4zx1^n)3Nuxpl2d; znoHg|MYl0_%E$7KKHSrbTOGwklnR zLUZHKU+4Dz;}q3Y804{MKEIBQHTdH$gLurgAi;LOs?H?P$ljm+&Fa03!wC@ zU;!3IMy&`Re%0kE#5j~{V6cZtav1Vr?ah~V5_pR4YpZ|-38J8O^yWy`@euJc8|J6M zE78!&Y~4nBR~g%LGN#8A(s5hMj{GjIhj@pVF5agb)7_?(W-4qnBsod%M8NmfompjN zp$~EQdFH&2P;{`R)`;nZi_HvD_3vuS`DAULy6|7mVngrVJnRx{pV2B!_ifJh zTwmin2OGRhCsQ2_v%EU}K_fJ5zQhR&s`D9I+;duI#OvMJp3P@4C|hu{?cix4&xHy9 zTg{W%9p~1M(fpg7&4wz!hZ)t^>O1?rrlOv6%LM4jwR2m?S}?v&$IWUZy_;{>A#Jti zEVv_9?_A(_5`2nE?7>T;o8x#>E%!yX%L(77_O15ijah{szQ6&{tYJVXX%8G{8BFZ*8#y% z=*sCMpPNBB?L0NCYa7*Q+6JCl*DnYoRgL|hEi8!}^xb^EE9F1DlEeS}$8nqN4Hb@) z-O@Mj&p8$Zy(?lZ=sU=X*TK?GE?z&bvsGN5nue{NZ~6#6R9_<0U?;k(gZJ&(A0n?^ zGSeNWz09ki54YIp$M3Pti4H%Kx99x%%u2~8trp_1U#0mKT%SG7ni;=`Qon9D*MEDt ze~VW~J_7_?@2wv@I`7)?ZYpnUUBDmkFU2`}H&l zTvBQFUmHt{y9Bhc=)BBrEa1LFyyg8cZ>xGRYXydTT5!J8-OYy4;2JW zBuzv^ky;N32!KRG>Ch7lhJv!@Oe~tVHgU)#PAp;)TczCICT*$dsAe74Qn1!myv95o z?c?&AedJBGt)Qf&1Ojid!uNQQR^d!G2m088MyE(Wov&}uk5DirD`gse0^QBM2tQD| z$SR)A=P;?`EodOH|?g-h55tx0|7bj07Z(9K;Q>z<7P^8Vqo2~5UaYz7n&X|2!~TKUZg`pztvDB!lsP zzeG#-%~zGXdlReJ^K?l%MXJewOHXgVXQmj`FkLt#a_6{3=?f!<^8XNBpdJauQ|WCS!a< z7pjc>o{A&n=wK{jmhNsXzqqH)o|6UE!fldu_u`dA`0M@_QpwO^lQ=lt_-Cj4d}V9j zczUV;%XVWSfAwI_PI|?TQajvUYHE!9ZpusHrG2k6a&)^X(LM5fLi%;nd6bLqF5W&z zxgIAp)1~zm+zL5%bF$xkM79p=NX1OYs_pwB;Bt2#El$pn@dbMKn5KfL{ZRJhiC{3v z^rDI0ZlS_5eRqNf)svk9kluMQLxZObpj=Jr6nu28Ya)$sxvH$9XQ2W?_jK|6fv(X) zf$PPaoD3Mw1^IMbWl7vg&aJNVMa=OFto3O?p9kH?d1+838$_QqCiAwU8`EpQcR)Oh zdMR*?uS479XEI1z_}bI@gc>s~bwUf1blcng`D2%IcL<7i(~0v# z)V@w8aQnFso^Le*L0Rj>+#k8;1CH2>l+On8+RJr5u6o!F5zd$V*5Z3Z{Qx|14zpQ2n@8vSHrxTp02V>tO<;&8?<=p#h9JA zgEtg3kUYdPY}-pDuS;cXl+tt8u@uzR?ceL`n8WM2n)IC~L?p~@9>@nOXUmU-C&$pv zO-%#xjW&zRlY_vf%_Bh8>dZD3^i4#L7<;hh!l9l$-|vf{i`26{3F7|MTIk<*B_|dB zK{EQ+h1?CP>tn=rkSpoYKae#OWV1EXxZf*w6|py>wt_(!8}RpTuqzqT_YHz_pcq`m z{fEq%;WEi$jN z(HJ>}4D_UU9&$f3&dqI-3KmIS|H-b7IW<}AjU^(w>lcE2o?BmaPtp(t`yiE>w8~5M zAj0P3;b9Ib-*9f0BofD)D0j5^i}%2LX$zf|n*S^i)2=n6lrQaxlwz-1J{ zD^0o>p|Y|z#K&s}t^k`Dam3wcr(hrzd2oQ3t;HImv<_aRyrk7{b)a{M4w}At>kd|dQL1F1elnCY2 zM|vH35u$kuskv5_30k#h@16X37EOW)5IzNQVM*@^j zyCulr-Dw$u`3#~qsvgSLdf{h7mHz&x?mjL(_la4j)cEsR_(xshB^6?<5#F;YKNy6LV0<ZYF59+k^R4Q#x@_CFZChQo zZEO0Uf7V=mH+hwLcUG?0CnBC*N}~k2+UDsJ+nt$RQCe`i zb_pX^N_T0VcGE<$LyAns)v3c8U|dm-bEg_{JCq=r7T8M`RNfia@`Y-iu*j-w z#}1Tg3AlPOtiw)((3DL3vfL*I0Cm?cqnW3@6Wz7ywsXb(rX1DpRzpZE5eEP#SL$~f7P1s zYm_y2b5Q9(k0P3eH^?BK#v4o=5+N0N^q*2IP-#vnj=Z4&8wAe!2Z9>e*3;sODPXD8 zbXx(3Q_7btluFhgoVfdPKA3v$dIhCjmi+?=4`+o~pbd|nJn0DZ!1s6Ha$b6Mtf_JF zM7iX35AmVWjG4$^XSMP&AE@3>9By7WeGs|Uy$7cXeypMoXFn~cw!h!|>Xzi(HF=h? z>qmBTtiHvTa%};UnHD$%FrgdGLDxe$iyQ&k3b4H~J%$uI448RMrjq9~O*@6ls);@# z$*bOj`8nKWE@f0_*rh!5+>QCtHwy?}<~@Am^u|$55t?*0u zSg$}AP}+aAYXktdEc(*yA1)P#D->4Ljf7{9WIeu?ie~Nzyi@52vYP@fb1|K$4ZxWZ zN$|L;XDK$uvr{!;3*V19Vc-h;l^^m?Q#DZbGm;_9Pj;L6^37VXU#6_jnKF2~qR1CYB|NP-8%i{UE0WRg<@@k(H!EcN89*fZc=dAv- z`(-+udh#>*jv_bwzY=+$0X>cm{cU?^3JwoF@4qB+f}A3%70#$dB&ZrL3&AhXz9^Lg zNLJyfl0_+15UQ9M;V76iWJ@JvC0SfKN-MlSPNwSa5RLy{^rzDq4geLc9K+O^bKV6? zaOF9ZUwAtgogxm4D+v9q6vAhFb5vgQ0?7IK@bmkseLb(N{dReE@ZR8%TUv>q7`OVVrIq>`Lw+nxI=xJfD*0 z$XZ{|c0Svr`hS6PI_izGUp~%;Fnp<0Y9E$*SN?PtxF&CT9DaqaKH$$sSNXj7&iBBW zTL6E?mBMK8b!p|k{4MQvH_)TVA3REbZ)c~D2kL58sosy3C=Cl>A6EYQtL@XW(c|@d z3ifvrN~2EBAoWek3uM8Jd{&zE770RcF8as$j^{%UsRKp;`=0X4-%*P5){zk(4D>WiL3YQDF)#y85qwOcOBBdXZ&2v zOeQyW)5ku^2^sZ(30P~n?fPm z-uWPt2LraWFVX~wlr#9jkNk6A!vtL3K)JA4?lv&No*whp8m5%@V4CS7l48G^6_!dn z;{EGDoWZ6dU~najTRV5r@tfAiUDA!hH4@HS4lGWiyEJeVRzcx2pWF)qMm=7K));vj zdLUX3Mm^Jl+S-=1iNv&G^k@~R?`S==$-h-54P1Lz*BAl9BaM=`jo!>;M3w$zhho-$ zkrk6F?+>M7SN3dp;E7+k2?%d#Dw6$wMqinPb9Sm4RWK(T!FzQnl8^+&6A`HZ!q5my z4d5WFCgX|losr-LVk76E71p?dE!QS6J&EiX!CWfvXVhjT1kk$HEvMrpFeQRzo*Y2E z+IF;HMf6z0-fB2Xs7(HQ=*v%EF#zP zX)@xPB+kcx=S$i?KXr3jMfh@cP`7`?vh2Nn@SkE^PD#B9FWB!zUR3aka*&kZb$MDi zA@1183AG&d8=Dq%eoG~fg^J%HE5JD}95m>}sH&*EY14()9RqUB$(Sf`eQ_`XL3v{=?# z<)DKa3bc{=9Pv`Z**47FW0T3k7OznejQI?c!Na%9995F;L>Llz(0wr9wA%bw*H!gc z19H(IOpet!i;xUg&4Iq?h^)c2Eqz3Q7)Wm6X*~`~fL4-AaIKYHR&P*RkZnx}xsb~X zf8_ShqJHt9ABG}U*+p|)z`1R3wrT4G7{q#qtye{N=uMIRm~28`L!{F?VE^HYu_R@HUlcnJkvLfgDWXaD$KM=-hdXEUDk))#v6zp=%LLdb$h}b8E}qcguf&fFN|}= zW+GOBsDx8sEeaIsOE8N)7Sk$A?Ai+H5qjfVA1c|2!604}eFSsJhin36;4nm{`hulH zv`nfPey(^VcwUl{9K2yx6|Bx-BH!T##~7++mty>wvJSG~LuG)|Dbwm|pU-)C=7z++C+N2Al!*u=M;zA&1rDUnEcn!kId;_j%PA(qO?* z&mFYuq7PLptZs)E>A@B8_C8%5|Gqdcp}mNwD3_z{qH45@tl;SSZzX#Kf{A3~Tge(U zrCBIFr7IAfsBPSRkT1&x5OMpS){fJh@hr9~g24zQcH_j*X`e|B(l66=gNPk%b*w>a z#6#J~E(__sYwASS-+p+kL-zRt%Bqq3Gp7e3y39+<g$BC#46U{ zsKy#eOe2?hFIMxYEi9CdGyRBIaic(Rlk9G3t;?2_$jdLY5Lz!Aa*ZN~Oz_PSmKVr2 zI7!l5ok+I%>ftQtY1~ZW0-qqs4EFRPHwfvQdjV4U zw0BrWk?}PvZSnK+2gRbZ4N^E4nKRt!l(bQ7&V)*msIixx;T)c6#{0xDQt98EX>5Oc zqcBZC?f+r{E{`V*GK%rAa7F8tb!W51+*m=_LZfbF;F`!p&nQLW*q8Pw9d!TuH~)O zxk`K661nXOZS|Wd96iIOPpXc&NrCb_Yqlsv$@T}w_{zQ982RgaI;yZ zDPx&{9$-!UD#YdtQdIXi-);QXCGK?O{dWP&eN`%3oCM$rE6Gtf-u!oN>HQYklCzr8^ouq{02D+XP8 zz9!nk^7wPI47Lrq4Jn_{peLDhYF34pF4|hen}&tiS=z(}G-;+{E|k^@)OR>PerQJ8 zqbLs8No7WwTsulx*eaVZK+3}lNl2&47{#GHfBEm=5jQqALO-d!hQk})k}Eg1!8~Zw zos|u05p5uW$)qZQ|h&QBtHflZpjF``4zf!^{qt4poCZjUG^kgceAFMvVPj*@H^EC`I zzjoiR?v@HHsLS_IaOe7Zz3&9&8YoqLJluGHC)DaFN6}fWA5IWL3glfVJfyrWEoLB| zQVzzb<=|D^Q4pHM&t8@CdxFz%$#H>3hyd%fMv2@mPd@kZi2cUi?G#YwcUF1yx)6z&cE}P{XqXM z@;{M?BY~gy*0nL$2PT!L-mX;z5j?%kv0=}!)G)5~wBmPA#7;7D4&xLRIs$X-&yw(; zRXXd0R5H|_H1#p-;o6+cJSKWyiNq!Nm(lu3TmRg5tF!}!??8bC3ZhR)J7JHm*~n=Q zLc5__7rCBb{o_?hUDLvhSAdt|IY?rEqyhJEo-t?JsptJVQ=VLQYOqgv2r?`>uK<)a z3MU#^azpeOpnQ^&S?>OrTb=gnU>DMEeWc~pSO6AyXAcB(j$nPp_#hot1xSFO_L38^ zHfH?oOh!6MIvZ2FDV%mxi3T7PZdYWN0PXiV@EFD6lRds)8cm~`*+qV4lZZ(nS3UK{ z960d_R?smB;0Vjj;FX!&K)2U7w$D16)y2!6ADLy9>v$Yq^wIysbZbkluTa$ApC}8L z-c+6Vy0cNi@H;&4l}YU`QBec zC*{^T;j4b0EaK@cIM3|Qa+Lu2?XR$=ad-!9trVxs7^^;C{|S<29LM^6ys?$V?PU!; z-D-b2^hzk6+*D)PeXKq3v9sS}+nd}>oIXvE`HM}6PkAn* zr{Vl6b%MtCc~15A56q z;ZN>tVOpHo1cs(@Xc08hKYupOm7WnX@^g2~>Q|WFpwLeLP>E6>@R}i$oK;w4up{c9 z)l(Y#8qajc*=q1VTd2f|O8v51yPDl#NANxN0MApLd_o73*Mqch)xr(Fd+;dL z`u>a{+mv2A*>rqHv`I72Oi#XW;hcH0>TPUN3xj<)s;STl{QO58diod$=5OvoI<}~? zR?RJWIt@fe&@<}aC~bJL`rGB!4}z&~MY3@uIylZW3-{-l>~8h>Mj zQ(yXri$A~11-)Lc%iq1(S_#d-{d2r%x7?V06grnn9%T zcbA4;ptHyG$5o(1P9Y_plB(n3b1*pfO-p&A&LdA=*T(V*s(mHy2qq`6(eqi0SR7~V z{GtmjgGZ_3ipXua7-nG*BeAoP5k15yY>ym7mV~V+Z>y46NHDGF%29d8(;NFx_2?eb zJVVWuLZ7H(I`GkNc{B$8rba_NG1C{5I;FjN7=JwQ zRC=axFD;1A;RYRrDk8jKxc2uzkS^KI7kIpZ!e)3_=SME=zlo=HXxz>kz@X5SNN{5*oG>gJprSCs0IjoAFGGze;*7SL7?y<2#f7- zlzOAV01qU1EPaNqbrz;Z^xYl>PCSC^r5lv(by_ey@B_Uu{lLUDGu|=&7tA@BArWjOwwnsuDScM`T zegcM3>-q`)I!TqRQ55qQg7X2n9;huaqa#!@b9xq%ZaX zvG2wKr}jfKsAv{x`e%~go=y?q!-7G@pE!iHAYhJ>qCbq)kHTXZ9@&Y~Qs01Tf;};N z$0TYGAI39`2`Z7-#tqcbyhVXTMf}KnKj_p_-@$w8QK4gUY|rgAvRK|dBqs@Ioj8kU z3@Z{x0Vh?Y7cEfa9>RB>+^{V#!wC-o9WoF`A#ryiQ3Nat#jjJESLp;P41)|^NR^ji zAWH`gg~gC7!bdj-72{Jts|g_dTkIkT1tAlkSQH0G)ko0oVLn&wAdS{7?YLl7c7+gB zFj5jl%>W7MUslh#U&K99wf}yRA6NEmtxH@JA}=9&R4t{b>I+G8s-5ZVbhIIWH$NUw z_PNKUp}JKNPGdmcG;vZ=sIf(e12%S*NTbC4iz!^jKs>DwuVCh)!X!=l4Gj=Sk;Fti)eSGdLpzKZ`pGEy!oh3RMbk4Ty#!}Gf z4IYa*MIdJn!scd4*2&E~ZH+c0IY{u}i|y*19_?IxJCk&OqJ)X8=5rVU+nP{5acey$ zZliKoeomjCRmkM2P<7eHHzjW5m>_co;J|`WWqht#doQ&QtzPG(R@>tujb3#f6*n?L zEQ&)zcnr}$!5OZ#DH+8Q;JhjuL7Q7zw14~!ZIyS=DH)OB(5IRA=@VsDaXsZC7wuqC z6*{t_LTvI7$wp>BVgb4T&akppOwnsKCdkK+eXblPJ-4*Uj_4Jglum8$+ZeciJgUMr zmcMM9*E3nPzy91bup~QO;l=)Ly<90yY|;9zCk%T{EyesK2l9MQNJf)K(2C&o#8<4P zE4RKXNK4lgac)qgu5@6?K~Z6m6ybZ&hy$h_&m9ualXmc2R;Nv4Pd=hn@?OJAVm+(| zqe4#{GiXav6KFYlyNuJFHV&D8+nFa^8zPg#W!lUY>1mn>V ztvXTzgrh{gvGm!5hs;?Z-O`rQ96#cv-w&tUNo&}ftsK~Iz7J<*D`cciUz`A7Vyt?Yo}%mzenbA9Xi;Wp7m#kB;e&FRounKmlZEVBi@csADRFnA> z`Pu{|c*tYp-YgFv7dpL9J|7!9+qjX!`6V**s#}BFW|u7U?Ouj?xctSNXp)WL~*B2y*wUt{Bgw%yIv$#2%HuDUs z5U8!;HJUe)v#}D~^EQ-X7rD)kbYCGsbW@RkWl6Ws3(FRKba&nKdUUzu1XTSYK zaoE~KsDT!@97vcj=>z&{8ifH_2qBXKlRj*0mcaa6? zi+|kJYGkMXh9|zT817J!Cdw5w&#pI15>sCHoJM?e6~p@#%F3hY*%6De(u2f-YLI^yrD)ubXggZ z>$#p+m^iEw3;sQF39!bK-e$lg_IT^gkLpw2H{sz1T@(Dd_rAT_%m%``0TJ0uR-P*G)aD> zJlXno<1wyuU$*h#eA4x}0M}hJ&%9Sdx5XQ%*`sfw%K=O;=RIB8EupK}rOG3Ry26SCQ)!z8n1@oM@2V;EqTh?`WJb>U3n=+;CX#KY7l#>7w6)c6!DRC{*8ZTma zV)(+ngdxR5M2xOxNUwz98g>MAMQ>~oI2++1l%>Z}Icr<<%IChC1=;|!5F3QPMkJl2 z$;?;cQK2K5iexfGd;cP(Gt3y9A2$+ITNRy?Z-rX^Gr2a}7d1)=LhsBmlH*&KsyXgN zo;-aWo{+L7fl?};eO4_$PWclgH{Ll0egW(!5)rsfH+Oo$bv! z29gmv_)|__7~_Tvy?K%l4jhew(`n-YJcC#()#CVz>(_2zzI?V|?JkBG3~1LP-hQ^q zW%LgxAk!{unZ=HwvTSvHBLGx1ddD4ePt0%bGzd+UC`;83+gkeQ`%V=2!j9-`q=+-e zQp~pw)nfQ$d*@L44S7Y)wd|#fY!hwaR!dkfyCd` zKqEQu`FCW96@3*FdbM)ydV_KDEqgoydvbXL~&_(PZV`7N>SQdoV0 zX==vz_zY-bKol?{)Y2hg_Z(FU*zIMnUA$`abF3(MbUVbv=s@qb8Sr|l#N^%8^Yc4U zd$1NmOFqCp_Ob99TjwQRP(gsvHmu*Nlp&$;Kyo3o_nq@tv}GoG2>A=y_S)eCP=YjU zEz3b$cylHQF7>NDQAwbohpjD-2w`A8qM*ua9J&cq!-^p9^rd#e+WELIFjcbx7NoZ! z_5+>%{D{iP=btE3{NQQd3!yc3$#Vu8Sk$(-I6EgJUH0J2mx?^RF85k}l2?2C^9YCx z!(gzCAq!>Zi1~s*803Lx)j-nG@*y?m0LkI>`-+IR?4x`Ww2^8y+;8ro=$~Z4 z*&ge`lf)v?0V>H}yaCfPlkFxPstXR`Vi93;&FCl@8dZ@HNO~0iEE8fp{CFUe)wIFm zf(t2@DfQ?a_8jQqIgwR!@OD%h62v>j7MiHFHl~h*!(j}~{>w>S+qJaJIz)~-RGLc} z1VklR!7djXUAfHQ$F2dv4HMFoXemhx&=LCFvO2eE$8c9s zX+~(Qh849}db{f?PImDHBlMP@HJQ!)fYnr6OF(Lpmnqs|)`77NB>7YuzcJ~#WkB)Q%G}SE{@}8Oim{hf)m37m{Ep<|lK$$Uz2asB-DD<5T$zS=9 zud0Td36p~>u7EO!0VHoe(aji1m+8WhuHb0JExO`sjI@YUyVVH>Y_TGy1zy`B&>({S zS~LhR^w=(q({UUOL9<4Vd*J+34qJ9pKV#CBC_n%l)`>2Z21U|Q7c}k~G_F~%V#kwR zbiHYy%-RkGc{UQ3OU&BcM`zLFi%}UJ@&LZ0C)Ns>xO;ha-45~On?qlch8ZQHZHhTI z2!!692)dT^eS!ke$R{*i5y9&U6IQxt5ksYoH z4Mmvc{GI*&wvo9xnMyi3Sf z72g^F6sf`6p8kC-<**mGcS$lOda=h2K=+&BX zZU;u|#J#OhF5Al7H4sbH`hSZ?$j*0xA-{UE&it(`tkTw54mMbRS!4Lk7ScfaWGR4C z^e@Za1V*8PzP~n+J{FqvdW zWO@*&v&yqu_@k#pI1$WAWsHSWzgshe1O%b;|7EjX^I4nurOzDyfe*J#_u%gy=0Hn2 z*>Dn*y)Hm8`$9v<%d?c^-ka43qVrlhDCn2jWxBe)lEy2c6f+Lfi_^Sh=EaMQZKG@m zcni#4*ji8Pg-*j|2*%!x;`C54sp+*Y&8@mc_cdx(t*&gOV{J$X^fn)_F{+4R>wg$X z$wsw4Pv_va8MZF(LY?8^KQ-YE_?{TVZtXY}Gk|{gkPAK|e3V1;MQ2)(Ejv5GG%a?i zJ39)_c4|`4Z^N!*V|=juSw5c34E?a?8R+Bn<@T^!A_?|{+e*t~hkm>l;@^^Ni|flTgKa3B0sV7uKp?Jwa&2b*Vu1*4ku<8IZ4Ij0#Cuj2|KgspooCU zdA{n%E78iJT&-7f*#hTn%q3?uG<}-CiireLNFMm^Z^KZDBs&>k3#IN;wG zoXoyVrVcjE^o%5ApV04f#>~K^E+&Ncyi%J zzrTD?c0P@R)WZE=)J6pB|I8lUIe!*G@z7uYYxYR;R@JuX7}=0opVyTYL{kMbazN@5 zF{Y7TqOn~qV^*|TDoKZstf)>@u{4gBYbt#jFmqYMb!B3|qQ+H`%WC4988W?el{-3b z$Of-hxhlGXiL+W5&%!Q0o9Cdkd7gSXY2AMM)q4WyeX{eutRLm*AN4>`75LTjvA<-j zZ#D#lKD>a}fuQXM+;3t+?m-C1q@O1l}lM0WrYjIkqdqLsRj**_b7KZN9DrLdr zQzh}kruxU^jI=WHk*7HScdysWBrzeY73W|2&F!bjIg*?CFo?T*rv;tL$S8r-=&{yO z_gsZLr=AbTqKb{~lg|=HZwb@3H(k0b+0tMgDVB-n&X=B?{Zc}r^o-&2CWE`*RfPK3 z<@ud22yN#sSki-g{a99#4C6|1aMO9bKBm6$4_o>Nd*V+yQW?1}Q0zm|t(=}LDh7~=#VJs>p9{*P0E(Os9DDmgd^p#?`Ty%{@j9?AdpG`6% zrD?ZxN;ymx11kOqwcl@Vw*pY?!xxJ38plQ0q>P})SINS!u+NhN$Ne&fUSe;stsY#0 z$sXc$T3@)X83ISUNa>?5yEQC*F|gU2XX>d%viWOhjBP3C7#5(Imx#0!EwqLC^F%EY z?CZuziKXUeyGs7WVDt-~0;Oy3tcK*6JIK25FGez4>_L<#xXGCYOScYNH83Brc^XwK z<^3jiL6>+Q;?dpT*>hr3kJ+WoSI>U<$j#JXWy7e?WxJ*!nMg{^z35C>4iBi4LJP&t zT|aArEAMek|9Ic9O(W7InUCl3Y;{pn|5i3s-8Q!P?pQHs60!RIot1H)380RpV`^4e zT0-O@fAcDC2QY|#|2d&G^M=2Y=dd!Bp18ToAUC9YfSH}WLc)rb9~#?+5h!Ihw%&}f z(9KLe#1x%!gUXC*k&1kGD?uPjabhOhd&(cdl&_Rp%v><8Sy>f_>$*uuV*VRl0nTpa zq^9jPW@D=xQ|uQKJWBdTZ@B6l+-GoWJM-$jdRr=7=BkC8=4Q&-#nl|Bx zC86zmKLN=-TqcK9(1f~PyK+wP&8KN@oLk&oRx3(!m4wUTzXCd#`9Sm28?_= z#(lD}ej|>O%1-u6Y8N&$;kmhrFYgg8{$hI4zyBH`ZP9N@|1uBYq*&$8;nF`Aa#c_s zZz00j?el{rKI^c_u269Yc@!?r`V(%jafz*)OJ^`={BA>^!koM=4I0|ARb@8mb!mam zPp5GR!Pz0T(y)lobQ5OoBcql4{mjMf@JosUZc>rNOGWe`WrVYk5saSGvlN@y22Dc4 zlx~uhqKRJ;sl9zgY(2!~*$d0$`1lkw0T%QB|~k!kB5RxG6=ae0i!{LQ;F-!fE0ZS^r-uBwSo$c%M< zwp-4oG1={HXFD%59qkuAJRgNm1g*Zo`EpQuY0=M=8s$MutKA^LWTX$xa(Pb7b@JdAiL(uVT`>NV z^mN<$+#JMu(g9Vipz!SXoGE4a2}tcqNp0qOGM5&7g3E=yA8a#ZOI?IbW2i zvYz3l04IpqyB7$=C~7-Ke;_4tvbJCn3ys?>hYcd7rN~phj7vYPz=7Ho`dXrlOGe$e zSLdPo3M|9lJ=k1nqSYDT{%>N8svLOrt_%n5e%7$+pS6AN%9Z&(X8xkNp79!|cbkmI zK?O4U8(L1kczdwRB53GquRt1tRDkbd^)KGSJ+C<{1MX#{WiatAw)NpvJCsBlK@NXu zh6H0-pLK<>{``dquZ5vohxf-W0-q!lqdPejMRqQ8m$kUcAGO*E4OBCLxx`_PLp_84 zmAtg}4=(h4u;^w!W+zo*IGdocVbv-Oy@)(vWj<;Q_v^CM=( z%ba57wNqE!L(5`C>DQDudj2{gSj!^8FNbPj{jbhHU(Mj~CL3#WDgrdsJ|9BwEp+Wv z81?X517}X{pSvq$!c&Rfwe+61v&b^B7_*%si5kBf?6HY@)?Ur)5`NkEMdysYwNt4Q zlIM#MrL~^$D6w1n6yjR=8X#A7=Qrk1FEp?k0)`RwKbY`Me+JR3BB1{?OH71aZ%qZvRTcncvW7(>r({&cOKe`Ddi2h>7e%nP2-5_ zCYBB$hS#$2lhR|)dwK+h*%Y;(C}Lrqr~qt|^h=3sx7{vC3qB^We8QFad=fi?If7uA zf)>^wq6a5U^>z`nMhZs?^pxe2_$vo`AixxTI+xvFYY*!+fsnu_serAHs2~NbBe$V{ zOz4>@gT!#yu~Xa^_NZJ1$Xo`+H3$o!I5HSMA3=xF4xyQsN|ry!VzZb5eW0njx}~Fd z;asFPzlw+`l^%QpiSbmag53uK6DJTOFA7fQSaM$SR1oSd=Ia*;ZiW8T&28vodSeMp zPuh&pY*P`bsn)(s^R_X}#_uu!!&0v54-SOY^-$lpS4mF;x1oSaCL6_6rww z?fK8eOa~awv~>ZKY{*Fl-u-mf71(3oK3c8uu9zku@NqN+1a*oTX);IhG-+Qe)Hu>> z#CH59aPNRh>1DIIF_>dfSrK+)j8_u9Fgnpk9G6buK-HTpC<#TgCg!QzH!3v`$@Nt> z^RTtlRKOZ^iv@dTChOm#P8b|0vl~bqlj+5**b!>c8PY~zVsQ9XEFlr`%kW>ly<%=7 z>;$Fi(S3j}#ml&v_VHd(31s@nStI7(C!o2=NJFZX3m25r|mf-I- zLz{kDQ>Q(k4=<38H$p2C#kRj&Q6+ccu7t6FI~BcVa^w;-n?SvTYN=1IU?v1CRr zOzFZIWNbZQ#fdpg3ZwySk+@ADykLW}9nYk#hcSZt+9qIup-qU~byb1V|GirA;Fivu zVy;ru(Wiufodo)CPcza|zVj-kpnp4hMbKlJ9TFMZ;S*Sl-SA*?3Bz~{`8=@&i0sLr zT(vMIPbqlWL4L^u6-SQbVQMK24v%@g$1%!Vk#`*!O2O5uXrN9qxXj1X%nz}8?u>yo z%K3bFp*8^?0yTmde&Hjp;NF^}3H#caCImP4l}v*=U%xM9?m7Z38kYIrz|WFK$Drg} z&(pn!4fJU*IF~<^z(lm4Q<=h34@ieeJFtZO92a$qH;$qQOiQu(*TEcq2ImO{>}P$H zf_0lvgx5;k;?$H}PkIq0uu5J^E?Bs!%&`s6EZS%J*0Rk9xT&lU+Sv zsKy#8&EQ$|ZRpZGlo9fdnKvQml=VsyAcv_q=> zizcY8O(C;`w~d%gC$+u>#`69BR&81#+&ihV_pyHri}i0F|J>YWw4g=cHWnuG+x580f;T@%>k@LCSIWZ{9RzMj3-`S&u6}^lFeY0YF^Iyp4zhdzC2>{hZkH4cf#Vz@# zUF0z*Lp<56@)&_gWY7VEnyQysaqy^lCH}NhfATC*hKc@=oEIIq%8!pcn1bpjB1zw| zJEi9tS6XJ`CmKd8`zR&&7e_){4*JiqKJ46VK74F{eEDAMegD;Qc*;xZ-uYI;N0f<6 zR*R`iK8YWRvJMNXL7|4&Qo*%GU9uC=yWe&Zrvu$HeOQx*XO1c`*4xZieG4H%nNCjj z;fd1Fhap*n@o%U|`FC#%LqiHFUa78y?vv_?ovcfVnea3hovtIYbhXV8q82!j2c5kP z%Dk*`GN`FP@Fo?e#-C<7v>Gg`XyDd9lQ!a}NE=(H<^_&8DyP^gU`G30%bKNi>F8YL zS*(vTf6&r5dSTMJCctH`Dse?Cp7c zyW@y2Wc9JHtWinWVi5}tFNTV?G@z{-YW;ck^E?v0t4f{xoPIQ^Gw@aQ&h_XpyJ&;E{qq}#qYjD#%(ZK z+QFI2=JWEpp9k&wGOF^KU!SjTY%qMhMo-`)xfM4$ou#G|#F#_bVk&Mzm)E4+T85JG zqHeDFfLfm&jm?~cbhEkAcA|1cNh{2-#=6&lo6Ta%J}86UmAdy*)a;Ucu~ou&D@c;R z2Iu&g|JfsUCcj~JX70RZDCw#&lB!Gfv_^YaN#|l9T@G4f>)rqTuPE1pgMTMYsKyQ+ zyv23PT!qA^^+v~7C`GW%EC_C!bgW?bDQ~yulK?INHxsWWqttU(7=@=3*wL>+ysC#I zWJ1wI&s%*+ZG}gv?Mrk0S{1>uoh@NLDsOpvz#~=`n-hkfLBFGVLk)A>HniQH@l|c7 zO61^Mym+OG#Fbb1bq!wA<4Tyu@h%IQF=>I{Z6A0x3)urpsMb5J;=Nk%xd?kgG>fnW}XY^JxDRxZ{ zbL0fdnC0?z+B$`h1)=<$J6eCgjLNTlUmBm4QRAoRi;=41V4IZ;!Ix@klVeQfm+NNd z=3dr>FR_7c-T{a(38h!}avP5d__jpkek;Q$?Gib@%`wVPe8=Tth$T{LWL?~;Zc289 z&2)Bi{7k1CK(xk={n3F!{Fb-mzP&;;^bV|(;l`2ys+_0)IYteUE2KEUvwceKLm8~& z^tW;5@U%f6s5=kg5BU_SF5*p3H$#Hky3kH00g$ewt2^Twsb@Bfub%Zpdz>SC9dsW% zOvc=?;|jk5xPuBw+mGx99^|O@{HqB@E0fmd72oJAkC+)--O=t9k1xk=-i>r>kcp`4 zbLu~oW|H`Y)N$W>0Xp%yr2gb$C2ZIlvA7>-yTt_9M9bf^)- zBYOe2DVIDE1pX?)M0l22U<#oF9eSb-6#KnV$6YNZVWz1$#LKuK6(P1aTqqo!s&q?H z5nMwK^x^(HGGKivbGAJ0^}D~4+aXBZcR-L>xQ8dY{-$D%(+B-fTOB zUYU8eu`FO{&d>*{n*T#LZ(um z``hHor;Tx1U_ol{`y2GneogcvmD^@mkx)=F9bg1H%4r5rL~rmx@K^A; z_*39S9d1#0;|St%x#05<(%RPwD=9#x?L56X4 zC?=#IN_%EhK&_t@S;|9Q@HHsF@hhe|Gs2{YJtOx?X6Nlzgb-gQ+62M4+Do2tgE==L zCQKbxsvh_6`MW#Mi}qN)5GaK=_kxkOd*Bi5o#0)$mioJfUD$De_uWNNy@k&$&Vg)M zAjx%rI-TxX3ZKXUFd$t71oChtU?e$U(V%K@8PN}SO1yCSDNhTd=XG)W;-15sV;R#W z4>W5`u~PD1_tJnuiz(k_zlr>j^PmuuAW7bG?=o{0QS zdC!cny$(X_sgM`L9uo_wb<7~834E}Nv9C6J6zaUqL}syqwr6fV#a(1C#Og0Il0 zz(nXvrH2u&yCw!5e-fhhJZtKv-Koa_S z#|uS4YKY-W!u@XMiM|IjY(@lsbikPg@4-$P8amU8shEu=)ua^P?chFSGYh}C@IU?3 zj-b{=@QRHAbCt0NHiMY7R-nMyHW!T)MliRb%kxtr9m{QH%?vvXO&vSiZ;Nu{;$D3) zMpn6nGnDEE-g2l;lcRl!ZjM2ElsRE0PyBP=!tU};KH+LaHDv3u{>Z4!oFqZKQ~zfZ zNh6J#kjn=@b~yAkIFBpFEc=zaCf7jE;GUPVl2omr*5@VLYl5Y;c8lg(OI0*-XJ?cr zK!2;Ts1HqcU_#+zFLZ}NjIB!OY|F;eO)^FUSo1WH1n(~{wL@9;`Q<|5^+3C9;h z_HsQ*nF5ROa`T*l{A|S3`#!+1%C35-t_h1?xC)>5Da}N(ri$_1ac9*kL9|yGlimwB zDqBnJ(nn>b_XtzwtsWi5hs3PopD%H_gd;UuAw6csbFn0W-u>V*PtWRKOX@Zg$HGAM zJ!oNq?+ciXsD9GfSN^+p(vtJi^6cXsgAsYoli{|G`q+bErUHl?Z#A z<3V%5*`Lsw{aYvBVCG_{3FR?6;Ik)Enl?M#C|e(Wsn#}=;rn2{o;7y3yHkscPdg+zN~rVIblr?#Q@P|fmk%FB4Rv$T|x-us&O zTd9swY{lTd^zgnn;rM&8)ssNU*Ij7yL&)gouLtCa>3Dxna@Zn=Uz?&Y2IZMv#;|Qb zt1WN*@A4ORQwwS~cjvI8n;kJNCdiHd@027kp_fMe9YnxGfB!G1M8nPz^`u>Halho8 zD@3&*ygY!*iLUy9%>NJqdF?LD)P{a-opF3@72XJLp#uTDXcOBFjuQzqz8*gc>^V^2 zv3cD`qoF=N4_(95i!{Q{}V#a&>A?xAt z@)FU&b1!6?)9-fCg%>k9dRaSCW5~SaJLs)0R5(;xcTs=>58j(4mSOO>p?7)1-HHC; zZ*cyHLg%9}f$?7*{u4nBIR6k=L?<2x{n>-E2f!%Yt&5nSQjCkQHE`{p1;6SlUAqC0 zU7u^TJ$(wT6&W6ugnFLj$evaSR>;p!x;E$&}7fbo6ka%)8%9EH<(6_Gthp%@GlC0~xMa$J?cUf7s?dq~^ z+qP}nwrzCTw!6!=dFy%K@7y@&-X~5*M*i5bBXW-zb4~2E=9t~}T93zq*;E9T>TG5T zVXa^Y@6Y85wetXU%In{cMM2=PHE#R<=#VR#I<0iEJ`G4oXK4`1Yq!^*LUy&oa|QWv zR4(pju2&v+mh`tTWZ3!h3XcO=^A6=BXykJo8K_AI3Hn6W6Z*yztfzBz+PFy%wP&to zsgP&S>8Vg9`5{6gqtU-5FP*)4y1ce);NJ^U&8^kyVWUR_>8P@#rKv{c5b)QFP4AU6 zQ|K%Y{$_*5q-ZysC%0bznQ1A9RknK^52I@Po~m|@_3KO+`uHD0V2-C%TY-T>H79w-kC(+n&5G_3@Ndm%$t#|+6 za-U$->}9U4OUoc7uDDnRd(d#6#)|ZOaN40CYu4VhgSPpyIQP!k^nRf3{Cs3?+z2m7 zw}H|(KJ;+t$`jU#grIV&JgklCaiXYtyd)ex8L4CA3ce%8RQwsQ7;59`;}D;;Q_KoCo*Cd@Sugx!iFb0qwquwnAZ9+SnrG ze6}shHmYDRj%FkU@dH9X_<`{2OrOyDh;bi#RlM!)W3#2i(P#?QBCE`b;Ol}O$kS^b zaeBbSHo?m**8%0n9IV3yFy3Y*)o(RSm*oCm$_G|dgw_V*Xf+g!)o(@fXD}+YHLZ)O z2Q?7x=IqJx9-=kVr2VzkHLZBm1H;2!gvP}Sc0eMeX1ral)>~x@ha%lQV$teLhd~X7 z>iF6%gH3vPPENoGR3!Y~q+AKEa~6%`UT)y&YGOe;wMHA0>s53_v0bT3n7+b?UVYKC zIV7Rwl9SsQH^G*X63=QTFAyh@WxFzpko?1RyYXzK#q)vDvpsU@I>!Cy92UaTz4eFm zYoq6b0d!rK*p|Fy!erAnvsV^M!-{xLfT4xvAZT9YltR1?K*cWp&2wR)ePN5=;y4lMR$CaO~{rIq`QQnN_87pWi zrPy-$jG|dRr&SB|;fJzu1BoT7pUtLzTAoOXue=m+7U&I^ zefQF!-?d~0Bl*QZYR|1pd_VrRIYB8Oj>$%+e$H!T#ha|1i8nCFq*{7c&H|mIF^T!*%LH$?JZ5e zEk>)?^vLZimkfGYo2--tIF^Fq$Z_clC#KMO@W!dcPL2uDid15%?)G$F;)3synEOoZ z=#!NvPs|1z?NY32yCxdw*(tnn*k81LUAmynxzDML9CuW$3-iV^TOKtr7L)=T^h2`6 zi*tck^P4mViR!!s3!n>ELQV}*VHT0p9yGzSqY?K4AQs;Oy(Iy8d|_NurQ>z!+@Ewn zjDHSBs5a*gg-cJ0jK%`Nsl1NCg^@vc&;z4CBq?AGD}m6&7VdytxHrYMc(p~v!z zx3~l8eWiTA&~RSV3z^L%b~$U0wN?$oWouG1Yvpf+0$)H3-$8ycvn&tGxw%G?pI)p% zlk%a#uM!j4KqcNT9V-0<#I}?ur&Ll7wI*d{5VOy z5jX*T@gZ6CLD6IKbrt@r;HNwK$EiAiB_&~W5ZYI=MBjNM55AXegm02~LC`kWpZ2pG2rJ~8Ev92|Q@9F(CUzr>#B6S^DesxqB zX2gC|2+pv!yHc^aEH)hV zm>=j7zT+JIp(#C(fi`s|Ro{eFr!x!oV}a^G1%fkv9@Vsg*F>H3gUuCYu~N+%WJkdB zqBULLBKF87PCMfE`#}5)>2{8LLKwHqVLsges6zmfB>x2lCtlO2HCSW(;}Vx|K}t-d z3NgL|5J(%I;)x0$^X8u)(nKQ6O^msj^t0Oc8qu*1sGXAX$A3!Sh`RK#vX{6k={jC7&bS~B!whh#O;UZmC8FH>w z6x4o)rKd&`jyNDuaY3loW~?|E3Y>2NGuFRuvs(f$J)4>zKnds38E(X zX0@O}Wl872vjr468g@MbVi^;+{1S!uu{dS01&A*Jdo<*8ZNhdE5%$WTo5Dr$yEtR? zw;^UXWls~BCI zM||AT(C>-)a}D@?*(caE4ad7Z{P*jic^W$+>(|KIC;#D*7zoaSy^74>(J3lJ=5F`{ zuT``TnV*@3xCWsuEYZdu!Ww}=rY1?*_+jFdv zP6P^$Ito{>1eEKAxcM~i*~z*0i%1g6qTBD_b z=tM4#jAs=?EuV{l^59kEa%bbv>Ku#Sn{JqxSl8fB^IoEfonr9IeLZJ%QK&}#FcdIT zU+TuGT=cda;bSp{%%d`5_GAdr9>U=LQm_>w%y%+|+&E?&-Qe>b&+r%Zr*qq%#YX;d zK%c9aY^GFlgBN>a&twrjMBog*Tvs_f7d{3we(zhM!qX%qRAb^&Q1wg$vgbR2US56p zjy2)hJ3zaiLq_LiK%BXP%UrIJH@fKJXHS_gFOU&y1FvM-zR5g2OuI20Q%jG=bDXJk z!!bo~Rk8lgpH+x&7kbW5J6$bnc>tf1FY%Qxu~u-FM*NpIcyz{>9n-{3TS)c|)$6^^ z`T9N1^2`QWt*4oe5ASSWZ9-&(ur(dm+zSSyOwY3LT+FJ=?~V$Su)73FrQ)E-MpJ(Y{F%Iyt>0cOc6 z;z?@f%9qd^d-cGEu)|D-^ou2MlB-3=t|*~SiCSE6K`y_3+C=c|&7?CxWF`&Ht~x(g z@Qv_GNHh%LRZNY?u#%rl`-+O*^&GkT%y^m{7HfC9NZKVpVSQ!@hQ(82z91gb09A3IGkVZ*ac2K4QGW(W#_b}){7^Ldr-frrBR zG?`VnO3LFce#jpY$zAM8N?)3kc*v^h*tZ5~7dAEO{%8iZ^p!D?Mt4!R-hvn7 zMkf#H3!X63io#-sEmEAVDp#jO4V2sTRP(3jboEr2+0;p-I3K>NO)Qt8i9Jo= zY{=>dGn(Ni7Bz->IPzbm>_`<&9k0QU!=+2%T<5nq8_}2&QoRax-qWI35{+YP)m??? z)n+do+3zk>0l8FT6i=DPOiO?jJ>B36VgTGn>Zo;9BisJK{L4k!o)(aePlaS1Kr1(i zpd!`N*uQOVOvj&;mnQRP{B$I6o4v1!%Tv&xef?mKPc*1ZZdhZ?`BZOyqCaKLUO#_L zIA2!UUP8?dwj%-ly(zmpV`m4jKFKVe^LBLsmMc#|C0An4JW6ot` z<)?b~dr5?o#@fj%932{J{N=WS(9jWy?G;)(9UP$It4w^#p?#V!dM)!Yyl)f|dE`4G z<0XeU(!tpBrI=yq=FH0SjdaqctTyv#gvc^3pmLc+J^Z>>V}EMMRqoq#t=;6~(tra#ZCENan4F}fM zBi5z9A8x9;cDk%8WV~1y;Xo}Tf5)wTSbvHmVfQl5{gE&MnZ;wiIxCQsWV7R zlFV+k{%4#6`^;WHqLk9Ja`nSB+K6CIgfC|d#jvsU)n*>FutuA^7+`5thIw>vkS;p8 z%7sx*n)2FHZgJPT8EejFL_E*lKwi3cBDjyJ(P(Ogji{ZkgsE3F)~Pg^hj>gt%(rJi z(eIJ3bXBP|BwZ66t%g4Wfju{T?|#21YOM?@3*Es(X6;xy>)=+-_UOxO&xLdUlK0#o z)I9wh3fPcZ-Pp#Hp-~p!M32Cz{tfHn$-moTwa(J4*br$>4OiFN~@_I7}b4H zr{1zk;Gj(bsj*``q0`!7tJmH#z;g0ZsCDIBsI4tI@$<{f!B-vLj2B0SrkBiK7k{)o zh?{=%drBC>nr6&O(WPd!j^q#XEi#$L#N!@vA(l|~6LKza^q{m?-x8yE%F3gVhJph< zX$zg{2On^tixI2x;KHJ#6q3?%lt8LNz^(?;y2mPPO;&8D$Ot8~R}6Q52*bANN)GC1 zs;2nNL0mVSUtgHaZugLkRJJ-EuXKkk6y^mIJDPQ!hZ-@BGnZbKq|j!RidpQ&Z<5NO zwqeL7109RMPW}gSbB}~{AK6Q%J-0aKTsXjmrNTLM67m-tSxWmn;YT7!g(!k2$?}y? zU!YYGVxLn$HagW`BvQwiu<9(6xm#+-K3W-sG(jh>T+zCI0`bQzoE+HC^C*lrRl3Us ze7MC#9XNq3zf-5$A#p@&5a`;WcNG>#9lDxae;Om%8~T}}&r-BR*$F#V&(gOcwjjr7 zlyuL*mh=OzbZOMi#xm{&E5qLn5SQ=*4tBDmJ8O|Rs*z=A;z+Ni08*+unI>&;i0iZCkkhQH=i(5Mu2uOGQsU;)7kbA(VFNwu@ftNW6BMW$-u{h?sD zSDjVF45PnCED2cAK~O-h#l?CI65NJQG53k3$Q}Uw`bO(Pab8el;rYCI+C0fR;*nvR zULtRVa3z7rGtQ(d5I4spC@0Q9KotQCl0KCyOaZ5hUH1txg6p7CZ=q{S1~vQ!nJ#Kp zGPPEyT}HTt)kq8J;Q^tr3}qs)Br^8lwZY^XR5GR02|4jddD^4wPWWF%KoZUkdvl9*DEyjVGwnBsS#o+PJaPe*gvs zXup=aZ`+#xG~Z!*bG;QhrW2REVeF%5X5d96$`BQ*UKFj+Mu=(tFtgo^uMLsj?mb{= ziU+NJX-`@R%a`E9r3)*QM_usl_gC2??RO8Fb6q_?m;+%M0t$;LL7C73nCZvy;x%y_ zk#vKW^~g)r)=gx}Hw}=>nC6Iu)$=xes0rnG>64iJfaPF?9U|RbYy$rkg=FSEDkS&7 ze{{v}P@hlp$wmf-G5i)G8)D!_4j@ z)uri?Sckyyc2)T#m?p6&T%XfS3`fTY42#ka0xguQk^|x(RD~ z=2c=`nfsj#Xl10B%BHmv#V;@NFO#eWMDUt;O-VMfN(!~J)N{iUixPMO25t`&XkjWu z2|1Hn5+0F&%5wu4!5_Q*L2#?GLM`nbOwj1lIG{V)LSIS8o$jAZmUG3$BKq`{g?b)e zc~37XBkqk(iuhzI$bmvt^xt_56>w~keX9@{Mo!Sj^<)%e+YOQcjzh_@B@y6)} zJ0}%qJq`%xQ$;(cFN*L}v`Id^qxya;8sVjj@%4r12l|i~dbNT^=_5{8DN9gPcwq~; z<}GUZXHfUgQGpD3EGgm-t+@JX6uMnk{ei*jLvl{pb|VqkY6il^t=sMM4~hl5*M=Z( zlwaF}v1p06$u#(o_hQ5V>VS)%j5X$2bb>wW0;pK2;1-xzJm6ZZzY09`j0KY~cvnMs z3`$GMz#n(cz?v?*US(!g4RlW1C1G!XTjca^`8Uw&h(~( z3VjCmjMq4JNgY0T_arN9+Bt^$v>j16BMf!c+?$yF6qhTzY8#}b@+4I?i0WgndbWR3 z_N9(>&Lm35-!3e>inyNkr|`HN>={c(yq-({@gbLxriHiYn$A(FnVj@R8|b3@mz)(| zuE$v4cw}X5@(`xRS3De>zR}<;e6-bUlhsF~v4fLXnqOL z;omLeW&cuXpe1<9B|c@0fbVAj0aDp8W&`o8R`ifh3Dlgj8>j6{Vb(cD>c>*{r-}o! zdCZXcZy}J8+XH`ag)&*2&2d&My3(WtIf@`V^zODOaa%Ct??XE<5!?2YK!r^7cz?Z4 zarN%pdG+KTg>SUl?tJ|m&J+4_4OD(ws`DlksHxLRSbY3bc89NVkYXj?-PGR`fk2g` z(s+um%+8IUpwz2nyyO#EOnVrI+A@XR45%_>8KU&n0GJ5K2q^S%Ig5*$%aQs}5 zQ10FS?Z`=5KC#k2UT)!g!AD7ci4y0qJW_FiThNg?7@>>Tm^U(jdIec@Ls5~tz+$dj z)fn^D7^LjVXQ9hYjQ{nJkjsSC{06U){FODlpnFZ>KcSyotuar}E9D-En)#mN+?aZG z{Yk>jyJvW_?=?PeUT^VOgpWLFUy_y|Vj1980m8d&^;Xvz1E_vd@{r4S?x7M6<} z0`RFS7xHnfS4Av|ss}5{gKWVfftUgxRXqO_Vj@0(p`3a<(V?_ers|$u&3PW{^p@;* z+XH1}DOL%@+t6MNuGdGa{P?&p(0IC_`Bos-=_ycA&X@@s?~sD3Ea>SFS;A{?Ir+6P zKg)?8%jQual{4|T#-D;KKIAJW;XzmI(1*(^eD04*vb{C7p~S^tUr;6667@Kzv65N6_qj|JjevwF;Ng{^&0zarL*wil)!_-_0t1XVP&NB^vYhtCHFx zLh5%=UA2^DytGB?CH>JeFM8Kp3G2;%whelH2fzsq{?I#Ff4Nd1sw$Fsz4ik2MXv!x z$nJYkY@*6CzEnk78+~j%Wgy)jejb-NnU>^`zYK|+H+GMBt3g(HgPBz#d*wBNZG4bb zx^KkuMjDme{EA!vX0mpA&x-te6REekQ=EdJapRFrpY;&1KZYt)m+T5@B_6g&%a2^% z-_>#2#-cDxnM5+zMg1ugbS29csE ziUR1|y|zdvR84clb4)ol^wyDsE9bOlP;`r#W1=?h`oGDAKo7dth+{QfZH;|0ktl zMWQHLa^TPM3UTxo-uKH*q3$M&Wr@`(S^1`F2C;+56xPZvy=OXyDTw5)wgAVP-jJqi z_{b}&Qt+Nei*DH>5M|VR1hpk>5`~`D{u4Hh=9o%_ZL390cg*FU=+GCwl)yfjdeTKH zGmUg-D{kU)RRw)9wll9MEV-Gl_^AaYKTSoIsdsUg1(+(TSH>!+?Kr_2LenXDm7RbRVf`4*}Fd6a4l>rKX9f4S~vNswJg#f z`n(uvAWoOaf{m?NKXC|7QdksnASENE1Uk&{Cmjj=ql!C%y#jtbau)8WFTpI;{oz=~ zjOXu=%WX0*9>;u1F50k(yIodpnswE`z{cNX=aPhx2I7=uon(Sr0DL%1De!h89dZpQ z`W?av{>*W{@PRU4##)uWWBGFk5O3t2vdO6}2p$328s;fCTaeV;5Wd`gyTG(Yhw2S7 z%ESR7s^`u&`#_%5y$wJjW86nJznhG69Fd*3@})!bQ2wJ6AdQlVVitwO-s%hD^vO%X z_gQ7RWxFI~Q}^9l(R4uvEd2-P`>Ye3jWqvyHg14k`DQa0*|N|TQk-2)H-lJjzWi~WaHkFxN_n?ThauOUnz`|CDXkw3m?TZB%#iE5vzKmF z(y=|%mJ=UqxQ_m%lASna8(GB{AwYZEPj%!(u}Fzx%eX)sG=FtG-jVY}ga5SLbBnyc zgt1<@@hiDy=R8YTt)3-@$7DU6)6p^I#X8!MY&&0N$sG@9E_ znhYb9fNL_bT0oJZxQEP|k9i``al;Ww#WiR>&@mmHM}55)ZZoW0iLOwR}>>0-D>z=)~LEYtO+`bkXL$HLr|Ys zZ)}*DtR9AhUrJxk*`h?2~&fZAaXSgHDgw1U^GS){!%IpTU5M zp@f*M6gTp55D6gG%C9>116Ozt(}648y4(yQz-I6fMRP_TL)_XpBtTew1c{jlPSs=* z)j{KKF3L?q=&Q3yPqfD`hH?l)!6ASK9V{w{3d8Xbwf-aQcVY+;Hq%dztx^?CO!ge8 zA4r-^6gkWagKorxtJ(TbMTAD>D@GT7+3`vOtftbV8qlH`$5_F@*npO7#mQUeioRGKikgDqG@hVdZ)Z zvl`b-JmryA)6a~vA4nPT_Wo89?ZYP=MAyXOR);$zs831z?(=*^nT;qGT+kvI;%x$k zpEv7|v2%{*Up!{Ki{f3;AL%}brS!dgKaUCWj*#(Jqe4g6FYuM8jh9!Y)ZWcRZ4Piw zcP6NtCKrZyg3Uxu-5V$r#jNOP4NAVB#<)1zMZcD5!JOL>D|I2f`%ECxs>G|bD$&k` z_6~U1>7J$ZC`=DtTeu|)$Z`Wl5%U3KqL&?X%d~|@oimffHFedqZ+a9N0W0Bxs;U6>|=?+3-C z+NIf(Z_Tvv#MmLiPf+b6@X4&dVVFn6k4sVX4bDaj40cU#dyS(%^(I#%3%$|YbAh$F z6?IIlWv5t~9+U^Yk~F=(kHtbwCXjvCrB}ZBPalsTd_yu6JBR-*Qmz@$dg#$E&VpON zw?pFo?Znh~k)jxmHtbremFBfb zGEK+({3p;q8-Lgtoo751WYi(zcKSmrxTh9Y{nM=Gl^0L;m}@{!|8-ruy;ajgeeS>- z#rBXK`lB_MC|Z+eET!q;y$1q^>SQfz%rJ7iL33_i zvvpKLBQMRtBOn2^z&NUSU8-7W%n9$jVqEZqQ0!DLguj8Kk?Q#(+{`edbUiD#g#5l4|uqmskDDBA#%u_a8t=!*5QU_Mvf%;Tj(~K?IUjXX2-pA!j_FIu9&Bw^l zM4IPPNr6HDYMPal0%@jW3u13|^izpjPTHu+hacjIOqNY+k`lOQr~UaNW<+xp&8lYS zutXy_Lvk%jk#4x!*k87%^8FP5L<5!aDXzoK@<04!)18Jj>q&~WSBBv!iYDt*#tcFf zkT=A=HmwRw`MiEdYoE{2>HAjkSvgkk>?&sC<~Ap$Q?z6xa?(vLKTq1-WE(_{X)Xw1 z>z27GW_HwWi-H9HQm=p{Sz<&bxyUiF12AMS2|je+O5idqduJwX;D zSFv4~bhej~vCD-%6cx~#DFg(uoQS8l8`7Q;-rEar7jZ;18-){akldD->QOHe7o3A^ zWoDT%AF-VkA6pX3Dq6i9=C!e{r@g<2m)j}r(8aP|N`4J77NwkRw=d~vJKIVzXi7@V|4fv!t&lXHss%*6xu0L0SZv@i*5fZ3(4bDlKcw{C!MY4v zG+Zpa#yJV?i{B7$h$+%AGGAFUH&+vf@Y&_`tqUk*B4o@>$v@UCk*VwweGrh1Pv@0a z2TysW6Q6E<-bV)qbo8>1ZNJ>cq|ZZ$Hyg-~pBceWKwz06HO@ckTfs6<>)^sjMyN@) z@_(e-o#Ub*QB_ki6Cle1&*c%6W)2Hx#4jFBw`u>v?!CM`qw?x42M{{Im=KMPo zGLV)+YoYX@lc8?QRuz_C4V8e$#|4`=A0}qE@V+tjzb&|&qxcF2-5l;18})~?*AK$B zm+FPx!w3|^CpeqD&B^Nc3Hh)bn_=B15#+gu?u}2F*1q?l`@`Zp@EJ(7vPAHYXI-5r z!PliPA0f2TE6m86U^PW&m#rj z@I9Xc`2amtLe1T8cdxU4z&8hTn|8v=Vvj_ziK3uZZBKtVZW7=35yjbr>nw#qZQ6x?+oGeLq%0Hisa1>zuj>X zEx)pgnG43Y%)o(~6r6J%en}2P=$ia?_#tfb%r2&MLe{PBF1UX;6}Kzefj@%09-}O9 z+dJdosE)DkMk-!h{MLXKWGBod?;UtDQ?0ylnji+veoAu5)V~xf2;Cl&m~Pp$R1&Ow zU1ZB<@$yWnYpk{x*QS&f%^3BU{n^__bD%ly{4fI{7hB-B^KM<>^w^nDYdYZ)0z3Q- z;1w}8?=Pei-`_GM^ESBhBsD}DYblDNF7YR09pwIIMZ)0t%Zc98khS@^Tdo{@S3Ezy zjUc5Jcb@_UYsk(pGxR}cMcheeGdJpOQFfZ3UUrNWuDZ{yUw+x$z~y=!vGn;OSR%hF zLfK{XOOgK41}6~#=sU6N@O7cb(49W}dD#>K87FVpbF*BD;21z22CL{IBkS3UgOg!k z>+)v8yhPw9Oicu|$k5Cc5SvwneKtv8H~nPlggAoCn?I5g%U&U48$AW#KwZOcJ=x<1 z9X5LD?^s~u;bp{yr?~@?94)g1X)Cr26KJEcJJxOtZ1xX%Ucxva5qE}{U9ZF{R{G9N)!XZB!7durAxmj!rvfZ zgZu_VtyBtxpTalf?@rs?_$JxK{{Sn@MHh5750-IsOsL1^(6aa~J9|Qhr7-Z2DYUdn z97_KCH`%EEQZJIOek zE2zJX*j`ij#E@4YS!4qKkDV+sbIL$}tpziVZh)b&j#7(3vSkNJJb@rxQ55*}UT?B7 zgzDhSP3(XLbJy)fu$H;A9^$7(5ot4?cpy@}=yury$;t;3eHwyc1_a3%KF-QWhi#py2xdl z06uX{dxuQ$j!@V*ueoEZaHKM?D1M46kWlh2N%)>`d1HRYK<_0ql|%d`UTlDS1vc#0YiPTB5I8InDM$ph6Pz& zpsXrZ|5uh6O*;fT=c+`f7IojTAViL80`1RhShYvwgKU8LoLYpAp;i|~!Xe>|Y)(FB zn1l9ec~FbG~$5bQL)H3PX&dcrERS1jqr3666l%vPhaga;!(s z8>{(X>yH#-C-#>WvGrV&zFSuj^sg5E{ZjH)a+N)H$G9TTD8KtbUk;7RenURG0Aq;4 z#`dYa=vmcqgUBOUad0BOA;^fmh-}lFYQq_mKJQAw5pajIRmE)y^UaNyL(8HGJcd^0 zo9Z)3`l=UW3l!_6Je}^cg>s6GT{}f-)d!WcrfZ z;!BWY=hpVfqDQv>q|gD?IARygxkmc$azr(X>d*Fsy-NP{gY>$so{P1yLwGMR>2aQ@ zrqaiAwKa%7^YZRnQubH?RtP=Qay)zXJRkQLn(AUdTr!pt_e+px`bth`#(OF-x4nG2 z^DXanCXw%_k@h`%HGz`mAFTdSsISx#)AQS2k{jtL*fOMk5Fcbnw!9`qe!#7uC_H8E z?|D0E+xs2B)z>5RZsvLUCWy%Y0}4yk^rUj4hsDNzd*h{d8~zMX@LNaWCa~5Jy2i@8 zMNre3iwiG!M3I*HB6f*=F19C2r&UmPhn4Q!g9TZ0-bwH5X`0V+m*_|9tAA@8=Trv!=)?1s0Zr>PB^CnV|=##~8`{L98w_u0++#cNX>cMzIKwmzDe?vsf|4i&QS%b<)Tpu%sY86u45X>Ptko_Q`l$Ks zk*ht|^Dbl}8tT@`Mb!!yjEU>}4r7>KzdbmAtg`h#8Ys~{b%OmYp}Wv-c^-~5RBgD} z_HlP}%bB^Wt}ZWMQ^Dopz=PebFCkyIPtYrU6d84N$=BXnOXmTNPv-MlUf-P_xcGIO zfOPSEFMIith00}%&IKp@_}BjZ!kup2+^>D?W8fKG6^ zeKTUaX|QYLk&E|mukVigBGHg#WkC5_qGvlH$=4fnV-9f}nSOXz9`e-A`1zASm<45F zw+sfig&uM)Tx9o_#r1wv(ky~u$?y-hZNEhHar|nsA1?2_D17n{>7fl)Wijg92WXs- z`n0QbB-7s-hbkt&HsszNao62jMtQdNMoXk&Ip`W;T+@xdBY;3e`w16fv01eZg52aN zD^0}W#HL}Wf|**|X<+*-N|*%>$`ldf(>agKNu`-9&*antQ_(QFMu)M>e4m_z`$8ON zs~u4MF!?DvRpZPd@ofTnb2`x<``-JO9!hqy#rJbcRND&ZLwkfLr(AItVfjwperbxy zCc^v?pg=PLiR1Bty_`wu)?ei*6{~Lq6Q6cqB+(TKQ;5KHYy=WF&L{dll*2{LnQA-a z@tQvbjS5?g6a__~-82d(8P&n)zKnM>5CV*arn?KSoigX21>(JZ28(LXBfJj*Y>q*R z*2YLDDSewQ$PTcLG2?pmHazBY;mh&?o*H@yVG_(dq3tIf%TA7B%y1@sRFFw+R@Ca3 z!fqD$1yPifhk<<{;Bya|X~J_M2m~4^{aSn;{HN$)9R;^smwj5hVu0-NpdNtR-2rFu z){s^R;3&HXBTussgHiUHNfdfX`ewZdHVI2f71Qy}2$AkjQ#)k`EEt<(m!O4n+P8!= zS)kl_`5(;5Te{1=v7fca9*DsEz&ZR*Y2F2CnQMvU-8K6{`z2{p6DYY3 zw0hUkf#N5Eb)LiC%lmy(+L}`AXtkXTvfLj`Cn9Jr-M^~yf{s}|$81s@>H6K~t(9t^ z%X!F`s%|xx8}!OevpckgYD^dZOl3nG!zbIomfzJ5$d$OOp5rUuRbsO**MIQx9s z`1KgjJs5LyH&e!!X!o)pjnVatq(T&sY!CpOY^*3ofEuK{u!sgTx_{Kru=i^DrO;8K zRF?}h`x~Z&0PZWoO7;k<&N?R> zdx{AiM!pOMBnb$TpvS!=M^;m~s=G6@sPSw|>_CTmYl4O0@}sJtK_b)Va-fV7WcsYk ztO-`})fp{+a@2v}-KZItTPor_(`4$}fz4|_=KJK;yx`XgUPZTqee3Cgq4@!$*D(>9HagVqB5FGw1Ir2m7Y1^OGw@9@c;;F#ElGjlW~K)oSu zI!V!nXumeYtJGOOCT7u(xcRAtPgNvR24hXZN#AxtG1Nr0n^F-S@-A@`jM(cg`@Fzm~uGT6-cRxT~6)fJl| z{}aTbggHY4w5N6^!_ox{W)i_duc!h4%pc9cOeXA*4{Yh+=}p7Z%|wYi@N1fY(%VI% znQ*&LEl;@FCW?Gx}<7FVPJzYK^D6@K|3)&``mcG);IW}nI%Q(C_+`)?D-n08UrGQE?E2w_~&$+ zGBCI0>CSdkW0asCIawznOV4KS!r=MRvW=1Hr6|X?@0$`oDISK+cpJ6uj}xjElzL=h zf15vC#qjrltf(jiIkFK<5ko9p(@-eigPwo=t*-koz+(`!6i~Fyo6XGcETsS605$mtiI4!nEl9iUzAx$ z93;g^aL3^v*+5@X+WK^!=YDe3*7Ant%j4j1(M*uBNTW0uUDi-`OR&;TQdVzq$hT)) zM`37>S+@VG|K(#c--Ho@UU)qK0wn!Ix3B46%yakuW}dtMnCFBw0ESXXuQjt_DUWt6 zS|b0C*}<+UQ?rJZ+lel_qe4<1e?#O`Dgk@e!E5!n!YJS*Vj~QVQad)1_XElQ9T|bD zmGYA4%P}D+wyuH@IJfH#)5A+@?+MF%G}g&>T-%j~D^&`;kMf1`Q!Y5ZKF0EiewCb# zyL6@LvX}IO5ZITf9L>{Z`tKL%A}o>q0K6MtQ19yQoB~;nA*>}3kY(4C`u~{cAPCwm z2wLmP(#&_eZr8s7D*)2|4|`qz085VA615g`>D5zfspNj0($>z*$)RFagb%Z}{muQ` zx(hm;xl^p&9X~dJzYAM?ZzW-ZqmOJo( zLRh*6@jRZp#rNa&oF2dGwDFS94#-a~*@HTM)z|@b^REM#ctD{|S3-2SfL0CT0mYP@ zh6vx1_-Oc8WM)x5f|5rZdapY}&bYUn0OtAmv0F${f5L$Nk;#@eRCGEziaWd0Zp+xv zhosT6E;bEe*OSr1_#;Iw5*!)lHOMvr6V%XO>)av~_{)Z*B||!(M}VTj|8VucM}ek) zTWJ3)3N-z*(0s}lg$Hy{-a-5Yxysy3x5=SqoF75!5zO3HRNKgf-&-a;L!^;zUQQ>H zw5#B#mJsDo8Z?Ni)M)`bgtd-}wQzKtVuo-Kj`o#bzHO~xfm1ZsOElX0)f7FY@sRj; z+iELmA5fr^^jwOJ8MQFUD0;0A+z@SMx#ZM?1_Ja{tJG6q+N$v|8b2oeu}Y^TRiEnRGoM_`4KtwP9Aj^MS8Xr0|+0K3t{@oU|BIY_m@{#4_;{C0H|vEu6W+ zuv4z zOb1c{FE3jVhQb{&w5gq|tpd>kD^wiy!AHykvYb~j9!~?`9U)>gVpR)M?JIM2h(k_}3NpLl%3ifBLU z+{Y-g-*l$QBYSShZoQOhJ)gI#B^N&PIP!UsTjrut<6lZbSIpq+xCWmlGEf9F&=|He=bkp6!d%K3*O z48odjg;8p^>x0h;mFRuM%T}U4_-wFr*F&3>5D=%9(YfglhUNCASLCRFJn12Hy>A)s zq&GOQB;2r@fzx8QTd!Av8RhO@r50a478pX$woP#?=daopJsa80m|PI(FWu+TK8%0p zz5zL0QtSR;QA;x_+VuGSSG-@Rqmc!Dby090yu!rdT=c03l5z!=bcVZdjLC@+E-mN3 z%={n0x$57R)c*?3RsSq0Kx7HO`%~dMX%KHc3L_W31_-$Xq^Hg5`yS<`B^_qB%1*7s zB&Oe?N79a=n{O1g#CEITAE^q}v;9x>4!`mAD6C{rnXSfJxP;C@u&I7{m}J! z+-tRkj`wbM8{&Vi#2Gw9=`AKY_R0#-Kf!qv1nm(7ZRU4i@wa;){Wrdf zfHeQZ*XTcd^$0JDlFO-&7Y}%tAo8xW5pVdVWp_No{313z?2+GUQ~@m3>}K+dLD-AE zi`MfwuRj`w7IcZ!{^ow*I6i_wpA``{fYjmsw$`pZn*L<8(U9E%tIeH4QAmWy`#jK9yL~iWo<&wZ*ghI z-z8ti%+?s_gS4Et=)bJ}A4Zt;Z=n8{5hncu%5>k3p((_T2kfH3cSK$+<13S9&=5{Q zO?mt5HL_`cfgt|)66jiVWU&mK$3RE<=cusZS4dezPn?%L%v`^F3C5c+Bxdfqcst(4 zujY@6R)xJj>OD1b6Y@6vgUw+MovTz3J%6veKC(P#(Lgl+o4BfXyZFO5KzTAawW|br ze|YK&6c5StUyNWK|VZACGPE z(HO53q8GBBhywKemU&%+vquas{=m~O%pBmDMvs*0xk+z1P69Q3hS%bf7}xeI_}|ZL z^m+f{?{5EZ{_gf4f5&X;j@BG=5_XFC1t)nKYTp%*Qj8H4;YQMbhYrO$+-FT{KhSuM z=;N~x$!@zuR>ilZ#N*qTF8ej3QDs2E57a*NOc1r`ZfMx||8VvdZcX>!`x`M}gw$w! zhz&*zM@lN);ONFNYNWJ;N{!ATF-qy~7D4F-=~h7y0kIJ5_`&n&b$z_PT%YIq{QiTz zcAs;f``qWeUT+H?T01^1wrM#o|7$>dwAhfaw9yL~Z_VMCi)lM%1zcsaUzE&Dhr>M2 z1v7nN>JoGU>0%f<>Z{dSuehr<(Ju$=k}%vN1~_Ve>-fcYf3zk0XJPYOOYEOw^Hsey z<%6>c0$snxN#i`P6;??XSa{DVR%g?j@$vq`dD6h??!wHF=Rex)WD~yDvKsQ9m=DBU z`l6*M>FJlgWUod@C|mbMSL|hulxIAX9}`}m7o=;rrkSk2vce4LR%jAbZ)T1#_oA9$ zPR%d6%ztmb}e7{xu*eUZBfYvsXE_`Yw;drecA@EJ;=8d(OMU9^Z-#%Ue zQdsq;!S?qqM6awVG=}P5qk8jH8l$xG^dgu%H?Zabg;9fv*PXouz-Nz7pPqP~*Q4$C!aYp>&Ql9}lI5uyZw%<#((8!ioc&BhkJRz_ouUZgip4(w-;_r4LVSJuF z^dmJA@YV0iCscev?65JD@b!WrgBupbuRl@-Fs2zY!@6=;*2*tM%M^UB*XfNMPcPDU z3Q(Jj;idjX6#uFJ{@n$b4`YdAoqwnsjNJ4;^fl7-&hyMg$heIQWrhOH7K-aAeZS9M z^sOW>6IVRvjFXzp4PnpvW{7_ohZuQbcw?dShI&e9;-_gbs`}hT0ZYB$VaLr!)Kf{X zaAsTB5$+?ZB&5{n6pMh;b-_XQOFgw~^z9PTYj^c+b4ku|x!x6`T(5(PAIKHFmONgi zNo+Fy33lHr3Xx2SRje@0ztCd2oqJ{d2qbxOFJp!d5_y&juzMC@zte5^6tIH!8oB1z zUo$Cv->*+ciI@cN^BmK)U8)j<+El&Gf~eeymzl={LT~x-Fi!Y8#*@b=&5}G`)hqEo zDp2=T&Z?_zc#?6fjH5{!0*QTo@Xl(~q=57@zs=o_lgDzZxk94S6+X>hVsG%54+Gy8 zQjwtT6pcs60@FY5b8v*RIB{oL-b{g&IW%nFgehFrr2Ol+pRFeZ-%lR-wtuO;stBMX z@onQ*x~&sI;)x*-j#EDXIUqS_rCS}q;BSu4IZe0hjy--i@e3F}p^P*~_S@IGTJM%} z5gFP&xaK{SBF(~90WQCWAtMQy6hJw~J*gb!%UJ__Um25|@3r8y;0!w_S!&rN{%w|E zxWj!ls8B#>^^xv2Lp?qFi{p28a?P8!oCchTzQZ;<?42Dp@VlFGlkgpT?cD6ahI zaxUiIa$-5qy^VhN@#cP7fk9v_T%b)8Q`+OYDL{YZ1RVW zsypj=zjajI>G1-eu~5W)RBsW=-b~|rm%?ye5S!bZk5`kO7GjFNFTN{BDNtuXB4<`j z8%f*v4qnJj{aRL?H%h@Q5YJZ-K!5*Zv-VN$)9aLX7vzbkl1s0Bp-ALLp+(-ro2JYX z8nnn3Pyl#G_TSnSMHd*9D`; zU-W)%1}AcoyxI!ODNQ?<$W*5Kt{z6&(f{DvFCF!Vg#Y)FcH!yrdIe0M?aN;T@&0nh zE%gt4Dy7ol=193BoS&>LQx4PeEPrC4OHQW#q1!D`Zn;TFyR6KIRKbX9oAbc?VMTT) zpRA)4cqxGu%zKe0%@dq>&m`@{=e-lD;k;x`BSLIYWJNB(fB$3P%)1f`7X{P%CH1SU z-#@{<`)raS3@=?j%Wd^ZR7MKuS}!s>84&?05Iw19c!^DCLG` zj$_3t(pbENN*_ziw1@pUZ#HjwzY6kKhIqYO_e$~avtF8Yb*X)$Uy87>I37tj_o5oj z>lze!F*vPm=52<%8x+Z!U$X9a&E_wGmdX0_HP*qQG^Jp;Hs0}?o-9UjyVDRGS-xjR zgt%h&{-KT5(APCl9@93VRrinlFVkKuPJVkf3&h;yzLXl1!FxTl`%6>EJe$GStDOzH zdmubAgcxAxP1yg%A{|7K9U@3e_U!1%gUPXHu}B`k{_kXqJ!O%uY6CA}j)V&{vWeUy z_|jF{8?})t#qk&3NwnAPVP~lyrmRuX^IE}+TU9Nenz79vT?`jaIGGm z8M%CLcbd^ZSv_`5hMhS5rSCf{T!W_s>*K17`D*OsqoZ)TBtn4Y6#!Rw?rd_ee4IOv zJ}jMxaE9Xecb_n{`K7eer4o+l!p12R@e=yoV1&_A#qEVVu9?E2H;&~gTJP~*@_#73 zPBlVrOAIX!dnX*F_(V~Q+&$EtpL9O&F6HpIsfRnJEActJaGKAa1{4*MBB0d;AxzF{#76H{`lO$ z>`d1h>AtwPD}e)?QtM^5fr;Yn_rhAP!_m*s(uY|8Jw8kb3jL!w`eP8%X7U418{`qejhU=tYIm!BHII#NZ_JjO%Ir{kvi{j z0nQ$Yy# zisdX>Da8>68JeQEKA)&=&9_r&KkCdDwD2o4Uh9;(XB} zk z0+UJEQ~}peq#=QT!!9(Ks5M_0 zRzdpkiAc1KG}e6{kS|6AZ_Z*WRXJHljS z#;6QNHgJik{IdfGW7|h~{}tkC-p~HM0pbmJ-afs%GyGus+aaejeQAP6Sf9j)?ry_R z{dc4AV7_Y9oGsRP|ASBPUm^n~i$StNIO+B`Gucz$4?gwYyB`EM;}SvHd6u~~`aEQ~ z*M5<*q}0T|cHQLogAAaWLEGmS$IlZ%B8VY*tvPWNCm-ZGD>kbDCV$i7r_|l8 z_{48Dn9ObId-aBtkK~QWy=M8NYdJp34^vJ&DCQ{m=Rkj$P<>}%_1lE%JH=|p{pI{|>}7_YgGa@tH@H^9bZ_3v z3!YTHKi+pTwNFW1Q)3GHRQx%5eEF{sS7*gF&-XHIip>sX?o;-MPu8C0$u7UV$^}Wd z!bo$DQMs>?H2m6k(ByaA3yx{8`l32K0z!PDrvnwTC&Q0_C&0eoU?}48UDYZLXQE~R zFB#>$8goFrba^rn_sa6dUcZYnX@q_Kmx)hyYl{GY-D}#OUnbNa0eE=Up?(M8;pw3! zo(I|D#S2}M#z5wMxaR@GH|@9F_gnc>uB$Caso9uZ8lL~r)!HnS^$ngT0f){PALriv zqUk9ql_qB{ch1L{<=kNYp2mzqo9(UL{Ewj)&DvsYaq?gRw#;u-3laiqFd2}yP)AJQ zmUjDa1LdRmxJH(?aH9Ia3o#RH^Gm!RMfBbRqPW2a^qteX$q9O^ukTZ5>(|Ox5BR>X z6n0b~0z|!r*!>E?KY0J&XR$i93{cIv`t!1@=0G*c6j`iRS%2Aa&Dv}!BqZYefH>sV zR%pe^hJ0{@X=_!*!PpzjaM={I6Nty6`*dWuiBlYQzTvr&(%BM)1=U$yEo zqu9}SCx1KkEB_uJ^45s!k6GdDl}q}B1*pFI5Og_{YfD$|IS;FyF1H%~`e(h$6s5ei z;C!kn-mWKRqqm+#9K+cZir!Q!BgwU;y-AdTr=~;GhB#oDPJ56 zRY2HfIK9dD967Q}R+3J4fNuCSp;Db(Z+<=a9G5m!@!6WU&K>_=;qlKUWLjIuw3XDSIQx2@PTIm#Usc^bXO%9AjA04KIVSnk~693o5LeWrToh|;-#p7?M_xm%l7 zT3B~M##84;*>j<*1qb8rU(}Q1_O|xTcwZlmE26cgYPZq<1))Z^MApzu*eb6yF5 z;Q>u8=RV248O6<>8baNVf$R5`!{X?9j#b)h_y>JS0qTd_QN|}@y91YIW(G2si|6qJ z4{{dx2PUsl7~CkB%rB(c_piSD%klU_PDP(}oZm`5`t&$2Q{O#(q2(^=F{ow9nqkv_ zqebM2&+bCYRq^{}xdww223_@GQZHgF-nd*cX>?LbU9Vx@^E2v4)Jx2Ya$xMCLY+(}CV^g6&*x#EjSlbu7eH4df+? z+uWl?7HxBh*UfudE)yR`bc^O>LG+bI41Dv69Ke0JHoa{I9h7g9^W# zh0<>-{BnxY;Pcxd+lFg5OclH9z}unrH|)kVf^p_g0Bx#lLy5Fb(C2yDdiKu-FM46p z1$ZA-hfq6&_{X8sw>f6D<|#;MxYVYTE!uBmT1FGYZNndx_y7=2P>CC|$t7-ruGx2-sG0yTc{mOkvqN#A(?665)lex2=&YIe)3 zkp1&lxKlF1XGuYP#5VxKZ5j!`sPI39ROf$VjLuWWi2v}(OY9TZcV?j;eIGhlx;*ur z2TXUbCYYFtSZ>_ssKxwT#TRE35+u#P)gZ2g3nmBJ$W;?Q1$7qEsq3Ra;rbjuIFS}M zil41n^!o3n6P)|)7}QFM4??zY1woFMqz)xNFsd*(lJuDI-FSheO-_Nm$y~ea*1bP* zNU{1&D8m8TcYW@muag=E>vEA`{&qOHD`>wTYJWijLkRVrcf94^Vax2dxd2Qrq=BDh zx&8aJ+*tg*2De5ryqr;|e!0D!WBlE->3+c@*;w~>Klj*^fe$;BN9uubc~N-q!cgUL z(TZNeHkIHrx=WYLoo-YtfE*Y!Znzb&N3j*%PPhKRQRVV6gvu%eM#_=mH}|jr`atPq z=?#`~y@2HsY>wc*2)?WAPHe*PN# z-L1*F@|Bn{0LXjT_$w;^Xq>nITjRWa+Bj1f4s)bf?zWj@MD;C2bgG)XG#S$Jh~15k zraWncV@sP7-HN)Okxmo(_#S5Rnm0Vvqu7@Q7Mf%YRC>c-X$xEIz%vE1;Ln(zoKuV5 zw?7&Q;&+Rw9xH@dMwGgX>&AV$*R=Y-SC!auPTuHAqEO+hmT|xcNT9{)1g%FMAFKqtsbqQw? z#BX1ha7qxBLNTok*-<^UcREw~hk4a2a7*ck_Q6sYnKyYd-xpq+>W$v2`9@lo<#|o- zgYVd&-E)BXg}1sdzA+FAga>HuBJ_3gXjg%LI}^4!u#p;{y)OqR9{~C=xa&X>zoTd6 z@cAlb%w{(O%(jVi*B~)y(lRY2qPkW`Albm@fub3|j;zm8THuehr54-X=MArwU^L#P zsUCTVC{4zfjLRS*XY<5hVgN6`pXisb`#(;J0^!qNePXtI-)X$bJRD{Bt*p2l3aLe? z%b6{Be65)GT1LRX%pwLwG{4&_Ue+;i8NU4T9sl#%?u7L25ZzbO>sP5`YTlb+XR9{s zP3G3*#wTO)nAoV)$}t zTIWIzjS?-nh?r|rCglfm<(p@$xF6V0sIY39AkGpj)(VFtU!DJwOh|X@woTB&zo%f{~%}d14&jxx6 z8^Ck+p`h&-dce;?rC(An{S3Q}`*}Ee3H#@{awg$;(8LXky5~ zqb*(zNa9&1s{t7O-T5b;p6q2Elhv)z{aY*}W>>qr9;PoJXEWSL6@Jt3P|NtW-g+Bn zZ`4*uDk{Kh$jU9IJKvP)J$QXl%(zgMmU3d@RzFYog&_U`tL!$Y%&K>kBe~0T z!gOQLI_yqD(?zi<7X~?IK~AHLo73a9oOy|RT^l)x`!9LBf6W>G(C&$6rPXiko_Ly8 zG}ComDf~3YT{DGcx4H(+RhMB;if)qM3gz0(1G-b8Hw%VoVzlWQ3PHs>S>S+Gb2S$p zNn4RH#Sn&cSLZ2;Moj^Yn&C12*_DrXA~jQ|Yt~e{i^Vl||GEZ@79*ZA5-KJ#tsZu0 zPcrey6w{>VKC-C{hADqUVa@Q>6)6kG`F>$-=fHy*mGL%$6^rO}_|Dvd?@VJGqqL~R=Yr|X! z;wzr&U`~(T&@U9XoHD*%z?+y~Dw9wFX3ZZ>-_PRiqCEHVD=6UBjg}yB>hf3B$6|X` zB0lJZYo;QPZI6Fo#So%>cM9kC09QiN=&vA@KGOI7(t&?O-oH<{#141kvziYA|B80= zzQBL(3ev7^_?n2owA}$ct*c9>;Cnl*v9Te2#VpN7Lyuw@FAbZ#{d9)~w*g*-e6fU= zPhZEwlWmf~UnBd;<~N8y8mib-E?+XtgotPzf*Q zxOaRGu%(7dSC=XVI`lbT2g|Hfn-^)GUf~q|KmgXQ_4rrh{UMmz&WgO>f~oB^@}MU# zGt70_|kBX%By!swX4jb1ki)PJ`1N{o272i z^3GP4o{S3d_}I_~JAm7;mm%5Tg&6L=c7DH_tJ#hK-%r9Q?Is~hsfc3L&E`Vy%MkFDp4 zvl8EG&2PLs=Qd3oJkf4gp-92WTG_TIc3!*Z=3Cy>>G9 znB>ctF(sjwjQx1}Tx*gCYpfG3g@X?bh3PAJ^rwpV8DPnZm!>zGh@du?OT@%vRI179 z=SAs`D5OWQ7F!cn=O!hv-O^jAyRYZZ8gNaYMH0VPYNk&~qRwdZf#cfD*d54@L*6Eh zf{@GhRLG+1$BLv~c^>lUkdVI$$kIv#_&jcmXl85we2QHPk(s`Jj!9dr*Q&Izd)>?C zwb)93vg>83t?28*R~mcV$R>PlUbR|d_IAJi(SW=V^_=8cs&PcF#McqrmN|>JV>YJ`eloXGU;x~8mW`2-NpOnOF4!Jqzt1)WvbEz^n zHD@fRxUZc&bUA&_$P{p2!=&k#!Ssh;_V4?w$}5lMyBP)0B&#=r)b7XE%RqNNIx0Cc z>>J}|wy1tqYCa#*wainXzEs{2VIFqS8})Gq^XOpQ4^4XUuF>F!Cr0m++i%LfmODVs z+eBsEX6gy3R0YXJu~y6&&=i$k`tnV@$2xhM7l7VY4Wb7rYd@=9wPN3B*kygx*B)iW zT$d}kG)7j`KebwhGKh+nXXtF0QDSr;BEZUsC$| z0XThN!=W^1*nDVdU-?zqq?KxY(_khrc^^(qUaj_QN-5Xoh0*=kDz@pSk0 zMD-GKAFG~wKdWXn-#C;SFtii_#w?LS;Nwrlk!tJt9$NCia_Ouc`Vv?Owja)5q$m(j@ln(Nf+2 z-yVhm&gPY>;FVp}*!;#g@P4rwYpGjE>WhejAwV)WtNUTnZUl;9kx4@K6KK=h_L}1Z z#`#Cw&ze5?WZH1aP9!;#9K^n{ln=3b;n?>yG17!xQOT~*+Uk|dhH68L2BkGXvx|Mh zjb$z;Q?Fbxvauoeoe=M;5=+S7V=pP~$q@#Rn{?@Kcwzz1?d zwsE}8%zvE^Gz%3PP3CznF5JTvv%VaHCDTm2Ij4eSA7&-Fa~&11`fLPv@vs-8I-zuj z9)ae)Hf8PQw~`U{R^wRq9Oxw^2j+pv5vu#TMMdx?M1+`2HjlDCJGm*OD}FA*PT5hH z=^V}X?@NWbvGF$}?6d&@O3G`zza#}CL>_Xd?&S6;2ZZYXF1_1h&}ejJkO&}TdSsgbl!ws#c`?x54vi-FIXbcf&OT7g;T*W`Z{~GGg&`GFiBQh( z&JL&I`b+iRXSr}=P)HyM0NDS!SSPL?R>{_l?gb^tYgoL&Wd zO$Q_njm6>Dn((A8m5k&f7u6$1P6Y?W(;iz(D_CXHV?a#A9Xx060GkbRcaVH_z4T=QJ% zAj!bjikduQ({-`H**S$Z3iLrs*tAjC0;B@zm$WA2b^M!6x!W}xC3~ZQUTaJ(&^*T` zOrlRo&!TK%BeNs#cA#P6!cdD>B)FJndMruXpjh!la7u;FlB3P3uhbT5<#izlK$s4A z4!lBeaCZlF2q*(s3N8BmQbifegYC)8MX2gA#jQl~(dq6Xo)rr=#Z`}&XUeSn{LOPX zcwB3gLBpbIue zRMA{gj;%`jhUv~KDrysPE?01Zr2ZxVa-c}{LU>%+ph!n)T%1TAi$tZP3S8pK{1^$h zrVwGd;{9|>V&!5V3%5s&wHkv z35p5St|q2!1K719yfD=;({Tz%LM>WILoBT_lz7AsJWkPzRHt@lrMIyV4{ObL86uR> zR)|LCZ3P>X6h&z-OgaE{ zF;Vk+h8s_eMUz`rWz02{SX*j^@iDpW;F1SjN>GMw?9mdNxC5vtdtQO{#UWx^WEmlf zQwVH)@|E54#llps)E;U~dr}Ehx2?KOU~U#}x`2w;+Xjt9=n1m7yV7@?AICBYb58cH z*3>Xc8nI(8kfz)DZKxI4fg+Mx3RK*M41tO6MWv21ux%$+mbRAX(Rbt_D$Tv6PJQrN z9$Go)hJEAA0a6o|#WBmFk!OM6HWR?E9l)n*Wb7a4bI2egWRQivp10&2koB{mp9G5i z4gLBl^jh|-?j8cxrQ&vkLJ?oYCrh}p@h0m$=SE0e z)5}pEpD01_nkzDA+cmg^89JGE#X0sW+E2R_M{23$s|j1U`V|KhaYjrQ;w zZ*`%ai`bk@j92!kfw{5Kvz&ZQup~KLUciwM+!$tPu&C6JM28~QOO?rjY{fk(VK5d) z=044pWo(_2EE~*R&zm8A&6v;rB43h1*`0}SKlAbi5B40eINZ|Z{CeY|!+jIbYB437 zvb&kyzU+3MD$2lx!WAL_wyUua)*Neu@#GZEG0`$CH{?VS!Bxhwim$3xCr5VOi^G*jag5Vg#>9xvMOr6+3TO|jG>Atn34-zP!qNdp5*lO;Nu*T46M%#VJVj8}Z)Vm-Q8+f0 zn59%%zd%-Z!Z=2fO_5L1u6YJ~9_f}x4bUZWs4WO=QoWXj1@E-MC5=IjR`bTZ81Dly(|QPyvnT2*&Is!$9Y3VOeu|7t}WF}eiYt(t`RF;2AoZqd^1hNy?P}?nvfwG z3hO!A$2}4~q_s5&bzCdAvZBHcUVaPz-hO=h2%~?v2qh_n`<@iO@ zJAhYOXp`xSS68*qWmnba84tV6gRNlI=KIFQyk?v}x%fk(wA%TqnwiSyngjzIY|K_b^cUZ+sv8 zP3+79K1`7bBhkv_h2g207GOh?1p^i{C!1=2T;rk`#ZZD?okwqH0(~WyX<*vk>bPqd zeYU0QeR9?$RlQB5zbR>$0o>Z8m!}}n_gQf_rLqKnZ zeMD6f+T$G&iwZ>Vy|hHnP?VIq7pQzHj}k13uCeYg2wkC%W>YKgDTo*dyQc+%>n>v{ zS^KvST=isQ+e)ItU-9)y#U;x6MTQZ0mNS&xqb1{tnsNxHsqRd(A7&Y&FRHUT$QmaV+!!z4Yc0$Xtu78h4CW^+`h+0(RAjI*T>9GPuf zH`Rs^xE%G$dcv|9ofGbuB`}l=a)&GOa%#jOjZx&|nFxkRHgv9=-HfZM1s1Moh3|J# z53>iq%LLGtw1@!)5r{50tJy_jBW;-BvOtj!G}j^7Sx}0uFg6jEyyR(pQR}4EYpcpK z@(3j{JaS>MKuJu1)Y#VNVF*jU6OF>A5@fcWwe&0 z-ZjbbwXE2Nsp6EO4$%%k2p*nTxvWhsyN;WmWb@^)HuoR(RPQ6N7Oi^MnOMdH%#=*d zbj8!t@p4nTQe%n=psMahg>6g)jhl2$R5Z`tyE9tV!T~jq9AwCMd8}b zp$Ld1XIE_@-vaBa`5kj(N*i6u+R4y#qIi@k)X4!1FEue&KBsR|K*JVJkbp}KGuo>r z>M{8`a8p083g)tow#E9jnov6!jPu%+>&vICF2oEg9RD@G@E-kQ0lE9No)KHduP?k+o{kL28nz3>ERCS!{ZL7&jJj z?{!s=g*2~`-jb_j=G4$UfgvD!(UE?$s$sFyNJgy0v8jzxgqyrA8xC8K=6=m%Ut(=~ z$R#0Km|X}m4{Q`GMJXb=6e#oJS#+qaJ`klTnJWv8O3)Ssv_!SKS*+AbIV!KTaFKhe zScVi*1-H-{3UiWr&=01wFrln1PIS9RU)AISsdR*aSNLj1O13&IV^R78u)+$sWVHYSxyD2L6Uy@sO?*PC4IWzf@Q zFEMN0%ZsbX3l51>{2mq#r5WJJoM=iCl9m-m*J$oK<>BbmdHX#Nmfb|*^eW(ei9P_> zgilFbmd$5#b-W}~3^PLl0I-K;{CIFiuK&A&qZca%!+rd=Vu`|&)8-<2RBvPGhOOj0 z#LIj9lj2YnbsLFsq6lW%1+=+f?I=-V*nq02^C{Vw^y>vTC7kv3kq;22@Q{&yP8b%%Stsyc&c_K3DI?< zqOma2dRTm^h#s;niacC39??OHH*|o<(x976;F8n~yr90YSeKOnH$&yP9tU?!ka@(s zAAO0NBr2&UX}4=pv^mqdiibT%y!`3q;R9z{-#G8LSlq)q*nWdITY&y%kQl+u4PZ=Zqqh z1W~2R9V@~P%0|w7@%@e3)h&{hSYZO)b{`xlAn>@&J$5fi3>^7R@rp@)a*nA7yH2ki zv@eY{P#8<3T-j~&E7v2)APr=Yq5Q~4r&r6*O7>phrN5K?&S|m}M~-r?2iKWi!|)91 zg2fOBgXk?o%tmWvSs<}1Q6Yx3)grsfPS^#Cp5>^-pa?iQ3KH=MKoin)9MMEk!3qfX z>40VK-D@`Trk#~)mJYCTj#QaJPr|R+G3l>h`%7b8&G{^IRn55Aa7SgCz%k369U5$s zWle`>Zyw()n!qb1oVM%Z0i%rI)r^gAWv~qLT|&MTGlrI%s$_u+3Jcebbt&7xF0~a# ztD*^D3vwlj`Dq!$p)tz}+Rzq3<1*_y^Gz{z2}%>i20v|6#6VF2zcQq8RY_50768|+ zrh#Gac~91smfXujXh@*sXviX>iK!~J9RP|8`YhPP3cZMCZC+D zauBf9&=MP7m`#c$U4s%~XIVOH@;O1{e43GfVLr7$zZruy+_Gz|Sb?RC?y7{HGam1_ zilpW*tCd1(Qq#Fw?sBjU*kAEXggA68=^>Q%gROFgXwHu_W)vcCIQ+OZ8EhIR|8JBC<^L+T_g5} z?Lybos**{fOlov)Td9`^{x(S_l#7LlyP2A4%`s+jRJ&xKr;(mIX>yvh*1?!ItA!F< zqRGKz$j7L^t$vJ~U#~GbLM!5Z5CqBl6&vOp*@7G-xlfE%eY* z^_Q#?_f4xkix?#euC;`>XhT}j0;wxeh@HX~m4%x0WKlS9D?S5~p=~`FW@9#HQjIRd za;rRVOxNqD;BX+Ac?8_HpqF2>N$|geM6m;{bew|)hLqOBQ0PSO^$EYQ9n--BuVPcV z`S{LxXpV&(ajRMOU^6HQVv4jQk$%^~AW$=z5a?H)O#8;8rzE9Yi*v1N^OtpAc zbjy003<#TxL#2uImM`!zs;D(&gGRwmE5u@(jsMa zks3SEGzaA}V#;b72^i6am6K+`3fJMUM|mKt-DG;u{eqp6}+dC5Rkluq7itE*uk2 z%{I#4*x^k*T}CIlU`J^}uH?CvuDhvQG0h~X-U1eag^Ssfn&|kjdq88EQW;-33~XyN zn9GIDQOsGgUWD<;{VMN&v~Zng!Tr01>pX=UqlWb{<~x+?RMul^Wsg7_w?ZbB7&(VW za@0&x@yGLQ-jA8GJIBm>X>E;zx2e*74N~F~2I(q#rAjO+OyU;XA{6(=bYAgbgRXx;<4hspvCxBqSFg z#%n=M+F@(oD%pn^WSIir;})V{3{POX&71mO*IY-kF=wcf6Qiap3bJW$3WpOS*!_tx z_TlBxE16B9#$i0U2SvN8jd9&+;-*3Z!nF}r?)vCzoJVk`g0`|Sn{s(skz#nVc&bNr zkjg}*-m8GT?AgP`87GgT+q+K0$~5egaJ>XpZzwWt6Q2ZS5_zR5ps=O}^pbK=uUJ-d zl%I``u8WWp^0Cp+IJsx_Y(m__)u}z6iey1Ov$Rmh*?`?g7}dN`LDl#wD2^_Resapt z)yvk9jTknPSWWF3#tr5z1-Xd$3$@X0E3ugQr%f+s3x`J_wOHYN2zCd0To&sjlc30W zXAZ0PfwEHAaJ*o(3-Dr$mmT^!p_7_BK^dJ=XR#I_EZ!_y`-DxeVs!$QqsOO0YLs0P z%>wYtZCm`+!u?U$Gta{PcVW*w#a*!@I6ex)M{wXv#o^;Xj;fO+?b&M9;4Rg1U{xIp zs|cDpA`)m~mqWqJ*&0Bh6z*^>FVa8GJVi4Gai;^QvS|z=6C~#JO_It~nYGI%%gCmc z&I3^}=6LlwNjj|f`4i}j`T4X8C|!LKmmgCc>OMH-34Q^N5=obH763PCU$+&?c-+jcZ!5;k9AuG~hK zhyoUQ^d_b%Uo48U88Bd8bLwcaOjFHI%^}3gcqp9{kf5jpsJe?;k_fEgbU3jH;)tFK zl2z%Z1{EhamwV`TFgKkd6y@b23j00?LLh_W zz3X2+wPMdQ<642*e;4-kQ#0-;N8;!}J4!rS&5)0-Xc}44%YnIu+EQXss1RbdgPrUu z_s_IhLYsJH_>73#3<$7oq#fN ziAH#A0qpxc-*&1q*u@Aq^b8K^NvtDAjT<>)$fNUJ%yoj#cZch(rqVRK2H>elb(ceV zJ81k*Xp#;_SL}&<7SVvse9ttct%y07}#J_X>mcTqKd)2MOC@6Vp)X=pUz2R9#sp))VB>~c`+Dv){wU7 zphYOvHZHj3*+OG=t2bL!^?Hk_xrTCIayFQ%FSV2#yQ5f2Ro^tC=(0%fX45~Y80EFY z7LJIeE{8(gSNvO|{jy0TFt~}1iiSO~$88{_;G$RSKh^sK2vR}@xg9zC_+-}d;VhKz z16BV<`NJv7aC07Te1TE}G@Q0jl9oo-!rwrtlf81tf7}8h+?k`^v_~~qA|339q!HFE zNLB_+(qF8y8=L2X_SRB0i2} z8l>u1SSGmMr*kRmB6T03tQR~1FU z>+`yCLKckGa)#s+{)8Mm;XWV7O#&q*h$32*T$rVFRxOG@z?vr|w9yU%0xOXNv}5y# z1Y)Ki&wGch3E<0V(~*)c2H_1hz1A9T>at9c)rsMX4Wt%RrV4lmSc!x=Q)Q+AC0&4} zswkJjXtajJs62g`Fi-37RI*xAvV@4Y2MlYxMD=3iiKUKi!JuHZYVCq>r5Zi22~3BM zxJuf>m=$Y!Z)K6E0?E^)Ets6U%QPl2sy>Yp!wbKYFl`GH5OWiB=qxkDvsHC_k#=zL z&L|7Z^!rK)WM(ZqJ^`i0VvQ!FADk9$z$tMcuDd!5 z32ADG?=Vfp>fTOM2gxbt6qSM~>Ibo&MZHe7atI__OGCa0?SOzmW|F70+-#e4OT+BqMF)W+QmO9BTK=(EUlsf@8pJoTMGC{fL`;-@G6i z&5c39Cagd-`BXYBRjEn2Jb9xnlEsS=*r<7=zE6QEO|6VEXN0|gDeJyZBWz!Z9ssG1=jIi@MB_0;*1uiT}x)e zS|B)x2(P>x!lreo?twP^k{C?iX;(+>qVAw*D(`XAq-VSr?8I}KEH1;1kVEzuO@~4m z#Ys9pwM83?OwL4){xF2M6M3)&DnrOwRh7H(N-qKoVSO2TUdBu*pNiQdoF8Wfd- zCjN!>AA_{zv+DJ4%XRs*UJK@A=CT7k%BKw3CK2X^MQ$2SS{V%%zWKUbxmy%e?9`kX z8kBT4yY}#!a0#HD&vmvBgH>*O#Z#I`qpx^hoOZAuKo4;{$6`_K0ryhnt$)u|eK3}K zpw3t90xXaq86$tBv%FxC#b;vG>&M*oE{|>!N8Pfj#H+Yqe;bUfF03R8z?vf)!YN#> z6%zzXshICqlFyyTg-NmqR#CLC9>}EadbTh&!w@TeXn(rJ*;KOy2KD;Hd~EY@L1+L% z1`e2`COxoor|)sj;pqo z)La^!C@flA#b$MbEtACF3lfQ80jNOL-5{~4gwo3QpeuDFdTyD(J*E^Tmy3oBGLo_C z`K6jRC?|DD_3hYBbQ*w=jv%y(7T*JVkh3tDu@5TKJwK=!N9*~%q_aVG&`<~;Y+H`F z`{O~lCOQV20@OY6Ant5?dYi80X%#Tdd1)@RZ5G!>XqGMA)sE*nre6 z9<|R&tRN=DdJc1}r1oy@7KzGMKHHN{lw5SOqfFX~bTV071jINrTaehp+SyW#RKt!} zkRICbUV13n6@@I&^=Deh&rJ5pD2moDw-t(*-8VKF40=ZuL8vHo$``D&1kmlt=u1wl z_CYAbg1E*U`CJuE62we9ksq20;&`e^<=yP$y%RCM`*1sD=>i;IhHIBQmyUX9W6kvQ z|3}$ZIJKd5-{P83+$A`HKq0ubI0Ow4AZQ^tBsdhP;4}m&?xjcwPI0Z^4#k}oFWOrQ zrQW{Y@7-_a{bugGncv@V&faJ3b=KNTud-3_hEQ5s1WfkV?FzP0GN`JDzO9B21H_&RWS&)1$WnVwKj< zDU_|EsSVC`m2I;3n5d_tw_YM{uYKGVhq`NNrC20Az_NM{pWaAoS z4^nbW5?RC&?Sk1CNDP{5I%I9 zDDhN_N&w@dA!?aLK@sN&3u-VI<0RM&Sg@0dwWGe08R_CkP zwPFsTAlqr9!h-20#!x`3V|~IUM3Y!Ba+8c0r>!u;N}=Er$8eUJVXH6=nQUFL2<%1Z zFs>LZt~Ck=LKuTITqYGHy-}F3WwpDqapN8ESd$l2xgb9gR6>PTP;y3t@W_xztv~}+ zgx>Nxu>lR2qK;hiSgKML8tzTCUBGm*X4yZ(n9m<7T?Sm#gU+pslOtpL1)7jZ{$}@i zPB|i$jQRb>MnUG>dQBnyp%0~PLAH@mJJy=y_K`Y>s%GPK$HFvH6}N`vT)|O-p^siJ z!edp=q`*M}$79)E*H@Vl-%baGTc^p~;o+EU(r>_{L9&G`#nk za#w@IbFN!y*`+udDi8op%L;ov?gG?iKzJkRP6+$ZgoLzqwHq$Euu@)VBHN0}`9%ks zOT-{|49wy;W?1N>C#W;mT;sTU0T|hWFUg!p*_|+$dLTwRm=oU3c(;@jbx8Rz5Yk;2 zEJc;TXq-YE^3ir|4hYl0E!a-7oR+=+XSscwFn?{09+a@u5h`)hOK&z_As52>y3x{H z3YL-`hZ--Q(L)j~u4~`?Hy-Zf-%k9$C4chwiKl0};1_&IbtW<$m?cXggIY87C*I}! zfko+J=@8xZ?B!fpN-ZlKrm2Q@4z|17C+#+>V=3)QO1sLO$4+D1$VdoK;|eiH(lwJ! z_VIJ|qm7f@r?FFl9_$IO5}Tal^o8s#wv7jekjAX0(2c3A<5^aXCNTnZFvSn5;_n-4 z5}%WYiz+61$LQaCK*QXPC%_$}(CHuC#}JKDxuQ@KE5Bt*tcj%oJb5==v=KTbEKD$otLI>bBxSJM*=3_Z!|a<#+alWud&`IuYijL2YC_g}hE}T%8NZHt zf{guOnTfOmy!;fT19;a=V*wRw9y)*zwC!1BE3RyGdCNvm;atJ(6RZ%nds>psUU`WD z9+W)5b?}ahb{-i_av)nM&})s{(bY4cL8y2_xR~J{i%e0DW}l`|ReesIhJ0VUPi#{R zASz2iNvu#;n`{{1tJu&V;;bRfwL9+XNX@>i%&3^)Lge>RDdc2osn`TXC`>)g23@w< z(jS{Rj75kn&#qcy#U{jn<|>3T9f}Obj3Gk&U?idiELGhAlN$;YN_NW4AXnIs&%n#N zR<+a$yg5Wt@vK(T#>RWTHclC7C!*JeNf2P}M}Qd_oQr}oS~$5SNlv+`Qy#0ZF_#jW ztmY!;vTn#W2;s5`wNhM~Dk!XMK}@;(#0rC_MkF@DH*Lq{C5R}N4_SGQs$6v{Lg3O# zRE#m!I*lykB(qRCVz#)%C4@EFIl(Gb_jT#gR0*TIfd(pDO}oLh&Cf)v!4R_FvQ~ZX z`_(_HssAy=efj?Z`!9b3d#)u9_i5-K!EBPI%_-W(ZaKXzrKb`SJjQCQkhJJmq1X~t&gY*4Qg zH$&LZ#pobSZGjXk6%@gxlR6HqnR=F0MY5Vq!>rWPRG6*6(nT$jITsWnh1_U8N{lI) ze2r76KYX)xt}ZXPkRak(7gEG*VK_-x1_Z+25u21|fC)XpLKe0v1N$8)tA0XCHyuZ= zu?a(2+G+dv1R@35S{S2HQiyP1FDj_RqUf_6;{hhykhm%4t20;)lpts)#e^8$pkfT; zyb;_!<1*;OFPs-2lM54G&}~U9%qMO$uYze473CWLF}K4lvm{qT-OxGI{n>Gv;gTl7 zvSNJ>^W=q-9(s9(R)ubZ4q#nJ)=b9bRiaYiz~k~Oyarfip^3;(OWYcC5;SwRBrH-z zXKBOYjKx@$fw^5}9DNIcumX&cf>G>6wv`lssEezv`yXKcKVJ6if9v!Aec7}B?(+@e z)R1hlbdJyx&q67J-k(T}5L-P7OCfHa&L(alOb(nD9JsE*6wq%GJBV$Ho^sT2t=}sr zJNBVboU1ukG4Mp!?9<_KZlRsFMvKf(atj;_+t_m~HcHEh^vjyybD2COF>?TcNn1u_ zl9r|!c$0tJFi%0r)j4%3fq*xuqd{0CPH&?H1+(fiH73X+SPLjDRfJmR=@FS2H_Mr1 zPvF4rCoYLsrEJmeJe|lWV}CB}F1f71boUnwG;t7l&|@#~=-v`Uj8~+yZW6 zqgCmVYGgsuL&xC$o@#W97t9SP!fUnyGmOaqvg^Hr;ds6zMjrl zXiEL_G6orJ*|@HG?Rpu!@d;Xlra*usrP{E;+R19BjCtAk9Bqe(Ees&2prl4Cs|B6j zxZzR@(5&y2`~V{v1%rhleV(_mgag$)EzVlohPtvoGzl2gW*{4FGm_lT;FO1VhPm@r z_;85JIIE+;$t1W{JxZ@KB24v2U@r)!A15fNz=_~+PZ=9;>kpms$XQQf=4TRg>pmn% z%W3BYkXM~30aheA%ODR%gkMi zJk!H?Y`7zLJ;0vIoN)uI)KSQ3v^d2ds+UU+%Z`>w)IhbxR3$_!8tWZO0SsAKdXP?l zumU$+6!a*v`G{Xvs4zP{r;@Yj6F%C&J&cp{u`Rolc9KE1hh-?cVKAyPZG(V?rPtGN zby(k9>RBdrG3Ah}jOA|Pc|janZe6^M6Oa!MJ!Ypq6KWkq>+H#aY4QQ**zH`BEJ;o} zeI?rC835I@TFTiuM06&@y`jP$5erqz7rH~`TGXiIB2j(!6WvtyD%&_TS9GXeo}3H} zZv=X@I8ePfBqX(}K$|DR(=9S@j5U7b!l~F*uVdr}D0G(bmPxj5|3uarCK$A#!erl% zufxS3ok(*yyUA};Fy)z86%Vt6CY+*|kL7As<(4F#wD3+5*RVfZ9~jszc0*K=ZOYBq zvx=?~5#?EZe)Z2Nhoxlcpk%>!$B+FX=*s`C-iL@)|3lD~|6T7FI;Bn;cQ2m@jEo6b zi7&A#+;G-FI*bH)t-@y@vMy>p0aP=wmMJb8G5j>c9r|AN+1(@SHD%iIK#d!58MPQn zeHUP0(L9C}C@pA5bIBO7#@G-@o}YXa!b2^ERPh3dPBu$hDReDgi1kRhGU7a9$Q=+# z%B4PVLx8rhc6PdTLHyS4A9i~o$%AAe9WjfO8(0>x73vp?Q?gf&u+n-YKp_+$y~ax# zBBgBerhcyD1W~|6QN0TzvG00DPpRP9RS@^mJ z8dJ;~NC2osWsbE9@X=a9P}zc23hVy4T73&a0Sj@NI&l%#Rc- z_BkIRvp`>>avxuSoYn;VUXb`4QJF68FwxXASsSz5tW3+rZo0!9S%K*jap}mk8A)r7 z80I7-57K06(WWykdgn?yO@VYW0DA6v1}u-@meM*f8gemvx}HW^&x)dYNm_C?I}r*A zkg>Qm|6&}(EuL}3+nwCkI5v>TbzQ|!3?r=|fycy$00(rn*?k<4wqd9h3ZcCA@$}Pq zElnXem(5ucA|lvi-ut2d%hdfZg|NWckY$R{I*E#_0RWLQwl11|ZoaRd+sMuk%;+QG zEk`hb__vGc(4I)?I{FLO>n4;=P=OLkT8I)k*ogJGvbsSip9CrnHy_TljV3~777h1m z!3k?3eROB`6I&S9rqa{?#I z&FODhFyG7-Zsx!DfkXoX7=nE^Xqb8##(Z#eQ0h=x`JYU= z2y(X`&935hm7PRq#NqKVljyTEsDcu^vKf7wGH$5yJj2v3ebag%D@0d6{=hOQD2(oRw-$ zlTvkgU#VT$k{;X;f7IW)YMt;Xy_mwg1Jp~^R67lIRS^5rZ}KhWN%`ZWwdV{kabpIZ zZHaQY8Rr?sk@b{iR!3CxPGYW?1;oR?NKU|xUq`nVxL9%|6dR2+7kfskTqcdwYZB$> zPzG5i2RNwB`7qrK1!A)bmJDqb!3&ARTLw?;#lz|XXuE)8N9%N4!@h$J^$_Poq;Iuj zQx-Q{d?}rjfv}z`Nfgr~PH}PX7N~11{IR^KT%3`q(F(#AB&c~ZRhHo9EmOgiA5O=; zaY5D;)&mn*@}pl(AM+N5n7d!-t@7!?2s zcG)J#Epf|M#?&S$y1b4#3UVg)0_Fh~5kq&b(lXe|tk)5P*HR+qI)t~$mo%c?NPT5I;g0*FJP}`wwR(Ku3A8*g|4lsNg))$&c1O1-L=4*=no;A zG>WC?#MS5$0Ws?}Gcx~R_5Xq2O8>2&{};cN{@u@+4cCPk6I#VX@+*nFh}V;}JE+{L zX=I+!5)BOC(t_o{EUxa!8|?h5vc6Qa3-mY|XHc1yAWT%3wV5HLQbzh6Fx_(j4ffQc ziUW`)>>+&qa12cPTs5{y#U{+iWQulG#T=`w_X4SFa*`R?|STz&&g#*-B;AC0VLESAmhu$hJ=3=!Gdra3MoD~9;087J7~>Ud3e@Bu3fE^0HL#EKJEHxt1Ah)l(pQ>9S(X@+^JhWG+E+Ffa(f$X%akr@feB z89cD)!w(D6W#y!bT^DSRE-(-_?;|cOYmix%tmc30wZ}qwwhr?4B|qtJDXtF!Qymo{ z-u#2#{)caE_urP^tbTI9S7|g!G@S{zXsqqt zccNMBT?CTU-SSXo)CU;U1abH>2joSd7wf~MXGprcFzM1>^j2#q&pj$_(o8Q2X|Qp5 zAr$LpaVX%dZNVn=m2{#7pd5&4_XXo2;HwLH^{1lJRE~F}!BA*9)!$ zTiOTd?dUq$#dBMaky#06m)qv%=uT`1t3qRs%iYBMtI#Fs~w(swGHSk?nldnR!& zk-*j?gSjN_<%!Bzq%3vr@^CkE>gbDa*v8q2y^#*zTXdNoft?NbJi#@~=8hbz%*uLC z3ZW^@4JXM@!khLo(-m*nwz3-Sv>i7^IF1E))2p^M2PI@10F?4|NG7&R`=?W6I9!=K z3e{9Q+aIR@r|IB>3yqHq97LHsQC z7-y`P&3mUySvJgjrX1N@!F`46b~g1fvrc%bFpBXuDnsEdf;F6-S$`lXxk<9uR8F0` zL^9ugKd3Y&;xxCMQ*Fvk3#5}WD=~=(&}KGd9x%$D09#%qz(Y00G4$pzZpGvud&O_A)N{F+B^o7g;PfqR20Q zB0V82eQ&-QWN8kjL(tUZ&lKZs#sZy0++yNXz3nD5@1e(s^NvV}Jan_ixYs z$v6IwckzF~_3D4S1OJ8V)qmdsBaENOjwz5b*9}V@syknwFiR)a`$3Kk**@!(g)?28 z8D1tNZr};^QeFq?OsXlGTmi)|LLsik*(Wg=!D>LwkNu%hFCm% zI+mnnZa%=CZmSJkD$-UzWGzP~$0FfcM7_3^$sgls;6PoQnWwqzU>HVvllYO3z`jaU z@!)ElOZ9rOq;`4_Q!yi#q$1pXRkPTlrWh=1(Q>tJQm<_%q--b5WfI`+6K5?*rFWx2 zSuj{NiP_3_t}ikS^g4ORx}z`0UXm>EOyN(^6MiQqA0%bE*e4{LW>BvpCHJTCNis|O zfT?K8l!kR{`zkqI8SO14M5g8cBu{JV)bV%4c^6dMaWZm-2<=NclACTKbA$v(jWMcB zqS@k09sE3|yavy#9m4wmSi6Ae(A3mGnOXc4xt$=bgz<}ZB;~49p+#y}fKX>9yM3hH znpT23YDWm@%{gTsXXsI^H&5arC2z%|mF2*bAFOGvc|2ldiEXWCp_`r8hi_}$$#p8;O_t2WmdjP@X{{(S^!tPF^cq z_>}z=1zxn8r&f#1?d=}qmW&1(^0vdy zN`+vT#R&uCGIjYefqbJo9nj~kyn>Wol4eUMF|_ZuTNuD0`Nvr$j~tNyb=R4rC82zl5d7Bl;v(S<*^3=OYf9?R4sLlo1{6ZHH81j>)u2WW((XLLXZ z^x7Y2;*z#;=7sZHUamZxH-LrwA+j^UYclMM^C_MTbu^t4+rnjJqNaPXhB>Jy%6wuj z6(^0GGR8f!EM6r~8fI64CKP%gG2J7T!XqvAqAu16`g}g^!zT05WoKmTzKEt*LZd9q zzFmd65X@j~C)b#ZOOW8VQ_-m`Aon1r_#>T22d|1RsoQkQJ9 z>42l$XnU}nL1F15AuSdiaf}>EQ2|m^*gl05GS`x%XxYfkIOEo=K%B5J5bi|iFxMA% z@mi|siiV>`mf<8>2$Cs^N-J;Y79p0vgaNeNBt;xQzj8W&#foDey;Ba-f;Et5g@nEc z5Zq><%_Q#=u-VK6wjnmE#qnALc#6boM@|hvHwGzYOt~0Hw;Ln?#xz^@>x!%p?PWq4 zydjV&VWK!gdRt2_G6z>Ig;+nMjy`f~I;tZ#LrlrTynH~idZNWQiNmB&7B(t3nvx)5 z2gQ+Oc_;WVALy&t;1G`a)kg@U)Z(vNaXqkMju=rq!z{gdXVcIU*cVJ;Ui5)k zoLo%@0ltS^S^e=3!}|XUniK!+sQ)WyPW=0*o6oCh$N=$fX&GbLH4BZ0#T}$L{*Wg$ zLM|ejofEl#)FosQ8Vdtj@X2_d8z&v(KHd$bj~FTPs9j0J7@%1#yOCfP_>_a-DhfYN zpZdf|LVz=Jv7?K9*o|=|uvX@=k56x%oAqW&cN)~!I5TNLX?34o6(A^X#hjH}`Ue5G z~k-LoqKT_&3C<0W^EbSFDM@OC}vgaloR~?gSm@uzvZ#Oo@ zK5Ofo5jG|hCEZ>YM6Z|OQ^05m2j&FTwUi^;aR_Pn7`GS0dUnFQ2zJh!6pNuuwO*5j z9Ks6MaeFurTK~y zw^%&k@Ar@!V{rf{N)}A_HCkD}hK+Mrlj;X~m8R#`?y|CPX_bP>?KCA5@~z|I50dD0 z))NvVx25&PnhJ>Yx;eZgJZ!>}YP6Ph%$uMgCaS_CZ8eDJVTqiT4b^6aLLhjzsp>$vGyLLg08 zQ+a7FPfRJgWp+1mrw!|#hcVemX`-JpVCiH{kwJSwu?sNj*df_t4~~OM20Id8uG%e> zw}mJLEEk;SiUzDn2O*%xeParwjaW`2CYQce0*I3-ogtcvdOB1L-GmlU}AIH|EQFpsZA%p%<)Xyn`r@r2m4(Jm_^nrOVVzLsB^3}`(zKyFS9q0vMEJ-K4f zgcC@)4h>~vEnBXomqQ1%WH?3A)q6~7lS;Q`I1mgC|!8Q~qJSy2lB%&1ptplx-O}G|SniN}{4}d0hVexHzMt}H%Uxt%6v>X) zlE)=(iUZ?&N%CtDRsYBw8;MyqiCJczCU*RZ=PM@tmz5GWMAiS0*|!ZBSBLn?PBc|GUFvwwRa&E6Wno@P6x$sy3JWPr^9`3)TM{ zoE~boWw&WS=l<&FwZ}*8M0yY@3=k79gy(Y(YQEWgbNhYc`)kzMNSLVfjh5HFD&J}> z)~1$>*TZrQRa8_gE?&QQ@#6Q-Q@&SoQQyjoJ8x^tpq&<8ow!^g(UZfNZn_77HVuk{ z`ro>1`SyIvklBvNbDwVww5N`aVY+L#OXhsFkG>>)kd?a%v66Rpgba3^*X`Vme7PQK z)q8quzSM5g-u2b^_6#n@58L-_)__G}lBW3cJ9FB4DT6Z-*X~!NwGXJ&0SZqS zJm1Sa1w4gs7M0l{fE8tqtUsvLl#OO*gYy+l*J?h%Hh+J=+i^wV8KwQ0oW5fts#faC zs{3_JB@bW6mA}fgbRXFz`Hr02`T z@>r(ct)?0(o!RjhROvy+=T8jP%<{ z`mxB@RgI+kw6gDrl3A)QB}Ueojwf!709J$bv7?qxcR-j0wxXaHjq05OE*Eg%okZR5 z2JC)$vJZ}hXI-e?&T~A{QX2uM`40?|@IHZ+Q_;onB-DIHCvw19Q9sGetMo%R)O26? zS$%waFYW2PW9R-4bOM2$$?CJ}UUHy|>(LO&n_AIr=JSaqPLb8EvHdQ1*vfjt2X|>#1N*cd~4D%c1S5s&f(n4#$jT1!y>(0q1ZglhV&r6@jRrA^IMq zYq)Uf)0xXUpWL8mZ5RDFoONNn!*B-p%OR~@%jp1Vqhd-87Y5p<@=T)%Y^1Vz6q}vd zk7!)jG`Zb1wnWe-AyogFt+m06?|m#4&26ueOOv`8H3X&Y7#c!!Py6q-5EiYk-33;M zWkL7oX{wm|mJwe*N%{h%VH6$9tJcSI z`^n5wof#x?H90%_n_DK^m#fT$s9muwG}lDq`-~6BjHX%Y;DK#1%ksjEk8$bLgm;dx z6p@{3PMt7@OG5!ltIXn~FFeF%=?#T817v9+>y+7QvlPB@HTsyBOE=2gVBV_?5cR!C zu8WY`YWjT4`YtO^{s8lZ)cL%I=G4ee3R{Q@>D^zj11Z?*5o^@?~7A$3m~We-6_3l zTG==6X)?vS8a>NQ0%2> zgFh>;DLAR(tLa7_H{%IMvHd2Bhu1zj8jlk=!&fhsEBhGF2 zJX}>Ke|3ku@>3B!Xj0ib9x)d57Yj9fd!#(*Pv$Cba188~(t0ePaki}UR|sZpPlUz0K4IokN<+1SbU!A|5xM_I1~*K3uG19*Xg zu(%n(>WEGMjOOdAv_wC}V-KK=P3+Y()!bX%aI2@ureVkJLv<#qZH|^}FvswUDW0*G zm2%l>BR!c%n^eY?xBl!?kcv#|Ok-)e<2D%<*(@>hnDgwrE3HBil%ehLf1;?KA3$L z|BBlL(qsxOz4FaM-)G2@BdRCK1Mc%~uzRTwkXM_W>hT#;L%Rasm`iFu)t6~3J8}|_ zxN4ox=!GgU+gMoW3AT)NkAKq!yDqouI|s)-H8Jtvc%STaWsK{?I|Y-Q4Bztp^c*uB zuG4vY+d|NLDS$Ly<64{mMK1Rw^9go3wl#ns>lb^VTwF|&VE4F z6-~Mw5t=#l0Tmg?Ar&_Fak-H26x`sI z>++pe*2@|>|L*hTtIMR*_9b4$cY!!PM#HvzQO;%Vy-5JCOP5dV{UXy(2<@*)6EL|Y z&oX04v#OJkw|vjHS08zpIHM-^lOA#>AJ_oX?W8IzNxs(6m^IEyjo7UjD^B|FWxa!q z1&xkfK6V&Vla~{6cHvX@Y2Iu|k|Mx^oFc}*e=+CG$`noi(s8kvCu*SA*_0fwLt{|z zrKlJwAunPUnA_>TJ(w6_V{N3*UnFaT3hKyRzm9y}C?Pg__fnIx-^TRTjK~Y8dZsdB zRb+7ERy-B{xu!hUQS4OIMW(V-nD*@jIUjT~GuPYjYs|B?-RSgLDb-(lafpL;w;I$=qv9qw%f=i-OGzCr4E3<5nKQA1xJC){+n zVM}?)bR!RA{-96grSVi%6O%eGuT%@vH~6x)(KRPAS<04F^J?{!|U zNKdX|R>01?HX6CmP+(n4t`M=J_4GaKN48Hwv{j5rj_-b1f9yL#*Xw+7LJca4@)?r4 zZn0JAc3Hj3N~4lhvSf%qK2~Dlx2t&3h!cnyLCKcVkYi07TIS~e<)@pQh^D-D*Tjf>ybK7#r`eIUl`mX(Gq^x zIS_o5&Q+qAMSkwcUaJ~;j=P(*DJ$7)&AZJ~^i5|%gFe2f_+Y%pDpVw~rs~I)b`VwF zKu(Q6!g6+E&A}8ry_{3@^f_U5pf+0S^VMV7%{x|8XN8V5fXDbRJ?oF(2i0sQQ?R!* zlR*PiR(S1+h*DyT%^E7G+^IHzT}1&8H?v;-I{Kw&CidPbMVyW)6?C)ajw~DTy+hpX zhR}2y*YhU<6Dd!BO})Q;oHKiKf~|4z-JJ-#Eb0&5c{7<62UOqdgl(eV=xgTPA=aFapwEo>_4)udZr+s^p&v|&A*;4ci_UzF&ycEoe7Us9YFIr<3NW?5v-A45Qx_IO%`k=QaE=?ZqfAlrbO$8vH6Q<<&)&c{bQ5 zp-10Sxj4%e%HI(lZ{t?&Pv zFB-YjB_{eG_Pe%{m}N?iC87Oj5x~vT^6%qK z&`*~6;AMT`L$Sm-iC^4x)1BWC2*(XX(QZ~$*zIL^o9v&**Gg@1r#8BGp8gPg-|LGD zYV{r+d1e*%{zX){Wlrc^XJ=*98B$)`zTR=|Y8VqC{(i#$<_|%~jnA9DdvDiL<-@)Y z*)(@>_ANa=IJED+tL_wvTm~*le)d+q{~Y{#Ww~)R)p059jd(K}UGi1`{o^O6--~vS z3NcG($;3q;AnzZ_Ua!`#_skbP+}QOE{`TvrDb_asFvN59>n+R=_TTR=yq$M`pW=eH z()tUIKRGUizv{Dn_07`{7i#aH?D#J6;drHglkfvdK}Uc|(A*F+Sv}Y7Z;#A$-(6oL zeKva&KDZmDGr35R$UlGe>Ge1I?gut>rVe@cHQ)9v^+!MZ(zG?<`=%ac^SIa1 z>eVo+&+y$ZZpXhK%&pAu3}&1z%ttx;`H0yX}+Y?&XHB&!@Yo$+daB47hQ=D zpGGqK4U5@!FN!JHFr5_QvHZL}e{D}{0}xbk?%Cg;UI_3~CDGm75w2{EX1SQqyhvSb zAkvB=TZsLoeXLWC z(#odMFiLXBXs9Q~5}0#cN38^Sx81*Cm?WOPC928%qPDm~gm%vl_9&ITb z)&V19OXmA(ZVnKPPCDSqBr7HnBeL`wsZ(~|L_^vJ@opF2fSrnqd`1eJ4@zQl9q628 zc{9PMjX&N4`oApj{tWDzk+3Fx-c~Sf<$TlXvB{d+;owQbQqIIw!$w<`Ek^!F!o1?> zHbv2>53#s^qL&ENvz+}xsXLtg{-FhAcW{htiSaAYuDA*@?u#8*9B4bxy-1RjzIwNdUcrXm83V+ zEt11BtLJl{67+uL%YEbSN2J$r%ZWzQUMBee9vrP{z zyZ9jSLXRYI(=Dn}`tYG=O6lR3WnvNKZvPn=d2R9doic3sLUT_^gr=^*gAyUz6w}r4 z%8fGaZ|#gPdLCNxPBz?0)3tK6-26S4W8~v`(7yj}!g{(=Y@OR~mo|3N*YC^ggT*=y zO)a}yvzA_X!S0`sO33)euIHQyVTE|bc6}f=gno+!X7?NZ=zJC5h`VEjk2i3GqyIWt z;OMUBbFjTZ$v12U9$nf_Hz#VDI|`wZttd$*6{L>6a{DZiK@}$SIdR&JoVoEW`sMVZ zvrliv=#ulTG7*<1eBeZoAm?KTn&b$&!ZHv{-5~gVM)9jg8J(t{=ggU3G+fKe75_rY ze--Xq;`G>9v-Eiv9zU7XRA8OFOWqwQB1LoEmQ;B+%nu|wQnzr6uYcfO(V&l>@m~=d z666VE3q3bm^<=@{X3ADqLAd>ULoLtTJ_rfrbj*!j#||-~IMSMKQHKzp%B{@ur>B>D zKGM3X&~)=z){(tG5&nR)gIi6gwjiyC5VM=cOfGA0SUp0boqtDI;a;n>U=19TI{vyo zW~Xg%#}E*q5iB}ADMGrezBs|td);yJL_JCM##Z{`1EzaQ)MDMrH|Fiz_SOA9yl!994*l2ny{ zegDFZI=dR2^+QuX$X|>L`#Tf=v;f_YAxN?)T=!C-ji`bjJ|ct54pkY{J%aiavfq4n zpZ~2|MRoJkejw%3_%{B%Ujo?$W;++@86Rf9&j+lz&@@CJNIkwTfa!j@dSUR_W)7iU zh2_qagiLYSaN*{SbTI}#^shL(XCwTw-xsTA_I0Sc;-$4|U%Bm^A2OV;x#k7bA8S=zzqD-|6*&SpzV5=l>&`f&q z^yOqvvD~u3UoYDa&ZuJR9Szgybg2uCrZ}+;YUD!kn{gV4lnWOq6m1Jq?{eSi*c&}L!Lo+jYW;%FQa`~#p z;q*LAj8Bf9`Yz;=_m?D+tPl8+pCM8xhEBXbXtIlEtAyTfL5DI6iSYsXi4I&pUjkG(Vv)5X^8!N{%)57*=CgyQflXgq_Q|L#wR)FtpR1^v1Af zUh@;CeJ*H<`9r_7^yh7)q*oJ*bE6%P_N@{Xci;n#4O5>`*o8T8_NSmZ#Hd`4*^}3F z$s-rD4oGmsr|loJ630R_Vp>OwI{px2YqgJK-u3wU&xsGPPx_JVnjd6;$LHU&Bw9BW zGv#cN2v1to(Ka5Z8Q$&Q&&%n12?(W$rqGCJY6$W0FcR1tCfE&7)MM6)Vc z@YoxK5}Xy?@_l2pGcIS9N^5eD&7W+Sq#-(mxIAlXn(K=Ed8U{2H2F^4EqsGnYB!N%#v86! z5)CKGj}l@M33sGE-Ydx-hD_EcF03sa`xX+>&Y|&QfM_6(ape z_33Udn$K82EfUdt@Eot^pr~J#7~I^m)INam1}5rVvY1jEUXvRQ1lklxs`&COTInR>%x- z3)@Qj1m`)$JuvHFvR2!Z+VzQ|(=K9K&$m9Ook)WX03S7;-C)zhy~RIWBywXTem*dN zm`6Na0@N21XpMQX`U9tqTr5cneDQTyfKKI)_4LzRWT4+eMa8M zi1i!Yr%a#L?h6rJ>RL-L%+0V}ztc=)%dlkZ9NPjdDL)CPZ0J&yaxxA}v1{!T&eayCWePO}#svGNDb?>^(0l2M^+XsTv!pwgmL zwqY2cQM>u} z+c+!Bg@eOJWczea^-9@UUSNh(rT$ON$s&3iPopn(fns(3o^7}=hjz^EE|$=h_oub1 zZ8w^{SJO<^SWW|?fx1>`pmi79FytzM0`6;DZg6N%Su=KNmh zH_#s9&-RN``}Qn@*PR}<_oEN*oB57$Z1p7GSf2vky|Q`Xxi^V9%DW%ibUxAlY;Ey- z%Z_f<)xD?&wpII(bbbUW2D^~(fPu#>YPk(Zw|*`E^GPiI%hR5>LoU5Ry_kXdujRDw zzSQ`ArqY-7Q&57u)xY7jg0Z~QcW-?t{ulIJ-$@6x=^VHR zc+t8k9_KoPpb{^)(y@oQxee?p&M!;T6v^xl!y zTee4$X3fZiAhUP3HL z1$jHUkFO>z__8Tm`D}nFK4Q(WNQ1hsWtI8;oxWaY9ASnVbq?)s4DEP$jG3}`dIotz=VO#|Nd!-b;0G3_^`@z4HvS9 zVZiz}eG9w`EqV69d@75oRxot;F6>$wsw6*T@I|vv(!PEaql&t78K=V>@<~-vo{Vh= ze&fL9(c1*d^ooLJoT2h;Xv}?49Z{c_6b>!5%JP@d=5sczAu5G|8dv^SBoz-XT=m(; z{5r_Yh*jjgg(>u<{oghYbB$gsk}w3k5z3%^^bKsrZzcR-8g(&WCRyirkw>@|r%!4~ zrgbW&s;(EvKMAdPsL?}Za68I6nd{{N&kUPK_@8ae_b*Aln7vU636WfnmCHM4q&olY z#4p&Nkfz0)h$B>isPXHmfx_2OiFGecYh=a(wDbecNt2IsZqVzOZmF6F^1OHky+Qim9M* zsu4E=GWvIb0&EPgJ&13A)!8knjjJEvKtHJeNYsEh#8ObG8(|?2%k!L9U32*?ZcW3U zA_d_sdQyzeLvqi)q*%w^j5(S8B$Z%6RT8-G)7FIXipe_W*TTsVD!WuRPJWQ;vU6*jx~%ycJ3E{4cBGAJPYlA9yq}1touq zQ$@$q8gBBL+d3Yg3aF=D*^^m(s2Y5DnCuaoG(9^)AX*}pV z(JD?O=!&J2`bL6hMk)!X)O%`1^#uoJb+MkItqAf#v%hQ$B_`^wXIl1*hER-BkApZK z+2sE#rJ=^!A3E$V0)q z=~;^g#S)fble;{guYCeis`ZpWHv@U_eft~q)lm-6R3&0@V?iLOC&v&@u@+9M$#~3x zZQBjv47QOdipWB16f1seNWW`FpGaIZ{e;y;$m~^$&p19^p4Rrx{9h!Cp=i>p(nFab zFGcncrKVrZ^jvmP+~$@tzwqi5B|9Cry?)TiMKX8^#J?98QFu!Y2B zWjLyn(DkfKNc2E-dh15mt~$E&QN8@@y`7d``KAZ-Y6*5RtT!#j0X`j@!I>cz@(a-G2)KA zdcrQ&hVaXA(Xx#^$9|~xwGIozX-4Z%V=Iqs^Wc^GvynMINyexQ zAFdUxiV4hmO^b(pdw|AcjGb@WI~Ds~A#<9!Gs+Pg{UzQL9Tnb$={AkCm|2huq{Mt8 z(0l9o{h&wxUvuGx&kPYNQRPH)e}dUz;vWzl#-g71WTUow>K^?gfM-~S2P;5!Zsltr zYy4P-3E9+Ef#KyspTinOc-9WAKKeFD#sul({H^Ox>K|STmcumtpps{dTQt@nXE*fH z>|~Hrm{u~MCe~vE9%$2V0&(;b`Zk>X4_E0T`Ba8i^JT&O6rnS_I^`yMDG|{bQ7>RP zqzNwO8OlrNBtw3X{9}pO*yoJPDlD9xvLg}&d6RAj=|omUTGIyfF9QV;l*c-*<1Qy_ zwJxSDARtDzEEv2Jh`g9r{|F)Zyvai$PHeIi>h(yRU6n!r68Gc_UW<3_mzLpkL+&dR zA+rgeRKjn$C$J=!d7Cl4A?JmppYkV=kwnb>5Xp6}f?|5g&Ixlfwv|mUwKa5?nyeIa zTzu$Yhm(4KnNEq+rTl-8%RuADK(Wus^nM#A9GP(7Tye?f}ZNlp79&`Dx2P2!CRP>BQ*#CmW6aMWzFsfY+*OLjM?<&(JvH_z>AUV@xz z8C0@js9^JK$AJg`G7YSXuE~j_!A8GYbabFyA3*4@4xG5oRS%ZSdP`wDJ5l|?V2OiiRuusMM zEG!}LI&qD+UL+PTMnN1BG_e_c1SK~FXDTc%&XRHJv9f!XXJFY3mLc2X<9P4$wX^o9 zy4({-kuZPD{3RfwS=Rv(WFsJ2bCQe(l>LizMP zNGY_CHuy>ddbzh_xd2dM9@X%nd5)(^!FyaysDV5Aaw7&;u^zj|5S^kj%%;mW_e~@d zC~!-cd&iAEu(QxJB0If(l_%b0)@#!<@5v#Tv@8YRTJ`0gm)1HA?reXZI7OG)7GKK| z@jv^-zS58}0qdEuk8A&ERAwW(wm3u@%4rV6NdJh25IMJ91ZR7Y4(kbcpiKK?@g+8h zCZq1%4|f3Q@HBr!1&R?Vqg#4!nft{>e$vt#?^qLTyqzXHzq~P;S=?^pahz3#$*us54A`<9K0^}B$0 z&1Y)J9p``Og|M4+L`OEIuEpZ@EW(-<83(&kQ&gUwG1JJCC~m;EJ}NlA?y56?*KL_X zK(#fe^avrc&lmc~%RPSD8RX6I*JgjGG<v^`AmL<0AfN?8Esd4Tk)z8(=PrBmv*{s3 z*Xn&1{&Ds4ykFi5K@Sg+GZ%P)z!vokgskQeO|AOuab!_;+`n=%M4Ha^w78UsmHT z#olP?r7L0Ai_Cs}TWZYrRf(d!i)Ui(>^CGDm#k)H?n;dJlcnxm_knQ=i{%~cPk z~5qN@0J{+L)zr36R#061m*#Xr53VZtvg#NcmTb8uj;cl~-rpcZ1+* z(m>lqpKIm09%5He(P|Sf-8!x<#O(ajSx?(}AXn3y%%x-O z@#4QzMfH*P)2mgxX`{%}X<){%(}LT&8S(m%qti*(3;KFI!IN10INDo>ab~Kav<$7k znk-0urj6T=x`Q{+@iEnid)Ue>(-{w!I-G0R{38X_{WYWXyIY^ussr zyvm6Gvhy4w1+eLJ)jOvC*ZqNV|pf)^CpsmSfB5TzJ&afX=XTT)IzA zeOIK`mX4kc&OlF;e7+P9u=$CUmF1;n&SAT`DN)~doZPUsrbDjg^_-s^w$}YA&>LY; zIu&rVkKiV=X-!#m;S)uag*Ev-->8u0u;6^#LQv9d(?@L}j6uX`|8~s`RehY`c*U== zdgdI(UhM!IjGBl@pw;&B*2eFcIRhgyG=sE!Fz#AX`!S&!%AueRFOwgEr1J+->KJYI zk;eeIGo1Y?2DK+b-YUTxgN;>gn+?(7mF1q&7x#TMSenENU= z1_f{fIGA7a?g!+MAthMSE!1j!6SBbX$OD&>o5@sn$@!9=0|ppea|-slfl)VwlR4cP2=gl^8J6g<(Sz#T^l9*13d z4$z{Z5&_wV!j(FSslm8xlxXijSL>6ljx}y!{1e`Y=>Dk36$feM97P+m7;<$MtaYnI zT+HcUz4Fd+?HWXwaAadtx}R7Krs(g1CzPm2jPEb^oAXjo==wVQOYs0AAI=Bw0c_-7 z_k{KePUe5Q3@JQAIwn39k-PPQt2QWnF|LnXNKOUY6m^}p4<7Qv#ea)HZh0#ay#jj! zIC^j>mjR6CGiKCIU5eMBwj%XY_O+ri=VhwU|hhDBV193*tQNU6ElV!d%aqtq$Gj$agdL0Zd)ab{*ctl zYc5;EP_#tj*1Y-kiq>cRAjMHk>e(o~qx-eo&wiL76DfztUM<9$e!qKaTUIYn>W<+h zFtR*qeD9Y;lV7p!6~l7~pD{U3?KF4_^~2enob3SKes>U^gdUM-Qn++uf-4n?(d8k< z-=bzumnd8_Oq+uxJ9PD8%rK|El;aRj1E9<;a@{&D3_}}K^eVG$i<)EcX+nycP(dy( z@wU)wY>Q#N%#4KLgIQl#MgnLu%1hT}uxI<3HxR-N9%n(wsF7z8lfXo_&vLh>H<-xq zeXNSYC*~+$2u8z=dY6mC(K#S_QKn`24+Ki-%@j5s^jf$a=@I~UAhF20d2yPFo{P-LC2rBDK+_3Cz$KK~$Q}=5=lm zYpf@4Qa_Eu9QAjBSDnTI+%#8Ps$wylwelf9IcDN)$?Y-y`A72{d}Q^#2W*JpIc6Zp zsR4^=7KNMV$&$@~18uqO;S)4)!;JuD>T5Dw1)9y-Kp&IY+5}fX(OJpmB_YXC5xl8; zSR-;Ln^eh&T`d^2gv{fPy#!7@m7t}O@mubqPBT{{s*oo95AUTaAmc?KX)#*I!x0YsZPkpj~s(w{G*=5z^7 z@8hI5e~MxESZ>XJV-H^Q76Vdk5;1qNZBRE%J~mOuyoW#3Cx6|iDVraF@52$_O~(jf z!WFj>dTDl?1-8AdfXdt!X?$O(8In^;IQh2N`$*>)?)Xn6Kfp9rDU1N%_A%8)sr|Dn-$@TLOJl$U^%~|AraTT9V{a~tg_O5ipFisYnU;!Zsk%az zAH%~eMzy&dJj34ojiiw^-D6J%2`|fbW|&CiJk8|`?Fi6aCcZ6U-Lt9hVo4?J9)iQW3)UZHQFW1{2h*X884v0>=!3U!O@*Fz*phd~X)kv*>wGM=G# zi!g$WL>Wk|g(Ij=+_dNIhI4F>xo$tbbiACiW~SRCyAjTv>{Z(J)KASv(c8 zl-6vXc$g#g5@R|#%L#qSGHVrgO?E1r+cmjJ5L(2T zuDKogNHl?B?VV>8Vu(dck}#dRB9s|nN6%YS*^OQvNHRC5lwlv4)fmP4xk#zCU7;fs z&0(oK%axGNBk8X85*@~ZlkmZ&##UKR0k;OZ9Mv6wjTDWMI)m6z-xVZ78nKw3nI%z`y4x2Po~-;E0fZ`#jL9(OktQM{RGUcaY)z zjGv#=a;N^#kkVeCmvUNBQ!r=h`R-VOd^@D|ioDxnq?7;YL|HG9w2-i?VS)$O;}1_{ zd=J$fZo^N?u!Nav~Azw4pNKhY=53W9z zn(6LHUV|I7k^IfmucRZ+crB$g$GzSf*v7REe1xck^P=vemydW4Tu*SvVBbomj+FO| zi2)aS&=ZwTIlEh*TRI2PV&)#2J>G9SO336tu$1UpmQoJj8rnI9D;wO7otC-(2NAkp z!lH(lc1llx5BnQCN`&gflY8xYDK%EvN>2*v0F{Q@HA6*aMx9We`l#yNs6!JfrB;Zn zeW{3{3P;+{p&(iTy|V)B7zk1ioVW3dz)NUGqk~6dhml{&+;$BJ&NPEv??u$_0y0){ z21BD3Ggl}w4~JabdOUEF1YVm}hTzg^WJ-OT?F6}sWhSKbTafXkREhZ=Odr(`>kSho z++0tZmY3R=kl-jn&=baE$=~_TZDquXW>;_}z4s-!41&^qb!*)Jg;3aHXZsRM13tId-dj!e)ec!#hVj&`sq*LfpV%t`zcXaNrf> zAQ$A5Pa~(}l}Au|7D2SsxnTP;|Oq1VgMdvjNy6)#18NdL~7m zC*D!?*))zcuL}V*p)Enj>m+GYyLl6CD}o~wA|4C*+M80F9qiLPPaVOl)PhM5>*$~< z&yxS02{T5DtLK+T`2dlZ5s)d$)?i&44=VasDJWkx(kY51U%CN=GsBUiL}+3|7PNrM zd<_W?ZNuTOcQ2_$FP)4X83ro?d^yn!MedSpm^BZmeQMAr2Vj8$QzUgC;XqCL<10 z?%B!vB~b75rSdY!jT`JL?3y#26gZKAov!`@?hjUkncX5dl%5M^k{U4azs#U1V)InP z`JJU*mUz+INbppVNea!L5lc44@?JB?`$mV5`F(*7XG2zsSy;3Crc999PQ)7_IU~uc zOC@EmfAMmNi)&oX2G_2>!BJy9N&CmA08>UH$+i*a%MwkA+D-mMlqzoNAioY*@g37& z23~_AdV@7s$9MrlMk}PshH$$}g4+@k8AE8P8C^WVeh(Q#8hFP9~uHZ}_J4wFZ%9<^nIeknI?IhdIGF_LW~y6%Qj-#~lGb2$LIvEJ>l<8F||{2e1hRGk26g;HnlE z275GT_PIvva_v@&7Nn`SP3;CZF|Yiq#!TmvoVxI1d~*$Aoz%oFFCSa_!e?s-sL#6$ z%efS;Case0EIo8wol{k&l$zMO>S`M0pZ3<)*Vo=3d%oYw44<=+VY%jM0&lAn zoemGxw%r;E)ax}EZ~4}4s)^x(>$i7rkCGZNpQ?+Txl(8T+z9LTHmcRqr}nDI;>{Ls zVY<{yM65fc$OgyC)F*)I=;slWGuxw0gxlwu-20i;;g#bN?m}?RS4*dFKG|=~%3%hb zcIRW;RWz&(3HFs(xT@$`4S5$D z8^SJi`=aSjI@FY5Pzip$is@W^xO`s@db%9Fez?#p$OPFx4T|{GiS{iy&6oEIe|?kz zp(>R@ZSgC}p6o5Pj%MpE@U(pSd11pM*;_6)N;J~$Jk+x09kJyClk$z(Mw533aapv1 zx1$HGuRn59|L!S@O`Vz1Pf2@|rnyC$u9uwA@8s@7#Z9n2*6(UJ*mHj!m=f z)~t46)iyYtx}|(cd$vrbytu^rxqU%ci%le+R-c8yxTvC6{)PyvD%f^nX zMm4t-!wi#42h=9GL4?T0(&YEhoYWL8C9}D0ucLV=?JLY|GWdTw_i<~1u~je_Z?h&$ z7?E1L0d6W?fSE$H!m+Fa~}v{S)}!aiwi-YYAg;|6tflCY<1lLE`4WdEIBl^APS;UWS#yattC9$~LGYAU}Ok zkGB;J)o&z6uQzk-U@TT$)mMn-L&Blka?K!iQ5&V*m_sKg_|<*8bd{?i*!hu^d}V^y z10#Zd+MB+7(@1^s_<*4c~Ffu~rm-?a046idwHhp#Mk|jtkBR=O_ zLblzr!-Df4yg~=S3*8En@C#ia!6uOKDWHM!>#{#W6ORi@I_x)%Kzb8@r5S$$J_@=$ zQ+RjPkA0#-F0mW8DF-gh0V!XB+)Z{+fj{(f)i~@gA?!rpR?*QPg9<7%wCr&z<1?F2 zCoAh!JxjtEvWWA)a93-RqZMKJr}9Xi3v_%8&BJ4G1nc#PRRkd($d~o04*=6OjWlT2 zZYTJZ5Ox;71!auI=DMpi!m5LiPzn_4T`$WWH!V@5>m=G*vOPiIac?GB%0QAAL_7Cl z9|y!*IA+IqXs)mrLoJjo2l}2ST}DS*io3EI@fo26*{!C7?_l6qHYf#X-@Yh6in9nD zl-T8}c;P`~!^F-o2N^JMiDP343F`6x*3MRSLgyLyHB(mH*O{t!-W{mR0p(N>R@D&N zC4+tRC$rQm7qBd`@~Js0?CejlnC3SF-aUPaYIYTUIu#nTD;3c-AB9cGARgCjMv(2FIa-OPz%9tCu;Nf9)X z`ZFWlL)sw12`TkFZZPDe$6u%2C`s#u8nG5WFvx1XDU*>Ib_eaHCo*0h9t6V*+!U?^ zH=;1WEZB@6$(5YSr^#riJ z|LZoW>KHcgPEQ!N6@E|aA+SMWAvqEfw-Pp#?x4dWGtRwFJx|0FfoPTqPP8eYY?weJ zfrH+T=p`o|z?TrNdPhG^h}5LWE}g_pJ+6hgR|e)B$CD^!$Lk7oRE5Zv{SFZB(98v z&|~K7yJ|dtvzm>qz|OY=WT*MFq*%LT zdJIM;MX#K>s|D85!JnRrAqrBotzqYZ6l-4xo;uwq{PIYwW@RW6 z#u;a-PZ72ajGJ06w_v>zJk@tF1yFv+2T{w40tKW0P>EpGGr;nxTgGSc#u% zGE|WL;psp{aJ?g8a~LqFu4yfez7x?BaY(!A^ySe}ng|KkSlN9Zbse+BW}GDLQyq*a zq)R^jV)rrf?gO7sK18^?Vodlhif?lP1>YQHms)V^xG8 zqO(RTOgjs=rprfR>x>a=>lqk@F+2rFNE56-J#1|iq8F4bBKgnh*|e=eYrV6ZkA3`5 z1Bq@Ig2w5i;?}s*I-0o7e51cl>i(7ENv+Gd8fW%pMCGTBhZOHd)F2YXux`v_ZP$$` z3XPN0|9e~)I`pc+NC?ubyX?A#wG=nQ`6NS{T9S036UVav^zNI_`x0(d0Yycn+2ZuX zib1z^=;Sg7Z{XFG>arur1BTNQho{sk2|8Pw$xhH+JJ7c#`l_wOaGp;R?pNm43Kh5$ z=9U^VTSnaVmeIXr6iB>htvg7k&)$iPH#^|S*p3(+ZN(g5lyZ-T zcX&t~x1b}u&6QeS%XnI-otXDca%#hB|L^wdkm0e+N@DLoMVb0bVCnL`3^-1<79TSY zeyuy5%zTWb!c?(#Nv3G=wHise9gJ*)Rwl{!WT;I&-25{_|G= zXO%bs_AT^srd6-868jw;E}Id_&8so_TSb2D_ybN5N|JF$`d^jP z@Q+gx!-!H*^qK{Xjq(HX|5}T|8vlGm5!j`iZ8A@yVKnszVbtWc2II*`!0|v|Lf*dg z^Yr};K?SRYl!tpyRM5Q6`J=ZhNrP}Bq*Q@?fQwvFQPJP;Roy8!GKsv9*7T-?c z`f^NAd1?#(Ec?$)goH%op~6xYMzEN?cIO54Q9%=+4m`Bl*E zgUA}&*JM}W_6yD1cQ9SvIU=hm@Q%{6iRD&@OD~2GqiXj?bT{GF1Dx+R%!-t*t(R-K z?w3HRYwq?3tX`&!Ua_5^jE62XAR60`S ztM-l&QXhlK&JHhpH(_vVx%+{Rt9E?bBO4edg4mzq4CP?vw~Ri@c(geDb*Evz5mlDs z?z>~Le48fhvNT%s#em)5k+QmEfY8k9(#1%Z#AN)>r>(!XHPo{3pWwCj$H0JZ0eOqp zXQQB?;XkU_l8hM+opLSi<1r;r2S_&K+#dX(QLgnfbj<1@IpjxRjNV{amo}4<0U&GN}49NcImqe zauH`Qa+V%+S$k1I0du>%^g;uBH2L7%;&3a3l9iUS2Co2#oCW zA_lP(EiCuUg?W8(bnrQDx?zwGPNV}sbdl6uw4NSqV5?-T)T(8H=zrNm2B6+%7eIIg z`cNS`GA{~~1n*eDl5t!7_LsU(R$p&o;{x6!z)1CwZ&o$_{rB8kRwZ}Evs$bGQSH>} ziL@f`!G3JXw*A!g86+9>=#Mok@y~)ihTNNK%vWCkBhE-9Gp!Ap%}-o!f-Dic5b(UA zAy%qY<-?SqQWp0yJ46kNO%6B`4Z3JFCYKNcsHxM64o!k8A&fb_Ge&t?4r+&2;5nbP z7hN`(Q1~HF1rT$DkzCYUhY_1UQs z2s9nXHx`%!{HleO)XB|wC~+{$HjF|w%i@w@;mfT`LBed{8KgV_lc7V${Q1T_;)a3s zDRJMk-XpaRa`1e+J>f)8+lhwb{~XRwKpK|K;GQurVFB@$P@664{pJQsXb%zZF{x!^hFeX@r z<=i$tNL+QxSq~B}L5=Orz&H-bor1+uUP{Po%nqfvri4J47 zp}rd^IXp8b*d{vZ`XbI@&uMP#Gz8&kf)4V1CSxd{2LAjJXRKqn zv*-L9Wan4pGx}WKL(eal5Lj-;(qIu6Zh;4;-6M#JjEOWky$RfXy5W(yk&4sXwA{$5 zeCuGThEJjCg(^z~{dz1|f+Dv&FSgr#qT`f@e0*b>O+yB41eibY6agwijMNCp6<2cR9HNi1;wMd+D=D9^f@%{?w!Cxco$vXTr6ok`X zNM4wtQtwY!dW@L(O7R@gB)({9pm?pdz8dznh)>v{jxTkkh~qH0t^5{j;A+VsiHS55VO!AloJ*9I0JaKR z4MyioUGM-04l&>P)43mY2Ny(e zegkx4`Zt-j-=pC&n4oei;xNkHUpUUJX_zr#saXtLTB{9t1XrS+8l*m%8kS8hpVZgW zuZt7~GG=o9XT?TZU3f$~E!lG5$}H~qB>ftrytVNGFDhciOt?Zq7bC^SmEvDk$N!p- ze4-WF2*{R;HWb*t_Yn|w{JMyP|}=Rup)+AT3l z#A~4rZ(`P$lW~^^70E?hNqGV)YxL9kWuL#9oz9yytZ3_gG_t(pZ?mvP<&P0(eAJJS zQZ0hB=)Va#Tbz7&1Afd<;mbV^JF(M@G~N)s?>MbO&J)gT_;dN6h6^bOhMhp>OD ztcXdc-zgjnCLvXv4){xH480~l3d$?bG=+!vg8``nK+w%d))N%#@|GE!{>BVoR>q~n7^{m>Ed)&6Fdz6 zFqtQdmObM5S6RhT8Gyl{L&G2)`Z(yj5D3{kRXSvu4t=62Mh*CX-YP2zQrZP(bt3H; z?vksDv`r_Syk5xLWTvs>8R$DMtC);LDTHJ&xI|ooiqU6V=2JUgMen&vg5TSW_KMX! zbb9;B-8qVShH||YHioM(syH>1%?c)aiW(b}6X)Zh39!kceLAWemX9u7r=k==af?;8 z$g{!5Rd9l3cTc14$0XQxb5Jc?k1jzqUJl^i@7S-yE> z_}#Cyg}Ip(bu(a~ma%*#JRs|1%=>Lq2G0NMF3eh1S|@Uoun+RHs}}^kmbv~{nGGcDvPb%jNxI^(~X@n zC!%pV*&_q{1O_b*yg!gI-Y3efLb|&WtxftMH5N3d@>+|1Pc!(KMlPFqO@Zm4Ko3X_ ztUO$h|E@6XKJr2a>P-Q6=YC!s$%2VZ_GQp3$ht=EXM)CfPb9!ba$#eL=)A|_{x%dJ z(TqejHy7nYVXaJp;eVJ5vZ6s4NyM&S&91-HR-DEv`Gc3{ayO>YxrY*Xs10Iff`w2e>iX9v~GEU&moPaGCbY5)$t@8rBKNvyE)6Dmc`o zrEsB9$+Js~7m9t@WqJy(o`@}Zlw~(k@s(vip1L#l;6*E^PI&Y4OjNe=C3z)_xYFvS z107acqp>AQUtZkMo5-LDC7fdK#wOHu!b|99zd?xo)8FYB-4jGjZ9~k7y@@|Yn#(x^ zQS)e$`Aot>XZ>_W{~l2Uq8Bs~#c_5=rO&uSla>?rvjKAdjhBWMnU3+lsFAHAK$IvJ zl)K~9W>9RDs{f-YXRWcRvMGURyi(;38HXeW)DBOG&nF}B1eh55i1Y~Du-(D}F)5k= zc$#@aRO-6jp)sQvK)b!H-05;n`JU_P>G_}JBhQHMM?;#5cA9O5iRRQl>koO5v8l18 zymxq0d#!ytxR)&-?*orF;=QDDQyR~;j!L2lRcdYD1Z&$^J*~y`na<8`< z+%>$brt7nXoz2CnzD8yt&*A=2^8zYdtiH0+qIu_gfZ(|}Yo&Cry?h_+q(!}%z&j~V zYLxmgDCfq%r3z+c(@bR9W$(J^m3+@K6@8Yw@G-K*PYZ;#;9&$WMBR#$yMAJ;&E+aT z=_fIk>kV{r_T7R~RuRL!IRnILtq`}u8}cCFI22jc+K>EAsn8B>bGb8lpd2^z>XxF< zTYFr7m2*Ck%^p&v3Z~AwzVXa!>8`&z@m+ppu0}kct-x~;7V`b<9X+s>>NwwWFoIog z@Hi-FA^&(!%YlS(u@J9ifs))VPNG_rRVB__UMoKlxSi3nFf`rl=Y*^qRRDn^9lJ5bNZ%lLR^}_Cf6rc?5`C4KTBYcymj! zuBbxgT@4pSMgQ{FbN}O+KM9s)=(k7X+N?~X3ww4`kRxEjYQvV7_5gVbmw_GUad~sD zmwqlv%cKj{G@c^6i2Vx54OtIYXvi@iGcKg+e_^G}DK$pJ5;GJM zU62~Ed{2ivoQMTDR!sT5_Y~coaFQ(1 z5S>K=PTrtm&gNS`3*?Eg-_pv#l3yzRP#L3FYdxwfF2tMCfM_$Di-b{=$JM3l--<^| zr_1mvWDu|A65KNzVYHzL6x+dhpb*@m~27Rtx5%6d&Y$wg_B)BlCC6S3?-i6 zjwVN0vglp$3Ut^J2-(!YPgWxBE^P0bi7@{mO0Ko`=WlaVydn<~x|kh5yMl)kjb4qo z6Z;>*v=2fp7BOg0SQ$HEic7)U;=ukhKjC=yS56utT8JqP646n1Fxd81VK%awk@R$8 z6uvo1rUAWzVT_W{sC>a@L?|l2K>}uMzpH&i$_Ouq=Z6j+X{Grl+43i$zqsYF6O17P z_!wzoUDntbzO&3>-RH=Fp($_q)-&pU>i3I@=1!ZqnHoPGhQ)Frr%VeWc75plS_i!P zXcXH`jT4i?++;EV1@t(1tY%IzWnep@(A=XzTi74hOb*4ba1XyV z0PBbexga}YKx4-AK*MW8Fi3s%^NINs4P14|bTAC>+u>J`qW<=+uTCXO6B#XVCUh4- zc&V2Wrl%F1%M+BLNeiE;r$l9gY6>!n193LDg-aA098qu7W_J3xt2=@PD2ghNFj^yg zAWDWPzR-5_VG3v8Oa=|sgi>NT35(F{Pa)zgv5LrOw`FJB6wQ@Td29R3F&y0Bx$%Vi z9by=VxPuh7(}nJbMIIz5pBxQcg@IveMQk@BQI5mI?l=W|B$5;Iwf$vR z;Wrm->tii$6v1$4`g7p~SR)!#)HS{ZjnOOGE3wC_rf2=gTKvs+wv8w>Q5m_=xQ zG+%7MmBjnm%7836RWnC$1{9vec;51*#XJ%KY0nc(7mo3$HPGzWt0NH$w5a@y@4)=& z8PpE9ipxyiannmt#05OWV5k`qVK z-W5G)5sQ)vkTp!GK^u@}%eGn0#ug??2qyX)HZU$-=9(j)FwL@C$5_W%Qs=D*HmqMOQ1ZN(MT^kvV72{j_$dmo(& z$+hT*6Gj)W`Vez|9ocWP`V4G)?aK>%f3f?j94Fe`bN3K3?s z3p8k7c}F`zFZ>pTWsEeRSyZp>tv&Kv<}tG>Q=<3+JlG05!Hd-Y{%zTcIgXGPv$F|F zZHWyH0^P~kl0jBiKcOUfr^VJkH@hpj!;EE$_(&xA6YJpO0c(+AwxnncFhhj~u${DD zaZ|X5gIOQvd2-Ez;#S!)1|m|__s;A5Ra*0vN5#M);D>bA6g0t`_B^K_|03Gc;-i6J zI4011@vtt-S+eNnTZgTb=tq}ef;T`jK-4>r;~fKKwUCla#Wd!}3wGlC2ix4Cz$BP3P}b>Bw!B+6^K( ze)2r)W4049gPyXXkDpI;{OoG%%1(~9W zywpIz-F6qmn@}~q*7%V@z^y#BDljq!-J}E0 z0vm&CWRX>{0Z2oI5CY9R=~}h!j9V6#lolk;%j;@xf4rgC6L2f<4jO3);;i=N!kMF^Jj&7MTss;yoKqaCBC*Y$v}?;xQNcY$E+$|Bn27O23)oxTPI+Z zUH)L!K1`_VI{HRX2^o#>Ei`kO~PmsXhl( zg)Q!q!yqJG<0e8bD{7pP-OPTQ^R8d|)*hXyB|?WKWllL!CW@w+tmu%2LhzC5?w5Zj zNpiL5-)h^fDH%JSx=NPb*O&Mo@?Zl&a%$CzQhEqpi24oYn`Jw-@uTM+cmIRxWmMi zA<+W5jaV2PF+U2^CCCyHcE>?q;|`Y9E1D_bRqSdTDV&EwfhUw`5Q?Kh6W&U)#ljq@ zd@}#4N3`|PPEW$v4(YAi!b{b6C&N}uk@*ilkgOuUa{nk)Oo}N)aU$N-c4as{^&=3b z=Q##CtyF$+DZ|8t&2)4M~A$@^?th?ksw`) z)<9)&svgDF8_nYnoWK;x{2_oWHT1)h0 z*E@DuK+Uo-uqo*SByI?hu}OZ#UnNC4h=WLbQb6sIJu0hWQqJ99wTF2I&_4|&MeeQa zo{zm9F=;^Sxf#E$`|@iSVpAaVo8Y?Bt2%11pv6N>@pRUOL?@1r6`X{Dq=z4(j{ReffJK# zo^_sXi;L%H4xdL*zW0SegkhomUlhql5+F(+9ZJ}4-{U`K{@MRw0sTYOdn}2;PJ4B# z_Ax?&0QpNn6ZjM4|8!3>{Zz@7o))LC6R*LN1|M*af~#?R!BcD}vU)%6~;?%}hLHE^HDwNR3; z9t!sPhO@15TUDo;;B#jQwCak-hk5Lkz@-{%^BJm#v z|AHSUaIRi7W_#t%0oy}#Upc+%SWWI)zsDTmKwjT;#lJ0gAL+{tQVwowrbt>D9z%cu zgv|zv5C6i{6s2t{|3(q!;L=`@g9-1mkAt2s1XRAWi_pP5_wPS6TgjDp$E;%uXF4y^ zCS(WiL>yN=S0S4&o2Ap&9arJ8(1wl;-p#nJ*APh0fox++uAMf>XM@0adYam>`LKbH z>d20lGn? zuKLQ!&wu>jqXbNggus#^Ziv#*@hd5Zs))6S?BSKhX{4RMrMK!VcQLJ)e#y=M#zQNY z`Z)vqP9+soZJKN@o`904M#tO}1Tj;HK1&$U+|YyvJ_T6r;D93CMWm1O(|&%> zZqGIR+A+W+$$L%xIb8UKh%FTutQT4k+9dkB_F!aYpzAylT`d1B>=Iqtalt1W{XcA- zQ+r@-(`}QEZQHhO+eXK>ZQD-Awr$(CZEI)0-|6!M*6F&c?lEi31hn?Ko7JWMdVSat zu{%u%cC%smk`iLK6ZC$X-kN=_pgxqeTpDgP$24lNxw-+8Q1Xf)6QHO6bPtpP|aAuulTkZV=$m%`mghCeuZ`P8N)%$9f-tWV&E~+

Mk zf=}N`cjr*yU?ly_@$pSZF|BW)VtjX>3f@M|a=OOL$@iy}3=J(Eu&V)(a~}C1xK$7< z0Xt5k4uy&s?<#FnD}4m25gSUymFUv=_(;N&=?SlP?5R74yi@ujC;(fiNuVJRFyCs+ z4i^0Jj;5UJm;J_Mj~4ZC9P!n1{BMKyQh#1dNYRA_?Z*6TnAFPF8Xw_gIr6;doTiFp zz`g-uTt;IGpV8TQncg(W14i0t`m^R1WHgw^3}r&(vf@iOcxjl9MmW5VN}Xvx^F2IX zhZ_~RNzXZC8VBO(m1I?J&^;Wx$|eCvAUPAX^c5wxBAFN zMXubA@4>%FwCEoKE=r4j!H;_YAQXa6K`C`AwRw!;z?F^XZ3ywvYgrASlv^%vh+iQ8 zMV~7oHZ=7+puD&NKBs1Enzf7ZUbIo#t1#7Uya zHzPe)HY&8o%w%|~j&HXmd5)kA3s17cfQZn_3H9dy&yG=9hLRhMf^Z)$Y2g z!i<19)W3!BlV{S)0I0d`+(A- ze&OU-11pPrM2t~crH8ZUdR`exGoBXx`Rw~{|iQgHK(ZW4`s?+6Yq<-7!X5+1%Vcv!SD0ZBDls15JfVw9Y zd~?d+i)ZX6U!u!^Fj-N&U=y8$Y{%I~iYTW$dms*Xw+gz@Td}V{ucc_`(xM$|$k-2h zWn)lB@jkm1OkoX}qAcYbVv?ptGt%XRhYQ4PpFCg+)LKPIwhOxg##txHXt%7)dG!z? zCHlL8!K9t;>R-%~V z%qTsWvCL_S<6+Sw9wR*9~1XmAaWdh zu`f{e_opVr{){@7t4h-LF7#nTn{1<^=E|?tu`%Re;dY0?EnKq5LKYu%p?qka4Loqn zuL)2yYrUJF9win}oq1u>v)(pJh8!l%P~s{s@Mc#DJI)*u0ytzGnz`Jj)NWf8bWr*+ z+}TP4*Y74V{k?{|dspjj$4iszJLP&I+6iA1N@FV*mtbgsNX)}wq>;%s05tTGt@P!! zL?dz#mP?JQU3Fq~YqD&nW^L$}Mw$_1cX##&#M{kfhr3cEbU6)XM)sFEEsAmU&U0&p zA_Mc`2g0^A?V_4dxhcLjX9Z}LuKE+RS(5svZYTLiCmndAbzQgo)qc}5@UzldU#6z} z4q(^1yW_XZwn#sD%?gep6T&QKVO5s$#wVIE#g>sb1pQ8?A~;luJbG9|?~J*s>jWLY z{-Skzs@j2O0LSU9cVOvG0t>em6lL*~YX^S&nwk8d#>{?3niqR{QwLJwr_RrE97JXMz zqlJN{TPr-OIGuYw@01(FVuEXUMj5VE-Mk}Y3{o1U++Q-ilZSMbuS8vprxzByj5L}lqiIIRd0zu-ZWe|&ai$ji@Qj|f z0xG)&DL7O4+~`WcTw&IRKpe-~ry_bi*!USR&gU_ZRjU-i`F$#)Y(3Or2sftzVpS;f zfyTe{c?cN8MwwGN`p=Q0EIKeo~sMbKRHGD zmD!}hIVK=cG_PmUZX4@KYdct0+YF?~$c2B09_*PFqB>UfH$cS+p!hX*Qa`qTNQ4Uv zZN7;DAD^XynD*E`Qnfl+@!*0>z}E0!z4Yxin~78aj912_xHN}_TEqD729w=&^D^!& z@Job=zA00%h1odMyK`s?CtUtYqY#qO;G)UwEY%M#DkkEisd(`qC~KXCM*EbS>GqcBj1hJV66Z@nUuDQc-vaVRUEW_oqz``>u5jJ(1o(sOv4#n z_UXO8{2Nndx7+`$_vD)ML7iiNuM7ZMs)Cl^NBkyAhg+aQC!|dl7|TKl_d(VN^1t-gzY0K< zMOKvk+y2F0wzB>oY{gr1E#?1@{=XK&VE^-c){G`?W@V+gL$KbJRz{-KK4Vi)zy!Ig z8Vnr?1jSX3hu*y;ChSNjXXm0@KGTM^ouBs+rT3av+j5n;ke-m)r>%VTA6dAIddlbD zbE`H8ojTjh-=RO^sk*ypr@b2|<*T~P{I=d|z@xNWXzpsjLCp5b;_Zv3n3j#?oA!5D zcCl8vNfBw&W)<3UeLVNgIFAseS=b+nz1LDl)^RR{J$a1mo_k@Q5Gt+XlCA)XyzhR) zb-R9=z0UC8rV8;m4Y>~x{eouxtohsWDH=QQBEPVlqyNF|3-0NDLga!07m=Ka$J`E+ znMb){fbuNUQgGzrjK^yIW{X*M(QB5fZoCr`16nC)jMvSF9;I*?E^P;jAN;xaHQNO**eXG8M{E}DMA-L^hc1kws zsdshC=aJ2=rCQs{8K1*i5#mE~lRJCxN$fU%eLmHB>Vnc%r2Z8`a{|`)udTfxfMmeZ z!+@V^+!;@E8}y@NHB^os3u6`*ck871Zz%PyynYEQUX>tT3eN`QvmC0#v}eZk+N3)qR0)C=+3>$$DVE9=E;Impp&RqXl-V2D@tPSJU74m51UmyrEC{N) zus(RlAxE2usp5P)%{g$6`A%?zkkwBDYe=l40xUn$SzJ!^enNh3NFs?^yTUP_?Y{+g zX`T_`;1vm|9~wzC{thBwUE!rs8{>w$x6rRpUAK?}yqwtdQL97)VsZQSxKOVvhKcqV zE-bj%pZ-6OfU_)1uAv`cHQr`;+Rc-rwPZ75_*8gt*oHOy!@QRxq z=A~%Fq6@rypo6kD!nmdGH|4ush4Dm~WQ&MA8o#z-w=>1)uA${u=>~hW0X2Vv@uESF zj9C5Joi_>&llxWoLLcZU1u9dx=NSs*+?qBHE&qrkWB!+U!n-AlO40S*-Bx?BSPc{_4Rdh#;4W24KMoZ0Es|f$GzE{2JCeeEhG0uX()drL>Oqm@5T zvOR345-@49JahtLh=BWS0SMThwp-@~WH=`sZUjFvNx;@l&V5Kxqn_O3qhc~4mu2o4 zOaLyf9l*gtH@TM`Q0gQie6fiDL9xnkoo7=HKR|zn3v^c|-}8yu@&ZlkPOd{e3#vIi zQ!7D;S%BSCAOfQALhDG?Qw$6xtAa$%0l6}cOAVI`mF(+0A~gE>7u#Av9|0HWN*L9( z?=wNq=osJcWuy7Bne}Bqyf!8Mih9ERuCzsRoZ|B76xp^zB?trW)K#E!Um&*pqHhh% zlf&cnBFxvHTKHDW#5^v}@iw;ps%5J91JQ9GCt5M?+=60uu=%2xB+%zOYHTO{a6Em` z9B%YC7aVqkPoe@EUS3Wf@mO1;u$FxPLrz%X3gNCzk zXbzpNR)5*L6HRfP#_J$v%wEQ=bqc%43a@0T5Pv~`r+u!Gm%Zj(Mes}uElXcnb0dJu zjr^gyE&}31{6KtahqMlZVm}eht6(t$4BE0N)xduFvFoL2a2sr`yO3Blawkla63lVR z!}!S$_^9Clr6M|~NLg7@kg+DFd>A}}2a2#7K2FJqXukw&ST(Ac=p1LQN?&lxC}r=! zFeXJsAeNzyTYii+9+eH&fkTXO!s~BVmILtdF#rhNM)5^fVJ^FioUxEzEiX5z$Ry?w zq9r!&v(<#UdNvnZ2OWe7D+LYwPL3N<}%71B%#80v4NXC;ONwbjwW)GE45&T4*DSH~G zB?EBL(?RCRKW*7z#v`DLq^3-Usr2Ks3LNsslRvyGP^fx1qev2D%j%5K-_Ryei*ce#|DE;NGgK zF*M1YNiRVDiEzbTkj{O(d7I1#G~q5$I7+o2XXb)@bX%dG5In@(rn(GgrZ0m{K-Dj0 z_%be)TB4L2V9BO?*dbGmNSS8#bIxtOpDPMls)PulpVOoiWm&!aZjR#8LM_N3PUF}ECPaw44Q%=^`IfoBO-VY*mXlYY&-p%dS}QM zi2W?=D8Yz^J!z_!Y=&rRV*(|z=&e6?4#dA<2i(n5uao5c_`syfAf_n$>VvStR9Mqr zzXt^@Qk42`n$@j65fqrj!Gc}&lXD7n1_2c-vWz2`>^Y@6UPnqPZo^V)#HCQ51d4h| z5y9FcKwx^?CC#>J7-1x4Gdbk3?kP0X(6qljRUe?Km^vP7c3I;DP+RrV#Hg|i(Ec37 zO~A)G-ZuCtZUybYDptgJGc~ExW_I#7Amf+iM=-=6`sibY!b}jh1(+L*Jvo(wQ9uO znuNtfq$DA5sR{Gb(jfu+$DfuI$AH+Q8(Z9oG_+4!P6OAt#W=cg{Xl1{l#LV78?8L~ zUej^QuT}>n7)XD4jz)CSX$^J#U%1b05^jR9 z4PJqI=!rK%D)~x92-R34#!`5=T;j;(VJxvfr6`@rx5ak;4FZv8gVuwDAFwa9wz<^Z zKsoMnOYu4LE~l|JL%a0&EWP0k(DLjvv6{hcR7WDDiGm|)_MP#x2C^-m%M)er*8{aJ z^@Juw%3a?U8e8_#wtA8tk4uNYPA(&JQw)H+Qp6wxHv?d)`;y(Y1?7s8`y}lgl&$8Q zblB3xpLY-=!<_ZTso10xv^8G7NiU{l%s_WXLHNFCt>aYfL)Rm}29N4cZ@!q~LkF`9CF z351*egnAE?e%~yhH%Q(#h3cb*8{v@aY|>)tomCJA#iLYRn_k1ab-;-dgN0D3E)TAW zYE#C6Ex0WKUX0yC(a?s$suqj-k8 z)v6Ny`3E=rd{jTH*l&D8K7}}u%~8!FOyIPdeK#$-u1m)7K&Fi^O`IE@>3kPSjN|cp zB4;VrYZ66pAY)GjyJ`A8`E*A{M~y^Mgx!u#!Ff$D1r}&-BtL8Obnl z*=@tlqAKw$$a%-+4L*x)xY@OvXcGKkAZXvxMt`_@#X5^QUC4tw4}CVlEc=obx`4ZG zNdz*g;o3aNZa4KQ{;t2%%OZ`+pk;nI+cQX{1&f!Oc_z~4i!#Mv@whix>(U7&;arHe zs11$VRp@xj?}p|lGKmYi4iN+pIqoWJxFG^dEbPG65OSGH`0*SV!?lTZ7Z-j5%<#sh zT5I8ah#AY5AV0B45z_Y1CD>CV^0D z2aiPV?}*puorpg}X-Q6fmy-m$C19QKG;ffHq5o-*nbxToEu~;5l!nN^mMV^A-)Q1Z zGxN7M>1%{4mpGBupUE#uILPmEMZ&qvn2%>G#A;&!yi&9Vx#9KO#`yX=Jw4_7 zdEGF4FS%aNQ&0Z6&Vcmg-MksLsSL^A)Sb`$&1Vd`sX1R%m)*IoxV}8eqwJKBJz~~G zkaV?GWQ2-6>Z>(fv~QtSz!qeN=GQcHi?wMwD{;XN>9lRJs@w(W(sh_0VyGJHSr>=! z@fD+~)KRJJjQ{rcRJet@pUONHSVYR`@y3}?T#F^sM7Nke?$zFkDRIFlt##&)oISMP ztb3rmG>}7j$W1JSoG-Cqnj9>sjwZ>~ry8$RMxL)&VA9)lR%q!JlZPI*w>&y$(8ws( zR5*|?FR53F-iz$O2m!kOtUsZAp>2Fce`9lCI&-ORbYq+QdcOr)xyL?5;_%!=7^}>a z)vk7do;-I>n+iJ_Z|fSdhK4&+8(?Bcxv*G)c}hcM1;%|- zp53Mov`wmqle)SF^-xpkxGm?7^Y#mz785UUy;T_3XI1gP+3s?!*_lu|SC#2` zZi+4Y4=U~b4;YiKvC*HHD+R!#<0YO^d}>wmcdHK@%`Fx4M5DW48WajayG~H0LJCzG zG5YY^odAH&$V5I8&c>0 zF%5IN9M&e+aBt3VJPnHYW_|WH>)L5_>m<>b!Qz2ChSpptwEhah$hvXWkmtT(pR>65iosA8h*SbJ%0< zRotymRFo%fw2`9J$2}|9(@tch1Cdp}lv4+Tgv7(e%*}nike{i5K?c|^!LkDzrt>fo zp!{A;xtpGL7G37#7ZWGjNqkS9y(Tbpt;3LACdzvdbMK9DSbiWNEyzm~V0I}7)nH82R{ZRexss~D zD4CP?ZMi_$n8!XXL1G_6t7TYS24x=CUjqVuF4Mc*(yayr(9MXuwPv8)AV!hv*WOdm z_wK@84b8Us!#*xX&mD`ND-;Wz8hv zd80WN)E_)6pc?vEe5m%?V97qv8YcW<>^oL*IeBZ7oK*!MAgvaFnM#3UyI{j|H;45TSaF7LPP zeyj3*1}sq!!*hvQNvdCqog&cuK6yItObG7Q-@^*}fo zsrhRQ!{b})zRtDzY__Q)y490d@EmCP4cmYt$+OX7g@@6WPuz5CI608h^n;VuG*Aiv z^_>%g)1qde*qK92qP!7BI49jE!goHwOI=+h7sfUK%YPba+B=K4g|*bT2Dr}R3dQ5% ze|JSuIm4#fK?gR=7+p)8;DoCi6q4cEr?2Xs{ZFY7Q;(J--g{}MJku0*w68_j+m(4-gf4jQl4bwv4I0ku6y@1VO=@t-wk`jr?!?^0nRTEvOd}s+P5KS(f4IUWh z)#wVXRX{95Sht{#6<+)=lL4THbw3=0mU1QA9u^?kzj6427@#tjCt@5B5gWuMhtruz znq=JjyK*@CHJ70u7gA0w`SA=Ssre8YJvO9gyV^xRMTZt;5cLRX%fw{l35P~a<@xMC zQROZWKSHbHyQJ?j-u#a}2KJaqDv&eS{mTw`A|rQ`y2sC}JRh{}xC%J$2y+d^)#|Q$ zQ*zym9mCn!n7DPA)Mb?cM9BlbX@)v&L{xw2CX6lrj)QfJ6>@WSvY^}7<0zclUb=X?NDaVz~7HJZrunFFw}gF}QE<^_2?Dn(0S;y3kRiuqU9 zgI-Oy-Pw<|o?n*WJk39jEbFt$u=?if6@XnqHz{!lQaAu#*?Kz;rPA!k%LQj$<>3!_AlT}-sjCub>&*fkDh82R8qr}0PWER+auh_~*Y;GB-eP&%Vf;W+=eYIzHSY;lH!-~Y zBwOAapQ^FY7hG+`{h83oxB{G{>qtIkQfsp@g0TjQja{QNDHv-62+5OuknN3Gw%e!K zO;^_AiELxKT`wxX<9=&fWI)zIwt*BrG}yu)fdgpzz=m&Bz=#pfb`3BH$JI-(^I4QX z9Z^vJP{gX*g2L2LHojnHek?3ka|zVIB7Aa zwQp25_VwSwLd7GpCEMKY0M)RN6g5uyW4jUrG42{LYhiKf^O$aEZAl^3%aPPOL^B)Po`x(Rra$u zh__2?&n6dVVjtP@4Z{zR7mSw{g@GNOPq6RP!)>x!0Lqx(y)*cC(_nf<>eTJd-S>cD zCXIP2a!=vO=5}1sHWz~XIXc2;2VZ&btd}n{R&TN&h&Qv1qH3?d3CZzJkf`fy zx7PhH$gir6blH!~*kTq_@_2md2D2pcx-_ShLXbCqhWMK;v~e`(}CRy>McuPEp`5NvB& zV(-#-5kIjCc`NgFBzNFR{_#wdvK!4?7V-&Yl%62{(%dh&(U7xgT%C?&So6lzisEDG zt|g@dRpmvZ(lHTOjbtiD!DQlSjYmR%vzJ~m)J#fx8@+?2z6n)94p2{Jz#fV6mh}d+ z0IY#R!O#Mth|XY^(>NQf_oc{28li>?5>DJG?lJ_dPrmtUK=zOhrI$ou_PEK}Y-o6t z0MbvGaPrMUPv>%}eki5h=z%eW*go9haCXJQ@ndzB^i z{%rGHftJKvFGUCCMBM!rH?|}sDC;xKIGVD*b%j?Zi384mKmKyoBk$qphW3AkfV56P zlua&_Ucc1|sQ;FaTWemqY_h^W z#e6V(FM`nWBNiUd6mzAB!?(7!0zBOm6cqS=Kh&r_$}P1!{|v;s+F0Rjh`H{xP_0yc za(r3e6Mdb3{SsF`S1LB^F09X%wOGEpil-He0o&_LoA+p++XLqBvt=2v$oYb=a$782 zL)_d2JvEt@zJjif7ZtYl^6ui?x{F3@&KG%@4IlKg;P!XD9gla@+l#iNuH$CR)yfDM zo&r2jwdmN6uf+vW-?FRk4~DNION)mVDOM-D4>M|8k(r2Os-Bw>IIH%xZ(6HKsqa-d zAM7J`yuZ~I*B|ZybY5&MEI5%q*q@-R5lm}>EiX8^Z>-hC-W$8hv(%;?;V~?s1yB{7ob1<+b2Cy zo_cI(j}6#LDNx6=ZLEjXE#KJtfg^Q)>G`>E3AIsksx;X)75nbSmQg7F1A0I|X%ryo5J^bR0Y9F%YjRAdk8bl0x9!y6& zm#wK{ly9DedMwGOFVbfW2!`KYsqlPk)^fGz2)}#11H55}Gk5c5&)B{i~{1 zg9{3&P-T0eXHFBaQG0o9t{jgOz#ks07SxdV0Bv|!c-W;3Ov9rm4e)A5>iLVi z{zHn_`!Xpdy}<=%L7X@?by2LzBZ0R0jMz&b(ujS8Efu?>xKuet7Q;)e@ucOF#_x+% zo8y?I$|DbcC5`|#IjsSyk@#n@&pcpSNxyw8J#sycFAR8&w>ely1hUATlJ$_9C6DBn zYH~?V>Mv*m_vJ_?ZLG>P@_TBr64tQ57#cOmsre(}AJr&Bhu>9?Y)Q}uI>TmRBKm$+;-e~y1 z>GlIgROFwcB20YqTP{mG`GF9iB$ z#vqFt3PSS|eQzKpSAS~azmDbh*t4?OJTz6Tz?j3zh*l^jfE>4Jg8aE?5>x|*J^1sZ z)Z`d*nM_z!9+-db)Mm*Z{nkP%O$E{+QQTwo79}S=Uc!^SX;7h;dWzsH9`y~N9M)V5 zyP9M4BrpcmkCTDu4YS=!yVn%i5;-NbP)#| z)Z2rDssoyl{+-Ohr8_}JOg3D|jdZL@$19lg7rF17poC9c%llXi9vbaHhg$%{t zN2dJQ#&SnW)Y+!*^`VilKh^g11=2v&6TPCR{x)O&>270&L8QbgGNv2>pk8HCyP>A+ z$rG!~!jyV2Ue=%h9Oi_mSi72VYJ_gT#}jD^s#5B&bG0$V6N6c}(KKV+6NW3EC3e4K z{wJ7_m9CnnRcodUB#yM5fBD8JQUu2&(4`(>^iz;Ca!@I=io@obbECoUniUElAYm zp6*{zR#={qV;&mg9<`yORIeQH4CXu1*uf~|XnRZPcY_YA;f%Lq3@9X-bQyS@zW#1{pla- znY>P@skE!M;qSzF!ZAc|W~Izz<^&H^9uOg|s+dZJIJCcly`+RRy$c?i&7O1tMz}N# zL1NIl3~6r}*6;`K&_@gsh6<&#LGC{dH5u9k^*VIQ$4Onxj>qLcTr8HX7ME^PyY~@J z<$cihhak!!s?YREbvn0bBDVPz(2NA-2f_}?r>f7RRI?RnLM;rtUdn8faoG{}-V#~` ztKO8U4X!{dz)}MI5^3_e*Adzi;~|gmNX)hOBHJrjprjG14&WS_B0fwiUV!W!XEqE5 z{?Q_$ecq=V_I})r%lyG3!SrB)aRfa0!0R)t26=gg5CZV}>lM**;H)LhyJMUeKD$ch zYH~4e@J%7OI$}{nX=N1vVvCWTLPE9}uqb&m=%(3X$iV>(ia6NWuAcu+JHX6d@ug9X zNWglK5xg>i7_cn3A&L-g{N)sqz(<96Pw^GS5)Ba{ZObo7jc+XslWg zEJ{TWgi?I#W{;>C2Y#|Rsd_^$PbzEO!i-(X%c}cJ`;RLMj}5Ocuuvk_HB5#~=dQTq zm(A9bhrVxFD~HN<0uGhh7>Pz_Kd_79%P^Ws3HIS_rc{^oQQ8PprHQ?_mo^Cd4t27G zMvd5%esD?MsiKHAr3y9lrlLeA6{7*i8aX>&T>xJ?P0uoR5?tg#4CF#ngc=R14PbJk z>i0cIUknP3q)3S;H^<~jq1la}bKpYKg!I)88@v$&p8Bi(#C>-W-o{rM{# z#spt2s6y#*FTF0}A{!K-ypXD?Cn%K#o#rdbeHH~lo^?xdSCMLn6AbeCZ`%KQ;x0;5 zhM1+bGpoG~tr<2W1tr?OIA^ueQTzvZ-QfHSxk_623G3`@AD=68abCBEUmA)e&^5MC zI9|-WAac-(6JacUMu*P8s}!qul{c6Ds6e)RY2WD>jHp}6+RM_$JV$f!0}p$WAU9-J zvKWBL`i`Wa&VbOAGEZ>7je(pvUU&OUn`&yk2?;-{i&aNFgV%h{NWVss0D*RgywF-G zro=bAbuNN`s<;xf`mTUcDZtdeBj6)U**8JiXui%&n>cwjo*T2Hr3C}N;aRgVcl5k( ztSRx6XSpL*EYe?OI&Q8QtBL;KIjil^{1f(%e^;Ie7fFkQz^%B>UQSi3H|2%)ylk2~ zUh7Dl+FCaH>&hP!P6;Y z2(4L%Jb_+0iGrcPNB>W5JcvID*fLE#n!tvJ+OT#X2CgX>!jltu8%HiG3Z1rvFza!a zgeqOm^hO<3sj&)8XUkk~oy71}cwzS@m4LUKtfkB(zu)m)GZNqwr;_3}x=(E@hYT$8Q~qEL}<8_p>D3C|MGlUZ}(>O0=c0 zy0gnmOf;c;FR8O0875_O2S>g_JxUQu)?Uk#!^bD$@|P+4gDAkJ^vE?ZX<8!-V~Nt` z2Y9|=NDHAb{pyPNu*xAKAAx6O9vg(Ia0Z0o3Msi6*(Wx30|2Z{Mf}Bo+if+I0;}yd z$kIc4;0Ziy%?v!5G5(bKY+I4hj%X=6*qW3TMFKFS%mF0pXU<@;)jOo;>Rt7N)nzYpXX={$(}ip21p8668W|vqX$!p z?U6U)@N&-AAiG=wN8eK1s*H0SQzuK@iBiM(vi@r`_~YPZ)Ps1?!NKapJbm_FhSzT5 zl$qxXA9O&;&f0aNVTv7x1#{4OapGZV12@kW&81z3jY+Lpi0HEF@&O2b0JJh8iRApm ztsHzwY80IrA;cELONPlBQ(%5`s?H?0k%!q50#Hq)=1{u2O3k)|_utkb+CT9szpr?YXI*toPU&Utv73QzxELM8jSbklLVt(Y@Z!GP}Qi&XgqA z8i1AnsTWFp0s2n1nS@S~vIUyZ;KL~?1VSGRC8Lnhg?!M)r42o-s&TAJIX&-+q>v#X zB=S&e`x+*CfS5V#fGE^d`31YWe=cm%?SqC5hP_rPJ6XD8Hl^c*QBeN+XvQ(pjB@`Ipdss_}~L^KCSAcWOu< zCxJWJrF8qqkJlx-Jl66pf!0)#oY|TV>_E_1rg*-3b{4N>7+i*0 zDv|2^&&8{(3ocp%&O%$&RQE21?N=yiS)TPwK8Tg95q4fP%KVeJGy=F5)#_uscq^Y8DC#)$Qig!(gwMnYf)o zNIM1|z#|b>7Sc^!`uV@9y zK<(}sgpu;TK}V^JeOeTw_r=0OXTR|oza*wITsTc~xS%B|XlSyo9vZDz;cgER|Ce%= zxay496vQfFC=N855$rKCsw&nLGLZ|4JeBL;BS#K0AAL3S^FMVuOV*s^wU8AH=lt^e zaBInIY|)FK2wS}WZqW>YskdEtk`sym?EHkqX{Q->n@V;F+EBJw^Ux~W#mv^thGl{P zsl-HahBs;c#5PItA+b}lL9PzU)x8DI0m8HA*wKP$Vq@4>@>@(#7?%49Qj2j3^O{bthf zGoHyI)xog|hfZgrEehu5&SMU~-V;0!CYMW?>Pk%ib7t-b*O*XWTu-OBV4d!d?V3e{ zNE`1>{2bC?c7>;$Dp}$%GW$ut8ijH{)*UYJubjJ>bRfF-Z?t!`&lI|MpG1hbGEH*+ zECz3|2%I88$6xFMzrn0dBZX$_O5agD;xlS@Sc907RgVaKn5^7cbkrn$T1Od(lcah^ z*c+at=|^!hL|&j9BsD3_8oo-|fcb;m_>IxTh3&m}=nfJ3Fo&0zcqGrv`oQ)_Kj2{0 zH3S4i44UiQi2mLT22dq#Zz2cOooMQ`acx+L#FhoLcJ16EH`Z4H5|?P*Z~qL4hd?y< z9Lps22J{k1X7E6<@NNy%npL(14_R?^lx9TRh&p5JkPufdd9{M9-5h3q>GO?p92p`3 zhg$5WSgGK1CbgL=_p(Ic$TWak!7k`2T!#Q$aSCT7qfw!-x?}O-%Ne5Z$*8KqG(%Jo zO2u}RaF39c43B~c{&?u(iW+cb^Nmb|K5{*s_&=85K}+aEXj{{mlp;VfO=Es^O4#nP zfOTKV|5%X-*E{#cK^giQlLqej;m)Wk^TKF2bJoL`uhW+bzHjn%8g337VjadfPyl&dG z!wl09^-Hli!Cr>SOAx4?1fyjooMs_M>wvUIH9}Dx2W}<*ox|CuxGJgj4QyAI%m}dB z;~J37JCuM*RflIMW<7Gd8@xtDL@M{Ee&@Tx>YWaL?*);GD2NbCC5pV=n**-qO7q|s z5~Nl!;RD+#WXv1G*smyk2N^|%6&a#TqN|I*ymv3&_HvkyY|=)VA5jK5`cr_G+rZrL zrF)j(mo$fMl-G&PvNbth_|jsohc^}^+5DV|u579Pp>5RkvyvEQ}} z)xIePk{Z^pMOknSopl1)39G_!99i6lYIsULmYC&3y(@4M2!a^lpi}NOMCw^%LMQLW z)E=?ni8+)%kVvl71ij-%tRm9?Q&;tndg0l4DT{IO#w+_87x{L^KjaeAg8u|D#7CKh zcg4Sz!F8+cD2d|}l1 zo5txFtS#VB%H4GUmQKe?AkRb#CnaC`mO^HYnB=_C*TpN}kn=<)2k-79tV29qYf8KT zBL@!*hq|;0a#xY4l&g;YoVv)?`_Ec$8>rv8qvU|T97lNOxP8@$jcqB!lWc1?7-JLj zjPuv2`Ew&PUXl@UU--=nQ@XOcaI6$Ou7}L1aW~5uvrd4{j5WMl!zPl|as(q1d=B@}I7v+9AF-@Q`}Aazc`p5K4lDKXbVMk9GO5YiOQP;ZC`llE z!+fMbBitIjyDQDrUM-PjUb|%mErHAalJT-lo>v9gFLiG+-xsXJwRh-opW{Zxnp zw=a&K{iWC+se_iISGY@S0)!LxCx4dVTtUDj2~Rs%v*#>^1o+;v-1pd$Xa|hHaZHMd zMoSD%MCJYY2cpG1kCJfLlrgYPSWtgfKWp^HNfr3(L3Kog>G+yUzH{9}^f!KK%Hfa) zU=N1^N1DReu1FfdUz>O9f;mj5%zj>Sz{k_GpW8{oynd`e`kPZTPM1SNFBmfl(N{zd zYx*hsIG-To+Zjc+N(1W=slGGFDl_okz)zq1Z1Y zJ~z_GI?TQKWInBgeEJ%;H7~WSXc|LYxlZu&w4VW-1>RE;7?m>fD2+MLX~GPc8IKiIYZ33 z-zQC*MRPx}iQMIMm=zuVTeO)xas?paqW)kBi>?JBRV>)AW1hcQ`sudgJON9yH7S9r`Mh)e^BVg z=FABS{BCumXZCMxVHD_F9f=AQ371FkQX!ojwl)!sga|By_=`@5#Pue(&S*b%mb0N6 zOB2Q-c0|x2eOjyHHU@n}$>eM%d1uh{#s(wL0ek8>Yt+mug#8hlJBj9iC+baSIhRu~ zv++M%>ee^rT;Q>u<$)v)UHj&I4np$iH;?iz?LsmaUz12xTTKuMJgiToBJm`+oV+Xh zg`*l+i9j}iXo#qFEEaXPvL9)qde~`;73{IN&mrG_|?0bYVGw^C<^p6X|<{>q@ z$vC?JRp?J!@pJ)lbJlNZ@=S?{I8tr% z`zctS5}KAYdBU8-^B;ul5)rF8b7qO%HJ>n!gc!6f1M)gFh>{9~kMlO?_R^R#zx>92 z9}6Etz&%lgAH8S0IZ|_3yf=Flk6ZZO-Zf!nfxvtW#XoPN#{_*><#ZR(*_X-96cTdJ z2a?Bb1aw)qTmmX`AkX}7*>MtJg@Laq+v@C(`@`nj7+C}#5z4Zw8vQb-uN2*3PW@GL z6sILk3s4xD)ot|+&ehmd2D-X*Mmx@a^Ks%T&PBw|{j*iJ7;>7@ZWt^yGSxH>wN3A$ z2`-&)b|mpUsP7V{Q*=o)exaYYELL{0e}rERC*xm4Jp|p30;ikcZ?Iq-C41S@Lzr7{?+G^E2D)%{!POcQTgzoe(CT7@^D|2B+k`Kd;pX`CXT89 z@`c~bw9$NX(x%5b?JBrkYDo-Aad<-4%erLtPk;c67?9f^%y4S z;sD{Vz7ua-n3W!z*$*O=>6Tksp<+x?sYUk-;K4AxSK8HisYi4u)Fwr$(CZQHhO+qR8PI<{@w zws|viU*|lWZ`iwb)ml&Vp%#^_d-T#*pP9UsEdy$uTd_hrtnX7w@BO^^2e&8P-&D#E z@*$qXybf^;pKZYwr6k5TNbfCm9T%3C9kVkkLvnT-xMV3lW#`@9B}A?ht{d}IP|yTO z9iG*@{tgz$uJ~b_-iK*$1v`JgYB?9DY`UQ8{a~j$YcoBeH%f{~hHrVY*O&+9H{R<{ zXnfW~Dz|2to~G)1Lw=46qj#R}(j26Zj-FpK1=uReXosoWOa!`&W7h{FXTN zJc2FP8-J`--kTB6bv{8f9z;_@CP8+8@^46JA#chy{WZ}!9d!3+v%gYENU@9M?HDp& zc>y#2YG)5;d)M@EolV#(y)J}jV(8FJuG}uLPTvBMvjVU}TbET}f1HwWa2qT*@trTs z$GRU$QH_ZmeSsxyl$<&jYkJ3_hynG%U*NWenTjToYFF&X@7?hUSczzX*uL~mvrxT8 z>Uv`=m3_=0`a6b__F#Ca@lJK1gK;k5hdX}N7|w8hy$N{1_BdnB_PIu*(UoOR5 zqd%ESPv1A#94AdzuYkh+ICk5Rrm+tL_~X{RS1~>CQ6Uxfv*Y6hg{P@xV=WLT3?-E0 zm$$YM)Yi}HwWI?fZxe;Wgt%71cp@>!^|Krb5?@8CM`ZUAp$!;2nR^gf9Fmbf9Wygu zY*tiMXG)S)`5=)$Gq?=p=ub7Ns8l9{x8QW9APh?e*nPZGRm2^O+KO=c{IL6(jk7rs zuzMG*rew(mH5owpfoxW6MPJuvQTHK(@#kZUOl{!wq8u>oe&qR4wg=pFB_Y+wdbRP# zCZBZY2}P$E9}c{Zft{4zt~8x;IwN+sO68TXx=$igk7(Ks+4hv7mQ7zxpNJ1+1Ar8l zqU;u9!vaN*_rA>x>Zqyc3T5~h>QcjYjR~^0Er*Aloqy`zEi=wbo^LVjh%F{ZFmFbM z(jXQ-#;^kt9HvWxu~$I6`&C7jvWuh2>%rjCp;L^b<|6_=0}RR}(kLGX(+pYa6)Fuu zi*e;Bb(}6hvhr3Vu~oqw8BHvp+G>>;-^kMLK0G(!h)4$TtQ8136ZjvWs1U+Jp#!x~ z@#}JH7W~siY2^{UM6sMANg)ModctDsE`Vqz&p#w~^XEJs_#S_I^SllBijoD@Q zgP66~!w|n52Ne(HXce#tMaROE;G#z&4O#Rt}@Y;R{NY{00vQ|Ho6JV+@vQ{dnDYUToap-%csj58*{Oh!N zTHI=iM;}JUGUKrrccu~e2c9kmj9qcO=)6Tu0>kxP2hT_}mNh_`MC6eeZSrB{eK;d; zb4|yT!}fktJzU0Yx!X9Kr;+QzYKmXdhfi;QKdZ_{B^hk}At>1`=a2ARYKvHZ)#mII zm`!pY#uiVTj=uL*1b1U~R3zarR{=PVNKJcL@L>^9&Sy^R7hxVI{f<- zGF2ioq%i7RUaL`L@XwJH;$rggSr?P2R!ar1V|a{#psX$qfecd%BRuM`ycu(Cvhr z8n<323E(UhJy{&iA1a&M@w*^trseb09mKOoukkvG^YR@lThhgfE22Mfub?BFmJwIkQF1p=FCyFQ``1AW#JA<1#!J|B15xG7)AQ) zV@~3S5KKh%vIA2!J~B`=_qWB6Q*)y?r@|fmN4^tuIdx2u=0~OWTTHL=vfcIm4gJz{LUgx^jhxVF7R#NMhDRqqfZm;`jTFq$zzhTY5ueQl^#;w$&-Kba2r+g9+QvQLHK^ zKEY1ieF>oU=1&Q%Y}F4P`lFu)lwAglRTPyYGwve{Ql9^vOQ*ryIbrw{$U>Moy)Yjo z`NRj1b6#e-y4|&!^D9E*R7m4$4r*p`6EJCQM2n?4;5=QrxCYVd)iiP>_EF%7ob|>k zodp||uM*Sa&zRe!&DE9*v|a8R^RCU<2!S~y=hYo%TI2i?J7`MLSvgZks6Iv1(Y!ZN z#_o|85<`BLBILzuljZ{NAp`Dh&wuw59kg0~jcn@d{fKuB~_DUkdaN$AAwsIr2^*ZqG;nDMMm5zf~a`XISCXG0z+lIArE25HX7l;wN}1}qOPZo6SK*t=_vRd zYbui|u>k9Gf;XxjN|0LJ+B)yb@Q*p8Mn_X0(bgy3P&l zXmGx|zvZ|fx{^6@=j`tAo-Wnxg2{TM;LQ4&H2#qQ?3#X5t2&T)+!8Dlq7vM*M0V`) zfFHv0gOHCkX=~!iTpADY7~zwozy?$AkTSu&ej9G^=Nld|t_REb_M*j%5qppzBI(^* z)+s5&WHs`!(Z~!fNy390SJZ2u{;YIMsMoO}YWsc*i}O2r6HmTrwjq)uM~4^U4tVU30DUc%_40$>sU4?tB({s6~4-6_jUK=-V1A(`2bS%4D_0 zhviXZ4U`ihmHgMA`*_n(wVt3`dcgcO8xvJvDR;`y!}DD5%AgvhMeQ$M0b++T?E~pUD@y6sD`C6q7Q*eNk!DfGKNU%Mh zmRL|#S@30o#op}58<#cnM%}c+PUAPjs=0{eY0yi0Ic*N3)p zm=GbGbzpUWQmS2d7LS>cahc>TbP}kAh7#zo5F4bfj~j*H0}=$&oc3CblO^!w%k->r zOpbOycH8UU3u~~IB1CSl9pbTebKN2&Ro9i7KNQ&W&M#+uV(!uw{{`xGqkqwkzq7Xh z8e+?`JXf)25Kw%fvn)bRi(uQt=^?TZ73a^~_fpSPd0fMcK@%A3~fVP3O*E5P2?!jvB}i87Bj={lu86g{AEu?BQiM ztO1vsyml&1VNO9D_p`l}nz^{gk zYwDma?Wm#PIH31x_IH-WTSWO=>WPW%G@OQYO~zPp7&8RNCijXCwQL!nY6=13JcAUr z{w}3dCA`l*j}4I#_mW}`7m{?I_%c{*2Pt+UNu+xKRk*RMiuK8%U5u4NR|hQD(R!Q& z7|Tnm;pu?nIQaXuGKY6oiV5klJO^xbt_MvRZk;6-n#jKvV;4Sm6rywS7)aYmYyuzz z8m;$eFGWt;rS0QD)Fu@<%ZeyBj+6KE1)9T%nQ}_yAUwt8hMYop#IzlvlSKp)3-Lq# z#f*^-=-at6NKsw2FAfFKKlQ8T{c(Evj6k~m_N79UIpy}{5#2hC< zZ8-?XNMIiz3|n5E8EB2IQ*5v|dD0;Fp&k`-Mv*z5$dlVDu zc%Gs!S9DNGub^ugZ+`s_sNCjD<880}VBHAPN6wZr2~e8hlZk(YG&k7$BAv0(vgOSw00f7p8}g7=GDf*zt!x-H{e)V^NU^ePeH z*9}wah$l5pcuBpVE)rvKZJgQAwsq*q&v^8gfX`4}+iUa*8=-g=WCOJA=$3s>KynJ} zRdOOUWX?OR!s9s01uv`>1JJRYnraItW@@!uxTr6aY0jBpSnaNK38(ATiwfW0q;Q-F zEh0=nT~G9&L{ZhsyxEJ~Wv%}+pKeZ7Q9G-lbAeUU<~_LP4^gv+WHED#K!@6gTyz z4V1Kd*VYm5cx_jKDkG8_16nf)JqzGQNCvuIy|0 z=X-R}>0dk%kj-mzg{HiNzjWjCJ*(1Cw`mrd#4j&d{9O{~PkckUb1o7K1biD|@s;;e zWun;5)4IBwC->It?6!nk^TcuYUfPEaEHoG*~k3ef3kUXIO>K z3Ta&t>*$WeuCi=7gQbpPd*Pvq>tqo|1xex#R2TUeXu%x09*i_28Wl&#$}v0CYJ0hh zW3?*C?NU&&b*)XQm2x=8yr)O8upy3R-3-RC!c2L_c3b4(mRNDi+_7!SN~gtjA9iFs z-Zz(rb>1X^V;D_F$X-w2;9T{OP`1-fGf@M>0~;|w%}fySfjF>5ugLbg2P6%9JQS-@ zE^bKp1s)JedkA);QAF2my(%DpaODjN`%kcJ;5o3wJ?NuGFe>LvO1&y_kEN%I*b*9a zz07XK4|bTnb?kjKITL7ohK-SB0kD^* zP`giKvQ1Syg%}YQwBh-mZIg2$0W4aE}f6Im574XB9IpMQmp|%o> z2WK+x;fia2CH7(p0CQKPW;;X6$mC2e9C*B4^hQd>dhmjv>xlJ;7H>bQtEUKlXZ!nykQ4iXA7UwbsS_ zA3M$ugD`3^R@-KVBgg%{6Z98l$3>QzVfOmDD~RU6_TDD3vRok<2u|q|;}jruLbJCD zQ58qL1KEj-A@s6wIgJMFA?=fbglRT7Pg~60>D$~l3ld-)RBdeAJfg+uH{XLCT~_!1 z58qxB0Z|I+P|j{Qt3a?(_Wvgr-Qt&vj`-xg(6d@<(`#zMUu_#S2iWdsAJN92N?BDc zkP2C1Y!u5yma)pDsPX`nmO42ajz*^&{CYAsH|OW~AqM(mJp{fJbvn#s*L!nBrrE^ni5LuC`<+rO;fc0);xzd2i`sjLoy_@5X z^nup8u-RAViv8YFG5l&23*kE4HN8t~!`d>CeVAg2<0e#iwLr^ib9yhOX3%j}n4xl@ zyLgy#MHUg0<5qBrjr8S_p(FSGoME4ia_klG{<+IHT zIE@^4t8?w|?KNk#O;3%h4Ij)BWhie>KaoBoH%s-OdVp$;p;}P2+@decUV2g3V9u1y z&HZot8)NYFa~m6pxNK036i*D9S4r ziD&%sN3?rEd3lmm&olezfHi5M6#sxh7tBsk-`(o6qS9XZX;LMeR+x*Y9~Z2XQ%V3# z3(p8r+)6>*7qLNU$r(xTHLQ2K>#jq);f+e@+tO@}&b%`G-&P#pPJh{rn7C`I>9;dPl)Sy9MR1>h>klO z*@f(;dfE7>U%-b_o#VDIZ{atPnYMdZ&rP>VtP0p!Ya$#dlb*rLM%gsbc)O8N8Mhsv zN*N7b{28=j)3kUx>j@B})u3DmudHx)^i5803=J9xTa_LWhlBAhON06Tu-$(TF6UlI z4v^9@@Q^cSfsGd}YlgVe`L=;pQDBJF+zEM*=r%MW)JKPu{iWhFc`B~TYfa2{gBalR zq@_sLz&fi`{q`ywZPDBG5=f=7cDOPj&7uQ)KKD=nL+k+6n3dql3t31!9wmlH>R;_k1&ris8&c((BqT${ z6JJssE%8cjAmk*V0NWC1o0d>2%^Y`+Bg4X}PD=yCcFr+w&I4&8iI;Ty5|XZ%FG}g0 za^I%sA~%?kF!9F6R{+aaAYKsidTlRwsNUJ0u4E+8rO|Q|X|ggz6z7}nLB6Wxfvp{~MtdJi zuS<_J-PX4o`z6VqT zI%=;1(A}d($5YTc;<#3p{E0bH(}c)Vq5m~5hu6A`Y7Yz=7^CA^VQ9oKFsq$Uf2it+ zYJ>V&;`d$^Tb-)9ilKUC`U;n;20gu=Y%`MSl-o^82cT2jQdi7!k7CG-^u9NpV=JLh zw&}Qm{!G3!X(=2z8r55KxB<5tE_Eq?S-FL;J{Bu0zDOaypz6OsoCLzF1&^b%)v89t@&=5pg1!3~P z$dpq};cz}Efs(Z~`%a{y8l2j%&+L|kF_f+**-c_+aZYGX;}>jcik*PLn|4Y|&#KB6 z6s4_41Vd6&>ZeQv(L;VOKQ=eahs`U}u_{Rdn7=>&LnKDyk|0lDU35m-3D`A5S6vFN zV#Ow|7i`sh3sYfFJlHiX-6xgkJ(sL;G3;f;m>>tYST2(2kA7kb&iDztr+^BHm>V0o z@hzu#8+&0+bRev^dQ3NYtJ}3&ZM4Qb?E4XpXj&1w21W)}Z@UYn$=an`gp!vK;f0v| z`5sxH1@R*`r_(p|k63MZY}ZykU;K`OHk@N(y$4*$X7ANn-VC9t2~>Hy-SZiQ)1dQ? zp*J%Hr1BfmA>1 z`#We}DQHYU3}|P}7`$T6m4Xww(r0FjIqYIx2A5YJXM920D4_A^Y&s{DTB}QBlnOKJ z(hWcpzSWRnSWSe_7ozit42LD!uQ=YoeJ& zXuPmiw!|WQ%Ftr+{0|vB7*31UrqkH(5i1{w?>R!491a|fF42!+wIq3xcChnJTwncK zL1!U-W)Q+Z1oggYxnM)UW;p^&!gHBNV^f-3HcFgecfz339Wp9@(QeVf9Q9ZA#GA&y zjQiDTb|-S;=XhR+w=7~O6VHr?|EH2qO+;@O{5rv;J3QhJs=3ySduWf&F>v!X6<2IZ zL0@~5h6IQw>fvLLlTfaNBt2eRgM_1y5iS{cFherTJ}9ROKQywnJ>9P0@p0gc=4E&_ zJ;InmY}dSjIYYz(a?7R__{E2kv(3`QEqyuQ&yKr_S9rn!OeMsg5wJgG>>@!wq$#n% zid@EI4OadQX);Hcm$0BN?r0CHjGz7^T_Dm1_5{C-^CpLKsgKVx>bEW*f$qc?N804# zo{Miit-e!#Dp#xFZ`*MtwpTFdM(@~mIT9waJkh!eNx|2|>^(hlO!sfMF@yywby&CJ zv%*Ce(WzEd0!r7%VMFE&4U7nN`sysOh9beK^LTxIOz!YUlP~nZ_g~tS5eOWsiv2rf zSocqwaamP&sJPoj@O2A76Wl!&wojOadaY@a+tCY0lifKM`*M=z$jGEk#5NHjz19P1seq1^QM16$m-RK6~p^(=ZMO+7{x6= zDSm^>62J;#=M@RFlsdLO7xerF8c7^9eFx8a}!h(eLI66YAG4H?c}`Bzz9$@JI7DgYyyNqGAWx@P`yRk6c9_wZ7kFpW(9_ie2(>BMmeJnd3)a73OaQ=Izu_EwD=?@EU0g1F1+=0F@eQ8;5Rqj-)RTm(1kEvT7c>$wx6w8^ za&&&`!B_&`^P*vJv1T8>it~ALU42}Ze}W$f@ed%*kq?b3UlpLB0UN*b=L?@}Pl`|5 zBpnVY{R_iGw)y@Ow}d9tN&MfIR&VX^S)UzcYJczh7jm)rKVBsM)6&`$gdrb!&^qvG}pLO#|BKd7rh!gQ3H*r7J!qW+s= z`Rl2@&BbF0y<+%f7BLSnPAi4DT%7IM^1(6DtgUEe_*pWXao1G-P}>BJ4g8x`yP?mP z{rC7vkz3o2(Y?^o;n{J;YqV7Q{W2S3xz+rax2sZVu;j&t4{j-{wa0*Iw$=Zv-sGL8 zOC0sp-+*n)(P?2m=xNr5a*Mwi!pYq9`w5NRFlGC2YrNZ@wDE!JpNm%#%rntfV3?t9 zRv=6d#D{^UgZdpkhQrx8LuQ-}cjymw0Ka^~g86XVBXmcyQshCQ1X}^7?JAiu$8)Qu zsZtybhe~O`2EF*odYI>>z8+=(QZ&}8Y=V-E4chx#Hm*%*PoRth{OWA>it!Ut2E7Jm zy_XCelD_SF?rd9K)`q7Pk$&RtEi%swqL-{a!{@j3j1%61#zjTD%?Avi!8^pwm8Bq4 zMlq?HuO7rW)jJ>-jL!mPFi=eTz^H2=6TrHSi{3AmZB^DCr(+`&W*T=Oaw}M6mCzCW zX##JFj{|jR*pwc!&$h>EFNa}JeISuO)m_GQAisZc?QJnTjh-u2E-b$yHKR=N!$&_A z+=cB1H!+5Weg%P6U3jjv$la-4aq}?%3l#Y36*MfY>f98YoV1~v-?d8`4wWg5e!2~$ zqIjCxUM256A(%d@ALdI}9W8hjanUR+a zZ=kAZO+?kR9pF5=7rhcE0`TstU8+J@^M?g)nyGD=E)Z*I)E#!A?rl;|n5-f3;KD3` zn3sWxDZHusi0NhLUFXR<0&sMBvPq?FdNSLavmihwyrKK;jlYHDp9C#lGh+^yfi5C? zFIZg;Q}Id?0Fv9(9F%?1<9v;v}lz5jH3QXD%jm=ROBN`efutaHQ{5F|%PynfdBv9sf7rmXvj+xbL z40Po1a4$1nh|X{{kDL~7>x!)D{#sBcgqXVh>i)!!I~X3xVBc^p3&2&GGEa+@A$n%;a0>hYnW#lNR`8# zCwl3o(14>*)U{|;;5AQiwT}W@S6h9;zYyXP0b1~j%{h)igO$HLNxT+pETjpFLwlUl zd_b+(iOvgQy5u(E>e-J9W!&~Px5=|G@GWSq1n@eJ7UW2aNIrT{ zGcFN6t#QPgR_)P6T*9j8tI99BR2oK4IWXgLCU|J7r#8xm;6E^(S!NhPVRita_1+5-2%2-0YqT$btH$amm#M5$zJaYqj}_m~$`WL~H^w_& z^PXqIe#Hv-)-r)#^l(Nyi3%IFQsR0Hy#=Kwf>cEFj)l|B%PpZo!7zN~Ac|&KM+OY0 za)-%fp){h$R2A6Lc_fWM1tMZ2X#w%VjbQ66YF;a#94ZAnh>3}8E?QIm`YL-i@*HU~ z#eXm#om-Kf1p3W3en|VMLa>V#%?rf~=<}9?jyXA#dey!#CrS(s++S;m6`F}1%TdwM z2Qg%k*tBAmE_Aknu&nF;WL$)r;Fs~4Z!IsjaQXU-845P~D)2$P0U+Ze)Py}C+WbT- z5uJVad>h_}AP8xU=`duN46UQ%qS@Sb5VsLt#apPJZ;_u^)ZGZ+G389W{?$b*$xLk=56iX{>@#6cY=@>R@-yA1uCE| zme?mEPw_FecH-*$SX^^3oSg$a@Ya2-UR2{Wa+(dcNrrWD zhS35b@_G7-_bEFN`?>%m{Jn{svNG zuxMtfYAenFO+iKEwSCVRBLCC60%**-a?!NY7TlB?R|ZXb{0nI{BAD|Cqu;V2|B!uH zn=|mi#rf=(#@|L=R(>0VF3q;X-8~;QS$Mhyw&PEo{UoR0L~(Q18}!x|aefG?s=GIh zc47T>#F)ESL)7N>Ww`A<6i8C0%Yn#k*$0eH8^U^0NZ;>9dV-;_9?h2WdHi%od5%`k zFZxiPjF#W56?NYzCbw{Jq>w|GsVd3f8X2=O|Jw^F7#`!fDV+DLQ=UZ5X|r@18fG~I z^ePKIBjC}iz_F3nARZ2nR|0f(!mRd*ndvmo8~~1M+pEe9_Za{DN?!U`a=0@&^)<%Ug!e8h<25G)cknfbgGu z+%Z}j8M+s^__2+8_cgkti`$6{PJfO-J`PJVs&mOGDcu(OmqB2u?b8&I@3jt3n(2O{Nf@2L;gD|JpQI$lYX zD#vL zgpP4SVuIR{N=3**N04Lwr?&$OpB5jVb9uP%9r14 zru%V6?t-ppXJ?%HWNf{J+5wt=d4ErcHf8HTAGx&|C(Y;(pTNizY4_L)$<;GT5GjB< zFDl5uUk|*Qx>iu}=BvYLX_8RZ2MOM82e?nGf2o%3v;~@igWgdw{sBZc6yP$zXyme` z6D}C(I)f!+TW^eqDy+_VLdhNDrq@9$&4Xz(jZX~x4;j%lDcl)2>lVs5$h&u>eYk$Ui-n*Gxfrjz@9NGucT^-Yk0ixY zF1O~`7(=6=-ed$ywN)wIk>a*GsdaA>T)NenuWRcs()C(#B24=>!s=%VlTb=mUHV@%E4lPcP}Vmm&3U^$rdjfmUH&$aaWt zef~ZV0c^hR!QS&Qiw}|pURRgmS3~e*R8nun(bQ=VM9Znn%e%=Dp;^otKDoaUVf`R(j9pCVgtXlSGU%#T(-@*|(>zjwvrZa|boZj|WK z`S#!TT0R&G016liH~;`%)bH;UB(cT=8L z#ne4iQU-YYZ*(?eROUh)k)qOWKjFt~D<)i4S^4yHT<&`Gc2_W4TAF*%cJw!{;5+EK z`@(LYODooW-moCi*46FCJ>Bs3jjDyTmML!-KN(Ul?CPyP=UmHADlx(U0@KlD{!uDC zEOEI*`c2%tP%e?S$iUgv2d>;AN+dz6+n-)cQ@u*sOQs-|oww7gV*dOd=s*H-K!;8hZ&8PnbeQg06NjXpS8e4GRh@Lc;8ea3q z;K1H-qTUYcj=50u25wM;ox8ciyy~&6gdtZ`hC`P#(wpC2v35E^h2epqVaEH~0kuVM z7xMqH-iKcS*2D@~ST#W#98tT~nj1j-P$Z1VmFEpAM{UpMGn{mK3JTe%-Rzh8po+1P zS~@h2=77F3{8zOuvW)s8t}b%hgwX~+wCr|H(oDk6493MGCi0=XzPk~7#eLM%)%lnEcV zby!0Bk|*1`w=OhpGZ>J3d`+}V%U_;VaCScLUD2S#-FckI1?sZe(_w@y1h0f8y*fDWDmJS8I{2OeU#ISa z|C=GfQ0bhJ__T+A;D9pefv!DYA>C0W{i&vWibq7Ikuyn*7N_?d(wE5F(V8s2d>^gY z^AYhZZ$)M;f1%PPH}cZwbybp1^v!V+VWzxXVg(ibM;h04Jom+w0u1wKVx+j& zDYw0O{2J6uOczCVtOVlyn*E)-OSq}{FYrIMc!-a}O5b^)8ygUgm)udR1TiMmhysW{ zR=&N$cGPVC=tOJLXDd(AWfN@LvDk&%#w_qOvV~I*hP-})hZ$I zM*a=Lia)XSoB{pRw+=aav3u|)DD0?E8}sG<$Ek?s5}MD+)=pw5!5ll47QH;w+&zat zK)xi&a{C&8jl9KW5l==`kAcf#3ahA6sPB77l5(kHl#5uFXeb&iTsvAh`V;*dC=vGOdviP-55><|usvpt6-o}QS z$m_nu)e$20MFPw8rArq-? zQfh8%a*Y9PO7^5d)40g)>vn570(;nLmaZ>zd;`}NC}6v+m{aO0te4p*T#Df#SLwTq&Q3x+ReutttK$h*p+u96U?XcpQ3n zFu#9ip3qh-yg!OO=Tl!;v8E74)GGEW$neO00{(gqwZf%NjIi{i0|hUZV`05BP5$hB z@=w$Brs?lCMtNwxJxvpbKvND{JAMun+xxtEzsOB_v@1pda}6Rj!F1jBp|bD`34#M! zb>V!pdTacg`8*qmguz3oy?GKRk<6?N6K@k|0rP(I9ZBi}OuB41DrV>ewVc`lE#(Zj zGx`Y!Y369u>mgXO#*c9JVG=8EZ5`))?dNM3l`ed!EIO0SpYE)JSs*aEwcj02_5$!o z4!wg&R0fX;SQM1sF?8{==`f{Y1!AdCiX?xWaT1SxitTTeda~KB$_0dmKq)mW}LYT$M_)mM`aHL!sDM;+4_dtoNAYGk& zBlnbqeCGZPUsZIY4U_59uW@-;^q%tuKOJof_U}FsDudjE`-@hY|MQN1d^W-y_k9eJ z<7M|Ext`4A(Qw0G0F_o{H#tyM=U(AxsG#ZP&53GWtU{Ip63}S7JXX+q*9O+i#LR%| zS0)^!!vgz$2F3AIJ)FP^eX=%1z$)Uyh@ZRVVeP+7Lqvery|G&OxLR|n&h#0ET%haj9pAgv^!{O2I0@WzymD%c%zifS^;YvR8Xh}PJv0w2+YY*7alhdyVu0Zmu6 zjl77stqLIr*)y&z(%SIE4%j7X@fhA-vY{0}so5TC#+Om`7eV(Jyt6XGfLf;!AoA@R zMqIBwAOJ1h@lsIft zlHtIvTc;79LH9*M2?Qza2-a4k(Y%((-{7Kbv)%se+5kj3(k3E+%>vp525SUU+U_i# z#R?JWVngcBpK~9_dcRL-ewVU0ju!(@Q_TA$8%-)6Y3u%CMl2qK=v2A7l8B^%36Bh) z=2-&U36!2vi<;$~bJ zy4~OdTMbS06 zMjuPY!&o9kVP4e5w>)H%efnKZ^kldK+b}7BKP?axD(m1KF(p(=}wl$M(O1XMMF~?BVOfsbmJ2r!T!#pyR-&Gc= zk3Wq`@DFY*FXS z-+txFP)9tqJGoL*g4;1J@%m1GDP|6Sp8tV)a7>*(F>N@dOvW=YKFoElbIy|`V{tt; z-&7X*@h&)_vZ$5{)G;3CViX+nym-6V*-9+{Z#C*VeOv-T$ULtW)9TV~T_|V95%e&} zT%{eb&M)#Z`G;k%ba20haZPNdpLB9Q8LW6iVpK-J+2RP_Oa4SWF~$TbqLcoAbe&U| zAV9XP%eHOXwr$(CZQHhO+w8J!+ttO}bLMsKKgfq%D>HWN_~J9A8Z_UL_Q_Bmhh!(= zwdBNH*)sAN{qjhpVJd5GS^Z&tezX&QGjm3wZZ4*&ixG{*)SB!7li8u|M}0M1E9UDI zzSUYt=2YF-TBw;ALt6ZSWykH`c$|NsCifhvHEp9{6~3I0<40)X>bALVn{~Kd2D|3L z85mNMncu<*)vTsIV&J4y`cqz{WC_yv^U!+Q>KZ;-c{hiD9`K~`Nf;{xCSZT3!QM@W zVG*$K1Ib1(IWBM3%wePTB>&G=idUW{$|Y*#dqg(=5eP`8s#{|S>~umTJ5XqUehC_c z^iX6-KLIG!f(82~EBv5i!U&!~Zhoi^S(-fzd^&;$M!aq}Kz@%}UFCW88fF{u>hXmv z76rCOFZ7h-`Q4mS;=AJ#pOKHkOh?5y4X4Yqn4b^*-;huovMHoQ1@KWEheJ=zlryNTy! z^B*V7%P!oHDlNUCXT9~C2c=qJFuCi)`&{S&F7L@}tD>H5v`c&S3Jor|J-wj_taCN8 z=jwn@W^_5Ufx61>oL$))0(#3&fpm7?STn}3*=5CpR_Lbcqik~^dz#a3E3DGxW6DUa zq2MM5&PM~K6gPaX0<|Zze~au@!*fm+Li zEKnOwT9wjt1AB8@#~2HK$8PHfQok-NlwCp7l(@CaE`)2S%Ta?(J1)a?WGT5LJ8TQ; ziv?6MPA;4rHOCKrNVtLJiwWWx`*tEI0G5$p-VBF&ZFJcUp@#s>_hOf1PQQ`Gj0tvT zWuDc+u61xtqaHaW@X#KHKaIh#1rBy(welLVmbfYzS+dIxK($I6LRBN;iZ!MB;=qj^ z-s4ez9^NMWYRwtLQhX|U?f$rMJ@ldNQZcm+P}#@a-$uu?0^H4 zL6|`io2fdqnhX9dRbl+*SNLyTN9UF<`Aevx=SL{Jj&KotZvV-&{^pu2v- zAHoM|#(bqeQc$dha4$HG6+;?>{b(vP;9)eAGH15Z$tz5)XIY`LtV5D;#UlwBV(-1G zps;KSpjR<{vuC;l|@kbSgYS5SK>uuLQ7`wXA_?z>c? zukmy!s{WK>rxGd8u&BHsF3p#1n>JiSrO1{j$c{xI zQ}fp`oIx07Qce3Y_$-g}l%B1eGVUS`I_50$z{J6jWyDY_j}Tuzf}yFFE5v;U-wYND z#E_VV?1Z%E#>*t7MY9CRH-#@IF#7#fE#`?7mJpB#350R6LYr%E`r=%fd=zQCCM$!J z0w*4hpfeG~Q{SOAR*b08FR`nz<_X;aYVKp3dr6p>3-tj6Q}UR8jKnA*j-bRFtl*`@ z=^{Zi>7;=54<0diEzKDeW^5$J=b4Xfut6C`Y4HMoWVZ!H2yjuSVjB{~&M0{ue$os0 zOK6zC8@JC&_O*9yR9+hAdC-2lS9jR+fQX-7u90@i&r?X4C*uuM=N&Zy(~63Cbw{uY z>2JcScE`~56jq7>fMvzNZiwj=m99F+>eAIsIGMZJuaiq+A>YPzqH8uV2H5j)iMQ(uJjHs*W~O{wfS-Db@+^`?l2hzo+Zts* zEL3e@IIVyCM14Kn%|3}>$a3nlZ%3JCTJdN@GBTSm_)Hpvxyw_qN4;{LL`2@D9azI z7n1vvujpVMlc5Y}NyoE^AUvGgGuxSAs%*djPqNsBI|@nYs^K~!-@eO5KxV^Ue@9(ba0MZZtpKX5r3j-&6_Qh9Q*~=_nMG64E1(-T8vP+RNH;S5 zYwX$=M)x=eGe`6p10zxDJ3CIwb6`4jXzU+-vEWgSmHQ`)Pv?AS?UN!ZCZ9fHmvq~D z@6Dc-+7lcU+h)ZQNd*D51;=S*on~I4cCJ~rKn|qY%SVWZ{GehRu%N~BGo4Md0I<&% zEk4JQSFcN9ZVaE&e8A)LET19+n||I>>1w0Ok#86iyR2(dHMXh7xvo;E!c^G&1`WR0 zaqYOuF4JXmq^=1t-nF6^Wgus9FNOm(l%S;p7P1NO~ z@y2+e*I>_&Wq;coHjfs^v=DIAKvQwL-ba<%R;Hw#uRiH7DW=&-b2N5DJ{`rkGl1zk z8?P%bh#?|W@(4?6NOZ7EhxS-vU>FC;ykO4swR|0?->!0wxd>4hgcS-UU-pkXp(2wz zcJX@#OI@&3d3g!unp7Jwp9B_>PeD&&iMN!%O^sxUvxF>I6S|2HJR)Lw=BC^e_fY${ zqTuptDC)m4g6?8liV*bn#K2;>NcXOA!O)wYrc$LkgD~4DadCMGB?}#?FhRB}B247O zIktF%v9pu^B^I;fEQj|YBHjudQPZAI3n}VDp+LTeLlMzm2zvKAjZ<2=k;MrvBoq1Z zjk7VG(%=FtDP^$l=U|A z9I~8SWsEzaq2bD>P=I-+6LD4)y9oweyLlC|!(;0Q(=iH3wiYN!+lxv=S>$;Wu^7xx zT{9u3hASJOtd|wEJNb$nXY&XAiCZE;EuaF@vJ9(c_>N79oty2+;L z6{m6P%L1JqM>+%b^%xjxzvg9E5x{_w6&z_ChaZ@rG$&G={cgO8a%^U@$Xxqc7IdaJZ$L?u+y$jr zI%RRKdMiE>D??+#tFacs%r(4$Md>tn-dEuQyV(6ri=+zT9OY7fZin_I#`u`}N{`PB z?}iP9n$E?(5)E~KK9kb3KNONdpA|*f3z~6sPo-1 zBVUCJR0SRu@ZmiE4;zw;hixebZywbj?~ZOC0pP)P>p^ImJgIKh(oZ0j!t1O{qi=G~ znoGAIMZ`AY@GVmr&w?KeyYa0rbC-tfWjp)K>=7=&Fo+NF3Db{~5{B{0ELeI_}A& z@9b@{oy^`0I+kr(=Lz^6PS1c&86b*8gqYW$X`bT=&)BAhKK6OG#s-1v6cH06`KEnO7*QGhCyQwbxwezR?;2P6?K=q6JY|}&=CL3z??WO;yiknfm7V^YjZPZ86 z{bC%$%ku4=T-qr1C;u&+?I z51^X4L?Y5a@DG{iEv>GAzPv;+51yRg&@m|d)N z2zQ8w=okkPo4MeSVd|>NRxpz7o7hZI6~$j}4HZfcH(U9Ox`aIKv}Anwsl&pcc0w2T zN1K9*SDT)z2chWpnr%D2SJ&AVG~9bMYwyNpdIeof{41lf$ZAi*gjyvctpJx!08U!c zOMKOhvb!)WgFO2n7C2p>Qm8gohx z6tf=+=I6+OidcyOHO*-Z_dMe-*T;4Kj-}u$ zl+92}DnbNnA$$~GTpL@bi!8tTFfoi7QV4_wg5v9CGv+7Kx5P!)X`g-&t5Ba(oYS?h z;a7+!mU^r-C0Z&jRfX{FHj@$g>8bDk$09blkp{2LA5!wP=rnjP-&j*`bnBTLEz$E7x!OD^x-pK z$;s)y;QcG_g0R1gb)>1pTxk=5O88L}KT;er<-6TO%F>O?4A7Apt{_Ja&GwWpO7dTt zR#Q<9%k*Wia<(%)c=K(3B(3T)Q)GTcsUEL=68~l6zRk5-4s;&8=mbv${m(-Vbu5zH$MUO{T$djy5Fq*RzwOuvQis@5R6$kK=rc{cjSlIY`W*{ z5^?&GKbI$85_^2}IMw20tCVT>r3uQ8_c|lu_c%$et&!lmA~;Uil;0sNgg0*_!=h_= zp~y`fsxnpU!y_mMQ}MjYVJ%I@aks_TusX$Lo5N9q4W#+JGOcNH&3wAon+kRl?a1VC zcMz8}|CVL8WlC6I$s`NwiD`ut;K{1P=c{Ra(_9mEuiFYak8WvEr9J`}n+e#a%c)c7 z^yGwoHG||znfa0Wgf?1z^Jk>FQld&kD|kxI^CAup&ggh{8taqe7<(8Jb1K`%`*e>k zXl8Pg)Ry%UY!+eUsKdev&X(CiH&nF7u(J}$Y}c{2O|DBl^TDB%#XJC}zpoeU4<*IA z57L01ZZuf4Cma@U*x8MYJR1Cx(PqR*jd?TU2}jpAwz;*JST4M`X?uIcEhHp00e^8_1bU{#htuuv$Co7*CMZFxWP>3N^-00HWt& zE-@-~M7h}!QPS*94h&2;(zrSJ9yzf<^Qe55cc#*x4{)d%i_yupEeR8+!@o;2?_q10 zjIJAYhlwy1L@5?+I#_%CD1T*xiy9wwy^Gh%lP(0_g;L}+=%nlnf61@4oYhD@L{i3?ibuW{H=7Lb&=2{V@Ml+KZwli*EUG@`L3 z*pb+cm-dnYm^D?1<=}OicHt~;B5G`h8vmhX1k(b0Wa^=y` z^xZ^4er~cI#Ofq)?X;(Wo>0I@(08H?e8`D{Rhdfa7y=chEL!xHS8D~Z&v1%+r(yepX?-z zYZX`?PHG3$F8mB8;J0;wmWUKoq_;nIk&TjUMHVfVILh*Kbx<|Ya-G0S$D8~D%8+wA zYu&zoT=pfMO2P_j{HaCuf$u0=9JNiy=4n|R_nUiom)qcC4{X9MjI{$2YfXiv!-%q% z+cfC$qNeNN%6P5cS>3XnF@`) z%QvEK$DTRCj4j>JvcEq!LEDeFvZ{<364sk0fr!{P!dq6G@|}10?4K538^NmfH4x zU2qxhJ<<=SwP)K1%{R}4AMCi^v|Tb&m7Ux%S=!#1e3eVjipupf9uE&j%G7+-p~ty% zwuH<(N44e0-FlW1+Z|?S*eEFu)k{(zWR9S26p6-tQUd~dV%i$~8r0d1zl0TvP?u3f z>Af_|xYuSHiD%Hf&}za6OD5$c>UgTHyqb_Khv#ncWg~jv%%&ZI35HU#J zjR3A$WY!h8NaVPTuRoe5%zeYenh9$!gyCZ-N`iD)#UxDGaOrg$gst4R9R_d5kdsBKIz1;_7vsi@$l+t!f@cV3jF#fHDCzUlAZ zCz|~8C&krDGN+kQ7DBS$=g4+MXq4654dx1r60RPDHw@2)Rh4j zQTf<|DC_|Jdx>TTQ_(%YZz}kP`!KRdFDz0uKNdey+xYlyQUJYQfgc(E@p3p?iH<4J z`Xnu;P$TJDR%EzS#O-@V+0}9G8j=LQIZwI~(rsJ!OMmt`E)gpy0fc z0F{K~j`HR&<|;@R5IMe8-0o;SaeQ(S6XS8N=cMkng;0jfkPq2ID&<(8DAk8}UZ%n% zHItv_9JGs3LJ@UZYFu3Ry&03pEqfxJ15(i_1bg5}URzx|ex~W9(e-SK2)3d1b4?zA zj@~XWVi*(Q{Q8EuWzJfgrK-KA61gNxXnPU0`)3j~G%AdL4C!N$O}J5q2bHtbi~d9p zzUGDP$+9_Sygm5YJh?({C8oDF(*^nQbh1k%s7#KX^3Sar=}t3V)kqtRdRq`0%Uq{x zPL?^@Ii6n3SW2nqXCty-IUxqng=(pZ!urtk%{X$aFs^}w8@v;^mas7McBw3U2bQ3Z z<@lPMG}qRky@sOWJ*;ylqTtkDE@1Lqq;Q-K(7%`jv8}(r!xWEXlK)i|O|^iiIqazS zX`64rIH=42!>sVt`ll+S;8gSZzLCxGY|l-#9W6>7%otj7Cx=b|03vy4#PZ1rIu zj~=Ti5^Q3c+p8MT?hC`Cd5;{vQ&aX(M$y|MZMEj>EqX-m|!l zOQDWkw3D(L>k%E80uYRj5YXahlm+touVyv)= zMfgYA*~`!=aX)4*8$H_d6dQ;BSM{ZkeyYbiH1B4qEC-X11!QOr{8UY+Q zO@_4!2iX&`y*&iV;A9X3czI--sJbQ3QMg9W8Fg#CMh=`I9&9- zw`AjOviVQWX1yD@>j`VR@!$} zeU-KwZLyN@`VU-8*m>F*+#N!T#T9BlpUnno4%;Pz2pYJA9E!Kz{aDVu*S@C%sE`{EaxEPbHfyP}vE})U;QXU}LB&3kzXp`Ni0=R zNQ0s@JKR>*Q_(c19?@qcZ}2%;o3^<}ENSqo_fc;~KZ$kUQkQYeve7hP_Xh0EKwUvE zJ)kD<2x3~yvs*l&#WeG1;+(rHX9yPs>?F3)nsK0wk9Jeenu3B&s4cp#W~~<;h!^Gmf<5!i&kZ2mcM^F>gN;y0%fYnrt5AJx#1PR zOgSO>?SZn%HMHV=!e)b=@Z5#Jb=9vSrsFy5W7e^Q9mObgjuh)1J;;E-VgZ>MnYG>U z?@AeZ11m26!T;pj_g`|rDf8k;qDRpcawu^)eZI;gAn*gIbByoBWJe5SX5^8Kz&caE zBN12($LbueGZ4lRWJ>u4)t1cEnI^R{*(5VMNEUyL=jD$wdaLX9JN_EJl%8}6UPR~h z08w>WCZ3OjVq;B)F&{$bB%{d;WT6qhnmd-fKwc5WuWP}O0eRk2{W z*q1(5T{F5Kx;#jy*z2Qe&NT^fw1Xf;gU{jh^A#`47^sPHH1LA&A zM!fJ4DI_l?I+OVgZ35UWwUn8zD}LEIEeSYzt=}p`jom0jCi%%0e1Isa$D)KrTp?xT zP5Z$cnQC%yBCf|rQs24}Y{s7Mo(n6ouehJ|z2#ymgO4(vN>|yZ8CR9#%y!ycLnBnP zxqTeLLnD1K#!I0xPW1ds{I^&{_U4kIb!HBr02ANocdR;Gx-8k@TrEF3#?Yw%9FNg3 zlRlG7-PF{-Q=LpjpI+o~rz73X_i@aO;Ps>R6K5j8%{TSJk4p{_bt-2cNjlasf;a^z z(FwK&P?QJ0)}2b>f##Kq1U>plmBAara&b?_1`;LD zno%po{)oPnt33;jBq4A;d>cQo!&QbkJ7#Jez9czZCs@V>hvf}^Kx6SOiErL5=Mw^$_r2tHAC6W0g;675KXc@@W7V=N?p{7L-Q-Q1*X>9M8%)^| z9C`ne4@6IPke%iL`d(YZ8=Y;dsq`=jF|lU&^9UIpFH>UA`2+gyram?{z_h@I$9^** zIv8(os#({+Mz($QC0~=g_Neh9v>iqZie3? zQ$LiD2AsU*$O=YRU|+-~80eNKEndgUk}F4uh?BFkI?SnEG=Y;cgcqa&_F>X@xrf9X zFX$c|d|{fRt7$uKM)VozdEa-i=3(4&RhH^Wk}Ov!;iCzc6bLiuK}<&&8dm&bd9z7J z#93x8-Xi|^W^Eo@_;{`-vdG^lmK%)QnL*AdviKoYp!Zof3z?>S;4Grnuj&Uyd{)BY zhUyt5Cn4mbbso&7wOGVj{Q`oqF}ttiE8%h2w-A@Lcl+=<;=%60QZn8rOi`UH1f6Maoq;*=`%Sg3iwu^ zl^ON&DMTfYyKB)fS6gBZfJ5;M5=CnOVBncZg`ICayp!bydd z<<1fBp{iE#S|_ZIpm+a(ZBY=yY+mg3NciRtYUf27G2eWD)z25kPkYnUg9VE$$#-`@ zj&Q9p){as|g=6Cua^#BmWLdkq@w4EF*LfV2^Tsl3O42#_*n|>>^@MD0lg(tM$f?{2 z`Q)%cSMTg~Gs~R5cC}Gr2Pt_uDCOV_1o!|-$l7t_y^T!s5k)PVJ$;35|;s(6=rJFpZC@L%L1?Os+H!pV>QdFIhvD`Mxil3DYt5i(bq2x;$ z9dqrVqt3OJB3Vcck zLE8<*542n@mX1dl=g6uo^Xn|yJG>h8%z}XT=!AYsz2$QD!|=oFIayB@|9oWOBiVJr z@RL+aLyDTv(EIQ7j_TM+!Pje=(t2Mla&=G4UlcK3`QAoeu1hP({}%Ny!euDJb?KaY z(!)Gno4SPdcr8H9B@)<7&uGA})YS=%bH1)u8f1#s3=aRuh5sDorGkcyFKrCUB8R9ASADw2n`CqE0pQ`>&ETCGnLEA5koF22y#gFA> zK3@qiUM1y_GfKJi20@L>FB)us?GfiC*;d|+WSsoQX#C;W-EB>Fi6CdrW+-KP5d!cb zV68GLE1p*U%y$S7#ULA#=`WZFOC3r*>r>w>@s<>E@;&^Lv*URhOw40`5zL?(^EPC> zi*=I+XWx79272b-m8dv3_A%0NJ+bieLe=~1whUQz7idH{C{9y#J8__P--{$@qt$P< z^Spe}n<}7yM+6nWbrY9+`C1C?Zi+wL2*|~d&UF`CdogWl0;WxqR(8atw zXHqu+2|4TL8P>Ok2H!5DSz+tzjeL)qpklhX__slbc*UNRPva~O@7f8u9|9Op5AE0o z3-c(FyAsu<)7<8S?f_lPNFlKi7Z3<8O}!##?ko20**V*}(w+@@x<_d@yR*9esxL`oXx#kPFAwsqQyS18 z^&U!`|KkAix1|t{r`usf9PSg`0GB<-*I>@$uFJ`MOX2foSWR&<6ZUo zHD`y+rvxC0RG}C*{@DQ=HyqtPFFdjg8zJ*^nRTzx=gRaUOC0AHmSP+vfaBB|FLgJT z5;=C@t(MRG0Uy`|2Hj}Z&mYX4)8d0F?!=PiV7;QplXPe{=$1i`$n3(%`R6~7L;u;K zVmcnF;+q9VL>R}9lC5AC9e3AJkO`00eW!|3T=9JNhH(FW-d56lO1MP0`GxYo%T5y& z_4^*@*G!x!9*R>;G0pGnlH{Tfi`M<#W;_rSNB9^eS>?>_)TsX<^j!aO$G18Dk2}8T#-%+jc}?Sl#%;^}0=NYKH}Wr_)~J(Z zO%jzB5_dj>K5qwrL{QCT_$>*Ns8MeRz*RmtIQZ}98o%BW{)+6DjP)L$%1`)R-QVBT z>Z_jwy;s!UsHeE>Gc5E2`?+)0`f4@VvrVh5uE}=s=jXPU z-#R<6w_)`y_h)-P>+O%OLQ~89+UmRQTK-~Gq6p3@cj>*ASp4eN?$U2P!vzT^&;D8~ z_V<&4FRrt&A>uM*6Uy2|1Y)w8_$!<*%& ztUG7ry#*=>DgTaazr1zikIc;|wJ`8{57W;EhEoVJ4;bsnzdrv?sLGm?nNr^RkqQN3 z4(C%~?cWuy80RS`zjQmu3kwp?I&^Bs8X5d3LAd7H&T;1M70&7eAi{y_{s_bv-yMoC za`r*&qI?GGn{9VljeQpbO?{2j7yl{VzW037sJIV!XH8YQ~!oQPS&o_)d_Bkjf*sRK3 zuLeuIzx-ET7;v3K4mYE(kOxoXE73>crTUNfdJag?1NwFS6-DT}M3cWFpxQh13i`t) zn&5&9c|(y^c#PZ2yfRnCm4+%ipAE@{NUsI&l(L8^HZYZM^@fj3{U|F=`{ZrTWL2UF z@Wpc!t^miX0d0z8osixR3ZR_bU)Xrz!+H)HNE0N3DxCD|rn)=-OpXZg)c21}_V0tI zZ39A1N+Lgk3nK;ayD@kpzwbd+!cjVN%+No`k}S)Koywp3g%}|)jPJYcTdc)&w0Ihq zkgjKvQKCp1O_+itaG~8u3SvF(tbfLaxX%9q;zz(;@6-4v`mPDwvBRhXuSD(aholD# zx2%r!#p4l>TL|5!1Vlbb%4_huj&vJu{4Ay2q*0;+3b*cV66uC1pTeLFN8g*71xu~%nd9C zfkwv^5Dh?lSyw}9B)3`ExA-O-Svky*j5;m^t5lOV2&oOu@05u|uzZ8@Fg-v-hM+;5 zRN#w-;W)5?+ZHmE6d^ZFs4+l}E=~c@Ss~y6eTuMT zn`I|K9Wh>BPHBV7-EwC8wV2AjR&D2%Om1;vk63Z5VM}2`{zlIsXe?f)J3MMsh|&#e z@PVA(yE8+h9S~3If`tF_l{ND;tnQTDJPlb%KUW>jejLx#VUX6YR;c;x)N9^U(q5b# zJPtrO!8CylsV@2hmuZ}h+2P6xkcd-zfJA#IFUtreLjfVHRg?v7VBt8Czzhm5O#aAW zroy2Ge9csR!8j3mMRI15+ehPRFey!XH42MNB>#r|zG6&tp+Cr(Jzx}U!Z;|;41@{M ze*LcQ>O4XUmmeXWHv~(n_&d=OezazSA}kJqHx&gfr?oB@%x#a9V+gGZ#vsiuV0NjV zMJi{ytEkU7L~J9m^yNMEl)Q%CVtrts6^EgC?Ek-SBt!`wtL#*@w_Da?zLOIhxE zinz6D@Ri!;?Onj@ia`Tau$Y^fjO3WH8jQV$n5PFF%koE_niuY4fN>%X&W&~{e_04k z`j6JGZnv_s9UHSLiZp67GTg+I93qvgg9otB!8i`exT!v(tgi(Ye$uybQ>uva6io!8 z5t1ertK!UpgGzfa0e4ilV*}|i+oNHqAwNs&`VF&z~?4w=F-P3$$oZovQj7J6(U5KZ5 zf|q72M|6gB_zRTTK*})8$ya{OHB*VYK^5Vj->Cs~$MTzf1J@C8< z2pMsqA+328`ATDe;5LBxune$S&sn2{XH7e4TgH}nhMY{Rsv1GH2cr9&wLE~W;P zXx>d#>q4V4^rts{0NtALb^EVn8-;79Idh3Qc_XrcDSqr(I2&=rAEBU~fU>c7aA{lA zm*7&C>B&hPnys36l>Evq@@2br=B|$qR3Tw>W}K*voz5-M*4r!YUjVVk{qRPi4B}?d z(U_E!6&!IXIxA)@t{*?U!KgbXpkm7!D$(;`8{NwRS-?MBE=a9fjd<&R88Sj=;ov&H z_@UHGEC$lhcm*SzuK4e>GBPDDG6j{`@P8}Gdj32S2*I*`i{n62Ht;c^%g3!W(Fd;3 z9Edl~V}|?B+7T}kv6RRn!?B)-X13r^I}O*4?q&bqk@s z8)f6fGDt_tDi1S9@Y2$+Npq)5N;Aa7ft@`Z_T69kZyvs%2$;*AhOoF!a8Z^GKIm9u z#Z^$dYHAWPE1(7hJj-`(z!;Xz>4#ILZ{i%7t{;&+J_{HyzYWnsAbOI@=z*?r3O{Xb zk}xu9x~7S~#|f9td8?D`-RgjHf}5g!PI#)k~Hnh7IA7TAqWvvuw~mbxd2@U?_=%^BiQgx%B&yu0FHePndj| z?ZYk76G|AQCO3{s1~lv=aLTiv`c;^mKyTfM2jV&RXUFPg29l0I1!4~z2o;gK6jVS`+kl$%hjT$u-k5CwHW-F7u6vpmfpk^;^Y&YMHgD+i(=E1C08=2K^Dbf1WBs$$)ma`A^=Tp zXwd7QNv43ax(m_i4m`*O#wdmeE8)tGV-qv4al!BG5Ya!M>?qEBn+2#%i*);C`T)(E zmr93Fj>mbJd6s}evE6r7&{YTu6||$t9%Wp)Sa^i z=Nf^68F%0wnY;;kjU{00wXlH{$AeAABh?{6UzuN{P~h|OG>K#dU8yR<34;w}+`+ls zBBG;x?%d0;&wZBeg|3eMxUcXSdE_9gpK{L$zYG@9qO|8*(ZlS9gSK4c@Mh_ zpm~5^3Pc5O&i=URyIKTjJGHtOD)2Su8fBkDCLvr9Q5UQDf}{CmT^QqqfGXxPQRg0? zd zuIeL`4QY*Yp_YbRHvk@`RrYT4nskeM*J{p%3dYA=TX88PxxA8*UW?(-M6uD3h^ij` z8otg+tBsQrg1xJ|UkDo$=p9@ity-C5@`jAsL&f9ZH{2ng752ZqNC_?fQZpQ=)eHMC zVE;b~;g#327}AU1=M6r#5glL-fS4N)#^E1elm7!tWM$@I(l)W217(=+gl}}IyUrbp zL&TJ*xvo?ymsh>aB_}8UeqZ6+@B1J1S3k45wy%_3T()5?rgO$FY4}$2ms{E3576Fx z7kx0XSI^z}zJPymZ|Fu_ZLlrT4Tg4Rk7J@$s8{>l8U*V0?zEvhe7c>VtV z!>)C?VpT26&Qx8r-Cr%MH{kTN|Ek?QgcVkee|CQ_%Qp;E{n+o7$+wM>ZY$LDt!a0# zx$kA!>DrzaUYsG)*=>3@drG>CL_0iqZ!aXYF=w&Y9ISj8`TP7}*1cTey@FS-HGpg2 z+IzzH`|rV9;#9kCz8M2kyT7FEtqmlWJ^3_ zo;GlvU4(x8T|N89TS#8sk~K6ntMD#sZ$ZV2^xJuqot|Up-g`;adt}_11=R0g??Uz- z0qhM|$8b9fn1d_&{1}9xc3@xX0N5E|;zsL_c<^VvGoE(6cKYZG8wsP4*U zl$Cezd?g_DbRT|sgw-#0AAq3NinMQKC^R6jTc|7N4_5|E z|67mUEymX&jMfuO@wwl~xE6fbzHV%FJ8Ky*Mj&4*{-i$)v`dx!lX}bpgnE^qu=jFD zf*{1IUDc~BIAo8^Eu+r{=7IQ9Qis5V6fovj!vKgQ*V8zM{{oXCcTAHVtPa0CsIvl? zAWlXPVn#Sz;3)l==rI}V3GN1CeC6dp z?|$P9h<{OIm;Vg<_9ri4r-)5)XIl|<`~xkPgzgEHKhCM;8WJ5tSNOjyXa%8)iBD6A zU+U3%F6Et{xJMkink|2s0A;t|3umBgKf+A_F6IeQ3t=g~;>)Axy`e~C4`^{r$TG;7 zGBkqSTuuT)2?7*_Zu>qvOggWC-LlJq(F-zPlr}Szu`sZ$5)?!dzR@|*`M5RRN|!b3 zlr?dm0|896_^T~1smVSQ;DM+GTsgF`nN58q9}U#mK35`*T|!z<6xh{`?busAq@Tu` zkQY}y8?IB%f^8(5aXezLomgV>NGC?8f^Wi;%ea(M4!s^HXSSs+NC{1s{IKqhAPw(M z4Z}7Qu~iicD#W!2V(JNneucI{d|R zAj<{hzfK?>#reye1B}`tACw~KQ;I2=PLvWMW&&le69$9iJ3~PkwRqLCoWP z8S&@u()w5ihK~`P0y=LXnj)7*AALbeFz$B#M9WQm=3xU4Ruwno^eLfM!%DS(Vn%hNF=0WxJ*UY2^1*AND3p^UihP;6b73fUj3ScadFSb z3qx4nS%sS)&N?=`Z`)!+&HT;hTohjC_HYr@Q!y8~?FLZ)$Uara!mt9$+N8@T*N`<=G{`>=wo?Ug?@_KXUtF&l??<*OH7%LT zwuABqKiHIFppe!~6(q&@{0JpNXEt%ER0nFy)#j$GDWb3VkpN8x6O3-?(fP_wjw!b2 zS3lxabWZHW)qgtHsti1h6n@_mHQO`kSx;BT#dV`nLP}{q2<&Dt%r|q9iOqIcB6o@| z5OkWv`idIxT|(8mB{%pV!HloB>Pyxswh}CW-ps4)bm4h;gpKnZH#<kYE z!o|#4?}t<0NhwMQTVQ2;R4k&H{HHWY3;PUj)v=pFX`>dmi4`TSlfySYgL!ANgc=E( zUT&o^!QBJ0f!|Vhn8spE`Z%Qriy-|ExU3Z2a{HB22GL)>D`fU@Gv8pAk*9;&wKn|9 z9Ps7(cPS-0VQ{V*P09wUjs?2YHQ@zrEB*7qx7>3+Pbh!HpkYYk?gl$tHlEssp@$9R zQ@O_iQ|#G#sG?c+;tW&z(>(hcI2_3M7D063T$@jKwj+pzfli}2i6oc^YG2(gt!^zS zXr`+}y(@NL0YF6u_Gau;a4%Pi5j0^DyRs0lh0Bn2sl)eqLONu$Tf_HdT&=~#gmVDj zDRiD97-Lf?@C0YEp4!5uy_z#+E@5XbBIl;E;$_7n^`n0l80Qcu=W(;QGz#TfjOfD1V@u5rl8-^)RLA;tzfh zQ2N{u8@0@)_Bqw1=kS9l`NC|fFBhi{tf_d#KcEERWN#-D1q&DKB(ugPjh6}Za&+_C zo~mocA)VCRq-}%o0@R>dv zlEhPe3hj7^669(N4#p%qvZvtQ z`HctMJ=U~y%!^>s|9DWBsOH>*C?ypbrZUX^lw(TinN-(H7BoiiV8CnP>?-GVZPN&2 z#SA3CSB;_f5!Z_3_~Ecc{C3lzX%eH|j_fsikzsh{ys_~t`62K`()kz+o?X35aaTtu zJJlm3WwV4BG0ktDGHxua^V7SK#Ua2|_*U{^7H3=%b2DYkarmsiXg4c4Jr!0#*&4|) z_j((8JKI}&Frc4O2T+ z1rgapGQc*F!57jdaGPL%H-xlGrd)Il+?M*l=HAz0h;-|>wz)UMhiW+^@&GEu1b0R<8HZbLFQ?FQtt`3-+jGpF z6DOxuO@cy;WGOB#-~Jo&*Op(PfqsY8PbpiKfCjoJY9V3Y(!eQB_-1|>RPgauqcFg4 zpBlIJZ{bB6i;%B;Ink&?!h9`K7)U8>__RF5xk!VxJ(l$2V58zx zkB&$)pYH=O7s00+Cq5UoHp4-Y#4P+a*EA^ctxBWBj2{hy{EUKDrHR$7HlARI(&zt= zt#jHE1jw>z+O}=mwr$(CZQHggZQDkrZQGjap4a||h_&Lx-FKf9XbRPJqFH`v1YPak zvyQkWXtuL2+lf%L`ifvv4_`EO{g+enkJ#Xq%F~?bq=v9!!?Ji7G(B>-x7KQUXC|sq zY76Ef5ek)2T|<)>>c?q8^jSgjh1xIbtvp}35EfRS83_FanEni_ybh;m2KNzj9=Rj{ z5f^Dk@U)DYsj1n*D{h>ZCv9)#`g#y9FN}q({GN2>MfuL~U6!ge%^UKMprF+enI^}_ zT^K#vP+4J|?PGHB_i_|@xxx4F=N^1c6{im$lh#Dgz;LE0dDj5TFq#Z`m;B6NeQmcx3(4Z4-SfIl>1=Ttk5jQLWyp!1uDYY=ITP(H`s)b>1Lc?`}MOF_j{inXbq z(12yCv2pXM`n7dQHwAYw4aRqALw=@_!{g_-fC<3LMgE+Fy1B`L8CL@O9x?bY)mY^! zZHfkm;~Xjgwr8W;qQQ$}T=MKxSmc-z;}J3UJpyT?g4y!NI&|{)bip$1#dPO1V8v;o zGUu59q|%aSB#(>A#ym=P^Lm`f z0^bqVR^%w*s2zO^+k?`^yuwJX8XgXmsL2K&zkm>r8g&q~{}mQi)B&Pqv7`0`q`dqS z7XB~3(e*!kS~C- zO)~+kCSqb@rmOsC|KW}QGNt@}ei>d{p(o8>WqtS6yP-BKJ+*^_O8@I8h_q9xYXypYwyiWrTMC$lbayU3cU>q5RUXmz{RiWff^tc7yU&d??Vl zUvPTx*Nt$(k^1zPRhj6%YcjGc=v`Kn)><#y3@C@>9-j>m{O-Q}zjnfH7X~H$!5cCE zaA3XeIxP*NX+*z}zh(5%@OL$#{-1ZgP{1^2g&$z}>`Pv{hix1aPBmR_0MH@z)FTMd zwVDH4(rZlDpU`qjbwUe&)^%KgA7|DLPVZ{0*`Exo$_*!J>$^mS8@_EWZk2n!{tn~T zeqK0Iq^xS zw9R*Me@KF*5uPIKqT@MU+y86=6(}?auY?Uj;}9~Eb6z8RC-37KB33nEbbd1YeT`sX zc*V!SWz&g~fyUQDuT`eudWmnJxZr9r5y@lD<@tiCgQ}oJsG2TqbY-|C1)H^SFnxuk z14;v^eSV6efea)DR(Y)(f4{6$Y$YtFE!)3G9U%e*@sFvS*KdD*Lspg$=~jaH2-c@` zEnKkF16_NLN*m4pkqavqFbg>_LnIXdcVu#98+amCjQbLyqf;Jw+7BY~)V5`d z1rI%dklBqta@HBAdlHjrt#*Vit}TPr@Ds2ftV5I)9$K)MBj`Qqlz{kg&!ukqh$&U!YR5c2jK^!guQf>vNO!F2--@DG&-Sy zq>;BswUNuoWFo*GWuhqq6~(Shd|G@A?_S6C8yQz4e)%$kNb5A_b?igDFl# z!mVSN6G#G^wM0$=BU&VRCp=+js5K-=1Csjtav3HO0gNDIw-^;;o_{=Ltxhiqc`cIo~IxxmkK@(75@0% zAknNpZ86Gs2eFhlEGr_BOC!YY3RIWWgv=x^e0ZRGGO}>_;+&M2L_$a@*$|;Ky(v|~ zIJ4*yAcqIs$0%NmEyOIbMQ82TaU~8uFyBCq)QEms!J6{P!P> zs*@!iDQxNOeD5qWBJPBHQot1EciZCdi*3n(oLOr|B~Sn@TjePJDs}Tq8WpND!NBV1B>xqC0{+kb499=oZn{x zU@EoG(4IR9$u@@OzO`eD3AIbz4Dw~IYphziOewtcCl51cK}EUW1X-B064c*uC~wD- z{)XvK$;g<7pwyyhd-DkNu!rHex=49qq7Je`R9?x-)&e71alq@4VLDUkMA2%yU?Kzb zs$R(?TpExe+Ld%FlGSHssU@Z%{WJF%EX!&Vll_ubuY1*#O^Qy9mf;?VF<4LVR|gvJ`y95B(9Jh(j7Feq>M2O3&^!A%_=9_uRo@Zj80-J&gSxW$LKEiBr8R+1B zn_dyTTrQ0#)FaA>t+(`6~186&9#osJ{K7$@0 ziJT`MqQ=uN3RR#O&!_(8idH@k3=--~A-O)>#bod|HdJS0RFEv{6bw+&_o%Q#8}@gk z^j%4sCa%-_G}NeqbII?p${vRPwat6*K%RCHicO1{;A`z~{<&;k66P4?NKw$dH734Z z=tctVo$Du$q*B}wn|<2Gn_%YQWkZbSx_|jpDJ0ZF-Z-ViKc}>60%9 zAz0to6BF+j@>~1ldRYm-!9$1%#MPeTAx-U(lcGsD*GVOw3>+(~aIT7jjcJa&KZo{b zoTMy8t9`~DTHe+*LHNx)`Fq9*v_c9z+G);dJAt9%&xK3u+jdH#co+!a^MsZz^F?rf zcti=Cyt<8GKyqUoVCY%}>f*J9kPKetrM`MP<0olFULu=R1)3(|b`}S!l*Y>t^AZQ5L0C74k!Eb!N8gdaTcW?Lxta)F^GLvtAAnJYn+39)(#xbHcEDvw}KxUx|d{j(+>6V zSF`3=ILTk!L<&iDp(#x$)f0z+!n8N*G-7yN?a494XE zu}gf}0BAC>9l{u9rX=_h1>eau3h?`=`u=UXk!15w-0gKU-lQ$oI zXGmrmIz2fr%!NLs(6_mF77wZtR_pi$q4ajL(<(!SZaMMS~N|Hn#ECd&nLqzvl zB1J49Gp!IBSI2tbtcW}crR8yv%p-ef(hmIOwi!$4x6Z?6`m+U%nddW+& zz{R28d=Y?w7kW)cEX(1a&NJBiojuUl2Jyz^g{mCgJO%0tdAFA44Vud(;P4e8x$BN8 zG<8YN4K@)zHX{e`Gb~%m0rz9&`et)^ekU zzXVnOD=hwBVtU_yC1#r(NdHRA%=r%7xEc=IlV^e5u6Q=$(B{7vvNK>6kVqVbl3DN_ zd8#u@<#-iAmA3UGDG?}8?6x~q_xRDHM}B{v&_3h%Nh(`dTd-ZHsv&3~_!cc2orsL`*sQ^*X~ z+GDKGVxprBh*iz;#7`KIlsOfkhc{Vf#t(s)-boU02kv<&W5_AGP7mUsBh=7y5#=OI zF>p;XW;Z5(7B^YA9mnmyAmMNnz$z0(de*=)bYh;f6mG3egaH`nxH^!Qjwawii0pwoqG5$qXxUa z7Yrs;>h1&7@P-w(MF!4n69_-NMbBZ3NBIe~-)Pl4Hqkn1%(RRPBxI)lmkuft!=81L z9nrH^?vv6DROG(jTnZ&>tw^(oJj0}K81$x4XP0d3M3}KLM=^4nZKfS7jX4j7QcAi1 zkCE!2QtRcMR4U{rpO!O_&*;>Yw5;&%pTp3}=kwADVm?9N1T3!HhA;jKLocg6piVMs;^a)(0;j zGWh`$sy=X|Q>92EcFjW>qa-FYXyTATP+-_W^fy^kC=v&V`!&I@)!R_3^;@`#?7*@*PRoBX&1cxC)#7mTq;f2ivxMQ z?>Kbg*2jBOVSaobVo#@R>uL2!t`A?3xCIox5R~ztr zZp0Kz_7c)o3oBrilSV1n@FmQaY~TbMSruv^oNOSU1@FBI5coJ3bg2q((qvf_+Pb%> z#Tg#$B>c$9_+nwa98}B2qiHrS$lqRpk!JqEl3-e z$;t!?%cBgvz3rqad5L;xGG!V=s>kR|u)<6RVDXU3N(S{_IY<|88dU?V5RCR{VbAa+ zV~4Du>!+>IjX*SpfG1&^3N7YI!p!OMw`!)Y=yt0`q;m|^Axu5Kb}fbO5iqRSg38lg zoXLdVo{}NI<&F9|o|M6uMXfFvO~o+nlRzul+jHg7pd$rzR?YN7iDXdz(d#`tgltri zQv6$w3zii{n-&$bKFGYmA-tUhm0Yq63S%n7+VL-dx~Bzl&}Ao~-DjKFV7}d#9gbDkVT6#~`ll^n}G^rVGPbkX0QeH)A}c<%G`J zKz~w*dfRcB(SsD=3T2TbeQxSTBQ<~G?^~`TYC}|zXWO0Eeq0aV#+>{{7+=|1mZyZl z2qDfA)3;%n@q5hP(|-EZdTHfyt%F2{ZZgHXje|OvfQEFw(HIGx9OGf%LfquIdedq8 zj&Tlm&}An2m|6)73jnf#%6XY?gPt`1^n`vXh+uoup2~{dU^C4y{)Vfo={F%!MZ7XM zf-E%rC^#X@*R)BFhEPL$Q7*f;V$yp$YI<|7S81Vd!@%Tc&qH(_r`;SD0N}(5R$?XQ|P${xJ2AUs_6yd;|_+ zTI?rX=jdd4OcR&qB61r4NqRmLx8g3ABJ0x&`9#+&wdkh8WmcZHIHqGI{yeE2UqKKr zVnT}=9gS9LRfzqOsLk2oW)RG(8C3qw*t=ce9iY61b*y20C+_2X3pU~=ew6{^pcPRp zYTRAfD`C`pA)HyvG{vNX=39Y0pq&=Q0fLfyH(cY_p0>9^i28#r+60XZRPB8e#mWeg zD$)$+jRvfF3#;QQN}9c8%Q-FQ`>Ym)hMv4=nrw^fNVRZqn5v?JNli5@v5z@w&P7Vo zF3Za+9S%)E?IE#2E*7d2w5~}nM0>+{0^Yo5g+BDquZcAo3 z3H7huld9pv=9*^5(X$J)GNS8AY|xTugxj_8xiK^;i{qJ1oi=2^#pA{a!8f)cAebF$L%4NS!llp+g%|=Wh53sZ09CV$HlRCjO2MMB}=lNxXhTgijEajHhWpq zHHPD1nVS>{5S+=BO7}^@*FSwy1BlKtSyG`}|IK@(y8^+#0PR2*f0{JI^)wF z1bQyHL+Y&J^_4yFK2U+7YhaOZ!)bZvTAv-vKey_i$`u=d%SLD#D}I;-y*7twuSCq( zBHO6Pzeyrr42Q~q4dSE~82wXQpcZx|3LY72x|t76>1u#H%VS&(DRQYR=_)S_=9N9# z_!}Tn36fFu_hvJfJQ9-eW8Tr&jo&C&c^+f(+cPgq2Y;roZhzTY-F}>BZ#_j-ZKmNv zShj>aS;GA*Z-S;pYH1{!^TcM<^74JqUyX>wfk;&ThZZ=>t5ALPi3NQ^lEBhYz)nqP zcf#8eYXR%ClBo}>0>HHwFzJJ!K$)DAKrrOTNK+#>C(<&xt0V*fISGJidw0P~Iuar(2 zPrESF2eXT7+`VPY5apJ869!1qE$wAEn0`A;qF|?Qy3*2Xb85ek%_B-5b##r`BO7s)8G^AMr{1rHhEV#UY1Kc~jJIp!!;gWhzz(ysY*BOoNFf0V@dH$BJgoJ{jKitd-Fosqgu7zWK1ZKnZ>o7V_pXT#zP9LI9g^aVqf&0%;BfS14 zxr0$eh&pZXGC%dYcu zSJc(|;P#M2b0IOam<)QD37i6uKPaCeyT-SeiY$~UUedsa zQ3Kt7C!0H|fT-OJs7G`4U;h&C-T&}G`f6SNM>GHI*VPAsDFM+2i5`kTAae(O1)nFH zw%N)SWkQmPkmrSOc)48KrS)Z)Bkq>xgr z`dzKT^4tzNBdo}#+eJrbIpVdN;@j}&U+)@Pp^fEv(Pk{@AL^<`_-ft9sI&#Ej}mRS zW?)j}%~jePo9{k)Hp2eut%bObY&biSV-uX4 zb`|IEpATq=o{_M3k2<_|e*eUzPpKbZ;hB`S1l;wiZ! z6RRiEG(S<%*J$nGIcMlv?XF8k_c?0JY%n|;@%Z?MJj^d;XZ-WGck?qY@LlNKkoQW- z-S1K&8#KYW5CY$7N+HbbWnv7S=@29M_~Z+$v-*?W%e$~106 z!6F`*8&z;fx8|JHd)o6@{W|pFMnl-|0qgq$Gt7ik6F(Bh$Hrv}(9>)N%nVWOYo&@M zK=tr%vc&mv+PvlAWe+kW{H0Q~*hJA9uGF1E~jtPT##(?7I9SbL*<3}kM=yEJsYi3C0M6NvwQ?h!|MUZ-?L;`H$Mib2#IT9kPzWtL4MEj2DO3i@*ng}E5 z$F0&ZudsZqObXmNtZF%BT)82WqDb2jNu(%_-cfmc2xTyYvk>nTL9DivJK7V1^qN$RBJ`&M4BFPrp)<((#(us{zIN<|AuK6M9G` z)#Ddn-wZJX#nE87Q4ET(&cYNq%ICb|D@Jg8P9bV^{PHN#%Yx%H#j^$-l7?mS_x?As z>hci$(}Bkv4C7YI6ggEErV1lGTt;w_seHyR30$?J0A>W+9nZ6y`8fA3(6J+Bn_d)% z6GU3l$kH9M+D!aqWOJm&HCX@EnSN4XscM)tH<=Zef3MdEPGr&K_jB$VZnIkV(#xsc z^LW$uPxnLQ68N>1B`>L$WdoukWzE6NY1mt?w}wcOlgKJxb^3iW(?DR^37r%w`&_z+ zeyJSTbdMm$b4b=^>&C$b1$W%%d{@#iF+vphmYI&fxt7=!$H&Js*5S!``VDe&E+y)#L|~1`bv&tY zY79h?1Ce`CUw<%NZZ^>$_I%$ND9#L^BFc8yT+bLw&u%UTf^YqV|5esOxz}87>r!H{ z>&x&vnuB)7igaD}|C`t=8s;A0X zBW>o4ZgSPRRd5bU0NJJtoFczt!5-`RLv1&5A@)usA4g3r0I--nrccxMciryBmy`sv z6te3v=vF>WYXwQ`opIV9*kImGxDdmsPbc@2>fD%_`Z1rBK4#7YuIEU0wF!W}MnLk{ ziAq6~9~(n*vB$`D#$=a&dDyW|em{&}CzuhVCfhTGzM2tYGeY*7teOl`3`*58FN zuGv-Ty+1`q&xd6c|w=DSZ!$8EZa3SLD$InBmEtHI9<4T!L<_9W^g!~)7vq9}6Q zbJLUCH9}&Ku`4a;Emh0wiplb2is+dvMW@F&t5>f_=1|s^^GaNA%(TUfW=A5pD*=Pj zpKABuxa%v%h>$}@2#zJkp`ZuOo9aJzx$(`=PK-bL#{ES%oj$%5IT3H2dLl6&5fAQl z1NIhN67Q8mX#svHB2RJ~41stEQLWn>XEMAGEA3pbv(3szM9azcZX{k(CGRFn1gPuH zIo|f=p{b>2-euOq|DSQMTvflmtReV(e&&6%l&L&KrRY(JL!7Q0CUxAWTE zztyW!#7PK&Naz*GBhS(dgw!!~i$WTK5Q4kT*(N@B5EHr@f{TUKkneMQfAQ7YjBC;o zUEoI&t1l51os?XqkR92wGl=^`g;(W(Pmy=@k}89`GAxKjg1x@w-Tt z@)a*KAqB#!i3Pv{AmeS%bkub}#?1en(PZ$7$}z(D+?Y># z=B>I*3G%r1E`xPG{;mc8L=?}_5zCcN5g)bQea-wy;ZN&z4&@h!ITr}Y4H&Q~;72-z zWPTf()e=JUx8~AKeWpD1ikr^q@Pv@M|A8G9Ehk=X8&Z0AAY9^$kur3JRw>?rwntF8q&8MCtI&oDKL0EMz=EWnST+{9NV7szri6V z3lO!63AME5w(4JS2>9P75Uouqq!*vZ8Cf?tO$bPl5P>A%d*CCTob(KIiz!=eWKAY5 zAs+R|Gi7h}sUxtMm5bWvy1uISQyaF=-M*iD96!~W&NbDUd9-MZtRF@n)!XRaouN58 zy`g+vop{%?DATPLU)HBbqlM7?hPBWKagDKFiXs1v4y@~Ar#!wz zF;z=)G($ij)FH7p9#PC(0n(gSENhR(s;mI}-Eg}lx1KztR@w}3_IB**_&c7L-Dg^j z1=wu2ZS?GVywi%x z*wH1?uOYSY^ywzuuE2OrSX%vE6n@mGD zYqDo(HHUVHZ|Q^Suxj|AUe@-cqYM3k(@zv7 zjh)hH9313Zg=OPG4X7M~)E!6pfQ23kDAW*4uR+5`03HbKxB)0zSqZZ6Ce1E05zXci z2DU3wj?tU0BnRAL>y@uh&BH8h)wrJjcC!{7^X$9!s)-_*utUbtZzBe8@#aQ{LKrnr zL89^HSt2w$v!XlP9ME^LM^hCUu?D(7voQF+#Z4fIFgV6D@=u54=cmmhnB$1~i~3BY zk}|Kr$Wnce66@xI0&&;~Fz7+U-oQWmU8irX5fKY3s6O&NmNiB&|{=5IX+`qth+>`CTht?%Rm1}&WBF+jWB=XhmE0M(uuVsykH*awjO zy28}FVS*q69JZbzR6Q)V&SpqUmIq&~wu4h0l3s%`Bp$_HJ02t)87o@3ip_ktLwk3k z(PK{jtfb#50XI{QGYFX>x^2&N(r-+UoB`%#1wewF;N#|{%b;?se>0ZW8>p%wgAkWT zAXS!RdFW*L13Kb@+aU)Az`TmyBMcr1T}lA8upr-L-25<0E<+!0jX7lA4stwiaTXw4 zgY2A$*h>IS(@ejCz2YXM6V^^**eseQBynDoZ{%A_y_V(-dGnjopnCU_is=Uwaxsvw zms8^4MnGE3#>ibwSK>^3G$jVe%CN3 z$toLtMV@&*V^J;%sDq-A&PNi;`8b_H3bN2-VAL=|7mU&@B%hI)B`C-al;L<`5_2p) zjtCtj-Sai+dCGJipX*`$KzPrwZxz%qK)sq7op%89vpg)iR1gJGoZoiIBD8q0M@OMj zPAL~NTk9!GG#`PJKtGM~LT&pF!mIQGZXU$TIaIpNMAH}&FzZUlQiQ-h52heOLY7y6!^ewd6&2@Bu#jWyz*wAQYGf0$WKF&t zN+XInOpDqf`HS-?0mM3)*$@HhMm8r+=!|nQ$4Sqt2_qV_D|-EuI~_MZgr!o9rWiT0 zo>FF&?B{LxTX+bZF=se4vNL)j<(ZKj#gQDu)e-u%g(2Z>CMV6PU;e!HdK`2i(U>G* zxXSTJh~arMG`Y=hT0}}T6O!`q!LVbmG&M_D!HL=wo~t4PYe-qQ(5E@2Qdpfx`@@lR zX(}HHc%JMONw~yh5d5%4DM-5!QfyhMwCtd2I7Lo-Zi5olzk|m5s&f?_$C!eBpaFui zI-G-p5>#sl0$}SSDT+*tg6NyP*!U%*!vqSLUJ=~3Fso1NM2T4{s8D+46Q-qtI3+Ie z9aq{VT`x`K7(9%v8HsbJYZbbN7${JG=%!V{3V|?@l>Jd=`-|NGBC!2cK~R2!k^Ko3 z#lfG-*7;=52DgPM_+25Wik=TAynv~6w8kxA(cI-Y zF4VGK3Zb3~1t~jJ0@lj5 zVi!|oh7kyxL(f>)x~9dNOX03UtEqXHiR2Z$DKD|eOql%+%(VxrEoix#a$7IcO*wX? zdY#!i4!7|Pm%z20uLNBqoo2WgOXWQ+*9f4xI0TnQMO@OhlH%>c z8pqVuf2`$)E<`}tdC6o*RhSObBZFsf>_A7m=sJtKZf8#el_baN1$w72nnopoWv5nP z94>3nCqpAnB{$ge*#G8fLL4!Ml$LL&)KGFN)WxcgKr3oW4)Y9~1Z(UIY;X`Q`G29H zWg5G{p16k8BM#kk>>m_s*TTHts#4B@@H?J$>B-v5*)U0p98f?a>iBDVqGnBP%@H6mUSyG-<7yfK({AUq`Zt5 zgT;)$O)@NUKv45kw3fi4bJ~zB9xTaDMiTA`CZQGX%Jo#@P;lA23e-VT9yv3(gi>@ zKLfwKj@|3PFcTtjDg*c@6_AggcS6^B@yTW^WFml?(PH67rc(96cCJFbc6fj={I)rv zb2__7u!F5JFEAAPQdN?a1PyGbb{h`EH58!w-JAE6lMp1Ye0joN&(uZgtnsAyj!2ig z4S>OBit-p{ST*~zqZ;+nT_%4zp080c^p~?sU-Kbl4gaQtzRPR9b~O=1of2p6{!tZ5 zz=4X+*CN6Qd7(;BL|SLEtd*(PiDGlf4txv)3K3^=B#+vgeL3FBrJxG@J0JXfNwYJ0 zH4}uG5>7ELW|Ct^_`1cqV_ayR!AS`&$-%`ucP|kPzKSVM5YFrUe7oV}f^wi6nq0_{ zdM)gRv(i)k?$dz!pGwe~jQt`!otTefSWYKoaS>$X3&c{!XfOxGLSYi3SH}DnZ|=(% zauY`BB0`F7PWxO6<5WpM!pb73W0R@+)so@P^MszpuDsZ-jtnWlh6(8qi5QOa7<=O3 z=@!)Mvc-eU8D~w;9#+66Wziq6}F0@9Tjo z?$f^k!~7T>iU7t`@0gOEiaLt@q_fFTCxRTvoX*Gq(S)79;kR|ED-GrK&goD2XfdZ% z?o3dk_AD-@%D-{H+UcPsu!t$<-BgXg=CRKbzlTLTjCD%5x(EIj&AQ6Zzu+>1qcL&+ zuc*(f{fG9kqwZH^){+}`QzVCkyUup zRWts(twN^_Yr77!Yq?Hjzin0VMg2z`UUkRpdz5vmjLLh%>bzA}{e#_|Q*+@yX?^h6 z^>xC(-qFN1o4?aW`?#k%$Di$Wvt_DL&;LpO?!YnoPxU!9kI3oW=K0^B7|+)wo~yjp zKbKtXK{~;04oO`4ON#X0;dcvkw}TB_k9ztq*o|G|4>#|Io9sUd>xI7nanR8VSR7&e z%(F|jx{6+}??c|ky1FU)7O{JK=XBEQEOWW+W1L#nXJEfhSgES4HEN``ObN;svgx(Y zXBRdrJRNa^K~WAu5D7<6{LtAjt3@;e%i6*}3;faHxxPA@*I9cy*O|m5wZ}8G;IENbKCg z{`~u+B>Lkgu zna_3|6tn^7dV;U?mBR3kORbt;$PTo99muf!6h11Tr(8rXoOZn9y*MSQ}BD*By4pH;{%<7d#m>o0yjiA{Nqh-Riz2`3PJhLPDxtW(W#m=gc6sko}=s5A25 z7t?3ymUg`J;Rcz2L=U>VywoP|&*ovoUI(I>@6*)aGxzDC0q+p_Q3=^%k`SUb-u(m< zDF!%-VB~>ofBKh&kqVzp1)tDn zK4VqwQ-oF_y|wnS^&j$;Fk6nOb~cW$%Q^_;a5_^yg2*&aY$3y#Ne$%5V4gv>!QjQW znOyMTb29b6ftQs35}DrE{vmv_#FJU>DC=Q>K^+5UmjXdsN0A>wuD}?!1{{j8P|>`D zS4c&tf@`@{fHwR_r1++9nnz9Qzf1PRbV_3T5X%M+dfx5Y!J%n|7f9rPk3lj-3*`3{ejr&c@qQdAuq@;*sJ=+5J>^iND zww^Xhh$q0m$Ds6cMGP`<9N5Sw%s~g^JF1hywA^?UyB8NJSaTke*u%~h7AFk5rMF>4 zOotHDSw09HdN7fKOigLSC-MFIH|5JD{QXEJI(^h;L zw?e+_v?0I~HrKoxicS7={X-^2JFoLCy((hfSed5Gb#qGN6^G0x&p5fQk6; zS6x-Fra$tg;lz1E_=0j>cSsS_<0L0dh`GApNF7qOJI9 zv|-Wl3_OssPOG~H<>wY2ENVJEITKgpeA3=_k4oM(7&N)nhh`+W%k>$1fs@O%A!TAz#`pf{r&9t@Pom zm7!V?m#hhFk4KyWNHffqigjsIGb00p89!igmN}133p0%?NlIzTEO;AQ zLq~yOU%xhCN5!MqSPI6<&lhr;d)w`b{a*@9kaA9BUUu0Cx0KTlXe1kA5+d(;0#?q9 zT1(doOr#Mxj&c_}M<=0NBIXXPhD>Oev1jVFroOCgiaA8*r&yXZG_;!$D}nCfYJr*g z4QH8;>n-R&17rwxi!Hf!D&+Y>MRnL<*{5;28L;&}kt(BeystF~0Lj+;mFCVLL?F9s zEPG}{0AEzgFW#7n#+>2OI-fZMNF~w^6KLIoAU{-44zQV5-_5R1=ViGOyj;O_j+Gss zk!ha^cgQtbn6wvd@G^A3TNIm&=X^)pHwCE4b_v+0F#bN7OezbKjbrE`&e`ml8kxU2 zz#c)0R(A7cK6sd(WlTPa0CIdAThSe#o8)d*#TGS3$ca5xW(q+qa)gV)jYIta!QgiDrtq(Cg367d-tLZP zCQpc*^vsm%0WAk5!FTTEOA^J<%QeXv6bOq|^O>)owq$ER56u{1Ap0Xhf5YUAZX94o zh?(Un^&+~-$P?K!{*k+hWR_Kz9g(D}Jf+_(13p-wOWMYv%Ray0h9yo4^-y1sdMe== z-z&{qEbVgP%nr`kTiSXs004#F5yO@dY0E&s4b?G;VwrEhYcgUAp5$e~JRw&o>uj_9 zEMR#Lx2(cQN2H9kQTRpyI`beQd%KBRy+(HV&QdWtl=1*=M0XU|qhYobHHy@T)|;h#ioj3>+a@2R@;Kz`z?6 zWRzm2lrz5oL#Ir8T%~~HwKfTc*?s~7@{~bns?t;!FXKpwy4VlS$({J6+&7wfg6y(f zJ}bgfmuiT+bz-=XoL)u*hU0v^^J^1NAM8xxvFB=A&^0B}T(MZV`Pti?_)kZYUrx>3 z5TB20{Sgf7a)w~GKwPRxPY!W7W8ZKk0FvqDeL89+*K$XUF@NY_x4l0LWV`*io0zDR zgX)^eqjdTm&H9Qs9S;x`enY!gf9W7ntWD^J}6B~c+ z9qfMaRW$BZ!7c*@4vCkJEY<}6{B>2noRcHO^R|nR^m4^nFXT&vsMP1)JjR!+3Ht2>hS+QQ2i(H{cgD)5;>~4vYHkgp8pf}gnSKs{Yxd0 z$XP0m4adHGz6^q*N}xnM@!p-=v?~Px(b3U=ZS5_ZUthn?wlcq%0=qCjbfN8FS8n?E z`mD@G`pL@kQdgYF#Lz-gvX+7l%0U(LD982=?&xg{ zbA?x*S3y7O=Of#-;D2)!wZ03SA8l6xC&|s7*LemFK9b~5-is{PS+@|+$|Gbrodsmr z9#e4~?SEbS9R9Xuv~Fs)!W(I7r2JfArE|aQyEHR|WIF$c(>*xg6 z7aNb0S6ftYLj!r?x_WVh6m*@{0(gR_^3cP&+67riyMsbhVtOV!OAd-Ox_Z0VmTnd* zbKMl6@#evgOxCVnhpv{_aAoVxfWM*sgmCLTZ8MBVb{-MOI>=~p!_M{`-NFj&5(|gT z+ty3re`gQ+tJ^`sc7ywf^f?Ppo$*yQ!f#es*xE-NYXcLKyJpo^P>zDuEL<3>=;?y` zIR;00xb~}L?SK5zAj=*BX~V-t=; z%doJ+sCZDY2P`35*kNPyv&!j9s=y2@J(>6y#r{dwVUMPSLf3H$yoBf>74^l3vk#7$ z(qPLTU~0ks4)&har*sovTi`w52$vAIZ&#P&YYC^8oHK}d!A<;zOD8Uc?WR0Dj{9Ro zez?nQ#vma1cQzetk!VZ=GG!ByeHEhm!MnuU56Fp4pk2BnL+1`KEdnNFxpzML`8&U) zoz)I^j->Kt05(GX9i37+0Ra}x3Iyga3ecf(ycUuadP#9iqoz=r6_CxFxVMbM))l1~ z>D6BR>Bl~SF*OM#M;gvYkPw0brKv*Z9*Nw8Hm%@5{SpCT5x2mZ3ZTqr2q|MNOBouo zIWmBDTbRku`qfFzz0oJ z5B*0vD&ypIyk1k5d3>tcJPR;ioc8uWz)APFj_S_hwO|q$Vf}ui?%P(Vcm&Kj0#z@& zc<3%_!1-{>=n|$*DlAj8vb!7dc?p{TrsZ*O>E)SHv~Cc6$2{5DP=rxOuw$*t6(8%m zavE?iu`ijbo!Glrb?h|Upz?)dFHs%-ML@Gpy>MxK-(fL32nhruaSG~nd0u>3gZfkn zgMSc?!|-PF-%pIF6vcP!ifD?#9yD&?EWy3tiWMuwu%stwYWX{>|{6rh47zhC+ ziwvRgXlwYL?u5iFA?h4&R1z=4x=1*smtm?~EX)b|FSrrmc6TLYDKX4x$-V7+ma;#M z+;=+uE=V37D+mcT086&^JQAa&@FlszIsyM6l`vsP8LaGGD?^M)j5tC#LL}|H%lk|u zT%vqKEfZ9su(QQ2OKT-5v~K!03@Ke`0e3XR8d}RS$%u6UA6HS)^fv!owC6`0!10)5 z9iFWH1*Bb2WF{5qSIMrwNQP{@>Y z)BAfGbyzq^Wajvk`I#k&e7{C7M3tz^MsyfSWrD4Pr{(<{I~++8ZlrMD_+Y7UKp?N0el_MCmmSAvV~ z?=|8{MMJmeV?grBc;_Lty-Vn(oW$9#YS9qH2WcIjsrLL6I;Ec(aGcVgbkanT-4@nH z(8tu&zH~$9CHBj~yoxRbTAaz=gpHSF_FM}n!8PeNrEGp=@|IM^mL9Tj5qe&I#AKO@ z-bTeqQH8;=lVe$-e{xK)1Ck7U&f;XKs1G}!hCksQ+ylTi$r0MnY}UY}Bt;$A?2NgK z_XzJ(jV54O5|b}bUqXNG?H5J(y{S|l-=*FyrRh;)BHHPi*@byghnxen8T6si*VX=1 zwS>=buA50pHW;h`;%5XMni+If3xcoCT&8G+C&S~v$YG3i0PTeXMCsC)`E;pdG$~^y z+At^ZW1RenV5(e%i8gCU&;N$fW*TN2@;vA0^`1pJFJ)b&p7)qj-)b`=8B0F%|EgY5 zMY(-lH%C~c$*I&sU&^h%XB1Ojc#*co66w`1Yak1UL;!u{kX=tabL@r#A{3yu%Cx$+{ zuFcbVZ)S4HyCzw;xLdu*JpI<_M-r9Gf&gn;IJ`POFf&;!C(28QucFO1q~x+Hv1?$TZYr2w zsO6-d&`{nzrEG{=1d=(@qFxf#r0n<5=w5A|E5lrgq-QcZ^fT!mN}ZC|3K(p$h~At$ zp!I%jAd^#(hOgTbe+RK-vpliDVe0)V_GbsRecDvEwdBFGd*5r$_xkvu^|7Wnscg)> z((pxhR(1sRbpk1$fFir?HS>S6oZsDgd|x=5P?5)gn1KXEJp}&>L;? z@Xr{EDmBca6a(po@VHI-F~mWo)LWq@OR6fla6SJbUBFmX>3x3lvJaSU#S*Ys!C(rvEETm?W+`$%iiBcCD#{M#2KuK$zTb z@BWgQ8Na-mk&?!vIFrdu^`Mc~u#a?zG^YW+4fN`A4B6Zux=JS>AK zXDTy2O>E0mTq@+YTU(+ir*BAPW?JY&$DSs8fuOIFdF>Fn+4_CD+IWZyzJ!*##xw&l z@{M@)Mdq-+NIKJ+(--siUF&^jPKaS;KJ4x9fj{17mf$5x@zY&@X)>oG_hxII4`2(H zNBP)&eHyU`KbD(Xk)v%2=7OiFZIoDSi#*L-uA;%X1P3&G9PKKz!Gk18aZpl7aAUS( z?xvaTpCj6X6hndjrWD9*Zh%i=UVuWXIn%hj3pI_0{pw|MWnVMq&PA=>agv)$SoGfL z=@dBz8p=3naf(&Yjejm6NlJ9FuSi9s)c_@qAMEG}7u;fq+IZn3^Z)Sj)YhA~w5|AH z9N#^$ZfX^kWU$a3X51ZgCyW622Y$>J-q9qF^MX)>PU{96uSCJT$oUSKZon?i?v7`v zdYq#_*sEAJG{4QkgeNKxN4T}>{8oXUcmC)TuCG+7L(Gle0%`o~oN1rtj;hL}k`mo7 z!%%rAMDldsiaIH|=+-{}z}0}hb^SsRLp?zh=|3iD+$o;!mosJ#ai5ZF3{K&thNfM2 zxl?Dmy}{;SUTb1>^6p6C1o0Y-3uoBdxuCc0$v`urRkOD&YxRElcY5#D0-UM;^YYT$ z?Lf~91d_aq`P?@`=5#P@<=K6&7{GmRk@9jQ$)13u@e0M2Z=pNAB;cn zko}{v{CSnJ>-b0La*WDuz1gGc!s$f5K2ZqpR?hS7V0I|wE0eO+GRQNNv##)pZ8p=M z)b7v%i})%*Q_)4f2a$f_Yb;wkV{f$d2l!c)-BOwIza7BALO|3~7Szqltn*(RSpENu zSXZw984vy5)WRSUASMt9^7$cPjK2VGgUk|+T5CiaqIty|s?kGu zZu!9(??PW??UfgAn>ubE%f-n^JMM?sBZZ&u0nLXv{;_Oj7n)d!Q;ScG+Y#_<*h{}6 zG1{41L1mN9dne@PeyRGN@}<}}0A3ay)aDzGJywUGOqN?#>xIzgJRUlf?x*1aL`;G>zk+u>rN^~P0aU0Q*;+6X=a1?amRO?g&v{6j$B zqMCDfyGnz37Afr}{I)u*3p5HM!}ZEnoIyg=`DZZixIgJmf9yzU@sg;13a*Ev225Fh zn&;eGnrAZnBKidk0_d|GMfv1Ua}20|bZ_9xk|NHcyyRD@sG~FtU4t3K_IGYzdV+GA* zv@4EVw{w&CCY@H;B-A?ZwgUN0p#*BkzzdfHM`<9}+1-!is6!VuA`7Ehig6Ms{_9|u zk)t`%h(l}kR*FJ`GMv4ILr?<>$zQ<^1=&e-$l?AY!2)7_ah&8KQE8XMUzX+>WcQ`P z(Ousom>}Y1e{AB`!b^NxH=!wTra?pZ%`uMoZvT=R|J{Jt+X01$itLar(nm8m2HaI6 zBl@P(0a@3$is=)|zouYD&aIB}00Rr2H}=NRwjEf4C+&?#l(-&4V^+$eR8rjl7+esi1&-{76Kn$&7uRZ^l?z)RL6Vo*$|X9eNR2d&{rS(X+S zVFtVws_?ZBkJ6Y;Ja!Y$*3F|PC*4f?;o4i5yeKyIN^Xo(82Ox1YG?)dfSiR@qS8kC zLOWs|68}!H^ad&i!3P8x*fBx40pcI<>{l?wjQY6Ma8V!!tGzv5Gia;_b8?aXLh#+9uqgTTaY%(UDs*?O8f0h7hS0f5^Tc_iuXnAX5L|0mdPL`L1WDr!? zBop6e>Qr={Wi%jvFJzYnMKbp4^tCRWASg{2Ms|&3*ab_pe|*(ksCL|`$Kmi(8g8Q{ zUjSD$vm&8Y?UV0kj!$IhaIJy4W-Xj2Y$TnJOad9J0_sK`C|&4A@SXG@F&W1yCib0p zQoZ|Bh2M_;?7wa2NGw^-BMu8~+>^Yq$Qm6v|dcR>i#hT!&kZ z%`u6PIJl#G7`34@yBbVDQC)_{WQ?J_ zr)vzgNO06ZvF!eli-p(2F)yZd@kQZ*hp<2Ht^;f4g*fH`?|?J5atHe+^1)`nrDo!` zwt+sojc_8vPPK<*HGECT*XE$afiJr>emdGPv*orwU$Bl}O~LQLvIYXxdm0uN!Rq7) zrwD+E3YJTIs}vE4MroJpoWx!;%MT?H$uLH#`bW`n=;Kp~yOuRjFQKYE>pd2wO1avg z{*|^xV`+S(jK#DdHW;kQ{SmP|!Ia{38zCBM7)@TQ!Ci#G7lqfXgKtZ&E~$RtT2 zRw3$TsQaM!#i`1=sA?R{YLql7dpYJKnyHLIcCrByo|64YleS%dCm4%)rDT;zeV2;- zy~xcQcg;*OI15cnv10=May@A>9omI_Ex4W+ zk(;#|dcX;i^^tM)@WoPgDNs9cOKkzrK=s_9yCNiuXIU7`5c`Gp9W}Q`lrBt|xXQl(6Lu%=cV=3LQgzV{ILPQD+Eq#Fgi!^s z|DB*OHq|I?^6HaSe9&y&9N1V0P_1(+yU+}4$&Gj7!=eu};L+|5b57`>W&>^Z7NIyQf za}R#~${4{rg%smW+H=YrlX>F+&<~;}L7i*|k|O>HMg5@^`^Y%T$ER#FAUN92Rx~!b z2Se;wt(h)U=EZDxUwsIw-p|2}g*+j5TwoSAp3;SmXj1VEgOw0#tWf%+Mqvnsf28C} z9Fc{eJ1(KAy}6dcaZauCrffUCnHXtLV2dty6?~BgrIvmhKs&M0SL^d_@H$k1+Lh17 z*Z@cJu2MMytDY8ZKNe_Z5E&Nt)oWASviM9faJu!fhGRt44=+V;#$*z|p92K?pb()i zYA$5&Y<8OIYkZ|fds3#Fq#BS>kAlWN6FVLhxQxfnItg0d@0mCof8S>e@k} zvvD*!C&V#_i+Y+13GGArhm@6N(r9b{KG>tptv6E1rX^TmL7uE7t& zi!y52-#V!|XT4Hqze2qHpJIr1948zr8_MvjE(D!Xl{;nEX*)&RWPq{~f5F#i&1!*oq(G@EgiQ+GKA!-1g3{f-~N zQOk)YKEj3wM=cB|tJ!Y@rsf5PQ=bM$ zT`BQS0k<#%$s7=sVQ$oO8cuY#D9H7tqE;N>GL{yv0ngiWN9Rg=%wmi{9`=PWo`@Oc zTmNu~jp{9in==EOf`;_<2UdQ*t&Lle9TyfWTao&qq!!jQGur*rj%@ULORad>QsTKM zmm+oop3ix;p4}KFN&S)ZHXPRIXHRRcL_6DP{j$#OB`bJWPzF zfa1StDfSm4pU93H{vJ07frI+*f2O711|lciQ?TH18t); zS0+_rrIga#$!r=-10YiV;lq3iS*keD-@ktWzUD>@@A~|{4u*gJrn=~w9E~3NsICTW zy1cdgFn@1!=oMXr@lLgE-pf5((M08~eCPap*jC@uZSp?It$xkNjL5UO^i4)~f1Lib z+WK}`w>v#{ZN{p!%uUu-Oc8mFckS1%zlrUBoMc)i*=TP*9`(~t4PDD^cJ#jRS+vm5 zN19fD{kgv1u65MVyZRCm3Zi{qMK$p~$cuuu8LPDx{ZVFHHRAJ>8YI*3*iBqf&qbvW^YFb^%YTPl|JdV3)YMp#BXcBV<_-43yd7=NT>&aD@ z_7ao|Wvxf`2ylB%=-91YO?qc96x&(wGQRAj`hvel;K7D>lZ*Y7Hkhh1*&^`eUWGU5 zG4N8u=~)YJaNFNFVzoy*)2TtvAufgrxr0~IbV^N35!k~x?=Pc(lt zIXVx(_lEMbrJWs@jH~qTOwsHj`*8PxQbJm(>(Bb(E>!WxdwCKo7md@;-ea~~JRX_z zK0#UrRTt}PIV4RBHLNQue5eHx88|W6o?hYlVcz781$>tJEUVZ;*HnV>X<`j z;=cPuIl5y4k|ikDXPfEE&|4sKikU4{KS((u2ezxwd9LNLz{-n%HvX{ebA%s*K@GbZ z&zB&-FOlvpKwV{#B+`~VdTb8qfyhm%Jv-dgUM!3@wuqSO(Dl)PHQ7ovcb2j>sG_WD&{79Q#mPf%8Gm=}8$nXdxoIsy0ECkS;R7yQWw>#HzH(+V@Flo+S|rs~?U$(8!a-H*OVQ zHdVD>%W`vDcECs^sDIebq(KP*In&LW1~wOdkMCD$V2^u;i{J(V2X)#7X+rNAjJ(>< z^%(~fIp%p=a5~<=YdcIg07`hE`am_##U1p(O=y{OMd1lcxTbZO6k%O45eUGkl=rWa z;W65QZ@V5F@~$azp3TEyY)%2jx+3k6P&o_6FvZ__paE7+36w`**wA$K4bDo5>DXtG zRl8{*W4la#l|{fz1Fp~4P`XI6#^}C3Z3+U9bFQkNN70f)4kv9X?Q(PHgTbm*J&3vU zQg$HuvJt`wf3Huz9vJTvt2A2yG%NWMy0(&}FjU{97dJTeuECr#CVm5sNlKNZcK+zY zLFTx48eY3;$aca#!hgBHI}-!PUbtn-eY<@=S$-T2BKXG15xT691WQoV#M!V$&p-zj z7}=u!JCZFQT!N`Zh&eu~sLwuANw$U3>z6Wi7Y})7{ieV77(m^_Kn4yxQX>;Q=km`e zP<4^V+$u*d??;f7#)mVzcK`<&Nbk&f)W}~Z2 zp&MSnhKIF9vUP@ETymfBFjQIYPP4Ek0oWLrMSIotn;&_Xyjuly7AY#+xm&JBrIt>z zZ?N^uaPaLVUyXAJ^me9$f&CA;a5UsVr3lF5CcEH9ar?o+OaNPc(77V}n7x~kjR(r@ zZ37R~a4<8%J|i&*yfqG@Pm6|W#l7++T%~{0OquXiFjhXrkcp)eWFDaB2gpV#XM#K?(v>{O5yKR!2vC~qG z)Wmn2_<}`}!A?iSoo=N?=t;kFvJ{Njrty;H4S7O(v+lp01CdJy0Rak*E!J`$@Di5< zK7o_lCHHrhd5d?7Y_LmOB@_5ZQx^VB3l(2wOuP!CZZ-4C`kqOlzSoq zYN>XsHCFh?H&#!^DlnM4jPH5@x2r054fdS}|6p+mW10>ilPW?m!r%Wl7Y+gSZHbPW zGynSeKX3wn_Q8|^E7hf)vyS^@KU^4&F!i-H{uyUgTvjPF^LW{)KQ)a@-XRt%EN^z_ zn@+=?cf0FCa;8c#tK<_O3~A(d-%#9I>~`0F12u{HWZxd3@;3RD+jnz*uoS%O+lH}C z`rl+6DJ5Xq6}tV|4{`$@K=v6XpyY+XY2;#MEDRB{;5+b*ISU?`meZ#i^QlyowrS=V zatPMi42BW*d%3p9Z^XpMUJopEiQ<+yPou;rCHsg`0Lco&;ENa3*qp5@&&<7;rlcyx zWPotzrLh*X4c6Z{L;(Ir{}!-M^g9kcc(Et}7J4Ld`Q*3mYP@3PfTY z4R3;eGp--13%7new}9Uwx>-8;THogHvagdTDIyNRXd53-lFuMLL9jd2V$zDLkDv`Q z7X-4{@D2JiTl6SbK85W$3y{M$k&70-hQvZIC&+~+eY_+OiXUys{=}Oogz=|R6}d3h zDN#SdFXqsAxwfIdHwuJYL0$lVa;Q@v&q`f-1ReC`S|9ypN7MwTa;$iW6_BIZAGcGF z*81Y}K;B8<@4ihT!zZX{VYx-m9JJ;XX^m`oLINutf5e&c+#}2y#hI=6(V4gC#xaH$ zB<8~f5W^#mj`Yw9piO;3#ZYx+dJVp?cB4H5XF=WjwS<=%x zSr(Gb!V*i(?oZ%4K+d1t0*~>o8RPi7KPf3xstBtQIG^Li7a^nSTOAVgr7~hslQ2y) zw~xHy2$c3_2@p%0KomL}$g`~8U)e(ihf+ANE(W3aRefnGjj)7LUn4do_Ok?BNxm|2 zbgUmBJ%-m>oyLx$eycPQAmRW%5INE;Jjs4jl5-8Vr)G!o?c_*65)Vl>3TgRMu!o-Y ze{1a$q=+zKRByvfI49pGWe)L)F@I{EzS>U8$_V1i4I{i5qd1+lj;YT?)8bPX{fa&B6BmRcY%RnSlcig+B%D{j5MALg-yk|UfFPPI?#%P! zj{$*aEa44G5o1FY)H9f;XQ06(S_>Gh8d)(UFa`zet0x1!MsTt0pLfSFxRojkjh6y| zTNIM0`{J!dopp*M#qQP)6a?EtLT1|3ZRMjBfXIlusg>py{HIOu1seR&uRj7`vt;~% z>5<$Q!3v}?Iphy$UtkTixn=58Qab4qvWfPEs?Hev^HF4=eyqbWVTpK?gJBE3SDB6x zLhUyNMg}y(HP(C!YbHnuW?uj8wMB{qZ|u7_oaWf)B&(7xt$@e?D|+~gaB9O>=|aij zwZp&9_VzZYUO4~3kUsj|7W~_2o6e1!3~bc5Tj5qre&zw&G?yVFCGVPIl&-R~y+|x& z-%uUuw?H_FY9Kh#Mwt?En|<-6O7~O(~SvMIcy0(np4DfzAod)j*%L_+1MVg*a~iA z?(K4gGZPf{kj~D3dtcpO~{C{4`_>~GTM3zA}f|PkK=j(KyoWo z<T;HeT2FI|%+>_gT-L{4 zVxJl9<_i^FsH2`wkOsdRlKO3EpXik=NMhdO9&liGGd|{qKQ0>~G;2tNrOkw_coR5u zh~Fr8t()w*c%J?Z(3%*D2m#YkA0Syp%&caBSyEktvb&vld7Wjhpix!*8zskT4Bg4e z@On4K-Np&!Bgl|N(7VPPupXU78CgX;=~|CE`SBsb6d^^|C3zlAY_x8!D^ZbZOjIhIe2S_qL(D~tAQC3{zxeNW-L76jg{%)(%uF<}~i(7NBs zqw>*{)7yGPJ2u1ytGk!ROjj^8-=qL6*QdwQ!Q{ac)ad}#ON;zx+V@GS8_)$`BjX=@ zL;{h&A#SZuizqZ~u7!+59iY7#StsLp^4?`jm82Y7{kghl7QZDs#pAtB!Vaf2-TFBp z$@o;r7P0!}gBs9yBfu`v55%NIXP?k66AlM^sbk8{5V-@D16AkxCHgiD78Ci>2G+y zPF%pT|LYt^rUIh&GNSg(1b4#XpkDtEoSuv3ndg!u(vvrHA3-oHO@KcTiFJpszt^vQ z`T?kMyOot9altXjT9jh1Sbgpjg+E$EEV{zvr;;Ur*R+UaxuxAx&6e-OE$& zsoOpe&o+8gt1ha}>z`2H(dW2#0yg5>Nk2DS0^oL1rtpu*9!DBK<+Q^7ov73L#d93pv(`{^ zb&9*H>1--bWf6ViW)@zdsXb;ISB^b(7Cv=#;t1k`b>^yVl{Y(oYH7B^x;UCpbaj%q z2>spX<&9ijEI@s};0)JGj(WZUfSmkJMq}$Hkz7Z8Bw0JC{S+{a(OHS*se{OJ(mV(h z<yw0~YM!QQ-ftiLsPqsxiK5|k z-T6wVp`7~EQ6|cRdTg??#%(jw?xu>91=vHo=v-{2n$E+7<*aqfnu29f3!&-8JONWv zj!_5QM#`!$eC!|Mf*sa<D+-~@Xj#3lOos%} z^U15M4^5~1VyKMRgbAUvV9o$QPn~8;pI?&C+KHi{46nqiJezot!`|{3>5(S{?&ZPz z2nTt~4Pn(SIq5y5)b6gjt|N-Wr`DtputNntcl?)bN5Sy|FqsOVme^6D&2fhCjW8~< z4eBHcqC4^?+m!qFZUiyq=Tu;Pw$D;)pHj|#)7)d?7XW`Ic^5!K_ExTX8U#y%5F+xq z`|&~-{XbjJRYd}op!f8-OMRg2e)jEdVM=j>9Tv!FnN(&+@7ETbbw}~<4$bX)Risd* zxCGIZ;EN>V{!p`M7AQW5YM;BC$gK}V_eG1)vK{pl+^O2seX8aK%Y22`qwpq2NaQ`w z0AXo)EU)~*7O}jyfcxg%-RscbA_Hm{DnrqLS!ClfMITg%RoUpC}@8Wmxp&NA5Yy+C3GJcR7kGigv@kbeE{wYSjkILC3X; zH;{(|M{0}J?NnGmb47=~M#^IIBhInEi)sJH!xUqYAJZR-PN{XKG?d!h#Gnq{R-hXK zNcM16?rWjqtFFz`dBE8!Y*>)jY|aPB74*RnjL#f@J@*|N%nC;b^%(Jrt3e#_@vaCKLgIot+=)!zPiq7G*d!6StpYR?klJru64Hy0zq4f zQBFtPn@7*ZjpXUSmId6P@9Zo z9g2(1Z_Mb_8&hu_@Pqc;Nl zJ+I)Vlq9ETGK}cvG7!Uva6h&8QC%vil1#iG2hiy0vKGG*eR~0Ava1D(loJ`vBj4 z=sF+N*FY)@)JvRYV`D4y1=qk$hPsky8q<@|B~9{Bu9%Gjw^h)P6=D9w?u7GK8dlky zrwOVt^NE_m8k%)0ruDxXl8_XLfD_YjDD}<6^S{(=%#c;1Xaq;>80dJ?VW&|WSA!#y zIYeT`znZ;CX40(wk&l7YC~^o*VoFEX_8MMqcO>~&Zu%0!LibL{z+MFtf~=`-7(XvH zeuV3x8!~wmA&*n_XX`~I6W@d1xt4%%iS>+%d!BJbqEMFLOj%*%EjuA%>_ph zS5S*gRdoOfs~wAdc&0*~5vAYXB0&c2TGIBqM*AK+Hhscb5<~zuyS{a~GfG}f8k=l_ z=}LJ!vf8e}CXEjWO!H&@X$M+^slRG&gw-_-&~pHiVhIjT)f7U|E?tbj0NMh6iBc9% zz&_s0ndCw#S0?K^$l?oC>rM`QWeY>u6)A*mvQ8(?Jk$PEjH;s(E`7RIob#9RbfP=s zeQke`V=v*)$CgJjN!yg46_!29X3COw>z_WiDp!z9Ez5*#bRsh;@q<%w?rJal)vr9? zV%Mliq@SD_)$|EqRY}fIb{8}$#l^_X1IG(PE(hq)F?DwA#heoll<$w$AScOGiw!ua z@nXpM$bnaYjX+`-0_>?pV|kjK9)+c~Skbl<5zS_LPiELZOugF@&%5e6uxaG^tpL2> zVF#qLiq`tH1PN!Q&I9yh&W57TR_6n`afv{;(Du(`HqeJECNA`mC;nD_mvKdEles9k z`lf{FD?**R64Gkwu%MUvrAL5ad(gsc*+7?|01!n60U{Dk@p#N+`tgN@3&hL;w$w`p z_-S;6VeYui0v_dThzK{ZA5o1Nmt{9)yWWPKcX%e$Y?A%OESJjz~CeCW6@;U^=9g1anPdbQ5~?yn4;drBvudos2_mn*cZsFcvEvr|qvaGFjgJ#n^n|T6u+#PXYmh3(vOuW`f?Dj1= z4rNoS!e=p1BsAr>P1Q9V@GP#O#^^>C!`psjZ_=3(YqWM%(%5evS=|%GSR8Z@Oh9^~ zhsKvDH3GFW9(9veBXgzEIh=tT7WAvw;956hb*0v$Yq~A>aG`drcL&`Q2KUhgZpsY? zodzGeQ*5^RHX&i~HN&C**E5RoMh!g|JNF8}RQ&~V{Lb;8b@yM)R?ahWAlr!grU^~MvJU);TCaKOt#Ah7& zN4n0h^klwv`jeQ*o|hl{cHCOMGqafR`8$wvXX7xWHe4i?f7LfM+uhWMoo;SA8%H#X zy4l6^R}_S^ryQe3elY5h@vOdo22VVh4VrWM+v*Vfc~bF$;cI&m}(T%~+F0c_ZUj(Z8!yq;N7Wtl7sOlqy_VG~`v#m79HS^L<{a8!*RAaWYj4QWd)@$`{Hq2oHTfhsCC1 zFAX>B9%eHzzhL50XZr!OQTc+|sjibHElQtY{`!p9NA}lbwbv;PwT3CD7%|g6{_s zEbsBdCjU7qj$4Qf?=ANQk=~FSCphO%%v2e2_>kPU+TuQrV>0E-$RIZElQDAYBQ&&4 z4-jTlH8)|c$5>yv0I$$&SP}G}ACc4t2=l^pC=Bd)G5guwoZo0hZM}Y zzPW0J(Coq#Ooi%GUP5 zUCgujKpIB-jPKoA{?LBhk<$#vT#|~*0J*u7UF!^5#V5ave|&8No)-(c0LC!x5gMtn z6)U5+8l|`uswDi>Rmd0IXHJtPjve`6M{ zu0eF!JsPrj!Nb^AFny-7M4B`Hae2cxRs;3Vja`;%$TW^iblD8(QIJG`}S!F*W>HVYvbMs7v##ljKdFS7BC zHnxY5Q?OBcm%4)d*~BeBCkDz6PWOa}45h#t5G%aM04c&1AIst)O?U#zjZp2Qn82AHV)!j7&cwM78~B@lJxV!2t^6$iOQLZ1=kgXljI*QWuC z+QF5(n_JjrCqEp8b7!*bL17PpugGKhyO|AxQS*su03(MYxmk-Dm6OR#iodA7RTV#E zCIaSq9v#YV4r(8Q)A*SCI_PRPvQmIf78gmGE6`}d*}dR2W5$T@ue8eymK@J60}|ZP z-9Zjo>hBW>RsUl!MGfV$sd=Oj*@?iYjx#Ngo~g~DPg8V;OIeE238ez5JG_=w54Yz~ zHt*JOdKp_V4HoUJn)#i~R2a8|=q8$@Rsc4;Gt%uJ8i_;OIh;Jy+B5!J+rluH1nVs8 ze+q0wZiwkIyl5l5p&~OB6%?FIOn)GDXPvXY;1@$yOnFYUAI=u~JmNKo@(v<5uf{>p zu9G-Q*AQ2EOp}VTv?J-k*vyY}!u|1>#wS}3a~4z+14Nx`ju6cJ^co&bd}XM45F!D6 z=)i(}kSW9>DX~YD(89Meo)I+SAi~p=C2-PGe0>Whf;j}BIzof;C|F4s(SuTe$|sg# zoxX|;IU+A)9*c|G`aSoPp%RxORdC3sI7OmM9jR+$ zMkRK6Y$(&cIuloFI2u#XHW@)VIDlUaa1m;n6`Z;WBQk7DFd8L< zh@?kJ97Go-k9Cj)tmVZd==9}lie%9>gb75AOI&=e)lqqyj_;8(lk<3A-S<#O@-{A< zOfye_fNlgM@)C+fzLZmw4r6j+s@Gp;V&Sc+xX`6cPcDBaUjh?SDws-1bfeA5E*F8B zE7CLq4+#Z;uE$ipzl+hUIl@DgZu6h_rP8fIL^ z;I0BJZOPAquuIh6`|quYltmZYL)cY$NXe8b>3!zR=-h_I1H$6d(VS|Xg*fx`jq$lt zXi6G{0K-bI+6S+90YWr*W`S3KwQk1X+5s%haFPzYfV<(OE@*^Qlsh^D_q%e*7O+OT zPEup+#hXq{Yg90N!0IqAWAm-zf~bt(Z%LlVe4zj(SdTjP3X`_Twz?tmF^ zolu%oYj78?L^8zZ4>doL0vhVZBnH<^)^#8xwK5je=SWjj;VU_>yhjOPbdh9bO3e>F zDVOZs^&=4S%}AN8$|P*jaJDjkk3PRa$LXmt z%IH(n6m7nDS7zcc>a_LqbTra~U7e$(E1e9P3%GsdpKJep$Md&>0%3QK=PScbG8YZLLC=J=lR_3RI!% z9*ur!ErW_IJ3_eDivXW(9YBm~LFD#=(d1Qq28r=S(|#p-pRpV8ycm<-QTB{H!5WH2 z45k+3h}~@Hv#1I_m=zJ{)9)OX$w4W&R1bvnp_T1U(UZ49&1yUWWvzrS3eeh>b|-U^ zJtI($?JdljStx4ZZgr2NMZ7~>#gYQmV64fgE(^RmYSe8+gOop_x*Sw(2rIJomEvl= zWs#|IEHxXuz)gCf4Q$D*8m*4O)~nepMx!U*9NEO+{#2T5apG;Fr><_l_bGubWX}y! zHgk0OfGDilx!c{6&8`-3wJ0s|Xp@A%;SehaN-L>+1B6En|IqTMcWpSe%AyemC_R5V zamUSL_teLgM*hL6F0`p;c0o=F-}oy>A5!Nt!K3EJ&0RRTT(%B+cWrhc7gqQcO9n6Rw=sdD$yk6EkCF?d<+j4^DeIW9J|uoiv?DWkX*5&-25QqV z&j%q-x5x|z2Z*8Ew=T`pgef!O?g@Y~af9#Hb0j~pxb{g} zm!#|686iNUSw`tZK;&@r8Hxy`m(5?86o0p(r_V1{lw%}csuc6Qg1-r7>5~q+zv|)5 zJeF=RAh{VO_FxZ9&Ha8I8dG7mzvlw^>%ADym!*{oA(b+zzBEA6Uv9APNfVDQDYIEl zFWxwETvJuXsNx^f7|um#oYCjsWU%aT57&d?qFaGU2Q%8269fY*6Sg<5x#Ig(m8!GU zomN}r;8=~Cmjmu&^R3G)_|^|C_0&?>7CekdnM39ihdlZZ1c*B{C%Cvvg3c~U2rAY! zXQHZh{%PJI-#pgW5<9r%mmqJfE@tHZ0>)06(c_KRTig}q7(lBe8m>UQ>7TkIk^zqB z)c`r$SXoqmDz)EpyRMt~xjp7AWbSk(WioakAFbm*cn4wS)df)1vW=vOn{-&VPmeJ& zyJavyH5Jdv_e5hUiSmC5d;LEup)E$V{`R`MAHv@M-hJq}{Y)T_IhhQ>fuk8^+2AH1 zcKBD?Kt9vtW*|+})zZPVXkNdq(D|%0I&LRrE03a$Q&Tb>N%lr#CBOWfobI;$KQ1-i z%4@3Zti}o}CjH&zDA`pE+!t;#{IH{%${}2J*k=k~zC!RrtLU=_d$rIsJlIifJLyLC zHrd0~m{=jax_>AHRPRQ3Pn$OZTUKX|7zfV?59Z7D=V|xRWTo0T1x^e2EHjKUjvJ+Mj=Aj+33u5XSd zXt(zR{p9tT3btj0j8Avf+6_p%SI4~tPKzI|upbMGStm!mx;J%Q5aOX3))i$fG4Nf| ze)*EIX*aSoW~}Tz@)~rg>)J0ECv4bc&0Rt8p37h%{(MfIjh{*7;^ZRv7^ok=Kl89H z<&XaumB)EyZWvYYa~dFY%r@R*NMu z@D#5rq8oGbkoP4h*{?faKCTFxq+o^m+j>jLt z2v(awKo7UoMfq;#IadVJPtKHmSzZ&$Vv*g~Hc!rhf)ZVNkkTaRW!_J7{qkHMR92k! zyGXEU(OkhWT>zfci-1}D3}?MZlW%w2P97@aOA3@4$9eXY6QgH%hA{p_ARt8nUp2~gy31m1en(# z-DE4ReM}0mOeCm{8(?(aCei`pZGE-T;fauGYb-4=vK@tFzEGCQp4}|@hP;+zASF5L zgkF8{Dzp6<@S~D%Kc6e-CB_Y9b>t8&Y7;$a{tQDB!fhTRRyX)Q^Nb?6sY)pOb97oY z#zf;0qD#BJ2Te{em7(Eqn3=4iQ zkoytEaZBI;>cDnOSzqeVNRAzr-2z+KV%2om8s;v-Oz3zGOU6G)5lpU3$92Y@a;cjU zxJ9qL>1MzI_a5%^k~0Klu5_?s-;f=uojo^5T#OiIu2g~4tBRteoTLnUwnLDNcv8A21B^X7%PkBalpxKbNxf%lYBUN9spDhIr#s)!?%h={-W8gM~tlXMY)$m*SE zqnsrpL1urmleLRR`3*vof&DbqmXi@U@Mu!%`bGff0alJ7F$prxv!R-e6c>AJ(P#<| zN@6cg1aI9B`dAdmOb7(OpYW9zR+^&u$68-ccYhuE@cR$qFB+Ed`0*DOY(6mbZ>u|4auDEu{D zRKc)x-B`REUyZOWl;5OYB`7QaNjJYcOJ?noWeFovHgJ29ie(^u%>@9u7;&Uj+ zmk|_Nf}YQ2U0s{qngfUU@EcsqNa-uA?fk3QCYBz0)l3B?=QAa~ZVQj!nV>C~?PBbK zWayoGNcz`^HHbK`BE@JMy6PMI!qb`LLXJ9#Qi{kBT$g0<<1|ckA;jvF;Sy_R&0LyJ zlAOqCv$=tfK|E$9ZP|nf)2i#@IqM=U8y1rVna>^dL9#dA%VqA|1K)R4hf z_?WSUHlytu$)v6v!PABwfLF_jT(e5tuvL?V^l*y#f|pZ7_HwCJ8k`a>EC~*+T>6q@ zohbX{-jHP)aXYn(pC~YA05tqbI@pBc^1WN)hEPT0@I_Z7d>0Lq|2b>i#J%nYCOTPb zvt*(ZdP$5FB?bBIYberQ50#&Vyz0zLL~_=J!x5;6uM9e^H|vBT#$VC9!vdPZel&(SZH`qP}m8gy#-mw{Aa6?WL<$OP9#|4+7jbR9k=U9y+t z6(d0^eMfcqE*Nan7R{PhYmD(7RqQO93%Ps1g|z0J>Ay^K7_w zmEd^m&GhbG6|I|;7NFu$bzV%Dt4ATG%Rpy@20(2g;K&ro^`znIvcVn`fO8 zW6%sdHmQs!kA8oor;HmLxT2_v4vGO2iLik5#W0-btD)X4dckPLfSji}({(iFj7x1t z=6TeMSv~9q8y69)xw?@K@xTI@j-J+J>fyW$@2VYasxanhc{GT>2^tuarmJ0u4~B#yu6z!?oZAr@bPlR!1i%pY|+ZC~h*@m!N$!PQ=0vNM+Kwn30MYu$0G^ zx%TE@JY`P2GiH#Z0bap6B`*OG0RJf=(Jj@8_Gk2*{&$cbF}H|>g+Td-m2pSIX_!&< zuvd5eBkQ%xB~oJO{HzQG$+aadfv>B zeZ{G)8x@3EDt0e_!>|=40==}|=884pnZ-gKjAu%i&>7VN^&wBM!wSlXt-=N-&(Jf~ z;0$gf!&IwlUU82p`FMBli=55s@t-#+1Cl6e{0hsdhIv`>e4Q~<+5i_cdhBZMB=IUf z^Y!reFQC4XRoclr@^jdg=Jcgf2h}l#;6(3{s@k*>zMxo6Xvm94tON!)ICsg#1 z4{Vv6bw|tAFF?MXdUh1KkxKpf>Nw`Z=)OVlE^Ai@R*d4UOUQb6ywzj+R|3|~yU{oh zwDrGkZd)TeWm~)}pOlrIfu+hqW)mX&15K|OCLG?s;kd&Mfd*pAb5@H6IC6TaK|Y@> z3z73JRS=U9LngjKvfh}Te*bru@zn^7mdA;9`g(B-_5WjaZ+fkZ!!P>1U&>{ZyMCQ< z+f8@K=a5(76M83_7nUXt!^u^BNY(IL6PP&$U;&9#{joTg{s#-r$;sJ&vmHIMSO5KS z&h}}(5?%z;-ALbUw4OTVr)G2R*j;;OtmMjY68X3S@y5!Bihl#_w}!Dj$xukyE^-20 zD{2{C5Uo2;lDuQuP{N(M~w2}G~LmDxD+cvuxYtl~w zvsJ0BKTWp!45eL*`%lv8(-m^2%*w5&zpS>JNY^m)Qgb1__#^L0uicpC!9+`j#=g-- z>1phJXb!&Ru(zjB!6Y|rMkifQmn8RDv92$rTjJrTf+MDK1-;hWTlzhAmiv9%W@Vv* zTZFSLA_hoi8FRm&zRPU>idULVW=~Y-(PitV5pZ$a57+5 zNhGPSFJ_P!o6Q*VNGJiMN-E6P`Rs2ich0&?V|ZkD5fAo>o*3ONS+KKfy!+`5zYZBb zGIR~9{HWx)><6cC!Xp}^D~29wrU!*){l1ImeD6x1EY1`LPVLe7L zPmnWSdVTZuY>$6Mf#7QWG2J2LJ-+Zf*M1`1L2E5HHZ87Y@OsSOT<_AMiu^*-|j2FHGgcLlYG{2^C)MND%7qbL*0Vy#f7!wf*4toMPm2X#qO zCjv|w^dw5avEsZ~(PhqeMgb!Uja2ftFv`VgCYa$ULcaOedKXJU-K`ua(+AVIR=QM7M^aLT|#pzJd=FzU1qGjC{pA4s#2c$v{9v;w&1((Ew3JgzUb!)(KKH-$VQA_l*F?8exh)S{-`l9&# zqEJRwLB!WpGz40fOOKm5N<%0nIK0T{J_D<__CZA2Tz(l8o;)R40&Wm9SS-Y1P5!pd zw)`u*K|>MwqD!4zj@xzUyMtQ_2J(jsTA?R(I@httSc^O(Ylv_?;;>?}{&h$*=%T;} zG4cS6&(Yuwc}K7xbi4b*Ow>)(oxZ$5n#hyv;9TfHII1;lcm*}0JjXe5J}Cy+0d~YE z;Y|Nm&|Kg&nrh^y@$WlAgAwhAcCuQAyf=rbr~~j98+F_yFabyQ z$h=PBniQ7E!+ez^a@PcT6N>~0a_gZ(0Bv6}&_3ecO5;mW*A=()w3;1%5(BcmYN6$> zLY$PoGk+;-698?hc6=NaB>shpt+BYdNcJ53z(K9-Ltgi~@I;wNJQ6+tWi(>;7lV3; zv(^|D8c8jV%wZ|LwFUSLv-*#PFA9u1MGuY~MQO${q+h7eZ<08?Pm6l)$9(@~vsQSS zB`vI0aS_|3T%5b-QW3?wrvlYa^ocaJ&g@fBy&ybczSpY26!-^-l78sB4l$RWyiD{P zY)HQ~Hx%m8ueeowpSE`Z%Wps7_2>3H6qU`XqGXqi9O|Wc|3hURf=-)TxNyX3kaZaPp!W+3g%?`x+BBsvwCu;ek~Uo zXoVP*AzPF^tN*GLR8p?Jl!lv6gSP_uOidX_HtdYtdt zAGjtkRO0eA8mmhE`)_bu?+R(-ZxTGOi}5?*f>JPDWVPl7xs&C?!VST<9z-JcOYt_6 z9_o!VH+*5OWRgHSRBIJp8@bNW_!<~w>RUM89Sv_uCC)AGXW5y#0Heg9=35<&h|jcz+*M(5mBCKR^ieqt8KYC)oTKqmj{shh!S0ZtX9VeVnV1 zIgi+L-=YH2Y#+F0XUff?!=B&d{xP=;P(lisO#q?fM^N`mQ*wydu_Wux?x-oH4n~=^ z_rcWm8Vb36#%5jw#q0eNo`sqE(nDT0*B^2FRDa@#OqsLNOKEq<-3RaI9~-YD9p1V) zlXr-T#^IVZw%GHT8#*y{mIT_-jso%nx#$tJ^2> zNbG2o)?7x7bCPpfoH_)z_<&efx#i!A&W4Ys+|yLZ(HRA2G^~lbSAM#zm9U+}L_^Sc zD1yI-r53m_Fvdh=^9OO6eT6*~WK}JZ@K>g_eQGRd#IPvS9Qrd0WFeR0hG$3=JkBu- zp^3dKun(4u3GiB8h-k5mIkg-Kl4Tk$yBYquZ`9;<32<7x@_kih$!CLpd55wh0Sd7o7k%ddK84Zy;2y-OQ>1diCg!W3^~z8+>zYTC-LG*j=!6}e#NUbABh`hs%Z%_W|PvBQ70Qtc$YmxnI4=- zN@#9!(sLHVsHKT8=HvJCn;|W)h^c+5XQsIcMxU3l`Md!dUwbr77Q!jhRZ4o3jfXvC z&|)2p-B~v4-@5&F8$_K(qwoyobv+!{@xvm1kGfOCu4K zaDLcVWgE4XGp$|sVqIk!q+ECh?XbbPoKBv{y0y+VPIR&7CRIQ>d5)u9w-LWDu$E_B z#6R!IsyUBQ@KhYtX5S3Df90gC7xlE%(UfYt)jk?YlUV*;Pkt@XcZh1}OL!7S;UTzg zc)H<>O6=L8O$+5%D3NZ*S(~5lYL_>bE6M0SHGekXln2;*{qBrmx~ERK@Iy@%9Aw3$ z?M84=(~}C}yJwP0tlR(;ShF@ZSktym%YAw7%JJ-FmYxeWMGs~y>u#{IF8Miz&SHvE z9U-}sS*ovxK09-9S!erF05smHniF6Nn9~xJbep&nbL1Eus<_D$2~J<1-q#`{Z1BzQ zNws%u4~kh+EVQRG_nw<|qy% zL(+2~2Mf?F?>R^KNm?dbu`_eQE67T4J$WX}C|=q$zemO|xO^Y4Gq|9v!cf}i55&0c zc^=aqEX!ty@#;p-&KTtWHJZKlxe-g`z-^rQxL4h)n@eilZRJo$ghx9QVi!e@D5XLx zUaIO(DV)clZG`_^kdfd8T~#*+4;-E#F#x@CGb~Q-wV`#g6Uc-wW7mKYK%U;_id06mWoj*-^$-v&!0F@vxxDpg&G4_ zup2Y^@U0CZBS`M}3tH=KsDubtih&lv+((cCpuY(QhL5hSZ^w^WI&P$`zBZ=v>&e)X zbkI1y?vVyb@l4%LPjrc}|FO}ILw>}uP{6?$E+6=knm>CRt{$b}P8&(W7&ysPgeDne zF07Ki9xJI%OT};A+}CA%&xl=3h4$?~Sq|5icVe*8CZ1CEX@->0M5OAv?|z;eyCn`U zP)3tMyfZhAKd8kV-0Nrvnq#EBPNDjI?;ioPSXwqylsNg-PKx-xfnof_EoHXuQ;hB= zgH;kZ+1<60WR-Zk%TDS-D;97{??0Px+55XD_owR*Xum= zoPDK-cYq(L*d&Jfn-Q=~TAmFnu9*5lJy%qTgGB%6uXMYx6sF3)e`v;)*XO(NQZX|# zzl^vWdk{___~7G+=RU$IgZeU*++5CAz-}$Xo9n`_ukkrozjQh4zgJ?Q0@x`loLb z^*@;k=i2|YkxhGFkK&8)@ovJ51i|5kMYM301?LpwyD8QjfI`C=ZA^-ZzZBm_y6XJHHMo&7yY#p(x63BG=hM|f%IrElgvWU_O>3I) zqrfMEv(Z*#rS_o}g+)#G%Z7rHPuo}6yE% zh}ye7D@#0gvAe0Ww3a(ZXWr9Du@igZV}-5yYAaD`ND^i=dK~1bf8O?ROKrk@N_4WB>u^xlYUop%Y$to4qbk~-cZ6db$I0o6EV29`mE_S zIL~Qa4AN31&fZWbg1**@JaRxToJDv4VJjagZio1>JruyeYu@r|QL$^=8OJbeKq9=$ z`SgpINlRon5ZE3GLK)s&RnleOYOC_d@TNct5DdByi zL$aP16+^1$av`A$$bFX*L>(oEQw;UbilH!9oT)E8`Y^8{sXgEnf9#oQFvbSM@?k7Sx**9_#(YM81jJyZjww5CK3Kwa11yLPIPF948QtaBR+?3+>^@s!F6 zCj!O}Q5)PFl2>8TX{)5JwyYvo#3?Z4mU|-v0TW9^3h{UwM?g*7PkWSL3hfC|ksfSj z;6mIMpyRlkgn7!pR?u^LMJmv{qc5WTF9K^{qIievKyTBHAK-wiS&aD(k>fEj!XdXd z@c5A5g2>pT->4F6VN25{LCI9D{Sk$ z3!zqE|6+QrYzKyWH+7hTWc>iNbK_A{4owlwUJ_9Pgszs*Lxk-GgX zvaGh)vrXYQOs8 zL75<1-&DRM98)?_T<7U}T@Cw}7AZnhh*87kXr2A}us*H-Osd$f)x%6G-WldajikA6 zwi2nWs~H)`M0El%x^B(_Mt2j|0o|g7PW+);@s~V()gv^STUp=YTcPz8>U9~X!XIQ5 z4lg`x8E6QJ(&_4cNO8j3-~L_)?m)NB#MCM#cLu+D5v1Xfx<~%$5bPpfX5mXojyeR? zgMjzYzS{pVeoLh*bbOd2_efPQnlU~a`{KC5r%MYb!&!OvD9l!kp%JMFLKF3HUQ&zQ zQ0Y^e4uhDUQWnNFDnBK0_V_!9RIEE%pBoUjtrsrz6IQ+md;)Jlb5UzB!j=mR<2zfm zKq(u=q`!QmsC=R{WRhA0pcOx{pTYD4Bn1d(B{BCjhbiWki;8uyi)Cz!V9@n@*RyuG zd0sLRUnGezhMtS*L3#KLbD%5Eo>G4YDEv;j!+s&3iTwfb4GQ)X)t5Ui00|zMp(Cu> zK-3}7oh(;CNs$6$C_cIo`ruwn?HZBOlC%MX(1A!_*H_{(DOgiGiFUpUC!BiK)mqIe zMkjB28t4G?#Jf5#J8LOWU>a@0v0aLGy+8+{DXKvmrJ1*fzYQkC^Kv48QR7W8`m{dm`!LVMc1Jb7T(O>d zH4vXAe)j(wpR_t^B&I=B%wxS-lI?3}S?%BYMMQaS*L7Cz4dlw-;YYdU=-C*?>4dCI zz=VSTU{N#4K|C+~Y)844B2gt9!$RL9SvKbxfpO;$ zE2;{pSQN~yZn#bkRx&<6N)ttTJZyFr5rZN7(U zOHO-nt~>~ulBOEJV!9GPKf7HTXaIn0-LyRyK@O9WJ&4B`YwA(bteKyANlD5tfH#;< zI@^{W&*N6A*vQjx-ygnU<(^N$!&pWHMP0w>u2e2wZzVLC4OJi|=@<$n|PU zpE)4&e7$3&PCy2FgzvfCQ@n+t(@E-~#p)xg<7TE1U(PdTSwM_`0k}zDGX>k$X`EVc z@W_;^0W#yAIe(@Q>gO3L=PpN4_C%NLDajT~8~wLjiiAr)YE9VN7~SkrJ49>_I(c}b zn*lTL?@CdXesf4-aq^b&8{!Ztqyoc>gM_{2b>5ZM_UsB@)NsX|^!zPx<_hSwjj48Z zatk`ygV!<;i&a!0s3{h{xYzmYFw9?CLMsy80_Kfm!QXjtbE89`9p=k1hw_xPD&L9o z;M_eozE`FpP!y_gQaaJS8`*5#Xco8Y|Gvn*I2&mOX_${S59V%1IFLVhmEFDOk>-l5=V;qdKa^1EPbr_#Fu{G3k{NimKs~X)j7$trsGHF8>$Q8pR;%oBKmBt z-!6f%qAi+n6W~aBSLUu0|E+4-3vktO2->PJ`h)xjn@M!O3?lI)&HzBzcki^@m(H|P z$FBB!=u>hpq@h&{-YOdkvSn_GSL~i9uh+CW@WmEH2ubR1JbHJ}A+%}EDaFlb;g_Yu zbLIG<-8q?(%PnrwcGNB<^luC5^*V2tn={X(UL;gFu`irF36l>F9LHN5`YY3|kXgF) zf$P@eH+B(<5=3X-uH-rT(b7lTiXKp2a zcPz0HovUgTwv+7`{_Ux9Dg!N5hlEep%vZZ|vzm4}xPc03b2e`A8vr9}Y<|)}{J>fT zxp$zZO@MKnp*Tww=}MIQM(9ikj1ptua}L%F;KNx6bL;n1s4!I&kEERXMxc^Zn@#aq z$yZlq*o|~Ty<3Tz=9E?3HbRi}xAtG~D0K*=UCEs+WbCIycz(;0Y3*B>3+j720PKA` z#zmbG0ATIRWXXY}`1ODjgfDpSJ~4hDscL+$d&%IpfGno8kc&G#GP^WwQ6Lw_?QbC! zF}d*MVSh{$!ANJk)eG<%mz@6_tdOx9d$}{$g3|VNBkV!m1JxF_-CaSpBens_8t9)W z53JnsJJ1?<`YnpE$Wzc2K~L>UJ;lvB;^rktY_Xdez$T+y|ayqN*^} zK0%st*w0kgEy-ucz;Z)0yTIpVefEgd=DS=H959-BXQlxm;I|n_`CF!wfISTiyj1P+ zukj|gKU`ENbP%0D!+3cmkPf}RLmAb_5ymMEC$yXbQ{1<1I=1GtgQ`L9mJw{Qw z6yTbxoY0L=3qclkKBSCtG;S4jF>5Y&)9ShvqA72@D0sa@L`D3@qusso-hK#UbZCC? z4BtzG-`t=-Wk?e~aX*lOd#u>q4_gs-6wir?Y5QJc&*|M;|CzA!{uPQiiJXuxX1?@4 z*?HT1?^4cfo8Q13xA|B-|INZ>{jlElIMEslD=&Xm99#c$F8P0CY;#RyLQ<3rEyZ6U z{g`|}_rd8a7zw7UOGT361fcf~_q00g{9VxJ_!yHO+F*#n?@IZ1@#^7w=Y>X<7!hSU6zeOia|6TACv0mGc`?g z@0u)Jt#XR#@&b3?&gn((%cOz+oq_3XUz|G4o1NA2N;u$w$5(@O%4S8wR8B@iVA8VU zp?$mT4b7KfZ1O)bJrq2Qd<_zN_SStEuR+<#MeL`ct~K}T23J1#)vEd^r}E=5j+cjK ze);^~5$ctapFC9-KTG)IVtXaFxOSEsUEdy8Tkv<$aoHbSc$kI!iyG!?{pOk36|`3u zjS-5dyGmYS_5St-1jCW0{W`ni{8b#0g-wod6)(X-34K7-wOTe`S^Nk5C&`=O(#22I zNSAQhjD=1`&6|(Er(S2y%2Ri6P-y%81zY+{2c_!F=tWqhqxQI&+yImmRab~s{oL1UTIi8Dd05_7D`?wXpJn}#xg;gkxV zE(hJm2Ga-CG2XQCAOk!S!Aarx1d7zjNPJvyG**qvOvF8WI$I*uDISFdGxH$2|2_); z<-lotBzR+p$DSJ#p9SeQy!=gXnS_T@jNi9Of@}TV09j(MLCbO3@(h$Clf+~}?i1(T zG7a58_@(~vKcT?LFPDRaZrM)L6xg$Ck%ut43<*f!2hijyI%{q>9h-?bxG|Ev+wF&P zzyZxGT*1ha_S#zG6RNKp0@1M`k}>E9W4b=m{0^cx@i%Kc>;RWi4fh_C#~DW32yW!G zGPJ2``wmJ+)W*ufb%boStpX#|w8Yny2hr|Sgz6R}81au$u7}k>mZ#=g#3V)sHoW_D zB{r$zVN=4)9jG5NN(ed0A=3v6JWr%x1+mL=s=5JL$D+`1a#iC~@2|4@A7I2KvnM5Runy*_X}*TEeiDF-xEu5%DHE ziaHtenR?FMFyAY2&+AfI;%|8gfbt-$6e}vch@oZmioOP#r*?$X50$;5!yNgMNKBH7TVoxf@os7rRm_2wV%@ja(X~P9!G$(OXc4Nr+1rqQ+o9 z4Y-Ca!D)rs5AV!uDTA;lw+ab}4E}EH6`8tJq_xjCpQyY_u?z*^y7~Sup~8utN`Wd$P*ackh%b?59Tl* zW`c++#it+gG9?*U3`TL2Rn*hI^NMY(CLz4%P=Q`KWMnoFD@>z?OK%cUNaN^~;C}vE zp9@mXF;M$)#J$FCve1#)WiiM54;FHK#~T8}x%1_XXn^^8o+k5E{5Z+6GPBz-)R9bw zs?kXcrVIl#xaO?#L8(cV_jL81dEwQ&f_>|=6q>ni@_A&)Niyg%5Wx-G)tk`@-r^PR zqLzsJ0}cjvAc^|~aE+gLy-9F&LWAVBWIG4=ILRTtRl0a0j}>toVXsKL8C~}S6V7j@ zOcGJdu{X&ve>^D0OZQ^RK1f@uyau_y1`K-^11Ie^+|t;oRkrQ3?Bio1&Z_Os#(iXa z8CiHvOD6gb`~?&gQgr6`^1{EgW9FtS!j9330%GWfuIIJAAjoM)WrzNqpu)W0=;qW+ zoL77ZBhPrQ^?c_zQciOJE3CHhF-LTWK|jyOA?1@-X-!!}9G~!WAx~w0Xvor8b6hiI zleOU<)oizGG;czGM%SOT>hz4hvG51wu3*wo}0Lpv@n6#G?dn8S#Cew(sY1b)d;boGxc&X1*k2=X#ZHmk|Vvy5!w; z(fZ(xJvEgOfo#frmmiH^ova0ZTa?vAAnkT9ZvW0aCczD1y_u5;GD^Xu|NdKc6H;ui zXnTTWj5LT&UaWT>q9V@dScUo^f4jTuJwSMxP z$$2uvvEEaa9pV{G(+D(IZYLY@I}8oyi}rTOU%v`Bmqq~j+)|hujMVZ|zjx`Ll`XAC zXnPsn9G7j;JQPCR+}VVBIRtgZ#jK9sjm8e88l~{u|5S#i(a3Ve3#GXw6_wwv<7`bh z>FeO^PnC%98-}@CE(jpaDjWf80w#m(>u4*M3{3bqz=JeL&yupUT3*%%@^RluzB?S8 zNjt9}duj!i2R5OtWmtn_qZ~T>SG+=zY0NOw5-I#lzYyQFy3Xme&r40cx%0N4H}&`$ zuH1VlrUh8ri-K8I*z!yMo$Zq3_XF2d=A8Hr$#W_95jm!cVj;%4-)D|DyQN8Szq_HJ zO@ags1Jj3>ApE%z|EDiqvvLyUYC&CdOm>9wL7^2PIW6ZkU248ksCf^DqD;t|p&K0x z8~V6|(V4fm_{}Q)ZxV?)uLst8l}5v#g$F(uxlQyt|_?PbF&7Or>ZBL1`I)$W!bJX4tac8wGStaZxs?Py3x`R$M`-2*$^*Yf4 z&tI9-I8SvJihEhL27FA7=Qw##{uqo*l$XQ@8VSPPXcO7#w=uW4QmvGUPGJ%sI|-B` z?|ux>njkiC5uc+X`fvj$&!r@Y2SSRX6U>bI%!j;R8W^9q{^FT1N&7Uk0s=Y*S?sdYy+ zM2tL8pzw?sk6s;`C%9Ak6HsZr2IF|6(|-{R-$G~-{Ev4#DKX9$gZ#H;YN5A7h>y(8 zmD%w_m!fS$P|8x_{9n|9i|m6G>{YX{6LHSZ#1o;_Ex6tEUR$|Mp1?!Y^nf>EE|-#c zN|ac_A@~mO0?)@DmOz3CzZj`LF>nPzS6$g)6+O8u&KGxm`SxB9s2N&Xa&FxCtgS}R zoD869OBt3)pF-vasr{S>QU^6@jqX8p>&td`wi1TcN%@BDIT~B!s{VpBgLX30B!Wa1 zGDjP6uK7)f@p~;&OB4Iej7W_>7>R+b0oyqrA@?>vk+7bX!{Eu+Rh88-PHW!EvfG*$ zlKmw0-SO+}lVsW@({WRd=YOFzdXnQk-B}G?%*jKs;>A-GPu7q|3j=Y%tTm=_boMS? zzK?uJ(*oic>^kXp+V7wvkm-?WVVkPr0~L@PdAs@NF&dW4g@n$@Qh=D9uP`)D-!%%| z>0tcuhSkWHh2Geg+C=qlWdS_OKvNu3ndf4kVtdZ}ovUc{2rI2<^0J@F5+mj?f12ji zrIt~NbT8cfmI&<3KOxUQG$$mZoPt~2s&e2#AI?Lg5JUbDyzbN8m!++gVoV63F;4ZE zY<;qLu;f}{-FHmZ)g-~w&RhSp=arE%?XPK}u-*E(_4?9D?r$bwym8CYH5`+cr<|U& zFmqmTzXiw_I2mUC%an_&14cXGKwHd=TmIpO|L<8+=RYFb#~+dH{;;R(y!^2gk0X9u zHqN*NI}=VpWzsN|oYhN&PIhi9EWILS0p+7aehV408knxGZovZ}IXU_J^PHgXufV&} z)sbV}nvuq$K(!oZG{=J@Tgc-^+xuRhv&wgg|Byj9Q4C2Qck%VRE0e8=BTu|sw?8P4 zX`|ioW7Ze#C>|}w47I8;Q+#1Xai_q|5pUbW4I;a<4%|rBcXV3KS=YCy0sE#nKUMU4 zuiovkGe@kLhHX;nCrd`=Nvmz^YD*zuaLXA%&(WN_oSx15oB0YxciUQcV9a9tf$S-# zv2*Cy>r4}5pRfeMX!_~z&nkPI8V7^VVXgW35qcMz3nz$Z9iFw`4qvf++2Z?P{;Bf* zb`rj?{5&_^&Bq5W?}Vk*u&0z>He>sj)kZi&jhd=Jn}pU!Ue=2q;+<1wzR$V6M!T*m z-*CA71U0?~RAwKfU>}n2d${ULn`=|vH#l4)LB(*s^0C#Xcj&U+AgYuR>Yb9p>TL5L zfXYjsGbWYCO2MZZ90B5KRt(#wq~q0CWren@vXPX6|Gwn8E34KP4BEqQ)S?%ifd6E5Y&C8lz^Qt_MSV2ey)EeY7PaNKj zSMq$wP4oKv?f5_Xf4J{mYiiXei!Cm$>j1FcTyX+=uI87VXP;O$hakNLIbTPdD*6F| zu2J-Qzw&Zmr$*wffa5YcVf5vQd=lFt`#-i9O?3@6lbKe*0dgPA*QPZ`RIqqC{q#L< zd0lX}A_%E&|4iKmsB9;1=1q~|L8mRJi>t9n`;}XUcYmcJQ*$vM?BgBQ9fHWi&vr>t zmXmtG%9h6vqr+@2Wc2N^bGv3lv2Boepz7q|wwd=x-Ieb_lXp?sgjqG&fCn4^?Fj3z z+SS<$5K|WHSZ!=ymif&WkIU~A0gDd8rhidWs-MPijG&e4lLAsio4^Ozryu`4Ki`R!W~fk_?{TLQwuqR;{$oNxT829 z^xdo8&gh02oY-*;YGDyp=AH`(Ce`039Yn|PwgSR!$l~~ftxl}AFS)`5Kr#Grqb$a2 z`NZTc)s;T&YSgy+>MmfzQBB7qE?J>c;(q4HLam%-6GV_V^DI0qUkz~XH@BYToL-_$3eT!(itBVt0qB@fLEHjxbhA zsuM3HG6J9Z3P@qpKh03od-!M>lRl`7wA8mE#ahVREQVN%VBM9GCdG_8O~m5f>PK)sG`2SoAJxHHdo{*wuajnu- z1k3YAWV7Lz0Mn*&G7$GK4s^Yxfe`_&Z8VH-O6$)vQ6~2y#}ThHZdqukpnij@AnLpxX1TJ^3dYX(G7tr?@;U{kLLfu2d=qld6kSDKfAXd9mq>@BCuVJd?!3W+pXazH?1#n>u)>6)$iC{nFAmMT>1s+vf236w_u_{c4Oy z9;ru$Yee)In5Pj2`ByLWMh?v2$G!>u^Nx6Vwp-=sY{%HMft1Bg96jXRT`nEBzOVFL zV56C9&#^d&pT`cC)Vrq5NB5Ga!dEciO9u@PCiHv_spPj$NsMa*#gP$@{6B1+Lt`L; zxG{T{B)uu;0xeM^Jag*l$TnQ4oV!7ABYnTw94ia@3YpiPM=P}&3I|D=k9l&3 zpnAlVshFyDQ?i8ZK`ueKohzbhJpL(VHc+oPrz#1i{(*zC@2ekc3d~lar?Q_;>v>-$ASYTP-* zo|FFv6s(weLdI&(KxXIETA__ho3yCX8r*iE;bf0Ef)G3&VB-Br45J8Lq7cCO62b-e zP`JWJ-ZrmvMh{G@WSbC%rqQZR{iaE5I=e+Y-)cgGwU2!)%G+>mHBGqOEzwN^QoGqs z`U=B6$U@>gKZZYqtZx%NPEPP!xx0BU-zEf~k#Pbzg4g`xBz{(EC&%O1V_j@YX1_i_#oj`?#4oE(jO!?h)Zt;CO6Q5wkx{1b zpZ`zbhj=miO4+}w1!~9Js(xgPC3kMPM1cJ2 zP0Mu=T)Yo?&c{a=-V$swChG@+Dv>7ppS4@Dyl$mxw;rGkq*`TX98_*FzWZkD-Q!LM zytcIN9eSxZl0`-5!OGA#cTWgB#~Rm?Q!R!_vFzUJL3G$AMIW8KGKS`Be`|OReB4Tz zVx%{c6{N?=;(r_{E=35v{;9Dk@axgFBMd_?j{Q`YmXZL_&WFrPyP2n$1#SdrxEeEU z!lN{pTex|5s!2IV-MR0nQRA6crHqjUeWM7924LU7Dx5fwoHfCd6qQBEi8KEB5V?n2 zEv@|f(Hq^c_SuAQwbcHmG|4`oY6W40T6l+W)~b*r zDmkzb_!gv@o>Tv9>CodsDK>sxTrwoPMfX#VqNtGyxDXT{3T*8xyr+5Nw(}_|KweVe z4p5_8JzK53W>OW!isZ0A4qB16n%kmP<4<%u%`98d9*@+2C4tgC^1VkXVK$b=o&=io z-nQyG1zEY6!5$FLJb52T z6u9qO3+oUPPVslxsNVb|Wg~Uo0n9X>3s&A4*xOAXU0(|;PDLI6#9chwcWF4Z5Wo&< z`yyJXfyq&$us44}r|QYEpBc15o1(xn!M?3(!cK%Gx);0fdXi%wC&^3Pf~k@NOUq2l zKFN79AM84}ha4)1eqA2@L^>d_Zw#F(Yq#~&w>HsN)H8R>;qT{v$1PYmN(LuBhVLP! zO9t&Odd7Ge*UKjzulgk^W5&lK5e=csA^=qyzVD30HMyPlfa@t9$pwdG8B zCRTpom0@OqUtWL%=N%lk{gVy;(0sCeQmW-xwG4LsI(*AUpy+(JXf>1uh=csk6eSTa z_64HQ95FUf@YJ}QWXfTk34n)%g{-*aW(%(Z#Sp5Shm~hVG|Zg6ApyDaboH zk~s{S2-uy%`Z7@TbCL)zE6bjuuIbf+$dYbFPwXOgevr~Laol(_UPzzToVQM9YVZQ# zeDE86TwOzlSlj?o`J>)Q*lc>oh7=Ab)EW0^i*4(Cm48hY)%`&c6`Y2YNWY{FY7P1| zK($G1Psjv=!c(m36prbl7@Ps3I&0>hfLV&{2#bWavk}7m21}h9LW9;!!Q=@Yn|s(?HW&G z3xk-by%*|&`x=;}3qr$_vpw=B+n_-{RSjh7Ly`ff5Rdhja2<(clK~*tz}TFI`!T7V zRmfEp@|jsGxaw7o*c)@F7QNAN=WwoNu~c_Kkn8)eBnY@mT}QPZ<^Xnv|5Gx{@G#sD zYA2+ymb!ym#`HVrpkLM@pxk&(u4_`0WmIuLaNbp35*N}n%41m#4=t8H{-@qvK+e#4 z$i1j@1VDJE@nawZ+zOGRL3+8(Q2+YKwW0Hkz{kPRuF>_d9rf-AK;9)l)N-a0a zSW5jRFg8jqC<*`yC<-_L0A9rJ>pu^Yv&M$Y!YJaCx@pw|07NWwAV7kUA3Q#~Pw(Ww z=isNW)kHIFxUmz&o#)is;zHBK{8GGoD$AgS^NRCIwZ+#z#XAdHh0y@dgYx&XCKsL=BH*YqYlqI9$mC6YRd|*rkU(uvU>8**sgrO2W3@e z^%jbH;ceuKZb-Q7Ag~F84)Uv|gIh^< zES6li=W2tu#{Zm7Jg5y)K$K+_sl2XGx_->uO;2S_xz9>@uBlNgs*$oq-!kr<Bw|DNmqg6$w-lKhnUlSLqiFnp^tz3N4 zm@!$se7hOX+;~%H$hsnb)%Q~JBiCF8aR zzVN4t$~L8P3nc=v4-&(6cj8))JLDET!(qQmkAdi-MGh}AS;Nf5M*A&F zE&b!AM|KUpxk~Joi2vNiaVx4N$*fisDzY>Z+>pBcHiCz$IvQw+)->GIVgu@1g!rC?3!~l97$T!JP<{?|61Zj;o=qc2 z!tT?YIB%Ute?=>{4m$4sZBu%MVAWy)0kO1$q~I4>WBDs}Q<@mQ>@LUva?vSB&;l~Y@A>UW2+U3g zjweujbrPC=N)IaP9fPaBKU_u5KelX64d$3Y4}AXu68`xO7Cn1Aa>$W+KVQ9Z|k1iy;nt_YY4hkQ_oZ;_u(h??@%evPv$4-N`O-IiRY zhg$2*cf%-wiKrXh>}y?thR?v@eP4V{;8?Pjsmk}@6+cvCkhN&1?+0;)v{qA_AUNk1 zTMUGg93PQp^oZ>K%1U~>V>}xLZ>)L}0rT9>pIBK1Jz0rObIvm~_bV+?C=h#8F$=vH zECw0FL->x+WdE;XgNPrC(S_Hd>6-rb%ZV&97iu%@>$4s4kT#&Xi z&!Ak#!%ZcL%%UD(=qI=kGGCm9W7n5YDt+L0t^~O33?4RAY!`y)j3}e@}#p9-|#X*#F3H$J`kNYmYnEfl}Z0fvjP<; z9z38cfuuQ%8)2Ltn>hSgNf%>)I04^<9L?mA4BtDF`N=D zar}p23I73}sfg9hCf!~_zq&pE9yB$fX$P%5nWlqXLlXI3!xR}mX&q%fk7%cms%*%u zO=e=b6DtH@n1rBrs4QJGrSzIxxul0q24^KMcAhp4w0fv=GJC(l=FaBd1 zzHQ#S@^w@lo-G8(`v${%fw=Z zB14-rKaoC#lVPPCgdFZ=Jlbkz-DWL2;H4?8RJ=hb@yMrnLxJH0Qnw4&vSqyUt!Dw% zDj2d;z3#<%m=FQuf_Y$31}uJ5D<3M`2IAjHGy+=*P1>$K0V|;Q=AxEuB58y&@_$&( z{xVl~M{2;TZVfplU(vzrSiuGifG?;Brm8qamLQgpeI+{_T4-z6=4{Z1~^SuYL}OmE_pgz*>*q_lje z@xKA~OqcI#zb{#*BRFG_h*&_~S%rOFN75k&QJRJ5a^}bsO3(0TNN) zaBq309XF0-X*~Tr+6d5m|GdpC)$j*W_NZo>dG8fNZdd~)LuUmBvUc}b6TQFcHqerL zq+!yuiAH$n%&ht_JYDn5oYUK1*y_}Hz{5+|lTW2w`0G9dJalC@N-0jCkK*KrLX$8T z=~~TbOm)7}XG8{@fD$n`vVc0_@hW&K**GXp#Z7N?C+`*7{KgUR8FJr381C6c*pIO_ zFx^1xn1lDfHo+P*7Qz&SeJ5W@XM2H33it z%>Ek>%sK@N=;&e*5n?pNv+Qm#P_5Tk2;ZflS)1_CV03*YY)EIpaFNlTh`JPp@4^~E ztsE`1r9@sDJ{(TG!*G_-eLw(*#|r)c9d33)tX(B0sj7Xo|nfV2XB2i&3Q|0WLGN`cs-e|AJ&ph{8Sg-U3CogJ+jc@D#o*jG#b`G z(5f)6`)0r5i{$o+mM5?5*qAL-^Pga*$2Gv#N#@G$$6*QaQ!MMk zcbA;l{g3V|u{dCO=2(c~SK4_A*A9w5rc&JnzDzQHZsHaA2r~(4SU&ev2uM%~KzrDZ zCE#EA?XOtD zqj7fyoQ0K7W$dDKPb2tPCfeM1L+h^tikC!E(v64dbWrD^nOlznoH9I}z!S8|Mk2~W zYO@r@Su4dsOj65#RLsxt$wL<`?^nRJh?XeR9I})dI_nIx+KgT6hkwyCEJ?1GkPg9w zAoF}9LK|b8OM%$9lKJ(Hs-58c$b-<57!0w!Yu4IWOY6SrlfcRC3S_Vw;~+c(MFGm^ z3VAjClLCUpb3?UKCH`SU1QY5^xl(_M4I52Tz@_XnmT z@1j2gjWJ?QG^*9yd$Y9obeBV;o)S=S~+0(-Q31YY;&^g zylDp0jm|X^AN36CZu8hIhnLwko%bIh@YYiyy%7+u!5xg1HMG9iO@Ypt1Ixdd^iAAy_Ha9Z-MAC#M{tc`#b|9$r!EoO(xj7dFDqd)h7iOkDWfQ7AfagvNb~cUq85aip-vVGPG^=FEoUh`DwJ37(pij8Dp*Hu;Vc%CPmYF+@cGxxkM#);5Qh_ zxei9jAsdjDOAxO)LVLT%@#ij^`G$wT>oWy~enFPO&^dP1+MxCw2<`_fc?6t#J%|^g zw)n6ueAnG5oTrh?M+ff89CZzoPXQFkV%Yuw?55^$gwf-icAvVo{gazo@1W=b9dU2- z0w^`hj90!$dir=vCHnRHFQLD6T%p&uGiLlQB3TjHbqZ&9qA0$? zeBu*gvkg{uRAmqeFT@b@t#u&Sm`aNYuT|7O`XAuIujT+gTEqGQ2MZAoz(Pl;#~ai{ zjC*k^6PKyNHLKwkVo|FOaR&q%lv$jIq!ay?%(+|33)s~sb2Za}J~*5Q!pc71>#E#_ z+AHTg#!5sA$tj};RMn2Lh*lpy!#lfm_R~*? zd)^OPQ+39dVvM$IsI62XH@~G#zCOE^)KQCdYSDI?-c1+W;>6a4|7q31{}aRJa-cA& zk^EzQF8BDP*iJEsHa~D+b1BMb^A$?#$)u^8XYJ8ByG5y*JhckrH5YO%O5Po_#MBN! zM;_vr*-DhKZ*J;Tbe^u=M7=yE+uCL}oaY8!()sZ)@;G5Lu)P44Qk(ydgWw#jv;XrT zCht4kerXeN&eqOOcUZZR?<;(teoJ&Ym=cbw!*XDdi+*|W#j&+;$;u<#Xtw;x*^RcB z2Or}_KEA$V2U9^ND{Z_RZ+u?;Hz%}BIr$6V>HPh)5)1u!}PX4C?fbGiE zc{aN}KVr?%YikqDx(O@Y06{|HAv6(#HC(z07np8|vaeLcmzun_y?U&@e0wJ1X;pbj zj8qqZ>`N*Gyc&bA&jj?W{%lp;11P&TA-o%A zBE0`4wgKJR@5QdQ(-IlWAe^v#iM$}W+=w6=1MqJ>vsJqTxMNl63G^qeC*GQqOf=?Q z^4Ff2npKY!jHj;w{V2^aE&}E}(DXg)Zj{BS-vPqD=K?=La|&emq!1{+LsBQ|pH{ja z_uHfzz?yA6Glo?sV7N^%MXcdWZ6>E4BdTr@a4!0)^d7;x@C3rV7_Gg4SB*9)Bs~%I zk4}C&6iX|&LMo5qbAT9wEs(*#6}v9!1~O8*IrCLmT>of`9aOj%?(82ug7z~zSFQRb zs=(Ckp=r*Ikn~&W_`wlEUF>?SQwK*Mg%{wj)D~$2F_V;D#C#?#KlyR3xc{-38--u`WJNgq2SIxABVK}uE9SIZ3S*ztHw46 z(Q18`&mk)sycK+jNbHNqS%Ry|z z&KaruB}z;3Tbz(M{WhQi2h9ljx?qb|%tkylYmFAqoNX+$h)`xSDh^XVu|KlvDPC?z zRdD^*vm;t|Or*SwXrU8KEz?=|WiyOIeFXh-M9XpN5u%-7s$tQePDCIcrw!^J5J)5K zYx$_to*jhP^zsH+setyp}Hqs@s58iCL-e};EF8s_xca~RLdZO$AA ze?JpV8&+g7_HOWt;Q7HVXoOdf(3}=6lIL4WXnhVN%3%NCVX|ZU(X*A1PnX!+e5fM_ z&Bw@WbQx)r@;YIDS(O~d?ZnsQ>B5bD2E}{sg_2imOdPo-I*;b$I`-$-2CNTILf0{y z5;LK1{e^5O6k_*gtLJcBWLYD0tz0`I9uzsH+#>EwSGZVbE?09w*!5{2Ou z2RQFDAp_qBqW((0cAMf#yGp;VT0Q<%nhK@e_*4f!-vk5|OUQq1mxv!ko{ysxC75fi zp;|&NAKPksG+SP}*Wwj_4+~d^DlwwpE}<|G5nIoi_?Eg~HXnbo%td7sW%o*rNdGdb zR~jAZ%Z5&J>&KDH#PitVCzAZ-6f1NKO9oU2D4sh`ZLCRy_Bc|fuuLkXANCj$OjDyy zm~ZWvv{{oT2bK@>reO%jqK##LN`(^2Sg=|S5YhqIC?w>?EZ?UlpYyClbZ zTX83Vf<(u=MjSOvRx*0;8dC$|+_A>u26lu&q0O5F|d;M$vz1FNP;A zB84P@AJBdRng*088K;{2<%frxci)IGdT9F7U|rWaE_aw4i2X^*Ny?ONMisZ{*Qeba z^EaG=A%ZNif{yMa%;P?xSdtJY6j|6;CS@qeoM>x{#dz(&yCqU%7zZyDcEMtF^Iqk1 zHl%wxv|n8w^eRD()No(7OsuGzYJw(<0GknbHhY>cda-luja+BQY_s)wCDDJK4kziLX zQDAk3^%6AR&x=6G@?S_w9_X$N4FTQHHBV}fG`Ul=*@B*vbj86iv@Y=Q=@r$>5JJ4 z>4QxcbH1UR`_#OeXf*Ez$^e2}D44k^hLOMsdSILF;=3=qUb`_3q6I1Vu?{QabA)1CT3C@^h@XBLB(C95gktVZo7$9P*43KE21wB zEGA|di*Zjv+%dgC!D)j448ri&F$)fYz*dmG+Nqjq z3eo{!IUgzOg4>l4-Ok-nbesmNws(pStP%{SsqsDruC&e9i*TZFX6-oEuqFy!9xWpG zE16hq3Q8b}j$#aHnmldPioTrZXD7k;v5BN?u}7fUs=EggrLder%K z!uyE3RR_|juPOI+uArX8a(Q=9K%7su_uDAp>=h2&u)ms_IRZh)SoO<679O5DHY5C& ze=mtZGTt}*T!rvE`j9MLMu-&4^H3tb*Odm)j?r^nNgSDuU5T7SmJNxf&b2AjMnK

#}X}D!@$+Mk$-*v0M}PF?X|elw?r<7y%6O z+KcEvA*v=@duq+h!z~}m`g}^qE6DU{z&fQ+8K=aloPAuWTvcAA;E4`Utb2jtP+flL zY%%xHnM4P zA0N&rB8I$%W1)Z<;7l$^>0b6hAfK>Ty>VIOJ@rJirupNI@uy3N-`y{vW7fv_#YT^U%wsKys% z$A77613|QO!h3|(r=t6%(b1#*=hHvJ03QKz#X?R*#mHgc{Jeq(gL@a@f8C8fm}26yORB~o;}N{lhcxYhQ@pIxEoke}bY#9w{$}9cL`O$Ed2apy8jSF(v;A)l*#4^p9;Qc$ z-w!VV#YVaRe@+lDjSG*3Uo^gVM-OA*FPIemj|9F}0G{3}!_0iLv1R}vhXn;1?!Rok zi|r-EOS(`RXI0fKPUrIVA02AcpU)Fruv^)t-}@cy!r}+?o3g*Wqbyus6{W5NvQEue z8#{bzb!7{^ryptWrn)SA*6Au4m+F@yGF;q7ciwGzQGZ%-Q5P<#g<50Z=b;_+TH@)h zuEeGsk)eFm?h(9^;%sJ@XS2ofB-dy8teG!#GA<1-yANC)jU-04zRO%^+74( zeT4okQ-fMpR^GLKhuVEWa}-sN^+ABzvKON91wT`8Y@2T1(8iw-F1;0ej40FJz2@X< zhQ=c|kkGn|!JRr?Y2y7M0@i7CutK++*KT>I?V{%aG;=zGTD_PjyNC6SEhTB2Juq_1 zQVD-iY!O13XqSunwW*rd7CuC4)p81dYlC*-w+)uEIY5P6t>}>KnQy;KHb%AwkrO6k zt4}>kbdP=w(AP3RJtTYzy*a}l7aieOy|*A8GB>OkC2y_Ab}+Gz+z=wy(m2YXl$n!q zF7mm_%ah=|*CBfAj`6H-U8>d-7-MG(Ww;8hEPEV`m>13q@2ZX%d4(=|3)*evR=UCd zXvm>w7Euu`hoN84Ve934>m+(AdJ}k2N9)tu11|0rQ>6iDM^UOhu@*?Nre>96H@l^v z2V|n3pic3&n9Q<=yjNbQ!6KR{jR}@wUNUy~K?<#`=yy`#0#$w3tMKhVk^DeRhK4Qw zUdSz95fQ>md3(QugFDz?HrsC`*{Z5~bK!Ctu8h(vZun2L<>F5kp(k(zkI~C{buFgI zNY}He1669oYq}sC*QlDeUkI=Tbf0lQBJVKoP+)&i4r2QviG^5@NpT>L*maOJuOVg; z%4jPv|M)QQwQQr@CKE@x<{R|HWl;s01)HKubxiQ`*Xqf{{k_-md->9V>fkY@o=?_2J zNqi>DTZTi>MkI+6I;IDAn_W>5u89}?!!7uymf%{JRveAoxz<#++Vh1 zsRm3?kK`IXTT047g**Qlg^76$o$r#n6 zOw@u;fmTWgYc9HZ6l><-5g@VhZP+Iz>(-p3ncpyNEKQLHIFVja&Gx2k#d26@zldU% z&<|RgrEDnlvf@?HOiXj zESdzGilN|7$r4`2!?=b*DO)s{ox2@TAeC4AlCW8%DUoC_uF?Q@ePJE*E zZk2AD%^cJ+`=w+bU+S9l=ZV?aGTWrNWi>qP_QCXk5p<2c-%z+E4}CSm4H$JV$gDsJ zxt4)bX@vXN$5xDNIGrKxl^_nC0P%PGiKf3gje6k>mc}X^E`Xp}HXkmru8JOqDI*As zu4EkX%Q@o{1ZvS(#fhV(8?h=qtN8n&WHRHVOi>DB_YfB84{bP3@<%Yh;ctR3sjK%! zACX1(f67pNP>t%o~2OBgxY|cq5#=+Aw{PTh^;y4 zFM;Sy;w*M+2x2#TJ|ngJoN8qplKk00@`Z za|H4Mq9sm9ISv)~IFFjnm)uIp)vZppD6R@k^$MY>9wcQNY5YxJ*{wjlggVnSmmyE9 zgCARNJ-qm*qH@myvGeBIx97PvNghRbZxg9;4&W!&q+D>Qa{{`!uR)Lo2y$vSvBS_I zc@ZV2brI!g=}fU`a(Z_-y%EQ1T=V>Mk3{BpoLprEjlTTOWpDDLSh1HAC^j9J$|X6!evt_gW|6rZ;BrTb{SMLPXgHyMOM|(( zS9Eiy7~CWA{cfnE>;9EX=~#Yb-@ZJ96{Qoz38n(1`PZ zhdLDj7GBYcNl+a3PG0IwYCBZPnf6mEq|;i0KX=_~A-Hi#mHcSyiAxijXCCx=%+b%X zttwY&`(IQgm2N9h5u-%BHKm}uY*y_i(WSQxHyPKiy;hSwy3o*zZ;Vvhq?+dKS)fLq zAi)-MaXD1DC=|Y9zC2*W4N#P%$6g#ORtnU2Qb=4C0RO5gd8#EEF>*Z36z7{s|I0h z_A01r8*sb`>+_PNBSW)wztStFU#n0Eqf0Dsk;c81E36y>wMV1ZQv{6>w}Bp6c?;Z6 zPZ`eg#h`hDnP7v~ z&9@y(b7_01@04LDGsZH0c&}r-F~r+T{l$()rOu$0K?2I-Ov;@kC)rwn&PGsBW&`n( z$&UAsXvvPYR%*P73w&tAF(S0*z*$0F1R8rPn69lKgG|ZSchLV(WqfqYb3_4y(5HVK ziwtQZ4W$@?Vtch;M_YQ?Sj4Lu(V@M)2fPFc+F~3$Y|rtFuL3W}Q)h^>>eRMaSnX5V zgbX;kD6WtRS#abz7`8cCSN1rWe93kRNLo4g@~hFT0HWeVC;n3w5keQ2+7}4kao9AO z9{Yo4K?rJYVX6vMch#O#X5cAKK1FpO3l>|_T9c;pI6R5~#}`gJ8eW36hv5dd=rgE? zhkAx9g2w$|YM?499^hxYxtwG;ZsM8s9(xA&dpGCUKRlPYnt{^s(M~P#nmhm zKB5A_w`t%XkDBMYue#ssVUuNW7=ug(>?Bi(8tbgY2+3CEk+1~p6dP(W)VfWTA{_l= z60X7i-)F9R!PlWJR$Hm8Vaep*&o}7#XZlC)+K8`HO-7Tr@`#&2Y%5!~TD%TrSLVjB z3Ty85566O3tOI+70dh}U^$eFU(p=`cwhS$cj9BFiG3Mfi{VEFz=9pAZ=HZKSwbq$8 z7Z!??&pAKZe1Jva||y$k$0yB7R|3Ic4QDQDcJ($ExDM+>zK0Y?uGh18v(9+ z)*17u#Eji9LDd>Ys3-qsc8tz>EW7F+YF zF0O;_itdn){n^BqJiPFd)fhP843~}SvJrY@zCNx39GsKO_4fP`7{mVvotgyYzXk9j z!a`m6W09VZY*ud1aA)gH+PkaQC|9+lu9&5abez_D!rM<^V4Wq&{s~zg zGFWnqEP=?Z?_x8dy{^vA=d2PPRRaY@D7fJO1pyYG`LPqZ`}VQKYZn1 zN8ui)KhishEucLwvEPZ!o9e=Op=3IPp|L?y8tS(Fvuo}9u{=f_o#_rFkDxS1mbVrB z3|u!!U-#o16CCh*XJRq@KD!98J{xG)O8_9I-JMN_x(33(>^cxy?<(9D|1ZaTXz>AhdPz^&zg%%Q*O+VPkFdrIY1XgioesUVAB z(<|`b+azWol`&)hupKaVVq%@^zH%;Gnw{}khE-T2Ixk)eCyD+b=(AuY&8)q-TkH{b zD_EM+j+oq%I2^(sG?YP?1?R*^TAA=rI*xLp6)jg+hL%b2cZU1)0NLKR;46SLF_E|t z!ofjH`eI|5ebZ{6kPpJIzPsy`Omp?=Ms_=6S`Wg`d=U$J*;ET8@mZ$bPIcUbCf$1_ z_IPC6SGQnpy^SyPIfHqsu*5VgYWfu~bUr{{Cz-u7X>v`~?L?tKtnznP5!a&??=Q@^ zd1}$8ed_N?zvEtp^uZ)3$|O00g99YwgG^b8w5}-;t{Z^z_iQPm zTqm4CIo^|MNGl^+5v>Vdy!3merwFG|&z1Rp{Unj;w7(86_VZ#opVhdv41AyGRkrQt zKx1iO1Z~mrgZnq#LQ#}CaU}Xqx!FO_UdP!)(t|_pPjp`5jL?GSY83xR2G$?_H<#W&J#cd5pZN*AXD9c_XNuj*+-IDPqPggKZ1= z(#&NGdv=a8-^yMiU}X?q!-HfeL+w=M^MxV*);{1hHJ0l-u-^--QOs4Ux%JgP&o-cJ zf{EbsAb=OX13y6W&Wh*!5=t{g%kn>N`&!~;hZDRiS=G7ec0%p-q={-o0}8ZB>f__X&?Dp)KPpork|o1 ziUw%OV2?qt%T5z&z`gZCV)tI8Y~$TQ9iX@O8^lZm^xnSeS6KkO-mz;h?7BDix7;@( zc056Z(!Vn)RyY=>gJV^XET2`$L&Y12L~>mrCvwUn!8-#jo`gcQ&_284$e{( z>{7aW1GQAHiKdtAOGf?)QH9+kE{tSADv$SFO}=HOh^8v48$>7tWdqNCnq6 zd<}ae9hxgws5VOdjD33R3c`ld$dzEsFrUvp07V+O#a9--p9NDHO>0hwH%=Agp8rLi zCrdNQPcPlq=uVwz_lVE_4SWM7!INlU%8gX`N5D*aB5 z2*|NNP^pEv)93chy#!@QalyRm=pIDAZgUY&@K-bPpjI*B&m=frWc67xICnTg4QA7Q;za}QP~+m;>S z7Bre9JvEIm)IcJBQVtyo;C)#T)fB&u3|jj791W zD&uuOvM(fVvO4P%8>UFY6q8UB>y(6mB2sN9hw>A(W3}*Gfl(w{^iiy!{4Hes^csMZ;LR(o9sCaWX&4<@YKoSxx^3X{T zvvPm87>xn$L8^j8IGqW}nA>F(y>c?Og4X_47VzQ=heF$QJnwiN=`AQBU(LK1ARqpg z26?cyv;h*EmoKoQG?jvQ>tJD1QO4y63L#c6ne05N>;q|_=X@NNZrEy+j>EuPy+M4~ zN^#t(4Fq>r$KCp_ ziB{N{gFT;x@)(fbsAtF5w1$A2)-jmqwq32bF{0L5IT&^PqyNK5fwBvNqz@o&hIy-N zPWfa50q0KdQ2g3M1iDTa)1ZmX85RTHYw>I4O>|G!-_Cs(N89YjB zPHyqvY1<07pVDo<`E`nptOlh8iUdJ{<=WAq&K&Q%tEzu-daQH=zLEFaB7*{*DBLFl z;OUVUqvY(I+BA2YkBAcsiPvG~OK zy_tq$I^sM%*eY(CMy1;VKMXAx?^_faB@?Fi%TzIkCqf}jJA07R$nJp3@cbEURM0G( z%wBO7InXy*=-i3l?1w=k?a-p}6>dZcq}80-5Qs7+sPHV1;C)=`!dn`#Ztzzy^p7>M zX`fuH7aTd~8P;Yh);=`pq90+j=w6S&C>W>h-zF4H{ZB%rRWYjoMvd(;bEaO(zdFwz zC_9+cwRAevrJ3#~0DB>@XA_ee8+z;pBaSSwLOiJ9KE%RgksZ9_`)}f?w}IQ{O^cXv z^?=k|bw3>L&I8vZ*8I6KQQ2%%ACyW&JGIuVym7`PRNpksslZjn8ij?UtvbvPbm_)4 zZLOZ?g~s#9)-^gVNV?;PAKH&YCKBUnuTBtIA1PbEarIWN^+pTV#K00Rq|e^tnkv|{Nd*@Pp7;FX`aKJb_k}v;yjD>&U-&@xLT(A)a zX0R|al^tQ;)c?t9%1hhJMp`b)R_KDgO3&{pv^l16-DGm+RHf%aZWPh&HWqpF;Lh64 zzR$?s2iHPubq?B`47XtacP%+Jq{^{Q?NMPaQv+e<}p&y0umK6gcj>rPbZOEHJa2j@&Yh7A{iGcXTavO*ENj zdDG*YFz0%;c}B9eL=0HqC#uy^F0Mte_8i<2gaQsK=QAfmRL!-6n^#h?7Xp9cUmyB(^b%D0UQx(Mc!&hkL&wPpaJT{ zhW!r~QIwou;ll=9FBPmxh1C=!cx_?w6zh=oE_uYToTmZqZq!21_s0#k^`*L==$LR2 zd#biA`Dn9%Efif3}%$~?+00=gD#9}#BIjze{n9DiDd zOpa~ALz^pmjmdVj$|P_x9FXrGdMjczl<98aGl#Lotx7w5ZA6@6zC0LMc+q7-lw(YF6ZQHhO z+qR8L+qP{xZ@uoX{r*CX5hKpod#<^XLdk-AgsxTL-9h0RkEXQ{#(>p9Xh?Jr`y%Rz zmXRp>?2k#fKipR%`1~rmow=oEf%5B;J%PZVS2&@M22O%j>2;|6QS@>yRFFVGGuZ~RgEPLLoGYVGCv;}Ao?Fy+$(!0 z5@y8}1xT?DJjIm!lLI_r4$#ZB6zM>ED5)2~HIBxSB$?|3?&a@VN%hwSGiEL|4xRpH!opvC znAr3KLJ(cj40c`y4X?4n`V%N2MzAT7P~36Ykg+0LH44f@!{c%;c-XzGvGX(F6e56i zXjhD(!nze^58%VAn~APN^v(e_2a-a;81+C8R-YNo;lA}`aXb)r(W7SN@}c)9>u5`8 zo*4rrHkP^jjT;Xn2&nzz*gg{BabvLrL0wk?>W$}WbzQ)VK(`DOD={!Z94aBv!u48& zHGjvm0OeIkA#6Wx_%fp{%Qs~^6}!E_b|+azk4T;*iTN*ARc)tAzV169U{KQW1eX=d zFT$DFzRDp2H6AYVP6j-#_#EY~qPpjb)vqBy_F^VIYsXo+c0p1{k)*^3&9I`j=(R5` z&K###UwuxP#>UE7PsMVx80IDid+~OiOn(J~v22JjS^5JKUI}FadHbQG31}))d-Bq0GRauMjBc2I zU1gstg3?)dVIhC+qw^cdQ;pQ0|7AQ5ZH5-F$hkH%bUd+2Q6q}e>3O3lWKTK##ja4y zd!up9O;!B>6C4chp?=~f)CehPuGG&31 zdXpa?^+;(E2HQqXurd8<|2CZO%84my+6Kn`%iXm;BaD-$Tdx_VSI#v6Rlk?q5P3F! zfQs+zQ##3CA>@r>$2y|^RRb3e*RR1NNSOs3Q0|wQZW5(I-nl*WLby$f7@Mk&W8CIp z!&1Z-mFTD?{&CWj#k)XYryA@k+>$>lsC{yrin`ROG;O-06^~S<qunY>1T zq#L&~$a<-RY1YuGsxa94D1i3u0`F6cyxd*YJB6ZANbNa8%6{Yh>#X;3m>QOBB#I;T zE@*n|;(HGR3nh@-YRFowEIw}pd{h5W>;?GCkCEJ~Y{cxlR(@mHMc%;EE@ zSSZjrT|?@8Yw3~$q{ql+e^qtDC}FiDS_77Bd}h#I4f#j#6 z)yc$h(veNeSr*p`i}@ThEZ4rdn<*6UuO>NL*{rR0rrm%kGF#(p5Dfck`v?!K zp*P!fOJJ&Q_L{_SeBc96tHL^p#wnBv@MC&QhZYk+fJx7SGbQ;ZLq56VrY)t4kEN0; zOCoxgAQ{cPuA!`nT>ASTxPn7`SXYr3fDjcG7%~cZdWo(fmpGjOhRprAw!lgt8_)5P z;sjWSL&G6330>bmT3 z0apoSy;rk4fVgoXGDjlhQ1xtiJKi1Z?U@XEMEJUqsn1>1nU~5y#`3Ru28o4{DgmY zsRV-`)i#c(s6unkl8~UHYfFXC4tXdh+1oqaSOwiCC@8G)?m<})9Y+O6V zgh?BpLBl;d8}X-Q+txE`XFnzhH|N`JwDqbOaSuK(`ChC9sReWmh4=L4$P--NT`_7}PpN zavnVR(J}?74h;WUS%j7bW^D;C)3U$?>7QUy#Y?C>HD6UmAZCjlhvQe7=bQOmuPZW7 zb8Cfk(D#JwJkB|_?PeU>=H8q;>^2E8wjY%FO6$l;H~QP2kyS80DR}94OhUcU#@DM3 zfzQ`M%kqP)W(hOnDpjUg6Y3$7GJ#FEnm_pS5Sh1!yrz6IeMt&{O?E#-J3SnBrC~Jg z$$uv&gzY6uliL?k8<&XI!gR_4neuK=7yjQm_>%A!*{Wq#zI^-J=^4XG+Ir9s$ zdfH1nz^JS<0G|wXVUGdb8kzluYwO3~E)6m}yekRcjpOYu@P8d~Mc%9o4LcBSkCBE^ zS$@X$j+CKa(P;{J{bn5ojJ6)Eh1g_#c?xB>$KGJu^Re8tk5<9WJWwz4_%Vkm+E%{! z<=cl+VqMXZ!aL7Y4=%NG6Yhx+Z7W$_Te#i7S&J22t(!Octlwn4IUcIVZf~%4YtE{` z4O7ex4?SqTXMbK}rNi2Le$w^qhVt2xjp6QGbKO0(2Xafq-Rum3wVku*R@*qjZ%;FX zKvtg*R9e+xk1pBD9?EZr)-y;u;|UbifN=@hWTNQ5gw~G1>siioVeYhRW5VGIMhbyz zV{>hed82I=-R>=2n*ow!`(j;2?Yy{6qq9iR@_l;)WS=@&_E}v?;eFF+mGm7L*4wYy zyZk}WWXSEWllvM-PcH5U&oh?blpchz$`4vv#j0Llsu(n5u>Qo--;(m5-6yi`zZp=& zGdja|N73`&%t-|8GY_|;UHPpb6ue+Sp?B{(qH0i)e?s)bT+ExG83;5MGY!zFRcEWe z*j$Upj!n>w-lZP1lwM#|EfP%a)+)3*?n|BS%z<{E==?-GH^JDgtFI15PKBQaK7bh@ zBU!mkREY2+^_hlFN1G+NYi;B>2-KjuZvksaPU{sNsL~QR%1n%=sG%u!#bST~gamMc z&wkjxO3|~!{2O5-2Z7XS;eie&j1&gBi0tvgQ~e2NYnsZrCNmS>3{^sNf{a0XqPb)* zSZuoU?*c^vs_HI|-G_cHa}I#wx4#=M%3M0&ReJOix%-uqc@Ul7Hf2Ks-$p5LAk~lVxv()!+ZGcao;dxlsnMBGWcJHn!}DE?@d!v zqGwDb8}X8zj+{h?kUmphGUl7LU?X`{LYlzYg*)%8 zauX>UXp~9&v_+cYE;U-jAfYfczm{^mluvq6QzKv4n)WGYfuJ3>+MN~s&V&i33#Q~3GMSa{z6?8#}?r^lZ(bnF{0Be(}>KR1C&604tII4$`?fzcA z6e4~o*I}hbU_P}xPk4E!iC-SFdlQx<*D~G+wrS!*QMgogqWb-{ijUij_cVv7cIBUC z4wMQ!wHuhhZ^0wpz*C#fF}X?t)m=Qp42so`A@TS=r@Gp*0@rNP3QZw(eng|_!4iW} zB$)4nGg0R|DWVHg4F;lsc!uKygfOc>Z<0JORtcF7mSBhX8o41lotsdhayBg89wPE! zKc*mJSJe)%`B!~Sk{kB84;x^v@E3S_JNBol$^{1)7q9Rd>NGGyf>Ob=gIJ;zG-Dc; zDJaU6JZ4TtxK%_2c4JJ3X-#^mKKkY`k}G1mFi-Ateh#6JQokTa^H_Pi{_a7x*2i)J z@&FS>8_0OFEgem-Q+tC4i!O@!sb7isU;#Q1bGT*5)!`<-;A|K451&=xf55k>jPP|$ zt%fP;5AA7K&Gh|)W-K2V#+A{Vm{9ArU|h+;u4gXcN{D06axUxCCJ@Wera{!;`2m4gb6dr&Vpvr96tslwqzGnQ*WvSO}e zL<%}cJ*j(kTHe$X!8AeceZG0oms`3fUengUWMk9gZ9ag08 z!JUp{SyI~eyR5=wKK*y+Lx>8(1_H$ZV(@c&A3B>qLN)_8hUUFpDnE|U*CvJEp098d z=G{vGB3B*>Av`jKs4)Qcs%!4e9p8y^1CbzZMhhHX%%z zR6q<6>ax8!}q&=rU$?zckA%W< z=`^O+Wog9P#oFx4P*=X?*|kC9pFvDa<{`ZBjS6s1vB^&%Fywgq@$PBvm+gX3(|jRg zxOukQ3-&CJY&xwnCk6ZEK#q`5xs0_=?pH|ICQ6(`z9dp4*Mf}IDrX_$Ey|s>*zhlV z7;KTD(BqH^y^W`eE@7Y7A<`6ar2TXg60~wwRu?fy!bvAppHTnTA5*-EQmYc3N2i$| z5AK)M;H2XQTcmfd%g6R?pt+L1>PA_tz!)#qnZb&=_C9S|O%3{c%VpHX(n66B*{aV* z9}u+44M)CH=8^nXr2*N$+{9(2$UxG|Z3FpDvP*^m1@iX^@}m{#nhM?G&L&{OF(?b# z62rjSKGMThQ;85tf)nCUSWkegnOv;N@5fxMm)ex~X3KbOzhOtgvU>*yv>8-`ogPow zCI}fTgpKX09bvEVu5?3P%42i3c8bx5Pn9%qO3W+bM48?3@_`MZdeWH_2bq5uQuvxy zK0*)y3{DA#Y;)l9P^?%s<{I~NpxT0}kq{BGL^iS35C``VNFr@=izF2}F4ILkM%=P4j&=rvWF@I68C7dd(J zcFu^m$=b!6-Ksg(rnWpZ5fg}zEeP{vm4;%_OSx_s-JQH@a2uQD6Ulo=N?YM12JdJ~tsnhuX|e)SUtd$KerrVVqcPIUz5cWu^{$2@J?aCK#iK^ zTxCkBk6dhsfg-=>T*;Mb+Z+~|(u-&SneiGI$*EVD&qe%BmT<+=DhrU_}7?yv$(M-Q0*-c=S=F;ZNLP%CMs8&WfKo)_3t} z2JWyC=eta*a;QcsQEJx=^i~HZxjf?gI?vepT{2hdRrCIGmt_VKJSRF=Z$ zMM5iv9>2x{XhlLFL?W%if2^{f&8K-i!{?dtN)eWK`_!a;#0;!aSI`(qSV$Q){{=er zSyQy&OuHG(t8EHiZJBTUECPrFJN3K$@EIi{wuvh0=CMQSed1nTAHQUBCf$TyoqQY( zH+Jr~&Yft?T|Lup{2#`cZpzRIp5g{bt3;_4=%~LPpk!O4KgK2G0p24~UP>QRhQ_Wv z241HHi%#dzNkma~K|_gUOP!@EnR{Unv&x^Pb|8Bt5!eO(o@y&~ z+`rA0C1>P^{@LBl;Dig^-Pt5Fbz;ZUeOOGQy?rrmRec+U(-t^+m~+)1-PE3=w@Yc= zPpT~05zJCKzDOfeJ@AHIq@~2&_=N@f@P$%woEt`Yqq1Z=t3G!h1}BNp9Sf{+zchYZ}of$P2m+Y{&$v#XvTbN z+6?CuR(vNIA$16eUvOIrO&_xVTJoB^0Z|iqQ135)6aW1H{%=d(zsSEB(m!I(46l1w z%6c;Py1&85L*WfDDeRvm4?u!VGMXR|P(W&8b*fBud!rLeAMXGMQGi~h&33!;#xFM~ z=lfRg*InqBo2{yfYP&X87z=H=(I$5xPR0#qqsx+`SVgIAnG5&sGMs(kxM>2?&^v0XbfSzH~O=3MLt?iAyHH1oE)#EH5HGcP2ny5zBqrskI1vRqVh8yc~= z`uY#|Cf>g12={bWwi)o#*RO!W{+u$x9tucZUK+IJ%WRJ(RqDF#BBMnsxOY8GIvmlS zMP~VYBd8YYWx8a!({%S>D$U;7Y$?09)!@w!F741-OQkK-zlE6UE&pC`G?$Nr_F`AM zyrO^qg$^U-o72+SjOVV@>Dg{|IO@EO{bZE`Pn-5`{1xj9eZ2-{(uCJK>6sIcn2KIq z7BfgL=&YzyRB7Ax)#~}{`LY?XqUwU3i*1D(&Qf(j%Bv<$HHQK&41G8u?*|lqw^`!V zz7B8Xr+v@uaiO;_1{x+S!0c+(0FhzA&sJsH*Awg!bxfB#Pk*78n`F>ZUN7GGx5ReS z-$Z}76Q2K(NYPK{)&huKA-%(MAfRdW7DXdGKtHXZOV-E<7TVl;xk@E0?IW7ksgfhW zFz`Fjr%A&+T9aMhE96kkhaI5poU=#$b)Bm>&r5(wdgumy){!ATPwO*7!9l_+a+MaG z+~?iy4k!kvunzC07Tze#M~{7T8)EUnlC-u`SPo80nIDRU%*)P`hO`S|kpj*j_RyC` zwdm8|XNMsP%?W!>K#Qo#^LYJozK4vmwx-7OaWyT%NtaRZA! zOAI@8MUaz%Dm88$FIcuf6(aCXw-{KqZCW2U**ebg1KNBnk;S7(kQc{tTV;EG7 zddPmnLWSr)^j?@!IFzP2+Ck@e%H2g!WPQi3!=79~sa?Z{-hyd428EXM%Pzt7$UG%! zy@TDT_Tlh^XyMH_wNNx#dK<4mD33X@NeP!oJcipLTeSNwTbJj^2w7fi3)CVY%dSwG zHEeK^q9edG+y~HGZln1&It}!OU-iJiDYq&w>9)6+ZYA{_VtvH0bcFlztC!FwtBlC) z{Ub__sp(PVzyaF9_{HA48xuQgS_9_1ww=Hy6gTN#k45z3JVPP&5=TIIpZ6o>;DCha z7~)br4&nynCPDUF|JI~GK`*>Fp_b#-z@f`mpUXe0WUsSAmHb@$J|j$0*teH9bLcoR zLiXw7@7ZD|e_n6#Fd^B|c*Aj0KA02wmR(qjQE235!`M`}a3d)|dORvwh~opobBG|i=R^6x*C z-zluaSKg_-S+_^PST(CXE-RM&E02>AEYYW zcw1^0(^ZQ^C&6=%Fr4bnQvLB8=|?|%CH}GAG5iE!8(1?YinGxdI4d@C6LZL|X4i^- zan1fYoY*aZW;8c&$6=8iNPIMw?sMdF?!j(w? zG>pyym&1h9sX3Cz;KOq|a%N{Ae{5H3whRB5mM4_;DVjJLONxswS5K-kiu+Eb!m9uyppS}(W~2fH=s+aSo)>pP?}>FT}GCViU%@# zO$narS0S{a8=*y+gFQ^C0#n}PWH-I?g{#1~r{)9Sd+X|Y!>@YJ+3kLhRzIvJNP4K1 z%JvPa==9afzqa<7#hkWNYhl0SS^>1kyt$3e(+Z>^ z2tfkb`7`x_rqM!+#U>`jKeB_Bsol(17p>ohiK?y0N%AyIW2%%jjM92+@Fh0}I*n-^ zD1Lc`nl44hry3i6kR+BM#p~pib`Dezm}pp-XN*sCul zJ)T2w;&Oj;dyH+2Y5pZ47~qBaCYv23YsG@`0uNXPb2=XGxK>!ibE)eu)N*Z?w};_^ z*2fvWMXiE$J*m1?R%(E!Ld0&Gu}eI3lnYw5{-Q6;BWtKQ1NHy097s5r*?tvQ$mgHN zDcB2hxvyFk>A(I7Me~_~XzLke+nDnJCj=&w2f2Dotl{29T}~QZj)QRx$_)km9uMD= z=Gq6pG*leeMVj+qN*yORR=K$*^z^Y!1Kx4*Ja3t;7L9HQYh-1LV4fr`mZBk%bK$LQ<0^}Q`IQ)rS&v2&K^d^M}fj91${kH`+KDYWb z-b=ATCVc%G(qdkic_l38I1P1mxtAm0S4*tuOYuqEBoh{2L#7iG`Ua@0+fGwa zF#Y^4Lgi(;=SeM}nw5t0dPajU@P}Y$vN7zRW&0!kcCtH`;}Oqe{cIk`ztq1x0SBEK zc4lGrgSFg9{%)+nqU1PGP^zd)GF;N3@u!hQ)S3t&Uk}0+ zIGlxRZX3+vw)(IB(i|C+z6j%ykb1}vz@$gr$Mg=^Aj7Lo?ukZ(i`tXnvEnl3TZqsF zprgkhNeBZ>HBm7^Q3&q^0X(oaB&gk$FOeon^Uc!pQlNisEx8qLznn`Sb5rZ(^n>2THi&nl>0p+p%3VgB|Ji z`9!EC{aR1^?2ul5yI5ppj|2V-vL4ykauX8vd{Czr?D8hR!R3q+vL!b#H+&;4>OE=_ zo4ICyzDGxWJjIP$EZv5_G*VFHB~H8ll)vF1%oalgw;)Bl=_DAjW=1CiJ56FkpXK!A zAMuO4XwYm&rC*d(i2=$XePbN-%%ZTW6!JEgX0SL=?kzp*F*M_yXaABdVLeykA)Y*F zE1&=aNR1`|h4rJd?q~_Q>xi}lTlxUHxnhHB*Cjtyi+Vpo1)!BivW(HRS|Vz#?{S)d zay4O7+Jz%WmT-(m-xgTBE*Ot@Q#bSFF1n8Q%rQV>H$HTF+MRa=scweozOhu}ZnR~f z-r;3#xY3SNO4jx_UQ@9Zm^*$>K_KxC6zJCycRX#^4-#pP0n|9iDwA>2Qb|q08D4Z% zd9#LcthfhpcJvoQD%e)nAlW?gEkT5C-#@kaLyIt_I)xttCX@I=w(Nl-yGExXZ>UpYuEguNZXd6* zW{Iff&`E(uTp8FkWWB3H=dS3-JNDq*S#-fZdAEqhAJ25?UukB%$iU^8m&b^8QLYq9 zi{21<0d3SL;;A`D@YG*|18$h{hNw~a@PYK@vSl|SmV{4cyg6+L-ByYr#M3`JOejgI zI}$dvf{Z+H?2#`y(~MVmU@wtTv*||#08+|Yb_m)q4+Tl`%uLEN+N1FwL)c1o-cbwd zk}B;GZTnF=!}@BPr?Q4^?9a_gKaJ$SMpIG;x2Sr-+TwnlVj|E-JK zU8SfoAT7BN4DlL(PbE))#}R4pM=y^;qnWUzC8CNOO;=t$#eS2UY$ii^^XWVRu?a& zn#+Q7w&hyg?{annZDqgftYvsjG(5LKHO*naI=eCk{ie;*s!d~;c6gl@;`0x)W>9IjW8X4wng{x(TN}Gg=5#s;2u zyo1m#hOK3tGqRnPk}jLI4hLARw^j@I-$L=?7lfaUFbAxv?c1mR-8iBoJmAZD7&~?Q zUFH$R9?j<~QKvXbtp=iJ(?RhIEof*ljej6~1|97se2Yx|lKM$aL>nf0o(HSf@(XIi z{;#OF8M^KqQ<8Su-k{3H?e6~ZDZ@|ma&`N*UW2Wihh`$h1e)+K@#rj<@;B|yPKd3H zg;P=M0gQV9h&;ZCh1}Ubu0~E`A|AhH`NKXo^jkUsnIsJM_qWXuurIfCMDbJfpvbqQ z+M=96JG>;>S!p*owEn}g&905u+0*o6OwP={OvEPJK3{PKz8lb6{Q{*BX_pCF<%N-M zEG>R!Yt7A%zKQhxzlt{W91YKdXu=`f2CNTcBvCbcT@aJh$X0^SP}^mZ_BN6VwObBiRi#DbyBmlt@-~F)>62Y2EJP4EhYZuC z(3G<;S>@od$pYJzz|MKy=-|2S?1GWs_YuOUjcI zmp(n&#*4fwESHBwYWCA0P30{!#-}%2c`HS)=56quG3oY1OQ?po>na+HXt?(5?q&br ztPdqk>+)(d$)P(reTZ0intfQ0pdeH&i<44BLkB;>DmYMR0RXOAiCohVjo;+gLbqYk zK1Ykl6-AdFvV^kM#piB2C>{Y>cnIE#j!1{xT3RbKOfyS&Nb(Tj-X?L zih-)uvgUksGd}WjOsnLH9~?2M%gwNI14t%ClCd>20hxKq(&LzG0ANQ?%`ccwi_HaD z%a&=#1!Tv_pu?`&BYoO?2(AjZg4}d=5&48y*`9WDk*YDf#7vN=Ec@Fh^f=iTJW7vf zW{R3iZv8F%1OpEDD#}p9Tc`nRYy)93~b>v_P z2qNIfWn8C@(r^<2H9M*orlJ~Q#>M#5^;47~^yiBbdyyb?lA#Z^X|r)9?S(i3M^ zTq)LM2tvsnXyOXycBW#>&NPC6`;KMY2&e+5WVHhFX1&M*GI@)3o4rsIJq>x2+q7Yqyslr?YG$D^6=_4S5wiWm7tw zZ;@FVV6{5@G8 z%v3Jr9DpiY-~Eu~^T6lnV=$aZP9#pQS{BFB5wxIv@Pe!)zLJ${$rw5lhpz>E$&gf| z3lxv1L60<) zo)mwoo8_Z1%3vrhJh!hmE7lQna7TpNsRakufVH9v{94~s5j+FF=*p3mPBB#OHXh=S zL>6$u>o9y;vgfJG>a&jZJL&6|>PvC?#~B-KBIO`UpNWZ!-3(+=Ti}h(=@XbK{ffcH zk#IUn3#KiYN-6Q&g0oK{979$x!PAs91dBWPT-EGvN{_26F+rq6etD53Roz#1=mSin zG(a1p^>tEGqKhW;Z;!&I<`> z`>`LD2l5qCecjKDGwPl~tfwQR8aaVQ63rwlbU3zQ-bwrko~TxCig!d(U}9w!E|Qv#v8hK}H~BKt=3Q*+ z^oq|5v|g`eO_^Lit* zXs`sqTX~Rzjnbw@xpK%nC0CG)sv);FkXUeMaVq)cZI~u3q(`+noRDrm#>90bZ`zYD z^gR>_3q{Crxl@vQ%phoFd%-8^m_ zLxl=9m#|khN3&AFR<8cBDDT%vF$}ak4r4Dl=Aiv!o77&Tj+PPjlx4m**cP6|zeJKthMa>=e0xYUY%n;bgW9~0W)Dtfex*WuXG1ldzD z$3KW-sV2hJ_Vl?_l^fiy+)z_Fr!5vZP#&}B`**%{%c8~Uu%*GV$OjV`wx6#ph6WBA zL#OY_(30K-^5m{35c8Xio3XQR@ryZ1a-&a0klaCm$Zb~Q8Kb)l1Epqlq4Hky>a_`W zoxcTh>U6mJsUSX`iiD3%(y9QwC2n-NX17Jzj<^KjXe0UZb?+bf>@W>bVDb%J)WayN zOIaq58m+hgWf#6bD7s7^OWA^I*9A7NL&xWQxENDq!SS-JFDAk-gBZGQA>zzmLJbY8 zORnX_p9>-#lGf;VhTXLsYdfqf>|p*R(QNf4BRi=XC8jv5tr6s>nbh|_c+7+ z)9{ShEsd#0!~C`eydw}@`;-ABImp;}x7)5uq$wDPfzYHdjzoD)3VX9=17#FG0qm)KD}Q33qMjjT%7i6*8Hvz@#CAMxxedFd}1Er*Wnoflh|kGfUI;t4TmmKHZ|b1XZ!#?oBz}=^05r)U zF)RUsB>EmO(31~O5-wHI+7FbPzK5Xjf_cmVM5eGC!qM5~Bw`}%8--YtM;XoJCIu3L zkBlf#c!*6*gf8|&ekZvI0XK99j`&6P0Lm|0nyvRWSSI|G;KS9QtA`}B*=LeExD#RM z0F69Rb#}Hx_`%vqBXfrgGiJNmuBni zx`&N>t~dWUU055t!e;YzEpD>?G%fLOe6`4V%%}&wTvymoodsXpPv|*KndX|d`Nll3 zG0lS{bwZw@TG=K8E${j?i7H$$rD3mv#xKTj{$7IDVV;4WLkCbat4{)u1!J*mP~e0@ zN7k$IIn^euBQab|Y94+V5>|o3TVkY1)&Dv6r-bu;U;Z+xF$&Nxt(o&@o-;KHH|K>P zs6R}CK19z~CCmUrYIVriI$Z^i5|VDfBRfBi~OyEol5wsJBSu!W-c_I^=E0wE++z(BX5Kwi0qfTpzB2rG_-oo6EHVSRT zZsYwXh9KG(!=D~}b-4{#(M@bQ2yW-kOAzI&tU9UV;~&Bi>ECiRV0Bm8t{cD5;Va7= zm+#-EI#i4}ACrU^nego>42jTI#N))vXDro$QS&BvA+tp;q!;zIatkM-841=!a;Dk#W%t{*2e`j1BJxzr^`1P9>WEkhh)jfT)dhsAqr2 zI)VOwS-MNlB?-7`U+zAC$g~`Q{YD3WnD}5{|NT)usg|zJ&8WkR`w)`H;!ln$;*?5d z*WPzfn=~EL^RqLF8UDD~*q_%)e%(2KZhqyKPdjs;;|o3*VHZg?9laPYe6|7`F-?Z% z9sJ7cC3XX>4KGcNFyBSzE=zCWs@t%Xj`gPktBcB4KiQwa8@6qR7bRY{0_@6@letg8 z%T0@}o;tQ4giuo)yhR~TWA>J2m)qI%(>x`m-~8A2yxycX*dn};x^LO2(%9PV(28wk zt6etd?QG*s=h_-L8Stdm{h7tymdkHf%kvxiI~U>CbBNx}w_4jV#ZIlB;$`>xb$4+& z7@5nid#p0AK&6GO%}vH^=R>}$P)pn& znM-#Wp3tESE;}p6-|uOE;vu80L`|^<7`x!tln- z4~cU&OR}oYxBwoQ4P$+lQb7~rYrOA?ySF47tAtf2mp))kjZY_3o#8PEn}^#O#1kY+$bjE_2W%rRciw8m7Z^RouCh0Rs$a?&S_sEOOcURL18x-gDGl2sW zc>Gm?E1_XThK}eqxFIy!yrKFsPeT=)6A9%EsaBISW?xJFo^9yZu z@MLHcdo?KpVmc;t z#uJ3W#%DBA00F``1-vLK61exY>tk|v8H9q0d^`5w@A#brDqz@NF^Aj3`G{Y#r9tq3 zcK6*)8qS6>mELDu$V|Iji1T#U^nwHr#FRPPcoThwKEkcG(C{3&_QCR50bnDt9CLIf zn8Hb+S?o02Jj~mROh$lGLpeF+BFRC4PqFfK*aj}2dTvI7G@Y}=@QYYEgHa?Uz*XvB z{m6KC)ZV#ZG*h0Fc+vVh#sEMM;@zZ9jLIneu-t6WmeE)g{ ze>z66l=0C9k4SI%{{5TeITjt?wg9gEozX;$Y*%~z$6E5cNWEc>vi()>{sJcVoGFi6 z^w}y>97#JA^`+ov|5lLrwHKcXrOM7^o#Zf!?eZp_8mY0aa8<#QiNO(dY;2&Y+`7=J zRnZ^a8k)g4v+sZ_4Wl27^pjXTczSj;9GX)!gjSm7O{F4+V?S7;H#)aq5E#+Z!$;2; z2hSG|wvq|*qiZ!op;FjCd0PBSJ5`-Cl*B;6xty;%mdRutM49)`7<^H-MBSj%eSUJ= zT{eeJ?U|~W8z~6Cp@5~FV>PXGLV}6gjfJpnZY}bz zRbfqWx=CHiRY=fK$(48(f?G zgH#z*98Qz-tskDTnF%^qatou zE~q4I$Pr3a)8+WyN|O$`b(8=BF4Xx;$YqAXI^ot^L&rV*8>p30?@^`+S8exgDs@w4 zWK?&YOzK(04tMeh2V;WeX9OYk<#4yWNt&bExIqg5=%|A#F@*VNQC#2b5&xj?JkZ?5 zY3OW{`*`|9cN4||O(}bZ#KO*(nDLj}wvu8gmBeMhNs#tX;@jRQ_CKgO-O)(%D**M6 z>}XV>4kK@NBsY1dF>IOBnKMMPz;U=zR}9%n7Rp*TWe>r^xK-_pB3+_uWEW!!Uq2?t z_Hr}{+LdBu@L?gF5D`Cpv2`_E!)-GxLweB_TM+{&ke&oncwFMFtfE{wMT9m87e#VB zvDa`ACExavNapbRh@1%CW; zwz-9_M!=|if4MGibH}|ZbuQJ`0Vw14E+$@srlFOJapK{oB?Tx_O{f}rP`^5_T=Ze( ze>SR0eLe^PctYBrvGrn5eo&ne$I((MDD?pM=bpQ`c`nFq*f^?o6R0VyCWE+l&@`hu zO!p1|S28_Suy|Lxmp`45eP<{u`)?BD6&FC^nA~GbO`!J7p*?I7>yS>Y!@AzI1?ui- zsQW~!yV@&G%XBA7aTq+E>t!Uj>qwi!pzGop{F&vnT z`rNet^fy{~>|n_i@iO(>)A5cY#Au2n8oAP`1ds8!*(ltT6OX)#yJn|!6Ab|0fX|N_ z6mq@Pw4yZc^p1g838i!;M<5T8u0}l3uWFxi%$B-fj(?2 z;E0rumx!(DrknnaA#T7@dt5$f0-6}Hd|8iu@E^VCz}AErUESwpX1}fA?WsonrZ$RR z$XK^uM1TC02ebama#b&CUT`~!k%%iWM(1;PG+kGmJ(X`GrK_+>7uDFk*ZlDGO-io7 znOFZ@SZrCn0v5mBb)(ZHsjc8|adCZPv4z)&N@4E?K2_(cxmzd^dV#1g5%QHSndMqYbnzB5Z|ik+v*2<8Id__H0=X!9JDj~S z|Ig+!m_oUa>tD04O=nI72DlWPv4!8q)NWTEB)}<^8dG{Cea9DmvOME)s_V`4P~o?ua&+%>nd5h7o^U`*6aNrH zQUnsNMCpMXo$+yo{Bv~`?G_PK#Snz+W{*C~rWgv%SJnAD$69|<4iW!8Ewf3k!>)Yq zn77PuHL&h%8lHR)#LZP4J>Cy9z&DY+qR@v?Xd2?DL$JxvTm{EOI4&{y54^RMy(;d1 z({j1xfT+D}s2hoKozOU_8~<|=^v{36fwYv%`$dspARmj$SH~K>L-|1AZD_!$pOD*N=8hLW+>MxMnoVZs?S`k|E82E%347>r zei6$`n)c2sccu9$=G!p!V-O}Wms=GCrR?2)S~uMeFz>+@kF ztpB6yoZ2&imUSE3wr$%^$F_}*la6iM{9?0X+w9o3?VYvPx!mVR%%^76TQ!ExN}aY| z__Ax|)834Wn!4g<*!5PS(fU~EJ+8n_;vYFI6V&?J7w2V3CqN69N0)Y2%JVaUuQ}M` zEz8xkrK;_+5&oiF&q7G-(IRnEz!rviRJukJS2~Cdo|~gmlK4<*R-Ha4vq$6FRaX7h zttO_w;LYl(^xEdPm)hyjOl2+GyLkub3x< z1DZlPusmW^6NF|pPWx}HZXedpt&+=?eti=(|6P%Z;X2}@^dHfmT|$@D@Ct*q_E)n_ zD(6a7G_tPtsGUfb;&U{=a-8|*!H~h}u>4hYmH-^Ci0C`7jt#b+>~0r}t)=mXQP2Br z=4GeA5LZ(&YjcGTV}2#mXaV6g_^d$5@E;dBA=7b_5J%%y^-N@6y6Gw~oP&>I_S`|HrtNcP%vHZhj z*pgh9_&a{SNJrrrs_r+fz?CL)zhGIsx|)DHu-=9fF!HmuEV{>~j*QYS!UaOjv+&1x z_p77D)tMFQ4A3^BdaaBxca($6Lf1(Qdm_tt`vH>uzGa(P7n>TS`ch+6#Icjz^=wAx^Yq01O!%Z)4!n%q1n-PeF$Rp50}J=2<7dY$0(f6J zM#5V{ium$HCFCz7VOA#wdoaRm0m5?D>S>f%`#558KtmsI{(Ob&_BmB2GGUaUCw?-6!}d_d0KG6P2>{{+!==k0lxOh z|-gJF2ti4Tm4hIgl^9rvg9nP~x8DOv5a zI@jDu6`!TQlrSjX!)~V7vI+^JiLqHMG?I|0Rf=?drh^R<_f1k|tg}PGQA$lHvq~B) zwv7QHh0bDQ-1`&t5S#8Z?Q_ls9CR!)%BKFUMQAN;s2;fnbVt%J{tJj1z~aVJr4H}; zaAiv*0^ufI#jfdXWYn__TUZ!h6x$2Q_AJY=8W#IDB6@~u@_osPjJNx+a?=r9zu5z+ z$gq#o<-EsLPc(I*!RvFO?gTn_xTcZm5T!zd)QwWkq@&ehi-Q82+0;%LdYEbtwIQrF zQcFy-e+j2%sS6(5*i5j!TN#0CeUDuaUiXz|skb7p4?Z+>MP$SBgXPHQpq?;N3=I|s zlbGukU`^1ZF;I=u#jvf$I&LnUyN*P}ZiO?|N8dDjPBxIiAL@ZBg&Is{Cl>4UA~LAU z^s%g{@fm7Cmbv%!5*CXR2L=Nh#0k*L~-q;*q9% z`;j0-)djhafdI@ro9h%)+JQ0DI&Yj7`D;~K-OgQgBS{AmdG%1YoW$a8&Z#L= z5>ek1OA7{4V}+;W#|;`h;(Om*^2(WTCP2t8=qeITU9`p%@6hiLBhEmj(YQtK-^)FN zWb8&n4e@?>fzeRHs~5w`n#ywV#Ejlbl~XJGxaxh$!a}PEp-HMS0}4jXG7gqJ>kJ>q1t(~qm{LN`1Sq{5S|Qm;3=(= zxwzC_ZKFQ_3K%Y?OO_#qC`NOx^DgHrjUg48SkSV5HXA`Q67p=Pq@>4Nrte5JEu>@Q zi#NUtv>0Pb9&&~~RZY2DGqdYha0#Ma+z5YlJc1Mtv|J4@M=I43ay+99{~0%Lt+JN* zQRZlcCUNGOcF~bBaz8JV$x}8AfSD@QWUw=6R>|*r_HFD1o$~;eUr}^X~lWxz1!9Q*2Zn%rdoQ08Q)2mq8@w2N9GQ>^;rX5%`-s>nCOMuBHoSLf(>_UR8k0A&9Ie4UQUv& zhS;zY`;puD7%9fZ<>S&H6ln1X5j7*6#L?mPG#f>O~-N1aK73oVG74fp~^7h(Kp@3aM zFI+el2i}J_7#&~+8gJVIEcF|9$Pig(CzH3k+m8N8PJ4{FrVO0n%ePx@&J^Kfax|}z zU7;f9oP+zhQOZq_fp8ql5{BPor)XhVS1l%0+_jpp)VAt#Q$r@5xKCJp|GSOdFE|v{ zmu>g4`oW0^eY^AV@{|f;t18T=w#%!<+5!y&`Jl}sa2jFHd8~VOVYiJXRC=$WjQU@a z`l3tcy&K9;lTCKgfVCdH=`PMuSw;~5;%%zSRzSjXQ4N=c^;`<7A(=ynNMqNV2vKU5 z1`nbn-B7j1zMq7BWtZGL@^oup9>09HcI7VLJ1WNq4#H5I<(1C%O*E0@o(CXLjb-;Q+_iZX3F(ItKtj%cZh;Rm?i?f!#YS+{QvaPVJ+||o z(@;YzZ=JBHh`jHQAJwY?FCQm^_Cc2w#Yjv-$h>$oS8sl3HKs$O_x!gZmd$&HJdFy4|PEX+HXVoXC47;)kB?orh61uH;`X;?RvU*EW-5U@^}kKM88&#H0Ehp5(l>fFJ<61e(6KBRFZKU@Rivtxyd+l7Y{l# znRly7>#pD6D%th&Ze?JX*s02R17fB9y!d$2;~?DZ{B(^8&d86MOBDQhIyvb{S1yUm z6%9wLUs{fJem8d;95T8}-eG-|(Lrc5E^#Ua*!lw3w<(oP#xw2tumxiBuXImxSHd4$I0}DZoM%xXmJe(#kTs1JUbc%C+R{=M-4vDw zZLdTb{iw`81j+q-N#Ct+$Q%@u)Q}q>^xeo5w_snvff1BH-&u^6-b;<808wU~O_b4trZ%7%>v=vMIU|B&e|>+sc&7TI)?wyoVtDhVqx& z9`BikejA*?SMG4RvE4qizW>TdqxY}cx+4xRm%L@OF2@l_r(r!l{yf37M=1;ezRPdm zyu)cnM4JCiTSC)-(Sw=M+mj0?!ST@F{tt5D59cInK?>pW(=9K*R4Y?vK^F&+32rvV z4~X$zC%Y)o-SNVZ*C!*dTR}%MEpdEWV3KpEwL*1%ZZ2U`P(fb)>%G>mGx8fZtl`DA zEKM-V^$%evpqgU{P(D)nprb_JzOER(u1MEz{>Us4eO;S1;Hk%nIMi-e`Cqu9dmExz z{YhozOXinU$5$Qs`)tjbx*K68($mn}?DD3O#H#Jk7n@f2%cK{TzaLt^_k3M>L*)#7 z!sE1cc{i@#nc&>Sn(};*F?(X{@g>^14SwzlJ3y-oK?R1k3w^fFz=d??4jkz0-ops`&Uk| zX#0Llj=GhjhGS<0td6^yvytB|0mUn90;?k{E<+@$mpAMJyT#!KF!I>&*8^{}Cx}f4 zi0&E$kxAIYrfr&kqjEnW4#wkx=hpLIkiTPHmuX0VcXk>7!1n2V_$^)_2?8;FYNG>* z-h}^=7GcuFs6{pelj+Kyhp+&lsGMauN9&o#pTU$+y~*93Y%9V)CwD{l1Ez0{i|h1+ zk_z2FW5yqr&%E%kL>$;92WNm~sERyGzpQ zR4(x&^4K@wF6KkINp4WPwkcmLwUJ{F zrHI~%=?M};fgJCU=$Ponh1k;b23xDV$L-3OjZz{oW{R|%Dk9(qp@JLgzIJ;wg6|JD0kV>pgvYmLD<6e8)&=Q7UE*1(VMbQ7HUDMOfkFT8dAuF^!l%cM0mxdyHLW?1? zMRG{DRHR;ORc)#b@I}=I&cm6}c~;y(+49DAdE zpBqD$OYX$4b|zOHZ}<}fXnyf*HW+V0qChf-NG1ygg&fS}7u#X>nLr0-8Mwb`uS{v6xcx)6E`zIB4>$9Q!4qjsHJVgZ&gAxRI|*ayX8PccCOTlPy?rx;1DX0S zCHAJb3ZRY#Q)QTrH9$=q(Hnqs79K~u;AXPi;%aMjxYwideo|R0db@IYI%s3GQDpni zy)S~EIFJ~d8kZvD_qAeE3qChOhqMZ(T<{zG7-eezLLg@2Ep(`%K-oEXc(Ey`)xrdPX(tNdWgB4cbo)^P8JtnjU z2qpbbBBipD%|xy+=~LLpzoybly>erq%m=ov^d*mSaJ5-vX{2S}s^F}1AeyH~XbvP$ zr1q~|I2K}Z9P_Q)c{p(k<8gS4w)fl|p&~j1Z(-5kG6J3kBz{62$XfGo@EDc+fpE?4o@oY~WkuGfK0X zCTtRWQTuXFfQwSe0-hee^Z?9CzJUJV#T8>PnxEGp>=ql*)cTP%u1&-%-GtOfFPTZC z+Yo25mMrTrh@#J9+pC$byN%&|^dKqDGj6(dRa{YHjGE!n247D8YWRtDyk0?C)7hdk zbjet*2L$yhszb8_9P z+9nj)oe=Bi2SFfW6mSUR=hAn=^7z?_c93aI5)utHl|m-k06~J zr=KYAHOvBS%QIz;qZ+UA@5}1mpPxW>--9H-qN4s7zjjbqf`4gA^zW6;S6Mw2;%0JI zQwvzc3kT&?+zOf_c8JP?x;5s&^|*iDZD$NFV-em9S7hy63D?#H6fpOd`MpQud)6X6 z12SeJo)zRd(R8?wBu(SS=e}k}N-_+}Wqx2u~lOC&U$^ZpL1%UzAKSsdw;f`eSYW<3fh6a^QLCPA}@{gbR1cU3YiMacXU!}D}Q z8^6BsYRQPG>#xEi60P$lSM-xMVH%>;pV<8ild(P>FSk9e1hHz2bKzG~U{4iTR`yi( zFE>33I$y=C2p+Lf3#QI%6Z))c5{n!oOB!89m3tBo7JAW~>#vPxp2yuc2^`RGFe=`+ zbV@27FH<{)p75IqJ>VuAYw$u6)LG;zebz%KPWD#)C2*%SExrDSMCL$AAV}0(AN6l@ z(rfe?98n+Iwk?#`Z#)5;^REV;!8scBG_5?&G0fIA^jg??n|eCC<{Gn7b`3N$(x-AQ zo6O!>?9+(`%8X`F=MqSIT6tT+VE>8DPKnI^ndG9^x+a|Jzbr?g>mPU_b;c%nf)smP z6KanfC1h+Kc~G{sa)&ne>=7t(aZ(n2)j^znO=G^OQK z_ftLig;IwUMglhbJ1JpoD~;gt4qe8zc*vCKaxHQ2Fj|C}-`(>BMaOXwqVLI}JkMK- zWZTIS8%v)PC^*1a&i5k$9Jo?tujLvgfX5Wm8@gq+wBu)@YT6+>g&K?Gj0a*tC1)3- zFMAN#9YUFQX{V@hj`rRe`K)N26j*ce8PXpk(i@_F(&{Q?(&5!M&={v}2G-Nyw&?Ok zs7doYQv{_zP2baYp1-gllvgP@<2TfAuP?+0;T@fU|7V^0fhcBkqE}9ppFscL5XJvE zYyWqZ>;00jfe9&8hiC^52?EO3f0OVYdej_uj4jE`bhZMLtRq2T-9yni==5n ztsPHVlIHvK;OAEF`(<6@t+Xc77!~119wmA!>53YurV{r)yMei=IZHU?c6M%ullji7 zE#UpGZhPKrT@xj6QBrayIA2TOrBl7E*HgZ($L#h&&?TH@+rG(Q8!5UeP!OWF@VBF2 zD;ThT2Pl1esz6-yw72+_73AzZ3@#N#?Jsx{Sj;Pwh?o7r+q%@qQ*`1J*&3a{9G`Ih{@oHT z93sec{ofCl@Z4C#UJgC=26%F&gkkJt1#(>MKJ8^!vD&^uFaNSX?k2>SD(W}u?-uso z6Lv)J9AW&_in5{4+#>GbZ{lTm!PPLTcLCZ5R?uTo09k zbBu;!{9IrswFrw43l)O}#?Z7CIfiu~_6w19eILqpj+AQ5s#5VUavDTUi-23_q#K=j z8M;)x6LcCo=;?fb@_s`Z#6cikun|94$>*%Xes{r8?;T&`Xsp4?Ldl9N98^lXY8;dL z8&Q^tPaZhk@h0Vv`)0jPom4ZZQp~5bDlnTdt$a(r!4F=n=al zcw2l>`yQk)(Fw$%1WCs=>D6pwKb!eNzTgM4ld88-r<__FrqDNNBWGFkKehJO0o)`~>(mCEmD>@#b-q zVGOUg;fIsI&_inQ>7yDw5IHU(3*s^c1vl{6=~M@4!onkMtj;HPDU559a&mq$`Da8F zYgtyV7lt!H!-dM2sH)u#%ZE(t<_VaRg^jE5O;I+sH!9AfmJSM%Xi`TdJ5IYe)obwV z;l0MfD?w&jkAY2@>LDOkaNJ1scS~Hx-e24qgYPktQa%0KT%}gACN7leAr`p<`7u5w zX65MilUY=2{4o&_)VF&*E5!%L2RQfM9X+{2`BPQ0W$tVDc)au=RQpRy;PF_*lWb=($Na)?>vTQ zrXU}9G*Sl%XS!DR$|-P~$E?YLe&&36@dWsN0~v9XJ$Tne%oZrqUJ;q{RK(WXf;LY65TowCGD9fy2v!3Bh{tEsw$I`y{Q)Cxf9i0J*B%h{SQ1lYCH zC8)Iij?Jd)8qYzDHT-vRQHjawCFm=uR-qrT3f2_p8{RuylkEcHAodgN|_C>U}6Wi!eBI8`WlFh;H=yEJdwkKP~NHbb;PNRek3L#yp?G$mVv ztLD!gT9S%PN9!x*spDkSu?fa|@>L~9G}C3r*7(TLne3SchA>F11cf5xQJ>ss4fwcm zU6?3a9mL1KcUt-#Ra6ff8I$fEe7@wx^TpoFxY?2aHeM6|GE`MXQi^*(M7W^z`m1(T zO~sOY(IP51$mDLHf<>PO|E01<{%dv+CUhOBP7%?a7X^OQV?%lOI>`Apakf`#>$T!D z41WY8zBE!KmbY%taB&-J%1Yc>tT%Yhz3Seh(X zCZ$b9Nm+=5FMEP-G}57LX=G$oaxt?P(5%t&y`-?cbXw26J+ z5r$}c)HIGhP^H`>@a7g?|3MQ~7Osz9)wMj#;U>u4#<^^ah#it9!|eN$r=E^ALylSd z8G)Q;6?+5|nRJ;uEsU7+;dUlgt#~stcDVI z-F<@*@qMpGH9TjtI)gT!Ej-_a!c4I&d-f!M{9b-*0mZnjiW|BS}cp7w7=IKvf_`9J5rdv=Nlfg~K;*X!X5=Hr( zg6Dc$D{zVD9a(8gqU>wM5{i`IU*Fz-*^`$|9@;N`l-`8g4Jr&@RHlwTWOZU`Df<&j zP3IAy3Y;dauQF`~pLscOZAMG44XDPJKD8mkKii|=^$MFT6P+_h=GnL^i4a8QkaZYC zzeBEJN*1Ydfo3~gym)boUSXwj{cSvs!3JBTDf$z{pj{h$A1|fP?$R`JTjBjGky$KC z0fO_1Bgvac6kR<&sB*c#zUMcNgH9kBUJ=4*)0k8F#Xk~3Gaj7*syyFXE={Kc2)Rug zpMpoHicVuIYr(|lwB{`W-f{Ma&k@CfkiRdS$4EsOA}{aW5qJ*hC}}ILh?pGiEePRz zOT49Xar?-%6OkOvJopI??BZcB2~-BGxk>LH1~lwem4$G+9yl$aNk{oc4EIBcr@@hk zlpLt#?js+iGxSn1KA`BudH<~3?H3H>Xx`CVR)7b{8a!5_1n0||ROxh1@`g#>-L-Y= z_B(k=uABK>m!jflg(^kFa%~*Tr0zbyvC+!xW;Ahe{`4GH$Glrk%yakU;App@yQ#9D zUxNm7=T!{vok76hd&tsBLwW*}NaY6zBpr}rSQ_51z?jyXfI$BJsRp?B1}g-6J!2fl zDtqBkKmH7b3lvZb%tOMo8+0?UI{QKi`|NUYND4kXUta3=PyM8L*CD`-p)yzlps`bH zUEWD0t#!a54v#Xr=v)QpCiu|TEye4E*98<$Y)EN5r{3olzRJt?E6oU=${>V8b(|fV zE1&xogHwbAUyEv_5h`~|q;Hkr$dTo*J$ot~d-5?U*`W9}HWsW1ze3Q*)W$S`c6e(d z9V>>fNQ)wPtX=&}j9Pp=LBXXe)z;9FqUYY`(5B-z5^Wvf7F0(T`^8g+HaIgWaZX%n zEt&9itt>ux>^ccmqe+~1K;IaC2t~8a3s*?7PYHcLAXBel9lxAi&IBo;-l)Uu)^vOy zAosL`Tj9daf1P87OxbK=z?W8B@3MiHo9#_5+d9fnl&vz~d#^D< zk;+Fxn9I7nGtVpI$Gu1v1#+&{fgd-EsL{+n9itLZXBZ$IoJcptUt=!fSNlamxv*6N zWtDGWJ<)?K`v$(UVehw?37~4Pcg~Rd_y$thoX!^bFDx;#8W_EV6McTUbr%i~ee3_+ z|66t2kU+Tnq2&)XpvyGV$;6^s-&2GT8VXLf)U96F#Zom4h(`%>Y_qJuswEp_muXe? zI|1wFzg!O)-_H1brS$o!>#4*o#>1=m860Xc^KD)ZBlu;6(qye^)Rj<8*jbMduVZ9Q z0e{U}Jn777Xn6n&HjC!MPpRA|KQ5YE8&1_w*+()O|yxCpsC=qxtsB$mD<(MsZhfBJ~uo?`C3zNy8Fp5 zN66?t8gp^nHS_ve(U+_kDr0x38R@bRqLo?(Rum334F-bF>cKz*HZv>M%qA+TBONEK z0|4hZTR3iNT0sPiMz1O*+K=d8mu9{}c6G~6=wEQY5SRCc)(@lGDtcRcG+%_?e?TgO zChN?LK|ZD>>muwM&K#FD2I4Qz+%AMx_DBW7s>k9Vx?wsh+pKNxBM#Se|n;cteBQI}q zT*4{VVb@+SAh0^ym!1!w>BLfsqWvXjTBI23?Jm9=FC+--%;yICkui`GT85)UH6W2{ z(VlAPpxoE6IHHp{OW0)R-2=|6v-Zfit#Fsfz<^G%3~iOON!@J2Td<|e2BWG-^Npo3d3U_9EFO?85ID8pS_8xr zHMkNO{wNx_0l6lQ4vcxcKz~jCo<`L>F+#l=urLrm+rqjqYFn`afi&334dXpgHKwoD zjET!FH&kOA2`2%6Rb1~7CS0-78@MfuPM|k3U^zQpFyj|dtI|sW`^n}i8*Exy5-T%O z>rbKE7HjQQXkZDA)tkei-En~NRXU*#;F^IvGVeUrVRcC72sQr2^7f_xsPjjRN!}0f zx|pFRLP(Lm0__4ZF>-jQViW}iQ|Y+<)B;cx&15r!f0eYqR;(G)3>pe9ne35seNE31 z>zFj|5#*kcKmd7b(R+%2%~ugm;Z-^VoggZ5oZBTvHbup_#%4sGN^QBg@HIV?1O7BN z7DFXk)P+Yp5%(*3(=jGE0;p1o{?xEjbxw>WG~{56+ekx45-RtH8Og$x)%~t(5B{hf zZf>bj*_vw|KZ}fSjlV-1P7^045xxf|R#LRH&~7sPLYr`GA8rf^yvHebI_%^;@)DJ@ zJ|EQ6r1D^u&;La_Xts|r{$gGkvJ=T_8h&la zePG864AE1hB|Tn6>JSS%2vtsTTrTm*0aI}2_1Hyp82v{a3WfmErgG>T`;#d;o3f!u zM@0{i7|a{&aGl6G`E5iOwJ$aRU)D=B1=0M>tIi&jCJ_5E zR4B0=ZtU&Xt+_^xWJ*sObX6TeB*6|yG;Mkps8u7TbzztnnI}wA<_c|Ds>QE=b}_3p zbII@)%CqQ)OBWwoC%#fmX3t^|H@(GY61Z2ifcexG6<#o0BLfqv1eOPbuu7SVKUOf~ z#`C~ZV(gmC>_Y2$GRtKwVPG8l{V!1bD4KDkyy6jKKw-!?p^iSPWR{v&@3W=W&TOUO zm3zlCC6SSImzapScp8D^4|xW#Ta!NVW8A< z7P-VILvQH(4oc1_G)1S3ylMX&Tk(9SK6hAOW;^W<5#Yltpo8|E`lvDSa5Mxa_JH@#-(#0^i#?T5t+9@|oF zye$|Yg;#{IS%m6N1zqn7VH>}=N8O@tV%>9e+Qap*Trx5+7}ll;nq|V}A^Qb(RRrGV z@@0@Wq3f6nN51ELa+0l+<``*qVsv-Tk!6j(i_t8HlSQsPTm;8+1HjtW6t;I2ypjPN z;ve3%2d7Y=tiMN=;cHgqH<2J@%)e|rQg)x;_$a}zAwkt4y2>{@hFKeSfz5Tlk7uGQ zLZLokz1~(UWVM5Th?|X_aC#D#;??G?H&|QG=KgYKn~i=Am7r7@57GfOB_X&=rLnhc zRh1u%!v#qH;xjBc9Mk62wf5|lC3yZ)LV2f<(^=abb_V{tnJbofag*r@Up8x0rj(7^ z_rRVoiyTQ10=K9$9VKlt-S68(5YjPUv-%smP2{axF|tx^(@?n~9o|`*7HuI2*L5&o z22xSIMc0hN);GMuPZi^4MAS38|M(BhdvNT7e31To!)*ObwD=CwDMj)K;G@nVNV7mG zTWyrjD>=CS8i>@aLxmU*d3EAZfi6eA^AIFRtfyp2-a|&H6W1cT=DU{)@w1uIR}3u7 z(t+*o%95r;E>jG5Z%GpTOiP?vmWzLVwSmJwWN1y!7fj36D^Jzu2ajQjMU;$WUG_5P zdAnm07kgI7wgYJ?_C%)hZ!Z=b)aR8(H!v1mW?L(rZblgF6t27ur>FusX=gtp`tORY zR3ur+fQPCQcsBF#lM`S$)xY84*3rB1*bM+|t1Q?hoGL6a?uD zC|`0MeL@fHu`>P6Nzg#J=9S~y$8F}a=1Dbe9+Ge(Qt@nax~$C7;`=r3E@Ux7De*D9 zYQFq7_L{wM!6jw<-tD*gzM@shtT|<&w;?}Q!O_p!6l#8BUNnsc}sD*_bP?pc*W-x(1viubVW9%hy(gg2uUK zQj%tiB@kV!JPt(R<(0lbej|hiM3&}b^A1%IDC3a0h5%z@xBFGL@YS*i7g7#kxR;&i zglc+A9t<~Kt>2q9fcNrX%=;)!D3T|BT+v6WNuoAi_D9!$u{-EAB1r(rlBxTIo=RJO zWJ)GoQzE!c>P%xx|Am;Ee9jY+{>@fIsXjk*N;A|4s{n5&l5+jW9K-ytc(n8*zUQ{iM(h$po*Z@Ef*d()kS))fsjm8& zN`wO^bd}^21cc9-lij2X)OuTVD|=1z)7x##$niU7g2I4%s#MG=q^2~V%teqH$v-@{ z;~m{IPD>8;kh3S@E*c&oH>gcusKd z8A!i+&`k{or+vh6e3qS?$luP?*!*yRDN zT@KkixL*%t?Lp(eG}Jk8hL;rs*Yqx&9aU7%KPP(S0yYt~V}4s=O)?^ThV#J^%6sDc zSrpwSmKXqm6(0vjJ73(7AdTup&+<~ZiS3uh^Nv&piRgMtSx4qAvg^=lk|-o$(Nkrd zE<2k`7CMGpyA6rLlqKG^{&yJrTFQ$h<|QD`(6L?Q>=5<%9=xiz9@3Q7$bz@*u#76U zjL#xm5?QI`az%!cC~!7r5wz}3>+RcCN6T9`4|Clfy&9IR_xBqTmBU0;{J-7aeCyB7 zATN61->>zbSnvOf#+O3)FB;#hK4($hvV%9mB#e;hlRa5XI$N?!l__hmu(e!sKQH|iPn$5w4M`~IHBdhGZ5vPNqy3?B~LcEGmj zJx}5G7eVFF^BeW~X?YdxvWcF{YK^8h?Vlc<~7GI4goO#XaZLRT}q>0{l1APm-TYNb}u^C{baix7XgI+zD zw`dHs?e8SfQQ3mz!ebBVk6biC2yAF>!)))Dj9smnCgT)Dmk9{f4VU94J;{q)zY%GW z?l+f8PpfREb!^S$Bk?;@b%2@)6dZTO#Pb51fL!}a$P!xXDcIiDUSMYGedU|u`7A3m z5g}dbF0LQHI0UeH7f2#=mz*o<(WV%AR>wE$6CqYA*e89({!>dwrb34#yvu*{HUq%r zxa7f-mv)m|1GBGVS1aRlRFVs1U?AgaQ3#w~))bYn?SeCP-G5;M1xY@U!3#rM8UTqEQa3W9!D zr71jGj+Mu`=lg@b-FNKv>(Q+yi{Jdd-Ve|dhLk!5Vu=jKx;SgjAboeCx(e(Ft;0VH zGsb=md93#*`s26m^p|dG)RXi|k~{zw|0ST7H7^@zmcOh7kv$?!pka4z?7}D(WbvC? zaUe+>OKp4zRlfs#tTKrF!KZ^Bt-xWAU5KZcK;zd~kJhfg z%%OfA0MYxP!VXcwR%H)~#SQJb_}~+wSjS9WbG;9?UvdIasbAO~7BsoeVQVu;Kv1xU zn_m^K8}cjjCeWN^^_{2)_$onfe>{E5Uw&~T(ug^Z=T++f~nUN%1VS>&EE&}Hy`BMSSOf_WIn9Vt4__43sPl{1QesDx!XFx=q7PL zd~@i-TX45a7#7BF?)O@3r*d$JfzS?fTZ%09Sb`vRRcTq_v@j(RGsX8!jR8_$_a3t# zdQCScT6dI74hp=rWF}ayP190$0LZ^R6kwrHC4vZtq|HmH{{6g+AX&w8d1ZWO2}Gw5 zeF;1`xO9WA$ZoMJ7#cRwHmpIfUHHMhb`!C(`nxM-X>}|44MF8v&|32RX?Q6&kmeQVm; z*%!zmu zCuMel*ML_IgisRSghnd6KHx!GIRj}L+$}olP^5(&OrbKpD07T4(Qj^JouN9N11@|H zRdh?H0;InPQXar&!HGhl8y&9x*-fg)b@I-ic4Ky*#*p7IXxw)#j zMPPVGV4c#4Z)>E6zBXYV#;76E;BWLDCpik41{R|^RS=4Iu@78s8iL=nuUL=zm3m-S zCOz>PL93mrLULnR_z34lalJs7?-aGVcum}eoD-&hiS!8{pXDI!xQ&D1vz{ToY^dSB zP;p)s9|jqdCUShdZ1u`U?^$C|&ZPX(^;suFuw1YE*<|nUZGK=?w4cgw7khsd^Csh=Dc1U_{+UzPMtkvUI`S1#nJQ`+bh}FM3J8PrY>^IVGjhu{ zy{3i#w3IJi4T5N;0KY7k^>`ULOkswW#L&{tJl=8SB*+xO&D=AET9$m>=Cdy=Ak~=` zB0`y+inRGY)p3dc_Qn;)0#p&-x26(zWo3(0e>_~M8Lm;sCD2?cVGo+~D7p1VB7CCl zuAmUe$wrsWcon~;#qDWM&?1VwhIyXYXLhDdFVPni#Ojls9#=Aqbn}RpjKl|;IE{j& ztK09(eJs8E>bu3wJB)ibz5dCLcJBXph$`Z^l9Cq0P)DidVpt39TgG7ryBf6L}#p z=OO&Pa!f2Dm*JA3Ma?8+(G`^$@U3FrXaYYSAdxSSjN&+J&`Bp zE9U0DDuU^6(cH$wVnHU^PEEy(k{zNOh&+weP-}8GHmp6X z`o$%u_EdGq5}7;ivr+Tw&RUqM#h-> ztn$-Jrj5__O>7x8Bm!@aN55ULJ{LPY1>dVGqg=O-#CE$%W0!m92}y&jY5G4j%flVOs_+EYVkS&_jHntdyYS{Po$eBKooMBg-K)wm&y6t$zGs}~u zT{U|NEbK}|zb|J*8?Z^p@iytHgjAQpu(SdXvUH6&(Q2(mh!U~9cn-mEjij8(Bskf! zC{D+5d6@!ah=;vU$dvTClnYflfK~d<(jZeDNBsqb#32%I6(xlki@Qh!$rm90gayBd z(!c*YdYEK2T*P5=B7+E_$fLb4;Dzy)z)0rU9ln1Zp(t(Z&pcciUmGK~1-^}Nehw-p1qeEV z4@tFa`9q)iE~W(eXR7W5m?p4Dp(Aqa140}pgxu2|{Z9Lf?<6CZf=+oF3VfpC^;_@X zKft9sWzb3&Jo?8Q9Go>azz4YRltg)_!%XL`@?S=+eqma^^F&k|+sada`QLyAZ z+qP}nw(UySIk)@U?LV={SbIHdPI@@)EkRpd0Rp)AazBJWDlUIrM(pQ~b*q{7tjYEr z9CiL22m6XCOnyh^3U1GQ`Dg<$Q>nWj042uR$2K`L4Gj&N7?&7}3*z_1wrrA(9Npau zoyL_`j&Uz3Ud*;i3smoMR8E&Ro=MRw+Iax5Ucl8#EJ9xHyBfqUhDb1bI;F;hO0=vr zab7!)Vw|IFMv~kgpUf$PZ2!4Sq(5M}arG5aH99f6im+E#9~xV-D~D64Yr2#Oe1O(z}kB(i0Kxd-SC z<>{O3=80tZiZhh{_|BGYpt3Z@tiFc2vMAAfADFWq7_&Fn=_B|3F+1qCzxpNPXUXrq z+qOE>l|7CrY~uT6#SC;SsGa8hGcz5Bqpk26B7qQ4ZnUtzXS^_KNmtSlmQ8$)sBScd z{twk0FVtuA6;CSfU)ox>0Hm~q%xOe`_zvQ3dQL@?hmXji7uHvb`p63v0*9GWL=Xc1 z-AhE0(QPd54+|qK9XgAq+i}I{B#56>NdjxE#Z+9^WsLAq-n5C5Zf7i6W?P;2+ce;(w_!qqR2O7NwA< zt27^xf4zQ^<{=`3VEO^^+@U;zHqdTnP$a~g1il|VqA7RMZ7)|(X+46)Yb&`f*XVXu zJkS1P+V}oE$F@GnE58i4Rd=YaE*Y~d;{0zKpm{nS~2~1RP1&YA*$LU ztXMM|QT?BV+T}5~RyjQ>N{1oMgsf)klf6a|El1+2tMqJ9>+^m}Xw$)IkDlKA?2|n! zy3aET+g0WX1AFIScfapfss8dEP`8Y4RLlp+;)d$BU4-i^_q~X|?aQ52cf~}$#pHBX znclop%Urd#Yo=`TfLD$Y$_^6yv8rm4>}!JW-u9yMLG<#iu0H%A-j-L+K$5x|GlK_=t-kM3c6Z$%45=-@N6meft|xRlO%G+sbqS?qagT~EmN zUsoHpDV}~e+A z+_s2sX^C$N{FmPMy&Dz{TXwfvu%!cPRtrLTwdI(C`*dH|8`zWJACgxeA;8-V)Ih#n zg&UkihIO3p%|5S3Jzk}(z7hS^J&*Cqnsisxns~0W5h7ShQwBafE?q~riAaoTWc?73 zhwqGh&&?Q0TYp{z6u$G60~$vgq%tW6D_yD-x@FTK^(lUC7M{;#Be8JFhr!Y4AWDvh zOX86cAmT-Gs;b+t=ikO;!^0mQ<5 z4k=qx=;hQ2NFvjX3v-1TE z+#;QVAVj%uGWb?*-X49<-FLUt*4sw zDeGDGvCnP)z&el`-%bMc^NryK?(xf5wI<>w8Z6$}`2u+=Fp39>SShtev1z1ZF!^CN z*W!oACT|GYfC{+aJl8g$!Jiy)`Rqn8}Kzg?4iq+{*zIw<7Bp0`dwY0Np`v z`FbLVbLRnUW#NOzeV+=Jkqse`p%rWZ!gII*U*~UBdL11~cfVJA64UV$R3I8Vw#9vi zb&jKhmLhR~rv}jL^pv#C`zBOk`6D)Q0YuWie+Ui;pZ{%p^=e03Ck9pU@hmgr4mkl+ zz{4r9k^zdrV2^iy`I#X_ZNMQv*VgpA&xhJFoABzF(#!E^lv*DRs98Iub%&;BBWDIK3kDYjgKnL4fsJp|>HMcr_SO(ocUzj^oRqUNG@68%_9Ka8|( zoKy&^b0rV~calPcdE{3~uW3Hc7uu0P)jk1-%eOBjAYYZPD3UkKIL{I7>>Od2&~8u_ z_5l`Ml938szAq~q)B6%UhLq0~Iq;dv8TrCp)7{TjZz`lj&Q?-rKeV}VjFZPr2XSfG zo_*V_s$A?o7_{ganwjUBf^zyVg5jQMN8<4pnGK@1&Bx0IZUKy8xtUT(kaLZjEH8Rv zEAlu{Io7i^_=fIH`(e~!Q9Ni_n>PpoNVl)dvcd3Lcv@e#y>T^Y=%G_Y^bS@K^ zCkO;6Ack0#7}BJ6bEPU%A_{fY>00?zJTeIf$B51YKl>_xPRtikhd!FJ;ACG4YP0|=t>v#7scK1N!l z<}dIEH<(lz=gwJX+t$TIKt#^;El@Gj;+!E&zH+c5oel`VkdzJN&zxy1pZXGD6ETH zg8Y#XS9@mCN~;q6B5}?T98CVJaadmdR#MJ_eXJY{36d&8k-wt+4%lW`gx>pBgH1xf z@nW369%K`Er|!N>WJ%apzbZShSoOvk{H@e0j#!H$*aYu((S&&P%WcQ$_-HtQHnRfn zgoua;5H{xatWi)*N!s?C2Zzd^l*?R&3+(V4ANKx&b>?cb#3F#+A`$3UL;I7Gtwje; zs15rF5S-nyW%cIdgHm&IcEMC2bqN9hH6dO@j!Gd&;CMyUOK3$Hv{|Uc3L4&&%a&oK z77rR#mGgsRzCb+#?os=bm6tETZUY43eU|K{RwBp3obh^*fBCOy|`Cwzonr4F@o35qz%{!cO{RLaj z_e-r^5=V?~qR1wB2Rt>uVBArXO=b@IDgAX>#1CTsjqQv;SkFW*U<0EBv!mK>`;3M+ z+^HIY#i676)QvP8b5sI5iARG6ql!4u$vsz&+LJEL&HLt@A0`Is(b8dxBgJ)+D6=y{ zhi!Eb$ByjuAxFm!nYOpVS5|JDDdK~sTMu{ae8J|SbZgZ7G|*=r;ciBjiD~7=Y2GKJ z>~}9sr8vbj6Y)fZ*B^jtEgeYl5LJ9M=Q1ER0Zn=2$&7~T?I0#Viz}+}H>Xi=cL3}j zNONHc<2*+(Kh30W`+_A|>%HCv^-WDXDFQO|%t9QIVZnU5-ri>}nBdj9i=IGB89j$7 zA49m^a>9>piN}!lvARjS&bL{N3#7{qY01hisRE3saDyIpf9DjxIwM%_@4N-dc^>@S zkqhs8zh-U*Xv^AriHDUKeh?{iFXD7iry;SjahU*h$?{DStC6u4VnWDuCX(q9aEr}E zM{M5U*r387eQ>lhHgut%m`FFK1m{DU5xlgNai(v;5RIonGwI)^wj>P?N~{x#i3B2< zt-}!H*Emz5lh;ql+M%t{YvnU$qaLdg3zQuA_@gIg?$Ll?rZuPT{Al$?%AJ_c1}Xxd zy2z7tim7N^S0{GP6=uCS_bq-W;gPjBCEFs8G!Mf#O^~m#5a?Nv*x_hqpD)6_!YsJb z#Gq>D7Vb}fBc>)Cv6)_PumemyNraF*I%P*HUGy+LS#Nuqs)GF&qKm<)Zny?zTi9m( zMOt&Ehp29LRY|^=)#CtX%9!q}N|K?V?^;Msn8I1{zmL^>ME0XPxeH_A7BtrJbMD91 zj=89+nf${BIwjxr<85cVe0HDl#o9^&oIRTybIWhcz7Mxs$h5z`=KE)1Q8EqBaI6JB zvSYkK;|U-xX5$Eb*I1-x-xZfxvpm zg;o|t{hBWG_gH_dg8&?^E|dIJ4+&88c>GDrttg2-NK~s@ip+M3-!B8waQOSDeo*p? za4=MD{8ZH5d-giHtpHBNc)ECoc6PWE%tOkWy*(rAa%aEhH-mnE|Lr%@t%%ORsF)ax z^s8uE2N<@WUY1mOh8%4d7x!U9yzgiDc^uA_f6Wc2G%0+Wal_VRub^xPX-TK^a_iqC zIyd=WPEH3`v2ll3@dFmyj?}b->F?!oQB329_6sqcZgdL_j`N~q@+BAB9I^a0PEv2% zido{ae6fgCgfgE;joIV<8QDy`DDQ_3)47K-Tz|L+WKpz7Q9Z&w=_2(UUn#hi#fe2B zeZDh%J`Z|^Hf(MppdvUZW!`rne*1tS=p*bJBTe&aCCO{dzPypqS~iN@%^s9?Wfd(P z#ii4bx}}Ij6#Rf}j&WbX|4&xk$^u5KVnQ2u3_bqmC{OxdvZ|}r_8%0OZ9XYd->9+^@%6 zKeVS=`CT;KND@~j2gPm$0_Rc(3SzWq_6+Z&+Rre>K#pjLt&(d){j`pj&UVGY9$iwB=bB10Pu@uGnu*eN#64)5ZH&XLils_@1y91m`TR zw&oznHTQm8o|@Dzt*Q#cW-80neaR-An<}|;zre6ILJ`=NuDxQ%QK$25R_@Hdi2DEZ z>6Q5tMi|CFDsYJQ^t^i$5+Dv*dh_hR*WjM#;A8M`3{t0R>icgDH`$?7u2)Vt4++}# zaQUBKR@*TU^nYDg>3GEE;B0H)e2CTCke?g!;|ml(yB4=#nvzT5b) zfF=LE(9;8j7w>&PItcGRN==T9NArFE<~^mLbp(wUi!wV`M4W23Ub~ zo}S!H!xpcI_xQYxf)558vIvai#H$^eCUmCvNzqbUK)#V%CKmMRopfP|4(w?>;${TD;=qoPpn$jAyP z^Ed(+jeuL$T_GHZwy#+X3Jd8^jPe2!XW-0*DUcR0fEz%KR&xiZ*?8PUcT!zQNEga_ z?DvW`ML0AbBj2{x)|v=3s^f`impHSBuGV6b(a%yy5BS_v+Xz* zS2fe!O0h)PhdW$fHeRSPzmfkuyEREtz;Q72@o1WiS;ocm_3Y@3}g=su(**pyvI0K5kz`*I=DFq`> zZW9Zm-ygwV)JVZJoJkidO~QDf5jy^usI#Ls5x~G+(T6o^0qy{C!{LA%;q zM$VB6It}HDg2LlPhUx=;FvsMf^!Xdv*&{cvs^gzzsj5F>ixY1l=`82>rj2@^#Ymn< zsmu^*mH68=n~kHK7^;+i=K00SU8}qnp(*g|bvon#sLrJYKhklsY^PuNFYxlYkVD{V zj2T`dZVugG8ol6$eD^*^T_dVTJyof5c$JL~wJV*&W)mqSnJ&{QQT_SlGIKk# zAmicC)sUsF;};27euD2kR3GeWOc*|$gX!AH!QWRRQB(vbrykH7&r&41Qe$1FQo#O} zN>?OAr=?R~aJ@$1^*=H;YTWpgs?g7RU#CKstp8a2?a4S)3W`<=z^2 z9cR|TZBKFiZ2<|9pUg4HLd_TQs>EUgBJ>ouC18mcUataXd5Bm}iR#CNm4zY3&@m+j z1Wh!o1vaXpMTF#vpC|?=$%cHsW1_`i7!&beL^G}^_Hpi5vH`kQ;+98qG_Y{GSXuz! zAcoF%p_(jl4v-|-@;(si;grsrK<^B_w zt_AHgRR4CzfuS%@u0-dB1!7W2w|{*pG0TQ*MqvPRC8{uLmICz9w2~~n$9)ODbwJxn z#e}h_s6kl!xYSAt3%4vN>?HC&f{Ewi!SiLfhLiLe*c_ZpL|zq;2#9!QIx=6eZ=#_G z#u3hCYG)`oNxVsuhY@?^-8V<}K;eKju|~*`zJb>Z{&Or#Uvd#H_?RFanv(KMCJJ@z z$#DN8$KtJ@nGBr?h!ssl+=j7AK8@cSC@yLU8MIvSju)HSJ-2yld?9~V#RSE^^vbCM z{zT$qFiO2DTxn+}G-mFJ&uF>HpnPvpHYXPUF|0OZH>@c~9LAG!H&o3MiOapv(6f$5eUvrV(Pq&$0HSpBNkv^UF3iw zEGLR;ZH5;F7HwjxF;6ei$Czwu zs$bDavM`xSend%NnN^&~Ev}GGQ!-ATGD!+$iTErC!*eRRCnO-k<>&&BA5rS5htK~N zwDu~uE3}VLbl}dKr$D8NwxVQER$w^)R6QR+oZEW5k1vzy=7F6Yh~;sAMfZncK+X_h zyX#ltk3^H|OA83SFH2&bX4aP_n%d6cIpeQi*P<{<9L#=^Zt>Wx(gPJAZ(RN!{;5X{ ze7rDYiQM3&Ft~+xllgn=%AS$T*A(%bp6WDrhV+kElse&w!~dQ=Gx^f|btF0cdLhW_ z;{6Z!$ahE673#e2#l469`F=SUi`=AU0L+zgKo>Y@j5@6bMfpsx=nYsb5zrn?1O-t38JZc%I*w; zn7y>v5fsI`Q6E9ULy4=O862j)O;sj4MWC0%s;z!wSr#0_MCMYBEX*y9eIcjD`7raS z{icCJ1)C3nNS7Hu`n(J{h;uuI73*XkgB!_5CtQD8*9ms?Ha)Z1zaRWW%Uw?$(w59m zUFAn@wNDtbKIdnxUpZ54m|-J!@XjXEg~gCI!~j%!Ld5oCbmEUE;%_HHM+xNB=hU&p z#gqZMqjK$DO8fPJDEX_h6&9$rP44Soqncp)yA@1R(#j>Std&YOjWU?_xI5kx^de3A zERSry?ij9DN$1z{(R`&5zW{ZRQ0HzU0YQm)H09@CRcW zAt|X|XzFaWhTkqDfD8mojz3Pvjd@>(IE0VFWJ~dhpv(*tBd<)6S!(5Pe8V809!rxD z_7B`f$D=B2Tz5-0Pn98WYd^pvp&jpZ*a=1=V$hG_7gn%f}-S}%A$5}UmJp6))|F`}-k z%{U)$+B<-XVQFKHn3D=C#Zv<*L!P%%!WW z{mWvTE#D%uFvW_pzB>wMzJ}|xoiw#?Bi;e>^MyRkZYJq2i`?43wIt@`S*Q6EOQ750 zHZdI;}-`nTC1`>{~_;kD>P6k2m3)`JOJUtl5=6gzD|(aE%?oPRduqO zdzc?-uV2nO<#$NzIkzUD4)kz`82P~`#&( zN+N3As0xXBSPno~0EUq=SK0@fQQGPm%&2i{K;ER8*h3Vh5iF?j>ClP6Z=@kGfZ{KF47pp$SF8okMmh7?n75Eb* zYxFJQ%969sp~`5?3qdd{TpzeiQ$(pFRuO}#f8V(-e$GN6SZUXhe=F>#-kcOgQ13GH zrXp9=($4DJljOQ-hpNvc4+j>00FGONU&uaCLLMQGh~MvQD)WJNS8K;G^dXd%Oj?K) zdd{kVuFrC!4_>5cT*^BywO8r>c8`D5r$EqZxoI%ob-@NynvAoamORaP3icu+`W_rv z9%QT8hnO1gA3eF{QQ%zjtLc=iDxobc;4qB9Nkf-l^l9JqI1_g9xAR0~oXOfjA5==?+hfgvU}4k0cW`<5k`8_qN^DS~AiX zLBWqUvLGYmqzSb8-f;j!+0ExL0o%#407KQS5cfyY9yrp+m?dAlyyh|ql_L?C36_{! zWdWyD>;;ydxFm_G%oO8@IB&Z8mM5M_M+DbVmZ-vCgAd*5aCEU4*F(XV@J0}U@F7F3 z^$6_I;IAPnT5`=U0GL}Hhmvcq#v?4ZAQpqFwO9;nEusJcA7goT=DNq%1ULV+6~UWC+d)TQXM^rY^UFM{4oF=NBFAf zXQgK_YiV4%U}jNx7w{kz84hTlMIz02Z%cLp&h;DYy;YvZm{IS%(-XWb>*GA?6zrUe zr~AkFo37h&Z09Zw}>evUqdd)xxqUaBqjE zD|8K`61vVZzB<4LPg$XIOI}y?C$ONk7l@0HS8#J-9DRs=22mec#&1@fb7?HwKac zdViB0PN%5At`((MXmb5RQ%juMQ>-*9QPB(S>V zB@j99a`vHZR1UqO?1sTr6B)C?X~2x+sx!Lqa*Zv9aH`C9x2!u`=@p^->Twhslj+^3 z@pvfCq&B0kyIj}SNO+yOs`NgGnkh%xl+ruFsl#Oh;ImT04y2+mm&s-$+e=Z;F&l?FuWG46tUYV@ z&_;@kqAtVH#ZG5Ai_GV!JcmjbZNmNG9o z4*l0(I?}HM9i0>2P`U@5JLzKk5f1_geU|lQ{Iq5qay3k;>Bye$J zhQ*osl=X8o3o)F92$96OrU8pMVk)%j(92->FEht}I}N1Q^IfxTrnR1slHPH$KH*AA z@xi9;=6b5+pQeCXqTYUCBK22>An{2x{6xZI&kkz1qI1U`5s{uBcGZEesWFrAMR)g` zDVKr9u^I%KpJ6n5B zN~@;G5k($2JCoB>T&sTOA6a?F{=^hMKK3BGJy7}6M2eow0%P(9VDg9Jd>rj`=gBqN z3VJKvF7b_FX{x~-yYL8HZ z%fe4mEhf`rbvyft69ojMKfR1ibn+cB$Cf(Gb0`Blp=D@&WoVmT@2%tw6cp>t+i^>1 z?CLuw`ga zP480OA48MZKqOfsy17T&aJt(}F>$zO*%C12jUWNdN{n=axDE6q_1jD7Vwz4zcjg?J z<~kNT(RKo!5#*3eQgt*pD7=20553AoNUB$eWa;8pXeD54u99D;ZEwp^=(^#Hr zQBSKT%j$JbrJ42eP)541WQwFpjifSaBIsiHEUL4Dh0Ha6@#No&hzJUd35(P*loh+~YbiuPJbZNga5)7^q9q13c=a5e7rsFfzKZvUx@#-}A z?H>d+%xOa(&tH`q{CR419tiJ`%Q_~i=uPIigFKbWk_=k2!!=Rp%Zioum^ve`2P*xA zLm-Otr2NK-L!a1T#G7kEA5`fwR$r8e+^9jnhTq`UP&?*;hZjS^TefeK`cM* zDas9$H;%MI;8n-MHLWzA@3yRK)xQlfD)LSutQ6&?0N&giFyWgH!8(EQp~DT>4#4d9 zUn09)5QKyR{F(Z@~xIF#Gpho@G>_XjZFT4!N4tDDcxBQUr>vG)jntj>2qpG{9GTHukYuJoXXQyq) z*6g0|3kc5Mg>zkRcWSqxHxzZ4eCVa!(o4zKM_{G7=l5YA<>w##uC`aV)qUpcgV?!+ zZN=}Xzzd{VI`j@~Znpnu*KT@_Xd%epL$w1&uJz}_HVg$F#d zyVHNc#B=lF0~>u7nsa&6yM_t35X;=&8|8UZ(4cx%le zeEoTsz55J&Q?=3Sy&qx*1zL3$ox2Z|&tI~(hc#@^4EJ}a+^8A-w-XkiWHLerL zk2r~eLx<|Jjm~Zd|DI8EU?r~|mfYAQnND4^Ux30zB_6Jv&+#nd9ta6f66hBq+g&wS ztSfKTJ_9DcMr)`+lij{_nDqFLyu=?^Ya2uu)+jf>Ahq})>9xDo&Cd`yd7^K?V16QgL=#x$xAB{nWcqVO( zx~y@JD!=3rY?ec;2xD3pkbfQ74_yzm{oBV&GWKW}j*k zABB-Z0b#uygJw_a!-bu(NtCCgpEb~QO552D3u+9E5KrJhCmW1&@zSCGUg(7;=ol;_>K%{rxitc4ob!h zj;D#PGES+QXU$~%mUbi_k$r$H+@{oKQZzI~)=QU&C)c4FMrCnHFhCjy8YTjAH-uQ! zE{EEPT0e9P?~-@YGnvJ8<{HH6V6q9;7?ETt0HSOSx>r_%NI%)Yb~Lo+nt<~~T|Bb( z#Z49(cRz17U<}7mJFg`I=3gZLL17+H&=H#& zx|*CIXI<}E{)l%677@J!uJ3K1^&%dxnElw}2r#SPVn*)|lV0$;5bnl%UPtaXXWmN> z+f5!5SI;qZpixOQAdLjXut^?l{<$gW!lSG{letw04AHm=-=!mlHZ?5Z-9i3nCJ2wB z4e2H?HqYQ%RAf9RhIGJ^2+fs3g!z!3Cjtr9_}+DR--BN+ltRIvp%q^au73-h0I$w1 zL?1n!>vYv~j-%7L2-mT=%ta5kS_mKTN2EP5=(^`9tf-zvT}rr7kez#H?5$~^kG%f% zVxP!Y#dR7z&wpN~p)@PO#4uPa)ciK^K&PbV@>=kC>}1Ur;HO(B9UZ-7rQGm4I+H~^ zWhqX4LlR(60#QK9$LNYzMd*3oz3gO^+@(v0zHk5BAeLp#arRbfeizK15AYPsK8W$sajIeHE?&T@9t821_3O1|*E&L2 z3Y=YynZj_YZLEhA<`Nx{NYBE)vEp^V)iA*EkE3vI>P+X-JZCq^lfBqW`pZ5Ma2_ERFt{Rxah?Qu*Xgw81dI_qE$y@}tL`oZ1~T#<)ckRc%l{LJvCxX&W@?c;3jEUk*TAPmGNlG;|Tjef^blX8Szh-yZoPQACmF%WrV=_ArJLBGgrfzZ00Gx_U zN%nC!8S6|PS`2Plyypk46Gyv(;3Hk(J;_n=54UN3G5wc3xR;s=2Z2mw^JFNaeHkeZ zB}wyFOr|pEC1ml7Yv~GRliqRNVN;^)& zya*RT;%ZfHnu8d{=$D?V&+9{FE0YC`w9}HKF;ckM!`??jwT}>zXLwymq(h34B4IJL zdYmbl`>g$vt|Hx+3Lm4UtDYh{ITNF-4goyah1ii_R|B_gkI7+L-2Y6MH-xk1&1>kbOZa63If*ONOu+JlRMd}| zB+sRa^4R{w{bBcT*f#{5OG7+Eu_S8N$)i;{x%|Un) zw~^*Uhcm7?U+1oAY!K>KZUuwk*HplWjU#4D@ph$iDLBP^l}Y3${~n^sv)7BEIXR)U zPnSy2q-|pRJ}>ZH305vF69u`lDKx-%8_P@zf>78Len7oyJqY5ds`nL-E%({1+!@6Y zZXh&(f*3T(U480M$s9SvviQd}Nj498=yi)hQ*@%FT!!3!`PxzcYW<1A{eeQnjh%TnCs&6Q@T`86Hx%u^oFR{NFb-G z%naKC?IjCYx_o#je`LQJMtYFbzW6;W?cj%{*p!_g<$?*{vK80__it4(&~N^qrFFPR zxIaNSLhF>i1bBYAysRvVY`pl(<;X$t?#6q)msR;`Pa=UU%!74iSqzgL;K`;JsRwFl z84u0@>BR10@iTsYNytTqz^9^0ASF;f zI$pDI5bzRb&}CqDr3P{!f>al?;rw!GJcL*tr3Mi+1wA zn8SSj-%u)@T*4oe=%_D%f=EXw(APiok=C}|*1AAK45QEc?med5OMO!B5kzw9qNZk} z*;;-4hmVWvK1 zJZ6~5t}`E>puhIxQDLutPy=rI7d~ty%cc`LbQ_$gqB8JG%s%iq{cOx}nTs>aof1Jw|BOTVHEEep;veWr>{bcrUI(>aJ z0}$jxZKOEk)H7?U+XI8|e=E-;;AJPrIJRq&O8M>-FP2@q0}Dw?B1=M~2TMITpMq(H zPw6^O;0RJ!yENk6h0{<=TMe%V2=KYWn5u0{mSGhBqGBx473X+fyZyTKcP|uAc)xN0 z>=k?>dGf!2=4d|HUXlti2M8u0}8`XqaJ z`WuTCaVOHh8BduI!~Etas-A<SO&atG$@ioHjChtPh-*2r3n6>nD@-bWs^Ob+$1lHhGI1KokwMgqx!?SE&U zG;cIBs?^Fxpk9h!bb-Bg$~onY#K~#P!59=so&64tP8Ms0*6@j^>#F0HM(XC(0QQ4iM`)>Aa=~gAEQyHer|Q-n7My-&bB2GQ z!EbQj2sTe{-(EM7a^8Xq)JA{O(3lCzQwC;!4iY&hoBr&qBGNaK$Voxr?Tp2XGW0E|+>QaGORE9e7at3yr_0xg${D6(#vt`F zCa+URhnbh@Z`ybWX-?xN3vJlef`{Q#mKr9gXymbjZ>X&*(hidAn_s6rKn6cfD1EAM zCnsTBylsWb+92?bVVudsRC)@7AhWgOBEaezxmTk@5;cJcZLt^^=7BSE3U}QDYA5OK zJfXypP6JgiBPeGcQA|}^0g$eMxjkZldlR+@AM52zF562HcSt9mA2Z*fL*;dXiOlvP zG{Pqef-i=$J2s?wa3=CB1k45#!gYpTujT&*?n#sy`UUW-?Q$WVKpoZZ`11lgOSkbK z*q7PEZXrC|hp3(d+J0%3Pg_RWR`Am2(8HQFi&(Oq;)WH;9ByuG7%`CRQl)O`4CSzt z@V?S@_8B+0OGzkYW*mCBz9?6xFJ-vua;~f2O$R@DVOc}A8FY&80OGM#HuwxvO#Q-* zlKtNhQ2W=EMi7b+H_fOp90s*xH3By*K#6z`+yuXaYXsBjn7-%Z;V}9&VD(N&dD*)k zSNWcs1CRH zx7HFAtR<+@@$AwniV|W{*W}$Ndl>u_-m3Mr`x~mDiO6NRqE5F1WoZM1&ff@=R{8`n zr`9SDB_7+TBLyf4XzQ1cAyGF#p05g={h|Qd- z2OFa(6aHknXz7{>vP#{dQX#jEt~p>@_9aqX#k{pk zncCkcTi)m0CP_?4l3XI;MqH_(>>s#WX%0Jq_H(paz)JI6bQ4!q_Dmn4InEzP3|1=g zE;*4v*T!G^rfhW#fhf1xBradBl9Us!P{_8FRUEfvdJZ~Uu*Im5jfl72ODi}pNY@JC zNlMwb6#IMRvK0GbLA}T++ovq_PR9z;aC(U9`!&hbp7`C(ik|HKxw{w};HMlrc)^X+=|`CW;m|awt*61Cprae*YWFsTL=WGHcV+ z@PXl>3N49)GnU;gAn!ltQU$nW`P`T;So6lHv2R2?HuEm)aYm>3TE3m9R3jJ36`dURpY5B2hyu~hE>W7 zJ71q2x>}`gt_k5zB{^M_Q*K;$5i7?IoC~DS1|kpO*PnMC>? z_IMf01xp4tzK+>Q8ao&3LXMp8!IkX;j_?^PMWp1vSCRKE6Zk8a?cA`WVZ3bTjJt`X zVZQHhO+qP}nPHy%-5BKGqwtmEFL=d>;BFtHgWF&ZHL~9%;5hMlPE1UAz%5GO9bKibM^EUD#a-XLH>csftL-Ixw z6uj*D@I|V39US1a8x>!OdMwj$5cqAHB=~16%2V66RLTHM__<`V(j63(zeB^_^KV2D zE8~P5psZEnRK+RSU{JJ?BkLGLQy!iPqfxWyN$1=|J+wDHByffgC8U=adOccqp?F>@YUCXaVsWspM z4!jfG1=DG|=)S|;6^13jPJhzJn-55Z{NsR>PnnXH{#dPK&I2d!TeY#+v}9_#opInh zy$VI)h>IQ+U%pQ5-E!;Tx8tRS9L03Tld}rR*Xqb7h8n!ffqBC5$}1wC-+f?!quNfT55XQ2~RzcT$!uEB!lX)D%d$zdgw9H5Hf+ zReyrA$sFOlG@oC{m?DTFPin!1X86OZA$Z-i2ON*>s zW&TM?Zs9jw_Q(g2TVKOAJBM!eLIV)ZDxW=Ik=@Ef5s93meL{*_EO6G*_Kvp>yE9l+ zhofRRGNppp@Qr!ZPKHo?{Z<@90o z{~;D>-$`iV-|q|>_&+OQNBTPYI=NOUSK6FgGC#zrWLYEXI1)t)PAB@_`Y9EX{}~}J z|L^hRrRV$3@B3F`sD&3!YdO+>8Sd5gu5B%(1Km-5=`vJ)S-TJSZ!PJiuck6Rk0v?K z@?E6v6Gr1{Na&3Rd5JUeac%20jo0$R9Q-lGS{N(vV;1?yd?|X0zo+!PJ-4d=IZ?%? z&QXGIZBJf#g>~ti$4k2;plbZ5$;tL3X~G%%v{TkjlxMTO>Sg3LgBOnGOL=cYdF8p5 zIu?Ph?xJ<=w|n0MS5aOW;k4A~<>yej4um6|U4ystJ^pcW%~>!rcslhf4_sXd5n-6U z)8phbL&B>s!*mx$yB?l}Yjk_9yAj;k(XF9teCJiS+*U8Niue^K#yRKS(3-V5cyk-C zws^6K!)q_ZKedv~JPy@cedLk17%e^8Dz)a)jUgP*Yqf1H>mk6sM0a;%RV=xfm~ysc zopR88U{L>akM`%q5iHS8#-qYNWz#2|O@xlPbKo^>2_MKu@WSc$zJtI zr-?;ZZkR@Z5}&Xkd?b{jP-WI;r_H1g&to8higD^*fvIIuv|+)0kG9uVOv?6QVu{US8+JgP4o0GKyd1E(9i z)n||Pu`W6dwOeO6NOQNq|0Boz!Sfbj)h$}MR_eTY4eHyM?cX&x>L7;8>s)T^RFh?vC?)Q#>F%9RJgg}zMfonK8Z;4Yq!G2sH$vqDem67h^_jfn6d zvUso#gR#Jx^KfP>0^a`jH$b67b5FZC`j|0D?Y(iME6%=7`$7syfWu5JQ5R57y!qa zGuNN8^F5h5;)2pB(QS(>+LVk2kRD92)hyZLc$9dSTt_s5<2h-g;}&}5{xz+=EEdhm zINk-(oYT$#kp05zeVn$P`o60HK4G*o^XLS(bn-+)u=-(k1wJbrc zBo#)3SRNU%nfr0D)WJ1xJ}a-YWYbs~^Lkwjz8q`=EYG@8gzF-v92k&%fz*h3g#wNf z17}YmFqrkt@|mC@2Oy6Lvgrn)IgnP%_k|>G+m4xzL_>og$sJmSzxlh2eVmyRj+V;X z=O^?Jk0AbVDjV~pBY}ADq)hfTT5dGKJtG6(xyWGGNQ3%=@tp-Yqf>O1G;yeCkZy$q z+ed95M}#&$%16r&=Noftf{dmQit6m1$6ehq7J&YYD#oQGIBq9Z8c@fP!L;AUq8#zH z$sE?ND28(1!B9DE&5fo`&h@{UFGCVj1H}7kpapvmo98YSYn3P_YjJ7Yrg;m4hjo5V zE#XT(R0t16b}RYgKy<4XQ4Qn_QFpk}Ah1=K)NPu6Zglp6nl6X7y{$v(58q5Cjb>!d z$GI%RC4S(T{o%WCO+F&cLa0l3*|Td$nDZtH$$|W%xA+g`LDAZ>t;uk+Gj=jUC=7gn zN{9D}xO>QGaaRr2FG4J8$6hk^Oe1L`jG^oM(EWXReE}l8DN8FUMa&LZELAkce`m|k zZNxS$S|aeNkZ5~pFj0Yj?jFRQ0TFx8zCVYe*zrU<;mb*1pY6)nI7!lQ)|Ct7)MJ<{Mf{T5ZIwTR94mKLM;b zA1@@+O3NKBHpMR2pXK1F8RVH_0^WZyDJ8CV2NtThh+2v@5a?3+lS}im5)j=W5qRA zRkAYJMtgT{WJuR>;6fH-OT}E%6XuQ!%DWs#4j>CpBiUJ8`_hR2&WXTk->z8bPeau( z-bhW|SxK;-F51X*rH7QObG2dY0YM^&+`bOR%S1M&q3(?p=AiW0NU$JHe$kOuCD<1i#2WTBl9Dj@>1W;3I)-%*(}VTcXuUkLDh1zp zINx$+qdoRyE!c0wB-|(^V`|KjG=n4*5oc0<(gvWiNSh-z=#bgH=Chg8n>-RCsO?8g z4vkv1b}K1w8g%|37;0Buc@1)JJ(niPh|7{OJMZ-z+)&=+9aV^ zm*$X|AfaSo41wKUOBa0ES9-4pH9&LVzUQAftnLoB{F>?Fsz!ela$9NO2(#uzNcb^F z3_WZ1dkgQn(l%G0q&N24AJxyN|FR!nvjeRcmGwCerNF#ZT*H~g@ta*U{!POI?F>6o zUosVJM#nXyLu#>WaM1++kb)j2vHrG0Y>+(0LgoQK(ags)d}O&~Pvo>upe4>e1;WCk z+B7`Qj5?1MAw7gk>tfA;^`3n}w@4ET_iPC*zD)nrp9>PIuGDD&Uufc@NcWBC`BYy<-xHSOmrSaHs=vH%}a^j<|}X3FqYlv0W^^F zMN3OxOKLrK7)RZ1WaX?`(LY5i>%G>@hJ44d>r0Oz^+JfjVE2>TAe+$YvA+zrs{5?m zie|+J)c`{je$+TG*EBhMOZJ}>>|J}k8{H9JukyRG2EdzCtv)JQ-nA;Wi8IX^r{v#Z zWjIe~qx`7?ckLlM-T~&DM&? zhvIa*)RMVXPUqNMjuv8p60{2Hk2aXKE7l|LE(D2VR~$c&3vn50ZuSZ~hTP6+lRfF0 zG9w*(rDVkmws%q8-%J7`n-tOE+pmGoftBfg7SR%}-UnvOqOe>Gb;k%EEyRibs+A z+SQWVtu?oU)2DgtSAjM}=~Y&Ua4X#oDrV}v$>F>?c|+(Xv!+Od8j7qUU}oE`zcQ5g z0klJPeW=O27}CQWR{jBOaY}6Mg&N$n!t3@o2 zY_ZxNRUEGem_SxtZse7Tu1sF2G*(zlbHTnu4 z9`-}f7#TaxsU3s@8(n`M@+=lhT2s3dM%hL(Aa zC{VY|^p7g-wLQE)-dSKgKWUgO2Pq0p4CBc<_P|bLl8hBOopB z|7#CppHvDaT4|ijX?GBZpsw)an4!?U?{pw)`0c=&@pYr~)0+E}q{Tvqy5Zhh)%=nF zF3&Plt?5n0rmW*Rc(ajzo^n*JIT!MAuc5rCwXry%uY9%Mjr;C4H(Tj|`S;N8z!< zTz$c9{HPuW>$0N^rIFwI`wItYfx!zrJC(GdI>q&{4RqPXiQjyb9 zozEbV!!17py%oVuEX7u+n4D2>{hsF8v{H*{31HT4$FXQ7)Kj6SNwP9q`FXs|yf*r2B>SPXRQ;M2Rai`CYI^5YnX;pO z&N`R!{VoZ{+rO_o0=YMrVp~1#l8)BVE)hJGANxD+($i2YeS#AJa$`}CZ8(1cRz5!)RD_ax{f2sV9-stO6PQ$CE)1Yc&0n18_TPgs*%@nm4e=@iwNY|j(#5v}ooy(20i;gQ_N6K;Q+X*NL|W%G4d^TE=(wt>=c+>K+1GHP`w< z-rGx+U_!_+^x6U!jHh*|2~xb9I|gq%YQCyK7o7V6;D6W;zDu>TQITC%RlT6NzTkR? z`;--A)hLzHPWA%5lpH5N(;=saSeS~9K!m02ag zk?~C@P?<8uIcU>m%E0qO6j(DjtG1e^kzfZU_A%Z9;UDv5cSTn5>f4h=*txlJ7K%_q z&+s_S3n^l~=4rXQ*JuhMeMYW_(7_OD5$N{$5J)(flQm(M>KpS`9n<5z>pNw5G-)~$ zzX)=RSbcz4F0*$r0uWRJM>X)Q?JjYYV1v^WoZvTj5nL$6^{GY(E8vc2zTrSp)9vfS z!VJ?Dt13YHGggetwTs_XVgxvUao8;g0xP$k>?Q()X0-vwAgcJOEqN^=DLeR#4zMqq zI+jl*8}J(sl0I@UWe8pErBq7xPQ4&do?4EQ<>KOq#IC97=@;QfD?tAkGevn(6YF>sItZ(+bN++s@K@uj2kkpqVxB=GwMLBcJ*cg zOZV&N{>f$E7|0TinQ0f)2l(rGMC7Y|zP1qdm-sXaBn>m%02DsV%HI?{V-enAhbzo^ z!q74?RFAlqrGQ1&zLqn23jUN^91?uTOPf3gE-^YxJlm&Z)I+n_VJxmXx0h-fU8}C7 zY*s z!mZ$IG?eti957a61(dE(ZEXK=-qThgt!%<L-Sj44qkjsqt(l(JL?%&9 zLJ50?PX3SGB=U5GY-S~i=R&^%Uo`+AC=qsKEeSxB&-krse{>2&2@RgaQ8~C|_QytC zq8uANMt@SVRP9ZK&o_|25$|O z%~+Bj`FK5tL~<4U{H{X)wTr%rPEcW9ES&W#hbd{Cxe2ju%w>!{wP|D-Z;_ywtOFp; z!}h;=snBc@ZVF%%U*|NYNS*=2zpsIO1L{(Y+|#49wqs8Hzn!1*()}@n6CvbQwQ_#C zC?%&7ZK?{!=~im}jx#ry?VrQbD5o#x-c0a0^io zC(5Ws0N*4dM1GhSt+gH0qqJA;eF)q@{0f$INb{CmN=15;YciwOhOUt?o+5D2ZuL;= zf2|G%><3ej!JS$lGa4XXqc3J?BhwR26@v}vL`Kuf9~W_!6A}p&7mB|s@+CPMu!Li$ z(DuXnVZs~9Ri-?yb?pl0hO*zTm-5@i?{gsT?Ld-JNB9O(!1ei#&|h%q!!YzU=&=Sr z_z!rB9tZQF%wNG<`kq=xsIj2HRvmLBNUUPYUIt+i92z%%&YqyVcp$OTrhc;tq)}PX zKs{|cRWwrAtz6X0_3Lnk#|>H{`B&ESo@Podg9enEpb{8BoS+0A4=PrZk+ zJG2dguLXNwPXYicx?4;>x#D{eMb_0ArD`24OdQ+Hg@1{P^w2$gbf!#Cfm953M#=ze z5uAFUQdQR?b`}afW6OKrm*9czwd_NNy5;0Qx48Q(?-X~u<@!IG@d*##5=nMrob0y& zi^&Jk=o%4l7^gAIwB(I!w19>gwr(qz390T*G!lTiLs!l&8V-2CoZ`ZVAz~|FG(tdr z@}qx8t7V?4<;4Q17(5sxuZWiI1~p(65iud&YRyDb?^$z&=hf^T4pU^IL2*eqDd};i zVpl&slUW3Tv_5&ilZj%2N7{&aDn%nWIeYQx)+9@&IG^APla(BdsNqIys2QBFxdo^7 zlPx+#vW_JOhQk10GhNu`@P+?14&&Wqaj5x6UwxMqpB9oT?Z=Q5>Nf}bhb($!&*C+t zLAO%c(y!-;52At+8TU zVv7G2@vk2vR)WorwBi411)3z(BX41*CMwq(du+vd{ar>plZ10YJWWSTG##Tj(zG2> z6bUazjkTPxBN+&l+UGU%aAc#*NglH3Nliicd2i#q#75PfiwLogBy~eT5(@O(6_d0g zx=@3&ZT~8khDAOA?~rf&31fq)3^MYNY>*gSr^rizOWUPt<#XRD%Y~!J~QAm4|)P?G{QKAC7zOXwyEsrjS_K_Cr&Xz5++RHHKlw5v(-QDWc4k*#hwtv<)i8d4k!vkDEH2p3{o=;rS}<&zO5 zNDN3@)#N>;7j#MoE><>JTwsFNBd)N_BtQm{>Q3WT4JF>NnWs26cd6Ii+^npgSQzLC zQR!j*F_*bi8*-9#c$%4HKvoThcTJM!So)eztFcxNh3TWr0boJ!)xe7ng%IYwc-PFa z*D%=X>^~XZIquz9u4GoBHKV#W9Ausi`-*gKtFSTa1!G3VhRT zJk4&Y^zyvzlUX0$3A*!m_7jD_x@Gk_i}9mv(P7Z%egK^Y7vg_&PKNN6**!hMVSH8> zFQ_w%b0M&Pl>=@!LJCsNJf0)HIDT(|;_FyQRCUu);=!Vj4CRjeIa!D$96d=pL@yGA zH|!8iV94*;4FT@pd}3<2_^kC`0qT_vd=&u}9V8&3d}(oZg4(>4@kVYcn&%gX!C6>` z<(ELTt)nd7#n@qB_o#R+?EK+fi^?r5##j{9fZ{)1_)?shHt~j@;Fhtse zZzD7Si?cIHg&)qMw`|6mK!HGyuL86lvh0sHE94zzkoG0p8Tt-_3D^vG^kd7Ex>?&y ztM$CAbWGYUr6XGcCGD@Ed!MfRGP#TDXeBM3 z9fcpS-MgE55ek7Psmy#V3eC9;R>D(GJni3o@?6QDv`SSw2Zp=WE`x&Z=>f+Fe6U|k#Q%nDEeWv`Hh9d)%B#Y6D;)cF;8ZW z+3OAq_f>VWHimw)Hqshzlgo5RNpcx==9%HTQDjnMu$^sgXH6C^&a_vF>GKZjgLZn4 zreMsfa|!kBhw*2u@^77bh2|Wjj4d?P}+%NZvfjo)(dg!wW&XvCV8U2lF0X@6r{ zs;vM?XnNr*VXLKhIDS9s>XUhYW`Ji`W442fGwgl;0R!$Roi~uJr({Ej-c+H&#oyF7 z%el^fp);=YqFxH<=Iu%g$}mN|9{tkDAC#IFLj09k9d++wIp2_4+0OzcYFtb3FuZL+cuX%)K6hyMjlX~Ay5%JR< zDgW6~Db9l3Zme_2R&ze|?6lrF92`b5TH6=y8@T6O=zX2lvj=-t8Yauxy_{MZlo(1i zVie#L?PclR^4UPCxhGzPKt0$^iS5mKYCgJuUITQT&^qvNc50bD1eEU2{{bqj#~;|B z`!5@|S__Dh%Z`$eGTa39e~oKjxNnHUKK)w$@VM-nrO$`M5*Sri!V17ly&p+_b9z{O zYo0_>3a9s)S274JZ9rb4j+K--C=fO_w(jdpLPEmN*99KPS4obF!B&${-Q}+O=eGsHJ@MaZ8Kgx?JB?hz@z8T1ByoNl4yW zvN>e?BS@RFWytva!Wwh+@%*No=J|+)CztJ}-8d^dJgcDrI_7k#qnre4JR3Td_qOiz zsC(TQl=hOY<(&`DEykI8e5pL+^<|*Za=Z?*XST|_hz)P_(@O_Nh?C*wB|C;=2eLa? zr=}u)``^*zfJKg~G` zHbwW58S%VDq{?wZ%RRbA4aXK!Dgj zvQs%Az6S-5hu-o}o0U`=Nl__BMvNgYFrED&k-HU1ZIx$a-)&4eyFju>SH+LEwO+JN zn|}r5Sdhc~dbI7*4EOE=FcmcAkXeo}QIENJ9y`RXhFJ!%=TW_NemhE8th@RmR%J5GW$| zTk6|FJ#}+~y&?tGv9{Gpm3y!YOhr*u{sd7gB;FxShmZ~jNE|eypMyFcGbmH{atytK z_h4qZu$wX*^4GLZ@k1Z?-w`vG=uYa-*qWA$y^0V@>N+tElwcN#Iq$4=Zc)NV^9j61 z(9zZj917OrIj5!qIkYJCj~)ewIvRtVqv1_T+yOD-S9Mkoo3n@xpn&r&MAUZW3~LFc zK~C1b6^zgupwijgCNC|+cu2CV(!=URbZ`gnFt>14RB)|Kp)hUC8WfpLfP5pn-l%j)&U|4ESeK3g4DgvoAsb}Y(0kg~?TFL&ar$DLu zaleSpNc&w_GNb=Gs9*}UoR?$YdDXs(8Tr{i6U+U1c!^sBd;gCSDu8D+zT;0t2;|OXw;ke zAM4M-X<)zTlz4-J0gPLvj3x$8p%zNy=ErdAS;%Jxd>{bAOzR#so=L)x!eJf`k`oFs zg(8Z^c&^{?|7wsvC{DW#y$6^f|Db3fhfZO?v9{5!RV-HH=jn7U4kLNMbp%2mUZkTS zM7UN&Wgp%Y8jBIK>ghrEdVEz*FYTB$a@|&u;~lFzTmSJNIlubtGf@PXsXr}qBWR8 zXhO1tte6bfTDs6KEy*wqJt9$VN{Ml!_5+shQ7r}!#Uyeu24!!Pe^4ZZKf^WoAe3cr zFlgkq3UM zz*-Z!xW_1%jnF^W#L$8BI|Zuw!9Z6&VTcv0hK^k|YZY92+v(pp6fjcq$4npCRkF1| zHB_LQI7L}ph5admxMsolU`-wYkXindOFE|O{ zC8TW)@q#C5VrDJgS5p;{$IGPKO!B>>O@L^eZE4r)py2GW;EXot&43_8_a?9;eAu!? zofw*=F_h4K;0LPdKVq|mI~foJqYNgpgS*2#anq&z_=?;?OtqE;hs{^n+uPd`C_ocugF+z-w{9fjJ|y&z+z14~EplPM_Zyf=W;$ zF{gO8lZOtZGo;}I=y-f2R(0d> zOQ%SWcUd({9gP&fBkb`|zWT@WB)o&i^Pi; zR^q@XBH@$U3Yq(qq4*2FklsfRI4?N*+dF}3cqb+rCC&bXnQQCO(keumgk;AzsNT(V zS{%0NiG->2)0`NJF2*DR6(Zk)c}P11@#i0jPWDUhH3wqPPr95gQeq+#G!<4clOngX zC2p?nMPOQa9sL=)fLLadqd$f0EV58YM06bebpDevzAOQ)0TE(wXu6Y7)EW2A>aw*G=7pCjs8th{bmj@D!!iLdGnG?vr zDJs9PYq-BjWm<-Kfytj#smwac09}6X9B~^fICJg$i$QS}E53=?$D$D$EzEw|4%YgF zAtD?V1@mu4QdQw|CACjS4gsP?y07I(5b$u#ExX3S0j*~Hs?zN4#UGNmx{x{}=?NMS z3BS3wiW6;e^qAU-{Vh)Su%qz-+-d=D)Pr_1<|g-|03yVa{?v_!sB#hpsMqTZn7ZRu zbrU+49GgA#9U=b}Iw%DM!GCPlX!cj)2ILngG66i`{yA>~M-9F9bTv;4syqCAA6vNI z3WH{^CuqA$XjdK%1jjU2)7TD&puv-g-AJv)-r6INRy^eXf(Q`XTxZ_;+Upf=MS0*k z5o7rqdL}Je;^wm71RO9#Exl?t4~%0bz5oN%ysu_t_)cuX`^8*$f1(5A zRsMpOBU{6XTQVo@d3O|PCgxCle^y;JZ%DYD36MRY1if+ZXmj>pV3Qxf|z#O z)KBt`-GP+VgkGX<=aVGy4Mad_aOsw?AA%5XJfexG>e`?XDmEh=s+(FJ@;T1i1DaFJ z&E2_`%pZG@la1XWqoWQ1|FFB`JIC?=AyMZ@U!>}XKMPUB}=uGFGv5EUVyrba6B*DLml|q%}_7jsM#;L|JM(K?-=I>8{tA$b4E7Jn;YFdd+vFsAHU0-~G^@j=x;W&I-TU&sqotbQ& zkB=6gFM%INtFB*-$W^dA_GW)eH>S;&>qS;UiSj}DQGLcgcUI1isH34qot3h zN55t0sc7$!owqLQC6=dCFjbm%3-`~72ib*Co)TwF(tj;g8+8Q><<6;GYq>qT7MOU> z$4uqL7HmRY_thE?7c*ueU7Z%rt|us8$LF|cazwIcHVgKZ9nQ8bOU6PQ$`jpKpe~fj zpj=aL*DySHSEuej4)+7%pmki`V^>o_$73j;LOtahQ3YDB+NzD|tSc8oLFPa@fvDXD zMm*Z5U&iL;+g24vT1|dfU7UpLdD*X|9bgN0pcxMi{!INl6?!f`I3066qc-nx-dzvY z7aFXtsNbE}*{)EGubJ5mWmZjl{5-C!VzlTsW_~{F^>#~HQTpNVR>L1ILpef9wh2G? z-=xA0Evdmcc>Jhu&gvh2di(#;Gb%E2>)We;#poh*^+nLQ*Sa4Ex9{PL(m`LxMRxns zfwwp`{M0^8q|3;Ob(~3IV^6BUadFySyNhZybvi)__Eq_ejV$yuW_m7s%<$`RMSA&l z9K!tUZmenW`Zq(Fg!*VX4fui-y4y%e(NPr+uJ+Vwe#gf*Uj#Fxj;JH~kCsBjLWUUY z%7oN;U4v-y_t3N(ADMTnT#0$5)29VkA6mqr1q=9*P`x>g0YnS>IoF_ZK-lS69HjxRvxZV9L36Fp4@XfLcO%yCMjBi@ zZWLpB9mK6)h8#;K=kxV*y&w&(AnwaKgoOo>h*A(@9NR%wycjg6_}WLvN>%OEBx^9^ z+mpb`Lt~@Q>{?H(Og+@{gNJk{I>ZVaeJ-v_!V~Jao7fxY zdYyaJSl(f>p?(JXwk>%)qjn`RC;%L&UYgf0###&`fnP{5@-1~{XQA0(CvVG6so7+# zzvKW`-Zfi8v=rPl3^K)W02K#2rsrMyDJtTd(d8Z+X})5w0>YmW5QRw*E2{PrsKX#A zdY|AM73J&%U$lyoZ0;eRY1z?&$NNz>V^iIkQD}i*HdGk#Bd0o`{JQJd_v5-u07}$; zPo7FgPmd-1O8MR8?#{})Vu?icVlaGY=8H&>MZyA{3+sxp2;7cVEMoT}{0%HI3hz+S zkfzyb8&Ktnn%O{aqpNvWsajTR`fh-yVy?W~^9x>UdyzGSMU@ba###aaK zwDc4K++?~WA}l|of1G2SDD{iT1~Q=Kk|jSIdJ9Xlg&ZJ~I!FS4VSN!?Intxqv1nHv zDD}v#!P9|)K>BFo(wu)4g0-WYM7NI29cR{pFw6j9>zZeq@Js{K;PYC*84}yQNts4{ zbYms^9MykK8Pyix#+J9BW&J0V^m0pWb})yCjKXKn*lzUSgK~ip4aKjdgk2|8*;VSe z28J_l+40{S@vYgulI#Qp-Von!%rBERsr6=AMMh-WltQ0~A^c28GxoPby@Dzj{J}r};{-a6niYQA?A*P17V5->w|rQKfbCF4KjU zmi@pv=KS$4ScO7mV*x&=QS_bo|7 zTkI1CBiN}JZ0d}2e*rF*#=}dLSJ?`XF%&5;e^+n1%YNlG?6mCF7fLuHw z)!0|v37AU(w@`JITqf5GW$VMH?sLe>lcjhZuY^YqRZPXCjsPz=@StL#aW_3@tiY_( zs*~6rG2dP}c_a)zI#D(Wuv&VcSXzoQ!)yCFUthhBJ;g38o@3=&v`=9A!fXf%fvBA< z8TsFZ8zj+NCQBeh5^j(kVP=1>h^8crmBayYM3CimPVz!8+#j%zzwpUjD5Y{^*l*Y% zA3>`%o$yNkAPqa*ti|0S)(l6?-3S=R`P|c*Zg@=(_Za?~55u!5nA|@~bcx^Po&fVs zF6;hHb$OEFT3uH%)+0PnOxAPZ4dh-hnB(&SF5`+=Nz@Xqxmze+m+yd}4JS{z?W~iF zJkP3-odYhmPicxI6aSw3YlEPznOqs6vSoarx*5L*w(_5_^t3xIwBpL&J<1Z@1R=Q| z2UwLb*5?!Q6VuNq0k>aF*V?*g+bx>Omw~ZHg+PTL^xbWaVL@kCL!iJGyp{2V>86(D z3DsUv8d1#AoB%X^%WICQv5p(2{RI?HjL($_9NlMe26W`w^uRvjX^iA7MNHo8cg8Z& zAFvg!G?sGPSV)k-G19R`TrH+JEV_5mhYp7DuL%O+hQKIr;Oq6Y=yo)WE&j*!1MV@U z4koZ;*)(JVfwOpPz8O1Gq*!4aU^PP@+bh0eNtj!X7xW=Em~>{|R$7CIx#rX1`KvgU zx)TU*hDIn~l25Ebu92qsqZ1gW%p;NUfusFktAQDEn27@95f2kbkLB1pRqhnU$|+-A z41yGMG8-yiS$L%xA-%*M@02T?8XAjF5aLL&;#B4jR4f7%Kfpg!Vw|BquS{sjz0nx- zX;P55SE5OAK)wX-{b<4E^pKL_0K3r$YS^cvit_zHH=r50ftafyS}{rqF#5T32$@;U78-tkXaAWu@Kp`>%}Sx z_XNB3Fm!8EThbHn_zH8=OT#A*&L`4}sajer0(;>3rFZ@-2zB6|6mi0!L?6KlmTSd# z?KUkRRC^i{8ET_@mi#{GuGh)3Kk)gh(#t>pKm|Sf$F-RFa1?A_pnqvrug-hYRmDYBvc)sCJ^oP#)TchW79&d4`RRHK&OhannqIon8Ut?s-wsjbX@% zNO?>$E>oC{Tz4w~HPYoxO4Of@!Rak0g@Sq`mR}__bZyv-40{q zx8%O|0pNw3dSRQ=7gCVKGSdrH($LxP?3TwjCEMvwiiATd&yWrcUn;4?@i@oir{e6M zLxf}2Lx>H9^h@_{ws0;Au?#c*9;sKGE*S;GMCIP3c^oqb8kKD2xr9`AztU1D+oRs| zM>ns3TI4>EPIP`RKQsn%p7KXY*@rs3&&Pd)N-j)~#WUvGH@hQOuv)axQDwT=w=7U6 zY|hh5AiB|l$Tn?txz2A-w(%2hJ@gDm_J8oY&MZKbXGWB&rNv@UY?SE#1+Vk^r{TOI z`kyA|TOCVt<*2(_91MpF^#mlEu8$#710cL*Jh~r1WC)2HLc{T5Lj`8+L)<|qaC=3C z^;Wy`l+Vr8)z?poPu2jR5fAekX}jMF_ynM#@C-KO>7z<_R$OI;fpr z!p+MWBIcr_VwTkl0`FNF+^QM3qaMS<^#>d`e`UAFcuDC9{=G+ciO;rXJR{ELzW&-R zhJ?EX=2j1lwd9tVTJ!KH-~>$_r#mEi%6vlgR?9BM<`0e>ss|KawIJ+>Z_5bf@4vH_ zX;CC>s*a#i(>5=pbY_Mh4!w!f<9i1`a^>VHtQD;H#>V+#pJ556?Neh7QK1KYRO>@ zI~_Jm1UZ-moEA+8UNWrO9Tjr>v>Ic}eiNa?ppCu$#&cJq4c$uje{V+~pdZk^{$#UohJ5np~Yi(KYgDVKAp+~Mt9se+3Rl<&? z)`0`SYmCaL;z|h0u;*c)y?rpit4A6y{H7|fW>XPbQ!zSyuN)E?yt3l$olng)T3IyG zE0&ka`Sr@7wdlhCgJ8V{ zT%{$HTVt?X4|spk>LGIuX9vl=BDF zX+;)FuwDu>A5w+pAm#FO&KWqPZvejm{}cIHkAdT~TatbY1&eh!q2aD_ujvo8N#*n(zg04hO){6) zLH33~?DR@23ikPoO<=0-=lGVeNh>o77%xNdh0RDZkCwa>nm2@0)c?G~1CMAs9+;$} z2M)s78?;Q|$FmGaP!oJr;OWQBv`hs{G91-#|1wCYY2N&1tVjYqhl41|AZMiK$T7_S zI>LZR#%?IK>d3*=YCK=WJz@l^<+mHqBZPCWSF!n=&KoIH=bPW=M|GD9N>4yd9UZ86iHAj^Y;KGh)cjW-S9MEH$XH*AmQg>tMKG)pU&i8EL=52 z+xQ%ra^S=j+Rj@Wwo5L4gHGAY8<9fbVj*bZ*uR*#w>6dO6WKXs!2hm>*#x`i!HVxV zi6}GR%Zl+hJsYwR{_3#2sGml3PO0XRK`%hO6 zSj+`#N2TP?Fu<}eIowk>A)qNDVq(&3NWa3TpRwc~R@J8jBQG4wp}n8tCLx+B#rNBC5P9f%>YveJ;LrdX^HX19ypbsgQN-WTbS z@e#Ci*6)n=l>aztS?TQ0n@PT@gwJ5F4X3!6d&>@JY_?q0ZeeMAfX(od=R)+#J$ToK z<3!`*{fMx1Y?L_TvEunFJ9H!tG`4NKW81cE+qP}n zwr$(CoenzY%YFCj{R=f})Twp$+H;`0YgbeY~Tt)oCUjg9zhi@Qy*DEwl z)_4PhG2c+bEfbj;h<%ST2{Ea9uqYs(c3|1Or~EaRyr_#y()LK8uG>6R1a|n5v)HS{ z4etWGBNg(RcQbSHU|JEsLN$?quS$(j7~i)o?4+~yPxH#@VY@Qv%1I)w1MQBh z{Ja`_(;m&%y2Fk|s1cr1v;u(=JeI>BBs|^*i8$B0Z)Jr`NxiT0YY2qP2K=yH&=RO3 zNCk*34(n)2hf+8>8|cTJd}=-V7Ay|kNB(?dIf|MHJ`z46IsODH6Jp{m4^C2wCeR>3v z;vIW6fVuQCr}F9LE@7uIN?IO@R=G#Mgbh|G6k5Y$p3H<dnup6(?;Um|SYhKJaQ~>!5SsRU!@l=TpLYPQPofV$wFM z#a)8Q-O}JQ?Sdsf4ZzX^t#IQX-DMw`H;fH$gtnB0DLUDvkfro6sRSjj0Eca*bY^Zu&e{G5CKH*) zBHr&FU#|ey=!7^$Pc!ku2}4OBK%6Ay*%=Pbw=V4*T)N8_0j(e22U=%6iNgl(f{r+e zoH(fl8$SK6HJl6|jn4RNR=X!B8G#N=9Tx9q?Z%DM0dqt6s5G5SH+?W^tj=`yYv6tX z6LkcUE=l?eI2mZVL4LY{F}Y6;K%H%9X_XX(H5VuR!S^p<(){VFoIvLBO-sKQ;OwI` zZQJ#3*7_&q_}KY8b9;fHnpd9XB_z6AO;GR1qDX%=Pa&>L%jB4+>{Eb8IKveWHAau- zvd-u92IJz0LQ5&9vpCh=^UalTKkq^%W-jI24gJ3Fcnqh)xD?tuo7Ti=YuUQ!*&55J z6KK|ZC58<=)fwYri07LoTs~NyhicKVGMXH6cq4RF_+u1S7bijJW(=+Im?Z_3 z`GubuEgTprHLW40WQAwJpv~5rVNra04WE-^bae{D#JA$;135sDvPatYtGe=4rg#Qw1G8Y^6VAz{Me-sy7a~N5XGT}F@kpv zZO50LN#d;3M1&{GpGg!!WT~V)X+lnIk^Cl8FHgu44~#Mohj6#Xm&bFM@$2Yrvg=$~ zim?Mbm@EY_-T%S|d(#0?-x*L}Vyctj{&%YR&!A(O9qH!ld5e#bKQveZh>!wd8~~HA z|9YvsYh`Db-PLHLV_ImM$xcVPB(duaw85l;woF!M`6qYA^yl+T&+Z`4V~u)emG18+ zr*PZNr{U2dW;cZG!ACt-)@g_1Q=^A#^KxI1o9#LXdX||x2A<&6;F4i9q3SGt|5j}c z`YavVjytU1ii|n9SMUU0q3ztYfKhoN1*V6C=mT7%!lUz^P*LMHwAq{_~FXffBQ zPa3Xw#8~tVhS=VAC$ak{f_Sz|57uFxU+2mncQCUyU}|GjIE9>Zyf558A)a-eNQV?Ui_{>6O*S0$NM zfj<4?WviAjea-zogF;OK!ajoKEWL>sh}ORm-j=*KOB!i>N6Fk6rVHavq>$o7r9zNFZ;K$jSRy*^R`bnr%9 zzkG`gil{~I{Gv{>Kj6AJgPmbHJ65krxWZsyC|u6T?dv;n_u)7u3u|HU?nFLrSv=Im z0ldOR3hY-#+!Cpdk9NI-MOtAV}mKKQfpccb`{OgeD7@x+AF?w~_L(({%Y!vs5 zTeXwDXx@l20p@)A6(BZ>^Q2VrNcwf2+ju$@jHblzV25mhinUq?=;jQIN^ws-M06U3 zRNICVCoPI8md~)b$&`xJ6ua+rim-#%An>Ai%HId|o&s6rwA4}_@!9f($39HfwG^1n z4&uUXz^>8u#jueUMppSy!}WzCpHlq%X=T8d%Lm(0=ghV>6 zP{7GSXiREj>^N-awcZXqV-_#*$3auqsrd-^FdFqptLcK%_eaCYZXh9V4%w@SqHcYTi{2ez@v3>yU0iY1RSL>V4?oo^) zgF?zSSW~3)`6cx~oqrfC`G}+X3pc^C4;yzE?1=O-x&~t@nmA383I8lrl}~KqREULK(=+C-K_*6Psc$xP1*Lq`A_044y1etPH;+EA$&o2wGI?!Hi)r^ z02SFzmm4R0H=qg!5T(Sz`hOP0wI453`mAJ|`I$lv8`qU&VFDb(gvN0oW+*PGTS%S6 z6ELUHVroQ!*U~eI74d2tmx?KZl+g;u_=%d5PFqY$W69GV#g)Egj3@(H?xKLU-C>xp z!y+<_v*e6hpgQrlXaxkf-)RXSf;iO&L0YMT5(89BpXS0#+iZ5CqH6NzdtzQbwDDHiCmVx|-U`Nti_Yl$s!ZhSr^c5=n)CElS*fEq4Cs|UfA zuxAcG!9TcdY)|@;yksO^WM>e9+Q>v&@N$%TkMJ`{&*ZO*EvE<#TS+LxuO;Ubx$TNf z6on<$mn91rk@Aj*SbU0VVVI8JIwC}vqA>LETv#sFj5vW6rFV`qef#`1V~!S4SUiV90hBWD`tq%z-GaqH zgHl30hXv~88?e)_lP=9lQegqe!}mwq;t*q18_E1wYo^m@-UU5b7bu{`Yg=IXGt{s} zzs6WvC?Ps_nVL$Ws}O60Z=owAgJX>(yfTrQFUy@EmZn1ak=KNh`Gw-S6&BT;5}!g` zdZ<(*ClsHtIlr5?<_7_voTCjb=bwRwW~h}Z-poV=Wx}Yuly4?M8L!{2k0Fm^>(NTnU+u(q%i@LG&C--Y!P*gf&`My%otNHWzem`ivBbc z@*FcV&DANm<6wl#&<9$$Gv260a-J~b_9{WS4>BCk;xcEnCXkLR75|X1c$d6hc6o6! z21z|&gJXl?>0m1u#>c`5rC{@OzO_^QIv<}jBu^Y|Yac%R?~@cYhmg zTAei|1u-)edET8pdUcx^8SYQZa^MA7TFoT-2-Zl(E1G^i{N%m#CT*Ew}# zXrpG{4Nra>d(&rPEx3aXi9hX9hVYXE@vBPHDgC01(yuNwW&Fmti^(|uQ^=a89HI^Dj@X-Y>sNk~h;CwE|XM7r*+ zy7zGHq)G#is{yfB-b@^~%%~cI2iT_D1R8TSJ95B858(V5 zW!iGm0H4IOE^nT$*n`7(UDAh-lE*pXnsO%QWFxVn($*J#Q41|CKH2{luv!}cQN!6$ z!y7Xbe?3m8|6|(X@(VLu7Ki&!F2N>1Aczm(w?|8iM|cMejq*Ti>+0N!GJ1IhKlj-8 zh^6^O8CE);nx;43G)i)Eaw@*<{rYWhe?G4Uf0Wuxl&W`nu`eCe(LNe~NrpS}%_puJ z)}_6^H+F?qA{$*M4L4mCn{QVpOFjS8w)2v(6T!nqJ9>U@3pyUwZX$>z-nAz>pKJWDW!g^;?o%yu2ly{bw;JORq&QWjnZ(sG+ z+Tp+J?h1XBVrw0ojWpiGG#|vDBDvK=^ioGEE580@4tZ@NzODTFs#IHoeg;nrov!TC zH(6j5(KRrpg?mL$*Iqt$VZVv0SG^7-Z@)WnyT**`IU7DibR~S~=2orL;EsJ6`O-pBPPj1TQ z9~yDyEUUXx`A;+-lG@PW`)zgT_b=Agc8M#u-|yf-id7lf>B1PG*$dxN>*s|B^o5L# z^dN3u2ktmpmgsr1$>l6w2AUrKp}z_ldVTw!7H$e#V z=f1)X;Hw<4W2t;G&Ic@%2N0!Jr^`Su4##R!^#->JQB*bw-N-U}P|;H~nOpS^KVUZ1 zei?V;9(yPe52o(tedTCl?=ylLIZAMUx`l7*wwpd+kjt8>lXn7dFb*i7sjFlSkPANk z<7zs=6@Gn)+Py9+IHxqCyE{Mt=RF){V!x!V^jUy(Xh%HRv8Yy+7qr7Tu0o@^{Cz_<8+^Aqfdbd#SIV{hGb1aj=#U!XSU}?2Px2BD=;;N$igNlMX@rB zq<%`Pk4pL!oMS;MlS^ViSeqUK6tV(rk9AQyZebmD9Bo1$qHE569CaeGP6pU@_gMc- zA*)F2I$9$@kEFK0BrS(9C=1A0xF+8b0{8x}1HlN}rQ;h$!ct-08o``f&HjE7h9gq% zgMU?2qO-#9O{5@q%p*jbrI@9p|9a}363Gf%VP>M5*4`F*@zYBon1-Z75OUfL3~~05 zWKug~>6A&8X**N;O6lg{DG!!w_sW< zAy9`a-+Sw$g|X2SG6015*i>AIksjU4#3|CgP zH7SUMwGP%>f}*Is;R!iN^k%%#cz?5ldw$0B-H{q$3?k}5lzfrngN7z)(JQ4$QTBRc z0SB_No;t^LrG+vQZGPbaC1}Z;&>nu@<8B(OC2bn!&buCD&X~M%Ckj_kBkj%kA<<3| z!p&IDNkKy#vs*xIDmmeR^byjz?qN^CoIIF6Nw{|K%U>Noy;l4TPCR9?gK4BDlM|xh6!T&NqzNpZ+UVtnQSs*4^q3L@jrcY6e(Scjh9@tv?HfM z+*uRMu%sKWC?@=dFmwC|V8>xL>sH!7LaJUNvTil7e*ABt%`28zaH}F(_s8N?L{Nfy zdQmagA;Lj#{W!$!v1W*gkZTkZiA0bAP8&L)U8!-!qIa3)QGjuAq}q*Kk!|W_vXcIS zyNb@pd|mkDQRYPY77D1APFpoml^0-U1PW(`aZDvmu4m_l*^e1^WQw3CUpSl*-6D$&5KrU}wU9Y(k?$Of_x&1)g-o1% zu}$#bs%S7R0yhFO(_o&yYRaj_3TTQV`KgdL0}3*4+T91(W`B74!`$rFlF+4ut#NO{ z2B+<&8W+aB<*cGzNHK29$G0G;Cfy5B)oM8=_NWKX2VUiq*>HwJcqgB=`I;0xQO%t6 z=Wws6;6kE%*zJt5>-v)yoHXgeJrfr~q7@#7dqhf&FSF3|=B(40Vd~9EZ2r`FS92!* z1cQr1 zY*S8nBa&73Xa_Da$VHl}*QfYnlk}}z!n{s*1=q=r^6?7n3IcBsTfF%Qsu9b4V!_W% zK}6+nmkhq9?8?sj6uom5K_rhm&F)pZC>TgT-9%jLrvOeJNgAAQ z*|Vkb&Zr}fE@)wGCk@Iz%D4~~-%LRi&Bt53Dgk@k`Dx_k6AMR$Rvb0!*u|P+&l9J# zD^-!590(Ls;^f!btLWH-=@>kflHPiR8wVfUuZe&lJ?@((Y&6)gXfw3~8!$8wp;>xQ z#DlL;qng4-Iim#&&9uAH)lemVAP$xtr@<51SLhpZulzf^5H8Zy7Uj<%ROSG+e-#VT zp&wlwSfrh6g*^WZA{274_K;;;Q`I}5wqAC8VpeNDF7U;i7q`RRO*K%e34lFFjl6%@ zG8}@-?M1*%e1k0`LSYkj2w!=Qg%=g6zj~|w>Jy)uW&1gh2Qh{R;xx=0DT5bLX`0)j zT}<%T)(kNoLl{1I58}@e6-Kreh8PVn?A5sEjfT^qac;wZKc75_SX-t8`JpYY2~Qy{rGnMYp3RSk*3(JbVv+tuWL83WbQKIAhu*!^ zTCVu?7t{JaBioMF3fIQOo3aN1HkuD&@Lr^4R(P2@9MrcQQv}0eJtiZtb!A7Xv`!CG z^dLwn@^VL8Ru%Uyj9zY~#!7w_1$HGPhn`Q|a`qJ?Nk^)&1sW@cccFm9+M@!THAcsk z`G;dPF1M!$=rZ~w9T&Tjk=6S+!FA1B@~V|)o^Rb+TJ2N3qM_qeJW)nhD2-Qbm~YUW zV99PSpXN!w%MVnBm&DX!QBF)oJdW_bRh|%@iJa^^gO-=}X!!$0`O1u!ZE^xSm?zEn zfTSrcm;XIjVJ^~J_-w^15yPCQ_oS%2Jjv3c=LsiAd0jOI&0{$jcq@%4I<}u#W#VFN zgT91L>f$_1*gGIhj<}TQ*nPleF|5?cj#Puop7wV%#f7bAv|OgR3qUSI3UiHWKW-pe z`|{WxER52j2HK)7%&7_tbZ~hvEWVtT>a3T~_ysz)4OP7z8I!I`>Fpb)W3uP28&jeW z8j#L&G-I>==nBHvDHMX2*jqpKKp|W16Ww0YobD!dH1*+Z)tD($wo1)CG+?V=5J zX|)ayUb-44+>Lh_2VVnOg6kJnFfZSxBKyo6$|iA$WWBC`CuT&2`XQpJ^5qIf^>8g);#~{DP8a9gQ198JUKzDW{XGHFdmfT4T=P zvW-KtY#tqx?5}w-!t41Yc}e2Vd$xJ%V_gCBy_If2^s(T#unP0$rvqhD9uw#y7z3`t z;9Hlj*jxGTJ7N1iA!KwZ_(Rln#FG1l@;7!wTQtV;o1*)U0me6K=%XBKYby1uh8L_b zHpBym$M&M;L5VWUb!eF>)`$U$ZT>>I>n7K9)dB6j^8l|4VQCNxXH=`pd}|hM20A3l z)il=FpFQa#%R)56J7n*2k0B|BkVcs!v)lP~7KSblEj=yD~3 zrNpSOfwYuWAW0eO^i@-CN0SV;97l+9pAAyZvJ4--*86p_-}e-+{&>{bHl~#F`AUt5 z5?d~Q9c5AG-2v`AajxjLqYGA!$udIQ=hWv~(h_+D7-{r7s{eY)R(sx#YHLDe-HUMb;`h!>^GK+=^04*)8{O5~d!|Q*%S;fD!#M+86OA6K%^_nq5 zt|u2Z*cqJ$+s=YcDpj2(N;sK_uqX#TRG*ft%*HZwxE=Ypp$PfHG#C;8Vi8hoAk zfe-5UrQMwz-Fbm8){6`a%t3`v$G0NdMgBL<>By9>aRF~EmR;>rA!=YdzhkoT*}crCMXmqxI2E#{O-LmIJpFW)nm;k;4b<} z3L;5k%D$RWdJp)~ebLdX`^8Aa7Ov=*VM$>dpdp{83F#r;|hb$Vc>?ng` zoK8(&i_@w|A3%dhE|J~`Dxf!_7u4Cp^CU>|)7re?@zrDUN0S%b%c(ri$a_O!%Qsg6 zgN8y}OmZ8e_4g7QM~}q0-NS&iW@E0N{B-i0igf6@@r*@qL#f-dQ@dtT`)WTe76^w= zY^f+M`8IOI4{CWzw<0I5w?_GbTJC$@tGIOKhEfH~`B1s|%jw-+z@U0N&jLRu9g~jp z>~~YR1xt|Pu>)0GSeYe=3}Uk!X}|6nKY50_R9jGlkl&5@w0$e?Fjz1}TS(2(K9!Fk zvKJHq1|{LhC4Dgf*qc<$i2}KdCiDN5Z@a3z!`*))#t+)q&+D6p8Jf3KkJmoWNY^)) zJ>iGg3TTP6%}O!2{s720+T}IdBfqoaPXiTf>HuX z(%U_!2VKOHkZ%Z0aXtm5J=fW`tH)9)yS^| zZ`GXb8WyMR!<34qD+5hZ=uZ8lV%2awm7z~q3X%chqi=C}&`6)uMC%Bk38qsrPe86!MKwX?9&Lkgu zBZwrnn?p{`VGr}q9*JV&#dxGR7ZDLVS@wyE6OZsxSm_5E}E?RL|U zcvZ{?mq=~t`yXC_q~a(?#1i8%U|E}`rtBfej?;$L?XL;NKPy}Do+rX2tARap9SyPsa!6N%0;_Wp%`V>i9!9;9+)R)l8UHlRb#6J z?et$K<>Wvr8XeC4Pu?U8DTcdi7r;xy*?Ez6E=$JQLBi5vpdsSn$N72Il(NezfXKL6}*6>eE|Rx9z^H z6Rb%%o+OgqhI=4u7cccMTTJ*$j-VLan^e)0%45$ zKdS?f>a2P`m5(EhErsz zD=GP}dQHeg5Q`|MWr!Br>l>vBWQqYP)Xp@7EV^GpnH*`2dLDdo)b}}1tA671S>FC( zWuGIl+Tk@z9F*!y8kMxS_QSx5#eI!%ygFI65`7(4d#4(hW{hXvdwH6?#)YtIp-lXr|6E{i4z(2-224sZ$@Y&)EJxjeqKNwt~``o_-cp0or*@#tZWBWutquNHb4Ezo?8W&q11(>9tZ za%%O^YbBgA!_(;P4h2P|AK#CV7h@rCY|>m98{mU=?l5JUk$VNaGc+bEeKOA0f#*uiVt>j(cWFfiJ-x9ZHgc9|dO#2NBa}y5pPgc60VxzkI+1OdNYl{mKqinCo zEpKuY@(NProg44|kUU?;EuxK{q=ej=v$6{Ia8Nn98YPw>2XpF&u?_JDw)Ny@Ht@z* zUZVQl_Vi=br4V=ecE#p2?@xtuT=JgM0=$?5sKA{eQFB!t{$zKl+=wF=xXb{f>9{69 zw=SFOlB|4@1K7n*2Pii7=yky~`$P)>DqqkI2%(d9GZ4;}eeTqb8CcbeI zSU<}pi*CXl%FyqsKcx+reGnN%lb-xQAX8ehpXr_5MHx6^v)VHc+dcKcW9bEJE%c>6 z8qEHtQHo$A7fTZ5KPqwjdoOuRCX4pq%vMt^KM$6c2o+jQLMY5!x)D{FW#-T8;W<>v zdiCz8(G0Q(C2b&7HOL6Z#&58k9e0B1&TU~|=t50Jh36r7;d1({Z3GJj|2D36-gQh0 z7wdBBfYUHt8iHR8WmvTI_7+2PeTweou&nP&QKvra$5vaRHnrMCAv!< z#k-sZ(90UPYWGz58u)R#dTWf@ObY?jr{i#HP20`;PD}@TSLv%wGKSMGm|iO59{<67 zv;E<=3AmNMRUFy~wfdJE!T;x2F>`soFSOh~#HGXJkO-d{v6}EvNAYws(rJcI&>-U5 z9MKfV@WBc3SOJJ^ov*hH+jGF6I+j;;a68?|(Ru9jSEkX%^L-vn6E=}m^+v$(&_p}5 zMr4!;ZLF3yht^af0d-mOD6~OM>^q*Aa1~cEp}7GYk5$2@XYk|@=h;Um5HCR&QKry4 z$li8oH@G9wvf}k2%u+mg2E5i5rD28(l$?7InGis{&eV13RW?G}Hzg`TqGK`o&!_5T z$-qBP=djK_7<1ym#Zp(hToe@YyNM1yyOELVu+ZzH=l7@oiN?y{N1oO+zh)dsM-Aj2 zJmvB!OW?r2WO$^9%MMj`0Vhtn`KmF~v5M>TR)d zq(KiIC;LPXH`2Ooz1i8G@lGZU zjMh@Hg>sadYrw#&6XNQlv(2otfi}}g8&*j5gMH|Bl2c=wJ2tY#X}j=PU{zUoF?*EM z825uYstfX|WSqcK73UzK<0|B!x4TMpMkP&b^X_CxdlXn$T{&Nejf1Mpn&4)>**SE{ zSFe1sB?SXaP!2`8|io9^dP44{4}#Hdf(OVrT1G|7UEuiYxUHL zb72t#qwXIxh?EUyg}9n$B|bLjyCyiSy(kP(HC})`-Aks2*pPnNO-V7|di+uS2XCWu zyUaElEJe4dR4qG5rLHBe^g_U*Z1}fm7|4Gza(^UxwGo)FfMv}6EO-)*MY}Okx z1D51$Am}|1amXupiEyAEj14BjwX@Bv0EiQnGLdCkg8_or!e%5R2N_xx4pEk>9?-2V zXaWzbU!t|tb^&b8ojHdVovx!5>!KwlphhT+PH(IyWRxIvgTUtv$ULGFRXnT~j{0*` z=CHN8fR!-^y3)9U=_>PZ*Q%24IfM<+X{4t^PJ*C$Cy8VISF`b^;Kp-v*j^e@5Af~S z2@0HmqJk=lBsK;|c@w)DFMdj z*A&j@TaHt&-xQnf$`#c!+<;Dx91wJFeklr;#3?-o44ViHxuzaTUJ2OrCl?`{)V&UR zT=ywp$C^A&<}<1ue4Dl4)v)@0O(LG*&Gaz zeH`*dW=oXzN5#qUe9GX;NMJjth=Y*v_ck?+z6s%k{2aDXNq%nAH-B{ zn*O1{Xspm%1}<)6j$0@jMln)Yeq?+eMb~zJ*zgo)<-7N~tpLCNGj2PWGM+R|o;SL- z4gHv-Dex@_o_DYy)Cs|+gR1{{Qt{GpmeiC{xTdNx(_VUIggDt%YtShNMJHODoSQXq z8vB5L72D8AVOJ!*G@hC`tqPvrzR4=9lZ}9k(o*s^@=8}vK-F-*S&_ZkSJ2z$;$M@{ zAaC)stzL8D;{FM&vP?-KghL%JUE#Se1yQK$76v>?X<n$4yH`0-L zr85HYz`4I|4lQVpwOkGg@4`vgd0vEyq$G-m{|1vx=up|Gh~#3Fk4mgJhidxE_*4TU z+}rAo!S7Rm8Bxj zc}w|Nv&=OV7uz0yci)aPk*gc3XP~WGrl9@l*i?mu{UvFh(y%k?9Bs43Y3i}p03GTf=^w(BA#L3T zTV^Rck`3*Jz=Jb(VQ$*g@H z=hz%IBFx{U&Yzv31mk3~GET^evDn+)kXY&)l)AkBexjvOjW4%MQKK9=lcz6I?v0q8 z#ZfAcUnDQ)&w*3h$0uWsb*(N?-V-WK#WZrL}nLe}M13Rv#6+NPaF%4JcGXDZuWjJC9L^v{+5z%wDm zp3)H+M?KYPlKC27N}IcEO^H9Pi7P5&!Cw`JeTxgqe@bsg|!9 z-Kky;zR3Tt(X_=}#^lo<(RW?&5KDYHdiOxq4+aw^7uJ*GAjiWU%~G9nmeB`F?L zp#`Vf{xRoc?Dh~g%lo|r*BK9E7Xf(@)c=m3Lz<7qg3)3}PO|l?kTMkcT;j}7-2< zmBgK2?+2`Rld(XE-ry1}D#u8f8}NKsBoI~OI5UD(maZLEEfKU9mf&+2YjznBW3^H9 z*^u%YbIyPT6><;B`ql~@+LiawW%U4Ao8n-TJW9jrKc_T?U;oBedb6CMQ^k_5yj{%E zg6*6mEc6}A&LE0CQcL4Bl7V&+EIL#AHxt~SqQkCc-SdO@O1Di5>WO6<>PWqR@zXK5fQ>DM zc`pYXCLeynA~!iEO0>=^_jk5GER)v1R6r$aiDVIpXdrTN9;?d?4OPnwzn8rg1u(Y;Cm{bb{BB z?u0%~r>K`-fzMnLsGfeAf{Khs*1q@Lcz3jxM6!Mo(7a$R&xh-Z{;Y7Fgu-PCA~Qy@ zg^c(085jTqEG4R+|AtS~Kl4VSCYf$+s5wuSXZ?2wMQ7g!j=c(qv*mmEHF`lI?proo zA>Y+QL?_5wtM$d?IzKZvzl1Y$y!gmXwqP-+_+oyW@nMD1sTS;UQBJj@H3J^}jKg;V zd1XX567p+OhhX09Hhgr^e`h=i>BpPI)?*LK?)oz_ov^(TO7$at5@O3=y1#bt>JeRF zYwpV+`1%my7$v20pR$tP2El8T6g!YtgP`lGNDpuN_2VV(JJ`~M70igBM;E8PqXJqU zd`(A&suRDcTqxvw)*G^qEg!%_p1-7_2LF)`nMZVcX-4U&4v)qQs5x%JEQpHpE?Lm< z+lU-fI(u%4;>U>`i_T8Js(wQ_R*D$PX0o+(TuO zNlDfin)5VRd`RMa5_7(Uja$a9B&*CzmoH_J1|jKjt6c0a9`1lSag6CjLod?@ON9O6 zS~B7L28=%q852gjrGc+cG5AU^B15MiF07>niN2HPLth(auqNDu>n6XBe4qLmPs*n@ zZM9NRP%7Uaq|-iJFB?cQXx8GwYuCPOJ%%3vE4Ja_Bo*6MiW#V7i~92I_brv@c35ma z%m^KZZtv5Ly6J{(weB;N!4#3Fzimj~rq~ z8c3S;IbwNL6+Km?b21QOJjqV(LU9e^C&*$wL`SAA{d3$;e19~!IhV}LyEh4Xj>lTU6^PnKGtkvccfg0sGsyB zfVNm&r)JVq2c@U5-raPe?O9p$C|iHwYP(?9JooMbig!?z)TF)Ewb}|gs83z-%-C~R z(?(R3`We?fP_wXvsRfr>x*B$e3%4G4+yBX-Q13no;fS^0kKv3(w7?@%dq(69$%Lg1 z#6l(S&ca@Luj%cxeY#Xqcf(-sROskV^bR^#vKYEU>zYgiLyXv1FE?wN3x;h9Ji@h)JlRqSXvK_aeK`dOQjpx z0NUC+sx(bny-!xy-%vCE$2@Vm`s!8CAjyB16hn1yjnYM@E$bAc*CxxOtL9#M-eujJ zq~N~QT`eGCJ{;#?s;4o9vW8ov5@GKw#hv8+NQsWsf-E4Injx*fpFmK)z!A?U9_)LN zP0)AxhG&M|*)wt{@)wk?8$)<4=!c6@On?w%KzfrL92-*r$2>1~+QFshGRlpR!?

AE6j&22C-gvC>e(SPk3=VO-`mq zC<)bW0q6n}jza+!XFED7N|a7vS(MFRp;ae**RIireT3Bm(8!JSf{SL9Bq=?oXWX}C ze$2drLH!^U9OO!d!WzmY8bD5GnFlvG&Ga|RnQB>*01&EiTJ*#ys))Ba%+bJRC_QSX zs1;JIL1Q-lb#jV{LB2T_d+W6IXC^yhe?B8)Pv#Gz3 zMr|CzVMN6#(=L`ssJX^20ys~nYsLEqgS-OYUK(Lw%@^2Wh3gFm*1Ah~Po)(GDjT*@ zPo;|jy$AP_Oa%je_93F@aOEB6jt|rf_vM!FUr}FV`ofQr@)|Wn4LFsvC_j4>ekrO4 z{PbrP9qCEP55t`;W?}BD%miG^TrICVx7DXusx=lnat|2Pn8l*(skRkZQE9*QEA(@ZQHhOR@%00+qUuNIs5io7x4(I#f&lg z=*!k@Xb=^cb;&=%>bDNN_RgSt0lAlUO&T;w+GI5u!Y8;8FfRpw;U(cjVafr1Xn9cL zTEg%7o)?(50kuwWIpgaLYn=#onzyXAv8GVRFQP(E;&^H=>QKt}I1p~C)u4LIGoo6s zr{XW+bgzG5W-wMGa>RjpjeQkHA;j9}3DqCc^|{D@AYP33Vd@`UiN>qh>ygp|KZ#qk zQA51;AE5k|Osb@&2Z8H3Q7D)3_SwlGLVDjaNa+dxEcFQ5XRSX5J;-q39U%XvxfqwU z^zLbJc>oxSSU?8p>?-;2*mOVSV^X9Cf&$zyN(>9=?SWJR^QhuluC`00^MIYszjzM((lirJtLSY;|I~ ze*I>GTRopb^iPW=likdXwrRXTq!K>+-rs`7dcut?B*yRpW4xlgm_wBl%o5>J%!4no zGIo*t>eI|xpR|*7+k3B9?y><#8R7WWsD}tIAH{b}9_>M*dm2*Nfkvtk!|P2}s_4=j z3SKf~!){;i+yS@P4fCCryl*`I>l1Nq;FjP5%1mvDNN32Jjp3ZGCP6{~xS!)8zfK|h zvsJ?9i6Zf*P_1gxA>m4786kYGK4-QHM6_2^3IH7$Y_fFBJU~@8--a@4;wewa9AE#P zsZ(staOhE~zjcIeA^LT15mBeqNR`P&gK&^ou+cRW$tBkA4h#vS^?m;)H}ib5%8n{2d<(c;eEcO{}~}bkHd9G|pOWdhBzp zY;Y!@Vlwh~D3#1o|MgT3{z#p3gx0JcHfAWov^UU+=ktQ@w= zQI3;I4!OK2T3Xm&qneFZleTF70~k|jeq8lXFvLCjM01>NM3~YG*OH6&6|i@_HlBBC z6oqs#u&}My1@e+H%J&q8x9*aDa8spdnLwn1@~iuD4b)2R*#?hH0SBvHkn{v7n!>%y zynxt?@_ZTd>-BH0lTuj2B0vhXNje)^uC{NpSzc;{e*KzNAU84XYy{h120}zh)Yhv! zg``m+2bXC1Ojvbuu;XL<)Qb_kl$yNL678|NwAkv{-AuX5C6E~*!*9X}m_g+!fN)T0 z&-zoH8J%go&)Y_Vext#mb-AC>`v^w(qbYnGIf`>b6`Y`aMBrrJY&olqq)pTG?XT%p zvdeKjNmINT+=2V0E_fEDga@nhqjOx?=JF<|Yi=hR_~PN7j;g|zVbpDpUh`{?w+0|V zho0kKKEORd1G7I%9&^>HLwhsKkIB!tc#=&Prr;8{xN{ffs6^8=$j6C%Q?Yso&GZeZ zt1vzuiK0g}=Ou@`*VlY_KeuL!9XCdp#d_$X=#|`-M|PDx9Bpa5ZDWPnXHJHGG4O~- zlz6H+#WvgqiUb{5wN!$BOj41mcTwp&1wz*Lh?Vh&k9ad6=StyMFCDg{=w8U9RstBr zP0zzO%Z5~GjhR-ex$*jt%`v~Af{a}q7h%|u<0-ck90Y6|9aCu@>h<|NSY~ZdCuCKv z0DR?F%c;ZKVJO=`ueLeW@fBoK-A^77U)5k&-Qemh=NFbwFefp1Xrz!_0oh0dxmV^d zwKDq`>giR87uI|?tE=z+F=-9F*vA9?Vb4=i1_srj;4kTlso0$P&ebS)nOBBQob*|p zJ|1cCJNqUMw&ZeLK4Fy#<;v&&!yg_K?2=cQ$28|1Zp{~qLwYCkI_!4Ml?Fon>U;S< zPGmEbQdK(9=kWaq2H%y(#s;r8C)=ciH(A_uGS0A11+M2h9d1w*_HI!qQX}Q+$={@~ z9pLd$*!1nj$&JOG09k`@+osn9C+yo@Cfj)@0o7OTZ;e$gVISg3Pw8>2P3a3a!60RD zkK`)@yjx!uow?Lt28x&Hr0LB!sJ~Fy@YnMaU`w|^dqe8P-!lqh5f?q7B8#!_;ui|h zE&Sg;i}KJs*b>wy{*PU}6{Ej#_BNa66EXy0Gfv*T8$Y|S5Ee8e!KKu|HqW^+ok8Z z*$V62^GumJ!HGVyt)-$v-kj)m0^azW%V5EpibT^;whyL%-}9n@6NtrSTrQZ1qKE_h zW-@U*)%%V3?uvc6Sd7_6_ z*;mjnN^+j7+ww!VHs@!Kc2yoP|JL|sUYuL<$971IPMH1JY+u=vxH$UAi8Ld zJ&kwWY5uqi{+7*_vuSyPU%So)eJ(eT?5@(h;l7_>){sG)mH9ct8o}0SSUW-hma`#e zyX_m`zQX1_t5$QJI)>M1qxyv%A*yft_9N?Vv_3!x@h_sO+i7j@f$aM}2;qd=uDf+_ zXR3=f+8m&+3q>E4wB`pebfURzkp{lOwztww&+&lhUoq^1t@rs2F^8T})XdKQK!f?6 zDj2$<6Q#Z$qiYcFc~FF!$9gZmBc#pL#_SjJ!8e>F6hXaGO!?94Xnj+N@z05a|2%Pi}eKU<8RSFE&!V=_VOf+F|BJEkR@HIp|lG zraqZtNGy(&ggrkuyDRc`0zkD-TkY*7Jv z4MSZlqDrJiK%OaL;}K4fF!6f_1~Q+>CV-BiMPHsRG?nTh?5o9~lwo!1lSa%uf-|8f zy!gT^SN(hxX~Uf)jDgZ8#rZ738J=dqoC^E$OBEz}JAS8|#$25OsZcEcBgII_D|W~T zU117TYu&uSC6dslNg>Re zh8hJ;GbD1S4Q|Ad#i%5SXoJ-8Kv2Xu55JCrVFOl`)Y)|{PSTq8Vj>)XF6VC*G9r?r zT=gv@y@B(=`WYPv<0zm|TObYNo4$WIZv04Vf|Wu2QfWg{H8HgpJIUq>^m|zF=qOn` z4|!ngs9-AHG}2gygm8GQY#47TC))VL9EmsMV62PT>%32F2holu9psjL5l%@Ob5FNv z#OT(s?a14G)(n-aoRH#f9JHZgX9oBbZ74&Fy*hauGy^Fz(RUX@M9LdK@rZf~N}GWY z(*VOi?F8U9qU5w{s9U^$P<$H}N=XRu)itH{JEr9uJMUUVktMGTMHxFbQ+ZxwM9T^@ z)G7hU$|0?-_+h4bPf_ji?Iw_AB2EJt-fAj(&Ry=(^{xzN>CeDaL8Y~El-mLxr8Y1t z)|PQFTr5d6Nzh~S;PmvYTb(&lkeFI)n?FRVDqPd*t>LKwTBwa)9SIT~*SO*-zfd94 zFV+J?9^dw0WRre^!}ZFDzMdBH*LtAFNAJnCWhNiuca`O9;xEmmE(yBXthjmk4SIsu4BTeF9W~g|tDNWb_!23VV zYM%*yX=f`Jle1YO{|3q*hZZFzI>E=4mO4bH95==V&|jFUW89M4gVMx3Pb5=AC>g@h zMIb`gmYs^f7q2jx6U#sM_Dvna)v5&3`~?&-#D3tWTHpI=p;e{jz)!@a#)47!3|mOY z$0PB~kZhI>ERJBepl#Nz@a$*p80E_9TWV2GFS=8|XGC}IS6bd^741_p4{{`^g~wob z;&t1%H==mh)cK8fPCsms`Z^K#cZoD-yH2M$HQDdDk0ldk?3T;xi%`0+x22zWs>0!J zIg00sTlru$lyZq-lbm9*f-YeITNRL|?&_~Xw4H8EOF99tw^k&0BvJz7PmlPy-CUKN z_>xA=1N|||^liQz zmXe<5tc|ONX@X4@II4}GHet(KHSruBu_1N-Zd)#kC0}2e(&hLxBSnt3YQ&fiH-AJM zi)=@q^G|AP9j4u_+nk3q!+kRlJ%*opPa>A>x1;3P}F}XJHgcm zq!wX3bvOj|ZZ~@vUA38R@qx&jC)y5+8L(sUz~BU1@FPh4@(^NZDat){9T4~fJeK25 z2o4?&Ms9XvXVz47+E>KprQ9(4 zvBD(EI-_Pg-&w=wh4*=TcWDZ0Rt(i?^itGqW{6{&e`eAaAc6n!7y>d(TT)aP83$`l zg2PmYP&Om2?!A$$&pEm38~qkm8HiS2);h}Tgxe_P`YIukARgF)z1J_Je0Mt(gEIML znQyAf=h{t8Dqb1UAg$qTW_t&RXSI-gM=ml_UhHI`6^xR7D|3Ef&a9F~#Ke6!!k{k7 z*meyKi{X)hqEflKBgQq=13MkN@6=N@arc%F1C289ZpmuVT3}{rsTrA__r+FKZFgnyIhxn z7kGW$x9GeVp&Frnqw1Wncubw42i0D2$2;s@Om<^qG|m1Ie=!2uqxTbY`GFlkmbMNF zt`?8vK)oL@^oIjO^Jp z;?I*UR=@^psGn#P+$QyGuL8T_H6M^_xXt3|AA`qOdS*qK{+iK;7=VLf$SYWQ!VTTW zvTFPDEL3__X|ydneBE(gDVU9d?d7hHAW9&3{j4f-x$V$nyf%o&x39#kapc*$J?#+n zX!i*@f`GrXAaPlg&-*aM3KxJ%zKMhhI0I;s%MdsiB5ED^H39EB2z~?`c|WBKVd0)g z1cR(5M}tJ{b79O6^uRx2xI*%)5~a*hy7;8`n@J&K)t3+dH?V&Sv~N4=5&fts!{Vw+ zG@jIocuwIznx8_oO;G1w{O*QH^e~5y&3s~LWnx4^g0B82g?SiSBowT+uktpY8(rO8 zxxe=zoOo%nDUZNh~)Hw+)?~op) z?B^hC&q`Tyo7#NMbn#uwJl%++54U1mnT~q%;LVId3=(P?NydB|qz%3F7-P63>l;@|BFD0_S;?K3ec)SV1JhvN7hI6k?_2-9sJq%xKj6Bx%hn(ai>2f zUiK>em>Ct5>L>xOm4t#=-Sgi(oc`o7`AvY(6kuZfbdihEv}i`{E7|U5J)0MzWGfA_ zIk@7*OrfA3H)3AM;8Z^25uZNOt3SV>8D|NXf}J0S=Bu7)q{^b3Tl z`LlRSGJl2>3%tW+LvlGhK1g7=!n0Uxa{6>@BV{SRgG0Ep$x{T#R^jc>@YHzt?te`a z8Cig+olK~M(SzH+O%qA~V?)eYW6fnl4DKn5;~{}Z##DCml09;(`3dkVAvX=5LOy8_ zj+RPmpJIm*pSxEWFqmjbHoQ$K-!C!}TJDFJgX3+-_xma4n^I8q-$)4uO@y%`{6+-_ zigpub<}7Ndldp2Hk9<@|9loIA<;B)*TJ^l(Qmlol65m8!&AX}7*@xPbpzdX+-E8vq ztZ}>0jZ!8Z%u)fxujq$*qjrtub3Pwp;#pLAcA@F%QS5LJ8Q3HVt%R(mUkhqmPA zr=?`e!u49VlRBF>X{W#BSCp{)G`&T^d#mHEsO-t>%%!gTH7>dd`?%@(|%v`vi}H!KA&loHHghIkcXH!}&{Y7R zfjiNB1>Rg_5(Y_9Vc?=5h}ZTCy|JZ)r$C9cGt&NP^6fS58QCUG!1wu^(Y#HL{&7X! zYd-QeOLW4^^H9D^w9Dk_gwDjMt;MUDMwsxEim8ip!12&9D;xK3sixTWaC+Erm{Vj=oAsPyc*z_{Th(7-wxVVT**c^p z2KFvw(9@@FCBa1(YKDt^q5XR7kBSAPDXaN#|HAJrrlS+nM+LH?EaCly&Rr-fheJoU z2PnV?8KbhdQZsyoncF49F?c32xsR#NR%^(}8NCI8lYpTbf_V0BKfw&L3TWgMP z1~-}nA+9UdFQ%2=;hr?LTO2V=$DOLOOgsyBg(4Y|+U~udA|&<^$VN)Xtb>%G(~1Mw z@O;=a1Po(b3BZ9L)>w!g?41J2RHv2L(d-E|LAHVb6aVETAyh zo`6nm+Ri~k+HUCqN`btK+c*qD#@kOj2t{gbR-8s{OV|<@g5xio^c_JMfQ>(9dh2}M zm={;GOASSr}64PQysa=||o$)05lKXDWNd&Gl0p*JS zQZRt8LejuhVfv*v!Tqv}oBomve08uSOICcGQmLT@>Y*ppJ(E38WG=35lSjMg{x;kZgd$PqYC)u;nN$vc{=Tsu)WA1C^_zXxPd6JhWglEkf3h%r@ z{u$z1S}FU$5KThHzObkPBUD&_y_8Y9q_O;XtQA<*4=sktFD!Tv??ciibR`lX&UUcf z9=-|!q3~U|&UqmhAw7tS=TrT%)*gSm?NI;oz3CP{h#?N75YY3a`Gl*-9z{1CG|j{J zkm#&GIXRX20^OAnmsFj*<6!RlHC0$Skg9gX?F{&95n8j|QlL-!5Gyz7sk;S?7ZwrD z=x|1F&A8#9Ye%a%-hR+!AYS&#@rV$VbDay~B;1+$o_Z-oA^HlNiLa7yCTBkE$MxRW z&#Dm?Lo>k9YXgbMZKXiROpr|hl`CSNYi*XSRISnkPNc*t?0OeTE4Lip(cLu< zA&vx;OG#O~9Jv&ztbrpFnoldQLs>v$S)SQLLe>GMRXJ=`u1pmFMZ~tbD1ORdTHGy{ zE(rO|3(Ec#6aTq8!y{`6r+Gu#isM8hZp=y*cv#Q8bi>7Tr}H9Vjx~^N_@KDeK>DCX2;ZQ8H2fS3~-Kwl#S=%0E06D)Cvhh+XrA z-}0ET%c}Jg-0ETCY;qVq(Zeqwsa%gqk8QD7Nkn*KBWl}(>1X~`Mn{Z;nbyzg8hu?> zs=(QCKs{2l(89<3D?PW(f0(y}cbwAb;y15nj*iX*GNTZOj=ye~0J%6F?kDcRusV;G zGhS1h>zw!_({g-{9WFTJ<~Z^s=NMSrk3tu|vUZdf{uYs0{~^IF8j-dMAtqY0XFv#e z(fU?Otz9l_g2gkmSCWjhKwwBC6;9qCzcUe1=JuqBQk(uB9eBlls~Mw21V1V#>|#a7 z5>9N|;pX4pZGzrOxD;)lU zO=(0Y;63 z5lN6dB*o6_vZ${NNjLA_LenxA5ma@L=6!Yrid%|#|9;L>$f5R^VxE%Yoz+E|N6ATO z-PMGQua1eCXS2qtBh~FLL3(mWt7fKeN`U49U!nBJ{#YqX<&`k`*5l_DGHci+J$JH*IUvJP-5~QxED9cac*~jz`l*0 z5K-Apbh7`dMLYk7=g7dh$MOZom99&-&tcB9U!8@kt&t%?D;RSItn`}VkIDr^4ezbH zvQN+5fwJn|wMdXE-o0YC2Wn|~6(^$g!y=R7UB#dbq>Tx|T6x0=Dq$JOFMfVPLw5mZ z=QL$28Q%pDhpQG!pQEC^wf=4l=z`-W>MdK#(#(sJ^zmG4@#m8KM)F)cR}QTPDv*2s zwl;wTQIye~KDY_>?~OM*x{N5s;z>Tk1?lxHqff~-8tg^Dq87wUiCj>{g(u$ugpV}6 z!E7dYftft~PNY&zwdL@K`KbVXk^p?+Ajumj>jfz|zOpIb2SL6|flSLZ9%7@7_IxxO z7;$7PbzA{XG*$O z$z?z6DN1v6D!ipQqriw295A>=!|F zao+(Wy$Jflv5V><_MZhd=5=i(FY$bg}4`&_$ds` z#184lFK9&a6^@YFl;uR;5g;?oaDCm~9r#m|Hs!jaj?qJ8^*MWZ(R17R^JxPtuZ$IU z9ra}uls&qc{^$J+&&~#T=up zoi05-K3@IlWzFh&$M^F};Co>y(?T8IrptCi@K5Vh_rFldw}D3IdqL($?S?89o0Zrp zmg&kmp3cOclDcG=OI7a;DVgui)mUlIS80*am4^!THjktl@|LVybI;alH1CcI73q~& zkHHSkuEQ#HvHuGI$=g@yau$Su|4zJ}2j+6Kk zJ8gPRb2})X4a~=o!p2c&;*I)|Mu0 zpN2_8KNUkt83Vvqq?Nn0o)uHBW} zY`s=jGZcDA&Nbp|pN*)pEQ{!|q;%u+5~KOGFJF_xuC~cAbhHIq*QuxxttU0dF~i+% zdcYII7%pv|VHSEzQ|N~cLhyDz#n~R@F3GTeQ;fXI?qp2uzRC=Q>rVKxtNyig65GW2 z_G7PQ7dnhhtlTi)=naVc4n2UoAoaV9NBaud+otq=C*2-rz^=6pQCs^1vz?CY>9=W6 zAc<8Md8zlc?~`@AtA_C`BKh zrU&$w&IXw^LUUI@-}6Baj^G*8fSd+#0DULh;AywqxnpE_shdN8@@s6djpZ15RXK`5 z>0p```wcVd&{bS1A*Av>m}m$d3LKE(FAlpx)RP-pB&!tkHllAEB>4jJCme;k-R&nN z|MFp!b9z5M5m-`|TNf`#C2^-;m`8*8R;M172Fx@QE99lkJOY|PlnM|(Ckk~hjN!)4 zfL(paMM|`15blr@LqE!u&4$z5VxJz`^sd|hdOHIv8_9gq{HQ4u^?^p!4#PHhhhO!MmZ_4+1}vHt$e?2~^B75m^Csn5Th2S~2pxf^5(&u8 z8?B*+-=N}pZEo|wv<{w6YWCv*MSppyV*fv06pl+XIA!zZ+?!V=IY83PC^Hz^+Q7r$ zY(|>!B8L)wbM0>&^c1TXtbpILKSz+rTmgjBzskj$RKXhiydrmOp)ke`Lk&zT1{KZPY1Hz1JQ5NWL_)BW z(u~0UW4wj#KEeGTQIaqU)@dG5vhjke zg)&AK#@l!mcLdAa=B3u^a-i(Il#Y~5LZ}>>9!@x*Pr&$l(nYTw-d-Q?S=EI&x4RahDdY_lN#Fkb|Ojz9qgyIY2IM1DT|a;^#-Q z9Q~-1;CO2IpFq|+5(_C*uMVk_0pi*`>fT8kt3@Kfrfrq_l9GKzNaOC&GdG#-lz?!& zlkoH%%sDj`+h^GAkkkWs|FV&rEd@Yt5zoF$g{$U9?X70pR0TN66d}3YmfPP{)SI@9 z-R|uq4~9e;D`M_A;!fOZmZo>7nyo;PIqfg66gGQ?2eq%y1JWyls(2ILY;$5#w+kU+ z;CKS&?1blE^7oB2mf}wo;bL0sBH0X(#4+m~uZcoQaPm;!f z0j3GDW2-%RzR9uid+RNNEK={lL7;+i1#sbx;%i?r`mfwlgi{qEUqxvfcAj)v80x;r z7l{W7OA12foz~5D$(B{){-`#-HFzwE)Ju#hZqQ)jJFei6`@AvAW3iaOl0a~ffH`7e zKwKIf=betcuPSQUF~@#ju&^DrP}${eeDlX9k#JOV8RPn#&XRcT8O8!RHq@tQO@={~ zH&J?=LGBoo>JRg1&D{r$6I(rNgt(gpm<>?)a-t>zSsj%AhZPpS0rtUK+FQD?;Y-1x z<+KT{)dOZv?N)%vQxZuM2c}Yvm~iO8q_8VVP(4Y#`G)N<6d0k7^v0AC(fBMzM{GZ< zGMOEA%7kmoRx61Z(-9!U+WWu(oN{FD-qBNQwf0YZAp;LmFynQzTn&FOCu-TRky8AyyK()-Yne2An`hfPib zT!YLcI-0PC1=i()0;uUemkEw;2Vnn9S<}S&hrA8!F^n~3g?2Aj4!yuxo6_H7i_|HI z4Th{AET(PyWPsLCi2Q+QGq8vL)PJI3^c?3MajaDt#781$PBJ4#(fCdYbm`*;IIobZ zPraL-!@Ed`TRVU1NTbtZPHS|wCmWq@4lUI0OIP)a5iKy5a&jI51PdGVo~o|EqtIv% zN$|oIo%mVMZqkmJAm6wx4$PLPW_KDe<6OTbkHWjM(|T2w%9^7bQAcj2@tz+f+GjY` z$mNs!OYA=5m_PbgzFLHl;|*f28lCy9Xm%D*dyv*LJ}6D0hf4g2L1p)SgbZ!ZZsk}2 zdK>APXx30hy*HS2zMsZFRhYoDu%Xa1&Z8)&qPx=+V|5b~1O-X*N0S`3a*WKahN)15 zg#yO`j zU!jjl5{OQrtFsOhF;V7a<{@pt^IOu8o{gB^!ZZ8QCydSP3)GlvN`M95&z&-TEV9Vt zjKlIiq`YnH%6*TL_2)Ks>nlK@rVg&(RP-hAPFBN57WU+8rM88(tB;|4=`4$V>=#4H zpuBl)(D!h>y++96FrX!Yxl7XbV8*-GVHeS}L8W%Znmc;cTIMf4_!qiGWDQ1iL^4+< z_D`W;qrIx|iF>1yX*Kw6^9^r|A~eug1s+M{$KI0*^}kI%Qgt;VL8qYq5sqX5rP}p! zLPZhblMMPJyHaXP@mP-+-WAy$R!-2z|kb;m)403n+#eh=TwASK{#W`=n~oUy&NY|==`)7sHHy)i(9 zBlit$oU^0Ay&SFHI_TEdNg#K<|JaG2eT1cq{_8yyK6%UJwu0)DlzUwwJMy}aFc-q8 z?*wX$B2%3s=osVT>rpQr(tpK!O%enEb43FQkWc~1dvpy4i`llgf{KpaM%8i@2uErzkIJ9)F%n; zT|Go-r+f=~Q-m_}v&Z=uji)3rR|U*A6D&(nGa|tSenuT#`pHdzA1d-fNn=_miU(811Y&^!O!MJ_@{iYvnBxeCujJbZ z!WZz2iP~wB{7j1}W@nXi`Pib7S102ArAO>o45zT|C6w;ON*L|O^Lb>=9$?N~lU+L} zSEqDmRFhZ{#?mRx2ya;}&-mF|;N;V6&tYeaFkt__47Io|kK*OK%}NC+W7iAnf?zo* zW_l_{$)w}c<1Vp9%{E;E;DfvFMcpN-dPNPdhs@?u;ww?t2(*ZVYdjmXw}l-^1hAy3 z%LhKt{vFFL-Tz0N7TbQSOApjP+O#;x>T-MTJdzVWbriP`|S++!}7H9tYkI7 z=(I7rB#kj)U?qFIQDWONRNVt@IL8yjRoQ-hVuKYaxO5fC_Db_viX~8&8CsINSH7>d zq+7{TU1xc+Rc$VNv)fI0d90!uBn;+egFI{kEh!r+2cyr|vSjxKz6mX9E4Y=r8`F(n zT2TFXR&ljQwE2T#bJ0;k*X-@d&4PRL+H-4(Xd|e)DpKhUs!Hy5Oso||dTfJtQqg4H zfBti^A=ln=nNe+F|F79!wW*YTANC!0XPd82+Xj0zva75b%Bvvl7L#QgXC9_evHbxl z+iZDJkgxvD?uFh?936WDA;!2a9@a1U1N~KkiDYnRY}xPuN>y01TUA%^&idHzzU1y| zX7`;{$zjXond>tuhAX5Ua9|RpePY|c+UE8QL(Yz<8_Cd+PaGh!uBT5Ek@mAw=kJT( zPxTVMUyF}biP6F-o-lZ8Ig4gLtqoc#NAnWOIsn_Gp&oX2L(OQh^#@0UeaCs zLX*B6k4^mX-)qeipDpMDx4#d+WVPt+s@LwO)5anLS5K_Px+CbU|Bel{{`oaD6klWk#7GtmMbWk3R}Wcn<_8U=w$9^ z%qG|Y8tja-zf|a&q)hfjD#UT3g1@BpI-SCNb8GrWIY+ZJ#;|igIM!h9tWF?M?_~mH zC`JvIEZa!AD!1HooUnar5Pg7wx>m$Z!tTt|;v?=nNDt}eQXCM-W~(uau}R9PfKop& zJcCgDainJyIX~At2+%|#`s!=aC0qAkW8j5ZK!WYz(d!{7oiqeb$X?0002`r5pBqfm zfPVP2dyvv%s}wuxsS8V*zbfqR;lCD1hbo4GJ8nxBk@$DyYkTAH{$ni*uMgN3G#bTM zw8^%$6Iel(fO#eu#9U7O__KQkEOD<~Ll&|Yq$+AjUWz4IC8d!&jZ2PdOdBahIIKDt zmZv8}$8)bsuJy}TW*aVha8m>(Iw7fsk>=MftP;VV(pE1@2JntkU>2vm%?ylPUS$F> z+ju#%bTJ>TAr#%_lNGDD6Ekr+oreBV9IC@a-byQnP%tJvqj6(~k^#BsPza9jGcKys z2$_-Zk&-}FYk?(uEQp2&;b<+)wP4EFice@NHhW)bXh_0y<0-nv$n#Sp_?vQ+y9Ahm zFqp~&SJ5CL`$2{c*5^3RKawRf<0f8@qQ)#i1pV53IW9^VPw` zY05QOzU?)J(W)^7buY5cya0q?NVS4oN1u8u-Ml*%8A9`l9}0J?y^8>L2;U)-bv*}x zFl*0i@hEJCY4isr{5y7)vxeGdtIU-Oi~q01bH3hh%$gV@U^x}csD3c$Q>pnes~-x zNgrmC>~dAl>Ow89!MUB(q*tCI-SF9_p!%2ny_J$3OUF~6@Xf63j_IoA4%q^d7aTP= zb*u2{W8@N@R4LGk7r{kB9ve&8T=`NU5t^HWm?+`oejPxeZJI!MvvUdv5-VkO=;1{k zqPH;N=|ez|x6t7X)^F=;lraqCa6}T{w{hfLl+-eKP^HKUIDYW&>`$fRzV+pZko~Yk zlx-V9S+Sb|%XjmvSBt6{3(0Z5_Ygj>WNU(#X$3mKdMdv*IS%8In0ZBkzf zP7?VljJN7#rv(iqDb~M_RK_E)hiYEd?baP~8cbBJcInrgbNV@U=%Ij*sBXf9eF)jAno-Su zyI^gbbiUG%nm~T9#XDb=8pM*l$8ki+RYeLA!T1uoV^UUN_5eZyC`%kwtFO2el~!t} zJegN}ycfU$$W_tLZ5k%pr$AYm&jJsmdmgv1*F31J3Tm+vRlTB*p~&U2qX|Xs0^{%T znjFY;pK7A%9mjQhh^S25HiU^nYnKc0(Dl^3-A8OO#kSWstY%nG$*RAO2Pg)U-1My@N3 zGvrV;hDkdJ8=)is9t-fL9*D#~2hN(hV3n6i>tunR!Gb2ra0_CBUNp$mvO1{* zl)Nc6mhB6MCUeNVy9yEjkSt0UCr@@g%M6aA6kD6v3G<}ahq`0p2_pD4Z-fw{ipc4Z zaOo&Ax=_hW((d7hnMA3 zJ;RCvM=!;Iy^cXL28FZG(Vv4GLb!l57$UOb!ZPf@MpmHXw!RTaWNwo@#TJFj;u_}a z9`HJ;#Wjt-AXyQ*uE9tUcbOHt-~CF4 zy=UKG8`Jz_strgoMDk&C#eTHymL;w5@i zNnD2udw$hN?=t5-I@s`+bvODD5B0m_NL9M=IAdl3C-BX%^rAB+KVLvrP|nuSh1N7l zq0wBaN%}CMkfz3USXCyA#t81$r%6pvMyW&_Mn=lraiDNHCAx&L1U{%Cj~suENvrCqn23v%w^M{j4rGquzX z@Jb=Z^%Qr(?$@qSYc_Ox?n0T$0CEbjvA{6Jp2!cW`lk!s8%R;JiP!~fCzPR=Mr{aM|^BYBJt5&9Ev3 zWR3#=B=k%1lL_>FF>)tW4o%}-;F8vx;i$6MIh#{I+Fkq)a3z2#n8knN>Gp2}M*$=1 zVEoz7FB=!|KgLsU4QWUi8E|LiuHudm1zOl|!;bdwYAr+sXF!_RsfIEb_Z<6HWWACCt|>eMBARM_ms>;+Xnrgn)dqLHP??OOjv(DVQF&C?#F3rSGGyds{a2@Pa zg$3&nmDNUffv9vFWyr;B_4+&YzglZn;y|=XgiVAD+}M_$yoOo$Pt~=nNmsLSfbefg@$lkrUqo;jzZ2Kj5OCF|42Zm2o{qWvK;R)){a-4)TO+tYdm?7P@3t0a?dPM>4t-7~$qTN7 z*{U*sQ|ASgw!Q&vNYV!eV32Khm=6O@n^gAy*gB`hN&sYw#x^=h$F^8 zCF$Gq({xC*b0#9og2~vo=0F+RmFG4+b#DaD0ESt+39oM#ircq|Kd?t^&eqziFWnCT zZ64Zb!hJcP(s_-EiqTzEB_LOtB3gGDL$LekkMxVGxMqKi9#LDKA?iXwaqrC(yREcI zxYjvcB~ue8EsYq&%P%0BOl;aRwuZ1GnHGmZM`nf7qQwOR|8>tdVaWja!L zul^P7a#2E0K;6%wlZPbpA06GNZt?8GBoYIj+$cx2@BSh!T@;wmS^KqbI#Cfq$%Fln zH-DMjifTml`HhX!T_$|pI&rlenT%NjhYK^915oDOV!_hH8m#kwq6d;`i)IZEFv7Bf z#Wdr8zCv9j+yqM6%dx{QGZUrCFBQ@yVDM)4{i80QVq6Zm zWXGa32r^C=^^Z_zn&MSgt|HsNU&nwZZp*Qac1k}MS9D&GBpFY`64mrn$0IS_>4edx z5K7RQ)oCM`;!1La{+h%#@i;@jLUbMfSq%Z!hiM?abX=eC%nh+-VI63IP>7c;}aDkD^U>aC%qnhd)`0Tn5cSfq;{)0iJs=X z!4FB!SVJtR3cIwj`eBa`-nP-YW?uTX-*2lcfu>76w%<@x!5!x^$S->{qkhc|>ZFg`9};6bv&8*oIz$1Pw3c7_ca@3>~$2UlVH%6OPYEPUrL9J6?`q5+B3AS~_x{m;Q@*G&-;!$KoFq|fy!bA= zSFW|&?~+lED#`}p%IJG`)~md$1{$_K&?B_nvTv2F~b(6lE&^kba#Sr8^`Ff1#_6rMr<}YBo1dfMI3iS zESTlY*cW}c#j{cBX2)o1TMJvac+c6w7#&h{=_8dn8gIMNSoSe$`&8wWNn7Zk`>&Eu zdrFi`96!SI=YNCq8$IHC__!Arr1=U0X`7(6Na*_iF^f&fLnfGnx@Y`r(iv>3<8;KW zs|P+t8t#+E^dUj2%)2u@lk)*JLkb_fyZITte8zrmRLbRj$7=xxL1-tJOr=cZE_ZC0Qc4ui#;7FOu=1VahAA-N4jxttHm5?ZReL#KBmt9ABA| z5{~1tWm%-XkqSDP(xi#NQCj$M-D9MaR+o0b^lxKek6)n>ptj#(^4g+sEC#0*h*|$g zx#SHdY|kkvGvuZDEE%-|sWgB@pZz?*C2^_pfg5J71}3_i18OVgs}{ea>y>3F!ZW_k zxp>0XVZY7Cgq*SMs%~L_cPt3Z+Ao;q)uhn~)L)p(Uy@pUc+SxxO*sLZ zl!HII<_y`ah!siueaB9PJVu2T=ZkW!OL_VUwFoYLWS;*YidK`gUsHs3Ex+HM*?@fbS?nf6m|cY9QK zfyL};S+(mda->V+HRai%2t|G&Q=wA|^(8sxT;kP-_4=erl&j>?+q$p&*t8|~F62!3 zYgW5(kF=mN4Nyf6;cr-ioJmV8z?cA;=+kp9w&N$yImoFiD!(MDWz2cllu53v3OeVx zvNbiJv0o&;YdcJHl?_)*sJ>)gB)R=$xb%!w4FU(y%3>p#@R|G)q0fzvt>Bn*Nj`Np zO@vVZv-xk`ZtgFtd1w)%9wBMUMj!jFy(JY9I8Wn=EV`l!`D&ONe(6mABZ6BgbfQHj zmGdQYT61$@&n$Lqz`{>5*?S_6>`E{vpK_oestT7cqu&2crhC?ai8no}bSnZ)o{%2b zi9p0xNIzFSH8RE=@IJ#+tjxuZOCp{t;w(2J`N12wXtY6TqHhQV{-&3c!ukHU7X=^! zdu6_1SpoL<{-dD|FsS%qBIZq}ms+mK!??JXpt1+tlG{}Lt&MPXLlg9%J z$*Eu44qs9v=VV64!2UDU5M~!k*o0qswFKD>8vkQmn)?yHk^cKI;;*vxBhs;1q3WIH zVZB;I*TIzbjGWBbQFYuDv;zqie};WL`Qh7ppFEp6m%=Vzp?7uq;c+d}fhIZUe9DQ} z3gJiq>uZpTq3)?M^I^q1fAYWt=k0$z98LdfwBFd!r{}{){=JYM{14V}(YpLk%kuQU zQ&`?O5GWi;9UNmT5aA*s!`E8c8b(=LWoNg;W(<9B?r0AA`a*(K36;68;Kp%>gt)jk z)z9P6(NV46_aV_2vs&|~{_cnBo7-JhyG`7(``tfbfYmi(sbP5S$atsg2J7|X-D#XB z-nYdlyYTM!?_BF|jrQt49S#)$UH59RPtPxhJRe!VMN(%!R6M*h)MdpG&-u{w53M&Nh86X|J`A=1OiiY(2ZvX@F&1 z<%Hj>?lEAT7g`wbreXf$_V%o%KJU|@S}`rX0r>N2%N>fV@(n@G{bsxEy!pp7!>`>k z>rOsBEDbyEb8x9QD8TQFug$m_f!b9&`ndX1t1{2x*0$`z7Y}f6^Hy2I_8n(1-3hP> z672c3x~Md8-674n-~_ntNAr)#8}j#5V)Or1%vp@>otE{Xu7Q>gLH=HoIV9Ee$kz43 zRok6uIxB#mvGCz|So8GrUyNPG#fp*(^r^ZbL@SfcHT5iFn8#z))$|F%H!S@wx!=o?=RY8e7;bt% z7)~4B2RgpxzDn7GXq!!8)+sqFHhBdn^^M4DHT@}JamP_;mRw4Bh&Pz**!1wZ)EW_} z72o!^B)6U5e0XV_J`)XhnjG(3u{py#m8fMnnqVJzb`lM(p-D4P1E}%n#{GhbioGX( zUuzvVsVX<^MjaD!&X_pL9C^}KmH@p%#H0}oCX1jtD zq=e>7-<4+zBN!bdjx;_AU4tT5_E(%c76QGpJL}B{Cxbp(j^R5)csjhEIFDci@5nC+ z0W#0p8i6IL3g}=(9ZUDc^W8=eg-glI<^u=-3F`2x#gx+-595TCvPB)dgvY-J^tAgY zzbu3Kkzw>P{7mskI2GhSzdfuy965a1Y1Fq>s0$0apd!_;1cJ=q_N$AKWD34HX@&@Q zT?A=E2XoS5Pr6wo$IMYB?h4myVQ8tq_`ZfRV_Xd64EIS6?wk|>YB8)KQxkJWkp@!w zXc78ilp=QhtQA%^OeGdVkfkA!IjOy40|q|Im{X*qW4-XmIl6Ed*E@WIQ<8R##?*$F z@~c@_M4>u`$NWBO;kaPf(Eq9f=O#Lsdl0e4U11V~bR;KubQvr<6LKD`3-{br)uIy2Lw@fFN}4W+WMNs22}_To!}kq&)Z_gN92q3Q@JNSqkQ8$X zF`{~qstDzZolS~`c6MWr4qx9sZR(}H2s!1Ua z`KoJVQm=|_bY8qH*we^OjTu^ioje*{&^Sf2oaV#ykS7+Bubd=!cJ6eF!zMgV;sm_;~OaLVe2i`M)~*zKWRJA;GACx-w^PxJHN6X9&9A^E+ag9d9im1mZ4LzeWwR;V1ZcPVx@11# zSNmDPXBsfSP~^2 zAG2Hu2K3vlW;wn>aMNZSsq@{1zIP@UnkeSBRi9wpnYH!9zG+aL}Ig~H_ zQEF_E3}IJ%Vw;#KG|a2qkK>e1m1U;%uYb^{c8DC|W9FB1h@L<(%aqZ=4B+ZHN*AGP zG=Sz@7$PHTjJ;;mZb3UP+(L=RoLv5AtwCy(3u{?rX`UB_zt%6Jy52X#i-Um3Ya7D? zy4T2L5^qFcP&bLE(^MXC2D6MvBX7|kFO&4qWs$YDLGEkyv~k6#T%}H1^Lvn2vb%sl zQQNmhlnuUXG$#iThl(@^zp~m5He~F4hvs0FX_FY9QD9TN`kYWr^L5lt@i=cqB}HCG?nJZLn6T*RMcdvl zMs49_EO0GYX8<%)Ze;~DdqYL3Ouxs1nDQ4Qqp`>w+QJtnf8l#*+4S#Io|uEmOfOLm zH<))Jz!fw!m^R&~Kxxrz_u(~CafU1!l(O%hdtZmnb>DK56lJgvfPW32#ftF>ya`PJQD z?jiZ%WHPR?OVWSYg4$Tl$SNchT1IM}#1wL5uM{#zq;EtsOr9wEJxESg)fjb0siwVA zo)@8CKVGL@*@4Twn@r%gxpP8!OL53-fSG6n`OP5Ce(mkcCe9-tZosNom5b1u!`G~P zHz3!6Nu}YMnvzgiS`e|4hNyV>#c)$GCK~2HU_;L*&hEeH#4tB^$*qqKC0y_Aod_e* z8NNfMF4iEMa#HcW(#gAMgBmM96&)M$yM8@JM8(y4Qtlpgm&N-$r8%po^w*bfA8NQC8?l z*h!j~Wm$xDl3-2tCo*pS0Y!`*Hq88QX;w!aFnTNpdQ@S`IpqJwLZAO&;eS6Q^T47d zvki>tK#U~%=0W=q`K7%8=3>|}EN4dPJU4vfE8UgaJKv_5lGPU#%B6CumwDvmJJ!G7Mv+<{soz{8ZL&u?es2#Ei2?#xD3LzXM>clKvdzrAh%_ba;i zhz`qr4bxnq=Jwi+ACu8rPrJnH8X@}oNyi5AZ?K4dIfrA|wKp3Oc$T|+YNNB3KMkQ>zUY6qTkmOa{PDG4H*IuUM>np8qPfJW z@7cDVI?h|r+hBI?k$q2_m!mG-&qg8OG)F0G09MS1vixyP&(v+7W!V0upDu{0h2$G} zsYAn7?cT<)6?VgV+IPS8X}K$tyxecc@nyqo%MN_5+;^Krmp|e9gJ(-}DjWMo25Z{= zFCQ_tT^DkoKq5`|voGc#LVkCl`kd+aW#zj1<@-7gga2t@J!5j3-WqcD6>3KG?$mDLAA z(7b9`!7lq=(!0QmWS_5p=aTqkxJOV=Z-92DT!5cjD5u$sq4sW~pA^pW%rPbkSVTJ6 zh_;jk&$6@ot@4C?6A2s|CK&Dj629FARQn&i(y8-JRx)bBH|@8{QCOaPZvOIws1Kq5 z7(mDVC42!$e&-4IxsA1Isap-QYdlGpOxxvck^W2W12t#<%9LE~lV;OE47{f6q?DV( zB4X1T(&D7q$TEX`=dTq_l`=22zD4vDoS$9S%cC(fJr_e`o zFn8D#Yu^U!X&nSeoPf)#X-cfNzNS(@4%jIOhi$QEz#MzLRTCRX**CEX((6 z!^0RxMrg(n$4ej4Q@|dDjPPG(yoSKETruWJb%s$x!5T3?UVjFMN>cOBGH&Neg=X2Q zXdqO0r4Y)Ui7ea|X={ITk$_V&;DUKjj8Cof9SF5oe89l96AaY9nS(@FptT-LrN@7g zF96HINJ6)H#IJyahTRlc{c!Vs&BY9EEz5}l8|@-T&xzM-vnmk++eYdWI&PthhvbgD zQWmmj?K-{NXHjPIh(dZNT7u~ZuM@c#DHaHQ(&C73LKux7qkqNpFc~UHJ6oTWCdcRi z;AWu6T*mNt_`zv9&6q@=Q*Hf-r22#9cnm;Xa@PqUkqI_=R!3@Xe|!aQ1+MBZaHS9A z!1|9*eT>qrw+r_M4%k#9PB5UUnVPg72i>l4rS0h+UE?3>C6!|wDh+>HvfgXx@!F4IB$F$Ez z^s&uOvDgbI_g}0YB9JjO{ZvE}I)06>^`j|@$N%@hBg<|e67j2Tk zyuBi+8wQ>*A}~(edg{XlvF#@@<><73{FRZ;ArWUKvxs7+S)|=p#ACGpIAwZ%n^Q+K z0Z%4xGLKZ0G9J0NH!OzeErDai%3o2JQqx3=$5A8tOVc`pO3(yZ9QA#ql?f-?e%Btf zgz>Bgyx?zw!UuyLz!KzQf<8;FY|hAy;z{ZKTj~SYKOxsk-BCt3)QnbyRoE^Ha^h5`0;OXdhF zf=>xEk~L+WkNsFkumBV<9XK7B2+o3(Do}7W!=C_+og!~w>gjLka-`3$kdx-xX@4E{ zI~lvJkQP$)7^t8aq%0B}vPlMd+eLqYt#9J{-2MnIp508L(zP3LZ`?Ohe0IXWx;vyc z_N%YS*ehq1=?2>o+zl8PiIM8`G4h%#Tx$f8!#OETVRTzy7qVt3ZpL2vE(xMo(RuSS%1_Ut4 z3nUU4>Q4Kl=^^E-fZ>{Q`YdUO@fkPx0SH{3SW*kUX6xjC@y7b>DUmc&ey|%XY3b&z zKPl4l_BZSFsTFZkyIU3n-f?}_@358-lygP`5`BUuI^4jfNsG|KKYUzKJu~#}iy9XC z!*g%pA8P&ZG{Mb|$E?Z+gs}V|BHI;gK~|P4>(^}xs2IGx5$P_NW#sqF3)3PQ;)kHJ zSzy-14yhUjd7XEpu-~v|-Nmk7+1G%NkNKkMZtmkI`4aVF-=D ziN)mRoP!6lFq)fTdAkf?r`Lzw-yM+D*X^KEO?O)_@#=UXMjd4XYZ@zme zrj(h3Z*01VWN30NxM;Din8j47ToA+&X*y55q27?{>Sd=5Xv?1ILw>(2^OS%enm-^+Fbw@3USQf)oX5JaZM1%>lR_v#jn#^`XyD|x5VRKJyTbuLRc1Ho>lzJWmH|!e^tqobJ5KpqOnXy?LcDXvRzmhEGL}{|Ip?YqGtfg zeZJ#8*~f+T%%*-Ol6(7>aYgUTA_=nE;oF0S1MR$<6~2A5TBL)XNJj0`j!MP1I<;74 zBtz~I7iS!^wYwJv(Qr#WmVuxUJM8;e2PvUJ#H-Lj*vF7hSKp(@b}nCTA&ZGr$K4*! z;-vb;7EqJfH?nA@z<0mX+4d+tz9rd~z0S52>3U93Ox3@}q{>dBJ7}=+7fX~Q^3Jr^ z2HRQWP4S8-_xrzQPKx$0oG+Df7toURsl+YI>nYR+_{gcRjv4#=_PZ9=PRL^_WRTha zChrB>H_B1o{!Sq`mDGKX>+ZEb3@R-G#1A$JyAWa7mFT0Ly}@G1~!fdf@6EjG?C^B8Ws{UEVZ zRF+>qI!vd|11e$KLKxfU{s{+pjwH%4q2l8NL_jqIB!48WN>Hf11Tvh&g?;~|%_D2O zskqMI>E*vZ4EZR=H+R_M=eYf@?; z<5L7mZ^9|Bg^-gy^ncD3AcAtVEH3!?9D~#ag01D9gdXpd#b9%=F@thD^SwO#jwao{ zgPpsFQhYW2h(-Y?mMe%xUmPK?vEL9>Jy$%faxtkk2M;86tBol$I)nMGV~TEU0-D9_ zzS?MfQmqxJO1Qrj;3W~q6EIb|C~t1E$X{J1^8A!=c9}?zFp+w0KUnqi9|H(~WCY}q zuPRI2&6z%{cMqr=R+Y+S;Yx67 zBts|D(k4NjhH9aDf81IIv0UqI@qWslDx__g&XZHIvEazTu=U$du zVVA?{4C1$naZqey2Q_gTV=YKQP4&1%!H<#lNb{y{X{Qr-4S zVF9Q zZM3vqmn2Z0voq)!(!&bA_Xt52~t+K>-+|5L2|b*is^C*Di(F;{cpstmV3igth*753moF=2IPL^INO>55?h24}K?OpX5Km&$E}i#^2gM1BDv$!rQXQEjb&Ma;8CHE|oXL zG@55-+HFH-RujNewS7bkXg^F<7j71dEpLp=hR&yGHtm!Xve0Zj{^;)C;FygeC0D74 zchnh7`!NHva~mrd!wu#AP;cM3x*)GoW`Nc?Kx zCAVj)WQTi{+u|9CxZHgyM1;Cy&MyeR>1&^mU&HN-k&cqwdzEy>d5m*bGVNEs5PAiM zru5J7K&9`ftAH%JH-9+Ot20pt1ZnUk_g&6Vq%t#&tKS^W)*Y^>{h*?)GqQcs6}YC) z-CIDO0JmU1Q4=%JBC9T${Xn39#XHrJ1k>X4SZHCTM{8c+@YcincWp=9aLNSM2(PBt z8};i(>6jCfHBm)ex1iT8S7alZnl$FpM~uBom!8nt6P)k!FQ|%F)*XfFIw|%5mfT;CAiFowL{9fjj_HLh0|Ar$4vc|`+FiW#zbMCMk zf($S_IE#*MZs0?;xO7B5bF4R~30O}uY#JGr>XsNmfjgW^hr-V$? zOZ3HXy!bU+tV}Kgv=!pPU-V=I#o4aMFzdI;K$cK~OtSsQ#k;f;l=Al|)CH9cVqn5X z$zV=SgN~7#Dk*yJ+*$DcY|d_-FgB`?L`_a+Hly|t{Cy13%3-`Fs9)%reZAk1jcSQYJMa{ox&vpn{=CEN;+wUszLC_G^o=w#>Xz6sS)6Lou z&>J6Md-0g4rtxG93{QfQ%+gRJRrQAYZtf8RJ*hT1wECOOHN1;SfVEMJ`lmb2{^Y%- z2hrg)TRUrW-l)BnKKQqnl5)*EZTFNMGRfxK8zB-DAQpP0qfsl02`CFNWFt!y>jLQN zw5*W2>D@qd5YDK!BRRCT&7h*&-Hx)c%qPE{xRb zr~GPj){ZB2hyc3-fK)$UjqFK1L?Q_a;zY4Gd9HqG8E=@U0STZ+<={oy1Y*ZW_Ixdn1@ZJTZ3jN8 zpM%;lGoqVYRG#pJcL3qTjdaGGfZ&SZ3F|BpQBQ8$`-R4032d4{Fd#Q-ksuPRP4+`6 zWn0{5u7}jOd!qEq1%Dv@9Z?!=jOnO$ez`SXTR1}zSE-UP1(L?g_Ujn!XkEsFdm3`H zCtpfi%}w)+Q8zRXSmDy@O=4#AVdsoMks2Wo3zcB8?uMY&G32f=yG2Z+sg;{?5w89b zdFJASPbVoe8KnmYM7M9=U}1I-_rkKNrDMf@{6fLN>{WV-;5R}>2zIImlAKH_)cQSk zGRvqOXNdyhBI`5C6%%n$@*410u8!waMHm*a<^^z_BA&np$ z%xp4xD&M*<`iMe1PETPJheQ5KiGoz=Cr9OQ0*`A}T@(?F3#QY*`cCWZ804G1P74Wm zrQ*wvG|hdVAR&OO!qXLqEAMfn6~F8~GBpy}F=VjtqV*$$8aBhFQoBAu#98hccd>ua zEh>Kf6l%v{xUCy#Bac!4fW7%(eX6tt`JA}0D?*QC4DQPB;rRO#G`Qv8nfT8aej@^y2bMgX-pn4Zle=vtj6xr|wjEaA1DWI+MT7=dJgSi<=}MPnynrEP z;W;~(cSHxa#-f`%FHBZ=OP92lYdO3J-j9@v1RFus2c9RHI5NU*B0kq3@#&j8aZRPN zt+pl+}C@wy$=#%)mrNgOHZiF9f?I1)uf^` zV~kKFQJ3?G+Ib_CMp+4K;HpP_6>_}bIyZ7CrAM(HqcP&6RK54Lt;k=-!P@fQZ}N~l z99vj}dx!GHj$l@;D-tEEV8528_HQs61A5lR$NH)~jRUiy%9iYnNs_T3v@%yL%_J)8 zsr-IyDb6UU;<=A<-5HfJfuh13F>gEI4-$20Uy_hWyD4!QOp88yLVzbmhgS}O$t;rN4ECJRZ*V*oWmY_>OGikWJLqy+4vF(9cjAl7FoNH z3K8u5=<`!>3Z)@6pow`ybWg3PUeuB6An#ixji+x|B`3Ii_!tred76))TxbS}lc_)N z3bBNkCu6YEV;(*}WQvQZ{|u~P>Q#j5Sudzi8H@Gq%U%E)7{6~Y$ewJz7F&ap(cZ&E zU$3%}Mrk*Q)Dxvbrd`d`Z#RDM%an@;8`G!LX{c$5Z@`)@r}yR<;LvUizg0qx>Vl72-vJ*!HDNJ-!YT zOfvba6TZP7JYd$0$gbTP>)Ml2bT|Ml2;V|O#T?&qFF|C!r?^D+Qvm@!5act>b&85+ zGKP3}EgEx0VvzJl@dn-$#An@kM?FMO?F(8P8~qyHZjCC8lwT&p$K=F%=dl3ZZ7|28 zgV-c*q}?pgKPdb4f&kkxok)K^ITJ;-TOg(;6=gV{x{vvdcUp=<@0t-7JSRI3X`0Ny z+X>UdBi_d_@n5E-y;&x`)T=m|me7WpRnOkX!2mx3{an9yMiO(d7+6aElcaZsImWOG z^ah+V18Z;FD%ju`EcTcty-*Pjwm0CX&5)EK$+)=L!99&Vkz3zH8X|9oDS>C9~)Nkt3SJ=|Jfa`-(qr~MRe-rJQv9V zeQ(XA(ni}1w#bc6NyQ2JQEod2rT7Pgc=gv!n079BRm03~eKqoiu%v$CM4s>Ogb3vi z75s))=(I0=1v6s6Rg@wqEjjo(PmlqE+dI|O_|MNjMEzw)^8Z8tNCHL=W%0NtugS{iFEUL_zA>YgiZ4xm$ONONG05afiU|I$Bzx1M^3`;sW{JoL;G z-2}?B-Y@fUNqoTHyvQ()Ug4_RJlRoLG+Y9bdfG(ux574kgC2s7zqlBC>s%I&6onHE zww@_&*kauGD*FJX-!W{Omw`q9{8Y2*K+ha(h|~i$_g0$bR}vfySY(Z63H;=rRwO7p z^{@9ra(>P_nzj>kAsUU4v-Z}o*O;c)<+}aPz4@m4;nV|NY8h8m|e;6nFs{P*2LvP*54<0 zX9u`-1oIR1O3&Bmm_GeFyC$Yn_#G`929cG<`9dpH$`{&ePjKst`P>NpQQCNh-Hrp(@4*!~^ zKp~-Tlqj}SzuV4NOokS)EMTqb1pu}8eR+uykt1LSd_&+IU>>QoU940&nGrVZu0Gm` z!?%F}@t#J5)M9FiHPQ?wYD|&n=OHf;7t_Dw8TK(gUhBG zibRa3ecpq?;@3YFY3jaumH|Vn0@11*!FY3buuwo-swNxa1@|508xEr@%&pJ?Md;&P zF|;@iCnrEECJo=?oxrm?rV5W@#TcC%v|+trgh9$uMHrbUHwLAzcA1!B+>33N0yxg| zxzsT`5_El^{x6jc)87LSraX{wYY`D^>f6qYBi5cH=z0kIllFqV28ix0i9uZwl>!na zN`@pDYigg!+NMDp2lh&n=ex49`koUP-v$raParSwLS@Ed(sV5+Hu--CLAfXtqEWUx zDNv)Cw*(o!pcN1wm18`JEk~9dQq`D`=TGF5CM-Jyoi6?qE>tzpo)Uq7_V<2$&9SmG zQYxFGA~FYOpnX`W_pHZ)ukl%yDX_yYPeOoM9AjP~drL;;XPY?&#y#cK8RYEB*lF8b z)^FLypH2?jz7W|uVrczn8rhQ2#}9B;+IH8d5sF~fneB-~R4cJyU`Uvd%r+eaVn5kH`Ig!%?e@o!y7CUpdD^Qpsfok+SXzqTs*r}#or zVc3DjS`{^%w*=NHin+mpj2P2_VY%L4ViIC`8u_N43Eou5)O`wGtPO`+8nAD*y{P4) zx-O9ua*ncQqfW=5J zI*BPbe0yclj<5@qy2_FGpuoTLigI2t8`5}yO;1E;fp93Ije@zHWH0^{`@uo_qRw=p zDy#3;nf%LoUc`IqO5v zW;w@PJ#%%H&1hLtz^YOoQ)`jD7i!UM?bT2wUMd680 zYw9*^r?bp4XkKLwSk9`P|=#Gqm! z+1m65pLzz|#`5~RAdG`P9xh^DhvarFb@F6dk1r-{ManQMuH%n~MF%l``=mRFef5AFqhyPXqc30(%8Kp@vv)13Kl`sO`% z1WD(fBv|}*6G!HvV@~{Li@eEr#FFl^S1cbbRdw{$jqA1uQW#+QTz8X>c;y?Fq(|#k zLOEiv74^8bsmcq~6ynrYO3nHck&oH*>;=0@WqHvyxxvA2-8rJsmCT%5mCe#AV`e7*u|K0*nOli0 zv}-rFrbX&V#E3&mMLAE=DJ11ryCvjX)6>>Xr)9xWV6tNN%b(pW?=aM|&ytb+sQmNj zE`0;xlp5DD(BhXpS*cmpBK{;JsC1({nQR%Pi9upnqDlI~PE$w9+~dI@lTfegX)+Ox zE#8#0GIGKl++M_X^Do_13_^Vzl5+YhVS%>PWjIm7hVlrF&*Xi&nK062MkasPU~grW z-eUz1@f_LwPxa=r7z^N2t3Xf+I1m9W*?tYmG8m_-d^7F4cdDdG+KEqwj?nT02e^XQ zJOdICNADRqhV9D19TDEzLrN3SZNxFHfwdZDGp-O{28#|WYNxfc29MXuNox132sg=F z<%b|O^fwr2$kSCW-N+JXdlR;~QO^x{h|;ceS!Sk1Dm(pRurt7i0b(nF-=q z4Nh=jz(f3lL693cC9Pt$I#Hhn{n{7$ccVJ#vsOHj{c zWvq^@p2K-v`LXYEy7#y0@ch$E$GSA(jc>gTa|%rQ@Mx&$m-kzy0|9TueiKa@GwrD0^6qql6navf=A%mJGDdK?^ z&lsJ4@4ROC=j1KP9{FVq3;o8gKarsgc=O_z@r<($e_<*}eZl%0d26DoL^>}Uu(v@b zlIa$9?xH)n2PMQj!IFM^+vfrFC{T;%9Bl|>8#`FJy-g5G{_Nn*7Ui>e0j9KhE!hTG(35gKc&-t)!uEQp?U-S8kuJaEl=yvbR z;(y^_<6qxz4-@*wPii4B4*JLcgo91bMG2G{?@o0|0}yC#XfTDb03a-Wp!Rh0gyWWK z(S&F%BF4BLb?wud)cru=E54=s+TM!C+V1Y|pYP8&j5~RXRAYTNW9t^tb?55~B)+3O zb5Yfoefqa{9fSh3T*1<(@x2D^vvW&Nr zp2+62`dt;|$XZ>|ty7(5irM=Zs_8@C3aE${*#=@kuOF+s=zj6hM@11cK!ocHEn zf#K4*FiE#hth8VM^-IL5!Po4_pDYWV!ku4Dcsn3rJRH@-Z1}alrl%jvqyVuDoMPx@ zT!s9y!68yXl0Y=T@fb-GPw#r_H2DoPKMaKeNAA)o;F0tQ{%Ro-66 zLjCvxBt?2e2>qWJ0{2jZ&CRI74FA`5eb;_D*P~qUq8}I+?h?!(?*FlMP78tnNfs^J zwr$(CZQHhO+g6ut+qSxF8{0ecI{%Ol8FAyDBSs*)UW35ATUo4Vkc#r-f{^}lk+`l*=q{@nEOfaQ0pw+FY;^-$)7+M%_{?leXD1$p{$bH+% zF|AU>J)1O#vC=sC&fTGQL?!U$CvX~Qjz2AZcZ>}ME7vjo){RP^u!0q1;GsZ?cInh& zCBQoM;OQKc*q=!w<#ePsbcz<;04a#9XVPXe=pg8w9f(f=1^$+7LqaCNe@`~=15u5K zWN||D9OXP8eqW^hXK--VtP45Ik)X-otV+HD+$YwqfSFYChJ}FB0n?Wu3n^zHah8Oh zw9+TQW{?zjFRi*{g`W;W0&@L34^GTpq)k^i>C0E*-5@X0r7?r=JaY^N=R2@p*7&0V z`#=a~Y_av~pxPHnFJ?0z;VnWN%@ATmfNAf8q@$kwVSO5ag~+^{Moay2O_-U&PMB0PErHX zpCOtmmPm2G=xsLksRulx@0X4(r4dg-0)QLZ*46^teaED;(pIS>K#H{ECk`nS*Wd!t zqTbLPM2&7jP~jApHPObG(qKffE+ec_8X@z0f!k%EhdV0)FIc%6oQ|Iil2#(ug~Mev@l zfP{ZuGM-ft-~~eTU!RpMSy>=t5CWMCF^9)B2^BmvW%HokohnPnj954Nr?AW&STBT&B6u27(8vX=-M2i z)J@Mxl7r2VG2FJ*p&CoMhpH8R&0COS5IIbtP1rNrG2VMWfNIgLd=YvUc}iZJSg!jy z*vC&bGDw?pCG&6wp;QwgLgua~%j&p>`$}8AXrHa{Z)Q!e#8ounxJ_a@l?MhF2IUBB0NZkF9F9qPB)$bPg~E_YBKtZW%3&6Rn+X4(BS)n%HR0nJoE zKlyHoV9rR{(a!1SJ*;RJTIO*UlrwY~NfpJG=G$o~#~<>LR~~`#{5n>}`tfhbutKJc#zT;fFSUE$6ypDv?@G1#nLTIuu zqXlpC8qPABvJF7Rn|Q#w3byjF{IgY4ta* z6vpa-&b#?3V%fw!j}Xs%mM&F7e$6A-1R`iYq0Xf^^A+Vu&v1@#yb*(Jw~5Ix*z7vx z6LU*S4p~`uaiBV6*+v3YFrNPu=uf-dPBzxdWLulj1|%V&=NVwX?>Ch({{)H1!H~&n zatqoS%JvKlYz)WfpSaqJU}Tgza}H+A+zFFP3^ik`duV>KCY*x7 z4#;oe4XL${S&1nUt=Jv|8S9nAkL*DK(UaY$V&Xl@1KZYfsRN|+GEFX_XzP%Nyq6@+ z6Aaz43{=xFX&~hs`O=Cl!di#Xg*-T2Wk%lPA{*GkAnqvX5FSHEEJlhF|`l#dQJf4{d+Z`Gn@duw;>Wzt3*CjF#9!f>J z2X^(4{GZ|Of#oqyulgyn?v43N&^3ta_M@IQu=OP+8bh~B!!WM(m>>?l9;$}@$R|I& z@Pn?$67_Pf;mp)nJrT;A3tiQwau&$%k3eHwn8<_L*y-lsT?{;#idh>iY=mdps@T(n zQLnX7v@jbl=FB+XBZ6!gxJJnWAw(>q=QeI+`^D!Z+G@Y!=o>dGCNrrx_Zj1k=Xeyh zN72VZ&(Ngn!RXx0Fo0`%sg0J*H11pN{2H)rRHp2YmFA#hj{SZm{aj18jjzdF8tOr3 z{=yC4qyep6D1)<4b)myG5Z0VY6sMn!EHW6QiKopib9$HzY>Im9u|#PL$1?Oe9k$~D zdlG`F%zDF6IV``64|U8Jjm zGfCYlFC<5j9(KY#(8{~B8qxA{om`JDq)77xlR9}oJ=d*?>TGYM)682rgT-c_%ZXfU zqy<|1$`g-MlWVsV10&LN%)$7fW7Vz_io& z3HhO<`b?Y}$z|KB<0)uS@r^(zL0TLVSP|#Qnhq_^v`iwgk2x1&!5-uyM%@|rvgSnI z$g+?~88Q(-#Ip@q!z`LU*`AoJ;g^F?htGO{?DTX{fI{djGT40Ofm`UOXWjJ1rX^xF zawX)a^9qN-gqAGw_dMwG&JV@^o}CRvTH3|8hfuweVlZkNem2~5TW)YLy$U02X+=H&b8-DUl;y{Rw$FTEy?9yxtY?0 zuK|6`1 zWy)Ot*AASQ|L@R=8MU%3IPISuxafZn5`6#Jf&WRiR{q(657|a*S^x&&M7sw8-{R z@EMh3qyBlkT)+W_9oua z%Ri=+sh6$tW7bNR)H4>;==zgYNNd*(Jqn5c-8ocf`0T&6i7uG);i&k8eS2fLqV@!5 z^o8Ne9|CI;Db?V5{6`yn2FpF@c&4*1t=B@G#7brG?-Lw1dF3#VK7smMrz!Yo@HqHK z!n6;nS{U)rPc6M&wVCCS?d_Ms4k+hxJg{1NC}ck9m{h&$oe+L+^EbgH%MVWN&Ei2( zi#BX8wUa&J2OjjWsCXmn-2VJ-6xlq*1?jW@WmhAQ# zpGU@?&cNO%JfqBxmFo8}(OGE=Obk!eI;! zN)jFiH>v*L)rZ!|QZ^f}oLR|DDGc2~pJbY3<*ggCx2tSV@aW)c{}@%z=!%r$(KCGG zY`6TuETs@M-(6XACoL85hoO?YFIt}*o&g|Hn9<3_9@05_BA2oR($KgVKvsahM?CYG zjIM=VhRL{kyR`FF*qYlaJoR~Hh9IIb$3OD(u!Ce z_U1Zy$Gr^vaD&vtR9d=S*w8LZ@tmKW1Y&~GR=41~ZvjRqhTc~Q1TZ?+s=?^~O`&G7 z!4jS{41Q9H0$jz2GQsWlwuN~TN2GuQ1gZv5CDsTzcspJaiY2J&aN-!U2z+2;#hNZR z8upaXBY$H$W0Q6m$|eMtUx7lZb>(yDD%#0|7i3NU^H1CL0GDYhL6~goO{&LuBmN-M z(47^=-Zw|dt*UEsRtG}(lcF=uOzA4O=Hwt7sPKTKirbk*_KNR|KW}3FeXYO4&%8aX zl(!1}S6BK;LSGb~ma2q8twcT@nOA z2qpE=>kBDmkod${h{S{~8y+Qc1#U5t+z{CU0>FTa`??Ve*Qg#+XR1;sh~oehtxYM%E=}?x$E) z>%lFl~XbUQnd(MjXlS;^Fk?Y{Bi+AVJ}19}xY&)F811rFcns3Ud$JO=M3 z<7rXL%EnpYJ1CoA%s59i`z^%{AYqTK{Cx?@sHG4UGsJUY2YXFImgSP{6;WK@Yo?+^ z^Dm`MVOWXCT?SB?UJsf+A+EoXi%_ppU|S8M0q>9@RP&csbJvRIwe-Lkqf8n(nwHJ< z5~W2F00zx&{We{Sn2y^wHLkp)w5kkxibvOR-vAl*qlFSDrQpsx#vJMo3POqeOQIqp z&XR6+lI2lF3uF|d(xv=|+(;Z$VWl`7K3{u*Yu{7G7r1;YG?(unMKFn{TOB4yea zA_ml&!yeLniE-nJAjC>=pQVytaXcGN(@nA{v5AN|LYjoa2)#F*1eT5`Wg02G<70r= zFKSDcKw*>MPhlO$j%@N7P?uHOh-99Cr8mzOvF{3ed8=@PlC%MW@V2cJ3bZ%T)a4g@ z_UzG!aF}`}46Y$PRhJ|QY_uV=iiSNBB*iS`YdI?u(~j-ht}5@)`w~x|!;5ut9_L>8 z$ujyF-LkYQx`80T`Ep?JLZ~hw`!mOqM>^gSp@|{*XSak6vD*;fyF$90gTkS>z2g9oEWN2bS3w}aS(i2C~Vf;CxmP5^4d80M%l z+|Alb8jH{w4;u0<=E`pD%fii7RR$@Nz!Vn1x~OgA(Fh7$89OdP;1LKg+xBH;ZFebJm~4K+>_X#oD=lu@=xqDV zhgJSTq<9ASs@PVdCB?#VK#TG0{9EJ;{44Yr-z0OOe|9%gjAKmDa4p%SpnavI!vP?w zXLF`CpD~;=1g4hYV;V_GWCf5fTyP@vlj54e@#(PQ{UCS8hWjQ^Vl+FpS@6P*$8#9M z`=IhBACb=eFrzox%h{9J>Occ7CZa5irb1_oBe-m=S~6_1OTcu`81s^+>=#a|A!#z- zEQ%DM&|B%UO%OTm8KcD`leE~e!{Mv|04xVFDgI)-pd!lI-SS#GyM%a^I`fSfK9T}x zj>g(+3d-4RwH4NMS2YXsBkL&iu2?7be91*Ih7t41o%aA{xFum6$amoTS$oi*$z(D) z@7G424du>PT(xa>%s}+Ibp^-WoUv$@Mn@2^K6xPK=h>L8NNRm-Kkj&v?LiB+PC=cq zru1tMW||CKNv07Q#XAn^gX|MXFPRYCs?>J`p&&Q^G_!V%xKwR!71c%EM+L?O$zkN! zI@i{sa!o-1KY8)3ODC=8xohlkke+jz|LHXJ<%#d#>BbSUfL0 z>h%&Dhq6j{#txx0nI0p~5CGZv!vtrJ?9MAw+-sG&AP7s zoR|O>kmk@+oq#Gt=6|0UAj-_Igf03C8`LRB+>AOQMWT63A-p-0JAF~Y;Q->6%2l#x zlrh}c{07~s$PSSQoYC=OvNAoOpWYtllnaS#pWQ3IW1GBW1t7=@d(d_V2Mh&MxnwtFe?e-CUA{9}o6wf+2sv|fN$A+w=#Ae@BEELJ z^=9jp2h26CGNjHW?UQiT%oS8y`3v+0--a(6N176C377l+Ged}7Xe(c zlVx2~7n5}zk;b$YY37I3+NQcdg^k>Vw!e+#mF6loZ^|t5aGM5pF|~Jo12-h2ye)BA z*>oo5ZKcMK2wM2ca0O)Ra`}Zehl3_m>vpyC*2Ni$Rs&YQhKZ-U!l-h=G1{`{+%$A6 z1Zk5KF*lQqpSe@Ado{rkl_^1)X2fgEOli5!$H)R!9XwK5!o_H~`S3kyDZ-V3vE458 zY%c3)(IpBJTXNdI1}RH{=d&f$D)H!>A?Me?OWsB}FJ)v#D=Fb1zxk9)bgh3V@sqnc zoe$Rysk5Y?kfki|dM@HnPK;pVlf@L|SuJb+nKJ6k&PS(ZR?t;kY2`o&)|++nsqR*v z9fZHLBP6|)<&9b;(*SC*of4On^46$@hdI7Ip-`OhTT z0Y%C-qjN2L_2yx_t^+0^g(TP^1w(K?Y+2i{#X9>xIi6_prx#gLRD|IkQu}TPJJ*_W z7a~PqLc1dF*6dHdwZkzJGF7fw2v{# zd!$%SoCG3GOR(}~Nkq9*RbJdD0M?lfwH!z1NL8>}(B<0kcVrfFYcQVx8Og-7xb*p% z{;~K(UPvM=!qAX@U#o-QSrSJcIplukP>LvlFG$kG8d z^%9gfhUYhl{XJsCi8R%Qz_^~V`5>2mLAzcCaP)(w6pjB`rY=s3WLpF^ryxvZRloiP zfX>jenES7GIH(5@wTuTfGh(d)8V9xDe`tq&|5ZP>B;fu{MCN=)9FfLjZoAFN9Cl+1 zKxO=mXi`*?ILameCa`BKv&vxiLWyUeC(Cf6Xo~&AL!c3VIkM#6pG$o6yFMFCw?eHj zE5#7Dsq6&am;8ON-r0wdBD*xvmf23OOEyC7)Nd=*l+PN-x&5@(sS{H`{{B?#H+Ksv zjl!CH!1gUa=stH@i>;c9&D!(6{p91`?YTzKZo8FqhpDQH2d1s#=x`rcGoEAH7n~dF zeIsPN&w00Nnok&YG*^9BTedoD5AWUnzO<*tv0e?HgpqH=o2rV=dbMn{-M-Y*o`H$!s{Knw^APBnEiTBb|bfi z;D$?>ompu|V@Ki2?TdBnu6y^_M=^HJK-!yM@>DAKm88c;4>+66;Vnp8(g| z1HJn@nj1^Kbm$!4uSsDTimQCyro2?=Yh=|_b;m^jQ zsDi)u3jyJaaUaRRO&HziGQ2TUnufh^DfNU>(d=W$Z|||2IEXAVSEx4|wgUdaV+9>q z)c-1|NVLXyupV*=xuPjjcoy^J>UP!OuL*90bN!$mk|V`#i-t_*m}w(sQRP4QTHCpO zBC-(h5bb+nL%rY_r(zX-VOf_L74(DTzSMaVx6d^`;aZ(z*I2m&Km$05F|U<6fp-2Zl!C|1GCZKO+&yZMr9%9- zl8=@T!e+v!>2uHVmjW60i3BE3%&OT@0a*tz-HWO(G4|^QLduU`vh&R7X{?KQ; zWug|s_43HLQVpEhgYS`E47@9&-FflTPMz}ldF|r}Y9;=QA(Cym2>VjIdAmni2m}xQ z!w6X8BQ+^En(AYyiKkAojKB$wHCcJfy=dB@1{KS|A@8ab!KYI23hZ&hV`B-l_Z9AL ztS9N_&Z+2<`e`MeI=wL9*!G|V2^ajr8EZ>$wATUjm@EJf2`nX|13R5)_YI5x)!o@9 zQH?N-`rF`Q%^I9=&kW5+>01uONMF(U;X6`WDSLA5-`ie5H2l4S8;@zLedM zpb=0kyed`^*R&q@+{C~nr@R9mQ!XS0oTWaqhP*yvpW)p!SHoh-Sz0Q>D~062v|{)$ z--eV%!_9&_!-Klsy3&O^#8{kg;FX%||!-=RF?1fkx?8QP;tIE8$rn#z_Szv#|kS`sPn-;NfSWIKNQN`UlF;J=g}Q{Zb86U#=o zMe86S1!?jNUxQKs1k12?0YdYvl|MY2`s~g!@ZCnJ;rjtFQY{1HsXDc+;R3%3D*3eH zxf~MW z@&?2T5X6+bC4aJrFH7V@uXPvI)4kWX+{q8;j7E@Wal$=97vJF$` zz(7QMa@VHlx`rL zvkHQFFU$?I75tFJq(e=T2qog0awP|yx*I_dT@m9!?b*W)SmBepLouovEx}ZacVBTQ zE62t{9tD*rBGY z28D1c%TKP>L^%u-AVW%TDVYgDM3(buGR(tvM^~mgi#L|}%N`AK*IH!iyQ;37;U%Jf z%cbv+)Hyi*Q39F^-^orm6%KND?rK8b0{N2UVAPM3qn&z9G> zY-`APF!(|KHs1`fxIOcibs3%E$awzC8>q`RUC8dVBFlYkO)mR#(z#0C6}SPGzZ!Sp zF6hTZx!Jb3%EJosx*2M)s^mSCq=bLc)pnvy(M0PuB!=y|C$qC7v-2cyxp;0k|GLfW zmg}yFo2k8f1Qv>+qcmaiaQPYBld&(s$Ucw-^!(6KD(>zgawmfo4-Nz z83e_soWN^`DQ`-vSA&P=Jpo3`dvX_qhsK9YleCwJF4EnO!Cyx=_&Vfa#Ihx0rd%@R z3*58X<}8kTrf$)eRpR<~#0%#oWy8G^JpH6=EW#T!HN#@+G%F%SQg9ffloIgb{EenLB?>?2$T_ zA(=Q~wSP=1fzELpCp_EtDtlC>j$`$wnRTA$eb?x?Tw9}^eTe2qCF<_i((o!XxQqag zn9l;L8wBD;`O2n|xoB;QHBkd*o6Yk^D96xo4IHxpU~Z@0JZW3ZiH=eG5fZ zCmCinFvmU|!!~Ok2weDaJme=Ain~*7*MD@+ev;z7mzvB20J?6+C1Ewx1`}|F_iy{| zK8ow{9C1jRQ+&qU+zU7o&yNsjl_CGBj&Zj{gNJ8)FF5aS(~@CDbb~d_Gntl~`}`al z5yXJHjIn1rWB}=sd~Hy_pA8w3I3Qrri`aG2<%6>2efqUbB13$mSJ@Hkd7Ee{DaMP? zuQW#^D++7DS60m{(Jcm6r>^hTJ?iqY%y*8{vgV4IvZOyZcPOtCSTx3?!9uDkkqFl( zS78DVPVdsdq75dgb9U1F} zMr7*UXpCL$p8{8zCC%={4{oZvYdcc5fBjxV^~9178_SJf;S|~Z<-9H;?n4BHW?b+) z>p}uf{`DGnJya5VDVEp$vgthU`UT8;AsG{-|4%I{;U8hWgadVdH$DHK&FS!eCqMiv(ttD7N<)oD(P+G(*dB01oSgw1V30fv0c62LLphb3DzoF|q7FPME=PlO zRa@KXda1_xjv;O`QADnH{G7Br!J?o4Zwf)8KyXwiP1GzWleo$QOT5A;eAnMySCrSlK1JwsK5B zKg_n&j7qZI(LZwJ1Ds+w?|ug zx|_1qRoXj)RP-I0mp8##hy&%s>ib!?;U)_ful{-J;Qofe?I@vWJ8A44q`m?=(iGkT%lPeM1U(`qJM;0ynNWLjrU@e~BahHbm zU1S|U^1>Q=euqGNGKg$0#aDf_QgW8Y|@=$>&o1zD* z6ijbG7zKpX?`+C`AQAU#Sb<%SNx#YwlvdHk)Kf6q&I=h>SWYl$q;CjNv8cq=#pdbI zwnzfNp_st~d6eF)6s4cDYi}ETI<3QZvH@Hv_2N*C1qG3m%U= z(-WKLEADZ1ghMFuU%tYlG5wJHQXZ2ivAfU6cZnTm6;%vM%%h?Pp)M*t_i6;1p43W~ zMd5CBAZ}&}13BPt?}odQDgsnSr@4{_d9-a7FNz8q7@~e+yKVg|kZ9 z_<`q%xTFB+6YSx`5SWCM)sQRrgv|6+IjlKDVUCToqFMY~H8Zfq>k{J>#?A|zLB z`6EIo!yGg@hhd{8gD}tek1 z&}BA5m{pA7*enefj+a%fM$n2EF!e|_2ZU&)Fh%`TEU6*OIl!L)1}rH7#3aIMnaG~w zui0p`Wf%S_A}t#9Q?z>f22>0kk_ZD`%*0BHC^{4^xq70M;-a%mNGCnq)d-)h6y5=b zPQ+6V)B6h!fudeBOp1P#DSLy3;-y|1XxRs8k3{*Hem|^NSW{wvycD@7(Boz2RTD6% z-KVH)7sEKdhQkT%UjikVcNQ0SgUVN;{?AIK%!?pey*?8>+^Po!J0fo~Z<4uQKd z-XF^T425h!aNoVi%D{5^1dZsD!j>)ULPC~I_TO8sRBWW$9Z@6X9S}`Zam;i ziQ^fOaz$;MjdfgRITzPcmOwG0Lde)E%$hyzybKp@88w<&E;2j3kxgMz-Bt11^<|Wl znOwZhUYV>kXK@7&OxOe^-5P$+5KPfTba<7pfI7fEn=rc&J7bJ5@s4m>gkcxtHa7QD|!)l@gS@1Z@E z`?>cEPBEnHC>s$u4Kh)~Dg71fqv4vs4z)7XZHtshm92M5IHe3$8_qw0F9)ZFTEeX# zAjx#BroE?e5XH{aG<@vvg?-f8QON(^@_-VV(+!SMA5Lg)qkoE8{^_tSv?<5=$P4;a zMOh(2A`L-sp&Q5N10T~OMrF6PTD9XPe=cfcC}3U^5^5{lU!Tq%|c7<<(!(Qdz%R+PF191TY5SHz@TJJSM8ZMTyK-0 zM=rr}SvDOVoouk}r)82=W6yQ*fy_HWbB<>#`PP%+0Do=4e9DPDTrd<3Q#o;$7kF8B z`w7?M&PGvnPPbRnIjL7(cFP9T86qkgFSO9k{-xFat9|n0pACyCA7_dNA8-_{sJRj; zMjsc5Om^$UxBWK!V!1t+B`+0yCeUG@C?0%xlUMdvktA^1Q-~TEX zR74TJ8&^8)GJrH~WP0AEPS}sUO}Yh;FA0)1?3rt2#rND+ZQYIkkv|JxVO-OvmU^NX z_PM@7>)xOt=QIwa;2NgB(cQR(A`2zGasbAqJ>!};F`NLVR*sV%DMtt_iRdjqGU4dZ zsi}le36Pkc&a9}RzFEc*GSFgk8Gnz}A*O2_0M9*3@fN53fPA$MQ5CZHNibphz073s zu)4KAr&3}B+2-V}pJ!8)&=MtB26y4I^a;UowViy91R%XY8T%oCTJn(w@0H-q#deJn z2|Sh0-J=b?zcyFfc^~E!8ac+LUX|Awe8|i*l{{j6M;_6ht%}}^kzY*KR(|PKvy>Dr zTTQ?`vuE0d2_eqN0ylnAV0iR!dvcc0Olu*V(?*63FpWDr^X2?rNK`e{m>T>LFXN?a zo}VLWc$U4wGOhvPJI9UiY)1v%Tq}A@_(Y<#JG5h8a{Y4J;8Ly=S z(epGGk|t1u9T8fK-a@-gc+LAM9hvoMq}t5@Yl7~q*m+%-ZU;O62mef?6}u#tG2FveMVi9NmZ9AA8 zMj@|`(`4Z?OT-56Few1|D=$99Bh(|Sb0cuGs8Y+dff{|I&F7cfWH0qBvj} z*7Y#e-5Z%Kjki!t&2|`Bd@mt6k!YjX`RWMU9}4`K2Ab*+m`SgGbI#HSF;#m)YvGh? znKS1Tb029h#D;y~@Bl%Vy;LSBA9V*jTeI#$&Eu-dv1}l$m?OWtbXP>Kb+>`PDYv>qG`Ev znhcs})P$?USzXvD_D;%%SVwGgHbs)7l<5JNmtoOAm@XEW`^B2eCm+4 zC)#n0*R$(!K{6Pd_o?iY`m1uSkPRaK)_h)R-Qo0kw|i-N)UQrwEfZFEzjx0sTi1n7 ztXRK1#M8giiVgg5(YloOTuy`rkupun*DuQZCJ1_BzIj}|OCeFe+pOgQq$}KA{2b2z zCPj7(+Pck-Cz}-~9Vi7Q!gCC!*}+t58l3*Zn@E29=K8~~%7N4SH_HD63qP2crGtBr zJ+thy8t!!#*`{CW18qeZ=Q?1PLv+hgEo_F0u3l2*5j66znx>2&rKkUB&~Q)RfgbHg z0yiN(OhNK5H3Rk`#dONMYEPtO*Rl&MH}+pDVjmV1ZJJZ*d~xwL!D_k# z^lTj)tnn!NP(n@W7URcr$HsjpkRT<$ycV$JSqi$$R&mL%h^Mo z7UVPIq|Q-CzA@2r!bjFodxE;t4oSe6)nI>ZCF>qsab|a(t$iT5I$$3MlGMW2RnNRh zqg#FT{bUBc9C^=9@Kl4r2LDVU8;XzH5vbrg=gJ_D&UkO^>MzaF#ppFAWV>fy(km*K z;&FO!%o}$Y%@l^HKl(1Ori=j43d;`@I_|5*64t9h&YyKD2S*j;AX zVN^SwR?nbh`C)B_TP?ZN{u#B_+LawT+WZG8y1~}EeNBJz#W11g+f)51zola5*}ff9 z-aZbp(^NZ%s(ba`{GR8dsg`ws7k-}deSJHo?p3qyyA3xZnFvZ9Unz`NXDz(V0;MhM z1MkZ{OX{*h;*2$_vjF|>aNe@TJ}y?i_?e?X|KW4}d$+rtBqzC7^^v326&F$`nK!e% zNYwEN?K_TLjN`(q*_0Kg^t9IgP>A)?YiF>Qd%Nbl6=Ra({t$T$CLU6Wmz_7e-gZ;? z!N~gTP9X$TKa+%4&wyfi;iSII22I^HZX&>&V;UCBgbe~wAn3WB}AgAcM zGU8|TJOa&UTwOQ>35GO?1%^X=NCbLJTWL`v9y+jO*F~9Y@)%Ymu1O&IXbjJTPYeyP zUC8*TK{yzbJ1!Hhh+k9eY7z*3SHmQ%@t|9kiPFfvVfh4(Fuo~poS!FBqwibayP*OD z{fBFJRGh=1o=Xa@n0xl`Tl1&Vk|=Ydu4#JY7#o4SX%Dc-Kfl{m`C1N z5$Ncv5`#l(Gz{Fr{A0!e-E>rGb5|d%L$5{srCss?a90k!>%X<%el>WIkdaw4%8@xW zu*()$PNxV1ac-W^PnHD5?AJ@|5@v7Xgk5fcjyg)ksb6SWN!T~b+9woA8Sk%fy+$t6xu<$& zLq5^`yNJF75k#MECKm-@4Ta!HD0@LK-HmZwsOn|B&IOr40GZ}uO*!oP%-dkV7_lS_ zhLY)O2ddGxUMYG#u;28II@95j8c>{biAcieP`>p_nJJ}D{m0<#+>Tx)M##8fBnMKB z8-C%_u*D<8E5|ayBN07~AcLDd@C8E1TA%;CLFzFRNY(3M6-lPmz2wBgTdYDcRAE6* z#x2LX0e@Jbqw->$8wnxW5L8x}F~Jfk)xN&vsoLN!FeYadJ(=K>C{XbjQHr0v#U#P-_aq$&LMTw>>%kE?eGY6G zPpVkAYOK7Aq=zQ+jR-M-DaB3KUVooEe!5j5aQ9V~Dy~Ma?;B@rZA&bkEChimQt3?B z5Q;+`*X${$o|Z1c67@>zbxJmt`Xq2GxEScEzLAevU-giLmx6jpjlAnuc{Mhi5(XV7 zI0K+pH5wwl4^+%h*M5Y!6D|5zM{@8D$s`)5Kj6JZ>I$fw$%|ekM0bV0SQ29`utJ#sRcL&woOaEN5|Wrd8G*rE@~QEA=!-L>(Mp@K<3x!)VvIx0s$}Uw zoxod=H`DR`r3{PCLHM(fQ$jqffsLAK;Oc6To@|ZSZekwxoS1-Ph~be|Y4#s!%Y}## zPp-@?Etum8;943on1ywnhqPMFH^>b(EgEE6o!BKR?*rF)pI%|f6(%%Ci5Q)hTAnXw}++HaB75r#7_jJnx(tdlv3{gKyKOLRR{P$KJX?GlrpA|Hw&hb0J7O=63h-qgkB z{`vCgCsV9EKvBnq7k(Z@Z!=3v6)EaZmM>dA$K9~pzD3w?x8>vdi~gK;KGD`n3<1EA z#DYLfu!$U(Xd0K;qymh9cC0EZe26LF4v}kRvwZS~FaOmT9(C?mw>0DKQ{^}Cp?pHH zvOjv-PJY3b#aW{{$Eaf-kS80^lC-tad;qRx&%717NF^y)jjj!Yo@|B$`?CPZC~hrf zm^CA=la5)c0k$lCiLt@tv9>#ud1)LYpv}h=I*ZggigpYlSyoPNrZJGP#dA1HKJb_u zniV1iWLgx0NuT;b`1Vr9$-PSl`_0=mM8d+1oY*ktsfjd>5(HIB3N>BfaRn)pQLSER zdDH31tzOo6RllXKohv$)ic2f+3_>@?v|iIL5XPJFj1!-@W$yh#7 zo>3!siNsa(0xo$a&gJuWC=ZyL-9Uc**X!xRF_8$E{bKL2{xtC#VYK|wqpOj3djJ4# zrFLc>oOyOx#PFM%8e>f&30+V^s}_Rf$zSqrNhRz^DuQzig&<|sJ;ha}s~Y9Pt_<2RS#&GEmR5OT*lT0_66OTyE6Wq_g`_DdyM z9BAV87toYsYRH)q(l~N2q7B3<6T4Fn^GG)0{jl8f9duNk6pagpr^q~N<1pTAM(Kjo zJ9nv-><==N|0qSN_Si}+oFz$jl$a}NRw$6rWwpU>%%A=tt4)kCRa!=H5pqQi53|XJ zX%b~0Wmnw$!pevQkhQ11grq^Dx7`(E?kP*H1sKBKxvv+H*;ggq_s$Sy2d zf#WpOAPI=*<>#Qlz)vk!ET=nUY#H%6EEA+1{X?>G%SI$OI}xxlsiU+A%%;IPVDzM8 zSp37JDbL%^m8j71v>Z!YwS!{m1_-bmnV#7}4Hd>?FQRR8b&Ezn{A-`Ro(W;Aa+?;6V>5ebt`B zjqyW3J+3E(gk5tC%`{_RbBIJ|tt<3$HpG`1V!6|rVpCewaW0i_G7(dQ-s#i)4A`)* zf;f}Z)^+R+>VNlQ?)b6OL`#K`4~(Ku4O!jNf{!kcfD%KfPiW(_ISe7!$tW=3qF?<) z_oImTU&$rHxH}NmQ}4Q(KeFcp1nb@LHbJs7@6w15%h9!_wd3==ZL}&pzKphhxqea& z^7FY>Men8jk-twF8{%A2ogmqCVdNx-`S#^ZdT$REtP&WLdnSERnJNb=Sw{E(4zOwO}a4SW_5i%lP%m&4Buhc-Q7IN8zNalLeK*Ylhns)@{`r1C)WARX z)R%r#SE&sZf8d4~s#&+lP(<=sl@y;x>Z$nVs780_U+^8&2rgISST79qs(h3;7!rr%+4H`Hr7D)nGuG|H$d;}c zo);`CJ5=Ecjkf3d_g`AN+drPGR;9hSWF9RUF9*u2vDTk~pxK4jE43!!Pb-)~++vcJoHR+}!5_%>w?FhXlM@~WE8L@UbS zGs^P#%Z{2GY1mF0)0c&@(dJaOlr!g%9!B*3uH^YCdEAesBA}C%;Gbpc*iHil75Ur&)JyBJ6i=b79~lTWQ0! z{u=R6*tqzLx$cA(NUiy9cMSj#<+?A;h+^5RQ%MR}W6s)%-#~67t(9qTQ~eU3ID(I; zH$KM*vvbgw-1=CM1%|T3{`dkh5xP&&hQ7^%8eoTx@NX|Qs0>4eprRD(HB4a&6ZVH@ zUlH)6{$h6pt~Mz{!!F7^UucZffrIU%i>NwxMF_jxKE+WEU~YA;c?+?f5*h%yRu@Mg zIE)+*2Fvdxja2D)ITj;1987hI*(E~=j~j$9LEoYB3Y4Lw7~ z^bAjH00*Zh4&0PG*c4_XJ4zY4LRnRQ@Da@&;Gd3hpVTn{XEfUd1g?tIC^@$-bas|2 zijEap!#~Ak2P_M8f$3x=(S|ihoN-YIXz~HWWPq?&nbBPscz+UFj%`vQ9m z@bUKZw16mY;u<8NZgJzm2gsl&F6HD^lR+fy2V&BJ#v9GO@b)7ZRe`mtvuXl`4wX)i zjVdX!E|^1yLgV_*I)ny3riulZfvqW@%i%bsBA_DuA4l4jej?Hdi~DzAJ=B!oJl8f; z0T&03Zc_y$$3_=Qmrh*x#pb6*(g0ix!*krdXXSUAW)hNYh6V^Ql%l93$@G%~^) z=!TF^noZjlRW{Rd1t#BE?jp^!JVg#17vN@Z5seG|oX-Fy#Gh}s^?(qfms7wPX= zb;lu*;JtsdMmE`;jELv8;4ac_q5&(gSq-c)4d>0in2@>;EFv&c#7q!Rv42?QZ91yW zE^9pW$F+^B`69U$OZ&snhjm(TSE=K8yfK*HH$uOQEr89K6~i5w`eMouQ=l`d(Op(9 zXz?auD7iY`yKM6NsmlCHH@ELtN$y^}U6tU&wOjrCm_ld86TFEs? zunhJE6NutD2AcDC;|l~oLWB?vdc)74MkY|yjJ%*fP+f`bj|lnl#aeB{RT%4aP5C`R zrE^c&t|gP+YJE9Q4sFW0sU|`^NKuUlC}PSmFT9l&@@*la62;YO)O0_x$lh}H&|C#- z1v^Of(UzJ6zG4n(I&-48NRgn-1CjgNYL8>6L)rAI>jk-oubLOvwL6wWwK^&E6AD{C zGV3VD)wS9I(VoVy6zQWq^zZ7}=Ad=_K?~@&LdBVtD{x~xr>%y%iq^-9yNsQlbUD=Z z??jtuPa0n|k*)t-nSQtFPe%|LZNvCCOaMVrlBF!~+VxaCaRDc31WJ3~a51l1y`_qy z?20dOS_2bi(9=J9b50n%7K_?_&7^7}2)8vRV?-{8Avo_tt`C*wj z_yS=K8pkoABv#iW)KGm9^qXR9PS!26Mrt}eHL-s(sO?q*`fKS+TEH>a&3@(MS8EW3 zH8ggoqAvgl9Yp)|tqiIxz~;RPow0|v7Ybe56Sk21zV}`|G~++<%U$*Cy;x%~3G3BD z!bv*L0PLR7^p0!8A4QWG!!oCqBv1NIpT(vwj~0$@-lTDfD6d+TT6jSKv|M#6&=I=x zIrhIQHyVhLMEzZ3u;fGe6BmK|S$0>ja>wBoJ?P2`enFKtJ(Fh z;h}x+#8}&GUYaDqR+`90xC{Z1piMm@tcru$tdJ!AQJd)N>sQf|Uu&i7R;(IK?un`^ zFbrv!O|f!Wxw!LM&qMyMWYmax36=NemAYpFu?13P* z{I4#9ar(tBQ~^s zV)}bxj?XnMP;T}#)^&ftybg%BTgzv@`-zS?pDl>hu3&e_VNof{ZF4obL<45h>$T5M z+&_Af^wX7q(EFL!xDi%DlmD3cQ{L5nOczppQEh;3n$y(Hm$(G?vwE`6IeY4LaeS^S^%iKC~Da-;8i;F@+9vysU3r_K8HFbUo* zTol?B5;nD%E+wzVofNn`f#KNQyv$Q~PMH%zas?i9teK0-MY>69pf$W@j8mt-O1eTkj5qPHdGokM-GTuFnu9{HzN>}-f9ES$;LP zkQuKQgfvjw>x;Ve8zN7e;tA>9VL7sMzt<_y=&im($>KXYtywxf4)$riFxVaN7IQu> z@q^fssMpST;{>^`X`ZYt2nYr+EX82I6zO{;WxA$!lC^)n)$(I@$B4&!F+tM)qhf{H zk47+eL(zI9Lf2V?caR#-0q&2flkVfooHoPw(AQatfm5My9r9l+>pES?3cb^UZ)mJ5c?#f839Y1brhDtPh> zy9USAa=u6L9kv^2Tw7z2&&*?6mAuuSvi)9ImsfQfW@GqR3X%nM^LE}2Fg}$c%cgE? z1m_Y!JyTLSwRst|4w$l#b=Sovzw+NcFPu@)nOXzX@gdZn95EXEtXT9_5j2U#Y1w-E zlCg_Yk}jVWZVZ+oL^oG;|qF_iNA+Lz=^MIQ8!NR`(}&9 z9>=`YFWsMIx`}Vn8(g!&7CgmkL(NL)_H^aqrTH6P(@_F6(PQwLeM&JL17V;3_xJxM z$JbjvE#a?aAVNIn4?7~-0|Xbyt65_zx}tqM^%jFbL4IWl~WsK0O!>1ZZZC#JK4EbPfP0I9<4TOVQWtx zo}Ti46xVh;ES;`eF_u~5^*0(#?c~)nhA!gVnihvvPMfOiM_LAda*CCJZW7>ppd+__ zn0&{%uGHm!6pync{hjD^NVzN6`-l9}B)uL(O2qcj9P(D)J*`O0H;bg)qf&ObUcNq) znVn<&W7^vu_VFt{gR|Oi^I49&NB7Ct>F}lA4@yGCon)@AzvnkCI3^rM)VQbU0K5F8 z;JU~Kbh;c3$<7&kbE@;F=kX7gD~c5zzds6D=f^iAc?8?6{#)KWD%+pNzoq1-witSglr-hl@=jGR z3GknzUJ79o4)}{b+^6wNx;_N7+}L$T4cLbkWr05wT8z{-L+&|o3h>w2fz$MCg(45$ z0HhABU()!565<=PjYKkT)5~hOx5=`*&#P?}VzMr^SzKSHrVi7O!o zY{fc6^&H}imw}1?8O4X8*pQ4o9tO&11ZXtrF@QNbfe|wmRr&tjGUr2Ajipx1ioU0hk zYq*c5s;h}(r`elO3Rql{D@LOzFs*+b9W0L9QY5F}qCA7}p0j^q#OJ4TS|re&!{t&+ z%?Z?VWR4|6Gx6gk97n7E%#Y*}KEUz$q{f(teBT!y14*czRP|;!+017oZx2;i-d^kgDG;qCT>vi*O%c&D2VZBW)Ph>H1-IW2 zU+*&f$mc%PtZOSuSo@8*x={|Rl>^C7nvB6zDFTNi!2Q~tPmdBE88lIzVKV{*qD!c~ z4h0cuu=Q`^8Y3n!2tt$*$;#7?QZNf6L}#cQhLx-8By*08Jpr|R2X`pcz6caboD(vc~YVEwtkT;wZTS!h4nUi}%Cw>(y@=~2yr3Xt)zZt0p z%hVu&CY4I$H|{%k>- z{VqsoX!e##CpyEQV9Y)qXT9Pa?C__(EL*ZN6|V2>>H<>+xX3Fd$#KDfSHX2yFUM8U zNBAu?mMzcoh>EJ4^mL;X5?bhsub1G!S3|7lTJ%A1pUZ2w)(AL@j`8x#KgROIkkFej zwX$%9pU=1v{1w>M8w}g4e}}<{V`}48YT`HbVzQ(0{{k!L*qUYvYJ~ey+6nWtVM+F9 zO$8?yV7Gs>&8K4$tcE6;<8@{IROyo?c2#Oo5&Z`Cf;CZ~&n9+vP%xuot$0f=aP?MI z`Ifq$irjFTIs?sszG8Y2vVcZhJtpyO2UsO#BRvGzznM_gTJL4x$NjS)W_`cQEd((k zycIR=N4ZzX80L_8;Z760n7NqDo{ABN7JsVleYA}Evv!O;CR~@XS1aaAoXd|fXKoi& zt+>a_RO|S{8l~-Hc|E%qhW>_kIyrr-`R0yeWaK2Oa zHx0Lzj8X9E%I77P{i5rw98O~8tkm9Vw1?=TYh2y-Aq}Jr$kK?fW z_1j+u5L$2fawBqZmcNLY_u#}sW^&r$@F$Ajd$Ga3v=R1X4@}i%(8`hR(c%mu;x|L z(F@P%ki8&n4_&jTF{D%bL9duAYKV`vGP&4B3I2N*7w$t+t;}e;-AGa7xZC9*1ZbG# zCA_&0Tg6S=f{&!$n_FXD);_B&?x&h98w*a+w#?>Ens+jeD*e+C!E0cF&&{<6tP`7^ z0gw*15n(JXUsuPggEwZ|jq~tv@ore_vVE+ zr`v$xLO4079?Er>9z#M4RDoyOy?ClIEm|Mu&P*cG@naV6BVENeQAdf6M0QYq)@rDO zk855^9{Yw7+XN;HDW`wz1uOOWd0GClnraEF+9kbYGUs~#Uk2p*2$_;rf8dHl{9AgH zKTGO0SUkWI!gEKpwv8}pjl9lj@uw2|&^!%6036A2?yCTbFRV4&|71y-!9WjO;$CQb=Q-FGXs0Gw07CNi$8SZ2X7)IZ%-RjCGfBiM;r+|6gIBR z*(5E8SQ@$;&VnJh+xO~bBx=(a%DA#Dft;`BOI!Zuv;|#a;-Bhg>W9W#Nhw3!pN2Mt zQ>f}AN&1^!R23OG1AyO~bxrsz(x)@@zUP?J;2o5wJ(DOEZoxJ7b6i-U|?g0aD^olg%g&+oQ zYdFhn+K*mpLsfT>DIs@zd%@#>Om9ps&dSV)Jr|B*E=luINdEPI#@mvm^~H#rX7q<* zV%CJ>SPiksa8ycNi6WZVBHlmclmo;sSO3e7FRz#TQWi^LDb>1g4cwgVRD_GxK7M-% zJjbYz_8pSGZZhgB+?homS+`vh%=EUi%$S10fghZ;Fm$0z@Y}Ac?eG+A&92M8kK5Kg zDVVJFEC~yAG*D0OsE2A0q~TCb0WTk^CL5OVy$m6XAmNAETNC;c!YF{s+%f1xwutUx zm7wp^%%kU%Bq=mY_K}WA)->Xbo*!}cAsJUszqu*SoAyk4cXB`6QEEs|Cvu>aq>z%v zx^KkN08LustxN3zCH0-Lr8tS*WK&RvC^SoC=ImITX(tL`va#^uMef_i^zyz9Nq#*G zH*9UqkY(791J&6kBT(bj*|-sbgNR zC_usW7S(?@A4VMrT=|b2I=f3TMM5c>{gQG7(E=s4;i&F{2tkK=vJl8PVbM#@96Z@q(9raeYm%esmUY+F_XP zQnn#p(P^HTmM(J7n8?Z?w{U5GzCQmAf9pLhF7m%oqcRT=wVfGtX1cH&0tdDAf9_n) zJdoRvp1fWQepi9RNsuUUuh+&_QdI-@m6EM(^`o3 z+3J{^sW7{9TzI#sO#WTdklAPs$r%q#yG|5qRRp(UXaX;2Zc;kQ=v--keF0Yr|4fi) zFBoL7OfNaN0=!(#%T*LRURh^0TzoMocQA9P@UrDaLZ8l!X}B3S{uYj)tK{%92Ewt~ zm9blNTin?7?1;IPuEj6@wt;G6%W;D_)!B4G)IMj{Z2UaxEDeC>13n^7FKvu*+sS{Q zjVLkj>1sKMeafp%&2ux}ihX-(KQqJ-0=wQV3pU+W|2|EePkYC{glXfv2Fp}Nr% z93BrZ`ZIV=+Wm3de-sch_yW1MJ|&pFw1T$Cv{b{Z?)CK!%XJ^kZzHfgAxw|ZuLzm_ zesm~fxc}F$OPJRUVEY6k*-FJx6%q%t;>a5si8mg+J+Pt*P-DtO8;0GzOOcBVVG(Hh zJe!_sc5(rBjjQjn;xsXSmJ|SQU=bydozfe_7sYnX2Ftj0Eq6_>8l1M37+zZ!4|=>Ut=d5hM&bue!ElL;GTuRN1{! z;a|&odL{TeB74`J0dDOQ9s} z_tU}}UQ2VRKDHMk}uev@e)2c&VhrRDKD_Q z?{F4A#X^^z4;s$(2^skC~uJgO4Ty3s0sA*SyqsC$QG(`m|<-*MzlW_uY>!EOvm#(}+uPzT`WbDqyw zO5otdOirbkKytyDWzdS`xOmN~o&ZtqbP;}Rza|9qgcB}A5z z5ht`IiP+5Zh-x#@6hrN9S;Pt7NaN54_9jIE%Q*Th`n+W;N(zF^CqNS;FXs=c? zQm(f!h`~2-jh|%<@vSpD%i+10RYazi1VG4FR~7vMG&l%1SP*PtJlhg&?6+sKGNCeE z3B9`5krxY&k)9g4S~SIHd1vqVn3R}?gkO7tNb ztq%lD7?D(}A^*ZBtG+6?Rs#rswoynHNi%r($ItJL(g%_3s;g|$mW4Z9UsHO(d*4OC zF9Lvq;_TGChB9f(|Io=9W&EH}1rAZTtQkOCcT#adpkVNpHDQS3bBKS-G)ZgnW>;#O z!d=@=S?1ta$WS2aq7sGq5@5N!xpGTooaG?_>vR}H0-zkk13<;P4c14N3ucwJt`I=v z4#Wt5l&RTIsC~K;(bZdc%dpTu!To~H6(FK>PYLGj8B8jzxVG=`i<%_jaMX}qM9z^G zvwqFt7@s+q)`H51doS;yOW8GFA-O-K`L)ooz7!W)RIGtORZ4VOqtth)+&c@mg_|VC zHBh5VP)4%vl{+>ED-p@5QNvlGHGXC!@WM`-(0U2!kOu;~;!1?vfe9d|C6O9|rC*DP znlx|-xU(?N19nplbG;xm;U}7G@H8v;gKrTjqqay%Tv!Qn#McX@`^Kn$gDi-_zZO~l z0lLtCBY%qE8q0;EG>kwW3xe!P1{-Wl4x>CBc{`V2N=uj}eGAM6WlWifsSLUC88U*f z-7D2QJfKd#)_s=1E>UTf_gX zl5}==LR#*ojyg>GGDAHOhTl+tq$qVR5(gNmz$vxt*$;38;=DdMd`obF^^cUlsAGpQ zx%oc<)-(^2^F*C#HH@T5OAgkvq!Stthg*lAG)WL)9*|fZZ6JvF5I$pES9nD$WDjB@ z0KufB;pDqqHe)%38pT#3T_~&6ML=9==wo1IA62tJNcg&+mPS;x8?tnL=dvdWpa<_P z1vFWdYCdZZ@fuIwtL}B#tJTZ=>T^uu)glV+9QaaEm>RP|TUGG>EV~Wz&A%812F*M4 z)?CWORkSzR;^!tJ@DlQdbBjM?NS*nbpvN6uBe3eYsjE0zzCfo#j=K8#l8TFDML^If z+#-Ip;2GdzKTJ}z?_CKQK}BpK`GBLVzb~jTuh~6t#%iAGy&(ID!NOvkZ?0V!Kx^oG zIt$BEEe4Ur&iL2z`Wa3lxo_f4!G*I4=ZdfGGr|ZWA53bvWwT~kv~cR0DjvpN*fU4$S$7LggB(Pm zm%9{FYQj{itC4#^cwC1F2c`?o3B;IL&SDxsN83lX2aBC%=;Us=7onyhNt}1Rx?7DA zjAqL^t!W8GRK}Sg_*QDI={9`|W~stWM@1pXAg`NMG2RhkLPf70=Pd9(0)s$j>vE1@ zx9MAiUU-KM$nKFc8w4m&CcwRpX`IIupGW!ueu~c{nhv}Q=xc$YJ@r<3a26F`Re`}H z21eVhuG@J)&PB9bylxP|Pi0BYU% zDXS6XIXv5^H6IE40=n!5>z1_31-#z**imH74s2785?XIzSCC0eB7Y@oL43MTd(-mH z9aM3F?TWxO7bbVa*11LHfKi{_I)y8r_}d02?*bi1k7@~jn8=<#P8j6qybU4iZW<{@ zHjO%kNV)r+sWczT+HMK}hQ!f%a%PJLlhpz|-U_0KeAP&o<7PuOdjHO3UN9iVF>z}$ z&Li`gDw@Cx$@GZqr=hLMP+^MwsRwB*JHt?gyXxA&+eBTu3oD=|z7ka7{5AtQWj~@h z9kcf|n3T)Szx_1$d`mn|htLIgwZ56|0TsX)yI?Mw^!nqQL|NaC7%dRLh(S0IbE9?R zqH@K1lteXv3Cx)e&OCUr;S-Uh%Rw zWZ2_&UsfaTih9PNsKRgQqVjIAs=X7v3C^QNnd)~HAGDO%B#eXk%``Bhm?N{FES1mj{tMSKq7F`jeZ49f&NrK{Yn2a?Etd9xf$cRV!+>wM=XcF~{2zUTgV zGIlQ@wA>sxb;Qj;hR1a)z5@Jsgytbeh2ta0Xi*4L8+DOX29t^qrA5ys-YO1~*bUvF zg;|?Gt|iWNlJtiUq^Wg3)|F`omFX4bapC*>a^*+^=pd!e~=qcK{n~bg5 znKi=<1(742IOs|FCcrUiIm1{PSPwm71`7FFj)MNZ)F{n|Y9!i{;JZ=iqhtl&EB|~M z2AV3x|L^)Bh2Beis`qi{eOCoQ}uf4A#AD80_k+0uKyL0Ph9t zjM#B18Y)=EWwQgrUHiJj!kb}HQOa)x_;f`~2kr-=h2LH3G~aJCJ+f!~1Ilb78Y)|h zF+zwwDhBLm2W%*2kF~!VQuFIb=dE{@BM%+l!V#G|sE*lcL9HbNief^P6&V?N_Lm8w z&5&gFb2_)9@$GNj*_cN19`%+CLp4?YaT$(WmCh$;_OUnLS$GrtWhJ4=iSnBez@C~C ziltd6SGOswBTB5A6e(8Ks?^!q17lA z>Xn(Z8J$zxakfTc-s<1-ScO`#h=#=14}24Wq^O zufeLc%o_R=RiZ&-7g0~;c+Zc` zrj6_iAI{wRR0v**;dAR0I^!B`vdd7HjzPkBHdwwuwqWYyz z`Ea}CS=;Id<(EqXb_e6AoPn22&nV=PT}|wy@g6KMZFPqYMpwo_jShauUO9$Ea^lZU zkZXD*&K_j4cwC(;-OGTcY}_g3iqYv=>dqS1kTV6PvFer+Z^A=HI8n@UEmNk3A?nL+ zdjU&sRn9=;Wx4fOF?m_>-ghr33PCmbhFxbe|9%7Ee?UfUcPNED$=srkz4@EtgQMq7sz@#TVet=g= zUj+a-1#bV@`2zxm$v?q{(hlKNWh^gK-v4e8v_Gl3W+RrtX#uv+Bvf-~7L4;&qLXYk z`!i$hD@5L@&^KG0MN4;_(iKN5xrbAKghRxNf+PI?afMNf`MuZd7+qkiye=*&leuq= z$qxzb_FaFoDrxzI#C%4a$s;Lki7y&A^fMF71eu?wx|2)$Xha_X#5FFq2GCsx157b zLoAFsJLUYukh}ETV^A6ZM&a9Flkte0T$g4&LATPxG6x~HF0je(BaiP8^PHC^Z$my$ZjbsC>Ts#c@rd{mRrK_|Rg zEjN=nyCC?601A2S02_Ei9S-Imq#~_^)D#+0^m__rS=uXna_Ws!wT9b<#IQS^{535z zg6k?5CLk5nk4`oV=)^B(Ce2&A@Q)-(ft#c(YVUJAkrTwY0V?dZ@uywyj6y1Td2*`Q zv!64k45zKxwg}l1lzk!$vQd=f>;k4t*|-q|L7o#X7?#9u+tFqh=}&k%-?!mXi^bqE0xmMbT=-}>;Bi1@X7GgsOrZw`f$a&DC<}0($UP5D_D)s)-&nBZJgHVA&-X zD^p_JYN;ez>1Bij!H>JQ>Iw5Gkg0i{itJ;#>vVtzb;1Ckcpz&OSS4n|Wet)rv3+tWqF0$7KqV zLzPlOKaYJ7Nw#A_+hKO?&mbq0gyDMq#?JW5cS_qo-OqN{!1xGvh{6mJSgf7YANCv8 zrKS&9e>v3Ae3o5|R$87A(mrRxx6)HMP|1tUyWTSioR%NNg$HcvHhdezb#As%3Uby3 zhsPgi?^BUwa!7Fc9LUynna*(B__=$h26^(5UO#E1SJ1S$OR`4^PLpm5aK26^`TVzn z6^>CNxewN0UJLFoSQeMrk)VYxkZt>KA$s@HPxjcmSly^gk5-L`E#JcwfcKN z=5#O4u{!Lckqg1qT&2INrEe3RbO8B7q;btnoO-sfo)``f&C7viBQUCzW3o~sFXT!+ zL~E3HI!S)YKhVJD29D05;64N_v@o2O)=q!1OL=N-Xt!7R95CaA;;D*^M|md!V-$rEB#UU?Z9PP!jC!0vz}w<68-3U5P}_qM@lEe>znIZm)u{?dv-HnSu%QqIV_=}64z1Mp+E~}c^2cunv9{*mXa9tt& z0LLzDfE3nuL;a0M&?09F0m^p6$tf@gXqa{N2hD1f&SZF-ott_(wuLaer&>6Br_s6M zr$Q3{h5ngNST+e=i1|l5f&16Zs`|i{3hm9j?8VCfAtW?Ndc`!I^Op0vRa>@n3I@m+ z3#J>|=q34%CP6*diU$V=t4*2|0?-CH^)0n)BwC3^&tmCo4qpoig@^hqEu14nx8H5u zZ*UQXo(G6~WukeHGY)rU`T9LpDivFoCnbh(9@a2mNr0Mzc*I^Ns7Yn;qmgbt0efT42FrRkt3wNICiaa~L=?D8 zUQO)dh#9isC#awWGs*?C-9DVpU!cL|WGg^JeCL&H*CZ}=mkM#)pTOl=VCsrR%qXpO zz$&eD(GDv8%qR^n7eY#w?RcXT_2)o)l6a)vn`@Lf$~B!l3U`o&jy=C@yZYECihVTz znUSB%1f6eh5&*?%&bvG!@BTH`d)ZV#K*z3m$5m!dwI!|7OBemr(A5JGmo$Ka4@69O z_|;X<)x9wNfUdKUmex)!H^@W?W_Nm%wFnK+& z51iLpBe|E@88i@bv(WBTD9*+oR3sfJV>f@2heW)7{4bJ8Sj3JxgAF0LM#X`P#%-bb z=R>{yP#A#|1)^-x*W$*4I~18K(^id5!i}-jTZfXg@wx7t|`wuIQMp zl#kX}t?DspTxZ146&uHD`L%V3+JOf(ZA#uH2yR4>GPU^wpJMj*eD>e9e@Fo!>LN30 z_io1v2o7rY|HzL2Y3FT-!Y%oHJjrDq?+-g?MoJBp;jTpB<$lejtfJ=*#Sg;DP<0O8 z@lEns+WYmvn4W%)i4d7F08IA*ivH-*p#A)u>D`>xRNMc@id&8GQvJ%cltDiw#K^Mc zt>9gF6DhOSakV%by3tOXk@Tsed_4>M#8CdGfg2fK^nC%FzVL6 zrg)9R%exqDIf6>-8M9>DQOvY+?>jM7oS~_C1*{fL zTXQM3+B}i*F5|LIyZ^MyhPR_vvy8h?PP5P-%T^t|7#nLiwHF;N+e@$^OkUlLg87A? z;Eq?)?XJA0kh&Z;qpf(6)#8eFFdy1xB|X;#0QS|;h>!aT{73IvbTnCFtx_Kf^FYH5 z=?!a{Wba;!&|)u!<(QA|3>QydINt&CGihWEVPH%zk61CCwzV4P zKovKNyN$e?IH}suKwKFDIl}D^hZf64b)Q30Z^@repVKV{l#Od{?T=6re`AYDuR5)r zNB>GOj(i)}hMB)bfMbCod%sANwQW;n5Jv@P{sFLqf*Psl>n=At zg{rZrlM4=8*|5ZJ@5w{EReDf_K)(mXs@uHBAkkL=jsKbKIPXZ(U{@{nmbm`lqSc^v zR>DsbyjArZQIasyDHddnU`6p~rNox6$>W|_*jnLv<~3Y4efkV6O)Rw12q*I$K*pS* zQo$CfKL(K>X}!VL^ksOrIKAFf-|^RYhrirtq$8Y_3_xm#RMyihGnM;Jhb{?NOu5s| zHpFG7$ok+1WnFy=S1=rMBT-GtZytB0A!u?sQ7I|)HF4L*tWso$4Mj6H&L`-_2ax}8 zJO8Wh|B@JcfLY_8EIyw$tzvMr@^+ZCWD$bODkKhe-S09wRg3C41KOdq1J_1wEB5Yo z1j&vPSAMzYYiuRx*&9Gzk-u^8dG}fh*kd<*9pM1}8F4@v07J!@jc$1CnGw8uu*=sB zJVYWXG6l8$lmOXeH&D!zm;}v0Q-E_X?l<8bFBU|+qs81b<4>-LO6?}>@V1Ne7BD3Q zG%|?)0$Aj)ecQe$eAzv&==qbaRqTK>mduUrMp(Z%gY#zyNhHo-JL5CFF=~9@mBS$T zXseYa-WxHMp^+#9TI-%vG+T>!3E5cvhSBwOpwbgbQmJN(Z8W43nqUCu0U)JUo!)q2 zm#LPBr&CMA%`tl@eb@gA-i6FuE@^B%%(&o@X3~snevM5`5$fq8trp!Z)$P!5Ii5;N za46VW0r+!lL*^=kc*o3R*7r~njK8Fw{SS4~Vx7F+p`g!W>PCiorh1ix@8Lb(D-NQ-!Ipvt(iVuS3QT8E4n2|5MR?~Nqk@knKRu2SI$ zYqk{eX#rDy*1G`WUbH+H%qrpl8B*GLtk>OooBmOcMpbx<3hRzn?ArOMn>RV zbtdY{Z#ixy^~#ZWi>9jsI@@f}WW%^~1L3y2b{HP^&MgDvWNc`B!%FenDADG+Wz+G+ zCK%W2+I^}dDCZIv*Sl}%j%*Q78gh1rsP?3^N{2~arWh~CI@Y0|%pD8r&WeT&Uz80P zo^8B?{^QkJ$ifV|-Lv;@g*>o~g>it#Mei|c>SreX?pOYyw-50~p1PuvtAiIjX)0J3 zb<3@=yT%k z5nb0Cm?x6MU(2duHmaxO#U}x6GCA>Xz7UipC5G6|8E1$`Yq>bN$9=_le}`;&t3(NH z1^qF_FAc~Qqp{BpDwY>BXre3X5*cVIApGXFV$@9kFfx-PR8k`Coj6W*h@Y7@W1N-= zH$D~50v{7pI>ZOH*L3D-T6g~yO;5`@W7B$Q;y+6@)c{!~M3i=uH#EOTw2pE{b(7#P z?va(dHhDhmX~=g1Jp^mR0{cAKOAw_sT_LQSUI&dYL?+{st$FxW+-Ph%sl_U$m@j#R zS}nt45lti=VzU>ey>w`v_jkueE(w9?>|+|@0)@H^tJ-tpOWopTgmfA(!PF_4BMfz9sJ8Q=~;qi6hzZ)aFkS-DJb@%cy_M56Ikm(8%BjCs?dsm2)4U0 z`ENNih5vA@+wxiMA48;JzT0Z5l}giXyUnFD7p0AmP*ZrjY!#jCp$4^(t^@myEMBTm zLQW9NKP}*6MK(TDU8eb8fy>_*BqCf}6DN8@g5)oP528(A9<6cbqY9Dhq#9tLC!*;i zjayCo+FNhz+<_8dYi3bT>XAVV3W;ee;iF?q$4t^;Z-{NpjmiXgcpc)b=Ytu+n?+Uq zs$*>|`QxaS4Ze13&bY$D=F&)gmfoJRBu74fD3ku3MrEMtB|fxA$4!G>Njs1^kKSps zUIptj)s}0>7QW;uabey*T$TX-+|PPa(v4XMUX9@uLe5poXrG{eMWr-+?1Kb`G6#OG zIqw%)Rh6irlrRI};nH@+OEhF2yuy}jW85J@CQLuX{Nejoe)$*Y#KC}^;}3?rMUzr# zQR#yE5R<}Cic?PGstL~w7QMFqSW1beu73-Mh?)*}#=yf+y=(+%D7#e`CK{!~{O;~N zB7LXKI?(|`AAEm$Kn60sA%Oa%N`C9T42^Z{BUvp^DlIfm%4|94D1Q~t8iq6XSeLrP zQ`V-xLJYnX?90PZZGQlHi9bv~Y6T#aU*A;KkMxN&W4rE|%M>-BQ}MvXbBjDkB>}_1 zW5>0yv0lD{Tj)ur4v@a;!!Z;;g+Z?5G;7lfYZJ;bMx}VP`O7hRc1j?QgroTpu?s_* z&0oH&SSa@wepcKrIvLh`Nbp(Sy=c;y*yKy06r?az(Hc8FYtK^)fEd zHH6rPOr83=7ZV2f^wheZni;Wba9bI{U0qE7h)s(g2sr+|p6UR*M^{wp(Jx|-#TBCJ zi!j~zn)TFSJXQ-_ds#4u@XYDzTY|8wsIY~ zu3b`R3D5tz~;oQEwjV9GAo|G#Tmpx-y)sO(;I;i{hPcfvY@nl(`>wfLdAhSbzeN|%`}oIk-kE1eWQb9{Z2@k*=Hia`ZKCY zLjI?`-fDy6Xw6IS160*uhHT)!|Fi5LKXxMr+IVMhG!!n{%Kr>OtN(eRJ^t+Lf|xf= z8M9BY8upg@=M%g{z5=UJRY{wqG|ujYGF;ANXtEHKd7jK`ri_#A4aeXlygm2z^$C2x zE~C6`udqH3t;Bw6vWvCx1WO#Ab!>$AD?K=2gn47UVcfW)UB4Ua*t5$~E={%RFW(qH z!}(*q6*>s7U`&+aw*ceUjnh{*Wt>wdf9{b*+h6TEwXJ&;Tt{_WhTH4HxpnpC?W8%Y z>4fsdc>}V}i6_GX9$CL>vgulIp37mf8mKyr zUqZ@;rIy`bczz=_0P62&F-)#8#O@#Joia_avMM1hBOc!Qo7LcKF3#N}u4f%ZPI_(G zLF*Jl{VvWq7!4Up>7L{wdEU6f`~^j>2UuP*rbmlTt?senS^#)-ETU_pGY$HB0~G9oL1r z13^xByu97U6p0NQjnKLFY?y*41P;4KN>9(7pNPK(v--krg8b?2Cv3PNIzVl=E`J_> zpf#aS1+SpXlbAjLEf$%SeL-lw~{VBWRhB1vWl& zOz@Vhrv{uQ-YxG_2exp{q`7P`st2Ia1B{Vsu1pS|N15QkJ8W~|yiZ>i3;7NAd>9eB zeJ~-WA&*oY7-MRBo*@yBbwp)UYkk3Pzi%oWPcJ+p7aYOprmVn-sxvxAo$0>xip^Y?q~gEV*)B(UOKXHpZSJUQ@FA5qMnY z-T;^=#pZzMo?sTcQ!k_(tXbCry+xtlrPMs8Rn5iEuqxVp{v^g1<2c7p&8~8$*TUh( zzd;cdc(Cl(l?|HM2oM{XRMd78Yxo3ZIio>~M>VeEFW7!AiMF`Q5Lh&|PG*yvC zn3VCwc}ETQ7@4F8#BnGoW)7++%JS4r^}<;8c(2^LkPhfA>LK$MSX#qi;q||z)M9k& zd>TAOql4*fm#%4rT;$7hnTxa{UMzLkuxI4)k6D%G4iLLg(uCMHMwOE0z!gr4rKct3 z@6t~+nvGDUCiq@bdUuxEL^#4W|D+8^m@-nSFLaXwOKkYtt}a-nVU{I^3HszJI?_OL z!T4MvB!jV2A*(=IN!r!=Ri;quHw3fgVya{@RJ8hW5@m&mrb0T7sKS0(@u1Y|=Y1h^ zW5%xJX5k3xPxs_#+_U*kRsOjuj1>0-=kBx40C}sBP;;}n} zQs0cFr2=$0OF<0%X0Q7t=H>#?Sq`;AlV9p<-#YLQr}W5pB&j1yg=z!J`Zt}@Cna`7 zI-J3%yVSVqSNaqa>xmE+(KeG%6P)lFm?q_z4M`?Uc{~ey3)EQ@=%Ao_%!n4y{f&vU zF#AX*MbzwiU3Ai8eYjPOBy&RNqa!UTPOMbF|3vD@uvy9FHVu3fPRrklnu{Tt_igSD zTh@5Fs;AqjGFWUx3XonUxy-pwkCtGAj9ri}>=@{+IuP6y14cby`A9Gz+Y3W2Pegh4 zv{N}&zC);sQDyyhe8rOOKL<%e?c;WCQ%Pc5Rjycnm*BC;U?!9TC2-K9EyGP8LQSas zYDPOah*TV_=nCLg8sG%|DH0avNfXKX^Ny#q4jyY9xfa|D)bv+89W(8^9UW<4(X>k! zAHPhRb<5Vcb0snd*N}ur3rQG{VmR>=NWYm8b78PJ|1I6fk!lGV=PX0bHusU`QqGA^ z5K$vUsmp~J>u7u5N>Ls4opN&kSZS`_l|aA3B zp7hyjYi}wYx!iUidiNnsuUC3HZ}^GcBgO z$RxxHy^3T9om@BTv%ZE)D4tm0i{YX(dh>QO!u{jEpJQgwI)N!f+m1qjg z9f@Bq2+x&^Q`BSzxXX`>mzMx3BElhLwxV5%xGnD4iviw9lGgI7?R7+x5~sTg$_nk{ zdz!PyrQ_Dj0}0vvWPjV({1Rz=IW!~kVFLVv`wYpejI4}HoWn| z8_*Hc*`AU-vagvPs7t}ztvl1&AV6sKf|kYCv{iYDjkvVDAZ+I@f$h~L9~ zh}ND@AYH6eD_77;$F$~##1L}0%dqODxFPW$H_`hN!q&zpwkmLHy`JR~my=5su247# zuzcEM-zMUoCgG82w0R`TTe~Jfd4TzH4So)$VKh@{64`Pan&#Dc>ZsEh$2k1d(;o>4B{W<&b%9L6GG!{zQ)@DvvuQsKa3s6gt)=uy+z$lAW25a zJL$%Zte9R$;`O``e;;FS;`L(|XlQ)8nq#%GWeVhq$50m>fC;Z<2 zKh=!oP$}L(3^l&~r%@T?Qgma@vlIh!2h*d-0$9&?gx3wXZ30SOK_g5&4!MyEYX$mB zu*5pqhzA4JQ~|>J)7%THh><48L)?GRhKRf^7%Ng)w?%yC+|p?I~a5K`t`fANQzkrNB;Z6am$kdI&8I2;9e*2BAWpwVjgJTWx z#Uv!rVF+54sTz0sR4b@RNP56Me};>mCDGbh?{gMV=q$;FL+Ddoci3xdK`Z9oXPM4T zgi{z3GKK9Tfd#1cXJhIAtoo_Ci;xbn!rKRrFdM8}+MRG$__-9UYsejOk0zbhOCf0T zS(2zip7u}XOWCj;@}CV&s<4rcQ}mFiUw1&;gE*& z9*~?uyLP6@7>Ra}{*wI%I>(RycadI8FJYx0`*>~nIN8Fb8fWT8AL<5#jutMPaD|=nz^dB1-xjB-9A5>89?NHfWLJU-CjY2Ud5@o*uzx#*& z7E@)-cV52|$WCG~xy-#Y-dfB+%WVl!KQ2k-*!6awi$_4DtSeq9uU{im&umK>fnB<{ z3^Ic57e9mx9EV$eA@66vJE2Vc4kv*)XyJ!M(pqse%aHuo`sf>9Z403m=e zJ>ionR?v+0F1Z|)iD~x@2TcI0(044C1Gz%uQXnGBcYxK(?z3BsIlg{!ehwvbIZU zny#lHK1<)13?@&xD0n2GNHO~I+3Tea0lz@TQyf(KK8$lua_S!MKcpxQqfQ_C@xyW z|IB=sKba5ylF#P}K}r&`9uYYoga{sqeJ;HFKTEn;QMU^?NjC5-0|J`vuNeb|CIaN<>7O-v+(|kC6{2MvS#9= zuN0pn3xJPy?zWzy*5oE~u0MA^Fx{!9@o`uC@ZM4fq3yDwHU-C5bwcDffBo4(ZT;nL zYdPe0yWR3stNCYtCoEt-OU&bZ*!)7mTh7j9$j7_!-fneiF)(i|WPAGiPR>uWLv9iE zv8JlyDO%s9)BSURV56HJzWY2a&Jnq_4x8EKkm0nffkK752k*SlMQt8ovkkwprNGTd zXhVPIuWMC9IJgbB0`p$?%kC*q&UQ2#o#Z;l;v4wirRA;b`6RQIb0)sKt7FZYfRIn4 zt2x@Bm^vf+r9A#Xv9lEqT)tMU&dS4iRJV28_sWOb&0fi`Cy;O@0)Mx!=k9yIfb*wr zswx-Frm~_%BKD7!e-1GY(gq>rBR}$R$OiKJ;Nw8lcTD|=Fyz2Be?|(bt-Eeu{9U_q zHD`R;{WPQQ3FUl-z-*Rjc&$;_L}W1`Eu?iG=0DW7_r2M3#y}NxiUbvH9u?Hiu+rZE zt5Ry4DzZH;dCus9mc3TNGHwG6WGBR?r@G2}iI1GN9QSX8==6k|?s#4h6SRG0)YRP% zWu-i4CU#s(pKc>MVlmRtzjc~!Oq1zUbUed@o$^t5G8uqhLWwT;Y5q6oMp%D>0>hw_ zXkhy1TljaneAT6tB?;t1D8b%3{>J_KR3UI#G$8+C1BvgO@g_XfN@Xu}LyR-bxovgE z9Yp$mQy#T1tq4wk{p}WJEi&e?>|SN9-B2rftQrF?QgZ^U`;9Y02R?iQzGm_dBRkRM z5Kh2Ih>2!L6;yhT2$<_eA9>z>8~r>dh0lX7A_Bf|m)l}Ap1?Rg%1$YV`LMo7O<3QV zSW_z>zNG60uFPJ!4$t=?(n=a#QkiNP7)sk`ksL8Gm7}>-Tbp2`NO4AVQjZJ-@*80m zmeI(g?jE{i{Wq*4Gut|9*W#~5--MS%#nwy65+RC$KoAT=ESKa1%THYTfheiy&^0vq z4%OgZff3p|W7Vvn;8pmbVa0`6P9~m*DCYx)TBZ0g$gsYPJ&N8tCR%AMz;ooL zU<2vM5eB>n2BfIHhPK$MX4H&C&vR5}?PMGwoQunfze!L&xyGzWY6iKq6mciEd?~23 z7`0)&thESlS0(s25?#mL&nFHZA!$tYRz8wRI$KIU$m3U3qbFghYB{mhc@L={P@1yk zCNp}GPGJbTwThqk(g0_!hWsmnQ0%pJMO!n_&ReG0R0Nq2*SD(oPBs<8Wte|WF2~7y z`q`c6f=_v3l7|&YzBZpQ%fxfR*uUH2Ygsj%o-ZW@VyE~L+$t@j4^j7u4XnR2oy4S& z0v)1Itj_^im9XE{>abKafw$c4*CNs@za%;1qdoUFDr;Yuvvbu`oFUOyI(tw$syj6i ze1Fe!kS1n2mlMD=>9}@?@m;{1UkOA&Ff_|3F)QSPFRE1l3r3UY9O-(3 z(`=K1V_kOFon>0WS1fga*mJ~iSB3Q|f|(v!TAj{&3C|7|8FeA?3uL(?n}Yo_ zW}Ky9jyOKf%u8N_pa+AptOygSGG%Y*qExV!+>-OU7$vb{3Hyr78cR`) zTXEswbs2N=sxs9>i2tf=F4QHRD1iz7!VK4wBN*H}mN%Ib7j)4;Hk0U2!q4`)#7AkM ztQsXotgwr(zp>k@X0j_?1$|HKtY?+o6hsq4s}tB6Q=$lX?gfO|0D&Xonv6HiDgK{$YYZIsbgl%KthtL{hNl6}#^YvPpOc;On5w={tO z=&`5-#cw(M&W_n<8i2TSnQ3aZ#U)(=zPq3OJK>K*U6MkMD-8)c@%AdprkPBNJXI?dubA-X7RMmD@@)laH2c>Tpn118x1obtg$B8QAUe5YW7bl8 zu(8qmFC+@XJvG+&UJ@1ovAaPsB#%SB%Z^<5QNkbpsYR-B1b^H%VB2?+4np?}fUEj* zgV>&~Cm9%TU#a%UGla0wJh}`sssAf`1X}8vc=wP@#G7N^+eeTVx1{vhS+wuabsbp= zxfxHN-O$|j!zhXtJxM9aI=R6xrrRNc@$l_6dZGe;SB!5?p{i+>a2z3qEFpAb~Js*k%|KZ zCz(=RO6TnzG~!w28m2K`LxZa)l60`XP6eOb)lW!}+m+CdMcdO_%T7D4MaWqT;gir@ zjNIBKpUJ%F_Auf#>Oz_krHNxS7eQf1eSGgKs;zPt$K{2N(oV;7KkJ??*oO3~W|*IqQD4 z^k01Ja>R=Z3%^q0!3Lki&J#7LI!TdP`7kz-!h^Y&l?;SvdfJ~7!KdF) zJJF?}{96C*4*uskhR7-4)AxK<^shN{4izI7Ji6hjhV5c4w3#mgH-%Q6uKx-u%Cmvd)|kRs-``Z*!tkbB{|S?CWeiLh`J|G2>0;x$mC}9y{yRcv5s*9IKRTpA zqUNSQ{h6=!A580OS%81|`=UyX^J%j*^Qj=S(<}S@sgXOxe%`gSQoruK-3NQyrnDh* zM6o99jk|fodVPA`ZvG3MK4$LC8@}kZT7-;sc(u7I) zutW9&aZCRVlI4UE%HDOXtu=|~`R}rFL?D7;3yDCpFalC+4A^Xw?ukkV4lh&@KEI-C za6+oo18PAHjUz;(-4~yp)BSr(Ly#Q756fDKZrD#w=1O2fC}nRH0ek(yXU{)oAf#gE zSj+wM2P0%R{~qos%X(1qpwG_qHeDqd;faxbQZa6S%VDM-#K|zlw^Xpt+-?m9eSdA@ff~IE zO4&+iEq`B8vKz(Ya$#H8ZG+m~k%*{x!YV4 zfy1Vr@~v`_cix0Gp{#b{5Z7iKiPL13+!1y4k1Urg{>yyz)ikvfcobP4#9%&-BCBE) zw;8cziBy}E#vN`L<3Yr~KL;~h<)j3HLeV}8~xJz&Z27eGZ3 zKIAlO)(y4C_@?0x$N+b;>W*)ncGkf{XMzyq9u3liM4qi?)(-ZY?kzMZKcn&>oX@)f9;E&)`|+!Aw2b*I>Lap9R+T0j zDi5;Qy2mI@#cBT9oygcUy^ODGOe|xRtf&$bP!>yo%rJ)qrP8ks)vY;`kaCNHixl@t zNnrrBM?ocB(KR=wY&MHf{v$?c&MaFHtO#BvCNKRPb~|%mU`+KD3&0SJv-Pl0F^FMm zAW2u!2?;iZ>rxmlzK}CW8aTW4bW8UpwzSxC5DoV};t*9uxMriX?61suoGF|e3cNVqjCJf2p22YS z?Qx+hSrekxyE}xAmBJoXDOwur6wEC|?E{0`>Y(&QzSZoM2wB!vs#*q0Rx+bLvOWw% zUV^Zwf%%inydtxgPrKm$#*i#3X$~4jS<(0a8+4XtM0hFQ0NRSDr#nowaM>(C{Do$! z6XKE(2ZmPrA~U?DcZ5MlCgRzzb2KLLSfPZhRcE<4NLTp@a5$d!`<+iHoE zLJb@p=<=4&Lrn%ov(&^QvL94jRs7OIeAo03q7eshW+`)vh#%`B)?d#&+<&D@F*1GG ziF9hV)3p|FoWpjx%3NY$0`--}@M~ZZh{w#P=IG_ht+2bGkFs8JIJikN>f0K^3g{p& zIIW~y#4!AM$U)S+S%k^ifHYtUgN4eO1%lI4-hH$bmtQv0eIfG3>l@}|U&ICp^GT89 z*C0pOZI^hr#r|kf<)s=Dho2p%i*;a3=EO&eX#1b_#21llFRw28=C6EBiPYhkF12K> zwByfOr#GFt)6??_3~Dh+N%~e;NnSxHD9vM7f25>3t(U(CyYIzJ$)se~%Q<9omA~cf zoD1@^$)g~N`+h*QcRuqtRbP7v6pw_0c<72rMX`k`Mij|^s3Zy2dWUatV|`>4plA5E zp<)jnvCgZ9hpgF@&=zIbr)#i!(4oc3)+M4+3{Jm?7ZG3KH{J$pz;;43qq`P(SmVYD za7Tp(AA_xys~5%;-1C6Q}EHgUebe*lAO7`U+|gJ%ubcFUkPN z3#B`i5)6293O*^O@!p^*@q3M;kvp(x%{}T|j#?VCRn=09^I>GrqI+mw?#_-gZKW=L zm$81FJ$w6_sBO>me-r~pMcgv8a8ar7CjFy_4Dv}%k>0$DBp+&wPuN8xgK(vG+oe!T zWvoL9Is#1ZQ?J@2w%bRe8J4-Z{^AaaU@BEEBPbz8GF>bXOR0`{gQ@0(sz_p+vT1pn zGsZM=dbtbG<{RNO zk9+7kUE1X%irGl%K>7^N+2_iUxh<-c6nv*+04oKG>1wn9B3XMgQ)7=A3~_SMuCsTL zj5Ae1h^ls>6yBTdLUqMS61+;WW~uo0$zGgJPNVoVLYE7a?L@sd%_Qg*5>-*|@SJF( zRSC%`^05@{%H{p6voqvuLC||`B`i$bz*gDNJ5f}d^pkVz1;#+o5@wFkr@W`Ys*{}d zB=LW>>}8&SaMF9#EAzK_u8_3N|6w zF6hWvdED$j!V}}wlU9{o;L>?1WWQMVv>%m-IlaF^X>_T@VIH7HDOM_HX(HGV?p zNhX%d5KL>=&5a)qj{N*m0g9`jVt(G1UXJk^6V(EvZ}3knkD=ZbJ@^5J8d3VCWPc{Y zM2Hr}+`}?z44t;kJ)S5fm-=Ej?yZSPiO9Qh(Pwj8(0;EkMspe~Ub=N`fbx9^4`!3U zGzwd|5Y(npToKuh()mdcq=YIjt zwI~Gly`rAuJflc{Q)xmS4Dst^p(7IG6wisW7@N46vBZB%lhXc)CSNul&s+8+yqV=5 z;aJI&RwgZ!MNb$&a+{fYSVd!2*d`&#A~U+jDR9`jJ6I-#e$c;{)}o){vsePS{cwH8RQb1-I&}5DW(L^@kgCk7b8j$~eSDEG`91C6u#R{4(;Q(ZR~v{*Ua zdSF)-lqMi;yP5T$3<-zNYi%O7_#9!^yCOb;rj&dDdK~KDA`N79kKWXqmNX&&K6M77 z98)8=v7?%F{bIS772s4~3#Qw30boBo%r5e8h8Pu_e&9SF+CrGdij~sOAvV%WAvqO( zPRF$`+X9YR{a1(eDDb$^Nzb>USk_C8YteMVk1T(EIhm_O1_dBRys$5S`T<~PWC~V- zDB93D3v<-H0KP=mhsYTj2+ zV&S3hA1MVoHY&yc3nzxte}ofEXa#E{ML*Qg-T!cU`~6TuKhKk&jnJMpIH(oO5>>~c` zedgXRxo!V(6XWsC>t}?oBHpN9Jg;3fj5IFC(GK?S-%iK~@4(|>z9Hj7YdV}Sr2bxK zGg5oFK9%Cl%5bNwayX=)goUYRTtSDjZ@iwbfGupj|<6-5sVI#$jCwhp&TVLIe z1GdZcfzwamPxDR3`AJRX#lREH-;Nw_D0rR)3467J?!I9t6kkh>yAD2$0ngJ4DW>%h z5sn3tUr!}TjQj=ei65oQ;tw*)V z4q@gt_T&>Ve_=U5nKNNu{KDn%TXe_ib?dBQi`PJZvGCnMEit+%v#y>w84-Z@BTUxx zRCki_!dM{?*&feeyTGNtuf#ip`a1?pLQ)1yPN<9Jf(c7D5NRNfM8Ccw6s9;8ZTKSh zvOpT7)#&GCCt0ZXx9lMcZf&5dRfm*VFaJZ4W-?T@FMDi5qWXk3GRM8Aydo zyG?rl0PYj1P~bnCP~53O@g!LH$mmXVr!BVWbP~LH`Vn=-ADDG2jcB91+8!uvN0_d4 zlfNNRd31WBzb+D{yV_p48^=5L#jt=jLsNh zSBjrr$p&LbgE=)sV-Wg<;vR)g3g0<+KZxGsjjW-Kc2gC$v<&sa7Ce880)uEe(RS@V zU{YLAJL!|x(DN|*ul5^LZbwuh2oJO1M5vi)v}>lC`X7NjUiJ2?_SJ{^?JDo`9qppf zr(7AQ*)DUDJTRGXKr28|{_#!Ox;%XBLZxprggaks%yquIsymct=aBwEQ+|1sFWE6UOF+1+LzNDIrGvSpA_g~Ood7To?KzV`6a@3&F(qV!7=dyl9t!)(d z?YHqsfEX=?*)U}+{EOg`Ejef3KxNUifaVjuZiVcnQ79yq%CXio`5~!a&G75B2rO80 z;VNav(c_Q34!#lzHhicu(+(P1LD4FFv*s0H(~UPJB|FA+pm|&Tu9Cj4AmQDwI`9p$2L3_X$hjoa)V?5yCLv*!r^$un_BerxeOG#w}tLM@sySs<} zk={6rboxvDrHjX7&wzk0vSirxh^5vRtMImuhJ8Y*WQ-&DMx&@o6J#SCsk;fMa zvd@xuV92~ZJ=32I#Syr}5Oz%bbGhwz6;muMNoU5*{6*3=?9-l;bx}2hJBt6*lM+8mhES=0Cint9bDZ8wM4tY z5HKo4;dhtv8I?a^wLk5>wmlipkC@`_a9$Rfdo4Zk@R7Taborbvw#txlh#~{!(*SV9 zUlA7x6olmv|M2D?=Spu}16Y=%T3Nh05`};~Qn|lkgX`o}J$QhI7GcO09Sp%dMDYZ3 z^$^8SXyv6)BH^ExEqk(XWv5+rJxdw?Du`HO2T6MBX2W@!+xjMkZnbz*3NJ2wIJ;hh z(ZF}mZNC7K)KiVT-Ou=#*N;NvSSX$b?9)iOpZB$7cSl zqz~ee(OseKAMAv!)S8ZOCkq6yx4R}zu=AJAUyY)T?PSEnb^$va03OM>57H2f-PjoU z)GiZ9uP7t4uw|>?VpXV#>g=mNpr2HS)X81N=vj#9#KGspDNb2&5kdY^|1D$FAn=Pm zjgMA=Mg^p-LWzYm#i}gV(lXI55B&oXk8O^EbiS|Zs!`$@JPg$FZ%tn-d3D=*9jZ$f zWX?kTe#n)NugW3B4CR~p1JjKwgMs7%Z%#$afm{{p@uX0UIE319byJxXp(KxXtbSW6 zvP}KhMwF3%r6kLru#;!{>ATQMixOV|gmRc8En26WT)AEHHE^VfyU1CXe41$D{CIm; z8BNdL?VMBk-Xgj@BvK}JQX`g^~Z+5V?Fo6i7^f~&c)Frfc z6&z4x8j&Q6E-2L!`;JjPJ5n-zxQ3+j4XgpNToBV^d@CW2y*AMaP5IF0@^=-JN^V?i z8tb}<{ay$IwkePdlA(&@Yiojo0NxUtij>MIXPg%Uoy3 zrvy+m!Grq-AusO zBkC7QRErKnl$uIh=PWLO$d+OESTRP-7XC)$1%J&L?~Q)RL>E2a`uq&%;{y@h?pcoO zq^b30Kyy3HJzDG{^I9m4KB~LAJ7qntJ|9=OVHLrxHSPrnzKHJaKbtx`Q_f8l0jO$R zT@NqNR1WfU4zH?ie$Au4rR_%p{gOA|Ssey%3bbp`3z#Y>@Aza~<7!6CbG6g! zlQ(mLp-x94=tXgyG4MV^(QS{L{-~-IxpbN_n{(ZOGut3z;t!xd1CD)xD=B`tSt6Yp zJ?SYLu3RY;q&joAvDdz%U)(453yV|wMC&Rge4aIeN9(m)ck=n(4S6oCyBm>O+ zi{}kTc7_duY^g%b{St+vLr|=|YuyC+oO=Q&S{cpI zo*<#xiX`Li+w?IRjn5O`?0{gPe$H+#F$vbQ;zgKLG4eQ27ShT-B9ZFiQUI^uBBs;K zQgRzY3>3)IFTq_YYceA1Ipv;Dc%l9bRXQvgZc5Yx?*0LtV^8(CK-#G@c`;cU92jGI z`+oU~tq@(y&+7?VOYN=3RzTJh{3|>}S42qR160_xvoS(g!G)mTXlx!uV(E+%-U@h2 z-)rshTn_Ad?wX-olin*kW*51&ih02#WDH0Q&fP)#jW}LFQ#yFbcRCO<++$!BSoh0( z9K#ZMfosIJh;F1#ah>KUs@s~rPZQKSTEpg%N{uk*TlR`7b^U=Z1 z*?0dU{ZNtEg71PK7bFTV!L-K3?h`Cj4n&j(H5h$e64I>1zgKh-qeTu#^s-4_tto9D z6ssq6<<^grbuW+{r9>BY)f=)<0VN~V2uDa-SQi1yFJQmat6J>+1xF%*Vp5i)1qM4}KAZ;7S+@XUaCeTsF zdhl*OBnMd4vsP)0EKzH4rR^vQpFl@#Xo$pJv8{@alQ*36s;I~ZvQ=I z3EI^LLVG)1Jrt8|88xKfM>IEc;ghNHE^Iiwk`G3N9XBhDJkWF=B4ifR{TC3rM?4K*B;@DLqt)A_JOSwP!yGoX-JngDXg zyLA!!??g42v-W-e&{5A2@Ip02mz!Y}lTJ5_A?^qDn|;Uui#yadD#Gpli~?Dv3Lw8GXOsn_p$X zjxy_gXY`5=K~aZLg^XVC@ugiOP|Y5<%E(e1u8sP2hhW;`LyF`_TLqKeqO zz%klU=iJvoF-SG%81cbPX*0gMiCqo*lwFREGku>`QN~P*K75S+F4q%yz=`US9K@Fg z!26#RNu`DJY8Ea+Cx_FsJy0yCc-sr$K}XD2&@xKabY8M|@ktHfOvQ>q6IvJav&SEYhBvC-Ia!^~i- zjB4Xbh>keN&nI?j!Oo{J@27X9(YGl&Obh+*0sxrgWXlq90jvrJM}rt9H|S1Di{_{S zQlHw?qOiuKTPz6aSK2^MiT?=Oaa?;S?2lxL=Y|vu1BLsc%ynKZz9Q(O(x9G3SFt&k z2FyR=@P1aRgH4WlzF<`-{qh<$B7*?x%sEC76@WWp6YF&I|xV55&_Q^TgbS(UUL_yNHa3g zvK+m!S!a!t$R4Bmf|}^3!FK#P18xHIlKRwR)x?Mz!a*Supz~r)8$41)0J-o0vWcFg zbjEJA1jz+urf0F6D+;_WHv~*u(;1aH)mYIGnyXwCrn@u)V>W6Ot26ofXrpMad=w_1 z?!DEtmSZg+Q?<9GZ6{;{Fa-BB@m^UMe~fDGk1?i%l21cew#RzuFcB`+e}6PjGz9{j z^oZ)iTa(%{klm%%wC-QRa2iOfndzu84lJv1>!9%Tbgak_;eAFSQ!5$fzFxCcFllgq z$3NSpX0b5$oNm3aS7tb~m=>ug{K27Uo3J4e4|Ri$)iol!M`iiQhsF9*7)IFTr(+@( zTV%v(OxSgF6fxd2U|+zaha&4DVw+wti^ul8-MuhI@QeE$#5ElTrv6z*s$&;#k3-oa z?WzXdTir_hYG9qj`8z+%9tW4Fq_^1782KVoo})`h2=YqrS8(kR;n}94yH6>FMJu&y zRQaRUFAV0m%m*S$?ip%?`Gg&su=Z?R$poJ(8i2*Kg8awdaL;31BoDMq?6V+>@EdwG zabO|`qHeQe%`Vk9Q%P>p%F3$KIWnl5+`3` z{B&&&YY4sM5~kOfR%vXFtUxWF05K)YRae_?*w92wTslVsl!$pG?ggvH$7*QB{DKdk zZ9Y1d6Njb7Qy+$&XxEz#Yo;8_&uP}v6Yx4>NPZqP>D}Uic;}nEt@q97+T4;a55B)d zoPWqkIyc3{uaCMs+~&(=$WAI|yB*cI8ALI>+A}v$6~V@H3iFnkZlf{RnR;SR{Gtc1 zQte6onB+G=!*~5M@n(O^xOBarh}fi4`%i%!=DCwr|C{vlCDl>F9s}uYG9IEYM?XSw zjP%8hJtfoIN_l0Fho%<~lY2oLckdNaV*Uw%SJnNV7%w<6wy|&Ntdp<9BwJevE*)=5 z68EL$p;W_mIq)ClvVmnxd+=-2lfO9BF>9H-fCoa9xF1TQ++}koW!}tWu5#G4a=qnh z@Xzm#6%xNH@|&>@wUYNWmzD?LN-aCHVVSJ4MP7jV8a3svls75&b)?xc`HQ8hK0aj2 z50mS26gJKs(1<4DC+zUfrn>W-TTcnn=@0$X$5t$H(!z`)^x+`n)*n`@$5QDW31w1i<)3w z6*MEsM$E7VX(ET>=i5p5<(9wPP3;cWCTu1P!uE5W*$vf?!0LZo?Xub4?xAxn+;8>6 zOuLbca4zlQhJ_wp#;_2(0pF}e$urf?@?0m~dC*vDAW7j!{JNsrdXwc#N*!@`3d2`q z1LqFqNrF^Jaz(yp;SnY8yjFW{Jc-`b`yIX1b8gPkA%Su6K??nu<9}Flfdy*1$cB_J zlG*tYT^^X*+?ISz@=j&UoCB%Fr-mxu{T)Pb8aGKkKNMcShFLcI9JUavDcc!kBPnn1^?Rh_G$e-1QSza1ugVOboQGmhN8}Iu;2O)i`4*8txCjmI{x64*p zu9DqUlyfAHwUJ4QOhq~QGiW7A##{OB{^9dZ*UhS`)Kc~qTg^NNcg*&<4Xy1Lbo5KG zrMn^cWfD`0?Pbhb8CI~m*jHo>t6cM;hzuGccEG+(+hk8arwZMx7$_Wu!oM2AqqvQ6 z&~4WcqZTCS`I1Lk%{{vMixDZJW4Z=G{xhIhy;XmxVS4V$U-<{n?O0k6M)P02nfqH5 zdSXB=sCkP5#X)`fA6}P>*1G4S2GWf8qB;mffb&^A2uh0pgC{^+dRjVq{IT)xHRNbT z=)mu;?{-}ld)0nMmN27qsqdHaaQh#p%kOg)`mIcBsk5r~Vh(1#h2nO1u(9U)(^?~} zs@CnUfELt$|sk$Qohb|oiFs~^R;fqczOtT1cE2qvn|=F{@5(PQ%m z&T6whK!0V}QZ-CwwmTHvUNwqbWdDfv>+=*`!+#(k^3Di5Q*^mXhCdaAj&}qa%pKst z)Gzk*jvC*Pdh)=JR_6pxpYD=S zLX-%s;Mfs0MYqu#6xPLzzCSJ9Q1}lx)j!?^+DtAEHtuXhKfrg(6%eRaD;OwvEnnk= z6`JagctnKhCyv_7rh=M1dzU3TZK!Sw1=kZbN9(r~P#c28(J`+*jaCH>CC`*mI+!&* z63vcZ0G`!|Z!q%I=4Q>d3Ibia*t)PoT!3=QE6yd~nk7Ukmdk8QiYU*m|6g&BaTp)m zfz)m3PXkiPCEEc7n?0d9>?AU?{zBbqAhPCTol7N1;pVz#J;_^SSCF=Q5X6Dgk&ZVB zOkK5rL9vb2WHz;}wpBjyz7B;|{U@ja7=9D9-{Ii4e`hb&D!fWgp2X{&hk| zYI0!&yEXlXNNSV!P_<^1ODBn}IJ~3D>{tlErK%YrJv1%DKu03YoN*V3-eHP#791fG zI(JCwm%zwxDbW73I<$b#5w<4)3?>KuMPVm;Q-m8dj5hDCSc-2w00_!Xwi-bywF>8{R58hhQVrEBIc5gNi^+~jGM=fxcg zHAF88dMMwvjkteTZb5@Sp1{zYD8%iaT|N~?8>5h}vlxO&h;j)P#4p)EwMNs3Ti~_I zvMAp;)sWsXY?C!SDLM?dYZy2gP`l0&pG7@>Y2C%519dn;;wyp1bGL_-1yHE%F#3chwG%43Jb^gcR z;^7zN;w0EW5VJTr>4qXX$X39WoBf9sxQ15$Rs6@v7ks=MCk&G7Y(v>2TVkS6{1xzf}xW& zKN5vZLEa-tYpW)gpR?6pU~PHF5~nIWuJBF62SR3BZonsc$EDHB*XdAs{bE0naH~kajN7|Kfp5}#+v^Q{MB<$boacEli=x4N6 zz+sNRu#mb=8OHow1LqHM$ftQ`1dXXric@w*xhA4Al${Qxw{~SZ^u3qpgf50BJ)mYL zXNg&vE9T^qxqv8O(CV6IgkmIRWD+QJuzxERQsBC?2&J@+8uD5CLd-U2$g061(Q!xc z4=Iq1WTrojHOWLTcA!IrO~~+PD;+LIo{o7znA27G+~U59K@ekIY~JvBLR1o{JeT~{hM$Tre}$ZB%i#TCEI+lWePwsFf43uAR= zuWt2RWkc%C7#3u`BxV`uFGYbU4bD40gDxq>oAfP}XlQ1+jvM8g4L@7Wy+k|{B9Z7t zGkSNOatVa9$O17&;hN158%u$dU2GpBJ$t4xziw&zCMx!4hNjB#G&s&Yc%M1z!Rh){ zH>%jnvsg^gyO9agRuZ-|-gKf&*;YT&&|jz#vRL$AKxP*iumzV5dP>hmz~sGLR%Um{ z-b6!cHYXJlvKp|cV@z-;FxcO?yB6EeJ^$P6!U(*NK!}s~sV$*Rw@OVRMiz@KR|n5x zWg0fTf?LpiZ)Xi4jXac%17~tHziVHG#W)dfK*f>^t+>=!KSvU@ltFiJ45+17)u8Pc z9AJxuT;g8MWx`g#Q722up^T^5YWbpW&%C}_P~m^YQHb7xUX6sXsCB}CmwP!Azhq3% zd46quM}NBS==|-F)zqaVTN>(cW-TaSH9zC~V!2_te|L2yz8)+YB)`(b4U(iprD7H- zGvGZUs8|D=Q&PS>T_%#A)wcdI#X%JXQZQ4$FUF=9RXnZPEuv2if877g?h@&=1jj;m_F@^9iH+xJU)PPtZ3 z{tZnIP?s@ABzl_>0ZH)YaIIgG|W&u=WsyXqS}N?;2uMJ;S$!R9%W6TB=I zDaUJq6Q;aQe1Rlil~e|4AnA736f(4t80DZrbA-050kGRF;;9}GakU8xdbarDW*XUI z`bI(Bre7TopOo2ei>NXBylcg^pdaRe1}5XDBhrEFnW(_zyQ7;RIFeaq3CnV z9mRB0SNl7MjpszQVgaSEpqEiGDJ*AFpwuz6NYv2tZ~ zQR<91QW1Uy+zRDMNDC!b0J^@P>6m+Bb0fV&@0WkRpZgwyiKXmI77(w205+EYB!!Rq zbeTvr6@p%;$!#n{z3qB7~y(UU_r5@)U>{TYpxD+ho00?J&pcfxQJR#$uG zI_~)(-iExw1m5@lDxEOE_Zg=?V;2g&&CiB}9lx#TT4hY@WbTM%iyKd3o|t&P{nf0%hDY+GtI_ zLLiC>(_!U!r8hQW?lbNpq4yPE!Y=M$U8oEeqL&I=LH4~a@Z~9$eYfG_twn~+sQ)H3 zQ93@~eucKndzy+RqTL-4u(u39Mm|RI#HnKj~<7W@^Q2&bvx-_di{q{{ec_>od01P$;tpJ6CVSN>*sCg zlbra8MYxjuI0z}WTF;|x|84LW&GLnpi(h^j9xL~e1)Mi4L4q>+b7ID|Emme6npx}k zKEk)yJO2)jwGG@F_kTs8z(hdQEn3vEjvd5b59@zwicQz02}DwAj{~#l0OL;_ftLDy z^b=q?ya!sW@;_qrG_V~w>E0H6qX`^Pjl1 z)Mge;uh7|jRJD~HyX=^LcDYZrHVN*u)6hq@P*<8|%?}u!4p7=zI2h;Yo2stv%$GHd zWg|qNTb;hFchnqR%6?8h*IKahuC`xGobt_W&_9g_ylkqBZ>iJJ9McnJ4I&cD;o!Ja&EUJ$SFEntA+HT}$$_ACgd}4Z+o6AVz+^-qH;b27Uz zb<$v8yV6Fp{WF!-`j2dJTv-*y<0mq`q0b~8hz3>`dD9$HU&G42=inN6v8HCLeZm1Ca-4w{QZ``tQ@WsRTSO9Dj?K17ILn2{TJeJMW20}#f6?aeK>SHTmj`k!|K)RisohS^@n zd<^5$%|igd|NNj(;Cam)#<1Cpg1%?b;*>rUP&}yTYfE#TQKv|s{jIfJg^3hQh7puQ zXt>^}U)x6@F{#iFOc2WK%nsa*71mGb3oafBjKY8^wM>h22yh^$ydNXZ!cz2RsVgjA zl&_(r(nx7Jr&hiWXcLOKHpb`tot4^%;h~i5YfJ*cIxfiIGHkM{$T&R{Un&JTBpg~~ zGd&W)`JfslX|Yn5@@y4%`9Vod@ko18hPtLCeiD4&5ix_MMAAa5uikW3F(dOpvE0?% zO9CWCf}c5z_DZiE{Xw|?dd|W6&rRPc&)vL9{()w$%Ddil?dSVv>?8T22P=^qEDJo& zup>T%WhhNA8Rfda)RIU*?_&Y7PoQ%gutjaqY3d-j@sJx(5!Q8-&4&-ki{(joJ@&G{ zEi63e{XwB*$+{elXRHo%y`3CMZo`>?iSogXBV2az;keIU(6f{oHo$o3xRoRxHW(8n zZK|{XwVVw){De|Kq`}NBI9)IK>DJi{0t^~4`#tg5QN|)OyIk5N7r4J{3a00AUJoY; zpvd=>!b*>|NHO#w91A!@*2MtO9jw5HC+RLr>Nxgg3qaFd7o1@0^mo_>^b5|Itvu>guXU578JB!R}j{k#wChh1u7g{ zkwCiCBRJg*UF9(hw|{SsO0|ywMF<7<@h$K)8OugK= z0x5#DDzisY<)sM)xT9X6hR2(Ow-Y?JeqS{9i11=zte(4RKXFK&x;BOB=P8d+<;B%x zAdu%Lfv1w~MUMD|PnJnLMoVH-Rmd}AvYGGZ+k^;lI%dUBBVA3yAA}|K5bZ*~b530t zFX<8lP>1C_820FZC4ASL8PTjmdwK+K`Hp@3jlkK4K$wed2nT29RcxgZBf`SG_UKUvAu+F9MySL$z@iIR>~l@txxl z!J^@B4LJ!fYWOfg>rdVtry(=rk+pt%tYWiPPAIJQ?I}Gb^FxqN$x={WaZG7)h9S=G zY@fQXQtxK2Bpyvx#1iBy=7kl~h0!s^J)6HTE}}&~B#hn_m$H zdp7)OjN6%3NFLvQJuz;8d>E}VY|QH@AL?3jwJvN>?|w5%$xPB)Yp`vS3~*bdAUF>< z7Q)K7jxf?4T~vP_Z5&~*o#1a{43pUK?J`VTa6Ts*fM&P)apv{&PmD*DJYDL2u|ZJh zef5t%yM}Q5MuM^-Yhc8g0QM1u)$kI}Fd-{s?7Ub>kvQtS?IzF<=?_<_DM&a>(1L*h zfP{e`vHR;-S?ts6y(xA-V$=2Rh_(~F=%Z7hl|!cB$h#6=3gXL^nQK{PIG0~e67niDZ z5}F4Iv;9#iF8s8`rkUjQwdjeeM&4_h{=;P!oqcAZvB8lz?TG}sd4-g^(VzITF|iNe`#-q#n(yH#PDhg6PE zprkzj7cuE`LWytx=w47s#Ub)+dit{wtF#o9J`=7hUAchoL-(9^yy|@_O6}TJ@a;mS zhM`@W2~w&Pm`gFb$P8EA7B>BO$9v2c^y0HQs({6=-CGp8uU+}q3tft5W&GRZUpg~& zI>v4p^7~q%GXuO&lE-nE10GdO{>m)n><*P)r5C%>Bsmi*(Z=kVpj(c12~A0#fzkC4 z8q;0m8Rrqr`%O-#rA$n+f`a2!U$+F^va6v@hUzmX{{Z=Pu>Au_{`(SG9g$*U(?|uETl8s(eeIg;AIFaJq;S4Y*kadFZBR2bNX)tvwaH z{g&Fww8s9RQ5{%ph`zNQEd|#nZ7gg-*GepmbWsJgY=+%6#`TtYL_F($fy*u!Mz{{L z8t9evl^gY8(*p{9HLfez8s|0<@|$!oJ`Ui!kLR0fG?iw$)|FszT0?VpsfM@}rD5B? zAtkbZqfED2d_cid_j_ya_$<`B1Xa=O?vb<~A=8t5p-FFwMhEAw8>%Z?P4MWKcCWJS zQ}Y@s=AwU*PhpRL zvf&!syUuO11UwUEFRp{?Cmi^R(91Ff&V!W6&Gv}>I1fFUgP@2$OJ=$tcE`bNNM>UH=T&44j<-B8K-L|EFu z5Ml7Pb~=iH6Yt>;+~DGJMxwH}dRq17tUH$oeGRvN+$ZHyXWi{JP7`nfFq+iP1u5di z`b)u;94fgOFf5Ra`jarsbKSaCEH?MhX`K{3%q#uaB$|RyL+K&{CqtVUe^CmqDThB+ zH)X#lKK1mIOK?{T?zO?Bco#`PZqd&*cEM9}RqHWCHUAh}WWj)dUtT8#-G-Bl!CpI- z&=xR0+O?FmSA@1dbqwfSTw8X-z?cZk?iLXVDKuH^(|TmE2fcp_ifF1+KPU+BrtYF5 z%$P}iHus+H3~E1kt`NUXN*b)sEW}hLU?A3gmm!mlV6g6@o;h z4faF~Qz*Neb-YTVAsl|wRdCJT-d(nGn!UZU^Bdyb_AP#^Ft-Q~HxEW?jO$pQ6w%&# zkB>-Ff*cv86%1JG(ceKZ{4qvlce2z|f-S{}I|y=Wz~Gj00a}9tB{Qa^DHnR`6#3l_ zf*P_J>dw?|V(RhJMA$%CIs3F072J41WyuQ)@vP)#KbeXeKI~5>>p`U3It%f9qps|J z35g-~X2Zo0UtyJSFDQ9A>OwCAbUX#CCZF7XsqmrI@R+9*kAzy;-bCb=8GGUqp(+4@ z+6r<}k!l&)yj>=8BEqe#E(V5f(#O9&@JR=t90N}T7>YZGsz-2CKEYn35q6N4Y9`U> zsSe@C1`6E*#iSAyvRN(OebAW16!BMlSAy;FPDM7LFLNPl&+`w>z(}6K)O30TE&0*=8#2KS+rNAG%~GVN}&QC?WX~B6Z%Ou1L#F zl40%iC!sh)o+DdbkUiUHu85pmOHTP+&j1?5PT?Ou)PP@ZsWAf2;x}?AB{7Dci_uso zv(Gpke@*Br8KpLw(x!Rei`Z3wr!dNvE6VYky+7#}MMV#a;y}M-@Pk30xF+rNX z{dsAqC9trw@F6!zMZeGDYSoI1vc2Z?!fG?2kx-jgx&>K+3}X9e?_{e+6UhJK}P?i zD~Zg7GyErjv8KaaO+|tqhme%RdHaSN_ixOS$OlyJ6$8tA#`RP8O!5*WOh*HjCqz*b zOH+LsQ}{>0!jBWc86I3UjxoZv8P+{qCM1AC@4F)>LDv`C3eRN!b^dPHAlaxEqZHzIU-_2BfFBz{}Zy z$a~P#O4HcGp~~Xjn=whx=`P7u;PJ3Zc@lk-Iz!qZ3l!<9P-iZ;OY_z8LtsEeXo()S z6E`Rmwo}3$?w#8jzBx4P(R@vI)hEN?ntf(;pVhuwg7_Q`!`$;s?SnAMM9^B5YMPaN z#QB2y_Tq!Y>@2O_TwBRMC$>4U-5pjHKLcmT_p|N=1;DS}O~^4Ri5F$!<{S))N#)yU zrGyi2{a~G$=3#mYqA9s!al8``vU*w8O#uXpg%yS5F}xCa1r{L1*4C$qn$%zVbIIf7C9Pe#YLvI&>q@4wn;Vkbw zJY8rd%LLg5qggd-OQ&a$sh!)WI!PFN)T+F?u52YQH|tFJD5E@SHfAONjp{Apqmeu zGv^c+=^eAT61Cx+gaAh2cr1yxV7uWmWjxw{5Cu{5v`cnT#k;RthSsdkv|HO2;}M=sq1y7Vq53=L%yy8Ri1pF*ddScc1PLt&6z zq%}N9idV?#5Y6w;yZH{1<8mkok+w-vr{7?3s>!za*p$$15d#gy=Abh2pyi-4TW_OX z9dsNi3HOi8{8;D)TwCO(-#;b01@AhP)dTsfHV3 ztNC0kOc=phBK#3cE47FxR;6H`Ttj3&)ul;Zli2Y6hy>0Vh4rlfuwLQqFK6hH_%LU> zgPtvSP}rqspj3L46Wu2n-6{NZ(Hh==j~nX8q{8>U_`7>HtmmZSe20O-tkVIJT|UA) z;vC0`m&IB~V~MU3yPnn8l>~s_RhVfi6&qb{zJ}sJmjNQYU|aUidcVHfDEV)a^>AHA zAFc@TOrS4d`Jb|2Iu!?gNJ$FG_vq0xU-KqQ>my95lQRr>jCM}` z=a?g_pSt4_F1-upOE*+TRXDbQB(3Q;=Ql23wUTt>Cu2vU9S)KfE5W)-I&w~8C=r}a&N)7VSSMsRnn^G zaVJfij1WtOQpcM>VMwiaUhhb+q|hy<71upM%)-JDRDI~{Q;eY}ZwSK5{^aYqHs-`@ z(W8I_$`Nech4l#-=?^HQC&T+@FQAFakCf0SGF5XfqAFg=e5Y*=FT7FU?Keac*-dG_0@t8EOmqr}yE4 zGtoc>HSW_WhfA*TAlMon`)(HFWB*aD>>r=MnYij*sHpZ;bW>6P0GS;Fg313s;p?{< zzsG=jJA69-OZbZYAB3;pc8cp}2coLFx<1mTgAuvh!CQL7tb=VNU;gR7H#CVLl2xXP zd&IS`NN;;OZ+gGK%4c#oNI?6|I`iG`pT>2+pZiw&{k$1;t6sZ~hFdhz*4N4fRnbeo zU!ZU2Ky#1%t86!IWL?XZTaPEz7rj2ZS8Iop_AaXK@pq3#NJF-YXXkBy|>=_OmP^8mj5` z^OMjK{lR)x0c}>&T_9=6ZD_}BPZ+uUaA{Rg-~-Lo{HM*Or~Vht#4;{)uyJluT@xG5 zauxOht@**FDy9i;;QCgDO$S_Juv1Uyt75e2qI0*R!QU`j2<+bt-MWj(6IjSHruYQW zTVl^f>u&AqyDQS-v*+)?DOcgB!jS2jW-WhFVpx@-6t7`z*5QRtt!basBQx>EgXueQ zcNy+}NoZENbvBoFs=z&)CU(wRYRgvBM)>_8b7ROt{MEaJOM*|rMf%D(5=$0!bu~V7 znml@u#dp+p|5Xvr7fJlY`^2k({*>VRxX%@E;;Qf4m5V5IrI?>1c}LdD{M&{#p0ZbB zQ;W^rfyX;pC!xeI`NMt|b@dShm&>vh?UM$k_N#P0a;h)Ds z+z(w*DJ!DdtzIli*zoJrVzQsfQ<^MGn|uRhAF-aH!zA`8%b95VhsL_5n@Q9}C-I3T z54MK{ZSj6AwRrNt-R%a<4-ESTiri7{UV;)2Kur%krE^+dOL6?+_<}A1L|Kg?nd$hT zE(I3snJo<5vBJrz`V105$$#K7J?Dq_{c2DUkX-E5UY>T|q`4;ygiBwA=lKTY5C!dw zOYo-MbbG{*xkuRFHKc>$umefPTD0U~=x`t&4Qec^d#;3R_{dkF`tgSi?o5aid5Taq z!;NM!AfvO=ef=aGFFoCE&9&pf4={wue6msJ!|36G`i@df`LZ5pase@2L?6zrO~9Md zQ&~Ec1Tn%15MlMu`*MV%bZabee8?`s8U|n5iN+SUw{^hA306UhtL@G>bq>8L#IqK* z34`Nv{XCN^DVHFZK8>eib{mHdtevt5iKu7%K&wIH-a#iK= zi#^yu;)fctCvPZfbW6+M7|KL@y~NRx9DAcX-vfoBxF^y1@*TsY11|EfpFM=|zJC}@ zVM%138Q8kV6`^iuj7e)*&M7j+0HAT?dJ|c9N&QL#N#w}W3y<;|2aZ$;a8Pr0YCSnE zYm7R2;Mv+vgWXx)81pO#X822(1LHmb_+41Ipp0bXfjv~B*IRFGOd6A>xu`lwdi``j zlUY0p5fBlsPz-W`fsp9b3^cV?ObqDQ0D4d+6TV?{c~ZT2ZvuJtW?|HNRe#kHZG*O7 zW><^VY7OkCLSa03QR=XLKSe8DDd!#LGZ}E{0Ut+XHY?REI)%rDAulTg!M=fxdu6lh zRNbqMlfJ8-V=t+|IC6|zJImPkyDsR3$~y*uIKfy%f-)Tu!&82^s8)}}q_Q$T$xWovL)=qWOG;Q1}Q8Id>dg!V7y&WZja?HIJKhimDSYF>=0;3Qt($%?w zX%L8{o_M)T{e`gb0xAVeR(MhLX+)Xm)j>BO#$$Ab9YkQP(VpKCZL>5dr9KH67Ulu! z;@eqXSVM(a$Ri4vu1WVC=Of?ovl40s)iCi11meUG6?qbpvaXYSJsj4YCw6xi9lY|B zQU-A?z_T6)ep+|MdGccS={7oC{*%E2-ng%5nTTCw>zeA-FgH8AiPKo-W0 zQ>+;~$5zY-zbIL9SZyNCeIEH_FoTrmxq4By?XewYfyiKxrbot2SffR(JVBDIqyFaa zn)QKglQ#(42sK(HpZE{wDzs?52oWm96vghA2c`X)440h_XH=WnU2$eV;ElW7e{Obv zeZXoguLH(T}0Z_~g#lqZyq>fXFU3jp;GiD=(sj>|^k8c$Y2vMNl^>2A9gKScPvKg?Fqv1Ys zvowtUKN~Zy=V+K}|8Y)m4!~-r!JHS=q~-aeBd*jqy8O(h$)eQXfdMp4(17E)6i87C z7u%IPTJx(hWfa|uBA3SP9)a%h6GU`-7Rm$)#H8yc zJ7N3sfD81259J7-Aig3bcJZQlehdf+7j~SKtW*{eU0sF*4wW5q6XTClYpwV<<2j1C z7FdWd%xqG&Q@&}L!<_KNa8ADfPc^6|4DHhc`<8a_m$iseO!cb6`lBOee=5*S7Lojo zbdyYXPP~%Ez^bqT)&r|3#QcAF(E&;^SFeZXW@v&0D_8F@z zX|02m>0ZilrT9~#iX!KVyJ%`y4`PA!)9727)Ba*Pkj2flWQWs#vlgqRnx3F)Ru0p* zlRMXLhRg)vxOkdlNpBvM0!B*$El1D17)g7sG3h5 zYI+<<#u^4~e|1i@2);%F7soq!H@hGD&->a(9SPfM!~-wh!IQj+`p? zt>q5g3L1|B1kt~ zc6;Wge23#HIKVqY=2fwRyvUtLtI1?0Xu~BR5!p)WMm1$SMgDY&(FNITKx0E4#HEs9 zDC&`H5Z_Cw8uLI7x^Q=>N`*BI52P=4K+X~Eq2B6Pz$U@9Bl^L$O zPni0L#}OJb`qiCdLZ@Z!%QrEB-^BBex>_;8GBIGAzU9Z_#FP__mVvBbvY&PC+dK0h z_C7%*$nRvj!cU=`CF!q|Vu7>+$_+i?EqZHU+~!Unn1D`yK`wz5#*cPs7jfXWg@bjJ zk=#_A?9Ub7sF}Y#sS%C?h=!Bbmho0_67OiuDJIjrZ|8dkgTFf#FjZKuma3+^#}^e4 zD92&xFJ=MFjpK%aZIRH3=@&&Io9#2@@3cDa%a1$s53gjMTEVO4@*oe6aDe*g8X1k>K8#ej4;-?>+dz;;uPl=w~VvaX#)560ZYslC#`aG(h zA4`QgopX5*QIk7zuXpILmctZhq*)LN3weVx8Qx|$nI1YTW`SiHnpr2~ z>SG!u9}FMt_c7aBJ*q=&;pS0iAg$Va^!;2SvNKu;dhhQjb2w5;6=k^(rPwpFI$pO7 zl#Bzk-5siROpcsw-Z$e0ICe-M*A;wqG$J=S8?rl}>fE7`)iOGJjo|8h31e&&qGZQq z6h(X>P5<71N^Za?k$%!d)wOyZ7R+u*P{##|2<6fR5)|YZ6R7@{CTjQfpr8&zc<1D> ziOaIPO)i)%0LxQz_nZqtjaXZhS%w)iHZqzdiS}OpJL1OR$~G=*Pc1eqK~r`Z|Dfwg z%a~C9ujoFK1&CV8gj(62)b|V7`d_eMAFXxQO$oUFtWqYS&N%@~Q|&(VL= zHmM|XluCw3TTiP`k;Bs|2oy-oI@&)^jQ`?@Oms(DAxJz1P(fYvj3MpSrF59#Z zk31cf)`aTTrCiqe$>~RbOW+R-T3!k3^CXq*8}5eWlAP%;2Gv-O=H{km5julj_hAaVt6Kcss40U3(xZxph{k#uR)fz_=rY*a0T<%5nM<~xm{MQqX}!{EItY| zvR%64;KCKub)CH$swNEmcclcYqJ^{Ml>M6HZbO#WmQ6KgRkPWNO;59Rx}|odz?{9S z3VO4it`ju+9{UBgNFaLmUJ-Vo^_l(!h(_2_T+_Ul-MZ_e&N;WdqorJ}s)9;Nf7J%x zpxUI@$6RBziGrtVpq`yJW2xNrz*O=!t+MsbNt6C*>?l+gG-cY7Qv(Iq&uAC3&^gIS z-L{prGNv0VsCZQsMy#{Dh*W`W-*vee1n)&`xy#@-UhC!iCdix=gSaeNj_C(pXO4tK zlw^+;nX!O)|0DZ1M@0-&_gA;|DVgRDQV-d`PCwG7xgf}PqlKI}2Ju3EzuIQwT!qFm zPuD&KuLXq#^|ONTZ6~CM$mpx#LcYbT~rMe*k zqveKe@K3|cMR_G>-e5M)Nm=#$0(uVu=4z@2uMQeRK?rr$pe{&V`UbNY5po?EibO}y zCEFn$z`3R|Qvy>ONWvsq{f6cubDVfgdR{X-aXWf(o37s1UBdKe?BLr7L@bh28`c5C zk=q#mwmGgY=KNUJ%GoiC4(tzNEI|=064M!IStbuh=CLs$$E(dZ3a&FFfqHNwD6t~D z7cg{Lp*|av>4}t5t{g7h;*S0XgyXuMxdl8>8%6B_Gp#pQZVCvhrGZD%u~`Alvx6oC zPU|cN*NIVZEVK>TIu<8{D+?YZ{e*yNM=lX2Ki6@SjHEPkI4T^7EG?nDx&R1JgFy9z zI&$}@N6I(?mqPpGsr7q7;h1$vv|%}a`V-jR0(#8wy=5lj4p>TTk%WRQOD5(aODfv- zGZ6h0$NpXUGnEeo$ZXoQ4vE}~wNZbj4Q4v>;7S*-+$o%0ZM}ejqT7dQ z)g_jYN{_srg#c*?G=J1yj}!gZTj4)pWq^B<&x^nJ+-DCZE!D)lc&8zKA1l zu7E=u$4#5ns^JWKrbDGx;H3$v#WO3#wERYJ%R5>W)&uo`!`o7>*90#PLrUXMD_Mly z#-AE6qf!pps?nC&rNN7;WDr%SaOgXN#uV#RWf1aSMvCDz(@}6{SCoTnSd}Uay|omJ z7^Ues$HWXW&P1>+Q$PmrQzSXINZYt!rkPQb;eWwpO57rF8%<@DV@c>JOAvLQXn4%) zY%ABYTEJBlKfd1}k_m;#J=N zlqVtvf`T$(PRUIC=_>_*kJ4ndQIV1T4lB!wLLtDpOOPmebXw1bxmSq^oE`&1m{ejI z0usYP%zf}pO^0eg>p)9**U$k^lpfYgo<7i#-&NUtt1?eRStzORnk5IN#<*0{4-?a@ zzb`-SRe=w;|5rxqPF^anx7~Vd`R53V(`FzZ$pps0MZF2(MG2Ky!y7381;{NzqovdO ze%zuhUBx{R5Ce5!OKxp&^+^<;>yfF8-yV1Wj1Lv{vsP*8FBB2dYQ+iPLHq#Qk+@L+12WL zG{&9wc#~A)5~oK%8#SbT_|BaUB<8`7c5=ehBkc=9Gn)bZgN~4Te0T_*LeyX)FS!Xb zRVL|4>SS%LK)0t?I?zxT6f||<7Swmi4_gMa871s4;MiIkJu_4*sitLgw7xTEE8GkB6&0(p2DtAYEV+jTTkDksv|UeXi3tYnTn87NPvf8l z&qL&>+=_jM%1OjQrH*7`&9Vdn^k)+eLR4b#wV#rf2syrD$P$~q`;Y^sj;KBFj4mxM zq!}>jK16@MQ3j~loydh%oZGCH%@0KUsYj}jIp# z&cT|x`a^bu5a02vAh_&`0|l5>kUpzuzZBR-7{g7(l?L9zAY7=swY0tYS==iNlBLzH z8M(5mGnSd+$oe6}LlZLnHEKHrNN67K%qxfnc-&>L*cI51pLKi9~t^ChnCK=50J{LG!8u6nQNHD1v@V=EO z-S~*MT&WE`@{ZAk#zTBRl)q+q(sBgJ&YwS9co8pjQ*=V+xM%$JYhd-cvRXFp4l948 zN{JL4wQ`={xtLPR0{*sa-F_S>PxM38&^Sc$A3TGRx=uQu#;O*9Xwngi+k>FY@Iiju z$8eWEpX1=t;>d(Tq;)2a+S9L}+im1o*;bF5oQgJi)cX?i>R84tst#k1GED&1%Xu^J z+o>ZEE1JU&zy`r9JT;c?w08Nhq+!(feMu~X;}8-)xrfnqbNwBm$)_-hOi~0+n;J*E zlI%0#w1JEYz^2j9bexgHUp@gN_*ovShFN*wCdDITCjFJzF!|wsC~C2q#jRWCu9#p%-wp&d5KaJy-)bSS3(r! zX0&*1SPz6f8K=BQz^pA~?tSa_CzCpFW?|n1k3ZxQ9Brxw%7>BYcm-JwIg;MMHjLWw z!FfgE8}*eI_n)^LK+b_#C}z(opA0isP%b@1q6Vs-aksu;IsR{ETKDdZQM3#fvzuO&@9Z1`eQ$I1J`n; z043)Wq8+}bI|<}%2u{F%pem}o;GvypSKYO<foqjSJzr?97h3s} z`urQuQNJzt2aaOQ)%nw!*#SJyCz)wd$a2mDpO<~22T9~Qb?1y#Cc<+j z;2*mwD6Y1id&7rby6s}U`_h2m5iH;BCQK>`o#U{nvM%^-wJ#NOFIcKP9&^>F>gXNo zpzD$!uDBGPjLRu3k@aek#_bra-jk*RNBF-TG0no}O+NLgd-}4f?zOO+8tcEm;l1Yh zTI<|Ye_y(oz(Ro6O60|xhCp8>!OEoHj#T|zG!_dQ9+;bP-lPg!;bs)N`#MG-ybx}> z09U7dlo)_8JC+BrKCjl9zA2XpYTuC?leZ)$9=6#?I1qr&zs{u>+4kizha;=hoo(GO z_@^a-m&5;4D`UTnh7JbQrjXMP2prU}|50i9ukkSftE&EqQs$aH6k&`%Ds#1k>mx8b z_#W{CKpSJi#!@b<)bq6YsbIYYR=m}d9mSE234o1_o&S$3Yx?K0wf9TxUx(zgalK8; zcS}{2skWW%FZ&lgs{(Ta{we#XvlicCnv8X_LKA&Z*3v?kwYX=^S@kgjt-94^9rfI> z{u;Tvj-s<^W94bVqo~zI-SJ~3Ef0pM&A5|3UCP*{x!j;~ivbvdWA0MQ=5FkkY|?9hmxl`2g!zf?zuf2SgM-B+f4xW--x@>`YW{qczNliLlc=VnPmrm zHsY)4zMpxWOSi4Mc%`|!r1jM0S-4O;JpVto&Z#jIri-$%ZQHi3jytw(+qRRA?I*Tv z+jcs(H9hal<@X!vqRy&1du^mQoOV!Rgnnt9s7fE>j)9DSmiz7OPAftvdf%J{a=s|E zBv|86O*>81=lIqvWq5LPIfNeJ7k&a#IZ+rbwQR$2Dbv^;0lBpC8Nx*z#8!_Llg{EwQc9wjp-vrv{1ND&#xXUnp~@ zvy(-m<-mZ!=ztOujD&Y0HA0~{`wO}dz@?@{b;Y}?xsDr5%&1ddu?(fT1-g>3R=U zW(^okS9Q`9zJt7lNssOFo*GUSkG!6|9J%H|QA>5asPiE6nfPjoI7C20gff%Ly+ zH0(q;x=3regTWm>JUR^Ekf?_`%J;K;_b$OL8--NYtJR@^Ls+I{U=?p? zsEvofx?Ln*Vl$)?iU6ud80vF9)@W}nJ}!P;|L3RDX-N5E%s zFbp$w4q0TR%l61{943(q^S<0f^g=bSgT;n3ZxBHTL&w{6BU&gn@6o4u;tEtRj=(zY z#%!rVOS7e0-ZyDvghA0e;2kqZ=WV+TTit7hi087W-QMbe4q6`}FKG6cpdU;G(*~%n z{#7tiQj$USKu`9VsXZKpf>&aP-w@x{Xe>?Xgr1too2~%b5`OW0lkOzj>`7W!fM4fi*XyxkLrRZc@#EWn53aK+ zUdHZPkBcVoHIK`La7V1bR3=(79ca9kDT#Cu5xv0^B!P_#!0J33+3x+ypf7B&Cf7p+ zE*F}p51KgXVOu()rcq)7w`De&;q7C)bl!~BpI3yb{$4$M5qTMDvc@5m2}L&7FM6by zk?-35BD3vyZaH7%PRq;5&Ios6kp3LO)GdD?8|v9t+4DW^?9e2_W<*)99`QQ`f%I&y z3J!ZF8{{YJ8dxh!qzNWkC>KoTP|~d7%NH3qM0}mIS(bWw^B7LvX|vJ0Bcp+ z=P7QWV5d?EYjEtlC78#%@QZEbHWNVcoaLIVmImcYI>7VyAhn9EwV6RJ;dH7~+ViYA zx&wIPt>mlG>Ox)Nhch()67P~r@*Bb8vAn~pmLbZd%tmwQd16V@^er5?9 zZaIfo4Txq+1&#nDC-UbO&TvgOLlpSRd8f&mX}N z|Jd-#1Z;s&B`}&hmYdv#oj0V4Yr#OtVZVj|iD8JC6w1Pac5fR zwEei5gt`{I|4v46Ys5637jUqb@viE8*0WwwSA1`mW_|uPj*n`GQ&J4& za^Nc)V;NjZHCm|)-1)I-yZ*+Qyxp&q)>~fw{VbuDI8)-jm65z-qLbqQyhiO7rN{u1 zR9`>B2zA?Y;-mHy!HP%ak~I1>#FHedH zg0$7WGKK5v^9a>Ik+q)}l8~22wyYaoergkwAxMFDMRkx1N4v2Me1XiY6wVM{)FMg@ zwZ}~wk9Eo_$`WUH`0osSev4grti(LZ1{w#36<*cQR{q*+fF^K0&YWtmi8^OzOp6|p zX^=A9>q7=6aF^-u`k8C?T12p2Rgt7Lw|*j(Kb=MNc(9sy&MO&RQoJ?+%Iiul`EbgB zZl*jKz@zf%ex(W=`s~MJ-Yv;vWu!euGKsDV4$h|>RWfuBdnScR)#;}*En_8Pkn~zt z{C3vY1GudhuQuL=MEaM~8mT&&z8R%Lg1xBfqN?uV$Ai`6`u{H8c-p!TCGbV<{nDfq zrqp8rn+HiSSq8?L@`ZB(tqnfFqsr|tZ=5@bdr-+9fwv_r^TjiknXe@Ubt`kmCf$VC zmXk;4LD4}0uwF1Y9$sPWinK8?NDnc{>UQ5-z=-z@;pEoFG~Ln~f8CXdgVo-1n-U!G ztGoK=^wIJ!iiZ_NSkw@*`-849U%UEFT%>!+2E13Ll7NVFUx9PCe=H$uO_I(}$hX%_ zCu|07y!MMvABGFuyKKSM#nZdyV$lpvyq27{7mAs0Z6!);&Y=(gSdEL!&Gh!lR)I*23|FR_`=W23SPIb9F<;&j%6wm zdy@{ku*d?xIFhdrTfEvB&B>ao6GEIjDD*Qv+e!}h)~Kx{eIextsz!C%c06Lvwy%!; zX4IvR9nt0mfwN4QKZTz4CqiC5!-P>E$pGpV!?FmxZ(PLa>KgTrZmd45371#U?A8SMQ{|}572k|>_wKiQ)*s$u^ z6=QV{V|{+}@A=hN)kdodr&|Y)Q_e&5s&D4hVbCfK^+PY+0Z1J?^@G&m;LTS7=bTxX zS95-~I*WCcN}J=&Qbxrra=P7jQ8b_uN??hk(_T4KQ)Zp zLvb8M=-san6xVgDEjCi02ZfbNn(J_msy)Yc9-%vpLOyH8z1mcK*V$6{uZJ4^OyF)h zBPb*m^}xY*r{hL5_@qzNl{z57Ue~y*Czu}PasGCl>Q{`s=~m=qf9Uwpcj#iy@2qNl zx9d9$HAZ3e?;sr*8TDzi!|KLykTR%WPd;@wyh{p4CJ(GUprs1gV1KH;bY6%}(5Khn z?G2v1Znwc^x{N;}t4T&eA2vqNwHuNBUooq^own<2IUHo0fGqYQ;j+@&5+PLLn1#-L zLT@}N9h>j^G~&O;3|C{=4~~C}$&}2G?&EjSw|vZ;`m>uQbi=bkJs-ol$OHh!Mig3< z9^7Bt|E4my?(uztZXj!%^J1U|TwD}Bew9HT+H8M5;@Qw-`Edoh?lBENtk^2(A=5?n z`w;N8P9XOR*VWI%Zv?{CjarZy32PslFdvPBMWT`*&F*X6c=hRm4h?_shUVZT#)*j` z1Pb2y1W_v&X>&jo9|)jUo}G=3A-4H&s)??vmHZm{DbcuV;bXVb6Mm?W|r*9L_iqluUd5Hk+(XM+SnRm5M6);;c6^sOZo z#ZnMXY1&lEj|LLjq{0EVPC2d6MQk?*C6LgO7-We9T+|lV7`K9BNG9thDLB#^O&VopejNgUmT|7OOz35huxn!1ZDnL!~n zDTYNvgnWas*Fa>j==LH@I!P#ipY`OhgpJ^QpPcd>8#WAqY-6Ac`JjCbM#QySdp6r~ zDxft2nxPhScP~E`u6^L%yuqT8#o%;-pK0Rd`@sWArAXEW$@jyg66EeT4cMsfNe$^4 zYRR@jukHZQ|QidBeHf;@t|mrQuexApo+0MhUdYGE%Z-0UUFa^!WKH(3FN0bFw&j0jT;4x+COy)K$pwT3^zyhdt|S98X|L z_j+)pPafDtjGPj%#i3|39ZJ`t6Qi1K+~t(iTrK^go03D@ga@Q zSopF4(sdL$V*98J%ItVgFqFaOlXL)AsRBJr3Z2l3r6Ix=7$^k6w(+gnVA#MZzQ?38cTh)X1#h5Wt%UeYDUzOm5COWC8J z!k+9G>A(ZDpaR6Gyhu-kTf}L1sopf!c8f4$h6D=AheIjus2WE9oKIM|!)uEtLEC&2C zFB#nUhs14b2hjfg8=5YNK=IN_?#Tt$3ez$koOriv9=7>3R1(RWivsVw`-)on$nGoX zJz%_b65=epa2V|=r;hoDqVf`ZuV z${_lWVl1vhBIJ`8C|M5bW;lp&^|T^k)?;iOJA>GrC9P---hhbfU@22dcZ;YJ3$;BPl3+DMo)#zs$1>ZyxY*xC@Ih#?XME>#mbHiv@Su{qLj19;;j$*(EgYE}8*G zj%7@-=!d=;fI|H7K6W&%xZsQVwGYaK(a-_vynbGQvz6fxp z6R7zO3K=DpRR9t9WhL!`&C4Mv zDaX}oGBKzcTKW}=*>`4kb8V0m)@2{rmZ>Nxcufn*y2VF`IXMf-ObJlPYttZ-6<#Q? z#~KK)pMv^^9@AMvJjNLk)RWXWwajyKE(GulJIX+M3MOM^)*9sW21nwtU=y_*kBw)< zCnw`Fpab4H$chg^1^v{umX6&*=OJ-u+H(WLf8NRe#EDfU#f`ub>t5VMd;xdhk--)w z`G7vp%+U({=`t|U@W8cBLU-TVu{RS@N4pT((T6}B}xWJ;( zh884lWq4uhHCMRp#1oUP&*fNfXRq^wmMr&ve~0s|uUih6mn{s3#x}A|@f^z=ZGN%q zFQpSO{+aU5^CZ=O>IQbhCzh#e-L2B!X&4y#^5Bf|xn zgD6@%WxY+OMBEGDf+l-qxqZAyEq%62mSU${fXjjaujg*0W}JeRRe{8o+scr6osm&8 zj0Tjn7;VH|01^*zi3Opo6?DDC zM;fYq^GM)*U%e7#OH6|VJk;SgIZFvMSJVC;suD37s9km%rOnLfvmc^Z;%N668_7EV zd%1?XGQGNZg8=uTld=3h>G&x2Cte*4E+)p{Z8r=MNM4vLko!A%Ef&?ZCZ2z&(;GV| z_qSQDRLnekNQ{PME{9_Xm~;&z^aKRHZk(bxntpPZy;2t6=Q5Hqt<`8^R*Pl)mamEA zSsEgGoRtBW!nkl#ybG(a=(uGXk4}$R<>3Yj>f4`g!H|OefCnjN&;vRWm>N5f#3D$ZzoU*l5)smzGl3ze;cmc0^^xLlu+LL0fHL63 zdE%R08;deM^pE&aOV*~)%&pGEuc1)^1VCG}y>*eS-kad=CvZ`r!fz--t}^TfdyBUh z81$BuMIk@Hooxb&5&z$qRP~P&IMHLnJ2(F&wKx7pup(4TJ9TpmUR7S+Z{zU#&e$fN z(0IePmESEMBf-Z6bsZ>+Mk16OKx5SLSA6$V7jhgabTAMymm1^(c0Hov}omKvP zUNPSM^&h6d>ntD$FQj)D*HqQ*1#IdvyA8J?4$CRWfBbsfT1xSCtkQn}q)53^fSaa$ zZ_5b-*wjSd(l>oRFU1XN>A~99`@QPPiS|7Z<3$TxV$kjbvIJ6=bJHxZqyDuP#%kA$ zO8a7k9cZZOPvFz5bLe*Bp36~n&};LsxpN`zbTORwSYK3>Z~oVu*O{sYm=LfjD{b%m zJ(~!>dEL436edtN-6QY%6dJ=7mdelU5}x$T%U2R^(<+5fW8GC;I~FXu>FOV5r2PCy zP_00~`br;Sf755RD`|7?L((F@7@Tx8sUSs*<+rG)(ztbm7 z&!f@5RdGEJbJpysqpQOX|c8s!js$ym>dr zpB)+BH{%bK6sdMWa)95OwYKbX>X9Zd{G8nUaj)$G9F|ZA0zigw6jw-R!eLc)0qsg) zT}>rEi3~Pc^kMyll(ZoiV;24rI)qhiy6nyb)-p=Yo;C173U&D|4gyp%#|a#UhiUf; zMR9(;_7Yb=RVl|E20=;5H((4BI96{0cF=^lT}rABIcnAc#0$kpll!D1rjnL=auJiG zh;alk{)>2gm>&3l>kV<=HRRpK`;0eN2B=Jvt4R&s$IyI}{uD)*W28KV>Ahy;vRhn4 z1*}0K&6^{WAulv(u05HAcs?TsgRGQ>>L&Ws2FPVftE5L!#^W%C{Hkt}GmXM+ZYw=aEvR7X&w0i5xfFfCT!Mw4#$ zd_r(8=A|+nyDj)z}l1zE1c|a z%Pa8TV8G&oW+JpHdqvFYGSS=@=*A0Fe!G}wSwi{(gaQZNe!KQbfzB5=wvg)(D4x>Nr*K- zRFV>!P^D`?^qz#dQ>8=HVA4}%#pAriB`jg;*{0fu%FyY5!b0s=?uu`Uo_~2tx~d>> zPOoW@k)$VU{Q=D}E&eH%l?v{zHzpkO&^l2Hx)&X;P4zc0O2VM}Go*P+#>j8deW3*r zMrt#mECloCSey3g#sp!>oU;qHsWp0NXPW_0B6F*hcyX0oj^)?DrErh?`E)% zN9wp>N~MT-y?!T}qB-Yk5_;F9h_mqZQJBC&bJ;j)@4)*MyH}xvU5HacP_~!n=e{wI zP$GEbdOwW8F}qE@%4!(9yL?AUmqDtq)I`aHJw;78TY`le9!=qn7UmC;O;}qIlCK}U z`gU22+`eYBUKdEeHm z7&ov1@6EB_+!rjk>AsxiFxn5#UncA>L)d$8Z}fHLq&Eseh&u(d*hO+7dqfEl-OL5v zutQG4=wCC;3!kB2d5vPT4R;cS{q!?c`s!)-!(9=By+i3y@!Bxhg2ZlZ>Z&8`ivssD9fb7FWzt z^7shSjOCO|&eWio$h}L@!q+ci%P$X}ZmY#!rTX?er`ySLouHeOqdx^@$*FkRX8QXW z`O6d5Gl#?kOTwzd%qZHKF?|fkZCps*Pyn{nd1>&E^&?1>hLhS~pnTPnqV=@K?hbLdK(> z+>61fgaHXVHp*wb$y{f(nJPPRYrW`!jC%_-CyEO;m${X$p|RGR@s=+7cAs`&^H8j) z0R(P(`AFe^5zv;To~NHxI#a+6!}x>3`zQS6vXD3xW30N~*3aO22_)oP-ZxJP|lwwuVtV$fb_93xH+rK=<4}TST zKnXlEOLy07gfNDP>bAzfAIN4&yZOP>o|n>S6C~e-C|;4)qTwl1e^{1VvPU0IIK@@n z@mJL0=?kBrFrG4p03DaxW4898eRHH766m!tM(O-HQX665UP`X`ol}z14@uk3`OCGR zoXjHCoOz`J_rqzoAG7j| zX>;O}@@m8dOu+r-ascVM$x;j-edsQGo$fCh;+`*nJE;che#pP5JsrR_>)UtuP_-$Y zj$$srtX?wS$u|+Bd+*Aik8$1s!jo~brW(bRI(9IfM0(uuA)?)t{^y9RXY@3V*{gs0 z!&uL+B^*t9b$tsBWpwwL@2DdE3oLN#v^nMbxrGAscZ#4^CVGnA2(+>mxRMMJJmPT| zlJc=vOQtR{FHG;43_pTbVm>bOK*3{O=57!$BcU1#Vg`#@PW;DY;;S6B)7>#`^h@M3 z3Q;iT$!Z3NgCUVKFQYM9=_Mw%eJOWa*u87165qPxCP?mFM`!;Z6+Cj;S#!S!K5aO2 z5FuMk!VHP1?3u^GjsfZ<+#Z}6#_6=);m9+uR2&Q?u>7p!n?S4tgVPeK7;#CP9<35@ zTK`t=v=;p6C0XFxoH^O2p&)wH+l&}yvjhD;xJ9|a6bXiQg`Q?&drJI9FdYl95=vd6 zKcC6af*FDJ5po=kZtP{}f-FR4oTuyLj`w{xUAVIuD1~XP%WQVMfc~*tT4vs_Ps&~k zmyFvPFVLT^EImu=4=QRlS{snMW=fj-?>8Fx7^Bbq)!^TZHrm|iMU8m1*{|_=s!DHL zr!s<%_jBe3(ze&JYktL!?5@t}M0Mq_wOPW|AX^vwbs?PBvDLfvOVv@ zuAU)Ugq!X_VxNGo!oM*Jq$}7klr*E6zsui7BKb*FpZ0Q5rjezEeQ@yOE9Z1_tND+~dRA4kqW7kMm-m|HAAN`mfuy z_;x_=Amp-&Y?g}7%hh>>MOcqFjQ)f>r<{88a7%@Lzxm5>!iuvt{IJ{)ZR~JGUTxXI z@7GcbH2_X@=eajc#l^RIA-ZK5@mKmH*SvDw%!?Jgz1wn!WN(Wo&aYR^S!0|@kW0_l zvh%&4j>Im0j6q7By^fnvfWqK$0WCkdNMgq_vk23-Zj5rphNUy($nvuP2M4+bQo~FE zF>fn+x4Me$ZuMCp4p4W7wiqx0f3Q8JZV8=35d=XKscOJ=EeHWiA2iRCyrr6cAFJ`a zi?SBta>@*t>~q?U=C-UQ)!ej?#3s_t}$KXa*viZQoYSP2Qj}noAfNj%Se2U4b-ASt$mhO~n%q zM)fRJ>e5fxUd8)YC;J*3Ds*dUopb)?i8(N8R$pXafhf%pDlV%zxRBf-$xJ;>=C-=4kGuM zs_!-X0&vC)euelN7d%rZVv<15A*q%dYnIiCyFbu8QS$rO-tY+rF_Vx~8Ly#Lys{7a zas&kiKpXVHlo_Q~Z+sSn6j2}MgnwX^ofv4O6({=|DW6ToEQHA%ZwN`>>YUb%j8*;o zTmrC%PZ4<+0K7|elY=@5-eUUA5k@dK-8Xjw<=YowOqw0!(Ja#N(w5%YS*pEn9$)~^ zM5<+B@DSr6E**I=TUF;7bG`a$2->Ab;uB=;6JudK0kaYOEHke{ zxrBg|vq*$$sEmxXiKI{OXRNT0J>c8o&nHyRUIotHQ`&0;L4TixUF$2uWp9j9$t6mh zyVOjKz7tmoOt;ms@VB$DBCFNBiL6Nf@g9lw#krIk|2%n~WE)p$G#=yj>x)X3Gs6Ok zID`?E!#W_GAa&LwoEeX*oP;V}2g+Br&X5YYi>NJ{aIp}<#Z?lR zk{|RYge~a_R&Fk>Eb&v3_lP*EKKRCYA8<{s<|_!}z_k>2K$Z}=+2iL5hZLkx+f#Sv z`4k*mWk2TuY)({v$f2g?#DHwqKETyIS6Fqy@DTe4^%t>P&VHuA)RK0-TbyeJEM1t$ z$J|WXTSgGoGkV1{LILPz1OmG>=*wyO13(@iG<*Zfek-ituK_F`zvxU>$FH@697*>s zu0DGlEGtgjtDShCmO{^30vYvL;9B_zuS%T2zWq9FC=%0M(`dClm^%z19v*T)Z@}6w z2+w}RNj;`l9@gII?1!N`oJ3k>)zJ@z2~x-t4lZ*YNqI!p53 z2JQ7W4L!`9r$|MzdO|k(`a$%?7!je^f3?WIqpW^)r9rZDUK^o)RyFd)7qo$KJI_S_ z2Od=PF(|gB^oY{3r(AE)NK_2GeNz5Cu`){;rlD$tc!HfyWFYjaF}*K3C`Dh@W@ZMy1?9X&y<)TKx`Y3E0 zYTn!V{n>vvO1y4+P*fg-c$VihEN&9^_DYf%AZ}y~-=pSl{kMN zi;Aa^UO9)BGndS#d_WtD1PM(FLW={JG3`!UF z!RJ}}Tqcr4xqINUV6}uJEX&f~sje$&#)&&vztDFwE+X}3%&gi%mVU#?LWRby;Xu!mwo zEab&G=%j&oe_0Vv0eNYeR04S3XkAuBU(cVO>B7EbbV4y+5!VH7FEWomvom+w{X%b2x6_Hn8i@9!YGV4orOc$LSnwHCT8!UtOTY#=oH z?~8)G3%QNjsDzgbK_tvsVS5qjy%=7Nbs9TIQyy*D2fo>BCEoRBC|cn5zw)eldBPJrW! z&-tu?8(=LZLwl4maLfrz!x&GN723PrLY72@kajAU{Mun;4!bGIo4KCC+sUk{q|$wc zs|yn&)j0?2g7q(u$6&7Ah-6gy7cwi|2*^5mdV{PVRv<>sYb~OajWhl+vId;{O9l>k z!NCxDYMU(dp^l1+CdN`Z8uuRWa|22sXnrW@A>PBu+#4Fy@IrXEY}}?T_Ji_tO(%!u zDm}1V_F!iv;VW88E=iUqwVBvcYL0D;T7TI?a{Rlh+!dO5ZEG?0dk2&~!rSSJF3lO< z(XidqLr6LxuaPaf!e9wLsC<%aRVs^-Ab;-emTpVmHO>8Dv9obVpGX1BOS^&BnT z6$Mw8TY^-mkcBJo)X?uY1G#RBm|dxQRH_>cJCu!Mx{c4TIxJ^PP`ZTa!(G^Ui7VL< z1*Padx9Aa~VJlf1zR0;P&3Uxci%C*B2UMC3w!=>(SV9&UU2JUK1YK?3FgjRxS1WVO zMAW(20S$O<#euBX*CHPX``)>>Y2IFnP^W3x!R~UKBKhCNy{f=^j8X+-x`r^Engh?` z!a>Qd>fdSDek!t(n=+ClhIKbO)JLt`h6e8{S)9~g9)?efDWxwUUsjIcQ^<^m!E2A! z5enSD+Bvt>@Ev14_v2^|z|T^*yVeoDY*|a0n1gb5%TGa})s%aeKl;Rh@jo`B9L1p% zIFqL6Y9;9*;R8xi<08VcG@b_51{)bSuv_jD2BX zrzn<*wjBJ+2a3?G%17uNTUS z;~B*+V)(;TkaXU>RU%;8?&up?uUhe3;@UpoT{Is2rBvJO0yP5s_Gf9CBxdH`@EdaA ztYtXHMiJ4{qfi(>kF`3`@b@hFNL(nChZQb=+#9~-6Cxn9P>a9&8YLN`)9X#An7!sA zGd_xvq#JTb+>e)agyCNx9gWpjl6B(Vbn z!SSHI!irhh>0BsG@s>XR+TVm-;dr>Q96M2_ZA(EazGi84@U zDHP}1A65-LRg3yRNQ=k?M&DpXUkM8fK)^#U{U4C#`pSGw%Z-q~J%iMH3DvxjbjKr;;N~~6EbeF$hc6N3If8JJ^zc^_t z8ek{uEw7_ZyQ#B`G{T>;yRp5!mh44U7yk)qe#e`L;c2R=^4f*8{EE}omW!#oz)I%-d#(h1UzOs~azcnOJG{?2mx^L&?`JD={k zNw+|pTgPsZ+yv7z@_iHSw6g=YHe}XCQ&D;)T7RsWja>6_y=_coZ;1%8!1Y8Lav_qP{s6X=xMm9bwVPfor-A!LW?zbWAm9YUhjgB7#vF$w~h zB|0@uF1}UQTQ}KvEFmo_oSgrFRg7@gwo|Aq-g!G!aFFtHO@6XWtHpgSh?ij+2gG49 z1)mO`Bf1V?m03XZv%1hvDsNDxhnRnx>rURjFx$eQ&bSX7B&Cc{01Z^ja}uLRx3OzE zq3Hr2=eJ)?nJ=|p)B;q#54ouW11jw34xR>@Z}ha;fV*-P)ia1J%3=G+1gtp=R(|&G zD*D~tU>WVTh`6l*@A?rmqJw*_gw+4iil&Ra{Q>g)4-e)r3mD7u#{>hLr*fmvpOxy* zH=H4CbgJXRwOaApL00tG^O5=FwZS3|XB;%ZNL~X4G@Blmk!LT0-lgX}N|iOiW|YVb zZ5_ZMDVQ2vakEf%TKWxHLFqE6Rm6q`3SHNHzj24 z1AqMmdmFgpQg0QSf+zH0J8e>380;*>`)k1z5fXft4s5jk=jChnG4sS~9qM^Rd4LZY z(=F$=2};%bH)w~W1}9n@+fn)~LeXk8MRCDC!e;I85(X4uC>r}*a_FVl`&}ecT@W`* zTLM^;jLF=$!hl-cXdg&?ki%^xWEx$D2;=@y9A=}(13k8wu zk9?F&Mm6j#na#q>F${#-KqRiTGm_hmfnXvL`{2x`<6L*(A6szNKQ(V5jtN8{I+F6t z3aNfK_UQH8lzWpL|JF>x<4MnBhbh2EPQ|Q{mO9vRDSv@^39G;;u@uVw zIxXLG*MCsK-0qTT^!Xm$n>P^@W)h}{do!R5AZ9W!b7?#XQl4L-!`NkaPNaX`2d`L-AzGatVPc-L;Ou`+%%5v6bT2_Wr=# zUF7dpxTPWEf9o%X*1Qc95ivChyQa*zPTneN)v<1r?jQt(n^oM1iKEgu{553yMMAeLgto3Hrz*tji+kg*}9Q zFpQndJ6~@T45?{lAgDR6ZZ&`mBDg01{d{1mH+_))sq%mt>Ui@~;0fdygEOCiyevpc;^JP-fD}K<=+DgG3_;$c(hY1ug=% z>^Z_Y>7%v7|B{?2(dr9ReweGuR7Cb6KJcgX3ujMUv$6%N_Z>ZsOHx}(k@7%Wu0L$Y z!>=8xK2UgpUWW)gbN$jbvmIoFy(lQF9-tdt!(>VTezCoy3U(&cWI> z@jeuqN%}#rW0n5^W1Zv&+(xBGiVTh1j{ zNCWoD@AkquC7yfVv7tiVxl?{>|HO%K*Y5Gav?~c75Y%o3Ia+!Rk^Ec`VfR;JR7^({mewN+Bwee27-a>*J%N?h#4S);9ErR*j{%*{KVMoaA*AkX3YoEIUfj82lIH9=*$)2iNgs^$%LZJ7%x0KX zkHfwa!O*x$ZvjpSLlFPkBH}xRGbOicm}y;D4uo!xHu8V!PTzr&UCj6a75slxeGNLm zVq2%M0R2fSr`wYnlwucwiDo(;JlyAh6Y)3NO)umsm+F3NOe)62}GHMVdbh)lTK)q?BkkD#dhq=M;{2ovCf; zo$8QlnB=O!o6(@Hwk@Z*qI?h{hsb1;joqYop|1dt?58L=EmK~r(4Y9w3IQ$hOmra; z*P7br6x62LsCs$*tQvZrHBr&Ilm?N-A;+ykwPgtiR+(om^M^8y<6=2IJ~M$BAa2+6 z9XWFQ*o~UfSX?Pl{29!vX&5-ZT6gMppnggaFo8Kr%Id-cP8JjynOKN2L``vCTG_w9 z7lFl#w#3CuTP@Tp>nz%4kIuW4Bcr7NhG7A*9zBFpSY6BM4is{f`ns&DkQcBgx>y;&?fO z@42#c4V=dxG1KYCGu7q^+wy-VXOp7G`%3qWWXH8E7tBT6{Bn&eo$%Pi^99WrOHiWg zzddGOVqXcY7i5}Z1^4XJZ~MO$`p#RCr-zB#T2hxNKNI5?kO z0MgUkHvY-stJLt6{p%};w-s+)%R2F7l6!Y1c{s642)w9~w`11EJ9F#t!DdaRV^(v} zo{!)d80beIwH@kitgTb?*dULm){4>3(no1<1z1vt^j&R4c#)}j_dYON zpgL8CYE0R7-aS=Zl#*{I48Hk=$uMkz#P{73(7eRHp*v|md?l15=@gba=9U7^n;LTm zh*Sx;tV{_cX)0lN%UW?mx7s9j5qn|-#9t`+?xL-|r(yBl^SCciF?w!v;C|cp%*Y9z zfhD(;wbjoRQ)^Kt|4FoF@KL?%`-~MK`ekZV? zNjB)nl=v#d@6E)K!q22CIf#wg8Ui-b=l6_C^3VW}H1QsOWBfsmDur_(wFcyfN?TJ^ zt~6HujQ@rUxa%@xs0FT3>Al**pv3Cir`4huxqca678JonKJ4s%Lxh_lH@b(lDobd? z5!~3kngj;5Ks$0Z#&ol1ug7cip?r}`Wk<`tkz(X-++u?S9r;v-y^1rb#_84%@GC#u zzSE3}^D%3_aW*84>=Nem%T<{M-C5E1!{b+g99W=3!VkYLwj1Z0Ls}PaO|V-1CY&RA zGY_HskAte}fYBSc(6>hNrh)O$PyQzdX}fKW!Cz%(f4-7y-cMD)7*6K9#CfidRW24cAD{6NV=Qce5wjBR= z{kEHLAp0o8pz>xHVER~Gu|qrEo!XTy0<-hA!F6(>hM_drJtbps<4)n9H?9*`@85|ZA|pFI*d1gq}p{k4+O{Ahpq3Wa0XaKR1t|xkAqN z1R3vTpG?LONaI5sAIbt4Wi&}xBrK(a|jSDSh8r@)n%*8wr$(CZQHhO+qP}n zwmm(6wtwCt_iq&uc{9#=hw`Yq?!YsC8685Heel!?&e>q1y+a4)^M7L8ir|&GW|DGu zEJf!b&~X^L{k?iL&eIp@B-KXvs`=Q2-j~B=Hsi7@1Cmk!eb965o{i+LfH1EIh$Ney*u>bL1g#*as zfoy$QB=2m(hgWYgX6^7#2pFa#Sd0RF8T0(~YPEcV=?Ff^=+Yo;zt~e0rrQLA;X2x7 z`+24tUf+>Y2HmhQzs5BgNFB68!#D*} zAHgP)ng7O^s&)HicQ#@NC{-e(HTpVYB8tTj8s?>~S5}_K3pxDOzY@uMKrE-iB1Bf} zFYW6i8p?@(Gp``GViN!uLf}ovopjMmd>7M+jJzLO;#soBFX7dXQ6La(OXHyy$@3c% zJ)LT{a|nQZ(=o(t972~o^f$ByWQ;5d8Ho!fgo2&VIrihbpXoq_pRp7t(B2<&>mKur zU5iNAX!~Q|Kv*}r(@Czt{gX~3- z1pthZ91@FbN=AB?0g1o2Uc>b}Hr*142BJoqcyU)D5e}YtS+B@p_p>S-yLFV)Ohq6? zKGF{fGE#qN0{uu3Ec)9y$-<2qc30LDQHV;0oXf^o4zZ_}qd@l`;R91$vMUB;b@N-b9+B60%=tC1W#VpeOC@&PC|B$kK}Q5^e5!+x=I$lOwJ z@>M&4;6p9@)zgEb3-)re3zaMPU_h){7fu=3n%8)-7^)Mcs6z+BR+!}h|FPPq&1W^2 zg9O2ITMd+0@j(^dy4u%!NRE^})aPx$Dq~`~Z#oy25I*06QEb0~SzOsu1$>OTK6Jx- z%b81O!`>4FJV(;E_fEhu=w=f;F^fT+j}TEOwKP_KUSS88lfd_Oo$?+(yH&bb++Cbu zBG?a8>TFuRd=ew0{(gbb2$gayGpr@ygXX*=R1D?|!EGB*%~q=6seSqsgUjqMjwk#F<6ut&xI*+pU zdP@Yx{EdMFnQvE-#sQscG#I7 zVRS}+x9r~qOf~qjl|E~CT7pTyY&gNK#kN;l2oG->*op3OJ%0NQjX zz+COD$j6MzlG+Fipp)9UGF|suNjpWk(7VMw&^}^_lBF*gf$q~^;)X}GCrqjV?Yp$w z_@mm=ktU7kr0aZ`{1P5iIoE=L9(~)tVW7zo;+|*MNs6HR$Dd+o4XRrov=ksk#eVB` zyJ--p%6Z>>k#Iu^e7CGnb}h7L_+W2nYIufMd|saLABjeY=Q7P;7Hr5um!w-{IP96< zYwXj0_Zr^z=r+na;28!jY(7r+*j25lcpcU^;}0kW3SG@csPwGBWGcx~v=g;2xx`u~ z-K}mD?<m-jqMRMqHQx8~nqo!WmM`$Sq;EP>MTyW3cZ4%fsy7;HrVM#4Ozz#xa0v zFq221-3z0QzaT}s)Dq(yhFRp}!Gi(SB9d^4Wg1w-mjeBt6U~Xxy4Wp5-|kAp6fN04 zdC$vOc4QheRF7AKq}oZL=n(IX#9yv|CJn!fczY6FbXu&yPdy~=xB#|9)xLOnr9 zu2MgOEtj34&UuOYvfSBJcGC&oGe?G238CKN{{WqK)LG1zM+;hz_uj`J(!E#ZOxPS9 zN2C3J!b9k(00}z3A&x-TZIVY>e3*SX)m$e-TXF6KfafSDRV+pMwss1^xTV{ z#=D_=3G;(*2e_+bPqjRr04g>)NwxGqsbAx2jp<={5F3AZ6bJ4_HF97W5*!vh2(t5M zP&|U5Utds65eiPiqn{I|CKloA%#vuDBn9qS4@6j-y=FjiQ|)mM*GhbfFf+48X*ov$U~G%?2NX_ZwLWVn@?b+h!WO)Q~y!YSR!I*G}01ThATqA(JBzTIcNx zj^TOFSn@zBKi|Zjpl!#wKb9*3Q~-RP-)VhR$Fu9Q(nP9ZEzPYtAdcqPLxOg`zFi$> zA0YD2J-XCz^(;F8y-JbbCo0XnCrzSXGWCDlPRctENllmoQYa3XCGaxL#8hW-W4tTs zvfj7q3{c^QoQ?ptX@SbVJ#b)4QEYPXXH9Ko{e?6H!v#Q_NBm-Rv3MV6xHKRJwb~k) zc?NLF3RHc5Rlf|vet4dy0ndZ{u-DN>YYi%Zl1ZyXsu=y$$IR3r9%_OKB$wE*EQu^q z!1_W))=yTfzLhz(uW|yG{%VyH^M~(UwvLB!yMU%8T|>Ds`)5B zcejGSzSW_FFVH+(pr>CfeQcV@24BP-%akIf65p8a|Xzr9CjP$Rg~ z?xMz<2FY92<{d31mI~FWSPeODhU3VOUAC&)bUpdyZXEPCFWs-DY-ju zmYusOr3GZrupvR?Me}WKmJ2*HPn!Z$D3G#34>I=4rBO+TUnLNL)o{U>FSGMBZmH@f z1Ds&H0sG{s;=?INBz^92D#xcs^OunYH?;BJI>nS=8wK$)16cSEG*drZo_DsRW(fL1 z1TpyMzjFl;iwUN1Ja?;@jkJ>$H?2X1M>ZrZD1Z7AFlVdgugIbH1+;Qs@D#WmB8Lt~ zj~dfWsh*9b!Xn-?vI}pxlh|3k5YH81iz5MNud|C4}_8 zjgxisM|O-h<$>GJC%q_h5p$rl3X=*1yLE!iwRt<8iSgY_rjR1qy)vE{D)!wbk-ppr`NpQ3UL9^n41W`KcfaF}IK0mK8~Vs1JFmals+9*zX1yBsbsI1<&^)Y$1C> zB#0R%eL25^4%OGU3jq*`&n4LNNY=XUpfqS%h*i%eBEAR?Q)TsGGaWjwt_g~ydyrZn zcpsyOuSje=EBM z{O^bKPdS>hDe~u<-Eqci&3yCvg)*j7uz5GU3X~0ReaI@0NFH(2N&%m|)l-F05@!#n zOdx(r7MHpK3RFu=EAMT#v9a;@`x38rvG+r{zA{4&ZS!I6WqA&*w1;ZIdh;rBUk>%M zWryZCUiZxYtNP3SYNCg>q6DvF(WGo6B&EF|_kC6RY+}2-<~FisEB9h6y7u-xS6P~J z9u@EWRZ*9nwIX#qT3h`}MteEuESq5IYT*d{kL`!D;CmGA{_*swextdp(LJdTGeKeP zS!W4Bt=3|rHMnAkik-j>HCRh8ZAKmK%zjPVE5gsFQqyj|#aXs9Mh1G> zcJqMnOJzq&wz>XCV4&UL<#zDRoXa=c&-jpQgN$UYieP%KL0|@3<7eh!xh!= zc|B~VU=(y96=<*d=I@A{6^km{aQz=a_0+Ms6oWw>Iig-8rMW-2{1I|8{b^1H1@f9s z;R3;nJ@{pRj8}c5F^0m`Z#X|HgLDhh5PP~Q(=IQ}Y`wyMe9L7OJQz&XeavcVtWLrC zo@>7Q(KjB={Yu&rz64gF8Izw%N?&KcRypz|V*Vl*cC|*hdGuGZD(<-gL=Hzt5iNqO zM56aUO*r}yqlON_>{&TDP{?GMt+Lm?!61!R+uF4a)m0)yLIMQY6U4KTVhl6Jg(EHK zAdE|aLIJhJ9*&s4R@wMOiyCjF=8`$zMz(0Go&rRG1k32$te51d&~9kkK#e7*dQ5L+ zN4Pyz_a&hjadaLU478cd+~fL26pUQiDWuwOAp1!h{b=FC0Q~aK(Tp)ZvE)UV(gsx z!-*-5b`@BexS<;4nR_!Wf&mp&O3}7W7z28ML(mJG=S$kebFqOyML>waYN!l7+n>y8 zQBh5!-hWC&F4*xw;)cg7q#raF2~L35J;4n@|^~S#Ck{g${Sumf*P=;0H5)VMWj(T`IB2N z;ti`!6@C*Y<0jxr;b8(@U+`7WUT*~@C;(_d+>}8oZ!b1V5WnWD^e8;s9u)w1&nMe# zPUndBH&*m-b}bkL!>z`o?0oCD-ud{bW_Yr@e#dKbmNN7TgECFv{K>tN16LsF^Ht5JK&p6m zHvj0m(^6yKRw$qPZ>h&ezN)SMH)eQe783>KThQ_Z3HD$LQru*@!$sOY{CR7}4bpLv z-9K^G-e0oz@yU6}&lr#lt||5j_mp}Q4N)n5>OTKhikW%=hP~QXb#Vu?;dE_8yXy(8 z&5sP?cBqvX(jg8M4yyc?UIU>iRIrcb#mgMzcno#EJ|J_U3+QlhMb z9O$zVnkVqnPON@{i(Z=uq-A5>IZ{6&Z7M81G&x#Of&t6I>1R>&W4RCquqju zcgTMbH3$V2c7-Ixa4-KC3h*+uH<3>so*N4HKTs*TY-l%Pkvf0rx;;~_L=;^^b$;6gcUH5x($0=Yl?@>lz z&+F%{V3U`aL$OiDZ-lQ+IdhBGo(NY48xp@-^u{VQo=YVA!+JHEg(h(S!&f$rhc`R z(*0_Y8lt%8nY7#WP=?Qovw~sQ#9}4_8MBZ2Wm6B6)c6s40*KOQ;f39;`eSqL)99xi z{<#azZK2I4RFJ@)qx(TM=;pGxTPkrmbNzWD<=T{RpdG9~>F5gXj091dQqg(PFYOIM zimTUT#)E(m9#@<2Rp;bsfhBRO+B^+;DP|t`359`iVlLQsf`mjFl`_4kq{cSu7CKzr zT|r4=V=`ghR2ZVdUPZ8CB zNeFC>MVo8DBXtUx>er@9n9@K9sR|V+7X`w1QhXF~QgwJL2zC{v}iXUOW4 zj~DbQ77G_e=1D$xE7#v8iZYJ;mCzL%(ej&UE+dd3RPk^4<>3t5aa@<<&wT;k2x!eb zF=w0jHV-**OQop*W{Iv9C@TgdEzM&-7Be=nZ!t0-`JXl!(ix6G+$>NafENwLF@$2p zHH$1|G=w?p*{-0J^w-}{nUrg1o$AudSkob4C%WRAc56ChzU3eMdIR5qT%$CsXdb47 zR?;JZ2;`e>K?XI_1SeN9CML z%EtnG#V|NHUqo%}2yVui34`<3w|AmTI)FNJwL6B65ZkF+e9 z-qrJG_xI?IUBFCcO zq-H?f2Pm;I3~e^Y*3VIYHxZworQK!!fk1*`@~suNiFRqoRac+|=!`S>b3@T{e}>rF zA6F`ATwu0i;|T|PcNLba6u7l55+(NxC0ADNh^KIW(b!3iHbno5f?PDpR&d_ek>*Yf zECn!t$fogtS}Fhh(wRu*bnqxk5xuM;Dh`B%#BZUb@$i|^?7qyH5R@yPBVoF3H+%dPUA^Z1`G*}PKyK2I2>dzUy9_h>2?$&b$Eo{fDsZUFiuUT9;2F>e@FT09s z{-%aCRp`CuR>y?uKTV-m`mT%!2-lpCg3JJv@rh*5yCuR&Si5ZG8yC9mn|l#1_HJyE zWm_8pp%RrxoQxEEl4GieBHaXhz%g6gayf zJNW`B=|sU?N1=)~!sp&p^dwxGAB4K)+ab5k%3Sq2XuUz)e(iq1u$xhFVu)&=8FDd) zQ5;GxLF*IC=!w_ZwF5f&6k48SMK)5oi+jl`cYFI;JR@S_32_OzVU)7bEj__^^n0px zs_J4>n(i_ChveB~l+~1UTXbZMS^DpNTgl0r3Nm=uV}TkZ?tuM#6L>myU(ujV&wYPd z-$!^YMaQNzvfLy)z&E}pFZ_#QZI5!Fw33i7*|;D}$*);YYwX%qlZDbHT0&o!BSy<% zetbw!Ckz5=JJTa96e|K_?u#^7RRH#jLB`(U<7_=JT?bho?2SrgIN?AVY+Ta8Xgl1k zN~*i2x$eBf+2drOTdtdXinpQAjXf+@ZRg3nP8ElL{3*;@!Fi3)3m% zEGwmXTO{04DR;E>JEB~!NT80%ns+7y=1bwP(dF-n*UvsH?SCfYvr0ge3RaZX@VCT& z?%ad_nx?M1EQujjvpb#)&w*u3Q*DpUG3}JW>EV8mo`5esbcR6 zax}iRn~fCE{Xj2D_@l)$gHKGzdVFsNam-y~TGe(LzFIA}%FnqufA4ZoGqVdMS^--+ z0ujGt0sAEo>!GTSK43(xhA}H@A=Fw2*JO{?2C)rpN}F1G^0nRhmENeorxsklone9F zW+!Xm6{esw8_FJB8I%&plDtJ@0+5M+Zg7tFXlbFfkZ^n>LLoEfc}yTwko)ASKQ1=hStELxD<7Og2dV4VfAx=+#ZDOO% zsyuuf$UeaZgIZ`)EZChD0qOQ_Qw$LV3QXE=a{7{N0Bs-D?8@53 zak-H=gSE@95st$Lf2z^do$P_C{VO}FV(s`~{AKHcG#E~@qF;$y%)}3W0Kp?Q8gAlM zOv*y9TK6kC6K42!GLFjw(AFzWV&FpFlOFXDOLft2Z2o{+0q(JYgoZHBgVbU~!9OId zL}~A?Jk9ylgbMEOOTei%>Z&5j^sU(ZokYn>;c>W%cqO=CoJaYsA`tSIbL9y!F$Rd2 zAULm8J*wNT*%0diCLspF&`F1L@pjdG>b-z^PQ}D9iNt@tH8K=Gl2`c4+zN8dsLhJ4 zSbul!K*pp}QKiR&uTVWL+7lvhk#)kd1*@XZM*kxs+ndlrOkjR3*?xt+10BLlRZV9U zoF+4je?{s;zT%O5DC_aTF0##aGOVJ2o*VXDD6Xbdrg6wrUL9J|3rQ_3fgBka!4-5? z{Etu*p{)c(tqvivLHIO%_ukmJ4mlcAd{+jP3n1h}j(VKv0LZ@}`lSsRsF4i7^*4Gv z^zlNm^1DW74@RR_t?*m-@3D_|fukB>2@XEM5rTvy9V3&!=o-feSm`&kGIL{cBm}xQ zA4Nz0FU){Y(GY&es7;lDJ1R>35$VZEh1C5^6th92`8fg?9apfNFjfOVE6C?hY|>mr z@DLc8!iSJth8-TTqc(sd3>)pd<462pp9`9g`@dg{D!?+IpdC7K2aKdkiF2ny+0l|X z*Q1zKr)X6<5RG9thrJwySM*WH+kYm@MW+U0bXRi0l9OR3=)Qxhq2yu#uT6DE*@||Z zOh2+zq-f;K3+>xuD2Ay0<7-hLNh0RQQKWnaU6FJcCaP6@)y-;h3F6+W7_POac&RH= zOvO@t^f!QvCs}Edpq%j*-5xm>i{fX+kIhCzQ;W_h>kvI^o@3rQtiQv1x@DRjsX9Sp zr^lGjS^y6E4Pc?0RvU8CaD+Y`0=Pz&$G=pmJ3Ge_y`;$u2(-vwC_d9>&x|uXpLNt= zzVWo5av}(`nci9EbcDd`JoYH<_w?Tj5PTuo{fZHUZLa!N#~23rUD6A8yzT0Nutfmw z>QtjYm1lJ zgpEpfA4CCsT5Si-LxJnkYm#0tTXjKr%9LfINj!pSXV(!_(|ojsb~ckwUP<@(^HO{e zm6W9FY4m~>vXBA|Xyv6^SWc*?(G+?@-7l7WFeB@sE`_TL>Zrc&(~u(>tzSNa!kv>5 z{KY53oh*PtK@>>n)AMJ7D5`8O#1sR0;KsjCCffJ;w>B5~dO0IgNU+WF%~q$Zugivw za_iY<c=xl~PDrYxw9n!cgnpM20-m1ae6{wr+m&d4U5q;Lw^ zA;ABZfzO;D{`yZbgBcd+Z{e6K7ua)~OX-Ymyp<$8E>H+ViEv&EyN zx))td4|mWz81c->L2hkST8GlfYc->kvd^-@{^cq}j!CBv0IrBtQYZOr*|$D>)s77p z#^mDTI{H{YTi^(=`{wf$rPi3cOM7JBp&@yUIB8LJ|{t>#v_btj7$GC-(o#@ z2+l_V&V*V|2Ii#|A6=OqD&G;-fw>jDoy$`-+3Rbeny4}ftd9$odCWOr7wr;D&xT^Y zsa5YgRL+36du9QTfH3_bovr9FZ^f&JbAx}0@2@t3a|Z-Ujw#Pr(bO|+N|Ph}=q5+& zOZKS9v|wkXFJ8^x;9#Goi-gz(PkM|&(!7XlMq#V~uG1 zhUQb_HvDI=mRb;{I;`9ws9eqvkP827T`_e(sDh+B+0>-WGgPt6+-e54Oe zZn9zew&QnvoI>CHd~{A&T-&gEYqc9DF7B@G&Z%K;JN1t}*f#ZGlnfS&kc5V5Zi$5{ ze(c~%%@dqe(;$ED;ZxWj@7O;?U4jjqQaEqP$`!C+Bg0?7Bk|H?u;HaHFGGmV0`NADC zrzMY&u&58@CW_2>sO|f40z7fIVft(R#~+2ZsWK2ooezi-MucDM!6ou{t2yNOw^<%CQ)mMNmQ1#tnyTFuju2iWKbo{{N2E397~xX) z7pFLivB-#pkZ90$zWwHnoscxv#qCz>^2p6_rp^l*7)0e~-qHT{u@xVwHB7xg*{YM7 zw(x*8S#;(F#VCjmdo>*f5DbFabtM>Zr&VXSsxj%tKF-k}g5mZ5?AgZNnREm4F+$_y znjM!;lJ1M#$_A+iVfBNl-83V-!*&aD%>^bdVh9f^g-E*?M@I@CR9R%=grH3LfEhMj zkM*+y95ryAt{q=C%$-5gyuy7ZUiibV=ocp025su4Q|CIEga58b^$uA&kB|L-Ic{N^ z!ZESm6V|mEUaWkc$WfGR+uid3a8Jjl;Zqf+nU>U@Q{`~%MmSA?CAk&Sz-V-mKLZ3S z(lZ4PLa!8pdgKPU{7FuzO1Fb1Ox?^7PIa&N zx3)_F@uld?ej~tPT#B<(&k}8OKLv+VoX6B*CWZ4*WtM~54IFhlc6`@YV^rcDi9W5b z7k!`HNgBtQ_{yROa~M}*fdr8m5OIY-Md$7Y&jh2pjNTpfooFl4KmJU38PfCphl-X1 zuGl8VP7twG)spUau!b}`KHF0#o;sIID~3K^V7Q8SexH5ubZZ z37@jts&=KHL1fx7ba_5sW(TKk1?w22OB^y%r+DxmwtTn_P_pVf&HD@Lwr!w}F)@RY zyDBciClqa`e!)rim7bu1do-lB3{qe#s84R{uJh5r_(hjAo;TY>bF^lG;wBnyXmez zj%c0cMyJwAn)w+m&9*e{yZ>mAqe$4%^rD340L^<^g=kk$9EQM!g<8ii@MO`k=M%+$ z=DmzsK$I9(l%KGrtpE4*?54|R2ZFPzs%`=YD{0K()>+n8;TiBks24ObKO&Zfk{Csc zZ`-c?Rl7dG1=kZ3D!CqAqv>pp|Bt(a!$Ys{_aXmRulBZz_nSAwAlp=B>wTi|YE6X+ z%0)ir2K>u%2Y1z~v?to`CiX_<+eE(lkHziD`S@IuKI=rQP48>%p`sp!h2(=Dm>n9J4;PO>SJulU0$tJ90m%i-Ew)NgE&7to~D2tD!xb4!spXQF6 z&bP_n+FVEN&Y|C++OYK&n)AG|>sg^N>vJudn)r13W^YBwtqW|+KJRtgd-}=jAr{ug zQ10KGx80lFHwRT}0UX}4)x*@c*Sl{|WjNQ>&w8xG@|I_lMKGQLIL-l{D;1PW()u7#b>M3D2Nz^YAlYuV2|U(*VYy2GT;F?1*V1}xkTZV{f#lWlM@et|UpLhmqS7M+q5JUOdsqH}uhTQ5oKK){M2Z*03PM|EHMxz8YzBUR z_N83dqJ6Li{L<}8zo>J<_?a~HQWS%_{*Z~DnH*&DL^Mo;ays=Bfx?Fz-lIi zcZfLpnf>85R=<4E>ncYB}7{G1E7*pALvx%=m`~RtOCOz2hT@^KlTNro)S6GW@q_1nxsWXzzVy^WpMvB9{ez;;B|O_Zn;;*)K*Bi0pTZ`Rq4 zc=R3*{;fueLn&yQK^rDL9nSATSvot;NXl9+lFP)7!dJr=g5uzzPx!;Fu+*NPmy(zg*?{)x^>d z&YB-2!-%TY2+E48_8gO~D5XI_K-0+Kx*oDDuQ-!xj35WW@JB2aBW{Tw8|+pLGlK<& zmA@&#EN&>~*%WH8+X6!=V}uXo4Q5EIt=WT&tL*0#r_;>MiXl`G;|@KFV9dwmk5^K} zh`BPl=`L?BX+$pD-_X{1Anqpo&Q7rZm$RaBWbmbbrJ>{C`wB`784mVQI6qve#VW$# z!4s1Ai&row4s`atdKWM@HtCkXU>x5HZnx@`nkAcfc=EwQ1Zx>xD^~oS;>(fQMFQf& zG0rkjxQZ|~9onCgX}%n(*<~(&A;4PLSECteA$69s!&9%CXsoLKX7wh-NGc<)vk-$0 z>3!frST|_qN1^NzRmkHV;_6tTqC}MHeTne0-XJo4d?T@PD%fCZjF*VuO%Xm*l%0Z4 zuB{n>hn9gi9Fwpacb<;Qc_8pZv^N__Q(oVs`qss}RXg~&FEgxipk-WCD0zvpq*($Z z(3bdin#ra>a~>mkJWWn5_ByyUX~fy?gw7y;@}E6XKW3FF&V(;Yz^%E^8kvE-Xiq8f z66ws2d^?gL1<{S?%fKq)j>i{)5D!pPU^B4TkB``hzUjai;8F&<)C}9!X|I_=Ll#DF zPE6>`0%#=1EK`rw&u8yLo@U2gb0;!c3q9@wc_x%k#$>hm^jI7C?Tzso-2;g)Gbuh> zNE&&rJ7YJf6S^H`?-{sqlXhN_E?QhECn zAKPC_-SDsSE zq(-Q`()M0RvYD(CN$t{GF3HS=@zYQgDTjs{xt@X!obh zF;+TlA?V@?&OGQ;v^7dpx%BY$)pU(M(9>xXbeZhNhAZH9mZ&l-CDz$10B$L&gkdCmanJ zl1}dWiKoDC8mr8t?rCr-66(26uqVHHPqUf7&i3tP)1UgM^|2+kbM7~SU` zT$8i_`6BsmkVHLK zl-82-iSTxO4FM44&J^q?0+@iuEHrmw4_=jcS0OB_6vem)Kz--HB-%z}nm|-u^|%x= zU@W?v>BFW-Ws8OP1wTLR%u-oXv@yx9>+wSLA>?WVuA#~?zOb6hi18iek$iQ-)v>IK z&itjnqn+(>U=a77{8Syz+F_QgI|{Hhqd~M-tQW;3m<>!;POJ$rDq^2)lvAfXy5672 zW-or{h96;=p-W}IOiRncv$BbFHp^4bHc;sOZv6et=yj7rgRfy}PF{8%1dd+A^W3F) z9s4~$!-Muhq)^=}g!{A`l4R~C-!v^L?`Xn+Zk@O>XZK!GC>T#Xmby`!b`@w+q=?M4 z0S8bZXw#VFGES6xp=K#`^nI!c(3q+oF0KbPzQT z8VJN4mI+*86C0FQsUg0AIC)5ypG4#5Yd&Am*vJG(tt8tr!G#^)RGs3d~%NC(d(oGN69 z*Ct0pf`ZaEl$?}?yKPPHaPhQ|eL8k#PfVt<~q*dG&}uycpRN9&Om&t*f#z;JXlavzJ zO2mWVcs+<4xB5Y$$dHd?Wi$!YgMnyhX!yQ8ZEbIT|Dj?;KXxsyDqOBEOAX{*^qE(l zRNLUB5q9R59qP?SP0pY@RK74Php#cbHA79vo))C};T8oSTXayh-S^Yy-C#6NxWMQ=Po8>aIGfGJc@FqbjTZxt^WVe`*b|GK`n2B9%YiVs0!}Rt<2t znVk*|$~H=R3!wMyH-mddy_=4^tUvixzSOHO?3*jht}84vE#$inkdOySnkHLnLj0~f z4K!(NCYiar$X72prk-9bFeY-c92269pm8T`)|xCke_NfrcamTCT|cNw)7;_anuJVg{@Jb%;Bf`0Ha_R>yAg4{;2I^g7DdM}>}E*XZlBn6&jpz8it8~x z;2#w0S5IZ4P3Zxz^viYl0N3Ha)I$<_jY5@Yxts;7=Zv(x%-1%MMmRo8B zL`~U^PKg1YP)@Z$FBf9cT}{5e5n^^xKyIjo(bv9hCH-hmo0RPH@X0S*)L(;c3btKU zBH#eQPD#*xVxAJM*c`RyqX$bwPLjKgxmGAo@{77N6C7GC`2nk$IrS>tBQXsO3*3ze6H=Z6NQvTn|3Yt(I4pJ6tAI?sQCeLg#@?X~ zupml29gzY?DCF6@M19hdM1!p%xFive4GbpfhByk0L%1&8pGb?}z-QnlX!Xb z?@N;o;N+3hmqK+AuRiJicOM<>fc6Q!>KxQ+rPd=-yg574L5Q))@9-QO)k z$8f6fP@X(rHkDH8)WEGxq_pm%KunM!4rI))-AZZ1XM=9e&ln+c?dEF%LfPj-q^~o! ztOwrhZj1eFC;7Qa?wn2d5QgW2D|a}dPMvY(sZpvwawu?<`IYDNsf&uGMAi`d!e&+~ zl{5VV;2yAW;XBC&bv0h4j4qw{? z3z|lwn=4QM1MC;yRl=G6pwTsiNqulq7^KN`>d+Wj=&~O|Tpg7Hb}O)s~ft+z9^dtE5}_@T|8P8BkTbN5*@f z(xuW%EjwlEP!~@B7#8Fk`=vIBocFcHM=4(;u0a?ygr7tvaG`yoS`gzWWbj1+P$N|NIjFY%yluNaIlD04cnJqKz1H)H>*x^0uCM-y^w$n2&`)e`X>D|1 zbYp_8THi!T$0F4wwHFgYKI@>^Klo1 zfA+%y*)t6~W1A>^|oa_3hy_?+lV*@#{gPwEuWifNA9BC@P_oSMu8Thfw)b?6kEdQRC@ zVgqT;CT+!XwnHE`98@!OyACTNEj$}b%qoy4X8A|L5s@@kAE#lGDTi5 z{h)g@E%%AE(U5~3g~ ztn&ij-qhA_@k09^<*Yr@J)pKVzrbhPNo$ z;fU%GmJ+eHuz(?DB_9Sr?h6Pk-v!CsXl5<(UeQxJpBdqgCNlGc8lZE3+O+)OHvcqy z#t4MbbvRCK(sr?2oeX$agMRs2$eU&b3NN&6Y?b272QzWbj@!dFHM{-Y!_$60dahT1 zk~n2F&4|f3Y$2=4>Nachu5q9-FzM6p@&?=YlH`z;THOT&FIjm`b7~Fx%j}yOsMNG{ z&f69)hWx{x&RuSE5ml@H`=#ZH5-MNHF7UYC1`wgRTZBW}vHz!2zLL+?ur+l{J;U#l z@i?8-+lgX5?nIGGn?Z0Va;U^cK0twqEeqU7XmCosG(Yz$Xh=RFtb?kXbWckx6gC#Z z6mh#o&Lj|AG1zP_8d0g6?98YP=bRwgM^&8pIx}vLCVsZ>yxDqkm?Y#J>y=f-*+9C* zr%xEVeWmT`d@??vYTkUUneQjN0v<>0@_*<$$L>tSZCS_Y7;kLbwr$%<$F^7--Z zwr$(#*xK1^kMnJv-|&p_+;h&Vx@s2_nX>U0J7yacVqAwDzo1FpKpFVa%X2UirC@l{bJ=Z<{>_?g{YTzHfJm3b`V%2pO6@B6HVJ9`ag&T; ztTkNcf&9?91SRRW`+4;Toir2NzbwCWaN8d97UklM+ra$B|8UZos)E2~>ivq@k6 z4yh;KB75@tb6Tr!qE?;tZuKOH6>peO%X*uV(|dn{ai2B7Elv-FF)P0PIF|}v#@XT6 zciP?MTA`{X;SrXBRr*#N8^ILy^gQMRP=c|5-%c^^Z1ayGS{)~5LX-bJbiD1NWDm+M z9Py!XUu?P}i3E+Tc-(boHR;m?LQ_eK&+Mn2>H&la;Fey&1!S-;%4Kj;|E+Q;gZ5!f zrq$pJyXIvr+rTlyEdi2r=89G#FZ?w=j|CclF!3 zc(vq8hLLh$hpO~L`p9F7zvw(>0A6F>-JG^{k5LFMj3^>vN=Bz%-(AbNdc3{Z<6S99 zs6K5+Vq8R6Z!!xN93MdEs)VUKExOo=n1JFGHNq@!TR(t@V)tN>s5z|n5e3SQ8h71s zOr!0Cp$wOM{&miJqM@d?TlUd)SpP{rhNpcwYAxUMuNGyzCY6YzK2L|EN!qV}5yz>U zt}V?W>h8PJ*6618{}?i**}#A|M!??YUi6O&7W`k9Bp0o<|FwmWf`bEkv1-Gw; z-HERb`~K`Ema9W=3ei7M%bTS)EWC@0fqMU*-FsE-nyXhM+%OGzQHqY{>nKAvPz3s@ z7poo7mo>Z&I929e{Fie&HqldrmiK(z?fwPRwb&^j?grAVCv+bC^Cy6oI_u|b%#lJ9 z9M7#&RRU{=~1VsQ1-MV`a6M?eeauA!Bba8PlMO>RuEc`$kLLpzeEeKINr4 z%sYUaVMmp59cD*4u3_IkG|$Tp8ejdd7 zQS_FFad1x{{`ju{1o_VUo-m$o*t?1U>GUeD7gNoRFfI;MJHkZls9kXgU}U@eHV5$= zu0i(-7uBvb63F8Vsu6Cfb(y`Czrhrvb*`h>tC4DK^(#GZh8;`R zmyFev|kHnQXfmjeY& z=y_ng8A3^*U>Q3Ci9Y|9ktWzipm280lPvmh9vI<8)v*Cz9fLils(3&sThyIHN4-NBe zE1IGrF@rZb{^{!e1HP32knw#4FIZvHB~ZabagMHwN5QEpsJNn0QY37frI&%$!7<;V zPc2pk(K&~-;@mVoR&Mx)!F1>W^=gvhpo&kNG8|9`h`Ogf(_u|(KjaAuR%Y*JdmRSV z$_06C*+DkXvT2XaUS?iA9r&}cvQ`C^@}`a z;01X7_*sQfd`gc_B6!6C`I&}C=>p*ORfaZit9ri${k4@E?*GIS_*=*fZdB|#04n36; zm*@@JEC7?a?B;)L&vF|@MUs!7xRLavHP4z-0cKuFx+2nuMSQ_uFXgTfmaDL`R$><9 z>G-_;Tgo>ZiQ`EX!HnhS4SXZ8B`)+j!Rb4|zdIJYz~@RcY2hHT8p1gDEVXIs%VA)| zpS1eL9Kb(?PvdIZTDUI7Hm)c4P-sXFUaFbSCU8pNo?^t3M`VVqNPq0g*c zo+pU9B$BHFCRG(yfynnAmN)JwIt+wUxc$ySrYjlqJ`p?Gn`AK#f7H*)NpajMhR{f+ z3Dcus-yfIzRM@~FpHWgcr-sB3bryu^(U}V*xS#j6mnA=vnWp8?OzV6nLJh!ysZicB z))AG?(v8eB zB-%$UnvAs0w??-Nbw&Xr@SI#$e7f-_w<#Aysokx0faxnoX2*E~63B+Dmar9~rjlxq z`?A(c4bkN}cyri>DfcyY#i>|>NzFr+M_QO4?BGrie5WawLp1Gy!Tg*m!ZJS+^md#% zSS**G%$farfure)v7)mQSzHNeZ03-<+co}hL?ndCXF`OxShL=QL$S{Pl90YL1mvAT?^?Q;9hG!Z>@2gU$gZN+1YwpUT%Q z$^uLk`H$Os7|=MPg8W(TJVX21b0u{p5@TD?(zKK9z@T|-M2bBWM-!rLZJKB(p1g@% z6T2OITI_qC2@}50eKyl+zzb(>(rQ-iGazL3QU!kZ)pT5=eZ}51Lz`LI6czpKEknOB zm4p{UfehktPXQ=sMK}x+eBE1CR+Mep6!YID<&D-T*!z8QCO#JQI&k_&kq<<&r`ZnDm|MuC#}ph z=gz8su4GYAuGIIs{l(Ke36Lif*S*K^_9X^=yid`Ahm_j6gi^ge0!Eo7&z85U-OX(< zoUwV7GwUS>%ELSU+e^VoP9v#U_-{?up>J(2wgxnEa)%rOM~B*$a$v9w?ypYq$PyJ} z218tk@P7AP`evDH?sN$h+WY_m`OAOS&o$2bE*)FLO~#NlmDUEqZ`2DROkyO3-bM45 z7&~>NsE+KF(M58H<5^xx*VE)Upv?ag%}N&(Yp0asM9U&@M|JMG3LHwzP1aqoGWj!_ ziyw)$tXZBZcBBjHAVXcG_ost;pe%J%Hwl0jP`%FVh!?Rgdo8ZhbZ+Xa@z^>D`Z%u- zto=;lhQZR7ZJjLw`Fx*Mu*BU&rH~_2Xx(~UkF8?EVG$NqF9k=9tJdW^R!qe@HKqVy z%D36Fj7IlqtDF}UTj`ss%#F^zw6vI1ODnS;S1P3@!E#ymNZXKfD?#8FQETZW8n2k%cd-K6{d-*SD^=yu*YTo8Wz$vA7AzyMo* zr!LdB&h#dHBl7k{p`P2U=&$7ID&{pmM;}zdY72rgF>&kYJ?r=&m{UEIJ!%ax)Net6ua{Y zjbJp6MIH7owoj=J4c=fz!Rsus+yNLQ|12E zX!CZv80x#%JO;2lP1NACaBf^9ZnpuIf?NhTuB>30+Cr9~A36KZ?R&*a`l8*62vj`D zN8t@fHu7%-J>&+4NpE9gHf;$fD3M>apljVnFtU|>3uv?>21ko4=vDtxpEWXQ4*8<$ z5|eEeUt$-{AA2VUC9708EfS3fZ1+5Nl#nvI4!(xd=^HR6xbpO|F-C?+_RiEp94F^# z(tO7TO5>$ALa1SMnl@3<5S)dlmk>3o`8cpPLQZ}qoe>Cw0#cIS zr=W2FPhkIzQ?li`Ab~vP+o3K%FwhSKEer-qC~yPX3r=50KbyKy&lY(&hY;a$)3*I~ zZAs}BQnyrfUanjsr+ShnE-vo({lh5f zxmn_iHon}*rx@_-AVU*&a);H&R~u@L!sXG5b7=Lq8_r)HWK2CMZ_UW8+h%vU3^Uqt z0rdv>4L6sv-*WO`M)_E?yJ!R6f1+MSxWe2#LHI&|-KL+_z`EpriJ2bYJNGEm@Bb7Y z>M}ED4+Q(sF8`9&UyyT$Hy~cdI>apxoY(#%^PLEOTC`cO699cH!*Q@`e&ct+595Lr zg*B1E{3-tKHLfs%^SUOg6l4Di7>=%s*o`n;k)0}D4auwXM`_+NQ-zgZyt?sGWpy8h zyjjv$*3}!p+rWy+KNxbYly;sa+s`L@OVfi!Xv!ahXM?Mg;1gg+GcX|RD38GyZyd(s z2&-XjG%(!i&AKhxQi%IleWK);r=IGsC zap@rB$d|u<+THK-)_1$<->}Dp`MBj$X?5Hwbzk*O`ZbGwfp*vJu@i*!| zQxncB3}}AL=A(THqBk!A)%;YYCqZg?nU-vXF-sA|6wk?1VCuXAXW#P;AXsQ?x|w2^ z>Qcwj7r<>bUi~W|+$4Hq^9c?~pC3~iDsGiKkc4lWRaq!)T{XIMGPge?=7aFPoa<&E zm59Mr<`u&*aWiyWG3yz|G$vs%UR|<}hcS|Z0CrT;UcSwh0S+c&!R-g|jA?%O9R|CZ z)XhT$m>l(!xmqndoENy%%QM@wu(qLi#C0Y)jIwmjqle#r%&6>`DFA;zk3iL^_B*&+ zPu3=K1-L66OENG8$=M6qj^Ahm8fP@{PM1TwAF+yQvB_>T;idB_Bo+fQ>AIPWN+YGI zic`e%l%AuvF3`3-P_*PfsEyKs`|7#P&ttgDQ;%PHRa3h`S6<6~DB_*x&#L-43Mx0^ zE|F=wvdRMI898D%M{;GYM)xo)OizT>nfzB4?ERZGqKBl>$Xd9PujfQpenwZ607a0S zUe$P;n2mE~DUc1mV%YZdF^nVttW6|5=tV*pqW5F21Z^K^LTNDXB@qR*7(Uiu_b>*5 zpeDmd6thJ^Wk4$!zDWXC{y7MZp0P}LahP&>rF{X19_6F7eJ|dJi?hyWQzC=)R|3Q!5N zA}LnL9Z|GAAS*M}z_+vvj|?oSMabeN4c)N(1b|Z2|F#r5UUCTQ_>S@Brp8382o0i@ zxt4Qkk~Wsg;DJM4S#gC?fmfK+qRh~T_-!?E7DIj0O#L1$TYArO2z*3|j{Fg|v-klA&7eYqgiK2z) z!0ygwXMtCwDlfwk)<1>^Bhl;-uPh+0WTVWMd4<{3nd14A4^Xif|5H*GT+DkJE`Sk2 z%?y)5BrR8Fh1KbO3cC1L0MfTVm!LczxAfb}NzbtWUs(YQS0|F7F3brV|2`Mdk%#o!LqOs*yHS0QT^jQ5-fIKTOA*JEbopVR?R z75oDe0pi-_W7AU+%c>iwu6sa4g$zw$F<3_GL{;CIf~&!{w_p{>I%&XITY#W6CrD5g_45u9M+ z>4^4AyjP5(4=IargCD|{fprEHF-3wg__$E(b%loaM~s9J{N#6$UhsTQ?6+h^8>;86 z-qe2Y7Q=BSv2*9eSGd!{@s%>$HLLkJxgjEw25Gl=W_-bty>>hntYg@pD(gT9jrp9x4e-w<-CWd@I9 zihNz#!p5?mQST5{aKrlL;AK_j7-_^~Ma-aR&~ zU4u@BvXd;019LuBJ{N(lK#m-gioKs^9Pf^1N24Qv6J1d}M~hcTj$2hFCn-T0RzYm` z3^1Ho5Nx=yI120}Fw0vA>X0xxK<{f>A49~feZ^~uD9PTU-?E%kMcaTRj%}>E*CfVD zX+8o;WI;k?5wJ7$q>V^N=wpLrTv2`}PHUj)UYcPbJNsps@KPfVNK0c1!!a$;wq|pk zZt>o@C?0V5+=Vc*MwXF3!^&Pg#;+5@w#uE=`@Xcfa_OP}(Vms=$|4Nc{KXYF2s8xejhN(ARo?OWJ3R@3(k{UwyFTE9o=hZ_4Y_ZPQy+>rl4@_^y{Kn z1?u7c@{2+<&|FnxuO+YSi55$r8!I!JAiAci<=|!@rZT+APSd^g=AvS0G8AC+d?OO{vh1uF|Ol#`)H zqx2e60|LKiFGvFCIB_HI@nxRM2!IH7ONwI>B7Fcbz%bi27Dr2A>8!e zcaLPo)VHIyFEEuSKK2hG;Vi3-Q}1^BbXSy5XlZAK6MLJtE(=EAnbfrfx_dG(Qfmvk!dSuN4yI$#%Jbe z7Sr%=y_paoNg`{cI^bB;uIXKA_SaeR-ETD7=?4ZFck-i8e6S|;G4ms;6eJeqjF2M~ z(uKPxff6idq}fCx3XrK%=TVJRawXEA3?{1*9bd`uur>ga@5Ia`FD6vAd}h_7lB^)u0xKf$-=L9&&LR6`F~L(P)he1T0d{-x z-9)eXusokn<{o)$*OKO&ovl_cY(0Ag6W6j-c_PVAE!-$n&gNpV^9qwKT#{zZ&Kng( z{z+imI5*QPHP6v2QvhRmKOw+8jzVdQwE3>-lQO8upJ)rghYM|t=vQlNtHA9} z(%9kk_s0d|7kYj3W=e-OcDblxRlU^<+?NiTs-D@N$97A8&N%O$$3}4TROXF(55n_R zefRkWp6%uGi_R7r?&fB&s{XNT>dc9Yz6ijlVq*17D4YiYH`AJp%`E%&KtE^WH# zfYrI+%bM==??JZ)O5847c2up}2g+~EdefcVh59CUT6(KZ;7z&fR)1|Y;4QRr40lUo z%+Jq({v~V$>*tR*i-ySUU#;!CTQSDXrSE@5wnNUCdbI{#Iw8$|AN!ol*V_t`yIXZuZH0OO!)PKFpa7SWqegfr4+t?fK6rw0Wc7;e@BBKoQMQ{^ zX{@=6&sjPmvf}%r{Yw6rw3}Sn-+0iq)?gmiYMF$t3i%z8v%^rw;#H5%JG+d1qu5bx zH`^8Ok=O#}9yL3!7a0}1qO1;FNt$wtoXM6Fc3}q5W6WV0j@Di{#^$pI*r%DR5?IC9$6Ph;?l63TI#__J`xwMhpk#V zk8_6wr5opJ6bVQOnBd7(=d7N=E2s^1!&q?@2yUe$;kWmaIlvEP9G_&nMmh$*$5R{M z*fQH#%3-ATLeK5_8x6*t)m<-&x_VB3lp`(c<_V*r1-b&jy_aYht8i)!5S|!3upNWy z(HtJ5*swUQwhmQSEkA{DeUQOS7Sld|2#DR%%_JX_Y2kB;^RP)iiVlQ^Bna zN(ghAeG4I1=z|1*4rO9LGLh0*6W=`NL{jfE0TD&PgO$;b7Pg^PHEA@1B@3Z~t-$t9rfnK{RbFII$**gdYqA|%4E#Az>J76n`Uf%65i2Pl92 zDOi{5+L#L)9XwQobOQZmBS7qX!0;tB?{BsU4tJk+f5|w*#8*3+&9qKsuW_+QwZI!Tw61q6CacizYcVvlBnNs z80#qOce0EKigm{=KM7^pQ)M_G(k6v9QNDu%Bf6kNaay?#x8CV=oW*BCzz z*54Gs&d^Z?I}1Cj5us0umTM+U8rMI1C0~O)JhF$LR(=5>bYwynjNq-d3XxRCkkE~( zf&p|AB&*<(j$@aapQ$kMT~R!)qKw;M2Lj3<)pVCA1coWOoM(#{K>1uAh;`BkV$$s{ zNi9~}8CW*FE1;BXSCvPv#ZIo(vYHOX_qO>FHqy4IZMin^E$&3J$Y@&o3PNQ3f zSEhtYD1V`rFP9&w`W@Bu7P1NovL3JK;7E2jebRu2O*xJJWGNYXVf@fDwUd7AIdr53?0#*L zziO{)kBWsEh}hCukX9s0PDoNcR3b@@8HVJ3Vh9^fxgoE@Z^W6fENUn>;Z1slQH6>%EXhg=QF}rLHEJllFs#K|hyq_DZJn3!DA7b%q58jS$ zqopxf;ll)pNT&D<-$EjiKzj}7@KDCT7Q$w@$dq%ba&*JFB3`r(Vm7F6G{h&T(n-sd z^m%u<_5vYZ`*2m6Z)ZxHMzdAJr#j|d97KNG`=qAgr+_3w4qxDTA%5H;mz=h1ir>Q|wVMZ#j| z(slo|qYb4jFI29O&}OZl7ZNvMT@YeG3#kX@C+_-Eq!pRQOy(=7Sxjen$OYo)1BJW> z6NL>2(em|?$I38Y)by;FFl^qaw(oe@Q7vV>)&NZgb!jJW6;C%huxY~uEbdCQI@_Eg zLmFC3IiBKSsVcfEBct#ig)_Sn{i&SQ+|7bonTvW)#vH*&0FvcSJY_8=4jUFh8lA~1 znt7HLH#bEJv+hnb4kA1-2$|l&@Sx=OE^KFAVi~f>0IcU>u>*^kkw4_P_oxwRW)X5& z#QbX{Z$k=qqUZV)q1MUAB&32|w5ePeRgJyF_? zf);qo(3k`^l@jbthYZmT2C8?mUK}2>1ZUW1&T377(BRjCRLT7rWl@ZD5H~eo(2BT! zB0o-;tb8Yp;9|qA&+~a_UhYsd;r5AvpDTHhPqD&e)GaF{KVTKcpvT|3qZ77W!eH5w z~P`d_!1W(}tLHOwc0oBGFKs z?Sc@Dm0esa)Ic6ufrziqBzY}c>xo)!EL@ya92>m*J4lvit0tK*humA~H+9t&DBfki z1z>AP`bb?Qz$0VW0$P+CP9GX1T*bBv?8C7eT`=%K(tChqR^))V3Zf^A+y{^^$1^9H zJh{>eO3P$qy_{yGILz7@I>ZcwCBwSy4@ObuPx;Oxsm_KM8bBGIG;(`&yczy+kuz^~ zVhW+C!2!YLz6l^c_el-%(Q!)kSPbWMtq(}P%@mx#JU0zo%yIL;w*HX|iZ_I`a427# z1Dl>6g32sZYP@20g_?Oz=9erdH%gb)UaU~Ja_Qep6HF zDrbm|LcoOutX7m5cxq@+cEe;E{TAwZ7UND+3hnwu1zc6CX!UfHL7{+>&~Dt7E4yT% z-aOxbQciA5Kn@~SL;AF})VYEx`k58miDJlB7`O1$NxTE|YYO0w&4&;MZrNdTRezSE)7fZb>me}tXB%t`R_y>W9YXlJ zkC)U5(X2^CBL3o#XLo@VCSk8BVB?;@fWkbJlJ;Xi+H2zexl*38WVT5$Ei;k~#%hs0 z?!Pj9*{jCH=1hzwLDRtK%y4K@2ok$6H7Lf+g~+UCv?g-^?M&1e^tYOoLJEo4S&loT zq$q?GnNX%oIUO%?q)iU&st!ZHL%`SY4#qVP=BTYc)rY$Z(ulR(qbd1)2N6 zZAXDg?|AGX3L5hRpKH+p*M384v^f(_DF`XIgJLcX|4S84~@VXHEC*6O4#Or6rLpb{hHE|XN+ zk!_P-yk-|kS_~M2>?pc8Xp8{Nrezvq`2f@B@{G|+4QV=7-0^oPUu*`!|H1R)Gk=Pw z832{d8|gpr{FnbCO8NYl<$e%>kJ%aIe{9&;(^a?OR8}Bo5xk=+$IIslq(f5Kmh}1J zP`t#7W%vzs$|p5!H(QbJdJy2@zn=5_d_w)6hHO?UBUM*JTmkFl@B?{6Hp+1uvzJkx zOTk7IC5#VO%$}Y<$_Z7L(bgxdaGei$&xfAv&4^=zMSuMZIBFdZfEzyPK@+jx#fVao%lp@hkV6 zF~krjvUwBrg4tbykDlFh3*M2xRcnDVU)yC0#@&(_ek*M!XwxC$2|;u#)^H)sI9yTj zFM;`wI&{6)2ccW44Hd1amGXUs#dHHf@4p+K0OL`w7lNFPdrsE)7waA^S9cxmsJ3S# zW(#TrPWN7&A~!+trbxH+yj~M^@1)uftVg7S6D_+0!{*m9W*AIm=an=^*grEnT0>>s zS@qjd23R!dytw?~UIbYvZXa92+hq4%^w^vB{*#iC26EUWK3v&aj1Tykn*+3q8+G%& zKW?W)y+HMIfR=pgJ>Sl2Z2MC<^KHEtO&Sg%-ECh5!GZ2sScGzTj;E$S~*z#lc&Abt8)G6agoZ*IDp zHZ_#*_N*FVXko0Pi^kej1OB%|3M!waj}|*zKbO%2BnG9{r8kg(B^P@~O;z1E7o&V& zZ{X+~Z`iCLZ5HV*0SV?q^iW!Rs}}ITgv~Fvk#R!wZ)I2)@b^)!w7;S?T^anp;Xc#? zf0bZKL-FJqB!ZKtlgVsuQ>o}84)@~-8Y~j>Wb_vti#LPd7>KZN{opRYTCiI{(<2p` zU^q#v3}mH!P#i#)f)K2+p+rpR=pYAC`{BBdFL1$=-MNYLfeJswSP2>Xl&g{PR!`7q z9?I)I(;OORp_2#}xqWR<)yos?ej5!}VXcu?sIc}6X8l6A}{7t*x0*>Hb43h zb>N2cs(vBIwARCZaf4&oCKGupXdH)w`MD}cqOwefIO6m{6D?(bJ>8&yP!?#}X_+xx z#=!-Qwlj94WM)jDv=x8I6sl?cbr@?Zh;|8Z!PUlgMdn#-FuaY>x}SA^p6*+<=Ul`3 zRmegLZ17;5qz3M0EBnOB=qdC%_)rEZ{+h(KHs4zP<*KqYogsCiZnAq3$2;bCKF*8ZE>D(k*aOl+(S zOVd0^wyo!hpo_?K@O}_IjOq`57v+yx2V>DzT>u`9XRFH9sHKeA+wZtUYO*%*Fi*Gl-R*+Mff11+#q>N1zPc$~uKASHb&I$bW z6Z$i*V672jy(K~Y@(SH`^R$mRz~iC$C0NEC_#hnNbM0@-9=)Adz|G)M-SSi z7O8onw)c1WD&!{q9zqcp4Xj^koaf+E4Bi(BN8cU=T$S;Zwer!21d>+$qKSNOof1N^ zkcb4?s8E%rzkq`ktrU2{mGjK%?l{Rx@wCfc%`7W1#Qx)jjW>}8d`S^$K>=P_zj9kFn;kc=gdT{ zic}6=e7y`D4Im{CqtlKq1BrT_f}T+galbjXp9v!XX@%PRn%#sQ?S}j+m<~r1~DZmXw|CqwsUW_ zbg24sd@#ncR$GlT;~;qlA8_e6M=kbxJ!l)KVEB~0L~2l|WJ|#d1bfsIT{xy1@o0QV z6fz$baf^b+TZ2;29Hvv|x0+lYyeFhFWzsj;kciEBhoEjE0yNR3_wr;b6 zx!lZmWNQ%j@sp*zBaqNX;T)3Bs%p1R7N(F&OvX+C}Be`%G>nxhfn%KxDzZ zC0)9P@U>PXYF=zoZCS(7+B@rfzwPZh)}9=8ynJYMIcU-M;N+6GfL?pc$gb(F^&$G? z|JZMe-WsZW2#v?ZZkmR|p-)!21-E!qL)?Ax+2wMEj2*F>XG)0pgo@o&vI`X>dtBS* zeFHu2X1_aFaWTRBW(DIi`i;`Kl!)>N#!6iO|gWl(y= zAh$%87S^--uN410UHZEJ_Pt!t9DV&-9HND=;iLV$FPNwwci=zB@Eso_j|tFQElT?L z7X_L5wwWrg^v zMc;_KV{{_Wv`#XwkH%)OCD`BGFg;wyG=>)EYcx@GUvRG=m`o#k`weI$msMlXilXWN z*l>isRi5{qJ-<)Wb@-VH%QuY0Vmqn!G3lURz7NoSOUY*_a4CRTk^~H1Rp?6weM7yX*fnMv9*OF9-)0kb z|1@99S8KVx>L|}~8?<;v-7r8Y*m3-o`S|R;z@O<>r_p&uu&xYE%!##@{6?t6RpkvQ z`;XwXUIh#&<^aU(tv7?=09OC2;G~_pDE1%J_(&_Zxi!`)1Q>?#r7LZ(w>wG$a1&L9 zRFD?c!zmVQ1|9l=1sJ3LC^BgEDsC9kjsj7WPSP9pS-iQf`1*&)scMOS^W&~qHWX{ zt{71c_|^v%My2|5cShPRrpng+`Ph~;SHg;O_#0%mHzF-5nm%VUa(hy288jXNwZ{{L<@`g-mZCPXSneq5(wHQt4JcET{! z-n4Fp8Wq;EN{*+tA?m^_g2nmTw6upmxG*n; zdF6HgP7mvT`pDrr%%e`GQr2lU)k!`DAfxlKqOKNK>}3s``IFIC<*F}TAo@C~ z%G^%o5mc|AjgQ6bAoyTF*!{_e{#}VD4QkT?&Diw`TT`p_NS0h!3J$IZ9Y&(sh6^gQ zGoShLw+(uXL=h49nHywFobLmN_CUlSt2=ZyHJi?HwGLHtt43mF_&Oz!F>=x(I*3+Q zCKFe)b6tBq#4)<>Ia!U{B zu!#pUFLS*R42ppBuWo|)5??^1W@zAE?e`ogy6Ge7$&$VtL=@63KhI=-U54uxSenj) zK1~udfs)~?vSg1vJrW5Ll#|9GegG_5=@wj}>YPnf0!mot<}IfE_F}(;Aj1hbZ!EN- zMJW^%J#oUiqd5IjPWYc@<|d9l{yx#UTM%1HI#EJ65)G7|uh){1VVjkMsTO|;j@5X9 zRE-}apT9|}V@|IvutON$>ySoD$N=Q9*JV^=cRoKvqHv{$srye4$GDrLk#ifgz?*5?Gf0|hyov?KZI?wy?jYz2M8Xq zwn(f(svX7_MgLR=B+{!Yw?C zJVEYcz1Obv5G$>R%0LUX%E#gWnQ+}ZYnysS3M0jF>}eCMl}=4MgeomMO)2Id3SQx< zph-Z!4GzO=W_?z?EzF%6H{naF^;pS{6}*JCtCQ$P)ibCI!jxz$;qGEi zNgDD3DDkKjRkYWtVX(LX#88q-OSt5&w&dkPifU~Rm7Ft%B}M%k}1 zv6?Cq?LQrQp%*Bf6)(tBQ7C*Iturl?Sp0v$A0Eub@%$tV`?2#6&6Voz)=68jJchGV zgY*C*j6zV7+f-^=yLS)`_0N$JLW_Y#a;vW*Z?ml^^LK%|C$4TGxs`tBI`ojj<38*O zPQ4Tg>j|;fv2M;6P`LsrD0CtcrinH9O^*-x#T6sGrHMaAoqSW1R?~exn4)Ivj_xWX zNeN;lc92oHl);2A<`F-mVhtU6VH#`V53Whdsao+b4&;B@NqNdDE>kuVuxDi$oJJ%) zH!?HPMN-}`6bMssAp5RgNMPD9#N3)gXF(#k^_;}-PUD9fv>XV_FSDA9zlzWQu+TSQ zJ3lSE6G`s!PcMijJ*++E%gI?d=?+23;@bM{8d>n?R&Oo_0q?TQiO^Rom;giqCG!vJ zsKHbhkgoQ?j?`!jrQ>Fc2ymZunznxtDx&T|L`T=SCrAN_qm*^H;=psVI@(BJjDPwb zhZ4Vpii7ys`)#=Fpnp0Zz4USB@l&VvgX3uuQ&$a7WHc@J%Y(zZ=k~g{v%TMHgLe>= z6gTuwuVu!-6%gQOK!|)21LGyQZNgKU2HTP5W~e7c^+jGR4o8@X#o$g0eTzWvn}bse zkaOn7EFT~09(*OYB|kF9AJgo2=e2qtXPz%cS|DhN_}rBk`#bSV}PBm-cwx-S(?P;jTX&SOuLkA5bRzYC{{ z*_daeM|S^dOl<350~%n?P?ZQKZ{}gaPhW)}`pn}6JTHp3J1|dv2KA&6&}PGx1rDt$_hncvfTiitlDS_$Nx*K&_ zk+emG$}{dcN+t-J+X08+fizQh7rngcsGk)hx^S`qDV&&N4YOGZ8>HpcDm>6kW67{* zrJD4_MFj3<8$F3~27C75%ObWNi#NNMk(lG!Hj}U$5N^r$;Ly|P2GFd9J6QhM^$2@> z;}3gY_^--BcfpOP=d{T%vP$+c0wc~Q|A($~S_}lhf^5>UZQEAIwr!go+qP}nwr$(C z&CTrWa>bvk6ON%QcJ<54mWKv>Gs&{4rE%AbzvfCJpVJ5x_k(${D zr(hSx2(0}8e#3c>Wjw-d->y+!l98iywC_Y%P|^|74#+#fHoSYILGub2X%m!sX|xv% z+-8BH#;k+`YG1#}E8gW_wqj%>39D>q2Yqh2&ZE<{yE zdj(E@vWfSumwb!&VQz4VCP@kcqg-?thvP(F5aKR`nKQR`o2D)Gl0q^5J=3~c(eJ!R zNy8P_sZcknwZ{*04dE3g8Cno@ zh1=uTL3nPX{?Ss?!!*DWWU!uCI{sRoYs{R~3EYc{v)JL87$9a#@v_IqnqHIMPj{Zcp4u>^fP#`#{1~YIQQ1{(NX> zfP>fYo*o2D7#Om~&_yeZzZZK=FAIqke%Y0RKp=#gzvF+zcjm9d<@lNYKZnO9+`KrWib64AWDb)sPX z_t=AL+C-e53C?GzRebrxLxt*xONHEzF2PcEMYR1gcm%*<8KVf%(PP>*Q(;o5a$`iq zXI^Zd+Bv8@{vNG}SjIyXKsoUp@aR(z@7u_5CTvD=daVRVg~0CIOGLvXcZ#|~O2UuW zQ2Pf6S3yG2`o9>v%2YsNI`c%twBREujWn_`H@)0u94tgNiVSj9ooSPQ%F zer~t2lW?U<>dKgbO3U+dU(7i?b*4t)-ut;EOpN=(QCNBJCw+Z=Ki^L=Sg(rDL)Den z=gI(*nsL>llTo~BA zI)hWIL>1L*))%N=^U@P_YBB@g&82J0x8W@dMb1u~&UAN}SBl#%SD>ow14ol_#!D^( zd=(QBJEuv_n?eIV`v_~K_eg=%jTN?`-i$Y$!$0Z#UGTV6;6u0ASsfQ0ADH`p>b|07 z!_Y#sOD$ZP!>HXJSCmv;VA=(UL{e66=5C-jb>}k z=4@D-RHD6N&%=sjM2`Ey#0kwXtK#Iji0yBhhs6951V>jb6IYWli&)Ir>loG1L{J%C z7r>{uE~lc_?Te)OdeZMF>Uo{Qms&$66>#5!GYqdwCc4)oa;vK(%@(+^Z@4c)isvM} zhKER@7fq*!!&NK!GrxR3P(uNIV0(Ec5*KV{y@@((6)S`Xo=v;hQE!7{fVbooC%K=z zgo06h_SpgsW&+`(=L;6Hh#cN{3NT9E&QcjvQpgpNT&&|gBx7!ITUi}u9zwg8`(}2B2@s5O%#2^o^??PJpux;|k8j0xqr*95SMf z7rbMK1k=Hodc7x}H3~v4zgMDM=~($_RKHY6^$6UCrL6}ktzoi$}J6&s(nyJcQ-;zTXA@7 zTO3`35MM@L5N9M`#HMgodSdu8fe-o+KhS9Kvi}-`I)1_D5E=9NML+_jgca;GJWasB z&TSNlemmY6*v8VhONYS1EJtCLNcLFUL13_Ste95YDP=S$?QeKptZcx}^ zcky*=*YRh|SE~vQeS{z-H$GqMgA^V$XD;W=vy#XK7S$vInSX3Y`_NLAfkAXGj!6@h zM%gw>VyyqI`9N=?A=CVf7zL;-7=#_3Tb)p`Ue~{lx{D2sqCPaKyj0a_M*s~R@B!Sv zHJnP|3UWN`jpB^80%WkQKN9zjw)2zXrAmTuL&Z0^dI2Hzx?cE8g)!yDv7x%cZG+{x zbT4Vqs_-fIblEBVniN7-=!2lBIT zP3q(84z);c@cR>?=^W~fgcfc3vkQ&t}cQpJ~q5a z7;LF}?Ky(@1l)8}CX>LGl7`=cd_UZhAwyhL z9XK0o@84)HxSg7oyq;Y8Bb=GnXYLVew@APsPPO=u>68y)ALx8YB|plnkqtz<+{vTG z@Twfpddx8_Q#kzr#AZUw1sHe^$u|H&jV%6;ZZno~2Hf{gd3>cID?v37-aQjXn%%j| zSvoI}zJ@y9hQrR$?X?dv;RIKLqj;3^6P@`@IRle-G!aDt>;4gw0&C#U=D<+(fxb)| zoEM|j_Qs9_&2*)wK`S*(#k z>9!z?V9>G=N3q3|I=7ex4pu^K=CtV8m1z@VW<26@50$+0=X3`k@nttRksn*5D*4P| zCEA4(1UJD~#`xh4rviEwODN64vEF4NXK;DF^@$6Mqkw0sqMY7q1 z59-EuzAX!3`~rYCj4HNPZh73s^EIad^+>YTLPdjq<=zDEG@3kp*fEdW4UMv*rEpuo ziOM+gZv*1R=hrLa{ntdp@ucGNBKyn(Z~eH$I>UN9JYR9M@XmI<0m~&#HNJ-tlAjnf z8Y4k-&p0G1;a8OqAl$ECzF4L&%FJK2d!(u%gQ0NuZMUEJ7{VHLQMlv^TQlN!H=#D2 z-u?MuBf!<3+40V9<(m8ys-lPUA<3(N)*a`xmkcvyq@;Ke5l1hRG=vyZo^7<2F5<>S z+Y@AZ_fADUJiB8>KY{IpukXF$rcp8j`VEqN%i|_XW#k~HMdX8q_PjF%)_j4v4+8US zJC8+jc7?Hmn`BNS#t>dLc9?Z=)gfdHA#M6|<~6I2M&O}khZ@+S`#+kw&G0yIIawtWfMvj{lb7b9{-n-xPj4jXB-*!f>9&i5b z1C9cLg&r~%V#K=BIFH0%qel!+K8ymnIiRhue!#*CMD!0RQMZ#3NAEr z`lBaFXh91}-WmGP#!NOZrSAMu2 zM;+X}Tnj-tB=b=|Qui)9w7>|i*TAonKCTxP3uNc0GBDqZZNhvDz-O2Kx_ckcBucgpd#x5QBwb+><$0W{_o0Dkj94nh6v)( z@^ZJ3v(-qv@kBC6wbf7^vbXQU?mxgVf-o3h`c^fCZf9QOlV5hxBeYtREG0rfT6Z?j zR)7yJYQ)d?1>VhP&M$EK*ECXEsLmM9D{VLDt`2v47~;hauXhn{JL}LVAMT=Ro@(4$ zo>H9a*jh~Vl5Z1aTV(UL-h!?8j~BJAXRhib&CRi`Z~Hp!oA}>aNhwjqA>K`^@{Xb9Cyoq9ptT>V7~Tx*_d7WAL|C`pWt+XSToo#Z;wn zMOU%wjVTny&HK(WJs&&!8qN^x>E*~~#q++Wc`Vt0cXU=Zqwab=4_)Wg$=FA=6|ee9 z@;iO=HWPZ7$37;*{1wHrc5Qvb<>pf-UlyfOsM%NVL=ICY1JA=>)cqEe6}AIpCsbec zJv!a}0Rxu+OSmWQZZNO1Ns}J{Wd_|C6I%-L&uGSUQwF9TpSy%uymhs{W9W2`b%*HB z_rZWubdZcc%15lU_fOYq?9Sh9;jn<6JgVhs zk6iMVEjUqmU!i)FV{#Z+zk$K>Bv5-T?0Y!bSkW*pxT74KB7PIC2V8joAe6FzfMGm+ z7Kw!SvZLV71%U@Sf=Z0Y`wvMLr1Qay&&MS}H3U!>uvr z!67#W^1ox1ptHZGThZE4U;P?1k)V_Sf z?&IrQMOXfLJh24}xam4_@qm~yL;TYN&Rh6y_Ph&jGl8>v0!3hQOV@=F0hx$Q?I#7$ z_L>NiK>)l4$>%6|9bYw3r3B|jbp04vAR2d@xmicmiEt|r8R`TyaNPDzo!jW5-gWl_ zbQe*Kl}UVKT?$En`yjNd7iJ7$XHsEV+3N$m5lsgCB7=yjzzq=$mfD?6vfYuSs@>U* zSa$=?C5HrLQk$n(Nq*De>>?w#AbZfA$4EkOaP^Qbljkod1;&WB8Oc_dT(-xq97}jH z7Z@RQ7K98bsLHGaSAzV3sdJ7rC?r65>}IKO?bLfht+6AP)wC4R{}>(~RsMPPd&rOX zO9otU2PSufo>OyEPwo+=L*j(de=r49?`fVjKOmV5ePf=bX8~vK&Sz+f-btSl9Jn> zbr;oG!#M)}buM&wIdTMul9I3|j8|ngq&XO?Fcw*eyfNfLBiH9NI-us@edreaw3w|weYBJhGzs)7nn0p#V`7W1w6SVcAH_u&yzoBznGmxRXB` z>>wA^=0(D$p%+Ui#4It~Q7KzH{WUxbHA}5sj$v8&QOg`sl-m*^%?)I(7_u=hvfwP0 z{BIpm%TZ}Ip(qvAimL!Ts{k9vO~HO+wUMhODH%-lzu1|_I3C|PJFo$P;~=FW5$xPe zBQ6TYFlK?2WX%{4X{Rt~`#-b(cP7h6IHmI8Uoy)`AK3ooRrnBK^(hD1_2JT-+IKj5v1 z+2a&`w!Mt&@ox)BC{bc95@Yh{GF{e6&9)cy0qv2^g=sYEx{G{hl(Ppevl);_y(a9< zmB4RD9`#46v=u$nAGV!_eRkReY+Aa znuWbh-3Wi|WwIrXk^EEtS+5{K(Whmhmq&jYsMCuBF;KCqHel5t74Fud=)moszc?%- zqI$Th_&c!rm2IA~66GAbU393gF<9dpUlT+;qU%tPif#}jPYIq`>S?qm&!c%*w>%k2 z0c9ZE?pLal${mP@%Z%+OR-Avssy)b@J6ys&D2h#tj_!=wrgo)g9kCP8QWW+V z6B2G{DhbB17aanJPA^eg%PFZM2~q*K#7qDWNiJtTh#3O32T)ht( ztGfj*Ux}$8I_cC1yz7ca>~y{ff-bgtmW$Teb$-RHn>ZzJFGOjy*99L(jK?NRP?zlG z(h&i4tk1Q8er{y}CfFBk2sRQ*Frxf8!Wh7T7J+x?FHIxrR-*sABZKVcoj9@g3Yaz4 z6)_;9Xz*Paq*h_S!)l3aZXi7*T&jml+_+k9bhY z?)TG?+;GFp0!>LLklXHcE68M=WTIdhI-#1|$=t`DviuPnx148kjT*7_%o&Y1gB zsUY{8$k+#<9^BAvQ5q=`kLT8?MRaY(`9$3zK}r_~x$fX#07s7A8GQ%|_mX~R5K1mcTQw!}%k_*Gd8ARw&lrn_)X)gX*+ z!u2(X<|EQsr7lr#8ZmluLN-@4TiKwxGsjOVoF=66zq7Yiih^*ji5j+7I=;lhY!o8$ zx+s6XJJR6nCtyB{16;zEtJ@%P=u(7qw|(Y@Ew5FXKBXj zJ=5NTr82m&IljOGn~nyr?*?liEMVcSp0edjyM)1C;MN0Z(!YTD2_YSw6S zcr}Ub%;)mpMB^aZ6Ke#V6(Kn;tt*yL@h3GRoEW-AKZt;!bS-vXB0o6g8us# zLy2VTk2_O6vD>KuLP-PU%lV5fveBG*JtnLI5I4-chG-D+E}d@twCiOK0393~YXZ^U zip3OsD8ncT8`W{KkJP_bIfkpE3&t@dyLI*?n<<&>9L7ganRAKqu~`;gqi+>&9pV;i zzrHhiDy_Jn7y+)rEs}*FDB7g(dVu~n96QvW< zhxu4KsOWex|3^t6^;bS%>Wih{JnUBgd}07mP78hE!U}*EpV73qoWF1VK-XG{Fu|ET zWT*>M2jTV|m@s5N-rAc{M}2?oTjp1I>P=Wrk<6YSl-u5rKX0Pi2l@6T1xHP5L9Mj{ zt~&F*-P)}5-*3$3SlxH>ahr0 zo8aLN56z=y#bjwSiFd9hJ&08iS56=b(}P?Gsmt*e9+jyE& z)PQbeW9V}LehwFl%!SjflOOk+H9RB`TBpR1_|WDJ&kiK`I{3eZr`q4b( zxHN&ca?Qlyk7?(}=2o=qU@^#Hcv4Ll*0U{G$LUV0BrGO> z;HXAjm+JPC)di>QT`MW`fuQ0He&1JYD#sA%cj;CQIdB!QX6AVhrG zvUb2L8B!(3&GJW4zK(Nd!MTcasRwI!PJhJY5ujzKG??7+?&%Nti|&EiPI7={n1e-&;_#t39YX3CHUp#FvS%bxn?;`HM`WvAoW=awfW>3u8Z$7_ zPR=emqy_7c`EI)y#^JK|6W8_N>e*XfT@AnSeNv24sji>bT4vZEosr2}i*#=_$b!af z_*t|_0S-~wZ0hu?+Bn-bF0R=&*pX7}3V=(k{flx8GNdw;9BjsS2Fg%v2(bl6rXC#`T57oq4kHk$R;V_hp@*)Uh{jtK!Y-#`!;qz+m z#7;lWRq{hNjmH??$QdBpP%x2b_~WEk#!2ivcgC>jB`j~f_|^K+XK%>LvjU;XOIA=! zhGL`Qqd<+GTZYAVWTB1w64Fm`l_ll}aPp{-{Xm?|MGt_|83RYEJHDT~F5vWi>HNtQ zY|klTX0thmNv3ps&0~ZC45DGY;)pf-RYifVDY=w#yJuQ2(pd(77}mSMT4fgVq=&R zqPVfN<^-qpNi|46B7~E||86hk&9<4p3GvVUA@Ws&LU1~$t=XROgy58I4Mnn8)skRQ zGa4z83IEd^D$9KDPhdb?C(nPaZ|JtI_u^AGS(sh$;F!5jW@*o(Pxv72OBoHH+^C)n zy{~g1E|mn~a^ah}4c&Ezoj}h>F+>in0(o^mT#v{LfuS2OKgghnc>-qp#dw0x2vPD- z82$1MY7%Cz-H<8PgI(z5L=<|^^i+zP#eni?xxxWUmc>L-ttSO+K8i8X_KtqhKs`m) zDV|%)O%jvgIi4#5NWUNvvCq(X79yw3-4aGe?SXKfm~Sn1T%=rJuy1~Ocw=B!Cq-hF zsv-dHj(+lK;^L7l#ZEIGtF~E{Yz3$YiOq053BAPib4f&DJ)X&pWlhzX9{7s z?I&skI z9F?q(Z00nXvvRy^KyLpywCz~Ou#*}7mvJT6s+((VX~0~QlgGwT?5SezU|NXHVSCOQ z2Za!ku3R9xT2P#r7WYun+-z8t$PRnXNBpeLwf;>E_<6W(<-*neH5qt60ohp0%y}PL zG8V#SPsFA$n~(|_!$a$oiMq(CDY-o!-j10 z6zjvfCVOz*vFQsrQ$ic7z&-UVM@|&BHr*P&1`^&8Jz5nyajz3LKQ&qr9`$Q|)3AYP zUUmUXR=*N;1=hOK!AE=rVNZ-JeLX~~E2;Ewr#Cq>UfDs6B?~z}Gb)|qMK|swM>15+ z6e_(}e&WHzxhF8&-&$5Q==~^#rtLS?WS5D}x6K8{bx&5x#@;7)8W;v`3(G+`$d3h@ zT$C<$y~Q40|5HFaH|uHTsh>KFax}TYuww7vSB9E3_%OPfP5#XGlO?)OINd+G(JYEn zaE%`TK?+IUa$WXO5z&_v-}Gr}N_B)Zdph5_d||Fpx<~cN!9W zl0pH+#K2;4)M(ue>^c&GR9Htxsy+lS)|C@>o5e$JJ-5M~*gGXXxFMMo9|p)Fx}}a( z^vF5>2$F)6N3vg19YZezMRmWwMYI_R+Z>#RF+1%57ER*$!8>LH#)@uL*OBv1@F_9ct5tYU8 zd&DcbFEpj?rd39(;fmS9^CxZU4pvjf7*!G#)2&fQC5q9k2}7zdBIlf^yw*-u@cCst z70jvfI9z1@a#npox?475^n&cA00;(Tj}8uZ(-HhAuMOvA`A`n(8H1ekU<;KIszNyk z?3}F3$Is(ylX167jwv=2OIvq*rHsmU4CU4Ttqz#Zxq_zlzj{B;PCgx8*Rk2hFF!Aq zDjt|^pXVeBpaZI)6DJn0ui2XRz* zwD1WVw;myo-{9v=-IO$5-vIP2CFFA*$v4+8jehUrIiPX!G&UuxZ(F08&&HgwvJh3yByK@)qLqfwZ53W9TO#s7RsX+M%BtRS->~)6b+~l&1T6`(VC=d1{TzgQgzZBWuOIj2 z7*;2=K}Y3*0FN2g-mP{~km+bha^k1kB1e6YZJJNJ0v=eFs_)`CMaGF5y%!CH!R_QQ zDg3C_@Qs%qz&4foU(+h|*R;xHM_I~PAArF|S^uAMMDxG=1@?yltn_9#bj?-0Ue+dn zSH0iXRMLN1Q`9K#dIJg`EJ!zzQYgIA9mGY1ME_whl=v6&f z9(e`lLZ<%XNjWL5ax`p1^<^X0zM|t#=3tjmDj*rVLtOMX~VVEz3ABXu&>Rhwr`v|EIz-E zq`@p>G=JgvRVzJ@ZF#;P;jEv)hU3Oy9YP7O{GIA9M$U8{CdMMD8^5H7KU&w{{uzSG zDN&kkYtp7>s>%bpd0V|TAeUe&dq4e}nypMNo=jvnzmy0KkXerrcwifT4^;613G$Uy<{#OJkS7b*TsP!>%SPW zj1{$T6l0J&^SSABZY`A*PdoXXb@NnDq3l`CNqUWac<+n#c1mYy_Fz&z^kgsc3TwYs zhh|0eUdGMr{H9@)z8sS8&6I&=ayF9Tes=y%Q&1qac73A?hSkuvYJA@Zplj~N6xl=} zug%mMn5_bnK?Xqm-6v?WeP!@HrdAoXUA|5H%oJ|Y+skW%4doIeBS;kCgib^Utrk{h zg3IHL)gUy3WIfFXhTV?ozxRkMwyM%V?jBhNXFQuuA>ambyhpp}_5`REw+O0?s=ohh z{J=LQ(&5m(Be6rUFnql{a&~2EDn6>cB1M20_4r*yd%1>l^AKDys$E@UMh*rK|&QqCE!K)&P6Aao)7>{8bh+OLhcQsI$F^(`N=1!iVfH4&Sf3}crWW=aBQbG z+71`vlV})H{mxOe$}^c_zbp&U{N}J8@ftLx)SMtSZUM)H5b!C}pwaUcEfm~BRbD4&C{Xmh&LQgSHBP|ye{u+?G4>$_6REFZ zH62XAWMiN=d(vm4$#9@_&DVpN*CjM!J~nom(TWHaFjZn;O^pZsfJIU=a-M7p^$w@} z6Qs%}kwiG76h^qJja5zU&lbl5r!IqtEXUaTXAVY9CFi-TDO6 z^tIToS;#}gBE&M9ecgAJ$X19oK5CPC915qE9QKE(OAKf;fmwh1Qtf3|=9w5S-LJC( zT^Pu^tft;L@aJVWaELmL9O3odi&W{lJsC$N@rqI3UDwATMMohzy=bmlO$7QqVt=-Q z9~nrFPA48Sg0!z`Tg%r&3URxMGm$u@JpbThY1b{t1O@`(AaQkPztWQWDd1lQ^I6pi zfAuYl;VM}Ds}AAwoCFH!Q<9VCF61b)Dv16IT4MRLL4n#Ib%ROvIrt7O1&(V@38(oF z^ufKAspxroK%Q0_{%W~ud3GlwYKBI%g3Lj+N{^)eIriqKlSZ-rEM#WnT3E>J)4t6o z!)~cg!ecKZ{`~92@pOE{iiF?@v3vKP^sl9o71?}ZPXY#Ch{HnQiLFp1$0pRoPC?iX zcWRfIQt|uS-bkV{uHVvROAntobBFcTd%Yz=*Lb*A-9>GVlu%XXf+$`OYy%BVwT+YFvzta2qY zUZdkkT)|LDi3h@YK}|*cQa;!3c9S$^?c?J@aWv6PnH)C(l04pYgX*(F8N5Gz5wmD$}cA1KOUYx6*=RCg5p ztUgQ>$qRd&$l|hr(61cWU+t@MfdGl|2m#N+ zG4K)uX+SviYS4L(B^w@Vs(V^2OAc+pS9wpG_%5iS&zb|o+vUhOq@XM69~m(!fR3r; z$tdm^AS!Zo-9?V{kbi-+<{j5GB2&Juv@k_;{lT~sf@A^1!~he;U1}SLP&%+aw@}yl z8<_GIiDjaoo8*Rx_BYhjWte&K1?D&g_j$glqlB=A`i#ZU2>|t)GG&?oKz?AF{HTR1 z6!vl!HVeZac7rsTg_>&#%2+E6CQL8u-2Yh^1s0#^j1<>91F5Z|!t~OA>1)71L&W5+ zy?EH$s&kvh*oEX3$q0)Igs>*|PigU*0vItcXd=QhhIuqfu%NIumJ@#vaU9u;CdK=b zUNevw&!teD2P&f99#t=jM1LwMxKfT9N-gEFl!+WZbfe&qO}p%!GhAt5>A`*SorE4LwDuD*P&tn+7M|)dYuJvX8LI?qsy+u^ zya}kwb;*|J+k{gQ9wP4(oX_~b@3|&ABphRP#hctq2+He5FfW1k4#|RXNh2X%_^sK* zo3*KXjSF-jf{bXDrs^}3YW%fV$+|WRb~rB7RL=lPfbsWW1f=xw_K6+kA4uc$4r>$6e4iP{nsr zFyt%bjii;Q>y#m_Hn#C<8;seTTDSusf&8ro6NtUSZIlWtDz(h}H?}NF%WXj7&V(lj zr2(S&vTiLk?;@qp6nie4ytCe!WOxrqK6TzYQU=w{5$!1{tPqfQg88|bH?_JB`g|jGqk$pal=5612%mteCX^)xbt($c=&x&@=j}El$8UGN% zWf`Lxb;822kp;@;h-6sT$f1{~k7C(5Daqj>I`tG$Qh>d69o$Npfva@rq4CI4tSK>_ z>}kix;|n56n%B`{45x>8KgE*ZieGkPVJFQ|dg#5;I`{}#L%_p&t||4JuGFD2Csvwg zBp5Wzwwon5z!Du&L5(^#=^#4I<_LJ$6awuoqLG%A3{!mZ-%~-1mne};hM+WvXlI0V zjvMt8PfSkO{zDiaj;J50yDapvHoQ-Uw=I)Vv!{HmP~aE<#O8{ml- zJ(qjM$t#Cj3?`YHLUz$ zk#psa2}Iup2EpR;pM27oom*^?L<!DqfvqvJQf+0o77z`I#nbKz*-k-b=yj%v zFr=pw zA5Bk4A^2aBa85ZCsu2(3n`h}g<&N{Qo6m9Qy!yi?KvX#Cl%GBK3o#AEpd8(dRbte& z|NdUdzKO}k$Vtb?a6XkG^@Lai+r6UD#P(GrOIf^Q`j}-;Mda@n6&ttwqZa6HWwfWc z<^jUq3Kbh}^G0kIj3Fa46)s{IBm7SqF#``4{CVzV-|B{|D&! zUB&&4NB)NhPab}>DsLdlP$+g^difLYkY|bxiFmK~k zZ<(&gyCLE7XwI+5UiB|aEGj!tSy5bIdj7SWSlam3&_!r7_y$X#E#+uJM*s4vN8Oz4w7rpPT)xl6~ZSb0$p}+ly{5tf%a*ONl-Dpi7%9MxP9B? zU(`#sQ0#|w;AO15Wgk9U;^Eign|c*gp=SzgqE740G78$ia&U8Mwv883y2^(R3*XoD zf=*iBkoyWYC6(VVOJ9N{M^X*KN%utVYVucLl*qN_G@bW*G@g^RZqw;(vMxmYGQWm* z_n7{nZ`)h5z=Okm930}NlkY-qI%z7aQlgq=0Q^EUhmwK)95n!}l*+j1JQ@*}jd13* z5=+#G)o($Uc15a|W=yvg$}2z6C|Z8B>m+gPyIea;e#k3;SD*vNT6v)9Wr4mQ1`e=` zKaVeesRLUi)=V%Q(g9rVP}6ZPx^1Fg93kbaCzP0i)5JFiaV^FVo1ZF&7{!32J7nfk zp*Vpnj0-KdCCGCTx|+ImKy0wK1{QiJnH!*Qj+JqZTm{=nm87;$RpT0?I&XN7jL6B6 zqly$E5PTO!WU-Zt3FYPwo6IOe?Pp|)BKr7oo##CuG&-+A^A>mzVBAD*L%q18pNdJ7 zjV>Xty#ztSSG!OvErU%~0e(GPW)uZjvhofL5F&m98Om6oodre87f+_k*_q!y|NPQa}3$~ycsL`4B&<}Hr1n*os>sQX?!$(vxF{tLHxkZ zQdY%dC(%BKtduFi_5(1j%^sn&e9aJDe#w5jtG;7#Y`rEkaj-T=ujx=;UNZ)f7?38X z-oe#ww)6P>8L`d{tMeDS%T7Y; zT3xVynIcMT;HSR2+?1pzq1nNOj?^hEMVxZ%jW2F0}L9Mv1PMJt1k zvgMs6)7(TDNECP5R|->^L)9@vboA``y*HCfIi#LM278Oxtf~lN&ZC=kkIz#!H4DFh z;5Vd^$zA;YD-$WtK10F78-v6KvjCzp<#W2F4I>GZr~A@^g3Y-j1iQY5nvB-x@Joxm z*t+($kXR~}wB)r>1(KQ3G>S&FYxYMCZiO5zAHtVv zvs}dU)hnwV3s+DP?Np3)=HkVz*PKVY$jyicW9iWjN7U>~?q+v^U<{PQ_)PJkLjatR z0}xo+mwO18{|{T|)E-#0WzpD1#kOtRwko!5+qP}nww+XL+wR=kkKOwhoNqs!xz}7{ zbcD^h4~N3AA7L_)3$BbUj7TK)XJjziZMcN}loeRT6|{Gq=}S#1)2Z2$$Atsn@|D4l z0?W^Q<8KKHp4%Zhn?yi9#~|8o2EiQ#>sVZO>RPkXiuKRvT*c$nw;j=Z*RK zYR{dQ_%yKQ!(sc4(K|@%zrX<~+W$_LTyyN+@N1!k%0}tQmmO}rBV}gEizJzBM*?F5 zS7u%7WELJ6B@wA9*yji-8jwGE;ELB#bXf8(*dS=8T|>apDT)O4DI-b+n$l^;qQ>vS zN%{LvEVG>7r*Qquk0PF(-0~h$CkleZdtuy1ZhQ}OLy6TNTqB8Cc{gU_k>|Z_VH<{Cm7e0Co1mPrTxd+m`O}mUZp@mAmeFfJ!^mJu7US9of9zdL*H1a@qWW2 zrxCMUxTNJmc%e=m@5U=Oh#VP;M$DRSkm`ERX=JPpFw8p-EPPyJ9Id%@Wl`Pdj^Uy+fphV8-`dEDD5aw&-8 zjKuyNUU)#BloE#?$A)>B34qTBAB&N92(M*2mRO)KDFc|trvZHBTc_iMICWUM;p&bW z(OBEDqmn2K%G0oG4MKbU$mDTYR`+ME%QBhrk<$&P1u=oqY!IM-8k;2xCzG(UAkxbi#i@%1h5^=>P36O zvkA1rP;i6^B~C!usBYm8rumkAVt-4{GP~hnkH*y{ zXo!5QZ#V5w{aagNYG%MH@Y7v?m1Hd4jzq|fkq2?(=MIV8)_{yIWo=UI1)dzEP7c&A zbEpF!C5>8Q16$cVSGqRN2&ye|!hz@!pX)Im(KD~3V?+B!;a=yTl=5L&#VvdSUhq~n z7}p2Lr6t~#t3S7RGK10uE^9HSZcTGrq4hmvi62_!8W-F%7>wYD>%v7B4f$JxP&0F9 zYuO!9yUp?>Lyx|fr1Tj_50_<~CPNED+&uk%*&Rg5MR*jTRFs6hD;rAEIv+xA3VjxH zLKvkIA>#d{rN)bnP$v4*41H+E>?@ z4YV=4mZCW%&6D@W>n56q_b@9LI62xPn-3uCY%8_wiLEHv;+BExZw}kob97MQV8om?+ z;j6D_jqy)LS4)w5acW*|^M{+tBAomHQ&5O$i2iRt^y>yh9pFYCylvV9!bZ&l`@eeP z|B&GSHz1ngR;@`8uQymthU}B1kUac1A&Rp~L=pIb@zcCzye*w+X{iBA|D{f}Vz%gZ zuI%{X$(;UAK*Sm4+Z*M56#5aXuX(v}R^j_Yx8M5JrGC>{vGsBnSz(!Jp>a8pyH6*? zU?o&vR{44YK^5tZ=XO_nM`X=PH}T%%bs0%(HM5_3bnn>?>3!F2q3WE5X1~~W^^Va3 zjbr_4s7uqkiguS-K*84FtF2t)49fFJN*ktya#XMde|AC0ec3VhGHm?Wy=`4ds-&mG zVjR7ERS}K#BWmDUz%f}nk?Oo*Wo8J08{yq}^K!SBeO^cXv+)hbP)~Z^T02Sm#c9>q zuoE$Cv#*;rwG^44x7sF#w(`O0^NiIs0)5h=?5o-GK`Xi5r%_pF`C;LQ0xO|77(zg~ zaWl{m;ZQSRX|K47+F85gjw>~L|D|(IK25n&{+V`%xBgH9N71=?X9)CbwRW7N zlg^$bG-(hbtkOsMYWzMm$)*7P$a0f#0wKc~RmAu8UveyH0|om>Z%}9a#WrGO%T=gB z_->zE$_Woh0Amt+KE$*lgk7|)1OJeh+BYMlD2`hsSyl->ne#_?>r$_ zPl@~Yg~@JU{%r(Qu>MrAeKVp9CSt6=T&V^jr8Pf(jGB$)(IHN0Wn3SxKDe8RIrDWj z7^$v+$dCw=nVVIoQ`ZJDt1v3)1#5WpUe!PIS*WN8We2oPWHE1p?O`8UKXj7d2Gf5a zsRg1W#RYA2r%xf4t)WMZx*oBMtDuz@$Yfg?>XoFmmZX4W zSXo>hzu@#aixGmcW&_-6yXi{T;&0!Cw!qs+dAQzb!ls3RaKf1;KC{_ky>x;rpPc20 z7%eqr6*td-W*4e$#7}3mJyQ~DlK8Sq&?{OKibc{B&4QB-&nu%Y8ofYbJ|NO$@LG)^ zx@`lAH_iZ^GW)#~*NL{1?OauCh$l(82?@r$EtS3vd@CscwFZ})O#DbO0(?p)EH@&b z<&>~FqWFGc1>gXk(0mOJXh5`{6Ot3!M9oY^Oqx7DIO6IsVZ36*&p`#glYm1dSyLXo zy6OB8|5t`(EnqZ%Bxw9`W0`a?agdJmsJ_-HYIAKn_FrmPP?x_890RTMinoDcik<}F zw^oV`3htvehQK+Pj|o&(!&m`Q$!8;!+El{Gmn3&MziKOPp9{+Z8}h;tTI&~;&QPdN z0x;vXsLy~7xeUsRsfP(fHn&ugXsQ*vVOYq!gsCQpNRI}dZm{)>NdanRad5{puh?Bm z1uxiyCRzcIdPxg7v&_Y;V1?x1-#&=?nrZzGN(zA3# zQw8*B*CA*A#iNNt_lnH0C|=1t2^o!{m>z^5Vr6Bei9cp$Q&>opCZ#>yu;j8l1;pIc zFWo!+`~DCsV&^!DRv`q`;JF9qosRo>3=$P3#s&%Jwi*a`j}ycoBDw)%Z(}8?_ZruR zm&|cgcRpa9905v-YF`2oNv;E7iks;Zd)Hy*Ip=S*hQP>Mzs^!fxB5~BbgP85sSH-g zBfXn&5V=Z%LMO{=q;$&l4rH{U5zv_8Wy^VS1Y-c$cx6{HXX`2)iBrnkN=rlP@%np_ z-_<`Ri)=QLvuLV+B?x1=0r35&chHLh|v zQCHO5pa`i8P3R*>#M?ultIsgX_%uzpDS7QBZ(eanvN<1ZLb^P6(X>5mx;l-3$ml`O zedT(Mwyw1XQhR?DwxODGe8LjN(@$g-lt9BTzAGi!^r6o6a@|Lu?f9soh6PBw4^jrc zi~Kst+0F^cR@W2`Xu9?kI)7}VdfxaA?*qs~4^lT$I;kE-k3rT zt9F867-YhE)Xy!&se$Of#RtPkyhf(f22ib^yW~CQYxzr zO}^8hR<={|1G!VI6B_AG6;a6nXSUnsuih4#*;cD{Rt}-=+!J_FF($#B>+Kw3+>_02F@vj2|z^ILRcH&%hp^ z>@OF?v3w(`R2Z3AfVzgF$Uvikp&VXfnRBnj6lcO9x0bZt$8xi77L2Oo3H=tr^|?%1szzdj{~m zG70}|4z~WhSHBq~y_W=BFn*OO#|=YlCetk=5Fr49x+FLzSW>*V`SXdhvVQk+V3Bwy zsGTjB1TRL%g|Nseh{yDKdmQ^a7&xDqRA?@XLhs~2@DiN zP{juiZ(lxmQK@UmDV_>BGQ7liC4G#4xog>m%-hm$+8FwCM9}|!p1r(AVD4W|4Nswh zDNf``|DV8!E=$YGi>}zE?|u3D;J|JB3Zq6)Q#10feub=YeG!8u;arT{Q@el>BVY2v z`LGbKV>MMxrZV_|`!rkDbrR!g0FTa|H)bFW z2Lw2WM{(n1yzU&=bss`$qN1w4?mXGOE>66eZlKv6%ZXq3g$qB08FdaNq|`V?>c{mzL)y^*37Ix~qk$ei$l5j#g$vB$}ErJVjmEMK6s zt#~57b(k4fg0l7<6~lUlfhzyCkfjuLe8WQ^+bh|;MtSj+9{cxK0z3D$k*zt!ejVv} zGEUwMhBt>zM{W3`pl0aOrx>Zb8-Au19H>#6GRMgAKwF)Fzsc}{A}O}Z)3T)J6|E%C zP=w*MTS|}>Qqt8$*N#4fpW1gcE|=kg_c@AWXN{gZT2Lb^n~YYeX9tDzbwBRi=yOLr zIglcM0@LtC$8~a_F<(p{sWo1*&%oIw8FXY4O`lT=aX7cDBkrYUG zZLJD5S60qtPm8jdUIy`@6qRua2&eLORK9HjX^6G%lNvejmgE!ByY>Nc3THRJnLG6Y zQqnAH1FjuMI(2+IQ0x5VU3sS=q5{$KS~jY2kKhw)w33xQT%_F!9?wTD&v3ZYoZ@bu zV&}lfx86qPuC-epfO9AvWbe?A8*4s_2$@N~%Lh*7D5tP%LV{$zU7y^1^1AjB@Omqs zANo|>Kx+Y6e6RS~pzeJs z@;2kOEmATx(91#5D3)Q%MQm*+JA&k;Y7{1jd<|d_1>0jRFV2jsF`H%bPo-!tvak8= zWb=J?L%H(P{nvg_(gcXQ$&OkX5L*q8jk^E;w|G+)|6@_(cmPI0i7mjw>`>uHtHq=P z;5GT}@vh5uGM5n>j?1{Ga8aeV-A#zEdojbRQZg~#{~|5p-#@HaKX&-OT||E>Z8E2) z)vY()ybu2(NgyafhR zWVZHOeU@9N(?TlFMw-@_iH_NuLM>8RKT=3*Q>Aho{uXy0`Q*hvj&^kv>Fact_uOP2 zv0?tDd;cPjWX6%1X4TJcs$BWdS-b(q6#1FYeU!zlOkIoa`ELH_`k^m(v+gi^yY%f} zyr^0-nzCUbb^N?cxQuhve5TqIq0vyivzXSas5U=yA<)%VOM9WCd8387c==xQyVLqj z0Xs~@Anm)oV(ad_vrtcGB!#zIS@AcviQ9II?VO3GcR3T}L|-y{#Had?H(yjo%Qs~0 znrr|Wmc}F7e?+nk_D0~*lKH#EnthJ6^{B&@ z^_@{>hYD~Lminp0GIu|%)nKRcgVbQ{D{9PSE!J&I?_49hEl|+P7moE6lPbwvB$t({ zKU!9u%Fi-QA(j}93GHX|G!?gT+!iJ$+F?=V60RQj{@?)Y#=I$76dMl~&!c74mn@=g zP;i!5$Uwo*Ma10!{;{JoGbV6R#Wp<@W8GlagP%is-dPA=ANl!r0BTBSjx4qkJzKfC zd$C`NH4s_$B7Fp3-*Yq3IcVMc~t0~|+L%k+GRu=fF= zD1^H|59asXHhjzNIPCV{5S;b7gctMA3zMk9L_?Obk0?n$9rEoaVN=4*(GXVT5sWO6 z!mbbdnCWRmNkrRLv8Q$~5AL~eJ=0?tDT4&H-x&iKyR_k4hMrk++dc-*5tjkKrgD3i z3X_*AWG(=F8+w%4yj-nWZIIUUVn9E0gP2WluL-ge-+0=OP!g4q8TmP59JLTz+npTN zB)z+INbrzD>rc<;S+Doz>S2&%YsiUM`5ciljM zj=v5-adeq+BY;#k1S+fIJ3fJeYlk6v}DVLJ}J5<5WnWzlF_yL|b$8`dXJGeLr1 zqO^bCuMLl&#U@MZ{J|fJE43jlGA5`9dI>i-0UD`$Sjg>r_vtb>4*(L__uH|ToHlEB z%WTOwm{3+4FM!3M!s_Jbhh@ps~Py`?I;7D(NDypNU^AvC}9I zxpa8Qc=y)zo}GBBY`kL}@)0wx?T9m(Odi7Vc?Hp8?yBWlj)jW*&Jd5Al_Z>v&xtG0v--df_LLk!cF9&^OodWs<0>jyo{0EHO^im1{}>E@G*` z0KVSyMJ&&N{c@E;NhuIyz9v7>9VTK<9<-*b&1Rfre594)h-olJpCI0R+?EExx-;vY z7ItK`Cq|gwGgl+QRByOOP1e-T1Zj&+k2bpzr5zr;sCrqA*coyn9gFLF=nu}2ivXE* zf^eUUXIGu%yiF&43!sURwhv|XrhKFK5h{Msf@koRz$3ZxH6iS%y;#uV)`zmqIkbyM zmBP)G-UVtktfLTURILwTqnfo;syEq!b=*D|dI(M1^KnwcJG9h#1CYwVAI9pAmf)&j zPop{Xx@YN<2O4{XyjGbLFc|zE12WbL&{1%J*xGacLMH+%&$_)iq^Z;uMLJY6RcXyX zMWwQQD)U-a>;53Vk4aQYo>E*c`U4PnmWu(b1f=qjOOX)L61VJ_6^s;i-iT~r+Z{UY zbN@Hub$v6-omA!3&RGRzmQwTWXm+Y0Ue486*xl{^bG93u#)4Kc0HHpeW1Y!nskKDQ zK8q*YLQfxkCOReF;nXW)s9aPon{tkS`z?`5fK+;<8#1Pl^OI>*w0#zNI1Ch-(K=ok zuqLbV7)ja|*jd=Mfg)Y%I-T+`IuMUcCu^q!7fEEmTRH>SPkQn@PK#7b>?jzRY0-gM zHrLc4)pcw}m*l&He6C`+hE&%shP&AP9ZwZPqpwbM^S`%-u-s1^PcM=Iz6 z($Q`PPeHbHnemEb>E1`bOyyR@-_nMj@RbdCk~`uz$8lo$_c2&s!}bc5)ZK8zc45HT zY^ifo=?#_ZL^w=}XoP|%tY26*9zKjmcbVQMY&sHPM*L~lNBd;32%Tegvet;Vng!ow z(HLKitZw8Qhru3+7ce__3`_DClx(3gGDzSFWZ&)glvW8n#vU?Z`l-R7DbCB@m!2*3 z*4IO|ncT4FwbMvp>Fcp4crtNG{<`_5X;p`C0nTG0Rw}h}{$dq(&`!Ilk^$agCVVj7 ztf`1$v}+$G@~FqAjpyLeF?vb;S}cl(+?uM;cQ$cj;RT*=AfzGB-(bSQZdN)2F9MdV zCd=RD>rqWiIKXs!IHnqnL9>=0-BB6iYKw4)TojSg%B%CKizTAQ;u8@X{K4vjeU1_51a2_tWpM;yR6k1=oWD>RY8>NDy&7C zVr=NxiR-Kk3`o@{ZPK5D1kW~6okP*re{_jcN1IS}jOt8_V@>{fZ4ycMp#^xQXh{wU zzaTN!{2T;5t^vXld>bV^>gU4QF=k>+xy!EQmyv<1@J@z&(a3(+5T=NKKP3ln2?XEN z;+Q_kwrvK&pJkzO9apw_?^}oG&C*=y8#_~^S5j;va8QxqdK6LuUEk{*X_@>Ynv^Kn zabe6I`E03Kw8@=Kd?UqayZME8Rknl#$<0t4Z6^mO2~LSIQo@u_>UbL}w(fwK_(v=r zAb)h97OALnyE-Ahb6fTTqUowY;9PHJJlX%)Cf!g+$gZlsbOitf_PC_`?i0eT^d{_I z4}DeeBiJ84P5H2ob%TWUo|>7&Ns*zaxt|=pe73O>g=gea@}=XiV#SVw_6`J&^rBd& z*~Si@txC~8%3hdZKG?CIF|5XFGrQ@03O^~vu`oXqkv-*cJwqGCC0@mVmJLYKUyX$S z;Wl&)CGW_6gTK)EMI|B@!;=GZTl9ojq+#-)Ik%RQEi zFw97I{@X4e`)B|%+{%iad7uWy_f0!xDsY5EW1G@25%*OULu}jwA zt;{>?CXu7ziECiPzOJu9pUa=kyn!GkAz_MN@7I+^`IkT!@p+CUSx1$@(?q?qMCW@C zEBc4QA-@|y!kdG_@L~gc^+wZtjc`*Q{>33g3TFB&dE;r3&YDoJD|37moXj>xP@uoH zwMu-iNGgTn$V#J=LYQgpP%HK}+@D{7M$jgCLAmAoa_F$dOkPF|WYQm~1X85GDZLi& z#Ek98#*;O&^{wdzT)Y10o88d=!SNIY*So36Gu2ixBEi5xNUQ1RJ8Kv|d*RS9UVi6W z|9z}Bvx~y!F7(qrtC71E?d2a##KoD2f>~;}arAB^dB9{2hkI!4_bX$QFb$ji#0NwB z2(69GeyFN3hGTU2iNFi$SC<4A9y7^~!eP#slVYO7 zZSnO72Wfwh9S|V-URjTdTbN63Uc3^d^#zp`w3R{%LX9xMp4Qmoec?!_S9a)DU>DUa zb7uT~Hm%glX9)$R29;!fRWCe_JKU^coIhNB? zS%DwDl9!ls%A$$=LWrj&vH6YQKGk_0C!5#S(15^VE~kDm7bj?9<9Zgnc8nZDjDe>g zT%P>`WSaj5$-{C$)IL_!!>?slY}A$iCrAEg28%55-yCT`&pNlJmqH5<+h$z|kN#gc zjUolfvb<`zrd1`MQ;+HWNJ=~jlN`PpZG8RjK4`ixA3OW&4d3@O@%JKIr-6p;M9FE* z!R+!{l1@EMCEgL`W23?L(jey$!xnp0=I@lWg(7q=D4#EO@Sv5D+ltFey=mG-$9BJ{ z!;-Ccw6?QgC!`|Aa+D8#OC8p^^DMFlQXdOkXM|w50V8#i)20qKQ>0r1b;EpJ@{yCNaonOk$fUT zEd!NLY)tRL{juP3KgHfo!`Ye}ZDx(_%KL#ch9)!9U^D6q9Uk??<|D(^pI&ye$7aAp zo`<(MR$;N=cRK}^Y(h-aV>NemkWCt#Z%8GVi)AO2>F5QKcE!x8Vot6=eR&-S3%AKj0U&0 z{l|tvkD}~G%AduCFQWr;ZRR_9wkC*Y&qyVqy&0p1Dev{_Jt50`(uPQLfkVoPl4u{9 zd5~Lb!~9~!=@e$$^V+&Nc!_EWZ=eoV2}~&Nj|Cd+z2I`jsFCK5?Rff3o7Z2 zi99rAJBcbHb5@>CC>E&W&Guvo^%}j~DQ*}M0S&2%I0#x6*VN;6$VGS#RTdDc;-}`| zmJ{mF?;iXx%cGV?HX#500-=iHO9Ka?ldQ>KJ~@Hhq=yB-wDmf3{Z7J)${L9G!jO)8 z+O}%z%P#8=v@pR5XWTr$A(8tRF$Z#t+caYN+Q2O|QwkH)Yw0zt9`xwCo0`dIf=At6 z10_AkbuYdK)_vSKc2K=31?w!~ak`@)t3Oxf$onPyIcbH$@UO!X(9ffsd`Phfv3BnL z?f8SDDO(W^+cALvMr&Oqx0I;qTONDYsH3zqHIT`xJ$bvU6yBa#A@8iP2Gyu_S}84t z$|MBl#QUU?9YNdngw)CH&PYeVBDR`eE>ZYkwirtiISrvT=SOq8!tu1}h4OCc73Q71 zQ1VSu`M3xdp!UF$N8`DYpUs4Ezyb=;-;gBcYZ$Bb8j?)~k*krofC4asZa4x1XtK=6 z$HW}$p$>%2s$>vXCI z2updU#u)KrHxHBdc7%7<;D?CL){2R$;9oO}tVh)&A)q4tWH{Z52z;(I=HirJ1qZN3 zV+W1c3G*`QbPz)mD4ViC$(8!CRua;%L9{;;+mq%+pUcRRTvjjBW^Pl^i>>6CK0K5_ zSr$ox^Y|Q9Y=|L`-VE1ykBEuO-m?{J6&9Sx1!~E8_%}(WDuO6&lI$`<8D$8r?`Z-7 z!)?6A-?Bp(M9TZ8iIh9r@=zM*WHkI0HZaC=$S_MaG*B>}R(Qykss4nw@X;%l22dV0 zs|uKIt$Vy@dmn3mCa7fwfUq__*1A1sq-|qMO0}de$ChyS;Bz+6pviY?=$6c`ZH{Rd zFWV+5)!R?ZFh&|cke!cA8zQpHPn|dY)BmD~M`$U+^kXDe5T-$p(WiK^5pvbC;YBu^ zo&vN!#NFx>m&YPp4otM4*y|KJ{w9RU;wwEuq}JIgU4tqbkd`S`kqATP*|bO8hGsC3 zWx(^a%2$jtr8Luqm?{pi50^g@bO&OtT4HNL5doyf0XrAxj$@|0Y|{SSg7VSn&=d1) z5mV)7uM0%gG4YAqfPjYmbk-hbZ_EhzRhR)aOVv$CWh)dh&AjYK?4OoS&8W_&Y@aIF zQ49N5$z#*_9957>%pzYEJ$deT8^jU>xMCK~6apyeLf%_Q7YCIfoZNn$D9PqkWM3@~ ziAsQ?WS8^K&Nz?B69fR<9ZmYU?@3^xxAm7FpJ!dk8~9L2u!W#1nZv;YmBBn~--)^S zU$H!^Zu= z7vUhpEWL>X<^=Lcfb=~6*sU*s3V)?JlqB;)&~kyH<(VqOkP{MWWR3tB6N8qfaC4)4 z@yTr#pH4EU{>iLcU3{Nz@3cmDCqo3)MG%l2k*I-o4n`YlZ3hOUow%0G2Gc_LO4mi3 zqD<`nBh!+S;n3bKB(A6tC6_ME9_sR7@AfA~(@kwGi`mfiZq41tb1zkt)Av zc}H~7v(|-cmM99`B}0_p5mOYwSl7or zqj%M&2`_$nKYkR?!NA&}w;vf=gy?K{$Sk4m9--}WB)jIz3H3T;kq}muEZD7Moy@^1 zYh1&;%@o0JMNZ?S>oWMB_33`pbWSz*5+ng;My#Wn@W%*8Ex?F(%1}0pib`xBR-TGS z(di>2wWL4bfi`J5L)oKRO2c^nXgMs2(^2-T`wJ<96=QcN2hJ;>J*#2f98`Ex{yn25 z(v+5}rp+&(yx?9l)=lV-cM;3vqF$7k6^)s6Us1O?eh!pWVNRt5wM313zwVI8CzNS( z3{R~mCnDWE;(2kK-Keb*{MPk9w1h;k#AS;VT1yMblDwJgJnmNNA4qtjsr;20q9^Pb z$8LB9#Q^bqTM~AhmoY5_))2f}i4`%3@v5~UI-FMOnYNM=Fm!D~+thbL zX$l+&$#uRJu*%tc#ePG){l?jQnSaKICS^s7L-DUh-gmbw25#6tv}AqPzE+!Z;N@4l zl7$o!6sbuIbmt=#td2J3o#gn?XU2|2m}1U=DmC=y>76S%=TqO-q1 z!LDID#eJI*jG$q9#9Og|vT`ZfqBeJY&)OTDOnvxXTTYpTP}!JNA_pQ>4a(h3m;!S0LB6+dJxt z2|h_@$wcH@tLc6O#cyp4#R32m{aWXMzZudlB#O)Pr0>@;ZN)Q8*v=Zq6!IAipE~E} zEfJKD?%9PoMLAU33U)P{GJtTN6u8QV59spBYmNX(S}EGzX`dRH43IePaI$eLHPKw#}m3f=lwSUDE{HQ5jk3`2?h$!CBissY>m1*1Rulm#O5CZ z9es9d9+QKP!{1!fIe8(I+d*sAND(Ddr!3cfSCQ zAwx&64kdIX6A(Wd?2fhw+qEp3G1R)K_D-}V#@^U2%}K3p@Ai!ad%FJ{=2WcsYiM-C zbXXG;Vf&)*z%%^~-Z#%1(+pN)O zh?K?mA^4L7ywo0M!qzdiXp|etvfx?sB=YSyAp#=|++XNaFj1A%Pa~gesQGTp9lw)^I5!)>4pM61e zXUw9divMPLHS0dI=2<8q{F>Rd3<&Y*U<$_s&_~)=HP6+38 zvWZ;Q(iYm_twQSuolvD=b5p2eERGmHn+36%K&nrTxLIS)-aXPvtztn7ut!XqXb_m# zi{2H*LqFx9?+vdA)>Y7f^1sO~EglecpBA+sxoYc|D|!m@f9tc>nrE7uW2hv$_m(iM z3@&y3k#<%v2QGYm4~97+TRL0S>f$__eN)IM42mZ<&Qkla42tFFv?n>8D8780obSKm zKh+^Tk@ml>hD?vcLT^kcnRaF$7N@BqXy4dEe4o&k3yW84DpzZ(vBS@VJ?l-*EI~cw zo$^DWclPZ#f@&}yi?1$Pu^ZOGp_R|PD1)u9FrLaX8fc*oUyYl?wyVa-y|8--KdZr4 z>Z`GQr@cQ@LmR1`Iye!DuXXFgzyV|TEWG&PDZ{yj&EFWY7Xvm`lQm_m`9t47)(4@L zEj?*7Z=vhJs1X8(rbh(gP)U9rirR*4mD2&vP++__mKvWa5jUESgO(`gnYd zOyCVmwoS&&I?BJ+jb0fUeUr;k6&*PLDKi5RnZ0QfztTd=d=0gErpsi~O>o6-2JF6L z_RM6{&gFK-{hFt0WH3P|qf#?8uUd5Vh^*T-Y(y0tHDF>c0xm4p2pab+yHb;JS=?ux z^&4m)ye6pL6+^scj;13I+iP&*S%2^qg7YW6tSg-D{hBSg>0|o`S})aB#Np%OfKiwR{~FqNlN(7T0xPK1bhg}}LWXfVf}1af zZjfag?}1^5~yZUeM`>AX^3*|Fp6<9)dT;z!kM4ZI=c_9-KUUoRR znS*KGrWP%tJ^({gPEa1gRFDZ0{>cKE{@V-H3#a1*GF-9vM?eBdEt8YF4^*DD32S(s zO|zGJ#N<+J&7-3z3|;cPM<`1*kf_I4lo~>-pv!wV8!3=9u^M81o2;%1KzL0U@=(Z& z$VT^1CHXur`)*(Q>JXrTm%}d{ZD`98${BjT4dw*1K)TQP5fEV5B0Mg#z!SFKap>N^ zZi2J0Tjj72#mO%;@>#YjU@_NVzvFeP2Tbq}971^h(5GHdB?Wr-LtkeW^R8L)sy$TU zxZtF)n=I};js6MMzrGK9C6~Ht0v5$S3{wg^s!gz8sJKas%jOlhn{jX-upcDN^Y%WY z-@!KIU-4fEiR-)#hQZ~TTAeM71zJZaE+GoXq$*-NTFlSQ zgjg%$3Uu?-bb{v6EgqzE4Qpvd*+cm&6G5^BQzcF?!>|WxLGRsg`vdn+CaZ6H z0D*}7>kvT&>P^#eqUMz=+^}$)D1V~^mBSMT>Ra?sOZuKP<2tE4^O=V29>yeaj5HCG zDqdOER0MoA0trD6p&(Afo-Ma=$CMLRUjWVHBP6GiR%u zCEkFAQP_+2a#-V^k@^%a=)#q#5{l{yu^#gwH`3FMcV2b>Gerg!Pbk)qJG^}=e#+o% zLNK@n#V`a3uiOOiWdq2{#vwsazGRP#VLv9xC}==D4w(4aX&AC;VE4h#Qty>QXbgXeOwsM4@8x9ABJw@lW#_U=h#n(|-sa{|Bd_KXBs##W z1VwSaeD$UoWg73Iu{acgW^;6;(=jnLQS!a*Cl@~T6X@uC$W*(seo2jB^AQ(NFzatg zsEZN}D+I&eFt*`zIa=_+&ZG%wcXM;`c~ab`lW&@8QB~hPvVE zv)%1y{6z;za5yXRG3$h8FgZ?$!tO-PkQ{mo%kr&B{s~4N{Q}qN`$>$kyV`DuWitp_ zf|76wxnJa*K^3+Msf~XVp!Tn*kXA(+)6ObwX|o1?C`?BDM=N5LR@+|K9Y<{LIJZ*ouELub z-A05amsO;DVId_Om`W-3=T#@6r3*Agdbai z^RK2nuAya_PW|z0j}~6c+HxX7E{f!v9RG}ycR99YLE@vEs#lyLA&VZg79F*QVWXHr4s#`Lo{;pIre!WO#d!&Y zRU^`N6>jWa_Kh4-#?W|;38GZ7;Utpv&I`?`IfVD#|~f#Nyu*cxo8bZ zG>i3QuxBt#oErS02tREFd)~;_L`EgTXKWbz62AUE9xQ4Z8FH${O$J*pDjnZFtCbFv zg`lMsjc^Mk%ij9Z9GmzveP9i~S*6oTQ%X1u-ixL!s8j6ch52glRC+$uH zg^4wjbrR`Keb4rg<0R%I~kIJC-qb|*qDzJ}Raqlo6^UXDf zf5;=HW#1L09TMeeWs3x&eOxWUfBo0G`y+`ug9}i)ZC6iv;KxmUb^-j47XwiNj2T%B zSCM9|{%z4(G2+cM+yGh>W5!YUUdeA5tpiz@Cuy4Nm1!!t%Pf|GNpG{%4o%J$TEAB%)nmY|l+pWlda!#xoJSO43b4$`($#GRpyv=Vl$NO9?RidS=q zAVEA%D`VK25i;Ww&a-SNcr=txqwdMWijukYOAKkCYmj+J5M${er8(XeqDnsZ!2eol zx4qUDVr~2>zFjz!Mh&S!Am(sB2zC(Ls5VRYNX&|c8idC)b)^wZ1@~i*zAbrB7G#AT zGePFNOV~&?Q!PQ8yNFn>C#3itNP&0p@;rvTPj%MgRtQxQbMZ_8-ha37UTWP|uth2@qVR>Cwt#kx{Xvs}RIm$tO+$lSjOlze*)v6%T> z&`XH4atZx95}a+BvGMEkV}lAaH{Wb^Eq?yXNCNmG{jErpT)voK@sXvi0QZsDMIxsE(D@ zk7HnM=NjE?43?^2=ig+mxw%F956Ey022O$q)C3Hy6>yI*7n!6_YBWVX~z4{PqD86p<^#;e5|iBVP>UpN^7{+J2(;t6=}nRF3J5P7m0&= z=Gawk(%6C&A)dDD>kFB~;|KBmee25$U*C&wcyH(513X!yqgi5K6~fL?VV|GlaP7~V z8dL7{opoo*aP4%{1J{=yKB{=U8=oKBk1akE>|R$w-|si!zgVNAy(wdlxhZ38FV;V; zyg#nU?(gu(9B-{iZ*Q(74mLikKY@2ETQ719xzIa5xybH6vBdWmc#?;p-6=me!ZkmP zJ5*#1_gcjFZ!>MDFOPU_KfMWKS9ob-d&Y#0h7TWd$mu^jNE%-!Nb7|s6&HVRW9eMV ze`><5KF@_`d~|((*rIH+#~OYAxqXibKmU{q)B0v#e0zp>eM=iXw@Mst*oqrn@!fg9 z%zifre}CeHbAJ!>ny!*~^GLQ9@QuCx+zGGvDiP-X^b4DOI~}}nA-liP5jTwOwCtaB8+FyJ z?(!Y!W{iFAB#fo%+V=FS5L*7265_pmJj;n2QD%m5y<;#Vxiil+yGgSlyU+A)ue~@Y zjBV(~kIlyZet(a!&mmI#rxeoq-N%ML;pk7)$7f;q=2vt0W?w!HPtBWoYvx$QkKA=M zFdVfduW4^UukZX~W83uCfN)RaiCf%Qe{THP=grBDZ%prxemHliXR`uf>0`UesMHI_ zUE{;=ozEb*>y;k--WoO6r*?bHtL^;YR`20Q@0i>7urTlKOKi^Flg`PG+~NPQ_3i&m zc=7+0A_?WvDwlObDpZo&ScSYx2uX4)l%!k|Hg-vJzbv7p*t$t_SyH*p=6*@6au3^V z7$&>e*k+fn&*%HY_m}T4=Px+t@jS2R>v`ET;nY@52fyk*Edkd{?Tg?orM9vJNuOGw znt>M^f8YOp{NH9qhJ|%{*9`Dg_c(Fg!jBkLBS!G7lM^H|i`-RYny>};NmwzHZA{$V zQWGJCDq~4gK}!*_$dxilPx{Za%ss>ga~79%dxCaTYjOFWK8=d zYTLhnf^KtqFGi0mcIDe3P1x&w8uQVJqQnU2I<8_VQyw81*XOpbzFIQCL1}Hag`6ItG+3!LSH7%-E#7s*1+I}KPUG|Rh zqWdUkwH0Mb9FCbz6=AX67#TQ|aoVP|rvl{nh++zTAdJv|a1R!S7|4X_yVSC4op7NX zNTbuV)WkK|PCG)XKO*r__vWoCN!`tJd=N3>I5o@EguW)`l!HOtcTHI#v0E^W4U=I5 z48A;Z^puv`-{w5BC4X*Fa9Q*@f%EuT^SnC&rq7IX;+&+TpaS-G+@Huh-ALAGP~(N^ zJh5reNV&@aAp6A{v7a)=?~<_Cuapo3Mi=m({r4G33+om>@>Y83De0Z%wge1kqOFEQ z({qku|B&r!vR~q+7d4W%z#3X=9BFhZ?m}=c>~18dSKt!Kq3zFYB3k9Diy+M&ICkeT zg2<3~MYffm8YH(m#%1J2&2rkih^KP>q`XcnnL7ItOXAEg_#E(HSC}Alr~dG2FHI{T zueUYgu=Y#yHoXA@G)WQGPb=j9%WfY?w7%^y5^_Lmw$_}bLf|SJ#5LLP;;_e%m~O?! z;=&O*skxdti}pqEvbzez{L z0}P&EJwwiz{{%fd6yP*+U&c(Y=y8mW@)WYvpgU?;A!oC(p48^Egdfe7+iZ-4A$wi& z+gYX)jSq=tc)}x@)-voFiG@JYccGO;@ty%cJ9@P6pJ;|C8vBXCZEClkF2=*8YHoSK zOaH-~s7uzm+6Ls5nsN|HwSML5@-)UQ$~9P7wd@Bo2>+!c{O9up+HWoT63xQ~dOXzt z`NP9bx{~gSrq1YTGkXSn#1DV4wz)C3P*USo#e~CEXqn>y0Gpc;P|!YfYk#gQ&l)Ww zO-9`E9pRo)Sx>;INhSWan2H^ioD?(n89!M3LIXz7zLn=_r=z=wf9lwEvxd2iFQKx4 z^h*P*ObDf(WZ9&{?#;J)he9sS|9jF!q!SsU1AC(4l?}enjxlg@uzy>mS=Z75Vn(^y#LM*55~fjh?FU=q zmI4Dof*Qpz=HA(&V|}Z6I)piyLJpAKc>-=s{ewn0Z^IQD!@`Sf#NtbcvOnX4uhpvc-|$r z{r=UjDaP;*Qx^yY-C=xpI`P6Zs-a#hij&#^*6x`{`?j_1#Bk&d2<*O-gz7)QyEJ7K z1la;ZFZ=_>!uGk!O4eGk0g&Pt>@LhVfb9Nq!;ovklOQeO627?GKoF${;LFG{9WADK zmrQyIp}jP|iD#hyB@vC9n@^_%w9Oo`g>LsfPBGG5KF9JHnJmxCDYw7cK3^dI`WWaf zsof~~K6pWC89JRB1J755TBf@CwqiZ9ylnTj76K4%X||tieNt>xs@76iIWvrW{8+ke zftCJ2R=zMyT}sYqjDXENmhGDPCr7lHngY|_B*tCg6Cdzacf=&EX$UW*P{=>%*3eHI zSs9%+fve}Ob3OQfHLWF|DHx2dz|Viu-GgETHb@mxRAFA=nc2`aiW+8WUa^ zm#O_;i`;!KIEl=Ypw*I^F3`zbJ&3qGr25fbwpB%YhWQTRk*;H~ko6db>~l4lF2v0p zY^5AXAgA><`W;x0aL)59jojah!?3f}*Dmq3%#lq|LVV&#-#lxt>rr`dfk6sSd@Sdx z+43%yUmZ~c8zYt&FTJZ8pHhg9HEbbPSiK>!GI+7&OU1q80P*sA8GNUg1}-e-mN)qY zx7-Mc4`E#8<+iVB8-E=_YvHF0u!{o9tC-9$qM&0^TYeaDtn4)jjzZ#TZV8}@HdO+9 z?HkmX{V9>8B^E2@6!WYy#)YQSNWv}y@74JQM#(xT#e>kqi{hGYSAD^c%DN*TeaOO443J#ts*WmbPPG#I zz6g-hHR1G{)IsxIORceM(i)3issaVVHXhy|)N~HcR#B)C2(z&K_J7@CQnvx!c?!M< zjxRKM<)yMH&1embH=`(6^hQ098Q$)3ctJ&MQO-0zq&Jk4GF?~ z3TmIH>Qcp)GUOZV8;D6dDgpHMaMcQpb^#W?NnudcZY8uiFGsqN4fT7;P?bdR6yj-+ z7Ly;dO{Z}utPyoCipZv9kAPXpF;|d?e1*KX$0>kzCrlbZ0Cy5`RG&Tlwt|F!fQFp7 zf_VFB^ z)AlQ&8+lDJM0Id?d;F8Nndu=fsa1RJUp8Gs;v-$l2_W;t0e*ioZ55i#*-wPBF|lO*p*=W#L7I5j#~wYeM{soPRR3b@4u^ zQ2J4v;QB=y@#c}2>=h?sw4LffA;(d2eMBLYYUCsN_Z88qwB|=O^+Rwsn2ayFwaVlL zw>v?=p8fc`D?qcLNsA>?gzLhqdQtcjWWIpoF8xHM1G%&6gwYhV9=*S1+#t5Rwa030 z`=-?C!U~Rahip5gu&mU&MQO;$lTJ>vLNVEKwNfb z%}8YXRK8f#^I7BEd%478Lq^v8ADY?~-lfFPxcMIz49KmQSeKc~jnd__L_2y#m6}kI zvI}R{{T)fcDaf2j(8doZGu#ua`Nzvtu)XWC9dBwNW6k9!PC)o8g+CefIf7~g1b#7 zTFgzdgVK*RQpou8G_y6CKDRBtPgTmLVQM*ai*oEsMxfe~@}yP!v>!s5?`;of!WdNTA-*6W4|m zb%H5+Crin9*hg=<-K8dA^X#t*J@mwisy$R6ot0y~s-aO2L&+whGq(h?3lup1)~701 z5_*sLt{k}e!2$*hr@mQjp|hu7M#XS`0_q594U@_8v?^y*kj2f0GFQ&^?j6INO!W}6TA4jA7g)D=FX}{8z{#$&%Y~rbzHBck0LwZ1XcZsEpzqu!- zq7DMU%EkC?>u>7GBZ)2_TrsZhH3cJD0ffnrbVu`_)~O-=(@pgjx=$5^Uw;M+bAO%z zzYdbz+siM&Lk)ob)j>Y^+e2d`>_s>6#4D2J#9pRC(QoEC|DKj~^OZ60+Oi*5bFt*k z1OqatOmvz8-;xS{Mw}O^9~`mX&3FgOQ&$D^eyuh!s3*8T^F;S=*dAQLoR_KjqsPLu zR?sYwz(ZLTL$!7g@DJMd2L*U8x|;UC0j(`8>k=>_Xl4JkW2A_RysU21nB^aQ%H|dbkkKH`V01TD~Zd#-)dsZeqpDzQ1kMgk z6Yo-sL{))!Xv_lSS$9LjoEBN*(9cx=1LF{?J=h>A?$X#lg9Uj!;|2&Wd~na^^J8qb z(8)m@JI47b_TtKNh<@_{6~ey9NE6=16!GeL@@RFOn{GGF$%8qk6C=y4J3`$U%8xtS z4Q(sv0yJIeKdcGe}%^z%@Up_?1mO?Ze$7Yr16_ zYN=yDk(4mFZio--ialV;i_JHgTRC46}noF z#$}k%)p6LRWqE`10x?vFA{THMljg}q+OW;W;)gU9RVeBhmUs-Q4@_K0}qr?e5% zP^Hd%Rc(A*JEBvPMJb)GKejg6MYvH{5JEf%i)hZP$Nx1YH_1j~(-$2n3bwePYaaff zcF_SM(k%qkfJ94yqw~uC1TOjt2>af+kQQRN#(S_7hok&@&o~NQbn?YyE|-q7ZZy5t zTtjmz?^d9fYdoey;HU<|_&33|<+g~}(FGhZ>!@-_7gy4CBo_|iZf3n=m+d1?Hg}jw z#ky;uY;*&qcuH9bT49GXkd8NpWI$To5s1s+{vm~s1>6?d^*yZXlDe+8c=o)aKVnU4 z4hMJHzI#lXDY&}GNHQ6kV_hpH|7K}pkEXw}juPK0e)CACGpG}~rZ7zVuweuI1`FL&kuMH}@%%uW7HS|jen_wS{a3to~zae%KzC8J~r86vg__%GN*BquUt3Fz`B zFB@DkDJ(HjUiO1ln`!`VCHY_}JvB}K2Z&-s3O24|L>hZ));w`3HoL>G;0L z%>2e=vA=wSiL^O@)s|8%8tx$3T}(ggAFpA6OY{(JWN{|`V_%a9RAUgiz@}yT3GnDb z*dkwX>D^K~!;qb4qYYVx=J};8=BBQCOFn~$#2mP)m|=}cYBl&FdS(e3Nw@vPF%d57 zl6^a*5BNWx5p8L=C|ssp1*Lu2oc89evE( zk~DbVJIXu=ZAq?eB`wW-bEs(iQM*oeDFs-ohnU6uIKZnU?-eEV)VVA?_llV>V8PQ_ zAQ;{NIbCbYch1ydEJn`|{19gADMS1dU9*(OrOINlobjm+;Wg~HpDs4>Hf?m3YLzzI zf1?oxv5W0pF9J$9veE)hI_{l#cnWwc7MaB;qeJh+_`C#P<)fgx9T9Zdd=fRD(2j7bE)Yg^)UR~Tkr@J_|kk+7K<=t_YJbt zFRckQ$yHb`>lp~C8c&x#=MSlh!}Y6dhYW;NKy|-@7mi&S!at*_5@(QJG!W1y&JozT z`64r!a)|2yfEV|dlD00j%m6fZ$q&}bFyW72I1<-*WQrU>Bn4EftS85JU=sxMzX?=> zc;^~dd07GgS1U%gn4qO(01>>9vfba)9qh}=b7Csga)0;OrfS8^KamA|d>7AkIf6-z zoYa9Y$}zH5wQ=(f#W^OkGt0y6vMF~kMvLZ!;tKOD%3D~^Ic9@1&cY}I3Y-ZjehW4x zyNKj(SrjM}b_&PigO!L0$8ou0x2@Lmjg##NYy8SMft1bh{oNJ4*!YI1zRp2p7DNqC zP5!FO`0c2MG9Rv$hji&!uka$O*nia}yg;xbIi_5y)vd~eV04gh4V>#rXi;ybCq=83 zVSpHjE9M-@g~FHx{^YRwmy zp{bhC=sO^K#{NRadMIWg6g!n{T~|}kr8`tT&jv8}#lU79h{5BrI*@=}q^4#i#K}C0 zmH4#-N(=h!mgH00dXPBPSqcWQ(6gp$wPDM|oe^H#;a66(=|m7US5DM~vIYAVULV2q zL_5IrJHW__uoyv*ANd#u4U`P4$yjXk7op8ZABO1RDhK~o^1)90;I3GQKfbk}(w3+Y zi#$QBU^DfqKN|rCAqj-D-o7frUU%6;O{OvTp8|5eYx#d3 znvF4aDOrGTUL{V6zkzC7Jx+?}{;*7c<1!h$M>)z7!d6w4>Lq(*sIV$=)xWJrehuF) z+9BCaT~z?X9(CQ4e0*j#HwkHoWLyUUr+ONNWT7<3BGAwFl%0yqU7<^cA`C%AAvD!dKJk0wsPdDTr38_NL_7M6VQ#wM=W z{4Jf;uHa#V=|qF&pP98ULBhniBB8&t^ysI9@{wOEv5+?=%Li3&Rhq+SM8oCLDLd6_ z++|xqN_kLSz*LAK=NbdsdrkIMrnp`L&FEuyeT}z9DR_;VB_SVH#H|dqIWYHl)34Ep zz(lXkWt#1M_*5aG|0TR}(LKeeCvioPLgUxSce>i$`vg9t zvM#uMnN0K57T0cMusBb`pk*6MS#;<_=&et-!ZeFKg$8cK)rmeqkRmCpJ>FbG3M5+l ziz@udz2%I^(0jgc(1M>1_lSwjr?3REk?II}F76G*rxhG9gEmCQCo<6-{K1I{ZcdMhUMP&@qHU8HO(BR~+Wit}lYJ}PKC!gdB889L|}(Y_QAWxQnqc7wEc?NkTT z2IjT%n%QT=Cc98}!K04}X11`yau z0BGobZ0YDXZIWTj8%2ECH7{nDWrCpn0#ag(S=&i+6d6{x)q)xrwJ~+$3`fhl#sXQX zU{O=7wP#gL<%(Z~t|Z$cV0*vnke8{1scKQ1&)POvx;-`T=%mY!a-wGa3Kw;cm-F;Ia$%3C}Ji_Gr@T-^gzx!9;H*v*Coxj%y_NZQ0L8Oqb?A~KfV%@&K6&8xBD?X z#2VTUraiF7t%|3I`6|;Y#<23Ez!}tDiLRh+y9ubhJ|~i);JVNw7tI+_R|sNiD#ZND z0^Ylu4$6$)%}I5HHtZJv0hx%UIXVW_;HyCBITE0H3D; zr&&UXbORERiVG3+m8|)-mb?+?Kad$sP?!;IgBZP>PVi!KB$Obu4;F5C`)r&Wf9-pYE9m$WFWp- z)mDKmX$B)a4}PN-=$qZTH1&YfQKBhDAvWWFwsX`|Gu*i%r$-~VsFHTj_eEi+?7PUq zD4zK{j)MO*oa7T_tZr$J!HWyO82>USm_pdvYde1iEwVgTBx%v$`A`%Rz9wrFGbrQ*7OpqR_G4|kpkGD4d1+qE;{Xoc2;t9&E4&8CCpt!9?k%EYh!00~5R8udJ5kO< zVn0G*Cv-FSoGuwt256XmOkANZ6g05ZmwY?x^C7e!tdrGCWfYEQBsQJX$SM{+_=p=C z1TSP{>P-}X?rldr&VfbiFftdG;Y=c;3l>Dgz=O&^k~Vd5s7>PIkG1}NY@1&F0(zP_ za8Ovc5v9uydnGcqj1OaK!RH$>ZJEA{5&xNAd>A~RKz zk4Q6GQm!Sg-LdAzNsgJmKVrlTLT532n`AXj@@FjlR~H(rVGnMCJeMW8sqvj~ z6fpQL@rP^5>_D3I1gtJ&nKTF2mxhR=wHZ1ZYEZ`V8`yZK37HPv43$obekn-S=Y((k zq8!-J6UHq(&t2QE<+rMssmba+Pf-FUB*`41)q5CQf312Jky1@q?IS4xMH6u!F!z`Y z7wXV(%(%F=)YU69CIq(TPCmNFs}1yb3F|4l%B%I~Tknlp0uV>*Wj<_mC{CS(tQpZk z-qAv02tU~SyQJz-D|dB6=@qS%dcue)yrR7)hY&5g5gOxKOc)Tyq#%n1UBR@K`lsDD zdF4E2cot(9Frvef7ZrY{g4nSfNHb$K;KZ?qkPM->tKZio^RU<#HKYSo2MK>f)1{*d z`@!f*h6>4>M)8C0w^LfpnFKN_Il%`5pEL@3ouxEXOinhG@;es zwfLf8V2yvhXH9o8$*MgbMDRglmjeBExbB{+)Peu2uCqwBjz+}jPz$H>nEX86g4AmpFbxJBG48SudcS98d0DlGbmYS_sA zj`|l1`S2p}2G{Sy%)f$jBadR@7w_V)>cZw!`;;?W@ z-lbq-^|4O)kldX!3q|lSwtff#OM7!$QU_odd}_PRrNI|fQzATKR?kyr9JE*HMVFS5 zBgo0IQuvKh1L1<4OE674G^X|XB<9VCU4?3mOczV(W|6qT1-9R5bdX4?v%alOm#1J$ zWoO18Qjlp?y8zY#uOdpxi(4q3jESC7IR5*iRdQ?LF<+~_5zfTV?{R4NR6Lid2r4=c zzNizB8l)$V)LrAw8>&mLArjKyKHTo0ilDlsGvp#onCgr~_0?y9{1b4+0(RCyQ3z;J3>ipUsihr4xaS)hwBv*+KJH0}Y$JpDZ;f=94 zc2_ZkkkZSCNH;GbUm&ka4XO9-B;${n8k}qr#6)h*m{vrxP}qx54b(}JvIHKsWYRgJ zPTlELnX=S27w59@>wCkZit%<%>Q1sSjROA^P-r-mC;K9pdZfD|=s$Hj=S%M^iweHl z7@*b^i|SZ=2lIYknGUYpkfH8tK!-)xE6JxpRPY@YV?4>9S_`|Gffi?L%_NG>4%_*LkM|D%O6&% z;Ae@2;S?O)dEy;OUJS{X<VUKu+>ZKs4&k^s2Ea0UTN++e{!0D8g79-c2q|L!;<9pk(0m3Ce;;z1np7X2f2t1b^QIve`9f75jg^xoGhUYNF2T8N`aGVCtW5S zZG?t|0S*WwF+G?bs24Fb$d9P<&}5j>!#bn?)4mam>9vqq4?9lNHIa+Fp9cN)`ADuP zqYr|8is;K`f-+D_5Fc78Wue>=-gvesOvSaavnll*nme+Sz@QFhA)R?cB%z%r@TNA) zFwt=w0^DLfOC9hi8Ysu|+9~ebYbW}dsmnJ(Cqr{SJ#!L`I7Sy8I8Y==gh*#mDf7Fvyi;Rb>&Wo$z1$DD9u2Fp6qL;!Z?Y zEbnGG@toK-sTL+)gz&aCWvTTOb05U#Y%2{Ch|9=<%vZPlvh6e%iZvK_rV_nGhif4Y zHp8sA&YdwPWT>OmKHwNnmn5M%c!~Z)Re5=IA=?uI)`c%6_7UF8R;i3o=!~+zO_#8P zW4jeL{@H!f0zED^Frx>el;N)P^Va>U*|u*XhUG!G-OxWma8U#$jCgw-i8^^qkS0K) z7}xkV2%3181Uu5aQ;N;vUlP9B8bg3DJF}Bvm<+;`$!ZF`%R_=ZWtdD z$=1gHZ~zt|UE9aJL0ukfD0>k?*b?s7ZkfF}nTg+ELYyX&nv3Tmz6>O-Vdo7fEvRsO zfSeRlvF1l79vzV_7NS|^bdW3LX(oJH=a=k;S+xxO)PYFS5KRU|9Ti1JX-y z*`$92wS0eOs7fTfB!#0G7baHE!Qjof?Q5)#DBYZ`muc+MJaTp@v>c+Vsd}W#^1-5adbIOUh#f5Kg9-9zf2vrd!H}OC zz(P6;8cKB-p}#5Jua7XY->75*RgNu? z6Pp!1^A=@n0-%ePA(T$ws+idY16O1YW|d#Gou~NkjXSz>cCKr2y7D_7{#Wk{j@;V| zM0ZA+1jGc&O#M&hz{l9UPKkcxNm~YAh0oJeP{+JHE^A@heyqnBZz^Ldw3==}XFX1oR{#N{Aff5R z2zYyGw=1`A{UUKj&KIE%sI2eN;-FJiG&-}+hGc5vi zRbTYvy|QqX3%^=1;6F7I1YOn_u|*ee(RJA*CwT8J0&>61m=CO?L({Ky%>tdIk-Dtn zRQ1A9))w4%DX|@8GWF|4F+k*5`7007_^@dkp_(zG2_J(_>j-I?w*K^qO?0t4*14Tw z)t{%YB?9=tc$={G#)r83Shx8HRS>nm0M|0d1Mz?XM8-21MH5I__%7$2JP?g!EhH@o zLL;7aN3ZJi_69^!3j!v(0hWRoeb>Gj;s^X%IHe2D15W2!%63!V#mKO9}IrPj)0`>Y{1|_*IyJ6 zIYJW_ite_VaBZ8Z6j(R(-^a6tS)?F}Z^g)Yoi0_CMS>@3nq5{|hdV!+hSlj9FJQQg zh}=ewBD(;l3u9(dJPOVkwj_59U1^FWMVLkzD_EP*VUEna(2<1XkZ3HWjQ+CG-gaU6 zihxxj^kXhF=#rP*#UC*FMCLYbs0l~x9vU^l8*YtOW8 zx4y!8X8PbDkTfO81J%roI78~*mC7MjQcOPZ^unip=UJVuL`G(tz_zHg^o3Z!RK3m7~L1Je=jG9*he_ah|+-;&Kw$)+; z|7N|^{lKFz6k=eXCt@k1CI?jc9rJ3o{aFS8hV0Ms{R%nuD*%7q4iEJ54_~Gva3cG0 zV6ntuLR(BwMaOayV4=I|v{WOI^qC|DvRKX{t|1J`L`4%X?nDfzR35d|zWfxt8k-KH zB{&Jhb8*%%=uZ-HkeNV=I>qVptVp6E*Ex>?14!0t%qR6-v{l`H|K;BApq2ArO@5y@ z9_fHTO|N(2yKkh}a7zu4CUfdJKUa_TCsRWV5d%XZeI3y`3*;&xTHpyL2kjyic1Hi9 zMe8u5hs#3PcW2gggg127!q=Hz!DI>A6(5=yurw6mVZqe_{f(D4##bkGh0n0OMA)vi zKu*E6-9OjG)WR?TQxh_AmeAKc@Kx;Q1*{TMyg?&-L6>#Qd&`ZMU4DmI_D`ass{(uB&>Z+i7t5|F0;d}Q!~|dkgHf$`|5dOQw|C>R$u7?%~~eSq{}rP6F55ALfh!z z>B%jUU;)brU*(9kCDYRkkaKaBG@G+xSgWpspr+M;%&+OBEuDoSnuQ8czY?)<=1(Ye z!5TB-84*i^Zoq%Lhw$S!2l3JcjDyL0pfv9~ z2%N3z|0^LIc%_4-tjpej{$o8-fc{a9%)^9;ZW)5a&8#>M>~TX>Ci(-vde;gAnMtt% zH)KovB3WB(dGsb&-)jv%fne$3Rx$`VLdi1QfPk(caz1FjarolI>g2$o` zO9MtDI#J$eH>3S|UgN`-sgRB%JQc@=$YrDcH%|c3@ptK34kNOss;di} zZA1yX=@^Wdu^i9Fe-)ofW;$};KJQKukUHw!d5CYzmQ84ywl65WAGNO3%D}45hgG8R z51R;_OLR-YN0UmHCg1DT6q*;!LYRTxGUJpjJWE%faqtU3+t#uO@wbhDDq4M1TYq)0 z?Mgrg;eOpiru5Y)PM1Zzf{pk|Y!1lbUP{pkr)itQ1m}psZKzl-Q>LKzvAX2WtN(T3 zckb5Y?ba+?L4RJ);r_0^?&Xo!T(|A~Kkv@QgYpwXxpg1UclC|ikQ+AdYu>O~<3{tw zt%qJWZ`yu}+AO!jeqYOGjVm`=w(Jgm-6F63klLcK@5#Q_t%ve&v??Bc|GIVCv9Hut zrQ_}U+P0q>ywRp?G5xwt1;nE!sepfNu-|cR`&s*)=d}~<)h-@uu~&BiZFJCZzjW3? z^Qv#lrd{55svR_aA5L`uZaisuv^(hO*>iht7bTwGd$*$HyjD1VqocNB_X!1^s~T>p zI&sw8RNaKlEsnrT0*=DI#H}Z{YP;`nPurim?^)V`%;SI34x-Mer60-@pShs--1qSX z{g?Mzdv+HKrh4SCkJK^_zkBMQVW_e1!L}o1r(XX!Qt|aq#!+0en$z|#Kc8jjd>stz zHQGvjmTBC`{*!4EBHncAc<{)C4qW!g zHXFG?KYRMy<;TgIfA6(9o99B7l`W>^06_$#nqS%+Ecq6*cH)!-Vj5{xp?f`^dG0=={xc+ znKqsEaGoE1_}kfH=KX+62755?oarH2o&$KJWCu{F+|og|N-MTvab&&8DmO@urHKyhFcfdd)xl_Q)$B z@z&XEeraaAyl)liIjV*z%Wb@V>3zxY^?Sv^|M?uN(^PkH8)EPBjp|A6@l||%W`keM zSd5q3#EwLO3N)Rwc~AVm4zB>@9RVM3BW8>K3&W=E=dayN(Mh=$m=QJoBQR?|X!nUM zv#S7WH9fAEbB;6T#kHqRYS)6FCDKQOv&Pt6z*pkrO}7gC0>|vpQ_0?k_X`8y$#rbP>BL<3wR4m6x%XxNNgw;K{Atw#$idTFtiDfvzpi?K`utV6`(mfh=A0YU z437}rteh2Ly+C~HntiorWUI2HZ`8`}+D(A2{f;j-_RPQ0Rwy>=@{PV|X+9eL%Rx?i)!eY9E^Swt)`seRAF2&i~1i%PX6s z)m;ADzfn`_HokM$@Bk&5mbFDIWsj}7@1+wnB~!<9?swlzHG@4XP8CD>Hv{(=8YrhV z&Ybj2^Vq3JOLN|!z9U_ab~`(LrWleFe=TNieN}!lnYO`D{v9pj_vgdzna2Kn+P34j zHRy>a`z@YtIyEd9b39G{cpxWL$GtBW;9laDW3_Zm3uS$7=p71tp!D7SGY39yJLPxA z{PI(|tx%0*J2mh3H%7CA-|v*%hz)lBFXMUev7y8Dhw{c9cAPyqgpa$JXK!gJr*7!Z zoz}Q~-uri|yL@U-{*}PTzI(Lv-DWbc`lq`lUF#h1c)q>zMqsAR_3$1&+uVo4w@ef7 z(Pq%9FSpE^?N7N-_~Pz3SlaZP!SGa7sO&&pa-9GHUU@Na{a65KK+(kfVvPfl?%;ySwClCck=I@ z=MJM+FJ{cyGy{TiY`4w(blV>ZjX3FYz6$xeIBocUO`hf7-+~Xy|1YL|0B8;ZG{wce zCGy&u1hxMUrVMO3Ci)*tA*iim3bdUwZ~Qvt$ill7Y2e!` z!g^25&kI|aC+?R4!aivDfAr4y(tp<@m;C#BdEoaSUcW2rR-To6HQb$@tis1{DUkck z|J#w5BDy%Lx4r^zzK1)VubiGKiUWR5d6=8y*3wz_^mF^SK_^9-+v!Em$FPgX;uUs< zEmqqE?mM4li%+@trRxaOrOHiEyI935>^gR6yY>UDm!#l@LAcwYp^7{GFWDM9NTN{H zufNsurgQ(e94-Gk_~6dgIF(ZdcfSp{p7MSV`k9=mJrc;g@%?*?C@b!~<#ciG;gr&U zaop^hO4!88s8K8obE={?8@^@~5C9V0aWl^{Xp_%4yeYh{yKvuNPHv{f{hBeq^Uh-6srAM~1o9$k-c1+RwS`~Evn>Z12V&c{Q^!!+t zHsh)zhxN``h>zdi;iz5NzLcZ>r~TPw*w*W(?{4eMpz8TncRbT*R{gNsFDt5Zy{R!D zrxn)~^@ex$L1@=h$==axy2$Mhy8Hh;7(4auHlOxdF%!f+bg1`?V~;*I>%otW_dk5G zI{I##YOn3rMQ(4IXdCtA@fz(KqpM|F*U}zi4%Jwf^;P_fIz@Rhp^YR{4`@bhAA(Wd zN;XWD_L&EzNB76O{Ll_+3qJYT&jA|!y#GqhA}PZw*bSbRxg~#EbV{(^m4wpPV<2?M}_sVYg*UW=C z$zPB4z8yNDq#eKaul|L+M(wv3iCaGP&g5iVQkyu}@UXx`y0aH`>FD3D1MK78auu#l z8)7Ql-s$rPn`Z90E7-ahdkr1A4%f<6z5kjzbY}9a=I~JDkAtbLr}vpYes)}RbvPq( zt)uHzutw^zW>lTo)ysY7jYd{4I4M<{$JpGsc~o^e#xo$v`K~vYtC;0?GZX%=Tf$^sJH0aBsJpZNLUWf87@~?_O zTm5Lw)6F3V?<^g^%Q!aw_a3;kE7^!7|G7^5A(zJ4aW*6AKh&e zFZIPy5YaE@Df~@2arN&p+ast)w%I~nJqhx(Xi7L92mwRx>$x9X*3E(BaYX048_w!u zevWDw`d>}oG?`*pal#RFr)g%^X(VK*$AOaaC;!eU4Qo5|>QYwpP+k8SYnAZ6-bzEc zDU_Z6WOXi}S^4%eSHg)I3oSp*lS`)_dp13}rN%r5xQaMc=<>Sh{S26`I5m@xmR#%h zq31<f(Mt58Ev4$>t;$L(&NX39}t-fRzZyo(FLFVi{~m;=S57VHr<5VUr7DA^_C`% z)}^w4POnalXH5>yPhz_gTyE4WG}l4ypb&zvn#y=*{E025+%guDJ<7h-qqD{dhRcFB&iDeJlI`HvBBm!r>(V(DA6 znleHclM4Skv${V~`#oH6E_;nUqBUXo<1#xF=xA(;-Jf_Kv22(9a}H1?QfxsSXaG`v zu2vgPnOrTp?l0vzHdz=Q4aAuaOA#f;Qa&r7wblM9rusnm4H8neRyeoocfI4iu#1N? zv^L0fZ#Bzabp97#Fu&t{{la0_N3XT^Z4{P%@6}NX6J%Vg-3Dn4-@|^i_0sDNSit)L zGuK1|AL`K4?`g8*pA%CPuyUFFGccR-@Euvysr9-lY+BzVeNkR`gI>i>=N&gSE^S~e z7Hl{+T6IM8>+7eq{O?Mw+mp^~c`ZK6ZIR!7AxY~Y7XPKa#kr$AX*O`66O;zlf$ey* z{}6g><0-<<@1NlNiU5jUGdphHLdER8G;s*;);e8yIk>-NqVsRE>7AjQpd9L`;(Dd% z+cKr(Vy8ElD~D^xJ+~{})EeCe+9NbfYCAM~?)r}!?VdGBf#wcwxs_Ht*2jjPYWGvVKi9yS_yvM^puAd>+c+~j z0zN%@Gq>Y|-M9R(R4)CdDqLW);SN!hUHb=* z2sjsR8BYfi8|FGp{FU$jY1G;6Guv=H<3tK5xX;GoNlDEGH%;Vx+u-h=w6>Ek-Hz}~ z$2U})f37atnLXARxMiSQx$%u-_TaUHTH|x|Cp*J4!m~VnlObl%?EgG&Jb9NX+H5o0 zUcFuk>Baw9dv>_w>0#qMI?mX%>!Uc`DNtN6w9T>0%8KpxOAECO3OQx{UBw~plE*Ch ziwWpuSX6Y@(4PHQs}0Vnw;7_-reP%L-NgwD5;Y^X3-9mMyG97fEJm26m2E;>;(Q!* zbv=o)P&e337#M-*uZW3altjl$p%W2`V{*TpIfW8RtP{D{Zz@|FCt9Hshc1^Ke_a}B zR}jf{DPzDdOeCZlVUUzFG4Yu=FsH5OpCh|4)m%y^idJoqfG!sSibl+{4tHt?mW3>$ zt-Q(l0mXCtYy(pgvGtK5K#Xg+kk_(-ih+vHJxi=j$@RRJoCa-q%>3-<<%#MKMli5Q zw?)_DfMd0#VJ zwNCq}`jB{!+9WA!2I<$kE5zzGB1yziYgD-7TFG z`wB+wLhHF9rF2C5CP^J2#mKJg{7>?6L0uq(yEzcm3wm(LpBi`&`m2&^cQdKPAI224 zU~D!=h?2llnmZZ5QqmMsI(A;^VFBVpS_86aU@&^VA!}Hq+VtK-NpJ@1%2MX`b7>+4 zMSY)cQbs1yV&g&^+g*gB#^0f7h8xi!hML8k`9y1KFBM$Uu2PT-zdv^c3kJYTH!towDh3K zSZ8}>61U!y53@!fc)VNf@XU|Say&cw&rd4AeT|o&Dn}{W|3_;4_cUH8Z#eu{rNm*N z^kAUW0caZcCznSv(fpfV)Bbfhvb$Ou7WQY4J zBf2<`W|u-_Er=Czury4XpveS+k*_=p%uM!wyvr#3(FaL5Zg}qk%l1-U#==OfVilEi zn5K^BvPAIPSG1 zJHjNzx4ON+yF%AcNh9;20ON#lA?%p4ZyJ85>Y2 zXUt~v!t$j#U(`Zt|PAH1@i>@}%e1Y+SjVk}smG5L`R z>kd1$DCRs%B?c&&6n3f4fn7uf zQ!gkS#jg&1_v)*2y0|LO$a)q+nANU<)E`RohgLZSJ+xy~2TaqH!LOn%Uy zrhv70i<8)NS^e0)3T)GOz9pR7F`_OL^a%?TE^^g!;i({8xk@5Zy<@m%CMU|e$`D(l zbyVQ53FO8K)rQ2}*m-??!WW4FG`l*mPkcjivlR+@>oCgqqDy*b6$>`G=+ri2;|1W- z#Gj=fx5CL!_>k(>rIm%;W!WP~80&7zq{TNzA@)wIf}PlWSZyDnH1W%dO7yj=*#)U7 zpILWV=woC)Atkq1xa|4XVtufB!Hy2Ni~zcPIxa`WNpeVoK|XW6)VL*OYO`n!04d#7 zo)T1POg*HbA-mY}WcmiOFQ|E;vCWOtjOayeWoSvYCS%`nTX1!);kv#L8=t$VnWwKP z*1o>}H%7ep%$k0+@hKazwVjwyJyT6+=**?H3%1(U~%Rlak_rF&dE#pNZy8`xUg z)k5k<;IXqJ7?U4YQtB3nrR&S6(#?=V=YGG*S|gcAoV!(gtlru>Evjr_!g4zxhc%0v zirQn`Y-#e{rF^m3*n@FW`KiV=@zR=>e9N=@v}mMDEqR zp}KxEQ~9iti6MqLSJV<;^g@$2s{}CsfiTOv+jWO@Kefx03+MMkrS8dcqi4_N&Bg16 z?$7;ns!nsoVhV=Lcl^9VmC%)MB&z(cN@PpuW_%M=PdTR^n!c1rQb$lJ0scR1@zun1 zKMoTEK$vYFjLlrP{?n2F0`!zi6fDiqkuGIH6Kw;%DRJyY#il_+m*RMmukS>lntn5~ zwJjza2V#4P5V$1!|1eVW2hbg+ah4EG?@k9*nzM&_KlRhFQ(;)5~F}_XV zKAt6g4Fu_-v&GUnD3xQJm`$}~s?auslT-9rWjaer3)d;vW1KWfeuOEL9;P&N7*MG- z1qL)bp~_N3TOTz>g{d~FdF+rHNj^}okreFx)UdBZ-nrnN9(bVH5a;b~G~V%&tCm${ zhFv8V@TV zOjGU`*I8>7yxc~wJ)VztpQ0)%;m*oQDTl{dKQJqINB5&6P>zZKk=r~^sZ)d#<+)&% zMZX{%3nR(YIZv7{CJ;i;Wn9C5DLc!LCFo;k zGK=$zs66|n#Mnx5y3BwM%P90inrn}eCt7YAHi06JUw4CR!(3aRgNdB;|s z(Gh0kqHHPrTNUM&o)$`(shQ>1_*R_gLn##=OI0?_Nw#MeDG|3zbumwh#?h(_9?j!* zSkQP|biDSJTwN+7^XQ=S40s4pZ*5N;-WYN}Ekh5>AbB)y!ofa}io)66hMTV-x z*eaMLnrjj|ERArhm%t&+i-IUEjp3K$rlgALDa6%2YtYsLv`K3ML5Dbn)K})R3F{Ht z1XT4}R}`~|0Wi%iZp9T=N)OGcYTYd17w-Ean8-=tRt=5n)aF*EoP`og&n@+r%SH+d ziH5@We(lmG%B~v&{aZ!w-LQ%#zM9LsG4`gV9=bZQPjxIKk0l@Cn(G?My?i#!Ciq{A zb}Jg2&?NTA<<@gCHEA<7IIdjH7V4;p8!-qed18}W$11f0#Hpy9|z7Qb)hMF1WC`Egz&Q>9N@5VaECVeR|5FRVO)mUh8K+tQG}6;)6*_aJ4f zt@%6cLI{!OS8Fahb+_y^7XV{aPHhJIU$?ITX!|87c9^xB`Xro*je0azL$y$vaU_cS zCnBxt!Q=bkidnlJK9$FJ5GYxc3w=nc5w?qm9t_wlMowM4D@s2vMR013aV+Y<7}chdzs>x4&YBaWmTK_rzc;&U9QcP`FUQkY^8OxxNNzl<|I zpO4vTTsWSh9NeM=|RgC;e99K=Gss%MLH`=xuH(1agt439_K@S z@=4t`D^Xo&*Ue_5mxnnlDP1@_FlHIBosop%&7PMKwx7wznJYVJpN3Q^vEco%9kNIr zDN{BxqlD5gmql%JjClu?>o`~HVqc6+bu2$g8tb*LZ+>x8%~E$c*C=ijYrnkf_S!fU zjCu{|k-AB%zuU{}ZguOsw%HC}?<&V)xA^^Y3IE^0FiImR|1a5Euu(p-P!_MUV}E0z zwEYjVVfVw@DTMz1wfl+`hSBPQp|hFgxX$;PZs`dI(D2N>XY@7j*#Sm`**oEe*vowZ zhT9JOb+ae+R1#9OQqt&ryw|1H%$V;{-E7%srmB78cY55lB#Bd9V_FMl;M}@(_3>9p zNvSAxwz7mCje%c#!?DIR-NfNJGLpjJgR>ovOKDUHxN zQeog6YXPkNPunQ1<9wPn!H&v;D>y?is8y3_FG2}!(&rVl=sw_Xa1VODLwXZPqHbIA z;pUf2-`2&tH21iPZ7h862VF1a1H5>S#PKPnv<9L4v;X*OHurNC_5;nInl^8NKUq!x zvmp>|I8|rUCuxeUITRDqrJH@{6J0+0Ms*Vaw$rpD_sq*7oi&uXtt@~`65goy;u(1L zq5h~e{>IVtlf+=Bv6cmI{>bjdjtZu|;}f?D7?V`H zP)@@FuQNnHB#zL}7b+?nl=qiMei86EtsnH41stydvOs5yKprc7$38V2*}&GiUhlJ@gO(2%)D@abYL~*ux!p z5fOLuCqZYCOS=azQtk5-`Sch}Up-lr#)H4(;pXcAu7c(t%x3;?a;DiM)DYXaH+gL( zCqU;R0-eBlI&pHbmSwn+k4)FTc||1%5hvcPChH>hfS}iZc}EsSmh}roI88N{cuMf! z3R&+Ng??U{&Y0O)t=0JF=ZShGtBLz@oRM19Ou| z>yd_@2r&E!E;$HG~P1tin2t2wXH^3_|bHDOcFg>=&wx# z;Z8EpgPIX}D3(FMPAY+A2?2|mDfs(PLXWfKIVzITnS$*W7P-nADolC@x-01X%E)uI zEIUh)Rn+Q7ImAX8$*63rjYRv)uT$Eklxaur!GY7szZTx!fMCPI1vBXiPbaRX-yu8j zY^?wn7MOO_->;)$N-1GAvI$G+Io9MawdY^WVON-8R4G;)i}UwFkG0P6qe!<2_%B8@ z6e{K}h>rr2dZ0$mzYd9cUOlWgl=-dDnrw0&DE(GVkxt(dym{cSMxXrRR#iwqyK;cI zT;V_F6K?c>e@$b#EA7csU`6Mh1yGq7hHI3omhsh$89GFMWD2N(W&~>4E}=mUK2u_q zh#^egtt#itsCFomYceB!x#6{tVh<84{Y34+9ITKSAmMdr7_6#=!N$I#CUamedH0Ui z^)?sQ7E9KHC*Pe2*b?G)TDItnJ2N94)4Ctbpd0oQZF3x@gFMz+MJFfSJr{gIm|3f$ zVfyRE3b6vO4v2MuM(TzFIxPhg(?q$EYqJV`1`gJpWJTlOub$d!(0m0=c34qqp%D^S z->@a~rB2|(zU4}Wg(r_7_otX*pb9jDrCQf5|73HIG&<(RGY`+ONxlX;12szU|TccS>dtwWAc6Cqy8v`nmJ-t1*FxMHBqL3mI4_mzXHkH@jz0SSYLi!|KFdjCDnP z`TfhznmH?beYwSp})^i+x4TBol> zVx{rE`RR2KTxk|PzB9l_jb6oxeQcLIRcahv-k&4&v;uSgI+2fU{JzapAtXP;5m|W0tLd zl9#pt$BqDnLVh=l7wmQPc-_U>+&zI9)Citmso&A!;;R4?6O<MIMN{A=R5vMiv&SZi1R^Qd~LWTNJ@#r@d9>bT4{HIR!i6^(LEWj#8Sp z%BM5gW%*-f-(Fxrp|2{=bSx*0<6(C^&Zk5vd7Ug3BS4qa>lOLs)F%WE#9toO1P#lPw7Q=|3vWnnZ^_w70 z5zhJu1!^(Rnrqk6P2auxXG9_XXZ6#EuRCmcaZOyAzYkxffM=j^#43QxA zu%p52hZ(KUIwWxaG-CNw&Dl=lKzj}0Y8F!rM3?LtW2wZsi!|~zn#J3DT05jGjG7|# zb!VD`2nhfh%swk3t!HLT(IK(CLl*w2AX8P7zjDFps}!RX(rkHLXv;^BdE<=>^D1HU zH}67e`H8S)3Gu|RLX97(P+d_QlG2|Yjzk1 zH{4n)fHorx+k#%VJklqAfOW~>md)0qsD-^fYG(4);< zPXU2x{LCD*yCKk~W$~xxBE|7>jmeu;uy%Ga9`Q>%S%&OPUbC#E1ZTRvn0*un^XZqY zY{hg=$E)tI??^1S5|LEl?%9fcPKOfZzQVq##wvLw3=MhbX8nE}rX0b*b(S4=Shsq3 zZ5625GU?7FGj1f`%>3L+P#2b6xT6E=VmFGyN|pVSVm0-!8*{NsS}>F2eS79_agrYG zj5h71Ub#r$BB`#F*sh|29_l~daHN;k9?U6p43O1mmy^UcM@X7-CrHxu*ajA;rd(-- ztxJd?0MAvKJGpx?K?hb;j`a+$e|Owf|MV6(Iyrc+|5*VyYZxLyg{-fxma*hmvlc~- zeB?el46m9c!{p#hf#}>&`0vRXt!^_w!J3^Z>oI6jy z)=Yh9!lsPll9)RUZoKlhRk$NMat|fwY{aQQ)5gFPBGP4glXOMYj^eW|MWgDQ4sMcG zMPw+cR2zrOFUHWbS^ZxVi(=5$$T!kCg$o(kaTO>Stla<;k4-#;wO) z6#d=(=V7jP<5&}VSgZrtzX3f|^Jzyhj``V0hstQsE~~u9;NM#NK5!vr7~mk+NM>%O zyW)}|trH=>1yDJMb_&d16@i)+1FS0Ti^e~Kxut^d@RhS$65`1 z%g*O!7d~Ci&UdD|5{}9c&LA(o0sB!=Jn+r0*qd2Syz0ox-%j2ZjZkJf)ig(jBgW1$ z1AsS;Qorhrn-?)f#l)YFNflUk^f-94Zg#vQ_8d&13C~cw9B*MrE8BXZJ%38Z-7{jo zWqzpwo|MKOi9K9M{vc6V$)-`_N4%g}@iSv{X(fl-p#%%xi#eD|nLEA4$PywA+bEyL zV>PjPZ|AE{fytVWBqq^JsK(^OP}5kv`Z03bfN{4vh@#L_4AiF#a=4OxL|yPQ%=} z_Frn~*d6+HnsI>by_fFM?UMSas$^(yN*{Vs*IY#}C85ov*NdJmtizJz5Kiz@cbe^U z*4|mdmeLA^h7@fnET`BNu>S!jBu|khOP)xr*;IE6P}@qFq2Cz2g8UkT&6)x;kbn z>n@tLj5-cmmw}a4lZ8%jNBYfF0k35tW%?~zVGAMu#Ux&1)07Z7#Syt4!z{KQkml5+ zC@dO;tcCe)s?sO|Sw;8v^a+MSqzBx}(L6U19XE7yk+MIrs8}IyIR$$nz$SR0lH3hP z@y|~wWgUIwz(AkadaDeU8RE;tZ;8D-Z<* z{xL>}2Eo4+f1@Av!!1FRl?3$R?|_cFmT})lqdwfixBD4gm#d}E;@hdL<7J-91XP$F zZ7Jqp1Pa{pz#T7V{{UNDp77OP=87ee;7W(87z@^pO52Ee^vO zIyfz}{sw`I?b{y-31PiOb7e7$4B;*O2|KVljHWo95KS5ut3pkr5oTb2Zzb4=!Yxv> za9!I>VAQxc-0&Wf0DLC49lLIs@bZ58!SHTe>JmpiJ4kyT&PSkmOB5`TMWwpu$Cwl} zVrY?pUPBdlKCnd3)xkwhz`y^(U0H%*bx(|Ndwl6D91{V*fA#b>{MaZ1jvoJD9BE#C z#btO7EJ$OQ%!(Vj2J|_vobE-LDh~20F1p%jiTXhzWWwRoD-Xhr8d?k$3!sw5=Wv*+`Q|D5O=SZ4OxioU*VxLI}B%r=d;pefIrcjQ}c91)HfDP}8-d zMbRD>G;(ImF%5F}h#s4lU#3-EowDCxMTso7Z|a`8Wo#m1b?<&T9bd0Q%|N3-s%*c+ z!pb_TzXAkl2g$NXnHWu}XPTXiLa=vA^k=jI#M8V{pA!AJY42Pe?Bn{K(3eLwEc2 z1__HaGs#dK)HhfI|J{)pK4I5k#FR}^kBgO6KEQ23r?p3Vs1C;&X^@Fi%f*kGeDW2D zniWUdn4Cx20^ED#6|(PEirV%MNv*kNAW7X3Fi*r&FuFrY0}Fx)H50rWWxgTTt&GKc z77r1JysHUel?V8k6lfy4G9#8U=5w|WspvR#aJDO2!KtKYJIsL)e|%6i-WRE)JL#uL zi!o3-y#L$yk(}4!Vkf|#Ni7_AGm>R?;E$PBPRvdIl6^uz-6&xzUOoQqdnfje9cr#I z^31QeJyrdz{_~IBn+q2dYFix9UgZ-Qj#UqG9&D)FK2d;EZ?IvEvfnC#c$qO z@i>C{pM-S-%r{@kA5a9r{}MC@8)X*@<@z`v`rqSJBKE4>y@=*GbCGe^)iUn7VgKFNo!M6P)^2@$D<$HYi)z=heMZagl@_K z>4r!zAlFCDVW%j2%i}IJ%zW>753j$+=^3^*0i!uBm!(bQo?H2`PbxyzP zeYiM&|AaTO@@_lLq@d9uCi-1?RvYcmAooyf^e(XCrA()JhRgKPhRMVq!zB-3Vh*X> z!81nb47Z{Pn?wTCOH!PL8Aqg{V2nF4{Y3FS2-;OR*q6%+as;np^o^i*qVkEC+an*!MdRQK z<{jwnZgR_z0N(tLNktiu>`HtgVxw}hn@FNs(Nh!pt&6dNHH}xf6^|{3Nk`QdlmI^L z*<4UjCVYMP2fLXyv3|2Bu*a+%HGOM38!bEIn#*w;P&*}k?0rI`#qvM+UJ}$qB&NXt z8INm*PV*Y~qq059W*pk1cxk5=T@I#7i`QcyyCPbvEs(%D1q513h$|hq2b=2PqkX5p z8wSqZA^z2d6yzPXvOpmTxtq-7s;s-6EI52=UUf@Xt0vi5iTjve{f)f_nivU>s5gE_ z@rHp}VluW7DynE^7fIfK0{TY$Z^PI$n;}9VJa49_NG8NR$OKz~N|4H6DQE^2Pl?2xvxAD0k=8CK-rTHqbd`siOil$0F$0w)c*|1&}p+j&Aq;vuD4})O6!(1Inc#jP z$r3&2xfPt|+<;l%&ei-euw)Qlkk`n{cgg6_V^;J9W2nB`3e4&6bL;RA9Tr6e7+ifp zQHu29kv-zgywa5Pov|vx?%ty8Q4Z`;&PH6IH6CM@YCKt%MDtE*DrXn`LflGfOqexl z22b9wYq}9%XyeRxr7kNU(KwwsbZ(+3>t6vH;>X1t&Y>=olS7GHdpfX8&9k19Y)rbj zEam!evtTUaepFPTW4@DD+{P;e=5|0B*m|2kbGs21&llrX5@~3Nhdle9lO=~xz#WPh zo4@<~D~w~vRPzP7*bG7%y`pb&m|XuxXad@oGBy_!Ge!Id_!u=`tMydJy_JggITvKHEUH4e{{84P}{|s#7@CpzRp%P%+x^ ze^;Z-PZB>CqyOBzh6(w;itcx53a0uFT)1ia%M7=p_|g zMLt>+nBQ4E2BMW02Na4TlML+}T29zJAnZ+(H|(G3MZ1Znvr2|`dV7++%Fkb~spgex z-1E2egLv31SmQS`l2%WlM5pkOZ_~UuAkiv87o%ZkT+x%wEIv$4>j_siR8xqohP)>x z*rV>VG|CnonMcgi6O=x;hfOp+w@TEo4rv~wv~K_UT1orDGf-xEb)QYOi-2HxCBi2A z#%;+N1Lg{}L6U70yB}>#eOCO+m2(te^@@>#nh_xBS8uyC#a;a3Y53gJ|B&tP^Xs@2 zb{QSd0S$AK&Tx|G6gJ0{GU6u@JF_AxT>S!$ucGQD=B%9E>>UB3xvJQNT`x%0>r(yE zam`JVoBKq#X#|eSMh;2j{qF0)@kfPi{Eao>533}YM7?vnmu+&^3EGc=MU`Om?Zd&Z z6f^8FVa#$;7SGtFQO~5IL|`Kw5#f${P|MK)MScl48-2^kpD#EY+OZ|s|B`eY6QvFl zWizL)=noc3!T<1Pxd*(Jkp7X>&RV}&D=pJyIR3OMuaZcrqH=*KD!-WnapS4_J_)O3 zqk}(jRrO!X0@0MssS*4pnWm&5p9H*^1HCTM9!+ZP$JprB5z_g^SiQ}Sa~@GveEHFv?|qHV*f+fdZmrUuNW z$O-B%&>46}L9RQgWiJn>b}%zNH;wD245#e8pYBmS(_Nrb8?gWKm3g}>0s1VtT#j^h z;|qImTxN#s5oTzV4S6Bk4LlW6NAR9IUz>S{6Tgb~&?39-S}u(pc=fQMXom<_HwCxL zq)+!Ly1y~KJNjT#31xB&bI+E6K&Mef(hEL9ckwE`EF|Z#7n0DjBxNxcWJMmLCthC? zg_(%v-9=c)s)uR<)EWA2aN0Kv_|6N7J+>Ab9Q$I%X+fFBLP8xK}y*%rsrcL+&L5lT1ej<1Vqy0p(Pb>V>Gj9JSg)XAJhtW)J z9@07F5=%ox`}S-hs!OFtO1uay_NqhWm4d`GCAz)LC;hn#K5VC%yN69ZpprRjpLK#i z_#zyiYch!1FXkwGh|A=)eR(@ZnA&{pV12mLF-EA*2fD7NuSS&fo=u}fJb>99Us1J0 zMd)!9e^NdcV|9qS`?w47uT{DWJfv1xlr{yc7qQJ#Z8RETO4zeD|JpsxY%=tS{ge%@ zn<#c$54-#QwAjFI0L#18c>tfk^|d$PCp3qX&i-PWBNw?8RlfvV#=b2RGUWAJW0eNC>=@+WA(vObr=)A&m>l{y<09 zmv(+HKMpp6s3$b8ZDF+sEW<=WMvt00O^Z%gKeAVX;1&IWtYf+ATN1y-mWqNZXiBk` zRR-$r;jRWXGX*2{-}siQelpmMA(zGktbJ)4Lo!M7*Gw$u622#gJR{IA#GnmOC*yI; znKxvE&2fd?`D^vKk;u6RGJ5h{8@V|g>h6R_-MK{w3D5VD8I(GMN_1m$UP)IJ^Pv2tB9lBQ3$t#1Chz_%PIx&cvCuwUf3H$~;MY>=oc%L3?u zUZnyjyo<6aj!4EHAE_O4650fWdNpB)xrMLQ2$7M(dsJVl(@gzp8*FBLdO9IMlwGUc zT-?=B+o1yzb}c9pk7R*6f=H<4LsXe$P-iXBE7dGUyj z6$BN=c(lkrT34FK#JCKPlLcpj1BVFa4g4uq`=TunKWV8ExJL0A)LgrunnsTjSC+f;2 zIcQ?s?LkTaaDO)_G`!YuF(86aCAVVGo>>rC4{lv;NpEYBafowDf8o>*AO{Y{hOIl;w0{ zK=G6-7{s>^ch&v2`Y$t70l@j7f5HdCjz*)=&Pu%bt-B{$r*1quRfc;Fre{&Juejkf zUqK7!Dpi{6aUnQ|)R*34XUbIQ?&rMKt6p*L&K17$L*0UdAh)Mt3I&p;$!|@a(3yzS zmEWF*tj^;X-uZ+SkJC3e!aW7U>upm=Q`mRUx9yjotiwIiScJ_Sy}XvZ5WCZ)3d&i* zDZh>WFkwHt10sb`chbr9U4MVXBSDMK(qLI;u=8}t(PW~lvZYQ#U*+{~cH(hgup@-B zq@zpP^#5VgE&nt!5dTzDmP`_*zwrFJ&3knIAQjRv>kh z+R)OsWjkahuQI%)Yv)LdVC0tQ6-?CX>Gg0Gf3R?`LZ7DfcX}R(Kxu>1{|z zNa$5cjhN{Hi4#3yt00QyFB*tYd5YK7YmI$^!5)+4GQmqG=#=5U$3Qj%rzbvCSr=JB zEsV251B_Nw<)%du(n^jV!<;4zaND(Q(N4~2iZn?u54LVC{=kp=McZa~HFKJj7AU$sC z&ALq6=2PVK!uOO6LZ=&TofT*qec{48i{ony0+I}R2}^2E!#wOShfL1@jdVzelN8R| zBFGkHl70yWxm4(4+_}Pi>F}*b9m~)>v-tn;-)yFkPA(-uiTpsY?f945jMZy0Yyh`6 zVYzIxHC7J%F&9YocsU_(Md5o2*lxVNa=cB`J@k^X+pLDjvI$R-I7L$Ekl|OQ^giM5 zXkwp(O24i=6$EJzl-BVh-65sAsn|zFSU1KxBQ*PofAxsjrg3bBcyys}rxev{5T=2S zCue?A?empS;UaPqgcXO4(V2h!jv1;I7-IJO_XwBH`9u#+DK+*v611B1o*a6A<3-)W zmo$RCSJlxmlM&JSx7aql`i%Um9>yMx`VsW&kdN<}$LOso>A}Y6F07Ne=yITQP`msx zYkVMM^hV3|(Lwx7G=C%s5J4d-5rb_i;+gI7d@^IW!gd8vgV5}1HANv~v zJ!fN>KsjZb!DK!uY5+He@+-?(bTEIXB(G`Wb@ZEclCj4y>N3Ejo6b}-{G9elc9=OfQx$L8m@`4_$+m1oN1C`eF1hk!AaoiE-mvILFn}TECUNk=wC3&Q&sJ5RH9DqYuMmNFxv&G zAAiFW>dT5v%MY_rOMR}66(26^W4`2M^*NP+?|S&9zd+C|Uvt$K4hlHIzFcG9JN(j^ z(Vs=5>gtO|4JhJBOQFMM0v81|nhJG^+jT?C8u4*#j4e`s7KD&KUKemstlVR$-{$1p zXgPb!fY{=B{c+hOGJ)GSoG8HUf8&Ae1sx6Ee-%gIPQdM@iEs?5_s`-q$WIBDOHYV9 z6wC(g3CW~CT$7h9Q%w@lpJWz)Z(#azw!H6Uxlpn8!l3J|i4oL$zfw8RG%)J5y6@q! zHXhXNJ42bNhjN4RaQ$PC9PCuKf1SBL7g$hh6x~CfQ<*vqt2USA(@;hJCg-~gD}IZA zwAp&zu0O)Pn&nfp0E)bZ72@qX*IvVmeqOJO;@L`ehxYFl|IT|K#*F`6EHO{2vrH2f zj^442L|SY=vH}F|{6~1!k9bzK)}wrUL}P&h?E2+_^f4nxm*Cm6MVD+i;^(_}8&2{vQ)%(1cN#ry9cif4#6wi+rWfa58pS7&cM6`y`WRI~ANRCL@W!XB@5R-wcS>*?cgyMIw6`;G> zBnmp*CVELG&5sGXn)LA^Y0FO5T$eP4dxO{BOmzukVyzB`Od?S z*0=|^MmpidIZvJk=X=eYb9QHsijI$FD?c1_7FLMi>OqDmuWk&@q=U~(5k|@^GKyZl z=Qe+ZdVGzvsDLYo$^Xw~1UBP|!_iiB*UxCd zkM!dg2k_c&Js)PFT*Spp{0~z#qc*0_ZRAeMD6h@CKJBbe@yw%KgB_m3_&z3YW2Ycx#CIE~#_B3(!#J)={d|jsO4BbkPE@_EzOP!gXG};*_^E8;<=+m3+21<&q{#N& z5|b`zo$@eiObA3`wej|eB}Ucr_oBh3AbVU;q^x4bVWGBv7zib7cqfLPvm~HLWDO6c zQ=T%9DV{bh3J)acV$2Jf&{8bfz;fR?#ag{@5;>Im5u8_gwGMa!|BZx?=|5?`3G+p-DS<*TJ2ZBxOw5$-x1Dv*>nM-IEZ!EsWEuM~twHaH=}Hm6us zBA+d_UA0fv+pd|zLum0W)`oGJ5WNR%_RhrNgm}F`TCk>yMHlN+eQ}+JzBUyl{iXP# zpd&;unPD}uIGS0$7VD%^WW{lBXKc(5@#qcI4EaJD2E=hF`mUKm&Y*`dIQP$yD~kMD z$52}BjHnwk>7;4gK4!>JFhMTNwT5{LTyq5JEs)&t^X?2lUe$#YF`J&OY`4H{>HwpUZ^#KLOfL^Eh~;LOk~ zf5#TFzod~;9`Bq_@Uw7M87OYGyMn!1=bvJRWc~ePW&3Z`&~`63134WrjKeL&+dt-@ z?2$H3eX)N(a!D8WgPFujJVUs@i)k=hV_$$B{pCruY%%sdz-Mzgv2yU3ibiufQ@Aiy z3-Sya_VMI*qQrWH?O3Q)l_Fckc<3AUf7p7 zMNv^u0g0g-h6V*@VCayc2bh7mob$hTt^4B4+uzGsYd`z>?q~1Mv~N-goz-5Ss6uXi9p*FMjuHath zt1Cph6j#rhzswo;H2zk~k(sEe>-Hpb}4xFPh<_NM{eaHPE^Gb3-5%o&i9 zV9hB)Xw#QEVk&G4v-TbRUT)!lO&Kg$G1~sX!J;kbYxv=9*j0Jd_}Z@>wG?Gt_I(R| znzuU%&#kew9S$-ayp3R%a*H^Z*F7CJ+UHIzwfd1?vwEZyPe=Z` zRGmz|YV++dRQ!tzKd}4P?ElMn>)~e;E<~=XS3j~ndl_Ah9~Z%0nMe_`I=z(Js|nwH zXE#gWQ%eN@`Eyrmo!&tESZC$CSU)Y_ih&* zx0bIU-i(k8eN-Z|I?aL{m_>|<+)>Hz?w`6jf^B1_vU{C>|J*ybcDxLXF^Ao$0uN|T zk@g<>F?GK8CdwV7_opvYnq!uY&)zED7o8s1mB$BA!S0=vXJ}w5kSmU$7N0gvIk#k3 zD@!IKb` z+5-h*kY6UY8*Cq>?Ya(&u9K_DF0l!&0~5T1XTRH3I?teqpl#20+C?Dsa)xjQB;xc>DRi;}9YwUzmt0k~(3Szx&~XF5b85iF#&> z85CFyLRacrxPkjIVFiUPWbcLl#@i?@QzOOyil2_16_ZkX8RpAXy3mV$bI#f zX}&8e%C|$d$8Yo~&H>jTC>ewqqtZXR832BYI*oxalKTE}W6*#3Q z<1a=tfVN8erax>F+dr6vK7K{3?BmczLaS;A0l$rw*t#!&ZM<+Q#`GmMzQk8l!G)qR zW$VWV7-2u}q_W^wC{~oEH4} zm+;NIQK1bBAifUfYu1gyh*6@&KYhqzG?De5u2>mS_=c=VF=p^4aY5ioHm=8SX5RZK z*8O8&f-_w4J_P)vx}xtg{71{4p%s3z6tNx#?I_EbII}FhvBRpbN3EUO0 zjY^I*l?EsNaN62AhfAnyBtsVsF_Gw8iZjE{OqyLU*p# zsCELhi3y1N+_mpoS*zA!#f*Tj`h{em9jyP?uj4AS)|L6M`teA^vqqr0fnb zw+O^U5))DXmTbb2fY7_ezxBIbQsz0pfv_@O#f}=Ue(#py#r0$wR)6;fgxA_F>xHS! z_%||4IScYnwp|-{s}kNEZ|S#Qp&*~K5wX!-7P{$+0ajIL!n|Y7+72U`ouTWW_1v|y zXV!?0@WVY-RXVu1uO@VRrzt#3C8L~j>jt4Z@duoZ%JxwuU;o(nMG)2k8b=@aK!fL4 zjbZM0ZA?1RrK*ER;EaWpkmR?9uJB{!G;ITj`7W7sl@D0Ob3)37{W&a^w2M#8%nG_( z#f%Jd^&v9_%HTa3FmnjTzB)Zq?APjm@j^beg=C$Pc?0>au9F;T)&Vv!f=^019cSz{ zjBldsV8DltAULKW>zr92PX*|pR3Wq&=au~-0uVntp*b)$ir{pHQ}}JdwhYyf?YMl- zN86E$EwiZ|YgYKG73c{u)TFnCiu%^j)v|@{$QEq6+Odl~JDwl0m1h*}G6s0hAnSc?en&o6JsmGnHZ`0nqQWkGDb@NnlMy;> zg93>~B=hcyqSDBM3O@5l)lJM&i7W0eq6DG`-4`+|a8-Ovitcw@9V z`q>$gxBN{pYQ#x0DsbNK%us!FAIpqYDkFYr7Hp&LuzH%pazhZcLdxKw{`TOu3CvpX zRB0KWD}2PoGJ*6qM`MaU6PNS3MJIEfCrCp1z$*FCL9Z$ts_3T z{{{0e)t3EJ>Y6r!=*!t-seas&Lt-|?&&m0-$UlRlo@ZE_BdTk9m#vR$gEWG z?#>T>sIK2Zg2}3UlKe0 zxM^Wp9_+CqJ7zj+OQ9#F^yoCQK2l?r>`oyh;ZW-SSWG76Lb9Injdb0AXhizfg@t@5 zdZuL_zc-7Qn9W78AYp)HbxGmM>ax3P|1yQ=jN?cx3zt0ANoO%Ogcv7n2)&6=+htQd ziVHR!-RCPqE>OERn2?smi80`|a-QUn=^AzMjq{me3N6BvnMN{|Jr`Cn6fj0u%)~Sl z!z;#hs$O_|)5J;~6msdqh130Ml)T?TYbpYV6#R3@N*dQ5s$UQ*D#ra)^<)K?rqL;V}72)*55>1Bv-hCPm}SB@R6SMo6?D$en^3{l^ru~c6z z#L0cfP3y11qYSLoAWSPd#fcaS3dKoe$7W2Uft|lnuJuBsxg4YeSV6qMyFVtQv(y@4 zB~+8#iYX2Nd}W>fRDG2>ey%|=>;V0G*3 zv{}%R)Zp+wrK2-aJxc4GPi^>Z6D1X!E@hNS?y0{#oQI09daEqWJcrSA_>&ivBa6|G zNSqv62TkHJsNUAu;lGk7VAiS zScd3v^spEnWhA?GRMa;S!MK6%`)5NgHS%ffx$w;44H~q6&Z0qp{v5#~O%VLbsqzUX z!Y0b}ez?6IKEZ;270EM;Cp=JyM_A!(=oX#|yUx|jzQy|Ptfr9eMfQw%@W(BDqVlC( zdIK@Vh5Roa_>Q9-VcOd{p+{hQ3I&AOFd_XCv|gwXNkg;CZnc>93$#wBM(cI+$a3YH z@D4;B(_O1WT)6ockrXR(QZW=h_PEDJ6%`$L4x7{5Eygc%w!caWF=25SeyKxL2DR5n zJtt*=YQ7jF!DrZ|D1`+`eR^mOJZM+Oh zFFzBZ?}y>i7H@I}#ove=hz$kr<8MJBgHZ^PC#V9Lh8BDKf*~V;jBH@gLkDly(OL3c z*%_UKn#=(WQC+uu2J(!ZeWa^|_9xr6S;m;3Sg-2H0Tx4@>yNl>F@fJhMPU#3_TmO9 ze5B%X8ndG$6^_E6qpe=Bdu!U$qH%05yQUhv5N{l;vBoxWx=fGF&lL~oP8_3d-=a!L zDx4GdR{TbbMN(y!5UzVpOx(xIOD5tA*q?{TvfJWwdOcG^STp)xP@7C7fX&wB=S@V2(-PogwPg|f8kt?$Oz!UeO}U^j)DJ7M(3wI-6; ziZ2K%M<>V5MV;Xii3i`A8%OO@FqSwl7c+f903x1m^bt^keX z$8%*2owu~C8*B{q@scJg#Qif?WEGHi8L?4f%)gbJ!-()XeNJM{{FSmauf2!~p2D*4 z(8=V|)G=j|#GG*kIpA!;tl=J7Mk+TW#`yZ|yK@z9FS@|ZIJX=kQ+BzrlUr7^ZJEUw zBeE+hWjq^rM3t;Ux}hJfH!v8FG1OHXE3SO*RC8>bmrVc?_9iCbOjt?%>j4AL4qA5< zWgZ&&(6c{RrYz5XHI_1G>Pe=-i)RBs@>lHAJE`kQ z?!JuBp~Y3A!kSRGgy9_4S`Erlc6?_M^IQiqc&9eSA@k){n!&sES)o>)v{m*6XQMai z!WWuyYP7HDTz6t9qa97Olt$}hI6B!gC1V)G@s@QM!Z~a>vI=tf#I%pbu5lUtYaAI< zRQ$9r#+d&Qn?VDwa+NQn>HEp?IlCNl5@OXZ5Q%EcP58;2xyYQ7bE>RCUmccoH%YvOz)}X`oWv6f_(^_2LdtB%hX&Gnndh)kimQUJ7DU4n9#fxbG#{kU%ogzH$!TOSf zgz7VSEtQZkyg^6rot*EaVyD`Kx^%~^q(gbfEi~jBEHd2VZlyzcwkC{9Lx@#`V121V z{*suS_W;u>&YatOjco4=h5_x44(@2D3}o7ebG6oSoJ08E9rqt;Y2Pn9$5U>()j7)- zGx5b#M#go*SU}a~<@t-bMxXk4!=HsaRs6_T6w^H#nk=D%amGp}-JJ`Ds40~4T<=t_ z1IA`~XVi&O(C#>2xTT$(J*ypmyUNp3AGjE6ajqCM)`o|>1$}RqKRIVSF|2aSW5u3I z_zsKo1D&R)%}65d!21M`CPJd=Qanmd*{H};@(9}|vy703Yx$hqz+E9=4}WZVmnFLjGw zBelUqL`zTikgbIyH6xRLPTojQkB+7gnn{?pBKFK>Sg-S$Y)wN!Rx!ER5Z*M10!bR_$Kb3 z2oqy2nDtbhn)s!m!Q~n6{49udE&uQry0I#xYtI@$2O#E>Il5vwH8c*&XFfgu)Y%T-ECY<5hAymPzfJn>)N7~OzAYmoah(%C=kjO zvs2iUXW^h0Q^EsqeZ`oYGHXRXruB+ScjpzdFeir!W6}wqP?a-HAzO8h_Hob#k0hEc zC@Q~1%v?*=SyoVXn+Yow_T-l)&F~o5TwVvot>w^v9d^0n$K4LD&z{S%+S97=66A5! zR49VX8nWw$mk1W_wqj)q=h^TN;&rmfHn_eiYBg0j)u^(jkahKCOqt1JRnp(OB{z1C z#Y&<)XML&rUI&i`>XJ}o(8^V-J>4W=Q(wmU&PDuXE2$DTRkiBTXpt;exsi8s)%o%F-cn|g$bX%HYjK1TvKh-myY+}j@?PdW-H&%)UGxApq$sYwleAkgr)N{Kx}p!e7E{IOc_!2b1!Kez4v+E-95JrAiJ~U5 zk2OHTqE6B$-T(i7mzRbkSWf^bDl{r29n~j#s`-PGwx?@MCnWy^;}L@s``N7!&d`0-VLClK2FfxIexrBni|PHu%R&c-K(= zrjS%Zv2iqoPdL?816e?B<~TAE*})*Y<2aS9Ge`4m@Z%~EZ0kjNfy10J{m>wu*gv=G z?ji~b?cUKu9Wfr4+Rr6AWTrS%&km_<>V_rrX)Yb&G%R&R#5{f%q*0rSK(+P4$aUF2 zDioM}f9^2_#S_uoEm6o|B)6j0$^64F-I%aba@*<}F%@pZ>+b$!ql_YXE z89A&A{A%Nw9fExIA#ORpF7$n-E_pe4IBn`vV#M9l#_MPq6l=y9OPR<`xD8fx#I@ac z1xu52)p|z*WvTf*T82_Uq%SQsX~t>3G^A3d{rRrZJDwKCR>f%{S*o0;XF26!!>Gya z{5k#f%nd`mAeFtL!Q3vDZ^0pJ$U>LHV$WZ&P{rR7I>Evx#IrMjELt(Kp3$|ErL3w6 z5zI5V(Dfc#3uDh^i=Qg|F2+NzKiec`<01B@R8ZU`LZ^8AnVEa(NxqaA@1sUDFU`iz zWI?XRxmAp|(w(5<-oTqYRxzuYk-b~yb}pb~E&-WG+W65ed7WD%XBL@+!|!2{a5`7+ z59I{|Vgex(RCEFY9I8{PG_As|Bi%Zz?t7oz665mZ=yhwG7(5&MGD3*& z-K(QA%UfT7Cj(z`>vMVi$g>Ktc{F(SQEb@tLq=XDxfhP1u8r_q{r9VMH8ZhaW73@@ zozNh+97{3Xy< zu6sWv-Sl`7f{%_=V_B=Zv2CF0|Txn|g<1KPxG?kC?{ z^JX@dl$ZW7F%Fr~k)_M!E}_iw5`V=!UO7(9*AOXKYIZI{Zy0w-Ctp8LSg21S6}+wEgng3`S;{6XLLWm5?ZRCG%q5)6tUEe zPGxM{`*c$36JSBC_rpv0cn6?SXenWP^Ym}Ar+TzT<*w#hJfE(UmFTQnUVkrO>M>z( zsKd|H(EO$#qE)VTg!PcSbFXcvz>7@b(uK?S%HFX?p)`6R4%e{YF`M6rff#5pc&h>XxQarW@!s zF{vxdtLX6?S(G_%$vyt5Q~=oUv%FMS+(>TXM>r45Uh7b4ZGgL){!hB6cH%=q@s|t7 z83V7*=hzJ&%L_y)9DeDQ%8HLR^u8(D6*ps^P~($$yZaqkY%bwQJYmfzkm=E>P$0J8 z^?S-}{5fjp8)eBbxpmB>Cm-Ldy;)@OM4sKFZnUgqJoi_^_n6Ij?h_Naubx|QKWP^1 zPf={lf1+1dRPXN5sq(a`KFl^uVk=Npvru9E|1pMAYoW-MZ8**u4%eMtgBxca*()zS zary$XH1@wQdCB=X>iA6GPc-q;d&Jk8<|gp6apbiYn$!%y&Tp@~dr@|fz?tB$fetVb zA{uy#SpNfj5sN}Wto(rNh;O)vsfU{_P$Jwp_5t3na!dKh-ogBZ_wzogfU0W7gPPXT zkL8yqERm#Hp6t5_*z*X?9z%-KztB5IaD<1RRmF#*q5La7#U2sR=)>pp`#Rl@OJDGe zsZeHMu`@Dcs*>eA2V^TAAFy_0_Hs3)+DnJB* zbN=c=|M6QzlCoM++zq9dR7osjrp zZJYe(sBrM|{e}Tt{U#`ynMLWLYxQ9l=Hg4CYz)S`UOsS@c}ze0;ofcv;|XuLtpdSF zP;YNm&J4Rh900ZViEn-(nRizT+G%|`P@`wEAb$!6uI|WT{jkd9Gt1c)5YTP|s5pv! zF|o-a^!PloSB~L~f9^F8O6S^k0-=u-y5rcifF|x*G{CVDymC@b>W}SLilmkwth6fn zgFtZ=XIXA6dN8K&4{6y6gXa$O#+bqISx5m0yXYc(eHzT}i`gS~CAY1}6(b#$e){uF z@$vAs?5Zoi$FRco5H#m#|M>kR`43=1*&#;z{O3{mm(%21+hO@*#Ix@66PFX$i^x6^ zkw=n>z+u43{GayVC2f#kNX#s;g#}-gz?*VBm*ot`;E5cr^Z2+ z-&$OZlQ?bxQ<3RlU8wTx>lr05;ZW$9c}gzB0Z|W^K5rRP z&QVN$EIYXiVp)MGBcl%#jam>&yf27>@?g&^nn@TYZd#QvkO;+h0=+LOhv7eDo(+8V z4EKc+S;C&JnpADC(lnc;^dy6IZqTA1)KL=WUoKv@A^LW&(#Khvo<19Hv7I5`a9)N^ z^cKXB+=ZSw4mo5in*$!kKF9t!U0yV{LFzBrPM{PK-qu>sg(LfXh%I;lYPI0V2a~<$ z8U`(*Ko{AwaITAA(3S}~-WJ6ubYGQa!#j&jntNGEAk8Wbimv5)Yp~5Q6&lgEN8g4`>LY?&N^1I)E$+ z`^Z5B$c$hDUDI}pA9r*M95QN!n)Vt8(T|m-Iby&ZZ_DIb#t`TL{Ts0$9}nqs1)6=N zjwcuDwT;MT^V#pu@SwekW~}5hR8pw_&0G*i$b^v?aLxq{~?w32P(1Xjwp2 z1c?kbPlFqL2%>|?Q1Vp*-8n`JpGl4uoniH%yeOhunhB9yP%%m;xUmUfIfIb3n(s*m z^MlMXft-k^6~0S|N;|_wKsW$5aK;J3?d(mj zvFfjtv=*u8Ym7Z4Rh zq<;=Q`?J^}ZSDXF(ZrBA5FD4o0gm4v1g@q^Yd-d0F%KUL!XyFNYePT^m( zDB|n`v=DSx3gARy#Z;e;%fg>9@`|DerbpmG^aL|^jQ&l9-}^_UK7{-+wP48|g#%Bx z^4ZE|xKzT!FlZUxfA?7Lv)6ZzKog#hK~W>)?v(nZ&l@L!oR=73ZAkFR z%4N^K$T;BLZ(|RZ+;FMK<16`yuK~B(n@rO&qQJ;*iMEz%g#L^#l4h^GAzOR3SsXBp zFaVQUC!>--+uni4%XpPEAZ^aUpy-@#&&3U81aEiXS;To29%@2~Ef6PIy3g~nk_<*n z;Asy+()z}QaXiW*6BS3j9)Z_w&xqeuh_{bj?jo8XOMb2Q`InCqoyN$E6S&h`Wd34l z186@$cq*b6ye})VNfHk{2Z7)^Pjz6ettFXhsZ;z?_g+!L7$mC&7`~JKt~?!;vSC?4 z(`FR*i23LGPwrxb>W=DeZCs!k#JK9avJ!OqHB36<>&1y_*!8uv_9Gv(a9F<$ z^{!=edSUcSX{bj`Q6$#QK~%Og+qKf1v2FM6#nC4OCwTaT8fv@=o4T{^PTgnXP|^X! zw-TQ@W!iVOjy0Jy7s0Fq$fPz#%bG!PK2Fs*D@;d>4rus>qNgH71kRrv1E zOxY1(X>6<=R8gA^3fk7>u=MM6#+_Pkn9<}8(lxI-u6la_MQ4F zfirF}cPq_zLuq2G-{6WIg1(+?J3V3^(y}Kdokcbkz{@?~~ z&NeO0pM5(7!C9VA(2Qw6!Mrn`il*84;8Zg;selr?I0f}E^hF2*OrS9&~ARS5YKF}Mp!Lg7csfW5mLPE!9^9l`9w zsT8dk&i|ZH3L2i*>yK=cQxA`~6Uz3P}=PQV`MuPn$KI_5IyGu*AcPpiU1oz2}%BvZudYuo4=K5G+yfzlZxOpOSq>cCRhtRrWoK2J03no-MpgSo6$acbs}@$< z38|GZ2dh1rofln#5|d};t-A|i8XH-E{9mtUvDOBF=}3FWkpwuA2WQ>+Z^Dh79z%c6Z_l#gh(eRE_*dTL5>-2m}c zi4S#1fiuV!Owc!IrXLD#W9U!5^BsG5hm^lYNgC0x}8bhHB3c+$!#}lm-j!J3dsf0mU{x zp17{kxI1tGEFUX~B3&LIZ|jC&3!~42BQ~QeI@6FNND8NapVzLw&P4lwc`lOo%Rd4W zWsAodK4kaXKn|v=fnpH(+m>z$Cy@sQkw>u#d*mpzkOnAn>~>>uvV4E*04{_^?|vEE z|I#9bH1_BnLg76*gQ{G(=_B}Pi3Vi}FbsqZ^oqbMgB+cV$pNhUc)zQ@l^uV0y^MLs zGWj|`nhq=#w%_);4Kh;I2&xG39&ed(XoT_If!$b2AE+u`o_U>1LBB|>Siwo;rq8H9 zS>aO;Bk)@2E6)A+yU_spV3N+_DIh9Ndf>)pKLGe1MCR}bwR|YD51j&v1I8jTT3q&0 zkds&6h2fY%j=O+ll-}d_tI5{(c|@k77WmcGu?IL!+Nj$)#_{5)f90GdNipDSABsQF z7da1IYJ==3Ydr{EJR2CilV0F^A5M(a1$;S=DTIs1RCNQF4$C{`BI{zf$5AAxvMVld z!Kq#}AOphK5`FqTutoEO$7!^lx2ZVk0OMR{gzp70JrUJRe4s= zLTc71U}6zJ?@$dZY%r@Tcm@vFs@ale|+a7s2m3_81*Qv!~KLqLnCdZ_& zN4jq$D^J=c0W4!gg80XjlL4MHCgIAPOB1lv2b?JJwW;dKE|M-5M6hsl$bfNcI4*&g=$RY&AAxi5cg1AH3IMBw3bR)DuNBqvNk zh`!D%;Q@pQ;&5GLPM%*`5-HMZJP;|j&lrZsSkn1?L+;2RKDrH3zeeL@dP;Kp1qp^m zM>*V)yl?E6IH2zD%6;T1#nhL7j>s#|4|IGV$bw2OZFghN@ATa(I(iG26&ywsK9V9$ z!hD+m7+}+L`qJ0wkQ9J?r01#GRk(i)Nsv}r@w5_}FBPMqCmML^sJ8-G7zj^ zDl*rV4YZC)zG`ezPH^x)rz$O;eU%^m3?*=an!6L+R}X%yMTovBy^C2U2>Cwcd49he z6$dBpxr0{hP*E!@A!xc&uc&9kH5@FW+s8UfHq5~_VT0n?AV#LIW*-b7f%{`oxEx9xbWPbs`eqcOd>r2kFk-qP zf2jDepy5*TzZ62 zp9rG3yTvN^H>VWO!BhZt2xf;2>QQZ@d8_l5VDKyuxe<^hdQyAFLQe2st@sF*Moj|$Tdrl-ZbQqIzDfJ`_Q3c1eU z-dib~EN8a+Fay9wV94lzL!qOGfoR>#mNPI?=0Z4T`6y5bE^&rJ1M$WNzf=0}IAVmB z>WY@?<4y555=N?D|0~15fwTVKal|i*C#pCa7Hc0G7Nt`kIv!_r9|oc1W*;W025UbS zrMXi-HmzHAKMrHiWXz%#)gn*#)*L#%^@l8a7*L^3jF$1F>BzdAol?)ck6mOQMFB3t2fO5IrTX3C#L>F%J7MET+Nl0>Bk zi*r^5i^^?Q6_2ZaR*g_PG?uAOX-fUL4rieo zIU!HYj+!3-mmyPq;U27fT(NOU3Pa*Ww*^C)-TDP1h3lOKV-;+!MJaVMivknv;CORW z`SAKh(=E}Z0yAS(szP%!6N^F%tABBYW`k}GOO|%QyGvG1iCje#q#2q;n2Lq&%QjvO z4a+azJiV6M`VDi@NCwVX6x)Yx6&A}!{2R7%jCsJec8Vj?vUW~lN?Ns&;%Z!V!53RD z(a%y0NjA>WI0UG+p@K zd;IhBJQ}-i%kNbwJ1qxZe;ejrd-G$-W9@CnCbx-?9e(wi51{*cuT1X^635Q(6TUQ~ z|8S|(y8jrLbVk6W*ovLYj3Tam(7d5`g(uS7b3~3n>T=GiDjL33Top6@Z`2VGaSLa8hhE9C z`3{R(XHgdCcOYvVeq-%s+=b|Bbv%h0ZA}7&sdY^vjoLfsBzm7QXG*3BujORjceJ%B z9C_L}mZCP0+Ny@2j=xsO#(6c+nBwIt_{WAHc^4cxs?ThanAFMd%yPAuQ{D3PgvZ_TzscOyQ@qeBwJI<*(seJS)AigfwD$PO zm8t79?p_=ecE?rXtj5Znv7VvpQR-n7#Qpx&C%)pW*J;BZWwuXmV}!mQn4aYUGrFD? zAx8l`mElL7o`g|1hdizT4Eh75IN^C{bpo~bK}`zxr-RxV(Z9TPF0%JcRWyHTaqDw0 zSP$z9d`h$%%xoe)29F*B|dLp>~&?zhBxRacgQF&@6qQ z&d&FdKBNbY&8NRlejT57IrlTPbhFIp`}W*`(|vpIPt%2f4{66Rpfnoew>}kO13$xT z$~WJrw3#wQpW_R3pP7h=GP-m9srVx*Ln(t49U@C-^QWa~*Z*#4*!hEewwQ6|8_`6#OyV`~UIOm<_8bGyNGoO?9n;w7&{VXJq-$L>~auJ3}cCxkFf zKTTHCM=m|phu{8F+L{8FQ(xxoE|q4!x?E0kZs$nW>HwUklry#*CMhl*|dSTI&sv|B9DQM_(5w|aZMMCwu@h4Io?k$A;E z@lz<$~a!tHatMO98e~9#*t}mu5oR2klRT>bf@0zlQd1 zn~Nx@{)EJ0hmG=TMp9EKY@N>p4stl0XJ~iWG8$V0RUIz6-P)dBMdjdPqZcJB3j2rq za^G8rFRR-#4}|(&%`^00fz^9(Z#~$SZ1b<*U&&6{lSJxHGQalmXBq0(k6c@L2A^H% z;TC!=3Km#_v*RiS+MGIo@g1pDxO&fXTcY;lS?)awm9FpHqfa=D5$m|k@P3h)uG#{m zf@GnvGY0MDCHVujr7~i}H}ZC}>hY$SjOWak_2sc61PND0!yye!B2PpsLq~Hq(KE&5 zbV4TzQ^DdEC4466uunav!Xh~v>la>aDu>z6@8rot#+Zm@Wl0xg_#&3_b2T<3u>XgX zX#8vPob0APMV!f0Uxxv8pK&GiM!#wmmR_$E&q9#_Yn#K}Te&B@uU?;YB#H z@$P4hr(?X#u?4jN9F$d>x33yF<)9IbsqX> z?{9Gk-ydineLUpyW3X*hIoDo7n|F!5ietsq)uE(pRihGfE5h0w7QSeB4oKGE93}s# zL#AdJhP)qTr^6rCk|U2N2*$Mal?QB)(W>$ z9_3Wx7Y^L)kI&pmRH*PwvhtFB=!Q%QF(gbcrtFZs?sx{T(9tR~l0I-w;*BOun12c= zba8VRm{jGlnolGjn>C^EZA$j+2{7%LT%&USm}=V~S_IG86?~t`AlK!4tb&-pefiE7Q9M0y%mLOuL5-Nsd;W-tECgaU6llyOF8+{P&I zee=8h3}(57k4CUg6Qg4{Px*~bWf1A75=Pq>Dqwt2n4V5$P6>}95KpLnDUc-l$G&2P zlVa4KSyg7YXX%ZN(U7c>3>x0M&pU%438NRW3UZpt4|m26|In%l=4K`__+M;y2zz64 zd8yj}7!#xqs>J=$kWTvkMWS#)4beufdg;YjviQ07c!HP47d5`r- zT7`cfjRDUV8DzbKRJoEg61;nAj`r&`gg%*M9~F6K9N_3IX_yxNUX;^JvoWaP(Mw)7 z|3GeGttLt(%u6qZsjyTaR2b*IQp<0FD;r3uSqKW8^2dYx!@|;O%hhucwfOW~wRkwy zL|Jl=%&#|}-QuVBwPak1EakJBE3}s>@3jh>-{UnWn%EJw>3yiHRcW@}^|tEyTa_k9 z>UtYXsS8uZ2#B%B*$UrK1K!P)c&3eDf#GY0jY8ull4MM zw7ovs5zqhAr5!q|JbJ2B$T7`RT>F2?QUFfoe`RUlKe7Z!q;`@ev#v}t(8*u_=!9!) zG(bGv@;J-F&sRsw#D{{4IND~KcQvOd<9Ka0w} z{hzl_aF;FE16rE??`|32 z84=1Mp|AeB{&t$o2S3M_NM|PkvMjjU#f40Tqkk-~S;ZNP&)WzG8TG9P5>p=y zeH!?KA^TtLfyla4lBPHmbV^N<2!0R_s~~kaCnuKOenKuyzIG@pr|E$b9SqJN)9T$2 z{~|Vn=@E1|Bz6}6U;%3vPeK?N%g9q``yU1WpYl*rX`*l~8-`lTm@|RT7 z{!P(*@b%A@F27($b<;6hGL1l2Wn97%pv5I{D5IpcD{+!6X&~mVMlydUdjDvoG5Fevi;=+D>;GZv9NRNlv~C^S z>e#kz+qP}HULWufbYT`T8$5XUik+>VxA zU}K|NObF93HSAut0G`xhc7SzyMjAE=)+WQh+ZG?fAx~o{_^*Hy|0}a5~W|@wT^@ zv_(?u=SkuR!R=J7LF=2|3uw6)V+hDwV6E1(=9qQrIx(!Q>6GhKb z6=3cPj-bKV35mqhuuOACc4soi*vQ5@kR#fu2VHoco0#;R6&Ig$I5AnF?O+^WYLOL1 zg1ThG;qkbP@r%r>^hg?geT?gQIYBG%sD6ofTpB+p60fow zq#~A<+~2@74c7$sBW`?JQ#uLod34-_G9g~0np_;MVZ`QSg_wSK;JG+NrP3}OuBeuR zXf}L052PC(o+m3N75GLPjPTO%kb=r$L^}%trM0sX*Q}dE=hALFfdyfo4j2CJ%GDJS zt}gCV#=`I+mQb=5^*86Sg5l%YM`sQ^eQ;~sIFk^Q2mLYqxP%>HQ4`NJ)`+N@=YTm& zi$XUs2XZFb9JFLTfPQ(FbtuT3O(V5f(0IztJ`|HYOwn{E0!-Mlb4u_jvfIYog&TX_ z93I9ntU0f7q z%W^B9YY~eK)u-OaC$N=!ldGR$CO*gpWFV+!L!Bl@whwU68_ip2mn`!&_(xn=kq;8l zX>e*TXIX`%8jm9t$CtU~0h@szv3l0%I~(}(s#EHVOD(|NGEeVFVgO^cBnba0oG6@D z*ujDtNlrOGFjA_s?3^iI6MAq#F_J5ppqOeNB9a{Kp9opn1-Yhz zPUgSon;!EEe?M2=eAPGIu=!CrlX2!NQ4LpViWvRWLY+bI8WgZzcfY6WjbJdz->>6V zTuY>FXy-*}NyViJtcE3ZYp%ygsR9e)rCS{5e$z4j&Lko6r`-u3#}^1@qaI zu7E?@J_Em$UlQP^`vAsFtf^Im4y~0B7vMkdSkZ*$R)BveMi%XDtx+XcC0bD*o~Z5E zs7J)Mt5T~>L0VVxT`?$Km$PVmiS{S4&sMK?q!EF<+jIwtMBL-=BxyXynrC|X%UCbiSFt|BhfRjj)9>y&VOg98|!?^DT!|6k{ zNf)b93OdoLn2)Bs^EGsKr@C;l{=1~Awhc=v!FGi}Y!|#wnzn8trQ~+gA{kQVGUyBx zM7{huQaVaNT5MtE2Tte@442FHjErz67TOvYJzD4!iPcVoplYu#AdtK3O!|rm0vx#Z zq8&C|@C!FrKL7^gXQt|L;xd1_3?Zgu1e5nvaUG^g4YF>=N*_OjLvT0Ns`ouArS+Jw zax0F}BTJnK(DZ#6JR8xLTwFXYX?bNTn;Rixcj)GLh zz&b5E|H{U4%@aSyxPYf0cxZpk@mch`l;9$P;T))D()=agG^X2}C(BZ;9+kyfoxseC zI{`X=u%3p(1c$aILn;$@T#Uz2jcE9>B| zZ_3DQTT%pc#Ygewq=@Q+rAy2Fu8ZAi)RLlhYyU$iSF9Onyvp8c6U5x{I)Wy9T2tTj z`Y_y`w>dc>k^=pOY8%Db!4&+c%ex4V8b2Vs4beZ+TT^epm4AZOZRpYI@I#Qfb!H9{ zjC>8>i21?L(E-TQ3ve4ESU;wI^-dwCwdk)* zJzYw1SD2hu>q*>Pf(;$$zt!DfGNF0)+dQiD10KLiG-w^*&>2x98pUf)wHw!E^3Z?( zTe>M7RmVCe13NKt(ohoAB7;a5+=e!68T%YTj)?C&9;BjZ61|y5t6XN-z01p78e|7N zf_B2*lk2zW#$&rcYrDO(aE@|W#D8by&*UT0x3+5)kT{1f;Ekk4mZlTDaKxajuM21@ z%uPf@fciej940R}w2ZZZf?+6s0uJl4JrnTT)Xq<4RI9nBzN6~?Rp~XlJ~#pImb2yq zGcFmwd7Cd>LvHwqpq;t|Q)-@Vy?&3oXky>G;)qaf+Ec{7s@U3_OS6IN3P;B-vgt>X zI2dv3j^{w`3Q{Je(qDb}vcHYcV0&AfF*COeK8B2E9pzed99PPjaf#1ErA*-DtvC}W z*|V(M`hm)GfrN2@>CJ9T8o9LvTBmC&#}KM`$&H6>XZAS=1JbsSdF1&&Cv9E<&%6Kv zrWXweSumKPw2M(DiBIFIq}OnYp5bcc&u*5WM#;8yjNC|vlsbf>@@Cq^mQBOad9@ZyQ3p{y>xl7H+%CL}AbVkd8e zjO=QbMOQ%LP&a)RMAk-ydJ6OXr+Cuprl@=wrDn}KS|;znhi?Ib0vA@$6L4Oy%6a(R zEu>aoo;hfTWX_*YrR_X81Xh=zlLXP(Ok5~*<4(NF?DZZ|iJ8^(XgbbGrDM1lSa&4K z8VrdZ82pl(*9n^AF;9T5Q6^z7_+?re#1i7;EYm&loA4o8B4GnYoqLa`#4~o9JaJTk z8ot>42bII1;&Cl6a&t$k0^=&PIv4q%;z4JO$S!TV*8fi~OBrm;@c)8q0UJF42mQR^ z^b!mgJ@MbhNaYaf{{dGb>R)itQ(yr8+?Q%OZ>0$hOl^a0_62Ez1mB3u@Prf2^GtxQ zs$VZpN{!1{FmfaL)Sp@y-55FHEG0LxXSZEXHKv1_Bhz4{Pg%sm5L?Hdj0})&6Y_*m zA^WuJHL6ah#Av{4m}2&=RnWIvxFDuh>{`L>)keFzWIF_tn?kR)dsuFAr`=<1>1ENn z5YhnDjH*m;E^h($2b$6w+{TM z(_(Xhc|2 zw%*8jYq=mh0h|W8`Hdsj#8${2g`RLIz-;22p$K61FZ_CiC7-<}i-Q58< zYsI*5yPu%p+I1|2q>KvOAy2E*x2(yl|Alw#XE%<|JnXoRM2I29pnH}E#WT)9vWb^7 zYxeEp4YX~gD_AX|wZ3BPZ81xRt4Gq_t%5FfiUQ=BDM(Y#odwRTAeFjmnkL7##Bx-i+>gJkq~{^27m$t1W-gLGI%n2+_>27@Cg};tv~pif@_@><4j~ zCB;%PE;sLqoNFaUx4gJT?1T$*VIl#*`5H8OoYMnki;-}*)-;*7ZfqvKOp)%EH>nPS z8fCBA z1KvB~Nv|@_mUvDKN()_#%I zk-#_)PP(>3elrWzvF1F0rG(4#QKXdMaLd(KH^LHS_+!{;{bwMzFlX-fy58j4grF)J zGzhy*hDi4mtEWd}dEZfK}kANwZFkgPQZ0lV4D^hf~6$sK$nfEew7BUq_WlLXEqEJWS2U zP2J!=W_AfIhP)>@#L6nrT>4-M7t8DfwFw0SJ3QRKM6Uc)UW1{yV%qZLi$jB3JZk3L zq{TzyspZwekvIgkQ8{=LZA=@pJ*Kht#Ccf!j-gk$#f1gk)=eB7#+J0v(>T}sC`MP@ zdK4e+#4L8*YanCB?ed)VjA8$XAg^alhQ}UhOpfO4r>|NT5tNi1f{0_ZuMIwQXPy$9 z-R9Z&l2_sff*TeAIRfsQqMg(6iptB;CM-?a)+LPupe z32nLG^=nFFSmd$6L?8GxAG~pe(0w~_qaWA?y+^%#NPe+W3p^8i!Es1Oj_*g+W-Qj7 zLO>uz?mHOvLkiG*93w%xLDAt*j7&vJSG+lPU=xr4M5eG6lD|xl1Pg5}JZw08n6&9w z9@S;2zw}=2^%fttZ{vF`=Cu_xRl+j;pmfKtErbh{Z2f?p>t;%3eBqj4cF1x0jVLS-j(dbSnf>HuNORddRjMavl7s9A}V%qU+kt zXPW&@%$XzD*wq%I1_2`GPVcqv$%4ZQnm=6t-w)xYU!zYdZG)?tAJx-|tfK$p^?==7?)5sh$xHhBB%hhDFK3nH0vM>0N>y(bhu(io8rH4WR z>^(gT=%7=vmx00H(>I+vq7^sS+2lN4^2Knh)9v^L*e=nGmZC(^1_7r`|pB8+rN>m7eeqqWNZ6}Y)O-exK_)*qr56ueA`Uvp$>mwA26Uw zQ&PWt&SX~7wMeJJBa(O1jN2A)_8|Z;*OQ`l{QWp!xCF|!Zxy|1Lc2(d`{DsWU4_50 zWg>;uwDAQWLw5@$e#A!@0$Qs5r?BQyG`I9td9J8uE@>fzn!Nc2TFr626xi?8(gRzQ zYL)B;9I#&xR}4)6**f-h;i46}1BOe<;F?CO55~ zz&lMgDOy6h+Z@ciWzJoLh$cza-2a5G4gDp=c?svlz_!+BGF->^+7=%+<2u+Gd`|Le zaf~uzozju$*FkV@XNynl#(};ZRPc@DzYX_lN5<>5>W8ACHtS=v|23dOdEQyJ_7QrT zjm~AA^?Zm-AE)f~+L`}$Fs!=6FwL)`Jvgb&?arM?>WDKaGO^^?_z}tNsgGb|JOs%BE;!i~nMJse@p zTk>}YNYT)WAwJ<`gArZ=CmC6-eTl6umr4%Hg6&my;zW2qj}rTiX_IGFIdO2C#psPTJYrq2IK;Fo*lAbJx= ze+5-(hufF~_ZPyrZ%$@22RE^h(2CBRiDhW5G*7HWOQ4+FY-fdP{i|^2S@a8dPKo5s8Q2GN$O5@^c3h4_S|%ZM3NjA zKMwmNUjOYI?}_m0vuhiTK2=wQu%(X6>};J zn2Vx0v+va%ms_EVb(B=VWjv)$rObwGbEIA!v{CqQ2`12KjxJn7O9OD0Xf2-+xc31- z!&L6Mas@euiyZy+aF*6Qg1vv_x1e(rDV-BKZLSr-DFwS@v*oYdTmy_nx7%3nSOK*>3N3CL%b3!?w183nn^L@jv zpS2=RPC9Cj8*Y~af%FaA->f+?#__s)ylL*px_*xM637wNG4Z7wRk>VRm5Q^rq+^ML zV(8@9pU6A6?)Sydu20TU#h$_)-iO2*J5pe;_xuy{ zaL@oXbL(N0(1@-!iI0qNCEBEd8!0(*1-V6-WO1pZxk0vKr_Qc)-vi4Jg1Y&(Kmu z29sg$(XokjB4M|l$AOTq7yeG#{m6&xb-oSqALa#mTDA24mv`@2=w;aGSEH?Uf0f+Y zzgdue5JCUp-P%7E(t}0x%ei~3pS&W(!%v3i;aak(}VryB?c6+S- z1$Wf<8Z}R})RsWAQ&8;ZRfxu?n)=75^sBcZ8G2J+BVL1y0_(XilsI9q zZwf{!zK=YR#)jrw;R{ozOq9*q2M_}=qc^gI#n0whJ0B#?HC__Phf|p??J7=DtGf5T z-2u}a{KY}<9MxF$!5tr_*@}_3(bmIxyuYSDb1!YYBU-T3K$=HK+p@*=b}B z@(WGk;P@xZ7F6xsPtc}kd(-}nfnk2&^^*s;3yewb7AFv^VMN8>2+IpSPG+r3JP44a zEK{0z*TBftI*ZUHZoq>9&}rRuBt~9nC+J|@S`bARw=8gXu`e4oE9FP!o=v~Q=K$8* z4Jj9z;dEv}ScywZ)}Guhe}X*Zyd($vTGqx24DnebIg?=j)YS?&WefbQ-qR{q=B|DZ z?mZ#RRk&M5z_AO%9!TWbk{-X;CyV4e+ac)DVs<40H!|7AHd8NHbi4JN+IzdQHDsH% znSkHU@gve;5rO^|Tu^%>DFmBRd1+j!pPaO-*%xbz-dIG?fSrRy=H9k1)-Zx%N+`T8 zHDuyW>*Qjq6A>HWT54lkv&OvvTji|xETFxc)@`8t5!pifY-P1MD~uc5kAXL*u9Xu7 zk6;xLZEUm#LgC?JuWe@2^bxaT5BP=0z8vynWItLT+keL<-~EH=7aH%5_w-l`$NVHl z6ipAC4KnQ*so;0|!=3+ETHR*G9UdL4OxrT=QjU<414gk0iNS92()|K&Ot0uD1k;iz z8ud%9uWNTX5s`{yDf%^5GY#<2EvaYkvs(9(>4N@64lP>V?PPk*&V;VdIAXImJBcN z-~0!BBljBM)o_Czc-KXr2F`3D5iI%!o^0hO0#!pcj`rTA(}2ErYJw{d*WOCa>UA%K zGmA=J!%AcyuOT>lSXhZ?Q309R07RB2EazDVe0nHSo34Bw!nM|V-8GBvh=4h(wPmiP zq7P$k2tnf4EO0vCbAJ~WNtT6i9up7PJFspLc3m~7NPk&JEbf^E#moZnKW&?`VzErq zyqifFWz?#5G;U={7I(=nsgRD!rcIuD{t&-x64*_UXhu$lr7)DxwYy{rtYuDV;1FJ~Mn- zlCTi<)yW+y!l4n38z|jHM9r?Vm3JoPMxc0h6ij>_Pe%9; zt*jt`PW}GHcg*`Yb@qS0W8OdNESks=510y$*>)>Qgt%yML*HBGrw(y*JQV=g7N|#Ey94g$GFBW{1pI96wybud}O?h z|7D`pu7?>>oq^_pTC1{c5nORO*ib~KzQ9b~90HUO%+-0bvB7?QKds)U`@4|HSs50e zQJV9Vv(5P5Y>uwBL%3l1(>e@BauI{WvV%M^LqVYdDqfVFyakeFD#S8V zuy`J1df9Herl@!hgTe9zuTtY*K6iWy+z}@;67Ma2tH9*qu4;WPol!&u-X)T@o6fS8 z$`<{-b#|s_Y^0T9M-MKw{-{X9&qEh=g_PIr-wr;%PVG#b@$^#eCA( zgu77+oDKN!!ui>?yl-SC-sbdJer>dboI@yNE^a(N@ll)KV=Io!CF$0JONRyH;yl^> z$~Sma)6SO1w(T?HrBdQN*Hf!9w<7@mNZ||Di`(*>hQO3d-Hs0p5944J*Z$FX&p8S& zE=Q_?=%rFx9DQwyL^Ms1R!U^j*AssHZWi~XHp0Ru35D?S5sbp!3^Gt^nZ;8?R7GCR> z5~&7DCKi!V&O_rs5dly`?uayYVW(RM0KNMi6IHv}}{uJlUw} zbqC}ScVwI@UJ>8&F9{0fZzBiz4x&%K%k7*rGO{p6v8XfpX6B#F$yAY)bpjMLUF66% z<@q!4g7uu3rBn(_F6uj>l_NunPhTl>j!-QDnoFDyqKB}Q=|WPVHgQvR@AA5F%F><1 zUXqazgk_T!!rVGsNPZHm;_Tou+KxRGz1|Chjkx&L;UWvfm8eK~;v&Ss9AK?f1eO)A z>+djJtr^MJft_^`f^r2Cz$u zr*K8_ZahE0_`7!O$Z8cPnXTCJIb3RFP9cXg`Gv0C1Yo$CXC(O*Q)1z z;H8b~;GjLPnr-9SY2w!M1oQRSS$5OXbG(IuwnWL6T^c`DT z!h@W#fltW9)1~pQ_NdvoR=A-;LK_*gAPPo_cBS>r4_3+YTA{q6v%={2D$Sf=Z99hF zEzrp_SX|IVIJdDI2OfmeC6K}cp)|oTg9anzD^RL~&1e1i$eg$W7_`p0k2MVW)QGK8 zufBXSJx?8j^2(5N z@aEf0hTO=Na)1~@mc)5xsD;y=I!My{U5LW5;CvNVagM**fK?T?_s`)~u-?*1=xCL7&{@yVth0eH~hkVwo*gS%`qa+WzB;!Vw~MO=j@C9L`1^TmH-P-0k1s6^AZ)2PPpCWeEObp5(3R(yPgUZ*rl%g7TXsfH}Eei90mF~1v=dsn34@Qgt`};Dn;8Q?7 z`9|<@dny0(9nY}QQ*qGyawD?-F433#8`+v5Wd5VWO8y~R3ri8$hnhIN%`*EI^ZO8} zY1U+ktS@SA-^QLWnOFd?~D_i~Sf`&(`lo9{D z9btMfmko{^&{g;3v)syw04RUgAi(*3tcz(}{5Di)F4Xfy%oUL4tPqZ(?pNM?_DHiX zXD`vHho!f9kEBa38mvBX()J7gJxr4rVkcmrRZ{Bg#5~=`cKsG1Yxvf+R8}IPl4U)f z=VS`f_h=}dwV4bc-9e%h^o~jt^J~kt6o}qT>&~*& z`>m+t%2Iu3)Abh(kRHj&RJMkKv}rG>l}i2vY#ayKCS><-VS!mwX=vyZ=P`rUy$(E= z9XAb^SS#E*-WWgj`~T*xw>DP@~X42E^{%aD@Ublh4KfpqnIyP**80 z<`gnp?MP1HSm_2ECG{_y0R*m@gu&c;mG_n3dfCaxGOn(-;XNqhX&rzU=o!aRBSR(O zuw#*cFNMcu_Pv2;&(gF{X0@(cWTBkmEc$D!G(=amU@|ritVJ9ib82%*xv*sJ3J zbbqG3q2Bz=ZUfomu94RAuhG9dYZDBT0zw?oDgiWyp*%DTPva`T*at8X(=LMQ?BPQ+ z3WTCQg4Bt?e?dhloMfA3mPG(7vGO;Y$b;k|V_4)8c(TvhN{`Qeb3T?@ z>uS#-7PZb(AqDkrKE{uoGulqZ=j&lQm$KKLBjzFTg~{DOA-dbCbGv3lvkRvJoAAey z);4x9*6@M$lHGN{99yco1-`p4iOO7zm`x?hpIg-4TL@_#6sx{lm;z!HjXc~}hMone zAP@{_q0*@M?a|g~xMyuJ`2S3_`c+XJ03qaXWXIpG_!fz!(W|QNhE5AW3{DG7HGg!> zXd?Oy)ocA@)-?xUCmHGVtkee+LVkmKoRB~kBQpOXO6;%Zs#3k!rVmY`y1z#_XI`BJ z*7&!E0QcnpYW7{n?zgFDkS)AfW8Xno5bfR=h|pMG8q+C@+IjBM@@(MPehGOi1+kGV zDc`fq;v*KEyE!}7Z)|zs04V4BpdcB3D2%-^l-plU8omk8PpepD)33_)IdC(TF+uObZkX2A09`G~S_f zR*n0d=Y{6FVZ>yrstyeNqT!1mT8)siwZ^{0EHb?O<)oms`b8#2rvXZq(vp%(%kG-Z zL27GKV$FkBS-#3lD5nwc?~R%r7}P)*`Q@o?7Dn2#0g*e3b7o}n;e54ZO45Q^Lb#o; zp0srQbwJ>M){N;4jf`YXh{m7ICzpGiy_YB%L4wG4^W*2+!S9G#ul?9twf zFkk)op#6L&R41hKKp@KU^>xPlLF}OoVOj={K`BM?CB)hT=iM#syz65Os1EFw<=8On zr~C)EYKY%upk5qhqx4Yeqx>LyQ4_LHoPv@14^M{!L1JN@CVAUQ#v9HgK+4@jchY1Q z7;r}pVl#!$;F3>z3>~heU~xwB$C%rx5Hv>H?b5Be9JQIvNucVgfE8JY{MdN~L@Zi%V0=+wCX9-sCwEImwBP;EYLMNJ9R=wJ#h$-F(XQE<%(bri`Ueq|@f zC`&W?nkK2xo!G3Oqv;Uds)ueol@fX|D5&U_bh%Ski0}rbg&`^+UHvlBm40?>a@>?B z@Ia48wPDkE|x#*^dZ_6xVLmbgc``Vz*_b)aX|HZjA4pCtoZdEAZs?|Fqj9h}| z+JT3TvM9k;;aN<{Z`c*SIopxBtg9o9w94*nw`#n5{6qcq?1z$N`!Uk0S#X36q6;FO zbOsj#{)&UmLAH2^U}T{LP&%B=zK-Ov1ad}EyU3Or9oHKfjJfr_-Okrg6reh5H#PYFEQp4)B;)jnDyu+iY=8Ur% zi%rJPa@CbUt5Ow4$;>WV*y~Vr;lTVbC)l^G9RXzp(^v8%Dhn$ge-B#5?Xf1?&6lQc z{{~Z)k__4v^>jVo$?jBw-!@te>t}M7WrG@La;mTzhC~DNmj;5apfsD1YsmDxl=Qd9 znA8H6yTdH!M20&Jl&UrBf~d%I>XUN%TODe4t zcw^@MU8YlCtVlh*`Phx6E}E92E*3IMYDF^%rf<(TzGKN3F;%vZU^xOwEffxg=|t3@iWP>;5sHjqKH6_0xp zeNLLcVZ9L4Fj`PXW=CuY4#RinOsvcFvf$|SQykfT#p0w>2Vp+s#eN$#vTw+IE!+TH z|46Q{W%6XOFl$UA^tbeVh05m2-Q_YYcolhlzIsvXX3^j1YvQ&Lk%USPGfC2bHeB0Q zOJqB?Z#0Hpnh0HM0a4xvt=RWZBMbs3%hMvms=Y1P$@(wrK1c@cQb)T;g{~M8YuJ<6 z0qaJbFT@45SJus}8gn}P!o@YCgYdJCfo&1l1e@h^Dru@H1QyQ8Yz~Gpr4qv|AGx!i zc=eK$&HN|pS(aYu6o;X>s`o(o*75w}G-`DtG?g}4L4ZZh50AufgtmLn-(hML3_HqC zvhayJm0+-5+J>ktLCTQK<=Kqm#7q+akR+|cIS(MVWVPp;tE^f3!=l?JNX?aw+Dp!N z?;EUB?4GJ=ik*tjZ*3QODL)|`_rl-vb1}|?VO(mU#?re=u2>pdj`N!}oCwWVID1!P|{r8tSzpb6gh%o#vdT6{^^PYjP!+ zKEvMumTV5SvFNs7E_}8sZa`?0oGM^0u^6NY^9Z{zZb3tJ_+qu>6`smWa$W8^A0Cj$ zhUO!2AHs`LwiPd}2vBZOR!0cv6fqdnlnlVqWO8gS&$g+3j)E$Jr>&!yUV?{EEsObqcew|77FLj!U8jfs7ZK&YD@m0?T4HTp zFglH@P$~4IKt4xw_{1mFD)B10Kfdjh4+3lkq4Pq*9f8gUO@`q;MJ_BC8<3xH|GP;` zP$5G!z-IluXr|~vK9qzAUB1RU;p}z*>ClU{Ih({T0d{5ugrQpzWzs>-sV;&4-?CSiR&l$m1u<(ED8ZDj;@V!P(3%eY^2nB)#SP9X z!BxuK9sxqCW+-H za0P7ox1#M`s)^R$`Av$S1w;yk%E~D=sqx3mS0~g4*TKS<1%{@+=i1n39eG4Z;n3~M$#Ds_YAWOrLEOQ7O6`FX-zYbne5IQ}z471Q5gyU4ImISTT-=VY+ znny#YY!(V|ilElJGsdS|sW%M_Wa~O!AZu|^Ha73~=49gJ{9<-~1e4o^TzTVGn1xq~ z7D*6z`iUt_LDM(7thuiF&~bK+pG|n*UAdO*l~t^fq=jOttxUWSZQhEUJ^wWX?;Y52 zm%lzq*j*Nx2j|836HAG242^FJ3Ge6T-^Rf*dBIRPResWTDHEWgc2V{*sT1gPCa|Xk z{DjdsIC4kyIXusU2bcwa5!V}kym}%s-*zvsHPgohld=j=GCwf`h_D>E8r&Qo`y-gK zy5FaAOq(WETey5~?Oiyt6jw|eA^Y{LG18d`EX|OlWqdgGk;Zql=75!WBjO#W)m%bF z5fkO}-GGd}>G&3Vz^Y!UlFIWn-mv8;K9({OZsHcOxy1D1Rh*DjR`Bo%#|AM8=JiLv z9CcK}{BoO5Zk_knxQm}!@pSl!0)oiz@UB8pjaw zQcx#s4*D)G`QD%P$V1L_u7$&I4{gV4$zLb7EE9pdN&+-O!lf?f12%_fF1jH;w{tK%QREh z7E?kx|IDD_brxanJNOctmr-`*`O63reYs0g^!jHcSKBMz;J5(lD;p@*ux2G|*mKogZtGb?!(d#>0i! z|L{$k#c6Aw?C(*T20vAUP=%dt^7AGH9=4TIifO_IIn-}xTxx{g4vT<4XrD^TbR>t% zTW(KA;lXDMRPT(jKz&&;-CgwS7kb9ynR!Den{S&GD*5LvX28SsYX^{^71CN07SVqc zTSGy-2bWnF&|Y>Tah`+5oR-`I)J$Ooub!ZIbq<3XgB@U_)Ojs+I?9G0(s1((!UmpD zKty(8&juad4v&|yE;7h)1jTCYGtKin+T!50Z47Q!4g<7FUyp_IpK^4`z+b-b%)jrt zHMD0=UEXGQ&MtgMU>Tx@i9k^3(=iw>9qUoYY%p%YsOF9;YdZFnQg_LQgo17VfqS9<~hx}3=fR~s&Fgw^~n$^_36QCa~ zBrLAcIbVX_Ug|e-|2tRh33eKGrYh>=Xgc?wWq_brF)YiTv*(u}1qd4eyvL6bJ61fw z2m*U4mI=$O5dP5mCvGww^w{ZA!qm6OtZ)xg)}cFd6Np*=Zh@^O_vT_UNh9iL`*G4` zSXV;dB8$Es(78N$o7`+n4KIlG4edllEdh3NJ%WU!sNqB#tFZZl(Cc%Gfj!1!yme+9 z%$UJraKAwMsWCb$D4D#s{B*XEui$@`5U5S$WmY?|`KE^8_T&mn=W2O~|E*MrEwzJ>$V)Ix!Pbcf=`RC!mR@ z>tG)=n++~dF9LL+SY4)fZZsHMcXGAB<(j8dT!qW9ZDJ=vJA}AbZ);cPU7j)!d3~L+ zUWS?V7^rj@M-13HAIC$Ic9QF{OWaxsaQuRl3fW6NG%+ure|rAe{X>&EQ=!vqcj7X? z8QQwalhs_1>(s-Re(vI_z5R`HG!?&U9vlZos{rotD;(WB?G>NeI2(*1vAFvAk3iEa)ysD`?o>=eCW zAUz3Cvo)A1asstZAOA_?he(F#UctAaa?w9pz%Eo?r@MkaSn6#KCa4&Ca(0>yZh$3J z^4fEZLJRwd?^O3NA>pYB9q3_d;E;T|J49@hQh2}Mf|Srzr%7#Lt5eK6!V9dMl5 z!7CT)Xop)D@*I72jp}~i<-(LwOfnMo_~Uc|iWm)i#l+Ic4@i!~FbOAT;LN`Z!g<+I zog51C`^QREf#*)V1()WWQb((m8J-k7XNR-_JN4S!{R~`TwNYv_`eb`vtzQC`5HS6# zoax~ZeI-ce{GyP)8QGv!A5j`Q60dO*O1xL5jimKA5MJz=;oE;N0giGFZf@$wIrz3WaY=-c)cP6omrxrHrp#t`!naQ)OQ092xLqyXK$Q4CzU4^W z3LP)Xgh0|GS&rYY?CkudaIUU$@q8|^h&U99CI}{}xyX?(9oEHB*~z+V5WltvkwSpZ z0aC+46=?{;0p$oM)Z+Tv*X9%$g^g#e-5(F&U@v&6!yx9fL>sM8H(-fF`M>ic)dsnI z?zU0qn>P)GtxSN1__GP0AD=qwpcZ7X(BSt4V80xy!uXVpNzBZ+kj5CRjhNnvOH2u75+#Amumm9`3w5Ne61EgC*_2P#{8Uf6iTOPV7b z+xz{ddfBvAAsIvuRX>oYDB}`VAJpm;I4tYx_)8w(swJ^o7Y5dxn#5t)wcL>zvkQB( zO~XO~qb|XnV}n8dWOqqNT(E%aEoZE%bK6 z2Y!NiDsY=1fc~0h2u;3;_Xvlq(%7w*BaSJoCaHiGO~5Bfu#qj29)K@9#6l>)rTfKy z6M|2VVLSzjJVlnVmz}TYj27kDHiwWF-f%!vZa%EF5^D6klgb6;sBm3Y_#UHiEuOM7O(y1UuE15ip zH|n3nP>%GMhMEl@w&%|+*wo%g6gM^DlkABhI7*b^%l7UlypB~r{uaL@BJxR-R9#{Kg=VI!*&p5z99$=8J|}xG)ZmLPS0@thQg4ss7c*TnZt8^*=uj zh~41U>fba_{J+8I|1?niKQO|kOWuQF6=kNiaer^nVhU6es$Ld#oN?4KnZ1EGD^`s< z1*~}^np|r}ieIGvaa6axNUBE-h+}uJ&lx8_s5(y1*aasb-2a*mE<|4tXSA5jc#CQ~ z4vV<{ChG{RbR}r_458-zSoES=Ak6>Oy4o#;F^+RF$kj$NBiK6txw~a~CvogxgcI zk-t_77KEo`vPKB(bb+oKrWIJeJ29a!106NYtOf|h zc>Bb-qPGxfejC{tW_MQVLlLC9->tB#EqE}K_!Ety9ZDlu#heD6rfbJQ7j0f z=7FPpC!+6jh>F@KIc?8>yYK(eb=F^PziZZpU?Et51c%}d4Q{2yA-FpfmmtN97MI}e z?gV$IxVwAtqAgI`QvJTnIWzBCv(}lvAiw0ZpIi35FIaKRTdCL70d_iGJ2^*KX?M0L z9@^{gx#^w`@afVlJp50&Yb`DjwH^>3@4Q5cO0T&O^^@PQ`}I%UYo-zRn4u2B+6iNP%a|?CuTnmot_mw*xeA~C zNvUV@w+Z2qiY2tG5pnxK>f}vH0-J(nEGSZ1T|2#0=*2 zQunK&Q{tGTDW39kO^1iI+Pq1}k=>4kq>8!5p8!aYwk zA2}8CaiFcU)g5FO!#M9VDi?1PJ0YxT%_rW6KJ)$`Q=T+wZYWM{fv&B++zPEsF+PU( z+@xgs+nqA_-@R=INBg8eT4Q9EA6N>?F(+N>N_`P#_y*y1N^>H z!A7l$f_(=c?hERz`e5gqwN1aY>Kt@sr{@KtlAGVhR3oZ~Uth+)`lb0j9_sd>A)-7; zZ^7PB{zEOWd5MpgdUnngjY2uMu-**!h!{8%goe(zJSO?7?i!4rv!rSYu$|cSEu20q z8sj>Hfp{~-EB2m0>ZqXicRyUHyRJFh#tY*7I=~f=tybG>d7U@&A+}8T8e3uVJi!QR zBkCY8O`Y@yA2QZ{wzF#POlXed|s0L z=JVM5UieVmVyyo($}2Q)+!9e|*sNE9&o+vS6;2S`r_OWCm|m$872) zp$&6A+P*Tk+mSPhI$#zk1w7;jFpFq=W%lyCAeLN)VHqWwiq~L=1TDWSVA-`%IDJU( z7N2R#Qq(@43&GnT|2>pQ57{-<`OwR28=tjEM-1x?qvTzY>pk&o68J>S)ICn=sb&v5 zAKXY3epMpg`%mZDPwy$rurAjRZW(E(KRsF3i7mfkW%drRo!E4LnSszlcBbV!U2!o~ z&7BZ1wZhE{E7ECRDI`)n!V6xpoQ}Ac4#qz+p^kiJHyt~S1(V2$mmToi{y{Jnv1+Uw zVR3Ajo~E1qeVwqVb}{=XNa9Wh|As_BO4R0W0r}_7^I8fvznjy0Ki5+*@s~SL@bcU4 zG)rY&eV&SVE;6s#k{}0V4>Du^kTbP7IsqBkISa^2({K*th8O=y_dVHm|<= zPh+u&LK~9?{^;3`+J9zSy{*5F3{=>lPJ5>ME4}%uZuL!)EWHC#!{_cC3xQBm41es4C9=M6K;74ScBzU1WyaCdGyY;z}=VXJo|%K zf#lA2zl?yqYb73JFI{NFup_Ua#-!+{|%~;wJ)^|@tqKDw5 zChj_LmQ;Y}&6__Gcz}9;?ZRNPxyx2Hl^2Kg#rQxie9QZdUS^Z8kjy{%mhxsr_E9VE z{DBnOyy!^{Eskis^gmpf2EB!nkku*?`*6T3y)64c$bALAZULCRdGb_99k zt*4t1<&o>NF+FA6B2TUsTw_t&_}lG=TD>e;S#@G0;i4t^h~<3w?C@<_F8QO>p-P^) z4J%H!NRjg4#t|*AA;d2J*Ai~YF|~yJ@Y8R&DfMiTjECwV zhd!Z*##N?Y{C%p^>GIp`)wlNVJ|`*2as{j2#TRZ49I~akKP-w-lK@mMl;s=C(tg-I z5oFX<$Xlk#d85RBGOVWhLj=OKzfa)d4(tqAb7i+JrU>>btR=*!KK7eSgb?5beu`6~ z!~Hu!L8<04`L$T6(t@Dvvghv$$SQ?t1fu0!1FJY-FOHHyZR!y{SlnGv|4-PKdR+1_WX;h6Oeg`$CHQ z{R)%tpSX`oXo9Rlx!<@2ZQf#ujjmKJq2>%L{o8mgFyuVu1m`*Hk7;YP^P|%Dw|RXL`l}defGpTR zPzyt(Uh!4!7kBagkXWy$bzg+-X;h+(*m0pGxe4Cunp0fE_o(Z?{2`S4Qnrrm>%3tWkIBT^zcO*ea!+WBU2n zYQUkBtz3-KT$w(nOOQ_HOFFwvGMzhq5Zm(0BfiuU6^R=vu!S)!#q4#us26uq)Qje4 zWnJWYgz0p`tpEAc0Gw8pU~BFWFV9U~Gd4#@(&QP)&{e`CMEHNM19-1B7p zSWeIz$T5Hl#TOEsvr0+G`b4u9=%C4TpAP#nur$(q4m~^kIWC$1?n?wqQ1Krx2o^DM zn!>q4$`Yyc{n#OnH|>T!9GQ8kYSQ)s+3M)Uw(3~k8xQe~_#3l7Zy$2W2e<243wX6= zk@VHI7JQniOSzdOQ0Iy%Xkd2~Am*$gTWGg@uOtE^+z=QJ4JS_!{{f_ZoC#-y|~j55AH0Vx2KF zH76y?Pse?MQB^Q=!uMJ$G$6J)A^kE89+_qThnnNrmD5=J^Yp+MxIL@WZ7&A<|6s&V zV+;Ljsr1B%#|zGVK|LUBXn>9P5yPd!hN`G&C?&}Xq=Pz2MFz!B)z57b}>y?oCaU%+ERL;Ose&gLxS+c_? zW4X^C8FERJ?V5h%a}EB<{F6{xr1LMhn~(0esO+du{!S(ENVI>w#kaJ9PHGXv%8BiXd}YEf0YFBt8)hSeiz4R=Or z;$3X-2k@DrEou0WoidadT!x`6GtnQsdC7~ZtL=|pa*3)a# zlAd-YQkVnD+`Lw-2xGyZD&#Wa#2bGC24VofslGe9_gn+e- ztP*PawBkGpwzMc|KiVI`QSzhqCbzltcNIStPLrK7;i<_a65&Mm?F#`0$or!^i?4`$ zI>;2`kB}{$T!3I6be{QT9=VF~OOJu1yxtFhCZd>Je8jp>nMv>9gxp-lpb$tI_uAI=? z^{mYjKF)6pGiQg@6cGtMlnnB*zLyM*C=Uro;a;^qkgiX$wn00m_UGeIV!o(K-a4dB|>OR z4&b!7tYytyY!e^Vx@arZz5Au77WBGDos9i=Y4{q`+9ip9@uTq! z?qvoM1Vgyc=q8V>{IeaBWvPllLb1Q{+u<1alS!fh=l%oetN&Xc1p{;o$4*81<@vP; zmM@Iq1g_I(nM+&gB)Gu-bR~+)DV+aPrp_J#>=J8$88|%emgsPsk`hg;yvT6eEOXx$#TcX{&J6(JTBnrtO<}&H^@Zhf1`bIqgd+kvDw6Nm~~TcH}4fj zWyvEIxz<+U(w+#Ng12#X?R+)WI+ZYD`&zCao&G8bmZ)Ok=>zv>ss2?-6^IJ}_(aA)D4}XQN zQ&ne3HujU|x3`=}Iy}n!uEklovz`N*123Q0>+cdeSVzc2-K{@_wb~H1mj>VHDL{L% z>AFt#)wgL*625xdl9@KPBkRTo+O1Lt*FkuK9auv<#EFDI*SWi3tR4J!p1;k*?3;UC ze-9h$KPM;2E&7ei$7>fmQH3kw`;+-*8c;S`*FA1Ywk3DM@w2=C9|?u_s@v<=^uHHdA!N`7GH7{D@%+;z@qZ+phdf0h zVmwVDpGfC;!CEv<#16iU&VyaYiW7&jiILZOq^&tjRsgqctBJcxlE}N19x|Ms@7<1V zolQ$w8~oGk1oZa|-G5To{V)xmyFRVN#}c@?UT+m^ou$!hYcvjx(kPqUgaRpfQgSF# z;YDt8bd!=DO}nQlAKetQ{ybi+ZjCkOjfMLLKKQ$d$ua&6asoKt8jSP**88O?n18a${OoVfXkG9gPSN=Ly!`7GGS{(3C_+aG8^e&LwlXzBc)+teporrBr z7r?Gj+X*U@j2QDN%~~>kS+@IU25U%!>_I&@X2I!#?q{1I^`OJM^7V|1OhJoP!80(~ zBFO???pJ~XTsht(4;l3GTsL{xU@-uXpMlKv*|R@fZEQuQ>8;>P_S+CSb-WFuDX;y3 z3zgc7xO}|l*$@;o4Ci(Z*m98WP%OZ|)?Y+Jc=%d+rszmia&f|qc`y8M7>rpMIjaK4p z^;y0RNy}n*iOFO}7eD&pOH8s-Dmc64N2>$FckU@o7J4GLc>c3Q4tmDGl5oCkr*Aq$ zj7lEp{RvUK^PJ1{7>A_LY^|h=26J2a{EmPAXnxenl@*lsrJa6gVc^(Vua`2I@>OD^ zdFAmvh)=TZjWZoT;!VWU#8E?pg%Edo42{jEWd9%NH|hC@LI<@!l`z4EwG4K(8qW0% zwB)Qf%P)M=g95tMg0DdEc`XIa_vPPLUenCCk zqX9>otU{SqD?SwR&e|hyej)su*6-S+Qr)P3lms6kne3?d_cDGF-2at8jmav*dRG#r zJTRz_Mw-hxlE!VqDQFW+3s|ZcLJXg8M$Sd6V-*>2wB2kHk25NW`d=E`jyfg9tSh^T z1)JtZ#KLsfgcE;YvYWI9M%2DyC|&e_bo=^py8bFf&|6zm@V0K0m}O-4SMes0zfG0E zH+7=zOiRH2>|y9+e9}e{Z9hsWOz&*nD>mM-1|q14H&pK-w2F$+TPclP0GQDlbq9Y> zG5yTD_7bCjsdS}=;9Ase7^a96(c^;e)5!5$KiJ$g*%2hK<}^8o)<#u5#Y;M9VZYXp zq&cgP7dL$ru*c`*$o4cx5>+}hv|N`-NE=UZnT$>`2(aCZL+3)OX5MQ3ej6qe)tW69 zgH7~1V&$$Zr^UfyoHP^Xm-)gm}e*Gb0IUYn)Ln)oD1 z|HLOs_ypH?wOo1oAog&BFA;$$iVJ&zU%F!56kq#yFgi}l<$w`iXV*%01S5Za4G*BT zxQ~<2^y`QlGHklZTY?UzSIel}ChO=PCzAiu`P|5MD{)M3_R}sk&VA367Q%=-3k62Pye@M5fXhFK94uQ9L-jqH#**S5OgGK>HK#wOE0GtAX= z1^vi`4Z+hhIce|1Fn(JNvgS^Iiu} zZlF6CV}6e@H(%Cgq3dBVa_Y`~=1WmSbFQbMr)#rht^hkgS<@2hswiC#b_(f>EYd1hVRDvPv4r@f^M{k zy4QI2ZxdG3e=L5cb03`fbeGxNcb+Xm`#tgMrS}rr*j*^nGe|e z=K}W3oBrulc><9X5<_Q+p)Gx#D^D-be}r3UU^e~wuNUYEZiN#Z5@I36j)spTC&On5 z0~6pJp(B-8(`CjyhD`Z$3962NgvgX?#pr@^)T@EaSeqfbSSvPq?oGz2IwN|1L%hu) z&I+|#FJtq7(UbCC5`}{+#{_{G^Hc7=dd9?C;r9}^A<5`$>@*~ATGh0qINIYU*6=Te zSh$qO8!&IV8T!YDVV3NP(x#`Fw>;+3glWrON3OUmQQv5!psb1yLxu|F;?aWaIfOC- zs79TFRJ!#mhe%u7luE90Qa3~vWO7_~&PRK*e)XhDrg}Xk6FMfJ%%V2BLbXZsC&OyG>PO}FHBpcSGM<%)Nnf!MjXHuK9++IX3_qhnH6|#! z2;# zWKCF%at0`%aB)x5L2HB+9Wk_Ijpx(LS0t7c_T*8tk3CiDgl~&!+bURnej>#%7pvEl za7zMTrz)g)|4t2#pN_y1-*O0Lj49D&0fS>$2RWOR*yealQd#IB#7y|uaBtm3Li*a( zLM&4f{X;{X{p*g4CXg&C=#AgkNZH1PnG=rf@+A=G3~JM(Snht3h9i~I+9?90_9lGF zje}{qBRhN|<-!miiq06djP=HnYw;~RtiW21pLjG@nAWJ-Zc4IWG;8t4u9D!fMhUqz zB`t70nb?%?ebY19Z8gpDYWp#u#X3EuTsb23AvH{wDB2IIw41BxloH97-B@pMqsGx4 z(?aF)Q1~Io`YPh?ohFY(yR0gMPO|2gwE_pKYA2y@1fH~oNst$bR^x273=nv~%4{~$ zjVyLX^>D<@NRh&@-l|`Ul$6%`@GHKu=tkL>Q!PpnO@NfAYt`gl2Z%>9KOPB0yFKR| z2n+dgqfBp-CB-O|P`#Tjb)p;Ls^w6v`65yV?u^7#H}o6d&=LB@&0~4{QCX7?S1k%V zRrY{_dk5{%jlq1nNbeh!BhYk3aA(n-THa;(L~5;J{d45c8Ulr^C%BC7o&~4&y4WXE z4|EFe7f|xf{oE=)qtuu9CWk;i!Zsc1t0UVE1tEuelSdSzq%hi;Xtf==@(1T;bV|87 zlH|y=rU6FBDIZ|FTW#aQ$Gnu$B6zG^uaVus^7EYsH0;vcMrn7|2F}mTqS9j#jTJzj z+u=bigkI;QkoeFBwAq&MUAlHhJxuOzZ)t-B=Z585dP2Es5Dc_81}9ogkN!9Wu6Nn^eF*fOLp#C3D01?+ z88IzG7V2XT9o=^&$mz=pw5Eo<$?#(I#GN`jsP&RK;qybkVMcq zO2=b#fPe+BJ8%tzyX_(>CsS5~_0@5f0gWzVLkw?R#+b#Hv#xrA|$PEv6!`Kj{9g@a4!Aw)#W|Q>l6DMb#^)Ek|ii;C|Giz*j5UH z-GSc4o`zmgGBd+eHiuu3{BfmJNk8eht80qGo5z%&<@=bHtk~E8E-m9E*vK!Oml`Mfsle-kjsQhVnfmt{8P>fB# zSj-2R`buwe^T8PaY8ONiB6T)V;VLm#c-FGqO~S|Q*2Zz+BW@?#!udNNx9Q`g+LIrZ z!D*!pRo;-&UMd?}(wrpvbuP5@d5OMXJ4IE>pwIFtu-ht`g;*CeQL3s{la|AHS^*>r z;qfkKaf@i;jZZ@ySJA}OidqeIw7uw~yv;m7Pfx|BMexU5`R*M+-`UQm+tSaDov2~?<55N%ejq4H zDb$**lMHgx7cE|ur97ns^u4Z5HZu)wWe?LP$)Y0}Fa2SNP{Si1hYS4$X={2KLi0f? zg(JB?z1TPGrrj++f$-u0=fv+hO*qC&SEj%b+bz1L-(u06uk7NbhEp6xwegZ#+prjF zO{Meq5j7W}=Q0FSpDh8*u{?N;X!l|v)C4QH2P=Mp8~Bl{B+@a@nxAKiCkT==>7Ukv ztT5cjtaJ%A*qiScVu`|{t!5a-A^J6Np&rRLg;MVom|l_3+N z*hVv@kX337=?^R(6}qPAFJ0X~lmA;s`oPdjF!Z0}ktCe|gIq(0dH63Kc_P=~1aGf& zO#y`@r`ddrzFtP~O@KNq8CwUo+hod9(DXImM&j?%nbAc@)jV)!l3h&iuN6G$9&n#BTVT_2YnDUB2 zlhq?B8A;!=(B-zv5~^KhFGqo!ZDB7at3olyEeDQHF$DY0y2vv0jS}SXd9f>1g5Yi& zbyn1&FIC{r1;$Ekix%a^(FqM0_>Xx63tpi&s-hIE#IOHm zxSQ|1IuWARtJ%Wy88`g_WN5gd-&RHlNEqj8Rgv-rxjK-pLG?N9lzbl$$f!l36pMHY zt{DUVO|1NwSa17Ppo|&-()UX*@JDJ%~W z6Lq$OqfS6A`}ut|&EolMs)KJ+e=Zp$BU<23U@)dy!09;g zwW!{avKHD684htZ@HZF0q&X}*PRyo>uZuRr?up7)Vbt|83r&+2_Gf?}uYL`~;N}d2 zfUKn znRwRc7V%xP*ihL-7N&Zs0eXf`K#X=qMjFBl9UrY0FEDK1bXUe6sbn>F+&nWFjsnTJ z@xjZF%#BcGfJ=UA!$X)jgs)>qdDvTm)7t>6CQglPB9kYCB?iurz>A<^KWxje4G9(a zR5!i{@|3Ah&Y~nL&o{$(rjaC<90uD57~ss<4jCofNVHONkr_;IwQB;J;^V0txqF`L z9L=oM^l3l^rAFv1EZL&P5w_ZMF*j`|kjzSFWNjPVw4Et?!ZbU+JJ%QIZr^oqgYQ^5 zUy7|7KUap=w|m2J^UyK<{jRLcIkOc>_9?Kbph+p`4YRT!e#-*U{SF%5bBY>fB`V<* zzR~uc8#=D(3xQdcApg?n=eQ(#C?3)>3PPcYUW7+`?aMe5bcM2Mn+hgs&DGi|xetq- z!b?33hKjaq!$BJ3H_Xi!WZ5fdacCTKFEx{z(8}@A8tBXOY&!xoF3FRvoVi~%o*9QN z4sIz&0l=&d9JALVp~Sv>Pv^1xlmw%MO5`O!qPnPjXIO?qwgAnkex-!af7|@D7IdfS z{w*h)U}zLEH1_tj(0#DU`6SI20I`Kf@d69lV z3;oDh?O!q7-hq}Yg-v@wy@}?{u6vMYnVmVX6TxXjT+VIwmE9bSPfZnJa9=4nk_rlC z)TARf5a0$?`tsanZ|9)7TqNubgrXf(HFBXy3)wnH^JiY zk@VEI2--p8H|=Y{gslAaGKnqzR@kZ@e}`zDvoAHe5FYyE!bmu*lB|Tg0!5jEO~N5O z`!li1bPA&X4JgJe0Kwy82fRVZuQdaC>(^Ol5bFHuY>Z$*g|zU6+`#OYl4j*l;y!hD z28YZsW5OpNl|MRjm1=E;g(jl6o?77tZWKZC-fMr z(C$q!ygrPlRhO6TVz}sTSj@O)y_Q#;4bwRY@)Dkk)blf*)g*#UwYyi2rksla=1skW zt+ri}oIDJQN?37iW+B7PE$GN6Q1bp1f(gIPC@uYAh&3 z@x_ZaLXD0*&OqTgNWF00lj}e~jF@D=P>=F0LPaQ$nUSSfX$m&Nr|qPLMf&_i#PUwe zQ|PKW79AK*XD6jaLqF04`6!4dVc5xK;I*WdC!{n@UL0jM7CLUBMjytms)c^4$%~3* z$|O+kDp#B?y(bU~Rjd&)56LxxiCp>nvSR9Sq1^ASDkc|Ew5|x=1cObRof53lFCNm{V0se6)^SlpmnZR9nr!;+}D%vdftkgwAM_o=Ht*Ji&9-_Gw@%IaXFkoF9{5q_6dO-VtzB>dk!fWjmvK@adhpqLvgzgJY_vE6ik-WdAr;ub;V^;@Tf!a4CGvFEdLBX zBP$j|@1P<)ZA)p-aI=I@QaNnJ4=FXU&-B#|vcU3g&SfgAno0ox6TM^);YVc&PDQ@* zktQmfnk!_>CknJ#dE67@mzcPkU#fZP1PWcOm&-i*QHkG#nmJ)qH~gc9Zk2lHq{Nb+ z?4(kj0_Q2bXWldmau}n)bp#iwT!AR%>7lY! zWnrsUt=Dg==UcPwR~}ScWoEiDI)4>NSgaY%fuT#g>nA32f`{DyQWv;l6%+Tr#U+dw z+C&WPn!Fu&>UsVn2|^lk;9uhMM1pvlxyZ}mY30_u>#DO9O};|<6z<3A73C4Qx(YAy2^rLJ znI)C5b=6hd$7)ioEcouMicxhX714OaZaq8hRK?h@j&p6b(*Xi$htm)R{*IV}m8UTl zSvAM9E_)E7O$jqBLwPVQ&xnKgn1NPE- z24vtCCQrE)$KQ&En+lQNsxcGQkaQ*t^Aa#9LiXo48!?2S%yRraX$wwE0<*WaqSq^+=T-UIu~sWZBYLNJ%`eExk4poESDSXNjIb<$9@t#>-AQ zTqY0|xd|W?c6Ftm-+9~?oMt~Hnmu?VL*SSrzk#}PSw6#=8g5M`Tts6c_lFl9s*#7= zhGrq`#M?HIbhxweRv}##yJfWZT_AT+CI7*JoDf1P2H8L+{&ejv${B-%vFabZZ#-IZ!t!u>J>Owug0hslRA06v$<5+f8f)r>ValY74%2%mh6+kE za1w9B5;JsHzb{$KIB>|_;PiVdmYTX#x1dYa5leAy-}Oy?sGG)TA|@vu1LgC-#c^Df zp-kg;K9(u>1Q!2IMPKx7AxBa06KAagCWL$bv@d_5h+OG$+`-U0`Cym`6& zaBd-u$2F!eDM_rziPM@}lEV8MJ=Q*A1x6-|AK{B_MvI~)VmYG#qF&~F}cgbbS?G-YMi zwu!`%AxfneL{8`j5n;1NB7IJ#{EN2I{1CCe+Cg2JG26fu@?twC2m)~J{ zTU9&*G6m0;u`*nz^GvoCazP@>hXBuVix}`W`KsL}%DF;Hjl&LuBA-2SDIipxnVFA!{xTox+^}o-7)BR|@d|qqQM3xgF=4Xpx2Ww@2wfY^ z9UF*qU~*4Xr~_U=LO?LMwb!JeOe&`0-pF)P<_^X#tSNp>^>ubONv5cjhEVfcu&&Su zYZ^l*wR1fBXFc28LR~D84ZIbD+4yA;7QGSW{PSobVMo39+sWBaQ zGiURapQ9z3+V?=mDKolyI*%6aue7rm6B18L$siG%GA21@4CQ*B@fm=zQJ7A*SFm2a|_`W0K(|Ry!iQO#D$6@#VV;!b$ zFyTxw^2ThG)HH3hOum2^kBg5?$47Ei;CM^0nw9Fl9JE!~d!ViVOGR^(C8=w;6-<_Zaq%E02It{#AQOaTLn5;8tqs5(Zh zW3?J3r=qTV~h z_;L|);5kd)^R9umB8CW*8aPqRv32Q4T{fE3PLJPcxU4pMc}&S7pJx2k}sLb5UmW4WW7sQwRtVg~`Vho?%(y05_XLgSuAc97@$4 z_S-dr>e(zPl$5rPif3H&$UaHeg#J-rxxlbH=i&%WkhmW zbr#LDQ?hVWzaYK_OaqtIrKk1>yw6CI2GwMngU3-aZoVqnZfl|s=GEAsQ=pnVK{h^{ ztu^*53$~D0|Md9DvP?8sDHsbg8A%hP-5l`xLFZHYyDJ|px@%+uPp-ac*1(6Dbel>U3lI}L_Df}!ar=T-k#Vmb@v_P;8g zr^Ixe;97e!7)Yb#ki)g)Xsb^LNI-J~q!`$wh;98{%>_}s16Wiq6Iu|3Vo zq4A`wqZbt}ji~K6nJIOtc>vGYDQ2F?1t^^A&+rm0?qcn!9rxi%SkoQa5hA8My_p6t znV+c!=GIRh8LFq(uf(TV^@pL1p475;@wPZs#YTTtoL|E1%20mMJQI1tFi%L# zj~h|k5xNlKuIb36f8RmFXM$ew3DZHH4uZ7iFD4WLesIM&v~t8GV52U>=cj-!*yoe5 z)rax6A%0LqK}$KY%<}VUPnoQwQWlrI1Wwuef>Hb5q)iOHwa~th>8Qs^>c9=d^?AaY zK#r$w#6?G6es#wyEH@tOYlcqfQ7mqsScys_>vVY3M8lFBDz7=a$!grt%XGt2G-eKB zc~u&L{Kl)XMNbi&HEQ_-I{8jLBjD%a6X$bc05QlP;$r` zhvO#!K^)%s4LoPRnE%cCMlB{jN5o_VW9(a_-FWH*Edi1^X_|w;a04>+t|=NSWv9dG zjC$UzW(Fk_aRaS+@Uo4d*-Gs5rd_J!kvYd1@T!|6!XT^*VW!&%$!Vt1pE^%t_~9qA zC{<$t5K5&<{r*!~)EA{6CzR+wwsy@H&^{JpO`GgSiLQ(_G7&)&?~jI)Iav2nlgUt4 zfCGWeuE=)=-Qxx_usNQ#!1~BVOxp^jwAr0FndS5-R=(ajGc70=+o;s1Tcg! ziA7SZdY=yT4Bv?7TxU+xMzg1K8}cW%Q407$*QNCU!@7CGn;O_hHcs`%qWI??QYpNr z(n5ml`rOB4=J4*$~IP!VpH9w5s|T_Y_wH>zQ&6B2+M!yqyA&E_5buyPs!Fe!F+qR0YQ=S@g>q%+*sXV=56Hj zVRp|5@udYAItdZqyIW}DaI197li{O%ROy>(>u%&ps^AsEM0_comQYEr1%1bh8(@KX zBa8`<6kPWP#9`4EL9A45kuRn5dO^=G0Z_k3m>qKvl<;cndrJageqSE9E%T=E{t5~# z1?^75)kBd`+7M5Jb(!B~PxEq7A%?`OH8OR#Tmua3r36ewZJ1Tticv?ZrUj>I{+wAl zeEg7DCGwXyO|dd(nqv*qluu!(DTehmfA~8)vk5j@u>guLqSoyc1KO@DEdVN{k-VW^ zz76ThTr4cbHX$ho=U!9^sc93tHk(tJc9LFh$Y?qkhnp5jscodtmcI0Dn3duIlA_sJ zhOt;oP48eXomGt~Q(w4qQF{xM05NljW{_0Yn+}3P#h9eSGp(O4gamD6Fg@1`$(ugu zQ-^2+M-Iv%7j8MRHn_F?^jS#4cC$pq^_qAQV2C%it!DGA7hUY+gwY>}@Sf#2K9cMK zfu)XC+q*CvG!G7)#dn?Ourrx))$FX1=9EPJ_8p#hCG5xs`Hx=2^f(nM<+wUR#7WY? ztLx;#;Fp_lxt_^*h&+-iOR<`uO_G5(e3v_d=i^q2r079_-LEzmqpEm>LRz~A(u`Hg34mX=!MenQ-qEe!6%d-D zjD>1C^I}hyEOqSTM(8xvOsVjZIS`MoKT?tM=gVlY?G&oE6FVD=pQ@9M6l3d}Htzx~ z$9i{L8WH5yYI2@T@yq&D)1!h|g1fWjtmGvXnJs}%t(u|+UP%9)yor+v8F0Oh4SjP3 z2w&Cd*O+nV0Wg#V&z*HjY8KZ;{r!}&6?aMnKx`B%b^}f(kW4UYa^*qS+2pyKO5Bq4 zxeI49>Xl`wj_Po>`SNX+of^2rvOyH=cHh=K9frPVj8oFsR6*;a79ur)dFf2oDnp@n zSi)|EsYKS0@JayylfM(xwqW+Q(GK&ij*g1H_NmI>Js#SBn~TKcK)l5c}g6{3Eqz) zrf=rgOt`L+{t|=rT23bFq};UlL`EzymqHs?#NZO$Nm`p&$?cRdb@KX<;AQwU^fo(< z0lFFgO+;ZGnXIF|k;mLTYwqv zF^Xg)T)vIN7Ts8Y3+c!(@lR`Fl|Q0b+}0edPSqRLs!h;2y2r7YPGk^fdTzYyNKJBS z^fVo;gwYKQGVfWLA?PL=6UaCpbreBW%T73AndW;#AhHVfeSB0k-XVIkm|XmFItP43 zifl>V+GHWq!ipeLJ*Eh-q>xeKo~~|Eg=mFVkR%nk*h2H}5~+`d1wp-;dz1)eN(BMI zFamyKP^^N!tSY1TAc-uTQOMXv2t(_4u0;TDOoN_BgCUQsuT2x0aEP`qdjMe(fYqR7 z7>@l+$*VOWmSyeyJ1mcbb$#r)jNqS4<}CZ!G>XDhHR;O-UIX$XH=H^laaoenH6!Mx z{i`D#6eZ{b=MD8)LMY4iNfY(wcfl_V<5jFE-X~#PuVxrdj8bUYVVG24;oq=6zD&@T zu6C~$Z12^1w79b)a}&)vHPcAP2s$nx_r;;giPocNWW% zf`MIIgW`WJXaVkCqQFLq_KbVSJM+doHTh99bCv0<4NR=C#t@UWOs$ao6wBk8FWY@s zF3|CoQf70`YmpgBw+b$fP-*%~ozkbtI4x$*bHJ7Jbf(Oz!efTG368!lTcPP|_56u3 z7G-3y@d&mAc5I6|zsXRubM7sh7siVLaeMD-N*{5syPm}q$W)bYzr@8WbDw9xT8no|w6XkUAZ6N*-Fdb}Upyob~ty;N%+{W-IrU70veZI>vyaTcF> z<;68gsQ{wzG?7=6np9g-58ol!z{E4Bt(Xi;|IyPi2PpSPxgFEn`83znu~<#Gb98Q zI(HBAh_u9gqjJx==@yoY$u+&jYl`Pdxz?n-Pkv~L{0r+T;}qV3Ynr87Q3kPtKqM<5 zLspl_n$b?G2Y%^eN75brilGEKtUPWry=m`!@f4uQhAhaGQ4O%p`(jku>U~Xv6~$sp zj*fO74T`rQwH|6~65Gahh! zk1q$u`*XfO=khuoS8bxU`OjBxY{GRz?`QAZvAgijI<(6UC);KaINId2$ORss?;0g= zp~Q?oU>^@%;eQN`_fonTe0Y%4fFXp3Jt_)WG;}><`L}$F<7eup9Y5u*f6z{wM*>c2 zQj_Q!*3~`RKfg2Bw6{T3YsT2KG*I&OY=4jaSwO;}i|!qv=f|v+QYv2g?h(VSsgN;J z?69YLs=X^1;#=Q(mT1@#;dZY9`QhXtS^2hFo2|xN4FmI{5IleL-KT2m_6wmGM_hSV z{F>tvIMBSE1@m7Hs8Y7(7pT(_2l?@cXb>r_$G4+XR%A2n8wRYPdFV=lHX0w=T1R&L z&HAG;E;!$y075U8)6J#4(5T|p^eCj8V(3ASz{{({w(PbI?!^7julmOehiY~}e$R&N zysWNh{xjQ?go(+ZC4IZ3e4PwqREDlCXbmVVuzKY~g=b034JC&zo=o!+by8Ww?U557 z&eWgOt}MM39dI_VxLKBu;)_d@AcpO@Q{JcO-CIQ27vW#t%WNZ&w{2RybQa_&zL*zZ z<-h@Nn~aN^yIbj-f~>zQP>1YOv1p&2=9@F^YZ@P1en|&vKxajI|&L$L3^a@K_xDt385Sd};3r7StCR?QYj>5GeqmMb*#+^9;}c13y;*CFb&JeUh>W?1(ckp^ zPX?G4IZ}@N?*Sq~qC1i3fuc{J)(((=BzJ7$&D||LP zHk>|nmf+eeZ+@yG>}_weu^ba~zOyh3vy-@9+)rt;p|a0ovP}A0%WWra9je@uJZRKw zcYat4AviP?{S|+;>XOMTl~-S|IRq0c!|SKn%n18Ygnje3;b*w1$zpEHr_lMO5DTiQL8$3@k1e zH^lnJqjq*0^cG*e$b3?DE*RYfYpjdf?6TSSotHu5HT66E*ePkp4zOe4=8VhKwT4ti z^d3?BtVue;<}BFwgQ@Q~)Vps!4cgh~&&x}Mt9~2CO~*lH z-GIk^o3HS!Y%N!_C8)Zsy=MWs*EHH0Yp$(;e~=jXN`cv4Nw@=>{fLI=5pWR5cY{@;zR!e-`bU2bqC>Sv*^TCZp;9S zpLuq%=x7_%(;zW{lIE5(!w3Er$+C?qb_zi6%-gzuer*+TGXveU^I}E2W#N40YXfXh zW95#&O1TvqTTI3g$NTTBtk;C{m`iJ;W?tbImNKRQBeVe=eq3vLMw94XbW*D;=E4d z?}-0Z0O{yR?c9;Y-U<5l%+8nz%qVPO*eud}f!sT~xd{Zm^WgzRkY6(xqDRfXY`CY{ z$1}IOw~h3N!TCC$HxFE5bsh*txmKvX!XlTPtTQ^f3K&$Jr>f7S+&9J*~$c zKDmVSc-Xe}=KgN-2L^eL|~_`{e$QDN9yAL^&D%|Mg6=2naH3#=m&#cLSlG%jv?~(AISCY zPRm~>WT|`1)8G5V9=a?)`5wTX$!bE6KO?av{gqCMayz%+NglZJN>k;YYV`9c+wLde zmmi$jdk8TZ#P?p1!56(2pr2VQ`#;Y82o=|!GTnTy$|ob0y!iD0&riIagoBV&-?=XGi|v9feOa&Btgn&D_S z9WQ+i=Pf>x@Vbbjwv_mKLyQi@xnyn<2WJ<8FP3v34cdzzr-kA_5<~a+-CckFR0ng2 zR-Qc`%D0k=^$v(rJ0lbs$BUw}UCuQ1%sFhD2wWX<>@@Cl=mh_UJQWxt>#fYU^?NXB z)0ou0p)st%|FFS1+

xZv}qUiUd5mSp%g8j3FP zUQ5iH$;V`U!pq-|v~8rsF$s+s)+acD&mL7H^RM4}{HLZGT4(t*2;3 z(t{3~u`2eJQ(7&5vCK6dRyVdp*Q^7fS6L?MD4YZ*peaMkmpA~wsH2LNG@ zgTdTKD^%9k6&uhf%7kCz`IRiuJzJI*!Exy&q`eOjP&t16#R8DQ9%uTK%@Z2Aw8FLP z6WeUk8-v#TW1YQD#|M;-0hm-uKOt|HQTmNmHie+zdG`Fyef#FSUCAR^uQdt z`V?Nh#`KrQZ4I|tSMqF;P&6o0C|g$5R%BwWY*>BuD@)76IZKuaSV;Qe^m%*N?rPBgIJ~ zOFn>dJl9A|m`US_!6PeeJn1T6vdyh!n5><3VYNUAy~8!1An4 z?}~*8=ELa&rUK2?UF4v7@E>?Vd16Ty(Ll{QTmASxjW6>$KjU`ax0Z@Vt$=oLX?P3V z7@d3Ii4nl>TKA~a=g{fkGxa)Amc&F*y{6%TD#{8)spfu-&Z8pwZg6zxn7-+49=jq=YW!(B77PL`y2(kJ?n zPIe>}Qc}qOduN^fKK#)hs+V`YjK36@|7DxSC-iCB87|;@ft@;^+**0?ozgllUf)Qe z*T39agY;2oap2DGi;Mk#B#nS=_=x(Sah$$g)iP=HGA&8`56nB7D?T@60#9E~_$9XM za20n86mLhc-BkCb5QbwtK<8u+qY6TJE~{>DJesX30$SH6j;|_}*PgrT7bs=FRJW=R zzgk!Q70KQykovlbZ3z^IMYgzNnZtkdi)7khz@6uS?Fg?Ie}(LSqbsly8NT~ZvxIT4 z_5Rx2^Fi0cdp@(~rIj2lavG~Ni7beTQv`B=V~uJ<_k8Bu{^$1?i8YY2)5eR|deOx2 zTU~zZ`<$rm16l3}fDP)>tCcvvhqN$blRsoZJYz6o^#(2@)iZlN3bx-d0R(4Ce4#&T z$>Q^1p0~3xf1*?IPs1DXzs=G z3TNFk^7-=%iiUe4Ip^Z!0;oG;uxG2eP{Rxb&@djU<5Z*=c!xy)LO_LR6*HLa~>&)85SbY0p zQyVTl(HabIn)O|FAg15Ys;7~fT)^hMH^Ch3l1Tpt4fqZoG5yrzFLTM|;1=SkvwnC; z=te%7CxBnmmwRNUoFY&f&eHm#w+%I2aR)QhpnQvI9;6f711K^$uSOJFS?yQQRS}NE z``zD*FLSNIZHL5YtZ!hlMR(e#@mm^$P`dq$opT#N#D6OBq-#k=7$#{FKO6!3Z?kEU z7mXOkj2R=3JW^r%04 z7F*2gm}Y&zO)St`R_2eS81;W;*S8r^Q40W1Oi$dt6>Sp_>Ng`Ww>Tq;$KCbaI#{eX?8|L-LMSh?xk9h)WX-;1@6R( zqH}y&RTl{vXUoHxzST{E2qe6SU-`6lXJE4~S03klF%n16X^UmLx zQoKw(!hE~4#%WSJXKt~r03<)+RMbo7DO6=W`n8V?-7UW$z9ZhKcn+t9m@=Kbd`h}@ zG=U#|4n6|ZMe9I_z?S|k?Ngmbbl=9Q@27}+!AGXDJ3BS|sDk1x^$~)b@r| zX1ETcW}#5pT<}^6PNvD5R&AT)U9%B@H0ALV#3pSIL z&gR`e&jH*VKv~{QuFNkGF*04i6JRydwdCy5w78z#egnF74;gV1qnsn&P{C9I?uH^!Qq$0rXRv6+A3wF@~Q)zV^u=*B zxX>R|Wrm$0;pWcN$^!t7Bu_kFF4rFn`>bjCHM_G+H89a~sKlt)$s9E)FP;!7$H~f>zCQ1g&XO_T~;6acUDgKp9 z8;>DI%O|yD$IabYk;C{w4i~{={2>)bG(8Z%!C*tLWdjCCmDLPfy2xlT^tqJN6w`S^ z#FCv6w>;!-5r}V=7)AX$9nLV` z*vRJFZAVZ5Gq3z-)ltJIBgIck_iBD!3l>uOE*w&fFCuJ`0QZt>x+V0aw!*{3%emKp zEcF>K<3EKjoTgaHp_Rdp^P9Ap<+gNs@5C8CB@OHpUkNroDby8N@hmv59)>t?a3UR^ zZ&kRXD7VgWmnt(GHaZbw80Iqdv}|9}0%ho?fTP&u!4*d#2PV@Q7doG_VFz{;Y^m%p z*mBqNX)&t=2!sDL=D^PtL#a2p$2GXtoVG3zmgv%S@&_VF+#)VNd6jrZbm3Oj=*Rm0 z#GCFI8s*~j$e6?PVr48n$P!+xoBi_~Wjv;{?$j1ubGoG|XIgrGAo9-inJQlw&8O{cQkHuXrB%ru!0_JR< z#`qE(dD{IxJhwt+8rD#J9`G%}LQ`G9o$giAy10IXhR0vePVvG7v)((3gTHxR*sZ)* zyc#JE0q$-N@rfz$zsH6Lo7kxyi3C$zlYEDgoM@QC_1SI@?|Urw?VF3wsn=iZTUF0F z)&6ckNU+FJ>{jNLBt|~xMUUqnJn`gXl+{4{CO(|yb{wARZ^7#0>o>uvg@@O>(pQEJ zagTiEWIkFtpF>B81*}prgow1ltp}*;x#r?$jx0J>3Ge;f+q}7Sq}Tcz$I9bos=-9F zy-x@h7o|p|3m&Ez+DAcs*(}Wvy)e^d{2YrZ1;ws3Nk)_9lSO;i5Bd=jX}c-mI%CS@ zvlnUc7j?Ku7WZ&HXE;aO&8hJ0IK^ERvfj)qmDqaUFu^LuH?usH!t;1+4U2u|`W{_3GUfYi z>!+?SGOtUXPcHxP1J}{dnjIs6 z@;c{BFxuY&QJabqwQRiOemM+j!fBsD=9EksWu)(s{L!JX`ew-ZGr6J)b*|>iJ?Ko7 zWLXkEV+)`Gd=R5xD5m<`!n9NyZc-kIF_SmmUh@=S|BuDA%BBW}I>~AhX3e3UXlk@& zJEy_e<^lVG5Rig2(w_dvviqsHij2XjW0-H&d54vf=+-^Y8(h{Z1WI27k9))^$jF?z-uBg>NB#T=;~=ll!3@J_sm3EyuUTICHdt-14)` zj_LWCGD*(Ge$`ED=GLXOhiZV8j9BUh5zXXL+8&r0tSXHWc%_w&{2cPMo&3G^*FI7s zm&oej3R}gNX}K< z_kx*@leoy-!3FodTsRh{jw+gPDYWOgv(jcTr{0)2qE|^?@V#|%&soE4%_oHlg0^KN%k4#&C<221EX*BHA0lXCr)At^OLtvRGN$v>;X#| z3BdW06AZ(_x9ARz*h`LLmh z@aNyiu*@F@0g{Kb1eb6#iOOvmH9c|#&3mlIy|3~@6=OpxD~1|n9n@!^*<$l|zahN2 z=X-VCtc{prXEEp9t+>kQ3pA9U3PxtBw&8kL#`O&Ub+2CF4rmYE&Ci>XI|Q~PkoFba z!<)19O!Sy79UY=hTVy$%^Oh6+8Y4kXI4Mw`r^r-M%Y-o@=-7&_e)QN!Pz|qlJ^aP{ z;;Ot;cp*&}E&(El+Q?;u^Uj!M$Z2PGV$&z%zTe8-F!czU!*UunM!`)<4Q^CHg(VNg zD6sod6$^L5Hd#`BRy}To$%TxDNFI=?vhc`&~de&)cDwOZo+&c^MIAU|;7=WKIi zKfO&?RP|S{NUh6_YvjX@;2d-V*V00g)m&~CbE*pYxmzE~%cwFdnIrmyo;C1NsypzM zi@&L$hj1J3oiSWC*+?ct9ajpu09mF92Zj!p&RVE&w{jgbT7L`SFiNd3raSx-{7zx; z5~TtDRI1iHI4#?d@JL+~oG2?^kA3PTV!M0~PaPMte8XNNTC*2l$FJq6-TCaAKqxCKPq}XP3{N`GX!)uJ$PbGGly!f&1VP>y zwsBTIGd?*(+i!h%o4$9P8#ujaO{J|OBf$4ZK)Zk zL(a52HlWwu)Q!M~fsL@R_SEm+H}q=B+^q#ohVmotgjRXEEtM8S#D6A#RH&56e&)}@ zwSTK1+z^eL^%>T*hq7#Z4)KwdD@}S?hT`0^R|R3@-}l&TsQ59h$`oRp`tAvDY7dzM znyoM&IqI<@&e>8|d2r4wBmJYu672I#_!iM$6bQ(fUb3Noug{u=4gPL@vU&nc2H(0W z#`~{1?IO{lUC`I=@FM6sDAnx;cs)tBIf-jIr@bo9e^+UGMr*yha%1cl9Xw4D@|}5y z?a8VQviV=Opz?!J_N0%SeQ+O%MuH3dzeRbsMAeyrndc&FkBwP5yYy7+<03ncsvsvs zBx;J!Zr@-;k#as!D*c3mgJD26g#Gp7$CodWerP{S!#_=bj-)(v^--vd@!cP3cosN- ztU68zI*yC$OGq3l6V(|dVr8F+g@S#$=HPy>hX2GOR0Jie?O8lUxG+SzG9qE?p#*?DiD&VWd6@&b;9f`suStdAhoH5@f@(o3Om=4t^r>K&}!cs~P99 z1C9ylF<7UX&)AyJvQg+U4Yz1!r+5pF|Z5io6O=JP+f>^-+hI+ zE@%wZ8_gI|o{r~K1{f%czCCyK2h2nCrF5B&=Zwv8_Dw|^X95A4-!hv}NIg*8ium^>mm2=|QnYDrUv^rvRo|CQp5}Fpjy|}UgJDTdMEFUB6YcbwFclw2T)C`Io zg^Mn}NeW!H`l4dc`LbAF*ARjS)-&m?FWkzw_j$@Eb$`vb)<59+ezxNMd|!6ZD>g^+ z7VU*9_URL8kld-e-r|#+;cQGGoe_U2ZPi&8cw__IRM5}x@Pg~H-TCyfsaW{Iz?S6d zD2$B7bBj!~U>W+J#i3z#&idzVmAPHDw#|Jc^wc&qJDRwz+ddzuQ@5S2{ND?u0^sR- z_om^edT(bKeH-FB5!I;~>*dkM!S zaDI&f#yiUmx1o){Z%^5B-ziuEYZaYgs}Up9dPpehUWgb@IZDtmmDQ{lmIM_hFC(R4 z%%CdhVyd~UV$fzVQ@Fmix%*&bFucPWC(3D@9G#{&AY=<>`nrQV2Kw!({xE2%I0b7( zcS%NrBY+0CgdhJIa?yD)zjHe|5z=BMTP*MzMBtb)d&a zz9uCpqyLs$Mw{pc{ObUl3WVBzPVG*%&Uf=_q^i?PKN1pObI8++L{D^$6$uVO$HEHY zNN>omUq=3oW=8=S^Hu>gdqq^gb*gASaXop-Y`&eV1&K=K^xuW=9J+c{PAjkD*8d`*+1^PQbtM)Ad$FH1%vV?-RZG zrb%g`Ax#PFAI=XU(}7QeP{`qZ7`QZ))O%1dsfg|Z?v)55;=8$X-rK4GtZ!M%tlaZbQbdhijs#ojXD@rV@4vMr-R?}={*_aSd+X6GcJ zH#MQGY=e>4T?YdSwqYJ_;`NIYs`CsvUoPlw>wf{Dp!4G$}j*-h)>J z$eJuhdL!PF?+ww^&)nK($W~W~m8M?z9o35OQ@(qnsR25kq%~}L&^fl=#}FEOX2rlJ zymhzj7u*XimdHA>YJnAq44GqL)t&@gy?1iV-(Z8OTUzdo5r|&GaF;3cr_7l zx6;K~nmgcNwHM+ds^3eN`mzfTJh!C&e7LP{H}aa`-P(~k7E%HZ_#vERQ4dBsp+XDR zMWSdt0rBdH9YVW`m=Ddrb8Vi=FYJ#Lgj(d}j^3ouTDfNgS(KYFtI#qaNwk&hR*7R~ zbjLUj6L_ogvIy^%S?*;Bx^ip<%Bkl$+lCkU;Y9eRSQi$&ZncCg4W8H=D*Nn~+ka|j z65p^MtOGzuO{JRss#pA4)5(!{*+I9ygIKgUTVelX(^ycXvkgqDvBXXO4JWp+BK{0v zfr&D?`JG|`*?K%@yM_p`OWD+7H3}sP85)apeD+3?8S}|wEw7vw*%wA3j!Fi@+@IuB z08kISLx(N)zj&1#8P1Q9w#7Axmd<=)nL1L%wm3q8K!-yQ;U42|as*s!N%n@RzhEo^ zjs}Zk2el#L-`->4>P}9DIJgU4IVSqZ4I69P{XVG-l|;v5;dC1?knQ8;J-W=xrgFM+ zTsRBmDN{4e2ee~0eCf(!5N*R!OFtdx4M%e`_ePSFd`B3V_Z~&%v5yKrS(0QfjlW85 z-=Mc%XfmEZqQ52c31m(c{1G7l+XL+3Rp_0jagtpaV}Cg!tRCGUYW%FKX!4D(&#(gq zomhe5*%{Y(^#oztL5SAE`Rn3-_TH}HSRO}&-sWIaYs*c7#oZ?P~804CotBb+Y$;oWI2iauEoTaK` zM;y`QMU)ioJPV7e&(uDw2&GEpA0i7OkbhM+LD*`Ua%c@aP<&Bm@(H%<(xo1_nEZB< z39TE+;@hZ?=~5*Jk; zu;tj-Xwq5gk|s$ospCPgUc|LYr3USM2Z#|>rE%cB&5I8^b708fk&WEVBuL_jdD)Sj z(TjjON?p-JGavNBGM%zeQiORLG1vmVkj!c{jF&zl`nPX%lHeYs=B;Wg$YsURSVTnU zN-%>yLOXRH%WN{NT*2N_z7wg`fBUNZEf8{6=9PsJUU2bm-#_&U&6I!~W8Uth zXlCzrn!LlIlr{p(!|*MlM7b5#y(aUAKDt975;vSIN-25%u?x%Y@(#BPYO-=g?wKq~ zoJ@D0`Hh{@`Qop6y|1T>V|^y%y(35zkzG_~XBo0Wc4V+wA#{h*myScV5WEd9pRX~c z)6~k|sJh|W7*&xj_S?xa#kl=l#=kqFogwh4{w%j){iWFBD{brzja~}6F=emW)oMg= z<|IYr{eyo@Wd$1Ek*-<@XRURsxLoB5F4DEIGF^%2N{wWY3w(vZ!v{A!VBI4{noF1Q0En zTbc{>F`jjt3iuvtq7S<*-7@M9+@+PUL*|Dc|BCV_qI5pzKY~eb$`aRt|ru9l3Gy2h9{|D17KEN=Y0lEe~ciG&!-jOZTT zEH@OUW;E}zZ-JaU%QJ>g|H3V@x>e3?n?!zYCClGgWMMLS^W?AskY?Am4D=}#;w;!3 zbFvpgn?P~U!=JB#qbaY8&&v3Lhb+8(5i4^Tws&y)&nt{=xvPjZQl|rx@L)=NsQ|!tBVZ3 zk%!NmdzW=Khd}N&ayJSU_Hi_KC8JzVa{fz5p}e_Dg2dv^8SDQ6MbEq8aGm1iIK=tp zal+vLc1uG4W3s&I?w7Vm_cmsYefxLW>zGy{%^JK?%k~3ehs!JKGmydUU}@0&%5U#k zAE47`D?u)=w~ zYwi5DEOczQGUlaMU)`m~k^%h~(hT|;YF^y-LjO`osm7J~Cr<3Cud0oSbY{0d%^sSfj; zXVk8@(bGYM)T-xSQauT~ujg5AsInbq-7du#!AAJy=85Bor9^j@?!lT=trbfavyaZtt4 zVbxM9a(Vb!FAG|DJAb`GkX!8tQBEH50h3+0M&oTf^$P{#BkBs7-? z%D9R8htF**&;oXue;yCMNPqus;DnNJn2+hBk0YmURyzqL#qB@_4F(l0^{i!{aXjq5Qr_xaraLvQehPUk=38*}bOb~% z^+f4_bu4)fGn{1d^B6mLz8&zBz@c3@W<1vpVpTmVa%nnirePI67b2I!o2Vb5R|=ma zh~aKs0gdC=f#eqB)pu1!t!MmJ>ibPd z2_sz8SI4q-N45loCl_|>lH%v$MSrMUhbF+B0}{FTA}JbX$?a+!;9w7j9V(S!PBQC+ z)-my0*Hhw7?j`7XeK*vt&^1|iL~YZo`)9K;)ehH`QzE}h1bytiN^b|2f?J-w*w~ZM zGb(&qE-Lo)q{j6TdmmjsrfjsocmVE6C_1|WV6bPU+tgxExuoWti)(U-Mu>YdBDghVgQY?)VKv`>JuXo}*TYqJZoRf39doVeInGin?T4 zi2_8?hg#ktFIaB~wm;lo&NSS2qq(-&8F4rR>pb+oOE1>G(JsYVdhPPCjb+!>ZAsBebufOv;6b*U z?Y#)uY63c&67sv?#LBsU$Qaui6~>vO2k#U7mw_2KGtA9sE-5}q&-ysJ{F?%;H%`d+ zD!Po@&PSif`EhGBg8kfU3SxV5&2zjOZ{Arr)R-9dT~scp7q#AG5H23V!U+z#(~Nw2 z-ur%kzkvQFI}l7?Msgl#Dhh-a-{HGidBy6diTg+;K5q>VRahLcV_PVHd8;3XZMvF+ z_&i#*@T<0uMI}FXZGoRi`+5st9{4Q((~T#UK&X8M<}CcXSvuXDb&DtMV3X;MG~*Ap zvpWCc{7Mn7pEXe)uNMmLTMfWEy`{GS}o zs9It4)R8M!v-fVJ?*UwNGWWX;RduE*PP81Ad93lOU-z$UU>{cfjkY!dCx)s(t%6Ql zwU5>PE=c?#DLe0pW(E;g0+uA!3LA=an`XgOiW>E?u1jsJ4xVqEds07plplZbzeaNX zn)&NwXqFWyYSN|X8vGoJk>t()*YEnEYu$rJgv-K1;q1A|W5SrZ*qxE>y zM3BuZt@I-N^8wg0#dhl&#WmRZ@MCyNTF-8O_AajW<|zf2*~lZFxGhU9?%KChGc<02 zlL@b6t?=~v@s4+F_&h;;vxp}jUf=SLE3>R6VP%c33n9b7`rkleVZJcC=C8`$R+Qzn ze!5s)&&Aby7=H@|9z21W_;7V^<^LWY=LmsO2WcoLEpNm??@FketS;tI^EuXh?Hw+V z&bKAQOkNnrrEh|K`|OI`jblp~Y`Lv~q^Fc967n1;=pZ6qPc?Ac;JLenC zDP+$bOe9H$=ktPXo>GC`*zuFadiRsBkOC^syi*mMW-1&loRtr?t9lOwG{G0~q%#<| zOz%N^v4FHM)x|fonk?onaC({on4b=%o*Ya*Osvj>GJA55Ykw_I5qzJ2@@u60^4STw z7LN;qJe)`$&*et%H%AlYYo{+?Z=v;irt{I?>Kz4Uqd^pb?C(E5d5B9OTVb{aald*6 zWpX`cRIeAq4|T#kq}v7wzw!d}_v7e>GVE|0D=cUfhx9{@qU@;vzML6OD@^4p>iNMH z`aBVT%L_AXpQBJ&v*@ko9UZptTK#d$Xdt1jQ2T~|)h*i4`N_jBfhC1siKMM8)`z3U zoM#kS8or;!0+eSArqVA^v~|aj8{eRg^=O*50xd}I?-SJ)52SI2`tMe>Yi##2y;Q{sDAu}g)1Lwhv`h46Xa zI35ni{UhH-1;GFx$pVmjNOeg(?rc{ZV)}MaQ>$+~OZn&vGLR|BAaxccM6MJeI7-#O z{WYEsd{=2ex>q*ybAd!WSw>`{x`~v0`wsDhPcq6=H;B#xcGI*r@%G?W5ufarz0k{x zyc+a-pXK;NdA1X-7f%PhwFS~&TsB(EotD0M9aj~N12La=XT;!PpB-9Bz)syA3D+FM0)*pLI%^7g(LJ%T#a%mMsx{Tg(ScyRUo8FC;%f48G5yy*lbnEi z0@F?s>T=Qk^SciHZhESl-!mhY+-#7|ggrHOAl&ov`BT!WyQ1b@T?yhfETfCpXHFY! zwKripCY#djpovBGYqTNaja(OTP>$Pjt^}9|RmqB>lqo1PJ}h1#zSn<}0Xq27&*)Io zjTxj4H8zGxO24s=s_t}%b@I+6-64xl`P*N0Wy|F*#UE%lwf@z#F}roLEv$B~T+G*+ zt(SfbvARJv;GhP7OKBLbz32F&ZgjuPSJKe zBmN*3zQBWTMH^V9H(u*DWe{ilmOE|`8n!KFunnaU+g%@aleqP;8yhgavRmAl6fq`D zTov^_$H7KRBNsr>$%j!Jv5qS+Y@sFD-n7JYLN8tT&Y)PY|JnWZ{$ewXumqlOOKOkw zn1(WJjTk+Dmry}gq==8u&WPB}=Tk|Djy1N+>AdUkgYQt>F*SedT7Io;`=;Y&p*_-8 zUEc#Hw-WTP-pq>yGAhI76v_M|O-1GKz8|YV!q;tNF#mVQ}Ny0SgcoK%M2K%4YN_ z_c&@%&GZoubTwKWjmo-$ls-Ae`A5#!a@ouYw6ARH#^kt@AWD$ew+gEAeT&Fgz0W>5yHk z8N#lMfAILU8%Q8#>(m$SR0hsT?=nAK3Ah|afu*|m&7cV?lrrveeHT*N5~Y&1g0O1u zS!r6av47B+eY|-eK%~bMT9SMP!f?il{eV0A5_!X;P}X7~BtwMqR=_X2p-COd3X@M@ zf?Y7Wk3qA~dpO&p7TfC@@ieWQGv7yq2L4Kef|pu;Akts@CUu5zR5$f}JL-qF9aW~* zH_#qnf+NAaI6!0e*{DK$|*4cy- z%B}kE+%0t5W4YPey>=6opG{|R;q$(fPx&V-xGs6yc$lyMmEkpkD9(&JvFFu?6C4pL zdhBt(KcIrope{5mwm+C|LL4;RSv*oT!$#z^`>9U>c)KXs?3lr8zY6Q_*@+)a&OYL{ zP-F1IZ15$my?myOc%pAy#JH78a?!7CV_aqB>k0OaVNl7+__sku{hsl67}I?gShLg7k?FW2Td7j2SWD9aRbu5a|( z;OzFKwv;u_x-AzOU8{5v~%Hb$+_mA3ivYXeWr+g7} z!_Ci_kH^iHc;i^m&q&LRl9B8)47u<&GC}R`3axx1XtZ0HHk+hhS4^hYMWno&6cP&T z^df#h|K`G5>#*Q6iTHbKqM&zLV8%JgnJj_oqKOrWKR?EfNukEbl>ahW&$xxK0e_~$xLqL5u^kto~)Wjs=mR;Zov*D{g zditOK_hs+Yz;`)8<|peGu8w+x!dsAVaamuSD1C6#fZBjmhFL}MO}47hwPucHd_ zYNr;Y+KYvlpdQ+sUp}e(D!k5jC~|>eXCmgn4iMjVjAwI zEEmx?u}1?>oc@&`zgZE@Sf1g~*!S4s_tTvI0m z{l*;fehX7CFe1nYR8AZUjaOc878Lvk-&OnOq`yA$JFxf_iJd_n#8|$dLs_Qma5YER z+w`OaQ0#q;XxA1O*BUJ$S~Ip3Dc;v+5}Tb9hsVaz{_dTaO^r9GmPl=gw8qntqQl#m z_tN_5VWUK!&3fH!{11{)yHzz@zoV_O%ZPwn=#}Mw@B964(?G#P->~FE!z5YEON7hD zY8>N4Y|zPoH&rH+VCv)G$GKdYef0zZD{=FGf2ikvA?cyFa!scd4OW}eVDkZ*)N<#_ajCjF7-G`@5*sKK1i zInH{Vxd1IxSEMbqAx&_~r72K=Z#=ZjDH~#=Q9!md5kF4Ex$hY`E%ubNp=8KQrZB$ zXY1r%9M^2FgddxFkv)4r056{Esa-dm?18=jf^P-RRRvt8c2quIk;mw7POK>&gJ9a< zSE0Qg>LtOaRlr~~A=g0X=~)#Z`c=)V{bE(DW9TEmBg1a(kJ`uO%^jcKU(ca7{V=6z zWHtMT7%lzbN?S>WEOWf0Z-1*@hAJSgU&gsSFS8+M)c{viv%K&ozAzjc1J@1D>V>}O z1j#{!-J1PxYsK3nLx(Z67an4=v?x}WBipX^T&Bd4+$X^utd*kgq9 z;_KMo|2;iiyBCqXS0Ho?+#=S5y8b`5&au0)rfsx$$F^XanWd4cUSq7kh+Qe^= z8(6zkM&MMFlvDRl%+QU%g+gcs9*_N3yA={$ea+FJr2^g7G5`B{%kn>Z>+m#nuspNp= zJ(BU%oOt&!XgW}SeuMXoJuh2^sYyyG#k}(y2URP#BMow;%otcQ*C_ZG7C~-LQrFTg zxy3yx){(6rRI+BJvF67?>DJUv@MTSs`b>G-XAars#5>T_h1V8iEgK3gM{v2c@av-f zqg?t^OMrSEX5D8VTK$Ia?W`%NGkt5XQ3*@I{P821HTGemyiS;kwz!npL*lREy_j(h zAk`{^#L{{+|6Aa={4Vu&#FqXc)R6Org8D2m6W`?DoIzkP!9#=SuQHCGxg1VS;0Loo zdd6f|eC=Wu2_ch>S#X0{FWUSj%@f+o9TUwLk}N?Jh{@v2t=LZVLJd#OyHi$@Gwll= z+s+jo54Hz3xGV2u#o(uvBQn`jc)Xn==u(*X#wjd9>hh*y^6xVBC6;dX<%ER!|IX+d z2od=og~s||X$8SpDyvM!ryy&1a!p_2(^iM>QQOlc@o30)qUJqS^KXk*?*55Ln#yer z!zp_rn^s+!O1=~B9)8P*MraT!Gf5?%7joc!x@m!Ibkr!v6wOXYMA1v2hwYGL>2iq7kEJv5-liB&#g@zsF3Mry8Y3UJQh%vRCm$NrRinf?{K5y51m!}jY z2fghh?~c`Sh%E=l^X! zjLLXy&jsUm(R24`{{WBre{-X^zy;ws5*Fg4fJnydv1!osj{S7s0MbUf47&sg+E;d@+Bg#2O{^pD$@x{W5>x2jFkKM zupKh5nGPsA7JpM(EwnxMG3_QC{qA9GwJsuE%$VJwhXZo3;dkY9C+k#|&Vo}kJM917 z6gBl=FD#)TFkz*C4+qS7-0FX0yXany7as~B(`VYF@k2)(x&=Jm>>55WJ6F)ARS!D$B!}-`J9f*^9|muKLGEZ(^TmZZ zrK2RpT!c(7!QcPi3pJ;MaMaVW-|{~&`jbYXaJQ9+He(b>dPK4fYS0)>tjsIqmT%TGCdqL zdi^aZ^9-~b0qTs0(hESC<4Aj}`_=d3zNe|K_>2n~GunB{WN`%?Vw8J&3rsZL~Y++gySG4>3wjLi9s!Ix(dGPO>X z19_Bhcq39uqT9y#;8b`+rf+aI1D32(vkS@JW1zD`;`!V z-j}>^ETKJe`YbyVCU{B$0;h5Ie1dW6AW z#`b;Zckp3X?IpchB9*tx68^m^Q_oZ=Q1L5sC|!f~>3l?&m!4D$lbmd8}ISY*!fld!Gfw&E~W&A{fXkNR_J% zGF;mm0G7`9jE)*5lihQ$r{_wg%DwGPo>JVW7>dYMXrE=#h`qLanImv6>!kPaN&l2i0Mg?`D;JWJj*>CH&$&x^@Bt5?ad2! z()dO6$V-{ZCcS>cQyP03)J}oZ^nY@xULH%}a}Oaa_B-)0$7Au1M7Hs8hsvk5#%WPq zOMb&*&awxp5^-!}WMu_W#wr3e-$0e)4{B8!?~vnH;OFoIW9L>v?Rt|TRI0=a*wP?` zK7rte?d}4qXG|D`jx4vc>v;-~?ZPu}+NC%}+EkUg1ngB^)2vd@gBzdNcGz16F#ljf zuY4jNky-Nuw&{hUWcdUV9MlEzWl?4a#OJqUK9sp;O9Q@3LG7ikMR)Sv=vkY;MbTD` ziJLYzf4``-IZEH_g+2mr0`^|rfL$AVsw7J_lJs(+z>ee(rwEr=oq8!IN>y#lOwjS;5!5`BxeRNB_XMjw^`TA1RP# zfpbA{JchAbpry6*=+&AJ+0+9+;d5k?_1=!F1;zVWc(F=7g;W=Mrdioccq6r<8HcM)@$_P! zXF(*b&J_IrDTStgltT5o=rQqtrXPAC|No*FGTc;DRxpOcVi$!yZ{SQrX+-g{JX08m z#Cc#itjkZ8FUjk~j5+5py$1Q@qM0pG*q*;GnTHY!>o`%g@3&udm&XOYrZU@Jvb%ad zuk3D|Boru;{<+N}Hv@`PHF_WsgPYh_sjb%<5kp*fe+co?y_MjmJb(IZ+UUunIT1JKyI#8 zh+I}Ct@T^$Mt>^iAQZ0yM29(=9)=ovA8X{PyCH1am=frj3$q?1j_W&F!}shQ2J0!W@BM#E`Icyc z@hLCZ_3D%SxgS@F05wM=Hs?m<<%0HFKmceO^aWRxIi1~EknD_@Va_^3pyjxpA+2Br z^xT zH;=(wcAI*?HDS1+SZ{&Wo(J98bd-R~@~D&4kc-3p`i6p3B5Z{H$uZ;}b_q-+-F(M{ zvC9bG*I*0i@4nWvx~tyabN6^Q$gKM8SL(gPYcBUoLBd~LRI?B>hF09#umM|bvxusB ze$`I0(%%^>k-Hff?&|HauQw{5 zI7Tk2Ou;YrX+#rARZ=K*o`J()cqX#4P%T_+>tBd+z8J0=kQu7ozeBy`elh~rRljU* z%It|!w#o$hPkIr&nFNn$F9!#z2|fJv!wA0RH0($}=IBAb@7t*2R`P`1mztySDk>Pi z5oqL{V!TVx4^sMgxR&EJCc0}!(=21Mt3$kLtW%Rih^rJMPGJw(UYU74e^i#8qRsu& z^~W9uKMa)t=KFFEo(w}xVu(m;T7^Q8{mx3YvVkm`?{xm&1q3B5%&3>j0G#2$F+l4w z!~SChp!G2P8;TSK0PMDOdS@l%E@O&GjP(%&{WwW7jtN+5aRFPT4!i8ByjCivv+ z4SI!d9c_?r>bb)5@*~X9!_I!3E;Pb(x*1h#_p$21-nJcq?HsQ3wG)rb;+0|Crx-gc zulkETYAOyrEIj$5yY_lxZD$}FErW(n`+2gypDVgKl0LIgS79ulLs-Sai;F|Xk}4yU z*2Sb$AM?IP1E!5HBgoUvu^vAa`lqQOw6F4eH2U(;X)Ghe1<+(S5ucDb8ZVs+1tMd? z6@>dXO>b%DL8~ERDM+qnWPb9J-GCZ$0w$V!T|?9|McxqFa_XY6#oCKi9K$QuS#}ta zisaUWM}(f1LED`WiL-w|oW{8PjnZ}?>mB-*#iHb<^Od`jx&!$MY(@F<+-B_!F_radx*Ao!^@CB>wp;M&u&fLSw8S?@%!xp*u z5CIB!WdJ@gWTEx#Ya;F2lN~LJk>NqyYD!2%vQcY!R}ihihjD^B=hOb2zM#kV?cN1& z5?u(kfrdeD9*sM%@w|-8yUCvn-8qQZTD19C_qQ8H?WJV#&$Ycp*R);#ZO+^#uUfie zGLp~A)z+hTt!ljzpmDnETB=i}b1MEJ)G`d`n(j|+g z#ApKFrYI|{sa#>*fpBv^v$UM+(cTUxxb+KP_yvg5e@D4W9@}PfmhIjoAw=Ms#OAk9Di* zs%WPzPHZ6&?hG7(n`lREdOYYJ5vFQWRz6BZ$ji|R6?i!)T4b4CLK(r*6F&}ws=9$TwN zO1!pDX>G>YMGAmp;Fb%)m1jLYK-U&tvL%O8f0C4vRxRx}yP^YYVLz;L#nk)R+WhxM z#|ZoJR0mKlIyEIFsbJys*{;MLI4S3Hox=~l_5kyn&_mK!dkBGDn=_@6Phg6^ZWX`G z0hDiLv^tXXGCh+!=>p-K0LYD283Rsl)d#kYIo6R3N-<{zE)`QHz3mO^(o zehs5++hgr`=q5d$N%$aPioVtCgU?f|J6K&_-!}drL4CPVg1&r+-VmU`k;DJOl;@(l zdqxFXSM14RdkvvL@pp}R98H{kJ50S)VdeIaK|}W$VMY=RHMBW`0L>83KTOd?A2(8v zgR&g$)7#UzDfI_ZgyTx5r*wm-)ZE#nHkrzdVA1IC+JC?|z}JRe*gp?nB!bC#kug9} zwAoi} z;zhBnHE9s5(#(|wYqY~yzhQb8?CS_k7HX4S5I0tSR4z{>N`xU{+-1Fd^g@|tMFzSioRvl=va#N= zdrpbY915GGJkatg0dlNsjk#iIN!x`qo-y-(S5m3PU6R{q7ju_b1`N7yz}Wh(e(BQf z1bv;BC(&E)5w8-wVAVFlNCjM_H5{N%m$k#D%PcJT^(Gme*Uf-_WIMr|mh%VK4@LX` zHEr1|yScM9OhHU1^WqiBcIJrYlbmYrj)K9)^WQitS{)ih(rfeWmo;*QU*;whRjod@ zcbrqP_u6F~nE5CosP4>+M!Sd?o-thP*cLFO8It95ZW}H(L76N*+4+U=%-@;vM@qsP zggfO_FA=LXT<1sQxaXgi?VkX=~JPHp39TrbhN4nPH54 z9#1@GieXgsMgfa3i}IYvzI;WobCYqK-3r#o6n;`J)FbHz>0?s0s0ksDbF)9NA#aCU zil7T`Z_*7vYv3J4%I%s363oFZrN<+;*i(pEKvo{>$=WOdMSP|>z;+PYdeye^H~HP{ zDI4B|DpQ5l#Oyba;XbQA{x8`&e9Sy|;1`+ry-96uI_|bfEn&tLtap0;+jzzvx*lMi z2@TveXSyp`73q`5i6A#F^1v&Hdr}3QcI@mBaje1#`yw5)@VDCAEg6M=rU}U{*{7YO zP0hZ}@!}%^_GAtEvKMxfrkC+ADt?N0ZrstK$%~*daRJHh+Z{!h61FbR?TZ;pLvJ<2 z|M+Ri+o1Fi?ZVpnPf7Xn7KSytof*Lu||qqn&-|mF*VW?a*(b z(v*i}gl={fyI_!o$H&3Na-P)bGf7Kjy|C^a;r;iURur+0OV2Vi3!*zqQ!;A& zU-;~oe?ph(*Z-tA0D< z5@$g!-t3dXbgi-k$YJ*_o^-YznIdWkvOmmYzU{wRc|LZC&u+SYSBg*ea z5bbPcCbujr#H!n9o15B+Kpc`T9F;B`9{9Yq`EonWHKRd3%bG<#$4UQ1QaP&b3g*Mg zod~_QiIf}73t5X7YQOEaA(Q&#^T*#{g?D#WjFYZaXsE(<;1Z;dkZE^7```4Nu0yDLNRK{Jvk=4Ojdqs*l$zH#6&>f1V}E>ViZ z0q7W0qTj(QmhXZ+D|%H0k-(c+{W&^vH|5+fpB`n4l|J?Sj`1OS^@BR+8liMq74y#b?Y) z@K87YUDJI68|<$|Qn$uQDBT=6$rpPo-AqQGFUx1ul&MfNWxC)^zN=}&_lWbErlvT= zYWhR^)SZuVyNEHr68Q7ycX_B(ng4P~-V@VYg^;gEjme1A1o>8>EAc5oU%UgaI=~ae z4>V9jG$KZ9eSx|W9{6`COU=zZVjvHjqETgp8DvnO`WAXx$>5)Ojoy9>Aa;i7d2rUa znAqfc`Zrt}QlwR)eilvN^R2xaz}dHH%@n>Q<0Iz=?Ah&xqzjxD&M0ngjhm7q% zNp+>@`e4oelEtV$&c|Qu@I+Pi9CN=na!-}d87zsau7EhAZl7Vh_72uu@%+~TC`BhS z{^V9Ys84LEENM{CM1^Dih_aLL9q2)HAm;n~4yrk!DOTe%dCR9e!2@r0+cz4=V4Nnm z#Z4AyX!+r3*d?287^>y;(_o3ko_yg-id*xrb%wu>sChZ=JK@9cGz85TDh(%VN*c(WFUe}fzTF89ffg6vQxReT|5dk&ljAe>>?l2DEt*ij_;ADxC) zaYJ3a^j012sYL#R;60>TD81t05ovw=uGq$1*6jKGYk3=ZHwbXPz=q z%e_at(+mDIr@uuA6ZM0%N7uFLU4Km@expy(eGxEZ&=v6(=L{*3vbHz0Z@u z0F3%@DIz_UUDb0SHbgL;wf)Gs>8pf__LGEGY2ut3lVFik=qBF6#FtvKL*O%`waoqz z!FIV7bR}iI3NPj#;3XL^^1Z-n`KJH`-umtX`y-}$&M%M2OmSpllB}k0Fb4U@%Mf8yS)Z31k#9zMO~eW-+zq~gaLH;!;mkl0xmDg}b5$m`p2X>cQ+7cV$&3pv zmb!`9T8rk;^@u}Hknn`i&lBrI@SCiuN3=^k2Ovfe-_l$Wf3Ru7NE>53qz=fRPv23) zsd$64G$8h67?78}3vFceFXs&>1#_TjJ78WZ`GJd;>YEANDNxc zJq3VA%nSPDxEle`1j4LN+v>E}Ca7T_QRdUxKvHjO#yD(;d^it+4j|AB7Ukqk@P%dP3xz&CCDLc_o|aK!9&du4Yf2lvR|~suFZD-nKr%3=8_7SDY4KVX3fo& zeQ9!wgz+x5!+8EtL#08s5iK!aTfo;rT!=E>revtXH4z*NCcykeXh>aSp*Xpm_|Q6^ z?3Vkd;whZBGq&P`w4gs`&FzMPtQ_h4`B$Z~> z&;q1!7q=+M8dngFggi1DpUIEeA?n~5j3sa9!kQ=Z%r|;Ox>j+ANdsezrrgQX%hm&n zk68saH0Cw$JC+9uB_9(Sg=QJ}D2MJr)HEg!#x`Tbc-pcV{z$BivB=-98T|5&dQH~? zXimIR9U%F}jtg0^mU%P*$#`IBE%A}l1#bWM zkbT(s4T`%B{@eV2#-Qc+U+Cq!=ug*UUq56(^#3(;|EUrGe`8P*c^9S0>VQu?Vd$ zURU&cl7G@!Kh_HdtNv7h)!at=a3g1G@ zKC9uD-xO-|j^I@Pu%EwR1-(@;KA$_<%p(Ib$0?tl_)yN_d)_b(w=d z;lCYdRM+YWUqhJ{aTsktR8M>LyT)oIx20js2*lk+5$PR`9SLooNp0d2cy-cq6yTNL z4`yQ?a}!2^4F)$UhEns(Ef^l*(&*E2Rz7c(bzENUwB_PExQXrzU5y!TuwD;h$v>f5 zXX~>bkXf9u`xgd*VCCm>p)-fnD#Zjf-;GOLbaSh=SzTX(NH@}XhaX~Wo^0TmxHr%hmo^TJ`K5wS^I%kO3!~P5ek-hBD_v$-gqeS< z^NFaFAoW>F)@-NbCvRZ(hr#+;+7Yw62P@Qj+T2OMRf|dAs3UI%8QVa~vr<|k5B%pW z;ZwSAwe%z88(aG)xOzFTflGL|F+?kLcE`l^N-5v!Z3>pAWazuhk?VHP6(-o`9Xo4Y z%eg$TiX=`k!%4A!LU7*y$d84~=|zOM3Ia0y#N>1w*WKStsnoNVv-^i9 zIfaz*7%AaRG@Wz~ur|Pu+pr%DpX05)N}VM{QG)fAyhRGj-0~HWf%uLRMO=qJC?)z_eS9rc<3cTV9ed+ap)egA=Nj@LqGHAd3`Z~peuM%b&lGt4kPo4+uz zuy=03Qe+FzR7(%RoC|qdFa%NdSydqHcsLVSVQR){-`d>5*pE8LuZ6ocwm+-LJN?7Ko|c=J zsbzpS?TR-OUQ6ZgJNx)Ckd{F`F2dF|@awfc&MZeGL=Jxc6Up$QSDipW=GjNnsIxyn zbiqK;c>{_C3&N8W5)ZTaI1Q%WDQL!_i^!lkXdO=9DPxARo1&)Agqw3Yfhv?1Sj7?( zZC#R8;w>ui(f+fL^%T4AoA;-xFNF7ZnIST3Efh^k25J*3^9UJ0@`hTQux~(DCoK-> zXZ7ob*zkMluYOdD0(*SDFN>8L&(IWCVcwZ%{%SuIRp)~gv;xeg3a)d+_&Vw*@kiMKBT)GRs~^= zXj$Dk4Z0S(Bhgk%T0{dvT7&viGpI!Nmow z?pFB%KZx-VbcQ-(ABf@(riAlJ0`ZnSnQ3LRwL+%OpLRXeOH~h{_osenMJXF+;5}l+ z0tR}v`e!%l^}o_@?IHh8&(;P&ozJd@_Y|4YYk$0;x{CAi|dNDQTsdenl zt#%&FGUK`oS|v`USu+Be505Jp?i=S4{*|9_&Ez5iK#2ZOYxWyejY-)1diJV|^DI~{ z7ABNfTFqW>qSGF?M|QOa<5*Xo59^q|=ww5Zo@~CwphyO!d6*8pc3Sbh^ySWD*Rd$Co zeaNvucyEr0qJxlcsg#V(qB8bwa9fCzRsL7VpI=Rl_B9jujxZ`ic_2EU`W7gpDV}25 zqvR47$DB4E$2@&g)aYgJe~(M8Xr1ucZ(V>dc-cBM#C9JY$Ockc(3He1VQ_YdBJrSZ zV4_|28FxWZMAb7x3YY6I-HB_A+~|#{(k2c{n2CeKD*6-JBq&8P6}PZk^!L_OmGy5P~~N_tklG5~r& zNqE>D@?B#3Q$?dlz+xgARf%{)Cu}P4ilGlceh-=gAW$!_M&JaWYyjleaEbuXbget_3d+Yo`%bV)6 z(ZHp>&`*$_$0dI)bfRgg*UIv00q98px{fh&{uDW4HhH*R!)g>ENPe&dmOnINN{pH8fT&aGk=#if3>|@K5oBny}q)A5QrNi>| z6%8qwPSDa6_iI)C7-|cC{kcH}(#vy?uRta`7>d=`IWJbP^s>|<{*V~pn>Nc*%t`x% zbM7iQpub$p)5YI=&HF_FSnVG+HD}S2Ua+grnJ|_jp}o`~V_q}0>78zV-Md|UG&y*> z$f$8e=I4?bfWzzsF9AMj_~O4n#LeWuioMwH{Yk*1s)e~KAz*?c$1|b>!0%n8?GE9n zs?bLGlF=-)z{`H=QK*>;;yM6c8D^T_I3X?D5KKC($)<`5)cx4=<27gybx~b=X?T<_ zX9ltEVX4sdgkRy-#s}G)%NL3WDuk8o-Xb0ZD7<<{EdFlEJxVQ3$cAb0I^v)WlCi9b zLb#lq{q8fq6YR{YKDw!aL}lG?f+?glZ?Y0PKrZLlg2=0%857y5BQkKwLWfy2ncDto z7cwcGG{t7_Yvp?dpBz8fCN5Z1A?@`T6@2WX z#i>s9{5LkyCc{)VRMGvK$Hv1b=#!G&rCXYg)b-R4Tzx$N!W~jWjGDd8V zK%$mvb*6fSq0o&R&-`WD^PYt|Jwufxg#mv){G6c`Os(jCFWL#M+!Fr>qF|8jwT6V@ zJAeo=;mjC2Vka!gd_rlvEx`U}eAmrmEvTnv>{_SwF7jat@qP`Qw2+);94;vLDVj2A zK7_aAkY53@RZu)5hBSGpGSjb;<(UkZSiNA&jSiE-Fm3!pJ+1U034dISO2sbPk1<`A zDJabD_^S$ASpEw?@(4{&7E|@QG)}V{?5iTG zm;0LGWW)d-xdfr~1h-;Wwx8eoLNHk%>7JuJ^z+w7l$leZ?5Ym6^}Xt&f`))XD7tdl z-6$vj3X6ha7+ncO!&j=MD?SXd9oR!1*?&)16~vO)+HxUHY5Y5ue03W-<}479y-PrB zsz?jv#`r=P&V*8C?5U0jHi|^Oy-ul0QrK2B>;Q&({(hSv8ANR7j-V566!Ql-0M08p zkoIL@__PTae;@1)N!ByLdDolCgODc^*9V`6_c5lUP$T&wDs`T>r>@2MWm$(tuJ87W z!$liWV~yP+WH@scLtq!w;IN=;`2JL;_oh-;e?od3e$43sTQJNyFtunfDDOH)Z|I7Z zy=OaO#f1!R3mR5XRuMA%3Mf}W1RHMogDGKi|8>YB-TFr5IqyW6L%iw}1E8pCE5J{N z`bK|fOðTz(!%Pjp?{+)nP6&^56=5z9|pury)jLhO)}chL_G%Ek1hAlw@sz998> z;un(G99Pugh6nH;QF|czPA`@}LimlOl$TZuJM(3AJ@Slz+|G21b_L6(&s=}*2BpEo znDd7T5s;iLgI$Z>&qznzJUGfN%Y+!}FQ4UBu{y?b)M^MHAKzAR4YWao#a$9=sYA#= z$JaFo)Wv?i=jyYQlTb93n35e{r+h?>56Nlm zNLzX%CJ1&(TnOit9CsvsZAn)hQT_%s7c*XFTwEim3#K>HvJ2}xbgS|4BKQ+KdBX0G z^_VQ4?SY_tFD-b4@k?R{Vn0W})5UmLpArR0-alXJ)80v+Eul^t&cI8Buo*7o zyvc6nJ4Su~b$%=Q^uaJ*|7eS)8E;@bez-#XGan>?_f}GQnZtBUQssR94dVBHwjD%^ zvAjtBxb~&R#qisbrAnj%Ws~L?E9>+tG5-C~SPON+HD^uCla0-&?~-P+>V-#rJU4Ro zgn6%MOzM)77cSWs)q}f_@6jDKB~%sYVU-bVOGV^OMH3R~W4b7eE|nv4UW|T8td4T} z#CTD5Gag$x`wFB=>I=^~vl$lFGDETP`*m@tBZ9-Q)Mrp`=~z)#SL@$sJ~-TkvVnmJ zC1_`^e3gk(1=MB-P74dQiH27-Z(24menAwHW01MO+H>+(@3-F}vGsoHEMnaeRE>!= zEsSA2rlPI0fJ9a0wbIX^#*pJcJJ~8F!w2!`z%7aci%>E@4((FAZz*Zsn24R z8@%c)kk&(mE$O8gX)$(#f(t`1?h?#Yae~^Z6pGg<{_q((*SjraJmyr`lsw1yco;j0 zRF(Quuq@^0#B2qKH$UOr{0cuvvbpR9RWC@t*Rc1zCepOW>)wOzu<~_2rEx`TRjb{H^(5Rh6o-y+=+RZTCuF4Hi`X0s?yu^1 zVjVVRWkAVoWox{DL-hY8_cm_%2gTsxXkKmw76awPqoILcmo>Bh-mmKd%pl%@vY6$z zZO~+GRNKBBL)A3If$|z%hZ5&4LNdv0=g=YB8ph8|U8}6P&`T+#(0MZPMJLOXAu!);P9^^Qy9sfDM$Gt^Dw9Dm zb@B#l<(FD{$%<;`%J{3KoD#Gb{0%+2}P($|z5Id3UQSbzriXty}7 zKjV?PMWMh!C3OPCwRR@Cj=T zD%m>SQ<$5GAqr)qQ4;dA!u@_nMIGWr?m51mXjSsqYH;__Ykbr=A z5xe3LwVUsoWEAg=vd?Bz;~|l41h{h}Qx~!>5s&}q6ViLZ#Xh914Vc2gZ$w%K@kOsF z&nH^SuQp{`#vZIc; z^0JCAA*9_SDh?sg%UXguwU>{?bw?3db`7G$X4`MUK6Cz^CEsZJ6Ge`E zB0H$qd7#Z}a9-(-IyGCK38%6}k6yA(vg0Ia6<6>cUhAF=Z zz5RXv+>}Y09x4^!)4Jb0%$vT38Zl17(HC?;Y)@h?e;EIv$%(-j|B!ipXesX=8yI!G z+^9BKo*nJxn45EIzcON=+91>onQ#+|2CLX0 zr(WK4j+j9m)KwnnASavZ3c7tOp&N|O>{bzl?bS!`N4rjOq*{OMEb(av;p7(*pv%e? zRbhUR@}PQeMe?AuCiBg#g8aAs7y3FMT5jWYCq(XWXWl(`ucj*ge^HLLbJ~Tu0nh0~ ze2uxk=_-|}GM1?iOYT|NP4kPwaM||Tl33Hl>`HKhuTom>jyidlZ3`bv8gHjfKVV{W z@Uz-VvJ1mfsdQ<5yud~#$B%4YsVZwbL(8}1gbY@B?JNEwIZ$`=4(YwD&H-85b;Qt5 zst@TO17^i+Iqu0bzdU|!mMlqnE^qo7_PDzd2i7mDn*owqz4NXDLC!1as*2L+pnK;T zgSmzT82xwZoZF#qm*l1Tw2hg|lxFE}UMJy=)f>pA0AiEPmRwzH>pxJTGCwj+(%5fs zgPgw%wEr8o$Nr4lH+9e_KC2n<(Eb09X}fr)hkLa7#&CmO{Y}ys%b3{n7)wI>Ws*t# zl*vrEb$B=uv?d+86_oh8C;dN$$`UksaeWoVNa-joX>FM+-Nxh=s;AWb4nKwNgZD4) zIJ_A?r=1l?y%U$Mm%+`!!Qzb#mEB1(yMS)(A2S*kJB+p76Jdbp)(+|^o*o_3^c#IZ zBxDK5mHd~wf%l-0{fS|K__i)c3S|*kswpQlT)KnwZd8>KxEetQ1we*Z-QX30Hz32D z;C7lNs28q5EFel>9cuCkmy*fEvRPgJfZBCGZ$KwN-OCrEYuvv&7=2_=9Wkt0X}1{?#ABv-Ca1Kst@ zDnKft1FW>{?cM~b15IsDeBuv1b6o>EmP5`ddd@+_ebW*MC~1U;#w}04P+G)?&Vb6L zejzH?Mij7YaBpJQLEmt1$PbKzj2}6D_8+DOYrlTJ!W-f)ll4N}uFpXgD){iEhvII& zxyDgb;!}!WXH3&O&)t3wUg2kni30O-sLy7k>dA9IyegtMFz5+)tFFfZ`p#j);dj2| zIxh%&MpQ96ux8_a=O?Jb@UGP*-Tk(!@xk>>)8o(hx5RE&oL5hs<_&O8V7I=6kID6Q z7YAb6&O!3Mr_18fBf8t$;(L4pxthVmOt|3;#Axu7wFwr+Ykl%i7{kGcy$+;~ZfLN{4{&fD5e>CtFFueJ`Oe48a+ z&5m-59Pp;qkQ=@*>#GV4c8i!i%>$F)_&rpTtwjRh#;irBPXkxm$@N#|ztuz?7#E;t z=6en1gOEwlXbTv(Oow_&WN7HNCSRj&RKP6%nJui)o`8Fuw;2XvId}gc?ddy z3P_lsR0*qUf=qS#p|83;7xZrHT~6YC;;;R{Wa^-@1|e^p0Xa{LX%}pKk!#{A5AyLv zW1Pu@gDhD3Fmo8MZAjH}B+b`yDmo%;%!oDa^2v^)UP1^2YwA70IUes|2=Uz)EzcO( zI=)%DCK{_oAcQmskSH^2Gy!#XvFI9Xa5C1pa{I3wTz}pfy2Fx@P%oGv_p}G4_}Zg; ziV&(XEj%Awa&ZNbR)6{p`G$;3IPdMemC%e7>GS8%GL8**LtKRZ2BiKvOia3s$ttu~9Ykk)XsTLKh7d^P6 zC`&z8Ra}T~%*p0ZHs!H{;nV64!oXuQQ4(9{3Z*%I+5Sdfb!HaKNoqP8^;7L%#+GMbRv-{byw?g=ql@ zk6dLWl@!k{^SaDZ?f9JkoP*dPbEhx{@(-un$_H$FE$k0_pBEz3FS`mbqH-19L9BfvDlTaxP4&H=%8qleF%=^X*<)LewfN?jIxa1$SaUvu zHP_YKp;j_@kt|)}+h6XRgJg9X%mHV@rCd-|87D*x=?ILZY&e20c zM7v@31&pc$I1~hdyMBO*R^FgUUDzyu(`2U81|oDyJiFXB?;ZaQ(97Fv62-c!SSVuJ z>rqz!b?Zb+_VuHeX zrFd=?zk(2OHL@F|kG)g*3ZH^dmE8DSSdT_#jsNfy{_vVoBQry?3(T4<@eOfp>jkp) z+>9g$F{}dCyuy6V(kxzX!g5%T^vGsyGK#)AjTv2D*;C8 z(tOmzWfjaA8R0z) zzo@1^*#D0t2~(cMXxTm4Co^m4J^J?DR@C?!ofz|QPk=~hZgEib@7@lGY>tSHJp|7B z43vKytjeYgl+*GIDi&*ZFW{x_%8aH52I6}vXp_YLbY)L!@fT62-rpui_w;&$2|}n* zf&jTY2JdFQh?-Ks`UTg_9suC+L@Cl+nPLaXS=2*YiG^QZfa-SwrzW| zZ5wxMegFO4PvUy7>wKN(anP`rr(!2i3YmCn@vbU-=@};b+$p$tHU4mhQ4^?$T|I(K z|JZR52A_*Pa+tGc3_Rx~s2i}$$~DQ;u>EU}jj^W2>BZ!89#Cb|tV+{cMjF;QHu12{ z8D}xBA@;6153-A)7!L2n7ht@CVBJBl!>Os7+`yYX{*h@>d%@;O7V@`Nc>bfgC$y`R zVXn30gnqJnnw?^8V6MQzKW>W0mfVDm4Y3NJz5LW6LX5U#t}|*v+lm(0(dqeze%hf< zlVCq#5t1K`O3VoS1yZnGRS$BJ0&BkN$rqM(1kCVRP18R>uMhKn4&U2gXZ=>Hc|nuT zU>LumlQQo^$+NC6|W*pWqt|7!ysE?K%&)uwIX7>(B$r4;D+9v1G=_tEm^oi=)SiOyCQH` zhyT;%SjdIvlx+dmz)|Z14qtEuu1KWD5acP;SrI~>zP9r|J*NvSoE$l*`PyRR2_=7D zii_QISifX6Bv`6`iE$Tmj@2`Z$ZdH!Fskg-%I=9x-ryo`C`w?FdSV_(RLaazMIAlz zxb{~~J|bo-{ZQj$RqK&{(DC9{9_TBz>55HbDfo7hBT{v~gL_$>z@yO?Eck1-Rf{*| z!tAc+Z}CmidD~r|I)5PVhO~siiH7LB)aA%A^-GF8g-3(8y%=fq{V{oebdrCK|9Fx=EZfE{Sf+VY7pT>vqK9eYLm#^Ve9QNb~3UT*yB<*d)i@;fVY0tcOi zAhAI*BIKo>-hoE2v4N1Y>d-KOq{+4b(VTCa<(!c3Q}tP*Z!pQ^0u0p*2~E-SQ#7fJ z9HTL^>kWW)GIQ>iKdlU*sco~XE24Q$0DXQ=1q0)v_^L?PV<)9{BKgK%#Koh z7ia#Qyzn>0&Ah^yQ;!r2 zM!gg$H{Ff0`~*hdJ?Vd>UNM{3XqdDc5VV#7B+5&-w`=33=?r z?)F2QTg6KL(;^@L_S1&0vw+z$W?^diHo|05Qz@GN&U=6X2 zK#0qdIRDdv%z9bw#?i?k?G5MLnjG##xjVZe472%l27x$sEADI{C!CjdO+rNq8??M) zrXS-zau)12ZR4&o3nK^JgPx8G1p z4!MwKW+`lz+ie~Q4^z+;NyF6xw>QtYPyNGYPV1zyV)bQ*$-^Hqi;B?#%@J8nUBS7W zW(MA}9}d-vu#7uHVIV&flQMD?S-FtAP!Htvx;TqVSD1>bxL+xZGw>N&&;x49AgsDTontk$CKI21O`{l2n{ zInlJiS2?ayz2@NLLBnO;jU$hDPqwZ&S3$(w5B_jB!6=SOh!}9N2X( znXjD#PJx>Kf@mwvzvw_@1`crc6y?c$j$ROUlLCzPSm(dLBmk`lWBD{A8&nh(jXblz z1hy9Pv;04ol#Sw9Hnd}74g*2A59)x-P>^&y;#Fmvm>tpuIkT-0DF4006={DJmqc#f zbUfRfQ}tq#+fdmk!mNO70ak}mk11P#8YZZ?-_{LVMl$6%L)<*+CW#|}c zR&I^_^nwQ~;mayv4hmV0zc`iD564RbUuXgK#++K=NaAEKc9=4p_G4E9dc*%n(OkWu z1~JY5Y|WIb#$oh-DQHg(_?M*(1-|WmOxKL}h)FVqGb;QLAWysGXT>KCpiU(wV{>V^ z)!or=u}KiN=AI>?GDhcNZbhM~y?)X>?;tsI)EPY5mk@q#QloV7@TjUJ!z5OmQ^81& z6krw*8+hT@_5yGke;9!H>m$)@RVctxuXt~+o5ui3T5Tkb5&ps z`X@%xLF)}z3*4qTizE)KVT-hf7ulq&?BaSA*gQ1wK71V2en&d1up>-Y7XAPd3pCbt z4FmAa@WCc<$Mva37o%ojis>ncI@?I^{L!Wj*DENFxSW z8f8Kee3(+oUaBAas})}KMT}jkWLZMK-?9n&c1;}D={4p6Y(Vn$a5cN(tDalKHVcJ}0kfFUxujG*;Y`nJcKm*1-BuVZfBtf;2Vm4k=Q(WhJPETbg9-K!P5(ftJ-t$w%pLfkB6VR!;Kvp!`(K+#)4an@DC0im= z;q}6f$abW4lwLw**fc$Yxb;wH#?K5^z3U&@CeghDaB&bq<=*g5Rb?zQHAl{Lk0^}y zK9Y(lp6NksoPnnSdowYEnz^Fq%66RJi#X4K;>u^k(l*9eCAMl~eL{~U_vkTkitg!O zRFA8xj8TX*Z#mh0mo;00jx4wDMG2JTzZ|?h|7JjJH0DgE)s>m=@h#MNs@v8G4IVXG z`<;t}X8oG*&0&Z8&dd_ zdEHkY{yU;=qLErEYvwxrbfzklAT%kmrr311loh|ThkaOLJaNvtjUT5#m@2xLuK2ly zfjoH{Cj3>9w;z|(3^O%tr0wAH&mE)=M0ICNzO>OFq)`<<3GyKh=a*U4{>0b2Qe7-I z16U6T$Nmf4(ww>FdlR&NlAn@$7G#Rwm_BWp{#$n>hvlvlS|T?p*tNN0O7_72YQxzx zHoddeX@x4NeI$SGh*czIhp(^dTxD0vELbPe(v26QnL_#T!<5{Fcl4m5FN478IbCr0}NGDAL0d*{W}!NB~ebzC#$Od&B^XE{lLO#$#&zrYP0;q z0#-$&1yoU{*>##|0>!>#o8u;ysx;R_px*EV#iWME)BJKuol(!dIk~uIpWB|=@t0P9 zlMCkPi&E(Khz+I39TUOivgBzoCxxmS$;gS<3*K!fYsp1-H>vZO7>0AxNSG2=~ENm*17C0O{3%q{kRdy5uG@703g})oweTLEXppnV4X%7j8sK$p?FsNBR}Vm zn(G-rxuAxgl`6Rjm2yGQ*HP5f^B^7RP6;1yj^rmT_7^F}Phb27^QWs)sV44|qDgiw z)MvtZd1mvRpPrJ|=?g3`@mvm&Oh>=u?gx#9$$3Y87w>TCn(n}kO{Jd+zX;Ar=rejV zk9T&dEaV06C#7z3M?xjX)Q>c#WfXEDGe>!FdCTITqdz*=Wm@|&X>~LY9lkAvEuHE5 zHj6}24nJ2VtDvvvQ;ye%Z7nXg3fubRJtPl?>Al#)8R_B!AoNemj3kuVQ*O0w!YUv9 zn;&ZZq|wg>uefp$RR@pYU!Oz+`ZrN-sF&IS$35;x1JFtuG@V1xYf9IZ&Jw*x9Ekmr zH3mhq2OvguWBK8oJFjPa$nMu#993E*ze~%(ew5e8!B`vlDP0nJ{=QVFC4(K;>Y!ow zw~U}j3h}__2bFqd7JnRmvYuc@l}P~ayv-HV#gN%fOfs*^`KFKu_zcOhr1?MJ<9BgZ zM<>PU$Hc9Y%v(|nI6eVJ89zOelaO*3a=_-;lj+D7Qw(}Wmqq>~v?XMJ3s&3cb<1~4%2Vt+J(dTi(?g9K zbRlWtMBcD?{YTlqU-Ek2FASZ8Kf+H!{#3+htAjA;*F5Gr%xFD$n$qZQ@MIHX>bu(? zrFOmJFx4D1MLA+S4S$X6o@YYzHiQ0oc*)fjMq^c)IMUDOk{t_w&r8E$IK9hRKVYni zSosx`$*UApUT zI(td%bN7KnL>M)J;v2w5gHXoxwiw}mIeJOlC(zyiJ0qtt0Eq~D`Tq#g1wU)+s`EJg zH0;T@rZb+&w>^0p$~5@Y&fkbD!N-4xmQkrz5r?9FpzoQnzrH8FQ^*Uju*IRE(zqh6d=x_?~>Z4Uw*(S)+(cL;0I$Nh z{pKMRj&F^0lTjT5+QuB#$Z1Fk{ES}+OO5s?KVcTuqjTJ&(Z&Y)#Nb2T@LZhc^eD66 zz||G$wAprmK&_>#s2<*B#ejP1c-|X&Q2t3F>$w+xy$Rx4eCG`P36esV(y>nv2~!SD z{Q+@Nx$nGh`_aEg4PMF=rk*U;Dj+tD2gO2$)=ezQv4)-Rqdi23^xId*)3F1t`{@9TN_gjdM zxnXOeaf)L+)hT)Tl<93Q>IZRf3Zk@0OBO_fnF8=aVpuTlfcRPBByGbQ> zvyit)Rz3U*1AF*l;Vm7$JLHZ@58vzN4@wHHQDT1*0NRR40NI53n)h)4spc{wzpELr z-Y*4bE!}Z3^sI1Sx%z;gvEJVrdtwR3FYvm*qlt!G-@rZ#x+-Q(n)AbXc=P+?RjkVB zQ}aW*t*`ZGB;rOmnUTsYscf|x-7WXsc6?nAt99t71}+5eEfB<1Lj3n?`zZU4(H-kZ zGU^t)7q!$(2l-+ncmx)irf!Ln98(KxUhNQ1bvnKnG;GP{k)EuXf5m?iIjyeygiKPm zc^`xSGxmA00)wM_nZhYN7eA&aS;}Kq;YydJ^nD}@oy3c~UiC9&-4#l*ImJA>XHGWxDXYChYR^MkF!D%4AxH%F3S3u2eD>24SIB~5VpdwX9A zgZ^fK8=d#{+DqTKE3aEm&>O-!PEpg30TZ*|p*KK->w)jqww|M4!S<(u-9&gWv1`#) zNNYG8{Ha(EQKL{Gs?7QcSJy!s^)1Ap+qu`GM{M2xv_ zOtyvW(m^8>xL61sF2y`6SR(qeXCK}8d`i-}sG$q>Y_SurCF56cP4v{jrKT~DGv}() zQ?Gtjw%?r+quYV{kNi|UGL_`ao|cX!7Ty=K6o~gWIln+2(*DmEO&^iSXn`oTWzF^9 z&L<3qLCjY&9E{GwS16rqv3Fv9G>rf0$uLvNh_qJc`FJx%rQ16h8#bR!!Us zd6QMz3gYM&k(WrDIt|;I3{5j@vYB{Bb@A~c?Y#e&{@}eFU~V;}Bzrd4_TYM7rxReK zbTHMp{9Uy@=dRt?8!Y#^l5T8WDj&e(f^X@no-lF4I>w`Y@qJStLPwHYf(AOGRr|Ye>(@yg zUHwd^v^T&3%R&tWA-Hi~9Q>FU&d5Qlwl0vM#C= z)h^p=r=d%7lStlvJi&x(RJAUkr#Z(a?aiRh*$wk{ArztCj3SLgXl2CYg@MVc1~L0V zKcd6Lw_mXsI4jJ2$7Qf6_EtQ~^r9zp>#4(QeWZ)o#^{_$@9eWt66}1* zRbdf%qZeIaEB0)QaRPMQFUV=;;$;yXFG_~a*jKD=qvF|#k0xFMag#2mD$}9b4L*OC z311Ym2hu^4aV|5wcLQ|W`yXf*DN?>i&!d5p{c_0C%J9TH`KMn0ks6mjxUEynYtjjm@p^9@mgr?F2k$Gf5)A~p0Opd zz=1q(Ss?*cu$m_t9zFrMk>CAipV3S;1kOKp{~dP?w@u^}GEvd~<4j}WuY(3QAK;D0 zn~@GpuUtq(0-auYXd`xi&d`10Z)=yhG_RhJsadI>6c@7hmhTU6${pYJ9?0+Uzqeg!HvU1Jm7w4Fo3n4v8#9jMYVr`Aqe~5DsisH^^ zClQIBqBvky=c+%jRq5Sf-UOR7m=sGTduMg2Y+V&oD+~bD5x?uS(}x6z9C22VeowHu zvtK=U_(uc`KsZ07${sw^C(hmdWy(%-*pGcEF1CE@k%#~x%k=DZ(6Yz_)i;aU9#TX5pH*lU_?F>pQQH+x%_8xF`jr>A!o3zoBwqYI@|(QPm7nq z;`w*^FV)ETmUiI%WLer>GK&<|bIng`YRG zr>}OtdWmO%k9tSGCcS;Wcw_JcGn(>rXlH z^v@?GLSfk@`d|Dlel(^V zD*oiSx(FY{z>~&_2tnTCAIP{gwZA5Z$lWp;x9V=jCc9~4wksI|b)c-|jykUde=7}B z@{lUSlGgYS%Pz@V*UFeBsI2wnzUsh0n@<={%yIk;ue$<$|0lvGWJ}!|znW!d+2GL2 zQiDSHOq4IADXK~@Gi374zFR>ZmL$H!VnVo@M7A@+i?N8n2qvR_XAUq4r+&#T`f_!tKq#g!HtL7kgG0yeF!N(ZRiVLBd@ zL`cC=3^{93>9JYS=$0>`Xb2|6#*C@7xKKJsBOzIyw})r!{8tLMo!CoX}g>#J<9E2c?aiChqh;|J-3Ii#*AB)%sMn%)8W~2eT-aT^Gm~&1&wSX~imWyVY+kd3LeN^-uTXhH1Uzpyz#HkMDxK04p3F( z*Cal06H2@mPFBu0G)%tKXb@-DX$fqiWq!t@8>?!ODb%`mP{ZpNx|sQ+T+C zKVvD4Gi`FH*5aZCRBW%)?u+q2res{zedMfr!wicP8J*&kM>EbxLsrHfc05Z~YZFL{m987!YRf~`^|;Dx_oTJ{5- zX4Fu7i`L6w{0V0uqY#(BGC%d7k4W9%MZeF5(`B}J3KQE{sQoa4JvD21O1<EC7e-%$cAoK;c(&X&OcsfbO0Zqg ztuz9vE*|VU;I9{ZH=!z~4N#0S#LYf2T9TsMBFVSgewotb5T*+0j5>J}&l*K)!Ug@;Pw)>6yhjbB zf;;6TxoUiaP$z>l9TgmwM&y-N=-kw_|9H#^qr^c{B%hPms|To;a9qZZl~ zVxdT&`x&|U?xoC1Gd?3kH{FT)evsCGZOYqBQXhVzI*3F~1o`$(Fh82HRI9kTJNn%> z3g3D92JJqYxzI9#rgr@ikDJEBb#dJdhM4_qzYMD>@N>f`U0fSsF8bY#*cae&X#C7Q ztaLAAoCA{lKrYB$&&QqO8<%ucD$PGY+koNE??H9vOV_T|80pEFWutZ~lV2T8+dLgL zr@<~`Z(Zy1yuj?Oh(0{cp6rEe8eeC0q`b+tHK|y(DVULZqIhDG|DdOjGZkfbwxltc zxnT_oKKFIUID2$lQS=_37bm-Al+37`rMPc-`~zjW(_=1s#FEL{ex1_y%(4J~_$tBz@oC#XV)Ra{Zl z6m{ zTcaoU}W5cl)1kL6AHMaITfrCv+3`Q5#pjKSv}jR_9cz?i4-1C z7czsrF*_h7K7@sTR`q5sbXmp6JaD~h%?Yp%zRG`uc_Q%xaI>uUpuKP#4+)cbp=us#J%e=a^~Ju#S_E&c9FH2iVkqF+Ckk$*SA! zN#7hw==D=O#9i;-q16#A^M*HKiQKJ!{kjo3XCy2cGTEgW>ve(m(VMx;eNFzgN*HIR zqBVRHGJLsK&j#R1;g)u#HOR(JMw^SW1iX&D0|plCvPO9%u2mZe5^xH#gs;h{!w{Kn z)7ya4nctkK^*cEQ;-#$O-3yY=cLy6h=uF~8wa<$Pe`4n^}1%0NyCRv z0pjP%Z5@&pR{Qm`&1P&4I=x;xLYV6c=-C_i7yV5?ELxEDBBvcSCXkIXbH)hh1yYvf ziHswgg_S-M*Qi~x;GgYJxmn5Ip&VFBwwXS)$Z&0CuPppY#O0>kKs>yrj56#L$7{z2 zN`rch!*ZpACn1)MC{hSwV{e^i5_@92e)i{JmtAf;ko&0QS3e&5J^^sj!xwmlF#y!#CVWv0^dghMm_ zL31qF(2nE^hZy%cR|fDx$x#TYG&4rKg8{!SgxUR%g}5~=U^S_I`{WgV066m>iM{P* ze?+Pw9ICf`eat;H>bc77wIgytZqgzUx(o2`c{L%Ek8S`0#A?~L@l8bToPmX0sAp9} zCmc7ukJV}C0Fvr;JKj5GZ)^p#v##qTtYcR0XoDp`pKh}UIn|;Qa;vy|?KYK5+4e@} z#*SH(xpbyLFYLZ4INciF;p0?A5{=O-Xo$g|Wa|Do?HLBXiQAhpqW5Tw1JWnmdw6$X z)E~%=IixbF|9AQ?fWC$x{aU5%A8`Ckr3qT7s?4rS0dX#nQYcfjzOGqLT~-Yhng;7~ zAbAG%@EwUSE25BOjo?Veg80y@$C}T1Lh@PZNA7K_eZKa9sh>iL`i*Q$3({T;aEoz7 zQ(>V*YfPJ#sc*#U3l9E4*VwH-hf^Px!q+2`tVAeelSyUT7``SA!YEYr`7|4VXYnBq zE|0ax-y(Ia9M6i2E~=NEQ$_pG^|%#zgKyLY}V*a_G-oC6-5V?*=(YTN;vY`A? zkWZq1!fCKlJ<_|0=^V~EY8Y-zB?H_veO&<;w#w6Ze`9?9sb2RW zL(!R}0-nhS#pT_Ca-?m3xlY$ywl>Q2pt^wNC{O`#WM(R3-5~&#Tf&xw{?pb6^tvtcU`rA%@ zPOp`IKw@kT0dL-g+M6^UFVK4!A>vN1CdpUPTMny_SJHI^7IN(WX=|gBp|C5Av8NlN z>*0v7yZ&!&?I)ZDXMm>t!8BHGIZw{x!kgNAhFXx43^DIN+OOoNTu&fmP3%s<>S1vb2#&YKU~4g?R@L&0*?grIrA@9J|E z@WQ+&+;PD&Qd~4MuBOqeW|r}lDp>uWJ{FIyTXxsqU0>ufyHcFPsYS-Aymd{(<@;f1 z^NLPt7N0A3Fo^*t;vD*`T2}iK890WU3lZg|!yxM#Cco;T|D;vRNX-wBX`EAnZVUcq1UOyw_I(=U%G$3qytjK7T5wmqtJ2l^bqI`}`Q$ znxA0gCU^$%77z!#G>xCcR>}exbED zFJ!N_1_c01t;4ijKVG132tw3U%`$&T_2T7#8om$c5XinJeW|Ck%ol^S5YQrO~13DJOBcwZK4#|7mGL zXZl(7SC;ttV}GvL@H91O?@w{#GAB<0WK=sN-kyygN{Hj>=DjGoYR3~B?XV(d5U-OP z7U$Ff6YmP!H__UV`CX}7e8=+Ji8iLw?0jWSc#uCdjKQ5ze`ub%T{CyH;#&VQF$EaV z;lHx}Tn#<@d|jok8;<@0x|EXaDC?y8S1r-D!>h%ps3ylz$7xD)+{Lar`YLZY`OxPz z(U=cyAR~-A=&~7V`3N=UCAvD-7!2p4e`{*s4S1Zg++s>w&vHG2x<;;bEMG5x%I-DM@*I+cVR9lT5GNaPo8h^cSYj(Q-Ip>>w03 zjc70`Zw}TTV(NiB8W>S$1ju$Lm}>d_?Nl3(T2i8Q#=r#r zigXgA(*4)-bahgUu@ZbaZbr1BTw!`Nb;Qu%}6Q*l^eL0+9nh zXJByeuWLJ?io7=Z!Nz}j0FO9xe>G6T))$ap?fZ1S7JUro@yZ#59qIe^P>SbQzibGw zAK>N0^#k$DsSJ+5I(Xi}si@WB#n9Osx?XEP5M3k~h-7_5$&8Vb3T-v-hUbi)UvH;| z#L!4A0lWSqd57lbd+^uDYtPeC22!(@b&(UgS`8!+Q6It{;hgBXI>&;`7k-n4MH49( zqIYY`9Vo4=53KcntE95K!i7;ats#XzQ2NCRUf=Dnzu_p8Z<_m|z%8u9I`RLdeDhe$ zXd+Ahn((a`(kGRpcf8(>n!E^e=3WQz{dQ!1HqX?qionVa`G9_v2 zD;#`<#X0XXb0c=J{Vu!rCT}n8`aD1o7{>F3yMphEUz!sFwr7L+;Dhk{RUk=9KgbG26nAA!Fdl9@6t zVB+#&*LznY*B|WDxTRTiM8x(;y_~b|tzfv0STc6{OMqiYi{p8Wc=3&TTk1hE_be(9N;tQbVdEbx!w}Jo#hg19LGuP&i)!bQwkF zv7P6{ZMdTX8at3Dzi{J+=3=5SYBoAFdciK<$)phKpiYJ!JUOE*xfl4iDv@4ygZ3qj zF?WT!qR(*&zeW-72Y7Ee7rxe1Q*rd@#oeH+A4uPXn~X--9(m}m537&U7z*@HThhD5 zs99jiByxWXk9a%{AcU(lDPO|}daWw%V2Lfx7mEuDEKr2lA{6c2;vLr=vEA)~NYS`A z&@u7a;(kU&o5FwLP{93tIE}nd(E`fI3qRfT2(xN;z`Ow z=AD{{<%wa|WC4X5(z5jN4cA-R^ZT16u%Fu+#keiw8gtmIQtHli_$`fx@rr=8+iw-rY-Xg zKDfJ@5;v@yHAMldF7hDeWJr5ShB|oUzx&uLVt}3tsWr$291O|ltrCFM$IhEB90@iI zX&^+t>IBSVN}0RVMH0wuY%wdYORj3~EB=GR5V!WwZu_w@T;yHET0J84v;CyLV_g57 ztrWBnGu}ryW-+l}1evtWpiIwU(RNhYpD5}Vu*=0nr3*2);yPvYfz`oEmfK_k7X|mb ztpmRVf?0jPN_Pso8TcTj?#vkKP58ZNYWBYzu@;FxT!7Rn(TfSJ#^Kf<8onkuh5$|M zyqP|cnDBh;wnMSjPj$b!V(MN4T;nQt%FFTJdH8w)NGuX+HANGN?vk4BRI6NrxswMm zSCSEBHy&|bHgX{Nz@zLSE5jxw+_AaN$Qh4JvB})?YFl}M`iln~B!F8IQu*tjlA||7 zm*(;UD}I_w#`rwXh2HOyzo|1;#*D6SOSS^G7sKp3?X|^j{)4H_h@ctY3}VJ_Y*9ByoJP@kv%!UV>wMOdvd@**!H{}|@4z$X2~L1dBF>u=^giB;#GYZ$B%~U7KCJiL|6-_ei>xbVrSWmPU!9w}6zrku!r~mn6 z&CKABH+JEKqlybvz{uAsb_ewMrhFM2m)ua+bwMtZ_H!N{-4g5ikLbQScfG#V&3EpQ z49u07$7>XF`eGo$4na=B&tYQUOY!Z_Bk+lwwc2-kB;D)M*A>n|jyMe3@gXk|zRc}{ z2ZFjv;K$KD6B$y8DN3$_`c_r_CwYg96a%8%68@D11-~q%!(7v{Pi8&WxwzrEZU%Ji zqo(?(IRDEFlo@7j~6p5cl zfprTq?hNq!uOD%u3a!-5*Nln%hbTF|D}6QzYEAN`Ki5`ys^73kEe4ZwpW)ISiuu6f z&*Rb}c=+WCKUsjx(~2X_gTTVdp=}1c+f3`I$Yu`7asO~gi^{3dd-2pEWoD?q9jbe! zw(R6xJHpN2WDY$xwiT>d-q)KyzsYs$uxt=X;C$zRkG4JkK)r{7yF(7B%~sZaN_vjx z<^qY;IcJ0RlkVs;Lh}jSv*=ao^+?3$Cs|CVwhm}A<$*BnMwc>IsBcuof4@s;^bmPs zpE6ERTpNvUx<^5!?4Q6v-f(glYjJbQ1ci#Ul!b)M$H;1*FBUH!xGc%S!4f#xSqk;Q zBix2gjFqJVuYWCIPnz~SQ{D$m;_;_nLXkR>33o!oK#n~2iJy^_twuDiNy&LW!q2Pj z5ng*XyG#>rD47Kf)40kSG6$_|p0FP2iN-Gs=^CA}v-#Ujw=~A+wcux{m+`xO)%cmP zo|o}H$V}*%>hoYx{dT72noB*XgMG`}UAUS4l59dknUvY0zorv2N?x=3T$>d0ti>Qd zHGyTH^x(Yuao-=4PrA`-UUDqpeX{#3xiaRY&V@YL6J5>>FD_qiQ$CY;Q!{;zlr%fi zNBHyn&F@a)sYx|uOm^ct0--JA@h@qNiHYe5O|!K|$HaGL$JQhA`w(mu_9N?r6vxyy z6p|2V-Vnx|g3OdTTZVm9I8Etnu@42G_d&9)`Ujy>-5H@XDOe69P{#i5UVodsb-X5*BDVer;RvyuAf&GAcGC(>qxz0nob8o*^(JCR^*3MmY~+z z@KBpGTJ_MTUR^9Zbo6qN;oRTSu0$bHO0H)xz0{VfHc5Inz@pe`y0 z?PjWoAS+P!55b87izk-Xr`SjV$sIU-u>Dy}BkT$-rp4W>TZCa(1BTYx;plxc7IXz+ zI3@9UF=5X0z_=w%NuAe0T^63f+!KN_k@W0i){;4;=VPEW#7>6qZo^F%>ihsQ8>9Bc z*2J%JUZIem7i@IZN3^fpRfbXZb}eBCj{D@{n%fD@B=d|PaS>MZM$WN94wpz2(c>HJ zO2L!)9lDW+wC68LtbU{b{2Sk{nGR)+aXsb#YEky_3ng3RF1c6j9?QYWj>UX@wZecT z182Fc9sb=OFV35-w4DzOJnvwDxW!juUJoMk#TXq2Jtn%uTR~66`kqV~Zn~juepT)H zpK{v1Vx%Y+o6-vgeJl>VjefTAER-}Mse-L5rkBZBIoT0^=F8!gHT3)^U0jcdJ%B|X zXYghUCo}{MMDdSL{H-L-6iDq-E(AHe6u&F7x9^=mWwJmDmFy3nlkL&k-gnNp zi917NTX#pi6DD$^cK=005;*-jAd3DCad3e>HS`;?QUba!E&4&7mz=VU=q zLynD(`N~5U_s>nFo(e<$)N+_VkM--U5numqgAQz@KLAp+x6`KzSXWrDl3edjnFt-VgFiH9d*LaEoHKbj|akez?8^i^b zaUfTGRN5sMOlx_>&6F&6e3PXxN#_g#TBV{+w*1}v$uXUJ4nO@FDY-}5T8*fEn%l#k?>Q0fGm z~fzFmdLpWC^1}+D@7W>1p{zoo%%JZIQkGiUOGLL3`7w!f(JOA2hm@nQ0Q)FPE@?;B@OZ_G|I80dhfK zR?xV=mS@4q@U0VLJZ-M!B0G&)SG2mEK1o4^s^7{tG9oyBn#vP7=@q#U0FHxk%TO9C z>?ZA=f3zvmz5MD_BTdO&vPS;#wj*3m;UneNw9}?(0E?bQXGbAbt8{Dj@|s*+x>% ztWmq$tUpkig}!MIQVx$V8^t5$vsqnmTW(fbTsYJyhG$DJd5rde)$b|5_&W{b<|~bY zkP*J?B0H3g6mBKlg5Z3Q;k;RF>#l+WiETrkhz?#|2@91E`BxkAqX`A5MU)<`_zyu+y~sLqh1w;lx;y&-@>1+sUj~)$5Q9FWd~b84{lfQhI$ZpC+e$>n#hC2VF6?!Y2<7?P0Vy5 zYb&;^rFHccwoM}43~#F3SUt8l7LD2$-IcDQ$xrTR@)uK$y3z$&`S;1ZSySGbFg;M5 zOy`h+2+iGdH9i4QFE5aFcrG(Y*at!J=(FeRxRa@9+C2o@XxG}{O(kM{c<+dP33Q0qf(0sAC)1tyYn6@iG9rKEzuq+A5Qo?Dm+DD z>)YEh4@F8Uq`J54*Lpz5cvQYwZ-h}n4LX&a(7>_T8w;5c0Q-Qo&wjxH`7TOl7*&6m z;1u?PjEy$D=wzsK*m>MyUD^|Y8o@b-(zY-eXY5km0){`u>Ri{34=l{=Iaz* z4GaD_df~i*$*1#G3xYmZ!>$31*^^yV&d9D`yc3IJ{y;VHT?@lM^G(Q5$!uGIh$h!o znww+oB@9%4D)t zPC5cD`UV8%C9D#t`Lt4?$$y;%`m`MsXuN-(K)0@I0$u012^`gSMqriempni4xxjIM zej+e-^?HGG&Yu<-GQUTlRx^|54=V(k4L@R+=bjT-ccD&TW68?`v$~!Y=)P_zs|~*{ zusO>>V9Cs1SaYYBK<^(^0t2t^5tvZ^nm|?e3BK7HCa`GF6M?3h5>_7em+W~{ptJRd z0w)(-6_`=GjlXuS7U)vX%H7wy1eSe_RZ-^}1@<`MUBD8)PkQebHebi8=E^vMR*v5b zT<~VKz_2$b3rvY)o#0+A{vI(SJ_UgS!)*(&d8+_frVVQvSzQ#^5R7-KMvcJw{dNMo z;)tPsRl7isrL6)(OScO&Kev)KUsyo&@J&iNHh5cML;Oa8Y62kZQke?O*=WvQgQo;WUxf)l zQ>P2G&tXkHUu%JK=U#!!9wL~WB z@Kg3$Lux@gn3nS*YGVTHwTWxme!NEwCdEPrDA15uM5Gl{;oBf4#Ae z++8B__P4PrZ8lyTyO5DFcM`FD|0scmQXGapqDC73VK1%Y6oJaga+tQmL7>TKsvtrG z_iBu%vCczFfmK%TVfjz568Y>hfiXi^IL3s?gdFuC<}GtrGxtyYx(TlWw@eZ^e+Tx= zj#wnn;Xs1Ggb*Krx|@vQK+I3@*;BGp0mn+JsSf`WR83A6EX@hPs`wiXaJT~dJ-3qu zkH4m3`Ei~}aH77_B2xIg63!IK;BfLUhl$l!#K+4JkMgH{gD(I`iKVIB*l+oBH{_zzM(X4Tzy?i1-(tbP*VEnVgzc z;*o_nYuc^AD&3+N1)7Gjexv3RF?{A{fu+_V0?Thj2z1#`{1XN*kl%x7LaZybFwGk@ zuiZubmpZcV-5zZ65B#bp7k+-{1ooey_N(++E!@kOoW5l%aBNSpKyUwh zuM%rbHtL$YG z@sYrkng0mXob?pwd>J&ne~ssFO1kF+uiQLTYBHiXao zoxCjL$)e3M{yG(0^nUn5H&o@#X9ey7O zobl`pVzL)I<=twy*MY+4Y=HTRdx(koT58;KA8`oQBVv7r_}-_G%AHb0HiJHXiD)?D zmv@RG`#vNKAysHdPQ$z49{Px9H!GO!AtqDyUM6dANE$c_=2Ra*Ep4~ar*d`Zxyiq` zkj=eR?x;J&Ke}KU)~eaD%(oVHby1aLK7{RAc2r07OMIVqj~ZWegO<~l48yf{86bacsn9i1pShs*}|50I&)2L;ZnppJ}^ zu*xV0y)|}%MPXNoq3a;nE{RL<`*hNpAR;e6i87=+EvMT(6?b3#8L!nQZyYSTu((aU6hi`Ni=l4nucOgrW-zYod2W;VHqp2v<3^YNnF zmS|_L#eNqP_R=eWu}K+JVsSg_8vmGn?Kw!VH2IBqrp7Z}jPgLQO!uL^8P7`{;S^jA zwMRozETXV;9U9v(n_ZMAP?eOE<6v4UeAbyk4UX2p$?)|sKO`Ck%8iIa+Co07UOO+) z$&%+S%&9f?B&LOmBQPy`H#S75sr2|$@GAHuQ%bRnn)R<{4jkhO(?Tri{$7JbMn9fd z`JKkc#m}Kv<1*gB#~Em>RaBwCg43vL?NKs5z6a*`#52|V{7#oF+{!#3+@~+ltQci0 z3xdO%!&vCuO}^^pQoos<_>z4Krj<%E+4N9JQ&%MkhUp3W3KqOSU1Q!lljP~Ab}(LL)=(1iRYurZ_t z`$s3j&(v;r^0kJ^r1Cm3aT$bTMLlSO@0Uz~nZHvhX>zt3G{LKpLR4jAe)6m6eD))rca&)bDl}BF`ZT-P%%I{r7ophgi%^w`e{2yLR*(Hj>xjw7 z#OQE^Gh^vXtYiH)5i2$65ZHGQwuD{bT;MntZH)XJO)gY0(G^zlq*LHcyuJy0vp%F# zXKaN(CXey1;UL_bd!EUpw~JXqb&qvSzQnIZd01t-@+=W(KzUj>&_Tk?ne===qK?MA z2cxFG0P1d`qYZ5TPN4TCYN5!7$Xjip`V2bI%*-^VbhlOH%Tn^XV+Au+%^KLH%pfo0 zWMtp{OZe$=5AVjlM#QwyRBMd4AC`NfB(}%giOCMKtZHQv9ydsiOhY&~rM973~rlp*7s~U)p=@3>`D2TH%fZfB}QA@)m_@Z0R%uuRjJ`9Z`gE{)C0&AsC znmSBXy5C`aHxEvTwjVIb=qI88jjNeX6JqFYp}yE$r)ACDUTPt+nw1mI5CN|<)LQr( zbcb11=vZny40ko7M~~abDM?R<^S+5RkLlfHW=Prr>PMlF0SnRq;t_PWX|Y z+ZV`R!;I;khH|V8*Jm{cV-!B;BC}$C$wIvDB1g5K!h{45DgGb9r5Y7p=xiopzN@jh z`dQ9&HRs7}|87pl1F!Mf`D0@1aSNOgb(~xT(<5RM@I9=Oc{sZtMh!W_kn9&YUno|; zB{0Q+j@cmHbxgTUu8sQWQ*PhztEyo7NT3Z74sYhK?G@o#oGjd9S*QJZdS!yto%0--3OruN_oNkYMsXiL%)C!ekxSh( z=4E`GIS!50R>Cx|J!CDqo{r#QM^{pK!30%23`xC4Mk4nTD~k%|k7Q_-Emi>K&6(L%>6T(@28;5P21U5?^Pxu_3ZvST}gG9D0y5YN|OfB^YC=$`G`pFBCIaM?Wn(r zcG5}mYj_Q=2J3K#V2k+4+M2EjY~4*Sj(B`fV2~a$wAjx4R#igoqLt{azKL`T)J45( z)A4k)BU+>yz?aAysC`i^RS@3K+1&gXoycZ@2rpiW7uIdh^JFIx9+~h<#?u{~{E2Xi zH9ews4^u~6BaH3oq}Oy^Ce9If5fWhLsN zn8j(rq*l6p4Q1u*i)25%j%Q*{kWEKJPLu@$oZ_|LW#ao=L@DeLy?$ikZE`0U+28ZM zr7bm>v{O1A(57Tg+sK`EB6+Eqi?v4IQw71RV5wQ-Y=IT0$d_&;c}X}-$1!b^`nz-| zY4G_FEQ-a)qB^1%`W|a0eb3yY9(9z}!s#H6#njY*RyAxQ<|FE@-M~E1QOA=mPwZi%io4?Y{XB2li6W_c_&n-1y)ttv(|*7+ zWWVD$Tn%<)7HZqPo|RV;&+aKOMAgapG4ow|u2TfQS=Q`0B?vu? zok`?tFHmu%J=kM)ipZFGGpmO!X1d5xlJSxIC94v;o~|{u6Bx6>}n}yTFChn1#_|H$*66deeB-Wl~{!D*|YZx)Ld*3=zL7N$q7S4^fuBd{FYMJ ziv7f}bT5qZyu~zIv;~`;-X$+tC-|-FSvp1c6V%U7hVP*ZnAs8x=xg>{vCwcSaWD&% z>Q@KWwf_a9lJ~P$o0e#dsA%yl=F$o6D8F$db3(s#)`(tBWTJlM&PS=HIx23!u1W8a zYxhA;oq21ess0I^bS@|rI5CZGZW+q@ey62%3YlK|XYftvkDU1O{w5}IkC=Ai{y@Pp zEy?WAMNZQmYN|uNlZi3*A~jgC8UEO*DR^ZkpAGwoK+vz8_p_hD)9Q!hqR1XKOSB+X zz6YfVKagL|qNRwA%yg=^5FH5YLZdWpFeGFSIuO!{jFx55e`AZuZ)Pd8L~#`F5%S*W zuM0DXWdC~3IQBC+-%b1iZ44ixcT{`8pXm3As#!kYF}-JRU{dnV;I7!R9u15ggXO;8po;zzsK=UG@*X*x?x}x@eJ!36 z;ZYsvPlj{`jDN~pWLPpRFeVnJ<%Dp;nrDT26l_DEbzh_6g1k5#s7&Zq^~tQ0uz+}` zbV{d~WbB`Lp4)~JC6-q%qE9Vx;5^v&U#g?#EYFOS?&_y5rK^sAf)0c>qc%m0S-H1~ zsnYHZG7>5E_leJ9xxpjex#$ve)NY1C?-QCjnj_qWun(GK$4 zA?eHJF!axa-ops$JZ0292e|s;|=q z5@V$-k3uaC46(#?6e{HG!7WzCF6LO}W;DTm0vz!Bo4M5W9^7`HjE^}TRMwb>IZZYmGmlgO`2D4AyyoeLc&Gw#iPZbNx)r6TR+AoXnQ}dm~rZ7$p0*KbEiO zo9G8{qr#69x~sIKdk=G$`YviV(#z@xq`S~a?#(3NkIV{sE6$$hAr zC_8vqSc+qblY{m}xzec`K@Ff&ZS5>CRpi@YC<-YXUVb0el{h(po>yJyPUos@`%1^&55_ z{#b6rs;YN+KGccNvoTy&cbk5y-cM{RIIPP|Z0WhWw@~b4TkfvS4l)B653*XodEUN7 ze9Ya5TP+o8^c@|ob2;3YcviYGI0S#a9SYEz=%WS6;winF~38^f3^VsW-2BG!@1~f{zmk3(B?iIP^MwIfi-tl z?HzEt+@Ic|9lH0SAhjo22}2lt$$|1@w9L|sSvwdvr1K%ZA&pp$}`GwDxi zHswX`dF0*yrkgZL9l`c2NAk$&1ALFXL}xdwBX@aQsmGL^ zaG>-WUK^gG7Dgv924&sl`&lW(C)fQNUL0jsDa2h_s~!~*{eWF!Qt9B;E%cfKJt9-{ z2|Ow5gy$X2=xtOW%A+$2O;g25cYSN|Ww8md_1;6i!Db!dyG2X=)?{~Hj{!GBGtHqkp;PQw=S-EcK=F-)jkyq=ZY=;M}GSj}$}N-}9B zDwnj4T#USHsQHb_N0G)ZnZ?w;Pb6ao<0;XL?ID(Brt*%NBS z=o5Nz>rA?5*#)VOKZa2zpP@)|{kR`+c+5FVE&T^b!!(H@-|{(q53_D$HBVMZ_n);l z=_U>Hh_mtnb*;0Q3awZ{wBxSujhPKSdTKPEW17I=&X8z*!+YJ2Q1?L%4a>Arqhm!TG`Xb9owlRc@!K-S5kM0Ka$zf zhj7VY6zh+?g?Igf49Y(xQ|4Lx)$fpW>7<`an6z+`mKyp3G@O8Jtn$2P5+ zDl7RKkD8)6f7LhB_ZxnNpCfa7)aT?*z7wr3DJPP4y>z{i|8*I)7A493QF^me&i^n( z{7l~r4`V{sAEFil(s(BEd3Y#a#S~=j#QGJ#NdM=s#GN?DzWV<>NZzQ=B>(^w02S_& z0TrrW6}zzK+yNXF&=?f}6$TXu001A02mk;80000JMVfhBlx5e)VHkFXWd<1bZH66& z9R_3>h8<)VkR3!66ck)=OT}DpNz~(_6pej(rLZmG7lc)>`dHYo%uj@EGA-b9%1vP#mkeQ_<;}wV zD}+@I<_cT3!&ca=U>{-Ae?K9tY+H)3)sNoh`J2atb-m4Z9Xt03JN@{#!g@>y6*g}^ zzV*M=$D5kj!Yb_z!dlMSA*|u;mxK)tZV@)_s*SLLgLi}tRs2O<{&R$_bvrGr_nr@k zeFvV~6t59Bv+0Vk{sBvb?JOs*=!B8NCdqlz^jUnoSm0#v6NlY}jPZK|+yH!mf+#C`wdC_3?PVMk5= znblsfV0Y^+!a7;q61Ft?8S)tN6)O*tk(@uWL2@C;*z4aAwy1lGuo)Nf!FDd>h}q1W zj-T!n)|k(m@mBW2PDo(oBqKh%z6^Q+PD)`b!p!l1N&*Ng-xszm!xwH`i4r#DQWs3v zL>|r7ZO4aJR*PvJ2lwLHQ9JFPu;z#GCurr1!uFJWMBW`>i;Z>>KG?ID*G-UYbeSaV z!Y0<7TD_R)hRRrZ8R*#z^4ueFIDBX#`)&tx#JKh$o`=Q>+Z)aG+6txz;hsyGrLM1eyl&N7i@rChzl1$y9XM zabW}P*+o~#cZ+h_tE(Qq+5bI3*p8>ztA6x)@SJm5*tl_ptQLqC&3XyN3!V#S;@Qet`kX((_Bm9=JKRjzrtQ3AC}-dBU3&ZMCsQ zcsQ9^JVIF84l?+p;?_$HAjM<2sRE1r~I zW*zJA>V)mJCP%tsFfg;~J7LZ4qamp}>QQx%w0X(NRIQ}U%BRuxB|51`+@qT`L_QCa zvFLomy48Fuk2hmC{*SOTTB(lVKk;V8Nzm}RMOR8T1-*VXOz7EwqE@<**~C-e8FvRo ziuHgS(Ia>sp`wD+D(YgmmhRx?M%J1&lde|VDf2Fxis4R1RO zsh5BaAZA@ju0s@H>-LljoVfvRQR5B?TXzb!*lYl+?EQSsxJz9W(S>)Qm2?ajSe0j#yWwUQB*^Xhp@TX#1=e5(!HuGZ}_ZyljtsjPh1cgcdaI;EiZNmtNDmL zpXC~Qdi)UF?l9746W*u08DA!f)cdHqqV6sXu?6j@Bro!En;xw`PZWg}X|U)V8MOZq z#Vh_EE-5B6OEg@i%PYDKutLv-ARA6fRKaL+*#+2Wy;~t{VhSG0!obk|BLa&pN0){< zFf-^j;hW|XOwM=WNs1HPGk;|hZ)Wo5Km|;2UQRZPMo|lK-@bxB@o0klBc?3dc$let znylIMqU}~ILBnY&u^84@vUogoH9H7@5n6|cDR~1b=#;Wqc8sGPf@ca((F<85Ox_lSvle) zwWj_9r8QmqGSNAmB1gYV)6|de^1c<`#udIJY_&c9r(HnX)4EtE^9UTyTFGbsL6}f; zluB9fHo8=|p&5U2P-dGqSRu8ONR3;-R(X`%Ir)QRr4l5~Uq_urS%Z0;Gs+No7@p*3 zWs>Uydxb4ElG9Q-{nv07{?Gpe4_#crv$$!uuzBa`<~8PEn=ut0I{3o&eoZ_6L^C5< z4uE0xnr>kOt?JZd8B7=ykBFK)egx(t1#&eqtG{^2Ry4PYOb?v3>TV=;7W5ON$k^95{G=rpv zJL!FX?-I4{ZDtkKg(|YMmAY`ZgBv4S@iz7ao`e>{M&;|0;(d+!RsV+1{^K|Dybd)` zd%-$`H=S`}4J^$wvt;FMVAU`Nuco!bhu|O3I?Zx)#_$~q>~a{af-a-FdJR*3?;NJb z-s^anH-?@#DH^ohPZ34)T`JMR6`$Q=@y*77&REUk9mP!)pzSZzAmiU)QhEc$2rGd_ z>Rqs<;yUW;l|WxJ)^O5^NJiP7oJaC{P|mT2{bc-gx?$r^>Z0@@9Y4Ux6FX0M;=BAm zFytz0COM-L=2w^~s@`G8d56e*<6g8h-mag$%&0_%$8?AMkJ%+?j-&>=ncODdh2e#N zgW=Sd$dpS2v4_;MTDm2*T>Lj#h<&6Ic8LL14=aV4z7^)HD+X;ZNVRPNlcEzVK)_Od+`KG@vtltqPv8*XfM=f`gh4p_Pv}0pnex)?&l@6m?6DwiY ziutS)Xu|0)p^v_=Gmw$QgH&RB61iv@hd-8=LEdYKYK=Firo5`9bNzd);Jt+z$vl@i z-}WY)k1`>@Wk)!pMFxUo(>$i@4yi-+PJkg6k3p|u5nRgLMP zOm{r^JHA6U?VcmQDo4Ds1tb%UlTYxwLr6@2t(m}$!&GdY{?;6z!m6V;LpF@^TJ!6i-EufjeR z&&1-QXHuOwoGj$*rGiFwp?(p+k;jU4^wz`!t`OJ`?PveJ5T!T?1R*o#VR#y zTtoeuO(J(QB{iszrqbO!m{l}-qHZ-oSK}|BfwC`A$eMU`Y0NH8SZ*>ZJ?Au&wj!2# z)Llg3Wmh;gckUz?>Je0#%JYG+N&a;0WK+HynhuLRYM5arHq$Gkw$qieS4(Hu+4MNC zMdUI49x)akf-Sjc(XQrwyzh0C?>au_eeVVsrMwF!ay{zQy^|W(I`)G0TKdTF@6ZAH z52&bvKhu=Flc-}W-UE{Xm>K#bEFHUu6()O1cMfM^!l*?0{yb+mpuO}1I}Q?QGha37 z{tHzcV+x)z25=5N1_FvaFp+O&_k=SXOX?F~qtz&GRQ#q8k?&}9Bgm7yd!2zR6ZnO<|W|RLaYE-Z_<8&iIo0@)^lHMR71ed5zPt{0eog{)Tf<-YYPn z^&Yc?O9u$od-#Fd``mc=#({jK4=7c+vQEx)G^%O~lVSY?@>sS9wVZj1H$A>%UYheW zr%tekXQ9_SAXl>ZWygL6+ zkng+7UNhE_)7t0Qv3?s`WSb@VoPq*Jc5zqW{Qzv6uEG$cjaRt9PrD!{C~p&LHhKoJ7j2=ZD{m0{csnApQ-gr+3`~ovq7MddM#D?Wv9$}~ zK+$T@?mq*VFBkyf^FIZX_{S5wQ9S- z-}C`>ZFd`;dC~!GZ*q2a?CKZRbup1W5&%- zI9GZmnYl1z#0TJMEM(5{+KZdUW|uPc_@j425eJ8I5WnMVn)h;5gghZx$!6&ORw};%x#$L^-f`P z_whOVI4AL9H%_=ok*I-YKDUKwFT<;9Pi`{PrL%^cE8K{E4>SV59mQU@@HR|3xf=)J zWPu4YoANl{%_O}hQQ7#QzfITE^npYBB4#A_d4t6M6@4dhBRyAXh0Z&z$Pu<`lr;Mp znFK01T$<>Y6LnV+c$U7;%%JxnkHe3!a*p&n#iYqhJF;E)Z|;F6yQa{)J=TL~?o=2U z7(2wiG1Q~a1M0O5jET7h*~ zgHl=FA|rLj&?W!R;DgIAsA8K1yEIGlo9B6YbjvreJ!QXiznaf?!^+SNuM^;taGoj4 z_9}CZdlY)8*vc$jtE8LEY$6ubPWE*hgFkVzeur-=I$+y8@?LlaWc+AHCMxNswCY2+ zJ$FCV=(hznA6lqeHXLzu$ZgD27gVx&yT0dGC#*FuU6jl zlkRQwLwuea3X_Y^axXWl0}SJ(UsyCHC~D{pcpGBH`^l4}dz6PQ z>CPvH+XlN#_Oc(&Umbph)g>QcZHw`$PD=-=ZvU6CJ>!Wn=r{byPvrDbv64B*XFJ*P z{~R>R_MnaNwLt8Wz+X+*a%bqanp?`u17H%n4sPqVF?&aSOVtc-<%C!e12VzVxnOL7 zbcD&}?6Adq22@WaJ z2~nAWzInb%@g()(n|>)d(*B9YT4|V1DlhXndmFtk$dg^7u5pK4t|22iLG-)k@tld= zK9GKUS_KE}7jxnswa>u5BjB_7f_`E9OWAk!MsV|cmHT1i2~@%`$hG5>DJAm0^n2MJ zPDTaWIH}bgr?NB|FeFtS2%c@|mBn;!0*4KPVgJt>verTrDmQYPPz#c}-#HODc!Rd) z6tRTm!Uu~l;C9GbI#~xYD zM$yJD8CK-WViRf{TSXLk&3M%l%M582O@(@$g$eaWaPXT40=-AbV|WdK%;d%Jd8dh-Nqb3<#OEU+UjwjBv0l9q3U5x&p%Kw-5t0Q^e&!PIKabL zGuSShLcYv9nVikmbKjMD42BrCaU<#-Dcum#9!*T!xryq&2s0Nx&xF>!28{~oqbsG~ zqSwg3h54lhqVxWq{+=Jjyd*R7CJ2bUUJjT-Y6lKf1k4+D>XX zX}=wn%Q*(N@w>^dPDxyz8(>kS4rVr-WWq@d`&ii8P7q6YL>8=ir2E?kMAUGddzBM&)r7eAvnItYa5|m12k)3vs3Ec&T{v*hEh!yP0@MYrazQ*Z8HBOp*n(!*CgT7XN z3+Ai0&=YGmuzT_|I%MRHTk>5o@|T%i6yKyb|TDd{f*cy9XU~GzteeU67##IgkY=RS>@NR`sWtNta%5b{n@;)6tFn(-Cv$*Kv0yp9EGmnf21v@5vYjlupXKgC zzl5`%aS2R|+f2NPEzJD2GCrr)azd$oh26t1|M1V7?aRrI-)KC4a?_G}jO<&Ke=Ka! zBTkp)HL%O(j&xV~K04OA3sxu}!tJW#Xo%~N;2FD~90iw?iykE$OIM()&4Ag=l) zXr{Fpp6eR%Jn9O_=qu^YO&dA6WVFDjCh0#+q91T8*tm!q@oq#9)fViVnk1crQdm>* zm?sS*(8=hNykoJK_5IE1{yt?qSDs@cO|M2_U3-cBoUgF1^N2S*g1;D54-_^j$UxYLliv#)>Gd(sN312H zEA_(4bvP}qWibAy6Set72i|_>zOcFx3Ssq4h6tN&d{0JIOUb)Bt`ODyur;UV@@Y2DJNDzu*eSfrC)$XNpWH$QPi2soh0lrZVF>>G?vIBv z9}?r1cvy5APlCtD_-@fiqUiaIZ=L0QmmTEDQo;El8_2u+onm2w(vRcM(r<+we6&W` z^xJ)e^*sg-xz~7a$t_TAT#bi8rG~Z#RG;4)4QWooi0;Dw(JwUMzwk=Cg@n#%^Id*221kFQ}KENiT)9 z-}xucp9{ukw{S4OgFj^oSP^=#5xZ(Eey+hEw^Pf7t@ykPJCeUttR4=MMgQT+prOKg zwZMdgm9-KMQTSF^hX0?yy%Z1X)TR!6GzVa~d<)r>Z-v_)55Y4&hAf0DKEx9nFx1&k zMK`QCDy&%_qR@Qz7<@uM0f$yjaGSr1XFdhV#NS2?N!*AQH{@$pX2S2 zZZ~23P6!k>^$@H}e*uy`XYr)lLq{^wpUmpcT_UXBFwSvZhi~5Pa3Ibfj@1pCE^N#C z_la=?_!ysWfknAzI5U7qJEs*8T@)t=5BW$~$5Kx2v1GEacD=|^RlBFK)?a~b@r71l z{g$pLcS8=5&A-=xYEBNmrNZQf5>&L{3|S8Kc!#K0p@)-9KqKLMYR$n3&g5y)%$V=^ zi__@&M3D+hy=q{r)(R$=y$=U8FYr04k}6a6fdkQ>qADhP%(3&x?0^RJDrl;V6TU%L zz3$>qL^QGV`3MY0gn>YF1ut>PiCIwz5NHHX2r*- zl;F@Q~) zZ}*u-SMnVPbIQ-bjlri;E$y%v7_|ikro4t5o;o0$5b?RNmJ)_m{+z6>$%R)Qrox&Q zuNKyS4;k_H*Wv9?(BwiBrLX~O`0iB0xe^X(@Faptj2s0IBRfDQtBCCDZzMY3g}a0e zlUS7X5pPiyqFT;Jr7Bj!K;0q~tkWNb3_6I)IptC5X050}@D#Gv&ktl=58_+qb@E%7 z=PYdQVyXK)i{Rfu(ZbfxA};j<7;g9mjs;Dpa^sp%7@fWNZ1FB5k37IIyyA+ruudxT9sKZyKJkn*(- zJ{0{)P3bSi=kWF0h1K}I5w>?4jEemd&6KzD_WZ>gc=8>x;B7GntPIdzWnVNf+lv}Z z+nLFkqu{D%Gu-e$L#FgosmJV#%tej|I4$+;H{{5a*wYV>7uMz^eZeXcP3Un6o(C`d zKv?%`yoxwU?i&Aqhm~@&T&yCeMjsGMN86WVKV=5Uyi0yF%$Nrfw^L<9qR^`_OX|ho zK~KDT305|%JHa-VC_<9Ri}npCTR&mCb6NKa4{xFdAv<78`dD(|vxBbW_o@Q?FQduh zy;1VP)2Z>iYn)?Y11sXMpe?g((F7GKmDxT7bKkRHfv`17TQin_yBq?Unv*nszq@!l zE*vzPt`kvU4v6)ei{iP)*Mmtn#|<9DyAoD|p~YPAvE5GI3%mGxWIyua=0aW^lsE7v zlxK9GqWos{Oc#wKHxicyO&jSzpKZJd^HaWLCe-amEfkJGy@Ova1(`oVEbI!IO*WSh zdmmd+$agCWndOXI$&2Z8urdkjQ06yuH~R}@s<9nSC=4XCnVWvc{}Q^GbBh;# z>Ru&ByXj>ie@nPM0Lh#tU7o)LLlQE0-r;Aa(ClmV|FXz)AQfbImp;;SBt0m|m%NXw zK-m%&@O*I~`qrofnY=LSx>qg;6x)+)gVRs&XFN|j><4q-+=Kk?x9BaR0?U z<-IM{`0UkD$~g`x!m5|k3v8i}tmIebF8vN7jV-|Qm|!YBavhjQ+@xv(7sDJ)>lg5| zFRbpjpIm251l8ymro@^s{vFjw_ivlUOcnkk3Zt<}=I`xjg5NgjjZ3LtTP=$HZY2@b zybn)&oq4;Z2h*C}GyDwe4QEn6y(O%7(?7zRXHX-BKhu9Fe9bvk8O&y>3y9IX9Bou- zt_$n#N99_651s|5)$n1p5+1ftef@tVUmjL)Pf^6X3g*}fJG6#Msa4RstJl-Wg9k9< z`K+WaMo*IXIr=u4vN4vThCkt-aAvOwWL$9+_Nu3Vd_oLrnOe;p=$}tTyL5n$Wry_M z&*%u=e^3Q3Kd`g(xQm|!c~T!YB=*SJoLqVa&y}y}Yl-98fTE|NGk$|n)T{@bpZ}6f zdFP>ZL+#*$>1XVQ6A#d33jRV>D%KEDOFVjF@)N3?J#-d%d<_2HE+|#-8am6AJn+fB zM-;}XsJ-DeblAF@+R;17$$Ha?bm;5@Jd-yGY`4?>kyzN0ev6oBR zv6AYTV2xKryh>*Lm0@^WV1%nEA121!)Gbo+2jkfNS#7n^la(2b*Gp{2276= zHf<$*PH+R;w|jbYI0d69UQABgsp~!*FAHfzc|y|Z9VIKsW6=^i^u> z!fVi0uVeOTSwZ%l*HIVcWptK+t5l6b&Gs7B7lfnqxgRj-h7UTcG{QRGg=wWV)T!n- zGNN3`ocV5FG)grIe~c3OR9(rY8F(2+C1v7a(_=7EYp?~u%)l0!Ps!0#e2!0I)9+>4 zC2VXrC{_KzT<6p-X;J1pVe^#e{IEtg6tBbN(Mm&J?A)2e@<(t^;AlEXW;k84&XIXF z{2Q-v}CrY_`^&N4iQZ9 zVI#N=u)fEqA(yBE*Nxz5WrV7PEnx;WP*M9MrqS^WPP1i~q%b!Yc9VU(2z1ZuK0P{X zJDWvI5%FdwNWI_}B65f(j}tC)TG>@;k66#AIkTv~zz=!G&xG0;whFCNXnE4fl=Gtu z$bQ;4Xk9OJiMOl3(5hLQ(Lv&up zX+@F4nLQ@o;HFFUEBzt|uw?@Zs9*b~%<#_YtUq^7nSST2{Y?E z;g6Ry8*wiaRIXz`_ZzWJmxP_U2$l|9!7QQN0#jt zK|6f1bQgC7w&WB_o7X-N_WM<0MI4)W$2$6pvlEz%S;gL(y^Q*b%!dhuqa_Rvumjp; z=aK7)=yN}7whzTeVC7{F4|Qk4z|xs?1eXO=Wl1sqWNfB>2TWr2OEgO%7aBH>bl{ii z-Qc~>D?zCsj$EY2Qyr0WsLctl=p!-F;FkJ`n=W|}C@Is>w9%$~8mgBCV*g0nQz)^7 z%!COR`Y;b|?CGct59QM9kHK>OAz^I|O$(^PYnB3*;r_K)WIcz%|OFAQMSbvi2x>X=%HZ;&e1aazJJ)QOj zyA0W;PeFCuC3sR@2nPb{>4~HMh69bU2yyCh?j(KYNmGq3{CPVC1?{1(bM`PZgwDt3 zo?Gd;E;Fx^9VNPIWdqNH$AGHkYE;x?8&&DHk%&@j$Ya?#s@1Il#a3l=TFnM3PQCRM zxp+tv2E&;c^UL*l;{hdX8OTjV-d%VYFqpm4Z9J29t}7K}yn~54a35SU{S3yYxl>K9 zdzclCzoR!*ui~7zu`ttX5zJ94iObW0PF?>Occ(=^k-Ly7JXt=G2`%#$=lAMzg;zUJ zC)IF}Px`+uq8p2Dk6d(}Q)3-mxNg5Rjn&w;ZQHi9V%s*FG;HiN=8CNq+qP}{WS@P# z|HQm_$9%?MoVc)99)qMKs@9@^L3qatQ*%ucP9?r)?Cf&gnXx~%&W($r!j3mp-b+`O zkGT;v7LcR*dpJADb-T-h+E{7Vmvm3g#O;Wli(}VBw1?W73v&@I_{rgU91xYQ=KpJ9 z^gX$4@;N5slz5{DcdUnR7;SvR<+!Sm8mQ(#&u+Fe)}Ph^MbXY%AdAenQ#gmnk$np{ zt}i?Y940~2fYK*=tF;;N-(h`3?1nO3Y3B^^;2KDG*1fJ&l4=IvE_bi0xi%zzs2NYY z2{g9-poMHw_QQZN9omMV>`0ml#bzQna2*BS6Y|5{NdA+_4}vRj@OS(3i)J?*!F#fB9T{hlDK0L*5g2W8{rlg+|ttxF)(-} zX2yj85mybk$PHhz66&YmfCWoizjYP>uBqpzJK?>cBve#d$nJ~T%|=drk#;9!&XG@G zw+WWN#uenn?cD6fEpkq?7cAn7VKKDs3q!rOKSwjz3eS7d1D~qdE!C}M#delEJ_qzy zTyIkVkyO*+-Bl@@al>ZMh_g+Lg<~{)F`?0*3vUg%A*H9u0j|IwoQUwWP2-v?k2Hqu zbGW9t*|Ze;AII$C_NJ^P9nC*#|+lbP>W>=OG{u!ImFjFq}pASrno8s^Nf8?`C*&w|nCu!M}X zFAL1mUeEP~NH0(YmRh_fq*@Z1$0Z){UETjKV|2qxzIBhN>r}{YYC{bh5MM?w8Y9U6 zPy(6qVpGo#1gy!c{kda5q>V6VTfF}dJ_@0T#?Hd!Zo4c3WpK93baEpvn+s)q*W;#L zsGQxY^-B)zH|lZ?;eM zd?`_#K-wBkJ;UMf!!kxZ*Ca{UStzx^3nJmH$NsX6L^IKB?4KML_1+4EvB%uO7-HW- z^8zN`gt=XA&?cm9Pf{pr?N=AS1JP6*b*oIQ^-J>HJ$Tdet4_gK6hOw+M`3Z$tpJIj zaXxHP>a{&L%Ue2cXYAgCd2Jtw5~G{pHEaXs5$DSI8vgR|LO))|ocbfyq$P{Ngd_G< zW$Pjs{I0j81FF7a&q)H%r#X9(;);sy4fbANywkm9(xm?n(Y8$xS8SHyc{^JejT=U0 z_4DsQ_alp${fdb4~}dS zBdS;PnNUQe_Ha##cA$JZm9z_n1GW#i`Qdor3qJ;neCNhoXV-43Sjg0h-|cH~Vy}p= zed(cu6hLSKqeHT}ssk!|d|_N|Q0Kj}qUbj`pU^^)^eeu`D|-9w*xPK0TEi*!l{16r zigR}Qj*Y>bS=Pj}$5zpH{I|A!?=aYFjJVw4nAM&;{B;^=BY$lColWg|-5Ixv<=rW{ zqigf+rIEgitNjD{ES#5VnN7#BpgD9yV^c_zdXNJfT;qf2T&y1__>+`o=Nqyz{g1vZ_VB-x)C3s{Y(lFok5zbkE4Z@Y{O*LO zG%sIkm+BnO#oNc_|4uaSe3sGs+`{BgA#mM}q`Nt(3w_>9rG5Ui&Yfw3q2fijT&W?C za^&m4A2#;xSHH7on6$-dS6m42>R&Q=$M62)pV>&FlQi5*jqUF&m9*s_<+!F|E&O7( zCz_M^6+J<5I(f!uF1{y8<>keLs=UI-^rX~RG|$_g_?UQV{}$Bo%s|#n{pl)kRr9PN z<49h~HH3!Knbk3bB*I_7dp&A|>cN0l!Z zBJ=)sb~8@hPI=w>x^(+Y`Azw3x?Fx36D5t9e-Fyex-Par7@dj4w$f8x0MW4~Xx%HT zm`*lLtv{rsZj_Cna=dWpGhoQm--im-e%lrxl+D&bu za#T>OE&oM$$H{twfMzoxjpcx|xeTNIhZ*?D#(MbF@*t!&K>2Sap_QE%AI|u;St5A0 z*(c2cBgaBB?$_e&+ESRCUz^Y}0PyGh7-O;UE*!Wi>zww*Z_I3E914^e>n&&V3fr9#BVY`QP<$a=8UvcfUN>6ARhPI^_{}lBkd0K8L(Rz*(lrN zNuTk0n;4)dVPEy(JB5TjK27E0CUJ)fat+|q1G@%^wf@Nu1`#qT6rp9d@ANG4Z8*jw zQek%rMAPg`YI`{u|ET=!b?btU6~i9BVza&dyA-wde)*pH z#!?c)g(GW62f*UGX~Q87X`r_MM++FTS^n3N{ZcWA;#9%FwTC_{p=UtZZClepQuznm zsa6^J<9cJ;aCe)+_E0XzM=j8FEui@Bdou-F@tWWvS{y)SyTtVlkM-ZkEQnn!U>+T7 z7u@I199^j2Lgd1S22-5Rlm;NlIPzZ?>?QV7NPSZ~1fra2A?TCj`MHbHc4h(W`4`wK z_tY@j0qJ?XJCj@*PKZPJ*>;XI7Z+7uXf|ets#ZN=E^MZ<_~GJled_L3Z~1{=PUt4!kzeon#>hY2xGsq$3ssk<@oQQpodYJdH3p z@X^0vjfc!Ulk=k%#hLO0yVm9Dlc2xL4;plKBXji+a(6byVT5}-JsmW7ewkR03^fwW z7H15&L*xM#Oi3&{lXPiQ=cWEotLu*v2eplVMF0~hquhe%y_0QZiIk<&Q&j5!PQ4I1>*lP*?6DA9`JM$J|u)5LPt$ z1CNo%6{*J1Ie0QV$CrN>GzqT5Bf>>xoLx8qI>V2c*BFe^1hz@OJyswEb=b9!YfO!- zFv02wVWKxYYmPpQFlm_qC^t6}yyqaj*Z^4X!GL@k1otDOL-#=}>}R;O2Jpryyf_+~ zMhE1u8MPu7RyL02)b(+o5KIU@Fp7D<(W-*ifSy!B{?BuIuBhIjTpOcpC$Y~>JF+<1 z=g8eJ#un=rM(0)|ZZ5Q@ppZ?&xSOqsR6_?)b@d06X(>IAd0Jp>>xHNwI*zC1Q)Jrf z7Z-MMYk~h`2CriuoUi%0aC)@Ho!FZ|5R(21)Q_Yno#F=z-~mg_#d~5_hBy8fss9S3 zJ#7(=9du5Sjbh-YYr~ifSZcvKRv9x{F&JZ3D1JbqKJUAnLOQ>B%%1!whnMl7fA$hA z?;XRlV)p{0ZKVkP`MZ&^1rAjEsjF$x87v{dX0b0_Nra5m#scyIx?zbM6oQ}3T5Qe! zj6~ftb#M8Msz2uc5hJ8UE_#CoV_Nh`9c`m?a_-) zlH8VdHvRBUr_` z@sJRF;q`7y(HB#7;!#;?e7{9LEFmt}b#D(22?_u=6Y2N=T*@jzMF!f{1*@HhZZ=~ z3WV$*S%7oSoVQZ)Y2&->EB}F;iFgq`2wD&zMLzQe8jkYKd zhh9h#*bA*8OVbmdWdtP%?80D^1skkS{xCRBoK{BHJu+eR4^?sv@8^@H63X6uCWz)8 z@s1A*5sKTEghVjjiL{$Y;x+T6CJ}QNF!mp^+-rQJD{fi+!E0Xj7w`K_94XI4jS_Rc zP8#>^bLMkXu9nu}ucw}`yszI3jmDbeCr3&bj7rE?X}EO!06hV3GGY+=>6? zAalgC1gpH>5T}$v*F{2UDWqHys%^C)rjhQqQb~u6spYW=mo^^akEGkEuCeguu!a5v zLwAP)luI_~Ix0o~KXFC`^_|BdNd!8`&JiAablMws{BA4#82nQPBChX`O5G^k#3Z8= zrERL#l|XTwhK)xszwAv?mI@n&c7;2lU^l|uZr``9opGi%{7!w>C=TjLkiv?@A|*o! z2Q^Ybn}6o9RCYj~jJJQ_;qX|G-o>a((h|1ABf;XO1r~ZC_{*rlsqG2%caA0@4kJ6p z3Ga>*V56$}jSoY&c7Q*T1|S^aV1%qjws%YQbQkjB;-Htv;)+H`TYE6ajqL~gOu~HFHs7|jD3v#!HU7jv=Jq3cEC*h&i;w@u)+2@wq zUsQH^-?6mu7KJ!3m%%~xu-C2rfr}g`X#9?6c4)J9@YyOio(j`NoI8OBe8#72;W8F$ zQuNSV60wE&WY)cgL0)p?tZARFYRj97X~d5%&q{URzIgmnOaoxn%?I{qMnILnljZ~3 zms9{j@VU7HoW4Z#aB{SMCj?%LO>l z;6*wNi^JZ<&j)gVF2k&5T3CE5ak{JFXW4vj^=SwQ(m!%?DP0NToD6|kT3mk&sZK+3 z043h4Ximef3lVB|nC0WadHI%tKjy~aeA4C{cS|_*p8klvty^E%Z;!x*H5cs3dbFOW zUdwJY|0xn(yf<#d^t&@Ni)>3Zdb{eDhO*|D!{i2P03~Jt-0(rQ&-`xiT*p}*#;e4O zlqFK$f1k3-z#VBvb^kxH z-aExWQjNUyCYW{43)JQN+gz=%tKD6bOYO=zAcRU zthlNO11U$s4WYmJ_-BQdfAFn^Lvbu-N*~k)#l1Yc&{hRZQFLZQU}en9-wz!d0O;M9 z3;O|#ON?Q=ypEB>I~#mhvKgMTX2v zV@Bkr5;E{``pWQ$oBx-N}I3xs|hIrVBXh=EVpP)>-8!os-HE15KLqeN{iu zseHa1P_=jtf5S<6zhSUwQIl+pgKli?~ zH7y~WKrPZqpDf$<%I&6N<=XhW0&eMK5+{p4Q;mMNNYTbrZF@hs8eTFjk5ZYDM_;|N zLR_^PfWM~x-!+vnCg0z#&h<1GryCbX#I4$pYbLe9*p-;8e&95a0@xJxA=XyF}~iHtY24dW5tf3-MaBi4@SmunC- zhQm-&n|=A+I-9lFh^h0UEk}B?Kh()gw^%V(#&+gA=k#<}xL``qxcRaaCwm(2D(1b~ zNReavt6`zyPh)=FeWLN=Mrf%ye@@uw(`!JfphIflbY&6O|qO#$PySCaU%xpxi? zp#V(m0NqYi!HMfLGpQ01FFrP=A%cc`xGPaOv#HXEi#fw{;OTkUP6~w`vbB||ulDl# zl#p~|^k)?SGCh%9w0!KTl)crRDo0cBsJK7-oa6!C&5-_IUFg?%Th3cl*b_L9X&>x! zyq*Hu;9;)>rUzECPoR!b6L?qxW7vyy?+6IdPv*;(px5BXywT;BXsm9Ru$-)ca75=- zkfQ<1k^PlCU+o5j*~KZY*DTTh&lp2lXC>tcWvqi-L_yTxX@iKP`IdDi z8s#2jcL4naV=7y{C;F$Z=N{l(Ppz#9BCKq6IH@vf-X<4f51SR!bhh$D=tp%AY-ZR{ z}|n={#H?kdPiB)>S4E{091^x|?3p>_p zv2N%Pg>CT!)i&$9lZ%zGM{U2~_=4Xt`xuh>hnp2?1C*8{=?XN^IL?2uBeCJ{fzMzp z%5lUWf8s(~E3w)U0#d?97!C-S zc|HlC`E0_2z-gHAS_gFkgoHTv86=W?7RY~K?aCqTMkn=sEX15iMYh;pc{5}GeU{`md zYpamxbZa{uyVc_|0smSt>scOfi)y^liG>Gw=5xXQPJL>073RmpswCTSKZvpts`>3J zjyvb@1nHixFqx(>;zzSYx+HDn&Z|E8r21D;Tf%eoB27`(A-o*{2Q&ea@-4KalU;QF8xuAglT zEp|dj+!4l7DJUwN%0&d669u2L=n6+NldLPBSeo6^YWzuYdjaXVyB!MW3|DeKT>CCt zK@dFrKJjnequuqEK*srXnb=V9qV)Oq2a8~xYnmhpAq?hV9dm63&dm3;NwM1SF7YGi zAF=-m>(|uEx@UXR8(In4oPAhAVYH33BTg-Wu)#_tj z=WfwA$yxUOYrU_cvE;Al8jAlt0*dN#@RRHyt`IJAbzGbkj;tsDPOBOtU&uEhoBb-dl-ZFj*;#NflGO&*Eb-Ct8mZ)o%O!?QqZ|bX+ z-lF7EZvfTSkM-SY+9YPjLE2w-BelI>4L8dPm?JgUBbLFFNjwz-Qgru^!7hb}Y~eO# z;UQZTG@7;5D`0$TUE+kL?JnTrk7^bTW~UXaQYIpPv#xJ^~}`CtE( zRxlCYNETpw#QSo_Y}27u?r#!(AURLd#`(a=^Ch?~_h-wUnJ7SwA}!C{;|xAOy0|Y^ z2T_LE2=(|epJw=BR6%EpCM)9Sad;)yr8c;^CSPvyF>@)(O{liC%2Y{1H6vJ-{egN( zW(rre`}9)MIUkD>xZ+N)aE`CLqMFG)9K`0kBE9eLfL>e};er{RO1z3jF|zFkcP(~B ze$WX(_fYj@OZ(#13js>w7OhO(?$;h#%>KzIF1@;;%zYX6Wq23K4Ke z^NWlWBb)Pz9W^3wW7zT-Kb&dTdC0e&%*`(<7xbj{KtGaiMddNF?0;FR4?1^wjXh%i zaI|`67_<@i$LQQLP&Oo(W?;6TI=?u1ruNedD!b|iqK@mF<`wmgqjA{-sXAjSi|MZt9@(3h*Mdg{6vAo)zkT>aE7}r&5%0hjy8}=0=&R3I^&#hEu`cKXo+pEIxHiP}O zpfXxtIH~`bhp1J;^dnT~bK&3bGkt)%F?NBUk@OAW31!urCw`(_Q&+;GNs$n(r31j;jI}siJPh?NuHN zmkTX)PGIn>wv4C=G|x@H(}8Kf5bzApCAbKc|)?B^ov_~QtNbA5m9L#$pV?u`uEj|HI|i26** z#6D;C54W|oOceF>690aI&$2=~{`p@A&*C>fxHm$t*gaf=#zT+!Kl#Dq5GAqO?6(b8 zng!;>Zk95gENRo{D(qL{FDd9y0-0aIfeb`a$WG#5nBroXN+PNWMUe$RR8oo~qYKc` zQ_%gN?hapGz7fKB$5rQ1*42~mRI3Mz1Vs|P$1Lh%+j#=e-l7>=SE*KZ*>veC?BNRM z7st(T1c<`xlc@4-E&|l1m%~`OXTxXp_caArmT&6+o0T40l_j;;o2R&%Yl%wpy5{x>(GZ$WmsIp4}#pIG&q|QDijXyZ1K$=mGfmYMxsCqcE~5FuVYFh^w4;B&UsUgg|D(QyVjZ;+`%{h>dG+mKqBK zk{C5aUhH}y2h^KJM48>{06M!#=5J3_LoeXV(3+~;LxA`auz(?n&S6+YE54UkGr-(z z)Y7|qF)%r`Ib>knX1TpT(ZN$qD|5{w)NVzh(7(P5b&)O-4>5TFF7uYMoz~kW6u;#G z!fAutCmdPq!Yj5@=B+%`PVN%2{nZFS+VE`Uju&w1&-kAi_#*Z-5M}Xnsn+*%KGkE& zm)zD7*DIRm71yCN(4mRa{aX43#<*;NR@|Z$SZuQ+RI1FfH3@FkzM~k5@Y;ybgoUC% zj(bf!;zf~FIm8))<($)GEwy}aW?JV7mUV9>Kweo#nu5;oE+YmtB09P0tHj%d7UNoZ z<^3$6xS9j_P`nA){>PLP;2XA{6CBaFg={i8t_OG!P{7n^CZL$g{%8ayef@W4tICCa z(U(-Uu;0q%4%fwR{Ua=YD@+|hao{M7-4i_vY8R3|PxS}qE=D}U6|HCW6JGm1h7zy8 z#vi@_auE@PaIu45(G|o#*cT)Y9GmCr%p^le_1nkEviRwL4s;j5G!$T4qBIQ$K9jwY z&^bftUqdQ+G(WQW()QGI?DBxG`kn<7Q0;>NAnS<5)~RTJjym+v_%3SLg&-^swS}R@ zhx)nw=PsvM^Gw(R-p+MHfHI?fs*(R}fnUelx)>k#Y)JFMy!r4fI*3O>fau()?+&=@ z(Q+jCmD}M%SOvWS#pA_A4)?+@wY<-G&02ybaS{-#sf?3TS-rRR+oT%ws?iIUDo3+a zK-j^O+d*r)uNHr+__S9w8W;JrAT2gCI|7UONe_U<5ICA{Ak%bZ+*jxZ)>2O1RwWfW z3+D-rI;N^S>`n>0J7`Oa8Iwa`8+ujtq4^wL9!IXzbVie3{Bkb=zEqq*LvY*) znIhvONSL+Ef3MGyeiOq^s#qKC#H+sYe9R7TNzO%nfFAv2yw6za z9V%CBcilgw#InxjkHF=wEUAt^Z*&ULrd5Em1rJr0dMunaw+iNdGLf>Z@Sf`gJ+EV` z<{B6aKY!W(6^h9Ki|Zh{SpeS%I&7?~JHl}#eQ;aWIu|vsT>I+J#eV=%SbX$T@-z&~ zz+7tE^v+!2Xji?MAy3&BTX>YiVm7@^sgoZ`HAWky$kKkDIF7W=o z?CG`lfqR^xPSVOe$%tG3F3n}96uvTqFG9;pl3e9M!ag2dieJ$Y9X9q;29JC4VeBeN zUo$j!Zy6SJ8^&KEVyt0U|1fhf-D9#MC2UH(({wm|@9=EOdxl+jYmaTxV2$gMf;Cg~ zfpC`y^|zNifWGr|3+Iw*Qp$kDbjVIyfFCJlU>WcFagqM;?*zG(d6j#PPnXx=s48Ru zS7zat(qXw_Vc$t!xS@BR9haDJJ$6oB)G$H0Nd6l`rj5WvNlT?+J{Opw3tyXOLC0Ey zpq$j!Jk8GstueW8qb;tWo8h+CpqK+6cb{$-9mUu&IryQ?$zBGQb$?k#7A}FPPy0h* zHQ4|P&pVoV8;Fh5uSJv0Dz9l+fBB}$^)YYiliXIFv=!$qGUv=UZXxSR7moJ?_iz4w zNq6lPPg|?suX_1@+(g4j40mpP zxqh7BIYj6z8$4P&Ch}XFRpR1rXyboo29E*vtOTREVofI1w;eOKEeX4W4My&mgszmr z{Uwk&$B+M%Q}ny9E?Dbo(aehp z`Q>BiXZP>GP$MTS{xi}RB^XpJL80yIAp!kF+`lq!YEZ6kPuN7AHiRioe%UAUfwRtg z*pneOh3&(19`852BvM>)c5uweW%OA>>5GM_t~>}_86k`GHOzOKkt3!nIw6^b8sPp zJ(twmyjJlJ|J(zt%g=YXZqubYAOb)b704R(aOUB3YKcWg+%tc4r8-vKwaTBkb0pz` zsVceBYZXYQ(d&SW<^#I4Vl$as&04MHSDSIo*365r2i=o_=GSqPHaM#K4^kS{|<8&U6*B*P8w{!uY}_p5|JTyT8qE!2ZO z%m_In#Q3j;bMSQ~8SeK3xZFU9H@4mkv-Sb4xa*$p&?FdKIiP-fk`7OKHddAp!%#eQ zZ#F(9k$MvT!0jsCoxaZ}2bu1bd5Yp`=*} z`>dBJ{x)?Yq1Qo6Ld{yI4LQSdop`PrHA#<<`s~5>j6G1h1lu$Fhc7Typn0_2RAHWp z)ud3oD|l94^%YvHyFgFH^&#X^Z92D(pC7U2{;nqsrp);DTudYM=b>hXF*swnc^7g< z-A+gs#iGZe#9j-{K(TY5j;NqeyR_Nh&AX*A1j+ksCVgcpsV`sKXS<4bxESv+NxCNR z?~IO`)WzUQF$eOYRt%Fk)x$L9+#C01zgn%%{9P6zk2gSe)EnXzR>Iz{(OV^;$)( z(;KG*vP6IU;*wOYIvN9M&w8bKr8!E!IK+9IqfFA))ZXDWT~)%4jGetaKAom(LXayH zA1Z3#-?Y}_7p0R6B#E{$OUQ{5xy|90vESRTI<>_ZO7EI!^7e@nMg`vKp8+43P>we%GmIpj@xHObQ87r<{A zfqN~cs)a@+Kd>n~a)+dykUQ#4w}4?^EXOF-dH|l{P*CCv4=WNDcW>79*6Aue)D$R##C4_{#%bf|DZ*5syL)4_r&68@R9SIL% zZzi#;f7S?K+-&0y`#QN>wo#y}E&cjKDc!W4d=psa5d47wFqu-ypO|WCbZnqxIzn^g=&w*!(-chuN}^1{1ATBWN_lx5&1X_sGwxJp}rb6nO&1ru{-%+iz32 zR`LL(?L&WTQe9`cAZ4*_%CuTH|3QdudE7+9e)?AcL03&m{O?l!-OC%dSJ5gf{ZPPw z?m%}w<;q^!+|8SkR3E=un+@bu#`uTYa=7e#WfN4fHf`&XG6O&Wo_{KIT4xMTw<~kR z!qq^9d?tYMBG`f3M|iRP$Dygkp6;yyMROCT7VD0Q0s^Eex8xLlbk~43bNW0alZ`fA z$t(i@aMLN3x0Yi{$H}JIP}(;%T_u$JUKe7;;0chK{Bvjj^VfG(5CUf?*!__uwn}oD`kc92ZDCKZB!X$rMukglDjlEj?p) zm)i#T$T3zcX0dGvXX|VYHD_cWyDwrWMaR!m55&-EcZF%R7y!(xy}|`0 zR+vg&{(ChsOdh|q*|D4WZ+*;bFe|+}ZdLIme(gU91VatS&?A1tx#UxOr?or*u}-wE zp(71swTeyr!}oiKAn_ZFrM6n98bl5pn?l=Q=33-joj20Z}7FkDbunqAHvr_mpD(UHg%fC?tmtzwRc3QYrUN)9aYkY(pHZde;xJcO}YlkTw)2+&0D^mbhM;4*Iq=eFk zhQ{SfIDVfFDlDl^xO|;tmObm9;5K#@0UAUW(6HRl7~#~f9HU`&ps=}_c~%Exw`rxx za^O8Fu|rs_kQ>B)T~1AJ-bX(hEsC0*WIeart+RI+TPD9S^^7)<9xu-_Mh=igb*b8e zk2c#Z$sOKf?gST@>-VB&W3G96DsR9r z$d%9-K9?y_*3sGUJ6OmDtjWha|<*fOUjhKh=C04vU&vCradM| z^|d6Clp81~+SIrsiGzRL@E&_Z%dt&coJra(+2jJjc$ETyLiQfX-{tN-txNMcT;;+% z#h$SmgAv`QvWfMHdyO%5D(MpxEsK5-%bQACyfvf=D(B3`JKT2M=hg}DKUTTEhL`7n zh?SOBu-ra|Jlq*&-+~+RzFW>R(i-jUG*uY)e;(}aTgaOKz}nDKp=!KO(;970%eqRh zu9e=1v&OIT1I!fmHOenk?`iy5xY_1IG{FqyoDfO*(rv--KFI8*mIq_2BZ!1O32JE( zmOW(9waCaE?MLxJ0kZVtuB|c6y+g|zzVJM`JkUI2lM$m_Uj@nP!Du0kM`Gqgm$6u)CUIX-UZ;PBP5$L^DJFP=D@SL_ zRdv5~d-r%x+)}RyIP~f)G37ErZ-t#_4zi3W)mv-qSaX2q)O%!IQ}YNhdb)egi}#0_ zeap$QM2)C(!#m0$+h+S9`JnRvmM<0ZVnEIJjp0cA*;In?8|hFsg2zap98RwLZ_9eK zSi+evolS!B4p>B$n2I;4=9M?t>z2pe3v);*Nbz+#ArJCCFZQ?ahYNmnxQRI-ab{`H zK8w)4p>-TZ^se9|KzexK znhf(7Wvrfa?r*edBz4FqGHZ~|I4asA0k^&8Ha2FhJ>TmBO=Q=*bcgfGjXBKhgnQqn z(^&K~zZ2FzTN#N|xw@xpk!gmD8<~EPbnCG-$ab@178LcuO{d85=FGf5fp<9XHkZ$M z<8@*KsUjlU$nPc3e%(*Jl5B7^D!pDj!@G$4lX@H;W*$R0RVW)3VdoYK{MQHqlrtJ5 zRba!bm=#&~S3LhmrE8x@`mH$=1LzLwnpXoWMA|x+vI5?Yz=)Zo*u-~xW8bLOi9f*6{oW{71a3=DL|I8*=fq8YS5GEVNf1E@p9ITq!@!EEq z#HVpEPZTnloYel9#h=8{D%8k>p`c%9kv9;EVMa|;FgXe`xyC6MRTbG>U7T^6MO2My zLb40JCNpVp4`O{3Bb!p zGAI97JxijqT_OIv@q%JDVcux{=}+|!6^2@vT$Dgnh1aNh%9S^uzZkHve_=F zbU5<`3;n`KNy^B2{af&7G!9=>wFQNna&x4;S#4C71^*sK!G~a}BuK|s^((O1O7n(T zVSx44r}Q^VCJdkL4`oH1{Q!75mFuB%oBMoIUT@ylc7r)yBkQ&C4H0Yn&XK}|7>Lz^ z7lr^OGE|P9UvC;}GXPSflCQy5X|s`cCAVNoA5KVlxj`a3yvM!~DIMvy1qJxcA)a2a3B?zb4(Ytg%6m^9YmFL$bb3#VqB}=Lk%>TGuh>5M^29~ zl=j!wxhX&T=h1v(G6&mkr~<4>ep}RsTd3@{`aBtoQxAH8CnX=a{&l(kQp|AfDcUB( zj(*E&27NJnP>$Q}HaeQ>OjP&-SCedY7N?oIcoo zQa)mkd_2h@NV3GWjFD0$V=(Tr_i%!I{hrZi>yuS5#pkWSUE9Y$D!{`Ai8_9AZICy= zU-vCkn`v-31FF0DYi{Y1Qu1p>W$ruau=hAc%B_mG`njNm)$l>u5G=9FTVHqMN<{`mcLA6szKYp~7j}x7gnp{WWDd7!eP> z?f-xZ{}U39V4Gcw7Z@Q+;y6ifke)J~^&#gkj2D51A%^+IM5GQTT9))M+2qda-Elj z=j>B6uncn(^Km7!Iqc7}7LD!un?Hym1GpDzQpsUw9SUS#D}m$n&(mJshx->s|ExcF zIaU#hAou(e$wCo%vF(H$;DgZj;xk41&iwY*k24#E<1D@iIH=nMedg?00jy;n3Y*|_ zlhmq@7KEYsZ&zn4P&{PDECuvrq?_<9%$1VcjWe_lZZ zBungtHu~*J0*-L+e7mf3fT)Zi5^Nj{R)Z5RRUOfto!P4|Ub@ zvbz>*niHBRAFU3g?Y2%Ew;#r6`fz1_TNH=+Ry2eG$*thFkE;HGASd=ex@1$I8$Q(m zxJxMQYt|hqghV7$y8ZS(|7o;zMR_>?F6hN+*Iq51FFc?|y1mQ)F@-0G zrmcCG(z=T7&vC^vMNtr(_43#E+7~>_4qCUD zQj2p`xLttw0RlnV$%+I#B5^T>{S^aly3;nn8$H84ir+uB~eWBQDqn1;2 zi>+7!0>O&_Cgy=J`e)K6rU61;~${c&;+7{zME(r1S*tR78 zu2_NV%Z^LRF1NnUN*lS9{>%Vb3$;wGAs*gL{~9QVcVT$lw<;dUm> zHA3d7q zg5%MRlPjuB(;apIVrZ9eHArw-VSW|h#`&Pw#8B}-M5Tn`9e8I4l;9U`vhxo1q|~da zzK;P*Jx6^3nJj`SA2r1~Jj;GM{B(6E|K_NlrWtQ}HDGja3>&CkWD9^Tu^FxWLmAi< z+uM))?V-84ivA@^Q8`x$aP75~=`2F}a)uBQNdo)Qr$iqELNK?~2cGJlkOnW5GjHQA zMr!f4Wxym2B~mZOp3+M{U=b-Y#XTPxTso)Mu64Wcva`jsf*r}*UdD_?J?^oV3fVc1 z8x~no(spaF+w0#49N$U1K=kNBMOF1WG`Hz#* zdaF6L8#h=9SlisQXhGSP-UW|zQ8$Bnpuz`#OV;h*4jQgJ&;@Z^CI5Y-zunyrDW3|6 z`m|r{P+nGnqfzUIBadSn6s>sZ*Tw72cqQpNU-Z;*!96jvQ8)I{Q zfyTdj0EXY~#A+ZO5v;yJIj_+9BMZF>eLHKs*}0h6XWTRARLOofJDgo&{~n&*5<_l7 zn`}>7gjxsLGqfw7V5}MC)u^r*0UfqYYUoMl8g2&R&T5z3IaMvBx!CKRy%-|ExUXz6 zA*X^bAeHhv@cF4rkyVOMbnX{r?Tlmk5!2U^TSuORk9k)-#XFz6SO7i=Zw{?nhl6P2 zH={DZ(ea=l8~}Uot+fg*DzQ}<080FP%37Iw^J&%(``GottOhl6P_%}qBr|Pvbx0A1pAB>s61A)GRet;rXI zw7OAnu~b~Oul}jJfVAGax3p?{zTpH(RG!5y?A$x3?bbdRPkRk{ajqP)E7GLCP-$P; z%nt^=4TKsId!{J$ju3x^t3H=@#pjeh0;DVqQ#i4AS>T7_M49;<~+6^1Sg z%rVyS(rg7eCvt}?lt)ACEXi{GQpO6Bzf4oVT_g{giy_3RwpgNyl2Wj4$~KH2lH3|B zR+i?xSm|K*S})HAK$EI-aquBReO12A!2BJmm>KDo?AAzYSYKL$<%Y5S2JgmxY@gel zx(~Wm)}kkJ%wRhYsPg13MS4lDkn$A|zGj9)zax1jN)M<7p8v-MYjS&n%=6xj>QH{@4XW&1uEDxnN+B<4C|=x`pXzY#C+Vpfzy2X*y=Gn!2<_0%zB%Vggl_UXjK%Zzm7|ok-inAKI0vtj#BbW zyopl}7j%jAJkU|1j6%WI@QsW;6$vW3yR;)thpK*dDk*3I26lO=^s{OR5>4LB7zV}}pi~l2& zM~j{2+la_Grm;@LM_o?3B|pSE#n9PmD>=J?kXZWBo@+cJxQ_4Z`T~``hE~x zj{#H8NaAW~VVF%yx9v`^{>d?7R2So!C?+Yt!%@}Fo%dGcK7%+3uPRqT#(tVD&lrv? zLg8?>ZPsnhYKXhlHhq}`HH~aLq^RT(AO{JfQ5~dfRNFmU7$miEjmTwMXpr3ZzUD^V zc+2;pHDRLWndKjOVW7Dt!^zw=enIINENl(A3CF(e^aWUSXK4sH3)Yj+<`XbHp~g?T z(w7u3xyZwR_q-ZB=A4+uT2;d70`ri2)Gf;|3s(tsN7g|+79%dw?y*cJZ%sV-$=~#X z;{5vq(tFJjE~}4d67svs#{+BtRhe-+`Z^xG{Y_8qUOD)pB_oICHBLcbG{XpG;P(wPJ$Tfk=RQp<#aT{t$>!xk}iV3P4ht&i8>PHGsnTm=D31x<5wnN6)h#WBQlG!oD1LwS~ zLY!o<*uJX1EXeMF?zoa2-yl#Qw?h6UJ)-UN_w$}Ml)mFm`I;o;Go=)jIk!7Ahp@O> z<(%^$1leDbd&UhxjsXpghVM^~V}ei`Q|P*o>`CHoc@C-%@a z?u&{;d5Q@whC|{@Gm!Q_=|ZAV-VbL%JXJmDvn_pGX3XB)Q!&&!E;I0N2P}}>echk+ zBBnt~p2H4)eSpMulPrxtU3k2q+yGO85A1coiFrE`Dn!HmM z@ys9jrV_eeoPRGEsMMki_D3bu78ju06oF^{&amqK6NSyE+uxZh=G3$eDyeEzmahVUQik_^kJS)Tp1zA*#wuNZb8@Zn^!!4O1%i_p~!UdEi!9a?0us2`x=OJ%bxT%V)#gW^mn3D zI@SUk_bwPtF|>x}@(^%m0l(MF$C840Tx9v74={NPrCyO!m^`tl^5;~>_DZdeLl1VR zCj+7j9+CZJSBTGeFL}b)THn959El_2o1y0ScO6HN(OP!=ZKvuSqA$51pV#n>b!I8b zE#YB}$!H-YjJFz+&qa0pspOf;C$qctu-71qjXB8e?*4nnOqs`4KEkYu^dTlYRl>otfHk?nDf`EI$KgdE z`nzz}AUQPb>HXJ#{#npAbUix$8pf`S@kG+gsxwMtq0PTL0ga7*CaJAjpbh}QXE{RJ zMpO-JqT{dkEmqdd_c@smnn2V%V}1+$jX*_M?^NPtQS|!?KXcuN*KNhxn3SGBbmB;D zX82uDmSwg}^fU61xu50>TlZvQ-#jMX%348*^;{=rgDQ<}r4^M%u^u2u|5_Ne1=-}d zF-V+m*BpQS{+;33Jue$zrDzmvZ)mOEAIfOJkx>(O(bJAtklGVBaX}2`8L8L5YE66- z_sUR1?$)DSN1rpDJv1KIf*7wi>}oDD1cjk}i)PN7w4K|UfDd{zUk=2G^cut9DZ>otUFRlwG0h7|6V0f2*B8T= zyO8GOX6zM}3k#U%&RfXrSslrj-Rg7&tl(gKsINn~<=y|&2vazgcDY!KNc9O|!N`0d zhyTxTaK3wq!*;tZRaSSL+j8s|^Sbg8(38N6HTAd67g4Y?;gt)*rc}sRw8Irbs}is zcJ)IvM!vno_5|N8d=ObKbRt)@_i`=nbp}ud+~kg#Fw){Bz%+6@J86$rx6Fs07h|=P zrdnd)E2%xLaW=N&QllfY^{Enh?u1tI>+)%Wk4Ic3tyH`2S8GiDhs zH&AY9i(W0HIjq+}fF zos^}bybXB-ln&D0l(X(%ZV7uhxg*TVh|p|?c?4xCv-=XuXmm7E~q_W8MgmM_zeD%k)JT}qcP)Shlz8Of1Fn|*!!n1{PIHHehEIJcE#hF^Kf$5Jo{oX(-+ zr?q3O=}LZuI(9F8e6OJSuUa}HJe%D;ZI={B+naAzF8eH6Epp?1Usd4;kji1VimQs&MJ9QT5Pu$ipf%~(z)h>fdc#EFb=oS!O)Vr*-YV%y%H`^+Cn zH`$woTvJHiKjwc>sg&Jk)ZsxX-3)zK=ip58fj|ArO=!`#A3rdY=D9RLa3x!{l{Q0y zb`goUcfst4f)tRRf4J~xMsAwBp4kelA^U?Fk0E-;Dd%yQ+}tTbFWVje9!GzCq;9om z`B~hJcQlz5w3NyBh=(|#$6Jz_hsBauo;a6PBV(EYcdPQSBXvtql$Hl~$L9@ZR%I_4 zGZy2c>+mFsmR2O*?^?Mw*iv-VC!mTPBUU1hIwLbu9$>cO#o!4bH zF%SFCtwBM~VnPLY`5p($zH@S!><79@Bkid-2uqjzw}yLAGo|lWn=a8@X&N&t;%(Fa z2Ca7COcqnp^tN31jlf5oMNrP=Ca9UrVKtN%YHnIo(w0#)+YD}gk;NjwxVJmTi$%|{ zFN&-QrAK^*f5xM|@Odsd4+_t`${jaVF1n#{My2cNVqJ>0)84nWT~%^peW?J%U%F6~=e04E zli7& zn=K8^@UR>cTGAPan!Sjn$)tFC#d1tfGXe2cJQ+s;oe3h07fjYrOMey!+>^khw+BlX ze1aM6f2E)vh@3JUL~D-&=0{I>7D${M5_*=8|0*S6ILj@td)9P%GKaezfx52ZX)lz{ zg2H64JdW~batBxT#}oDiP3jC}%9DCuOT<6YB?j0!c%FWrN-CbwBeZ&;7BV|%*4v@? zx;M(T+h@Pcib%QCf091MtkK_W{I$lyFf04!H|k@5Lhm?kn7ys+uS!Ax^fA)Zgf?4< z&>BKiJ3&^BB!&7YPm5>eFvQm0m?Uy=&q|wIV3hzja$;*Hz|Q z6tFOrk7avSpo|}hxku9unOKf%*LL^U)8eYBcgkgY#vQu_96t9mjb~f%3CR~)<(G6y zv8LvfkwXO$1j(dk5-L2Yuwe!b3Lqm}!PR{dYvUbgvr~TihWQP%^Gig0qldY?urv!! zfI0NP!~|+s1MyOn9giEM#>$kbcRa@(XfyB@1wy|HAVvNviYj%BV?YXHKtMuhm6hOx zh5b^)M!^JN!;pgc`TaQB*yNnN5Da#w++6dV@Z7k~a=Z-Aii(=}L!}D{zfTL8PszY) z+^R?hQhvTkXr`NdfEG^d&^YAODXrs|`2igSl>zi$+AOQ#=W0wfSWXa2-KV2YTphCg(*5+-r^>!m!gtyxU{}TSc`% zFU4EAZEA5q+|_LKMoU^>=b3G(!vJak)tRwL2eB5(r;{nkN+$L^yN!T9pW* z)kDNyu%CUb;b61r0!a2@SN?pa8m$jqs~iAEp4$o3Oy~R&Kea zjG?B0$}S@+>zSq7iU-FqpbK|XKxP6Qi;eQns=Dl2kP<6kz4Kj8^PLFXOU2J|pYe?y zB;C_VOhm;X3J=KFSR1(Ak_OOY#aWGf=3vht8$-aCE#;o{{-MWWw9 zT`_YI3Vd(7=BC)xTKc<(1M6IMG6F%Rg zu0u8U@{K`X_s|BQ({B4$^2Y`}IsLOCxBf{4cGw)J^JM>q@*slXo-w3vW`?E<*cE&_ z?_>wkt9GP`X_+J#UiqNI)J376 zi?+n4pEA&110AA1g3DyqSzopgwWrv@wzdE{TVqDnHVLNsb{Ii$VY?X@=!NAPHO+Vg zH|_j?g|Uc^Lk6sqgEU+Ii}=9ln!n8im}0t}FoN$J;G&xhZbxWf>hs!Hg9XvP?S`xY z20AMzyP=rw|ffB--6^7^IOyad>MqmuY95kTtvwFfI92$yl;}pYU+wl-btxi zh1X)=_wON6*3bF5U8xDHv)41cAwj>U3AmdSpOLhXxgXs2F+ZDy)CeyT+G$^ZWMc+x z**AQaZ=nw}%-cK$$}RW3FgyZuL%3m18KIf?lds6lDU`|^^Gs74y8ezqs*d_{zM%Lf zYl59$%lX3le!;pUyvwmrrt7PhtWJ~w=g5l1y6Fb;>nIKJIOBHOHp78NT!}>cLJJv; zpza0V9KNJ3TzD%k%rPT4gM>$?EZ8?#V~}@9zYkaqIi4MKO+U4E`3W<%O}bD7R2ST; za<{(GxNx&6jgO0uf7Je5kNf?uBuKuLG%pm7tGW&RTrS#F^7V0Br~|yK;XXjHC2`KL z7piyqpvmoCz-#!6(Q@yy?Uz)+;e#6RH&LO$s1Xz<&UGlH%_Z3tZM8b%YnIx z5^Z}HdI;U~1jE>JP(bD#v-gNu%A2X+dWeRg|Mu7=U?QCHpO-Ymbfkru>*KGfDE?oY z!&d=K0;sC3&NxT?+3`H$HPJwmrcP-7*KYvu`>7@+39_?Qp-ovigUlMLrBU*90TcbV zQY2R7PWQ;()nP*O>PHR?0T0CWSs%23l2@$v48ZD_wZgckAmsA?a43M@Hqg@%!;8aJ zlFHxegTA8QwCo4`UuQqsX&zQ~zMr;K{)xotSI@m z!;2v4LEF6Dl(BzG-_uLLDwBeA3mL!`Py&^AM3sp#=KvD-m}RM(exuJB7Yb2+XWO}6 z$he${hLnHwkZ#F-ent`QFN@DShoN%S*L>|j$H%CQzxs@Zf~w!jwzVl z&Vzsh(K7*Z&7%Ax0^rZx)l*W=cnz1$2=5hP-#>4z9fybX?Hjo9819p2O5vjw6E`%5 zD||RU0<@*85b6LFg2V)O>>$xBM!LA@NIaYS8D5Laa<3ixesMLkafO1hxDq-3Zp0hI3() z0)y_VD~b0&_z1X!_$)aDSgd?G_+C6yfeo&Q@TZNsBO$OI0zN#a*n_Wdr@UZ|wJa&< zPci>`?YU*)eMQ=x(Nlc}GYd>d@sxglerrBa3_`Z>aLVXTH!idP`(g4)wh7HAJgXzJ z`6CZGblQctC1p>oEZ8{MQT^lUMRp&$MD zLkcvZkjQg@ltAmj`I_`0)^R7sf>}@1?Q%|#a)osW*&(TFBGFH<;FR}z$l&+SBUg4v z$qrp9CrnQTyA_O~@<8KRk5B&=`u*2roK8w1FkFj^FLtt+$M{h**C30An}UmQljJGG z8dgx|^{V&*nFXgaK^G7IivFE}yUcCK`rsha_K@Br*${Hf&dw;$#Z{6X@zI;UVp=rG z$2MlTg&Y2JJ*&HB0@Wj|V$TNPiF}FGNFF*-^{IvY)`CM~Z9>%IxGZOP{)LAv;Fo-t z_j$&pvolq%i8KadENwY>(-eIfw5_&0f%EVXv9`sIXDs`co=)Q zln){6w52B8l;LM&JV^p8Z-~6>xA9}V1Fu7+5jetRtxZo%0kKvabdCcXp4e%54SNt#zig7qtau9*9HTu@ZrFd~;7PDy5^1quo@wqdiS;`L+dx;ZoY@L;9rM^F7+u zpjR}m7|!Tr7K!M!{b=aPD_5n(dNB{3F%g{fdwBNwA{lzg!oQ&|8~Cj4ic5GDEm!H~ z+f9#4qX3x6URglSgAPHZUsFC`2&n^&uk*(6@y3la&Z$RSXw4&s5_x;in2KX&3k6NB z#8RzQ23}SjB}3aZb@4Q<42rhj&EtHm$@A}*qK7`qrfiSNaKj|^Xg)^k>mA|%`ZRc3 zdY&<;2ioS$c~CXNM4VRmFUmul;|kQW0GTiigPLYTr4g`~6)pi4heSxYOfaSdTaaEE;{l1~rG+Hu@JU@7UXm z_!V!{Y28$Rov8QocKn8KQu}Nt!iWj1yz-;SWoFaQ`I>V*)B}!hkuM-JO1h{msG%a~ zInf7j1z()bMzu6k2hYx`FQBy)gMCHIk}MDK38!8`wa#3gJd*3z&-t}Ib1vCIW{Hvx zK4C&;Kfj4+%}2el!z{8ikM5M8&sEvsA>CHDotS^BkZgqbl7I zk-?|g4Qt2{!eIM+b&wkX34Pz2PwJ9?!`z!DF=arpvd7tpqaPDi6>H|si?;S!6lom1 zEz?wNXyxGxiityLfd=Lpwle59znz+*+3k{+shRf9VHl3yzI3|btPB!VJq6uUwJO#X z5*h_I2>$BFm5DEqL(~=6GkZ99x#^yCHr98gv$VDgM%J1cpGiPFDPIiMTEG78HNgdc zI9>&cTsRd=_JyOQp8JuF`jr6F+8sja#v$2zn`kSDLufE|GlF_+fVorhrrtuzh^7hKA8KmZ0~JC(EH1W%ut; zxUKvq0Jt#V1Z`x*voaTa*A^JACC49{SYCI{bWU+IzaRQJE^NLf%S~8zgT*TJ555N9 zR`MOsKecouI|m&8z3}!dd292P|9$wUBH}OpEYwO0r4v0f`$@t-bUTU!YF9c zt(!cx8GW5v<4>>R9iFB70?b5V`PKL?UPU;gX_Yq0SSasArX5dIox{uq9Y`V?9Nk8c z9cSk?;<-;R2WOJUp^WnS2PeHF-IVgU3tvS`#7!GgZty{?SN`;0uNy)Z<+jK~{x9ij z7RB2&{#!HXTBF%{x%@~_Z= z5G{g^7(W&sFlrxn0^0(7W?YjaP5qsUpz#yc3oe}#hWCTHvwQ&O;S!_ic_qc$Rq z_M%9hjK1`=`%y?iUBO1aO$0LW__pN;%F)tX8{}K}S}HEiD$&iK#T>Awdtno&ozfd}j#C+Zz0_kRq&%L9 z+=mOejeB!W4EJc1w`eZF1cp?1Vv4!yV5O7i8N#y{&fy!owX)P-_gU$p60$l+TNL`t z_xH918WL7qy$3$u93u#pYOdwfZ$ID@zGU_u!|xed%m;kyVqG!6f3-;PCner;dvU;U zrRNc8wQ)x#J#UMy$f4Lf57Mz>3aEY}C|DbeJ;A^84)A8*nU z-^$N4xEDRrB6{8} z(8+l_UL5|l_hVVMYecJPo0`9h4sP6)11fmK?HI`&+v)=8+!6+`LrfO}4<%1PAriuV#;`n2@! z2!G5pd^bc_yZBfdouN zV42~*Na!jN&DN$p5*E8z{$$ypdH4%`7TcnU{?`uE^faHSbc4^LJuGe9D|R(>OyuCC zS;M{ag{l^dYsj66J1TUP8C?*(R%qao+12)a->b?aYh994<}eL6L(ET2RqGk`CA^R@ zLVCT|ovIiHiF1$ja(=_`J%+|j?E~_}SdXM*)BTqi4edl(*rP$`spT=DR?Hpck^Y`X z7?_u|VBA55uKmmf{0)l>3lUFsF|YY$s>&vI(p#xe2vwnYj>s)Q9ms~E9XxNrnC>aiAmvH6)(RNd-F4KrJ z*%_^+;r@kundhB(p3DRV{P3e9X1~wH^E+u_TWxqo%UU(i?}5Yf!c5lxpbITleRf1! zYy9hr zrhi{(?@T5ARk>0f@m+=$wkG|sc5Xakn`whW=^l+b6+BQ~0=vM$*t1GZ{!mc)4d}MR z)9DUG(YF}$ji(Pwdv?wywNnFNMD5f zWB>)%OUpYpsHD!%o|cqms5z~Ut>*A6klZ&PK2@sl6AITLBU%veV5qkT&7<*htGJ%r zF0h>Agv`6-K%J{7Mu3c*l9)NKX`KvcWtT?WRXml|&+;ba1?A@e4vc*I;dq?qWHZ$I zY{ol3+Mwx;O!C#J0fB;H(=Nyjupa*NwTWHu_^D*f_6&b4HVHA_!ig{%(y0?pCt%Yl zI`C_CkU@Z%ww5O}OQw#$9!`jcVfJGOeR#ME3d1A0+T6&!w;z!+sXPG_bb=#kQu_Wa zZ%ls*CzB_%C`uOU8@#GB?<89O0`F0HT?F-LrmZ`S`17_kGWUpU6{~mo-ezfxcoj^oj>E z|LZINJzlEwK#>N#!*NUorFTuFLl~Vpaovl_fdkq^?}|#$2z}Z@4%ZbNN0>$oX&F+) z#hf>L!$J%BF5Ms-iz^32<003Yj0y8etqUA$9D1N0Jg{x1KZAG>V^|AII9UoI%(=AP zA!hkcG2Mth5u%*AyKTDsRIj`O?B>$3{GtC^I;(Pt+b(H{QybW$4op_0wsNStMkVhrBjR}d)NxM|% zS}T|EYR0Yx?C27wBl#Em?}utD$KQ|iI{_8fl~F!omgqER{^5ipI~XJjfN@WJvIP&f z=k@p_v)0h;6$iasv{da!g_Dtr_*s7gA6dBJi6SCvO|%CQ8)=opj1rF~*C4}Tiegmm!`f0t@=S9Kame;c7ovT&Cdz_rE&!;RlRDjpVb zOTR&V{5G7Vnvsglu4$hxwfdmjylwYg-G8-tcf1I~InP_Dp%RyXbWAZKNB#J-*EpYC z?^vvu<{t?nYbqzG^?Bft7l_y)dB~zGe1<8_rsA2s08E^ZTjRsP#{FiK4cYGgV;T z4ZO&m!Kg}pMLv1@-OutwdS?#Z9$&mnTeuTJAdmE?P6(SdBA2J__iev-C$m{L)61>? zzSUC&4mFLc7)e}OvlbLP$1`_9D~Ax8X$DvYWA{qq6V{D|RQkLVm0}h67M6OMhlqJ; z^HCljYyl%G=U4X@Zc)JB9RpA*ujz`0%`_!b*zkmhm zy1aw=WR;<~9L1l5@FK*3owuTZ%7JKj?JjK!>n+$fT%06$tBo(}|Da_1C4g0CiF>o$ehY9NE0VV@MBqFoJ(Px$Z;xcyJ=%Viga%8|3GwbWUS3}dzM zPztoj^ups=P3SUn9WU;Mi)kAEE-{FiCG#$*Z>)l zea~%64z7fgIzQh@ln*{8lEnsn`1FcaB>c>wid`f_WY{3VhwY}CvQ*HD56q|9XVSl0 zq^!P8yl=r{O+)vWOyuSeddG$e-EiT4dU?3)IXChHUcBgfK^J{_)Qo#sULCYkmQ zuyXWCbkF5Mdz$&ElUYG;J5 z;arbeucU}RYm$u!qaUUn$v^2X{iU*7p_2wcC8dF0X*=6P>6@IxAJs&{hu39~7Pb0s zL^^PM2!^)ux7T9-G-8Y0&>VzzuA+f#SNdDW>Eb@0j_$^r)e1{t z;@MY~wi*EF!%i@bE|*?(`i$tM!A^ONp;BM+`;;RYiJQD_h0y72roKCr2;D!(c8@jk z(h0o3vX;oK*Gkiw4z+M7k|)10%kllH>Yc0*rd>Bi@iEoiX@g9U0#3B6?Oj;BP0~Tn z54q2Ejs!Z_%d09HBk(BJLwb+C!0NR`^g`cPXM!H^uS~^xr3=fRl$HkE+ljNpd?FA0 zQLu>)?MC~RKI+r71akO9SVeoA%oR}1$?oO`Dx8w7Y;!y(FnlP`Nb4k?!| z*4(re`>o$<4Pb0ZM%@Rkt{A;S0(+Gvlt2vjhb{ARw4?0A_yj?s;)5Ze2JA-FH4^=_ z09L-QAEPPn7dQgN?(JPa@cxK$QNZX4{r1@jXaoH-Tf<@KoIn)!)pXk= zO4yvrhm^I=TM9;p>z(^1?xN)SE3J+?R`)~^A=&`kkZpi28-|aQ4ou;m#fj_fTVdraUayF@c$14>(%;?%AhgvCz-_sA9B8$Dy=n+$hG zTgVHuY!DoD)rH?~1c)MjWIXK6Y74?^D-RW``vCiALVjO3Q}1a%x((VQPO1^hG+ZK` z)@Q1E*m(s5Ss#1&H<6DSIpDhvFMC!0c%u-Qhf}nUh`h*bJ*t7^R}et`$!vu>%nHA1C2kAGLGi|4gghO{*7od{n{)s6-rp<3_ zJN;=E6_S-uV@Au+!Vt{Yr2>!aBqd1`CA}cXB7IA(_dBqRTSIO`PU$1gjji$Y+wXO! zz>h=#@{Na;(zji0C%`3JnP&1B!#_`T6r&%wsr6>5z~LF5T*$)!M5cmWUhG_iL@oU5 zXN|XPp1_bYhF;FDdmw3{x#|)S{1|kQ&`te}t=gf+YIm@jE8e58M950u`@^a_7r}*#U6ocqAgK(?*WbePmj&Vjg zpnX-Rh}=Kwj=kA_ncESKqpY%`vbFY<2B%*lg!GAT;j6Hs4OLU|#fc@)!2mYG>2S!&t+rkocYjHCB<1C>yoE`gWy#*uk zrzrzhTW5&$<6url7!DKTdbnL-6i_vn7Td+1Sz3)q42w3&hvw`Jo2&mP0z}6hRcdO! zLaqvq9gCi3c%_S}AuMgt!>CJ6YKzO(85eH#N_mOZ;Q`i$7rwQMS&k|LoW(|S zbzCj-MA_+F!Yrm6i^8z%ek)!ea|y{t0=?`{q^_$?AyIkmM!e~7t7A9+VUAK)+*hzo~()5Nf8f49luQB z6=q&n3Wm#E$-ErhGZ8$F(A=nSFT~9iM++W@?MlIk21?qC2T7A~T+g zogUc%_X{Y=x&qwNnoy#ohJf%lvm%w`KMC9;QSr&9yqJVOJ9LW90jxA;T7nLo&v+pc z=xEsa@*dee=)1aow)O#4zmp}Z*o_Pd^#=pCxTLY=pQ0GDiwnyS?wR%7-=(Upv%^?^ zHi0N1r%$R3Q@-pWQ)QI(_Xou_! zS>8vu+i5K(1Li=T%(cpl`!kevvY@y#^p_c1Pna*-C*woUS8OMaFk8Mwh1Mi2V?YZ< zLllz_cGe(&;28DccWn}TC?}QO538b0v^F&qg%if>uT$I<;0OM+&8HsQ8dr>hBkz&F z>1paUn4R4xKJ)zIn2&dx(#n*1S-0J{kHPgu@LO%C_zD-wFd&tp!{?;pFL92toii%t z2MT%LG}#As@8C7eOVNDghN~X)H&_7cD%U0?L7_Z4gPQb*J&*hnw2>|mG@zVe0z`&q zxmJ~hRIa7m*MwDbLta{hp;sd2Oz4HQDK0_#xnrB#hGu*C$lO2f6=}=$421)Q(Q9UP zELQzUi|)22Y8ri%c@*R$kScw=)0c}Qq%`ZhOP)6*3cmX`HIRbBX0z;R%uN!NcJBch z8Ch8fZ+LQAKSFKqfXlU9Da^a?$noVy6J?seOnKy_#?~A7|o3I>m-Lo={ zumt0UvKjo=whv{OISK7Jq$?Bc2taRb@;kgynOUv#1_jb%Jw;LsOzhngq%^#SVs@mXTsxF4Cm_gt0 zQG-`5DE!rM#=V0HcP`1>)!vhNrEn=MAJaTH7jL*zLJc?UGMnGSj#-34EnSN(f)IZo{4lb_#aaq=3KRlXV zS_FWIJZ}+_tm_Q_;H!&vKdqbvtfz z7&D7%S{8e|*K}@+w*iH!FmvQk*}=uDzUs6Vqp^Ltpx`d-+I(YfCiM`-g&id9}N ztH>RB)yR@r=89ZsS*zoNrhWKV}WPnqum!P zZk6=ptM#?rwfg!jD!4(3cEo53mBiUp$KRJM8m>s7lGY7pG;PJK5=E|@(=Vspc){jm zq>gNfxqTq&$Ekn2H|Yk?i+y_R4aOy7f-R-dZ?29qeTqxda;2^VBlB8u5;eYi_P2;u zxrkdvl|RGyim;1j!i**uHf(o^wO#N?o)mR5b9e{OIUZ~PbxhB|Dhk)E%DeEDt|{%{ zrtQ@}J_B_H`QQRisCUL~$34d%IyD&cEYX9(BCfV>e8q9G0r*>3E*J^*X46T_R88kbx|dP1}NEKSKE?R^sb%RdR|DM5Ojqntv2#k@x)jiN5D&V zY^_KUfn88H-%C`gvb%sE-*ZsC{eH5!I&g5dkGb1l*qxcHM1>Wt^9<-^gCOD+rufS( zFOGDI?Kfg?^Twv)tb-~|Y!TW92`&u(m65Z3k>Ol6WTgRtA&@zMwQ#KdtR1^(wXIam z?mdpe)FrA3xefwclYQvpE~TEnyocQnpgaECnMbHVp(`SiCcPv@+M`V7K3}o;J}nB9 z?qsK}_tUP52vIQWw$vM==$?k!$KNIl6`hO;~ zKTvTI_(sMSl0$A7XO$tvCz+b~Jc2tw{6`-vZCAqUy-{!)wyvdWe)9GKvSSy-p=+lii5v7*nI} zQI>H{vAnM$PdaTuUc;QZGqA2tgvA0#&Sl)>_u;4~HM_dUP5tn57G5}I>22|Rrj-YF zz0obblg+_){D;fyeev$jL*cx8cja?Y*B9#Z z{o)-JZ|C+9n@5eH9UwSKxn^Y=7Z)cTVukKic{4vPF4cwa=Z|?(43~IlvRQd7Dq9a@ zmn`i0d(n9$xK{uuUSj^FxvBWDpfA`(;1rMjrmpzR&hU?z=q?eNQ1WA8 z;FFun3c9P#s1qYxdMCcA%IylMC}5Zw7_zoy%4+kKYO;~-`&?r+xNuFi#W#_a2iLh; zK47%0bSS*j$Az9=EqmRYV}bXvG%+4GfNm)DK}`4kN7H|TWoh(`)}S zvAQLn1v=~BP+lJ36Y67^UeCVyu6sw}|BdEn62D`<=wsG*R35<*U_ShhXkJYXt3N(DD<&&BtG)g1Vdi$m zld2;)hG<4g1P=uUB@N^IG5C>?qLaa2B}rinMPq7QTu4X=2y&*99E={^*prOSan6sr zF0W+I$z{*;hQ*CTzIrw`HlK!hj0Ky=Ly-F3J#4!3ZVJ#O{3PI14VbCs5`g)((G3VUD7uOL3NJt{E5X5bZ)MRz`W_iTmbHe08QKUuolD{u3vBMy?ewzNZZX)3imkiPi_)3;iy=M|Ho zqBJtwM>HEw8}kWI38N$W@Kd%wyPPwy!R5d%zuprhf6hf_eO}iK;?Q;LxlpUfhwH!1 z8w`tG74u1pIWkdrSo(_!x9llr+y|;*YtRMD5n#*P5po7lR$*6n$ z0J+z8xNLM6SS$8aPX6%gwdweh@bQf2S3=g>E%!i9?X-YQj0W;l(yh#9w zgBPz-dA$GtS(=_)krvbq^CUel*A zi+@Iuy$lCh-;8xd|7ZYi|1}m~IuZfbd&joMFB8t8oYGx-qmGgKK>Orvx^7w|w=d!0 z4_!hQzpA_>1$fFYqO_fLflQMGF*yGG9{{#MNxuLLdzxp0TPw&62EY^LBP+?q@@APi z=*8%g!K*95U-H}tcQmG7glWDrw1pkG#F};YV1;uFt0_-U5jMLjOxRc>Z}!^O0s`jk z!j|OM6U%2iS@Zs0c2R#Mtj!!y(r*UgqO?fXyv%C(>#3=-e$Wo}=1I~qJa@UV9p;>z zCTzmaAz`!Rtlxf*eTxdng2yegAAb3-WNO1!a++XG)X8Kp{som7{t>v@uHcD@FPxvf z@_S+R28q;bG1+k(q6*BuWZ$M=J;_29ENZuL<6A3sx6PwY^P-mvTX}yWE1#h@GYVP9 z{vnvVWT6<<#`Yk&iAYONQS~t+yTLh+s3&I*3mY_C3Nl?w;pzsM832_ubmr#>8@o*pAMODYonYS7-q8l~pMY4el_mabQWp(t?g|?xy=ncdS=fS=)aiMO1We6$PUw(I^z7})b149D zN2|wpJMR?88=8Qj*95p4znZMo+Q6uY=pI(?B7-Tfp!{BcQ}ONFWZXWUYSLZIzsjzI zk9j^dKI?3Pu%(AU#%KlIC!8QOB8K5u(W|_Z^#YD%?W6{+))9NH<{Z$QJ0h&f+LwgY zynbBRpzU<1kT_zQoKBU^IL|xbU!VpJ2Z`4x7lrY3B^Ff;G1@F-_wYp<`AXU`Qaec4 zrb$$Br14{6S8TZ>tijB=WF!gRI``5Slqqy@vjjMxU&M2(+1J4QF=427zzXeJG_Y_d ziXE0TU09t_Z0(=PvRy3sRVH5oy`>;vy&1*GxkkRcUqM&PVx8C}j{3?N<(&>A9bxsp z!cX%glp$y?f0rbHfZUNMk?#2K(m_fb^8#VT@ET$J-G)JL^=x>x4;+$H@X*0@Dmc83 z=f*u?RiRCvQcfekJ>@9BWv#^UaC(aVKkzX0A$`_h^9RB`uoDOAAZ9;Pn{n^a*BYGQYVtd* z6B~M9x>MeTOQ!Q?fySrgcV;2)c!mB>M2(=~bPT=KmwJcU zH&lw@tITZaOIXKj3vafmh}tO@f9wh75^6HtF+}PaN7~;W!`FL9y{X+_-V%TSOZp5_tE9G&6yvv{K$xZ75&BE z-v~z4(}Qf!FM(r^z6RSr(aj4l!tl1Y;EY=Yy*OnCo|LP|ug>4$p+`Mvr0#~ps>M<4 zxQB|Ea0BFnoAW?289WnYZwp(dN0!wF;B0V~SZo?m{*o$cN->8w>qpV>?$6+fss<$w zvO^zgz9eh@MUuvT087)J!F#!>h%)=A zQEb^UqA>UmttiTZ9F_Jb3M>Ely08Pkz@Op=9k8W`8tM8D+^jFchn#oE`1T6wr~C{L zy{~cB^F4r8_$-mMi=%+X%uZA>d(AM>H4vkb5j`SYep%RLQ*s@;g4x?whZ9ZuOZ36< zxx)AcnYFeCPqRyW6=q3JV|c7#Z_N93jbPr}hd+uMJn204nXoHm@Xcs1z0WX(ubeia zD)E!g{jq!CjH4RV3rf|c4OX>@DF4uAJ7FvTO6Sd8jq+%_qeA18{KVLDVI6W=GvFZH z==Y_5_4`0i^&9mSK1nHTb0!?iUB(=1d;1$+}rYX{*;lR-W$gxV>d&6F-*h7M2ZC7a`S2=Q~71hu+)0N2R z9r6(-D>ENF$$bRVvafT8pi|A%k#Zi!=6DgEvpaK3@G*K~!xC0box@FzUo}0t<1YA5 z*$VQOdpOI)D%oXP4B6}*f^QS_IGL6z@S*6=FJyeB2X9_reYG@+mF2*SWY-^rH9bzn zsb6_TSl@m4rWK1n^Coj|qqB`#i(E!EnVsVVG`$TL6)&YWW7ZO#qc`}gl&oOjMK?@e zh5Gs3hrQ#It-*Jor;yHuI?K2jaQ`hcVT6 zZKI1>S%=vaB;GDNvV!|NjBe_3wSDX+1 zWkxGAp58;0Dx{FQuneNZPF>46qAmt(J?2xVS zUq;QEUIXoE`gOt@O~cQK1kSmItIop~P5h5`C(<;?t1<5bbiQ&ew=}~(!(`ctNW&Vj zE|641m56plq%&>WoyS9+B_qNHbkl!hex<)GeoP)So}#PX-@}uXF>up40V|?6u!3I2 z6JcGa)4%d>lAW@21a6`x-6!e_n5$Nz{Pn-1uG4J{gjN2H?CWk}K6HLUJsQ19*1RlW z-J)Hb%N%@ReyfrSirz+7s-8k8GFgu1hr}`O*(R}Lu>%v%_{lfm8F*$kQ18t%P!9uN zI2N@J?zvB;F3P?oF2@hq%eiwcd)3gFYxXlO*yJ$jxi7$X6BF_@1ygtDEp|~<@V?_|>V5po7!!>O+25ot z+#ayL$GS0Lefr2?tTjCM+)GCM4}g!kJ*TZmUwkVlqV9}5z|&kGKaDnFJ(lyWNdW|s z{S)7kM_9oy8njb;xJ5D6Bt}&_oV02t*Jk_SQtNE|oT5vo3;Ukg4? zS6IjWIq&#~g-Cp(_Cs&tn<^G$BEOX;^LJk7{YYj8yKB-k-iAuc!#Q(R1sLO13QUOI zd_-8w8oY9nsy?F^Et=5HT%i~`4dzVls0D7`G? zYr45&F%0xO3QH4r6NTw?>LN#rVa|;vi zj7TC?{)^pHYpK@6G-l)UQerpm0lm~QGdee_OZ zPg8JWGMV}YS#~WWmJa%F(I?ct?Gqw$I7p)QgCzCKwZ&QVH8gsxASD+Dhvr+!@f~x!8vl6T5t>?iiW>XFWGar5z@|pd&X}_Ctq8- zdBj7|2!d7`$_O&0_Yu0(GX`Q#4V+LW-eAoLb9!0KT{1Xy|A??Q+3=@Yh353H;SAz_ zj#@6r=gjDtPZajgnXVnKGc(ku;(5k=Ccu=XoT1__5yc`uCV8XV=#1S7a^dZcRm`Er z^JZ~2H#v_#sir8H<^Xe=Nda^3(0cf$a6;c^9K(~T(q48;>8GYM&}5H`a3-gaIm&M@ zm6-M(wbooEWpnMd^SGYnCUSuMXawAwxie!ABLrg>!LKTF5@iUIK-Tg zyb)gIXK;ecxJzAE&EwmKpICW*GSjcA^xLRXlgv6?0H3}ra$52W*eY+MJc=6W#9D{$ zC2Zg;qeq;ALv5)npL-yjzY4aPkKtidF?X|psj#j(k{XPhj92A`+#`9&&`B2?s?Yuh zD$e{vGGe@zI~heIzmXMkVAaq%Pt=)*rMohBDfTM5m~{on%ZKR8nxo_>LBY2f%VDo^ zCTiw;l5@A_Fwf&`vF$3fB`$+?mTJNow;AN6xQtzyMxX8xfq^= zJ^~-N@1#@4Q!;CR0#+=jB5SQ3aAtxuzgd|PyWd;%5&a$LUXTfu*fs#pvPuxn^<)0` zSj%MCd5F2%E~YZstwxTby?$jKH^o_#G0j&~xi% z=^kFX-x${l%P6|DNha)-g|Uub3-QXvcr)rGQ&87oZc$8M&>zR=K}s z(f!rC9!u*mGYt5m0OP-&1bP1^WrVu0Z9+rh(^)yYkJIt^nbH3#nbJPPX;<9}{%R)> zQx~I7rjiaP&%%=wtuc-NpBxkB!^tKibj;*w9n9=SuiU>S<00-_4z!~x<39%5zyo0GVFD{M&vP>q{{`9Cc?^qW>p3fx88fvO+o3rL zZG2TG^%oCkD$6DpetLeuBoH^Tu5=s*BHJq8YLsE)9AB?_Z(#q*kaST{b2h5whk*+5GoEqPn|KNT)i?)Wcp zi4(ZHc1n}Fz#Xz&y@j4-vlHExYrpeHPUD9`wb2O9_slsP9hpq_ zu7j4aj(v5)7RsgyTlM`0VLJ!E7q;fxE@8b+tQOX4aH+7}v!)B{d6Um`Cyo%-WfL}P z&thSX-|H3Da@IXzV|Awp8<)g)Lwj0;)rozXUB0#zHs+^tVJ*zI2&?--ldz2gU-P_y zbv##R3u|UjEo|SjtHP!nvllk^-n+v3`o1M>?IPZ+oIXw1_}<6D7A5^DY{=|1VGR^` zsOv#5zIPoY@hqkoFQWevWTRO?~|J?w5S)70^hz z^_8%Ro1BGB-4iFQqALfi+LDCLI4+E=!Jj81Cx0xgPk)iH+7<=E+NYcrR-L;^*sOVT zgmpWm#2T_!m;C`@nu_0o74)MIp+*fZ)45$TMyZH@DuX#2kTG0|AVjrWA%kC zOUvZ1`#uoX@cWs<+NeM*?v0!Hnabz%)?8uTFR6r0IAQ@NEk@*Ps8QI!gRE1lCJI~E zjlwzjj*ae;m_FPS;88fB+Ern6Y!@%TJ6hz zVLMi`d)ufeVKwJ|gDrA0JMZ2gdF1ARMe~!<;UL&H4wZvj^3_R1|-V2%E39Hrkqp*{Wo|0>=v&8t}5@KIZ zbe3<{{u|@CX4Y)2W*zqokc{brht4CZrl@7qh-NA%jo3mgl?Tc|{(ic!qXG)?&}Z|PGtp^sLO{hjf`+W-0@D<3Z$@88#1gg?~%c z&wPV@wcbUAOeZZA*5JYeVRMS{BxoC1o3y`9*oyO1huKF@z%co!u+`(Kk>&-4!iFr} z4-P)yZ$8PLj7W9l6Gi=+9K*(Z&%SOh&kI|(8%|pOy_IiGe#Dayd?@H;$5tbD(JN)& zux+fPz5-_g8YK;x4sgj|@O;GYtWdn+1bc0~FRb?)@ZYZ%oNexq$CRVgR8tk&60p<( z54VD9?CVseMZ_gCu8*oXDnf+K*x!n8vGCKmS|zOAa@Nf1+$C&{id`b=h$6$`oa}!( z?68#z9qogfrC-Fi+${We??8VVc2mWXO{YMki;O4u>EYpHB8q;>Ub6$kgf+5(t2T1N zbE-g6C)}iicy+qMmW{Afy_3x9>B3=$0GL)}6-3lhFUakan7PM_b*_IaY@0QB7Ijlu z@i#veHm;ehm1d#gp?c(N=13}WR>(XuYYxwAW#AT>3=>j!{Vr_h4_?Ar6`cp|FjfvQ zz^kOIHNuWDr`{(YgK6Gd!Aj zC`uUQ0rDm**(GEnDwp|1jIb5)_~!0X45Ln?hj!=4uge?ggXc{mO*{M%D@+Cd>K=Af zTHh5m>K!tk;)xa|nxe_6X{cYzc68F}?CbFD4C|{-H!GB?;q-M@_UE(TjTAheK>emEUlTUR zmKbZl02w!X^f~SdnqR-1`t?0d{Z4%iWh+xt5f$%3+!*uAe0Gqsb{#((7NTI@hl$8@ z_F5wPlr^;$-v;>^ASUMXvwjE@lMQc6wov}GhXRleu zw-gKDQqJK9DLYI}sWDVX#Gh#OjC16r#}j4F=)$%e*?|0ZdVP5fzM1~O3V~g}z>p1S za`j!5);tXMre0xp6Hlti)E!ntUt)Un`kbrMOkxXXe=fXYt^Hh`fw%PPv7!{b#ET_9)TUg(f^s?rQd@FtjdT1Af5*EF}UhW!jF!~)0 z@w_1Q#8zfJMIb7qHw_&K>m_T-75ngH0c;FtMUhJO(g8y)r1|p}%G1!mJ9){>^Tw2* ztauliZ@Zc~LH-YHY++%Uo)OH6p2o~$wig|UxkE(xfAD_lGJGpp&68?JY~{!3k7Ec} zx%>!MyAH#LkO|DX_C36#lsb{N4xF);6vJ=khoI#2Ojy?qOuvdSxD@xmTv&ZAP<7XW zd&7FwvIh>t7sD&R+&@5a4mRNi5p^Wf`{LWtN!2TV30r78M_8*XD)Jc4OlzQxCq*hU zRU*~0^%7LwIRKty`7_nDJ!2iCtN3946}|hp2=+DKM5XBFgJj7vSQPA42PWpcugZXj zie<1j?Fw_whkys9o`44S+T zWGeevKW%O@c^Sb9ZWm#rgADc4e1}f<9YXmN5AKB<5{IWRAp6B0bj*Mtc2tz2Eyi*1 zL9qwbvZ!TR@T#XCTLxwbn==6GJhI@q{t{F=0g?EB7O6^zq9%@0YSt0ZNSMlJw_oU( z0e>^qm``9XvYCXkmEC8`ExE`%tm(7kuj$}s@-BTr^9z0^wV^8N^K_qrO`KeU4xo)X z+tK`x1hQ|j9F?%n;EZXq0!DRjg_+8KSjX9fEO+RT7x!oAVaj6WmbvRt?DMs|6oqMv5^2$x%jh3d^UQmz!?V5gdQz z9BPon3d$3Vn`up)H2wC0(#WfW!p^g$#`S_YgOo&rQt2kpuIhu~T0Io-n5zcD8dlJw z^MhfFr#Ja(xe8X3kFjHHD!ni2xO5WlVva4H09J}#DW_5uI6fi@izmdM{hKrqeaU$z z@&+rUouuBgE-(XBN0Uuu7hTEy3e}|T*dwf~=CZK1neabwHM%q*ftB@I$ig%oBC@cC zt3B47=u_8FaXwbyqim-(r(b7Ub~FIJto4WR+yp+mZ(>@=_F(Fpxt*TweF=sbJx4cp zkfvqh`=D*2&zcq&VMY24qELsya7AkdtoV*yZ2jp|89w#=^;7r|+)tJ*50mAPbTrCj ze4ViVDk|M^9J6T7S!NiEE9A(e0?wGu<6FMB(QOkXO;-9d>RKd6Q9ZL!^2k~`h*yd` zZ>p&pxe0wl@Av?-HZF2 zz9tT+A*Wy2NFkhbOi_#kfy_|k~&{f(HYD?N<&Uqf%CCOMNF+GuC$hp zL=a0}g|`{AV0&)v0y4FUCzpEN(~8_L7@uluHd-Z!JLI=>QpouMWw5_Zt>rI;V*z$d0?u=#Gl(17 zSgr$ORnvJV`#RAz^f5&^d^%3p#$I@0e-aE$7Hinuh3tgAB4wwL8yMZGd{<@1bdm4D z+0gVntd0xkMx`e`R@ka!G$*AEmA3tw^(&je!Fwc#Mb3i{@*pOtk@@I$?AP$NnUVoW)I}53LH%VKJjEFjW5-OcGg}V{;PM)`9(Lqvf;d6*Fe%3ri zGrg^)zP16a&_0CD#Epf4A+_|cm>6zlz2jh}u4cZl1^38u)EUlV3ER=h<|K0A#35T& z`Xk-e=LuakyC0V3dJ?r_5&QajP|I$XaI7c*tu{HtlQDs0*5~%m!dhN}v9&9t+Xg+5 zl=l%`^HpjgVH$cHB!}Tfx53a(QX$vl_)yd;-K9*YqscYYRM379Nd8J%?Gkt@wh>pf z9y7K}YC!D_H8sXbZx0vWW(@@C70yvrO4gxa_`D^kP5Q}#vYbrzV zDc(y&#V^p65>CN+t2@lwqm5NEPGm5x^c%W>akSJgbT}buUFC%0SjG8WcLhW0a2KxM zO^m~LN6~;M)4zr?d)C6_?gr+f#)ki6+r1{qS4nBKMCClx4 zU>L9*-<)5BtIp3eW#vzS37R$dFTakTDWAN;=PD*Qs~&n#^yk!0V;=jq2E(qAPtf5a zSNiypsO2zl7TRvThMqXwE2GxoZIvriQ(6bTHKUtdx(Dgt$`g2KswCI8>zL#to(IW* zA(SEOMdI~@P%=MV)@eVAZ=ns(+ z5&6|bg88@*b{zcy6%joF<%!-c>E~&_TlNOGH2xaCWs(p8l8xAaX)rsEEvRrX$^xFwJ|AUR>Bg$1FHRPIdkwYP&Bv{l$+#$@880CB?drs7;@6`mX+o z`@_-4nZ5O_sr1>N#MSf`5|H^F(Y3dNyv0Uz*mo~?YnD%`?ubr2H?|d$S7x`oKZWO!Rk+u z1)r;EQQMo;$SfIb&vSr9VT0&f#8I@gWQaUATEfrXbt=|3WIk-Vi?$5^HsR?zQP|)Q zId$q~^VhMPi8tvmSdEh=+p^7kPB?W?*cWBwuI?V3QMmEf^eO)Tvcl(U?(xE2px+fg zLSfQtnMtaR!8tFMZksR(RHsj*doh2VaFRU6oFO8m7O^-vfwSHw@)Fz$TjG8v_HsAg?{Fd`?%QEh)FrZ< zF^F$QH^6P|5dIhb$hYjDaF3Dn&1D(qAfg!k0Ss}CreBZf=VZ~=iK>^r0cYY*Fz2WT z`8A|%3nvAI4L->C(z!H7QY^9D>Zrn*LZ|eDc|r?X6gv}+^=sj0b_EDbYDO#4j-c={ zD_Fo{X9V8iDpq4mBf*H#>_=8EbK? zkUfOYxofDPv_8(QDQ|;A$VU8Z@FTy&Q=;n}POpY5;Jnr@&VVkLz&|VqBuiEBIn4{+ zPV|>@Z9qnJ?ZLJ>o1R|vU=j#?K^Jy9#<@?il0I7j0J63x+@;h`Cl>j1zU%u9&xg-n zZks_ybA&8ZUWA|RmCP!JKCsB~3vP96cfo2q>E<>flzUh8OtLw@nloKa8FxQuFLLRYQlVk5QAhEx>~HicISHH-=1FJ9JLp)Q#EqIA%xR^I>cGm8 ziCVD~gd>ZII*ZXyW?Rqt6aF4gq+{SoLOt&U-veihdM0G=opWH|N}e<~F~iq>hCk-_ zcr*7MBJ%U3E~57_E$5FWmh6>iOX?(|Yh6ocH+qf^<+~fS-7Lw%^TsG=ToW}r{7X$x z7oDi)1Ex5?>Etmdiut6@jmjN8NZ!p9=x|6gXlUQShnU0MjMZIWW$!Y+8@HAiolLmv z*7=Ax{eC7>MIXaYvraJ4l!L#?5hkK(HRN5L!nD@$8Ci?0XY#49VD_++e#MC1j@smz zuyUmITcP|V)(O>ypEey#<54$YVC2F3>=HZxbBedYx8Mcz)3Q70kBb|Vx!-suf%ymMu<;&rM)`hJs5un9 zn!S{+=jX@nDkJ#Nn+4~y{)Xq~lbOvTr3u{Blqllt@ua24O<4bH#Nu?86?ANn64f2) zeo~sjbo24wKY|mn+8J(md_+ZO&B0sG9Yksr2=Xzrr2l`APNT|)oYT!F^8To+oR=+M zK%WbzqOtM-=|2#ZJT-7@PQN>PvyL7IGIdj{Y>*`AB zz~FSIMDMwd$_;b{3zh2E=BFFLYT_J898N@ zt60I~E=Z0DMF$$S!FKrUrby?z3BD+y`ZBi=`3tEUtwwSYB>i?{c9wm;4p9ZeHCwh2 z+$s*>XL+=ASMLj|RU5g%ws6IVuy84-+8`5c%x7HKm3OoG) zd>ApCI?Z?pi@cXHAygluI#M-oOtBDe?Y}{kW2C=E#Zeb2L->%}0~0KJIe(3q0*iVN zQy047%rHtbW_Z&(;OzGdJnIVJM*KH($k%fG59>j(-Pe(Q)tlV1&V8Be=UcFv@*Z#6 zzQVL)txMKC&rlsIdr;MUh0ng-sB7kS{B*s+DXi!wUO6U^o!X`7e9J=AK>s~%HN!4T zth>!~oh_)Y%{2ZRwVl&MN+L||JPPKqd(j_rX`){E4!9+FbE`1?i>KC7)HwYYR#Q(= z3ETRo^gkpTay0xaw$)mAsCAwD+s3)vQWicX@9EO-CIuOsDHM(}cU6WweHSQNO>any?x)Cvp#Hlrr~U=@tB($@(b75W$z02Kxm2mk;diwFP!000006Fr%CcvMvr z#y6Wyvgy4io3`n_mrXXkYpQSk z*y5)(!X~co7j{h7kHW@V+Xx%bzg5`bH>`wpo}Mdg=!!&P<39OJScm6X&2{BLVbyd0 z5jM4#6;kWo$CfP>HY_qtSht^s3hQ|CD`CB+v)XXwcwwVYeJO0(_EKTJL)(N+m=nt` zi}+Q>Se`ty#7gp}R$}1KLJL&qX z!a60tENoI4E4y@s6ZM%UVJknLBWzau6=5d~>=U-^!|#NxdHODP&-eInzeQNXJMHYg zE=yRe;yUm=GEvw@YZu;`T!x>wMhaW>{M-2Ec2C&g3%`K&`n$qrwigO(KAGn=@22A6 z0%9>5HBH!#FQSBvY-$kJrk&^v19tIWe~|PVNDwxB-LK$q)slEW@56r;tC_zRLKH#G z85gD!Q6gK~84jHMa z6*hS~Y%v(Y3W|_Vh1Kj{C#-SRC1LY|l)}dT@r~D$e4$ z8%U;5$ePKb+pN=O$jX~v5w=wYK0PDZMb}KFq+Gi|-k0(uyhO9aI2|G~XDr~{H zuh}c~9bxC}-^n{>VW9Ud{Ar5CpNg)T*n8|7aw{1>O=p1HWIWHdqT-yF!`r~R2w}%A z1Z`a;Ofwk|RvukH^LJdduwgg%2ICvxFxlZJ@}m3ONLXJD z9wxOufRoPjPro%)U=`1Ei$by-naIk9Vf1tPt3!mf4hJRulkl)`80&aEO9q3MP6nl2 z;FfV54%mkCB;Jg_17^`hO{(u=_zVCUkdXDZ?Qu?N_HmAsAvKzY$jRyY&O9c#k zhF`^;hm#iXgVl5AsN9^#yl=CMnW9_>bFv zyau1MGEzV|bQu2pIuB&nL>Bpo^)ttTZB!?f=uw7ONdr7jJ9YrRmBT%wf0=Z3Mq`DY zHI!Pb+XxR^y3BZgFP))bKTM0cN_AMA1oOsDdX)M5OuE9S*}OR(R9%nIKZh)$_qyER ziRLsM8##HkuvUdH;MEQ?Yb0Y1%J+}}c^F;772svXm)WJq<1Jy`=7UdADtRgWkbiex zB0PCNIjzvUN%pmPTdn>^*zzB!^f_6mh{3t+w;$wtzQS654|+r9fm`lNZ}8+Cn2!x6 z*Kxj6$+*i8M09~1$p;*St$hejECOi#B|&(loeBRf9P!7pgU(j|0xV5yAzsB4`fAKj ze6#kWGZ?n=rhW$=)EXWjd**CEejo~qgT>D>w=N<>p1C>L_V8On{!sgwiHZ>nJ^|HJxgpE!k z3iV+sZrDXzVZ-Ek_%ji9DM*@Zv>!+o+2V=uL~;=qN{2DO%Z$)AdkUL!fhiFCD~v5W zeTyf{LD=7wocg}WbaO6;C)SB{w$#(iIQ^MSjif5PlFz3b$Yro7*#n+rECm6B2fUv> z1Am;2;ZlDlE zb{|e>aOyb)dTO$-U4g<0A_+3%V|ZITo*MUS1l717sTzYFFeLR2)``By?w-C>mfkU@ zs%92_AnVT-Vqecxn!O%f7rz-Ul@Fwmm#BDQhaLx?#UCmGs^QNF>YFV}H|C4USQPi)(nQB#^m!`>0 zm=?Pcf9zM$4XO?>->0fjxh6}nu8+`BRgdX=7JpKS=6UkiW2n zuaXPxcGQB+s&rWT0JeBd28U=am(OK4^x=}$!jjY356I; z8CK5IvnLB$W^tU)Dq#!0D949QD2mcva#ZU|MD2}4p-QXzI(F21?BxdAT`Ix0G#v!87Q%@EuFy)(0GHYBXfOf+}>esTBzG2+~ z=T%pK2h}?A71F^vnVVovi4V;5y$nO#UnUm$D)eN^Y!6{GqUa)QE*2YQ*3tyvlz<1vP!l?yA}JOyvaH zP2>~iMUy2s&wUG3T(ATreI8LMeKEYB|059vsF)7^H_^Qj)gKC5Wy`$wlrzgxAHmz; zaCR)#5vhFwe5hJ5mlgWRX1)|x@+1^pcmjVH1P8&oO3sGvztY+ACsGSOOF+180t&<> zgS%7%pCZEXqfx75)O zqu!{bROJQ}WG;0!cpyZlE)w`S}7nT?3lg?I5 z1d|N(Qpg%IlKT{u9^p=oB6o4>u^RdW4?kd<)U8I9`92HUzIR~AkblrHL!V{k-da4* ze_uKwuBEa{7lZJ#FM;I94)SPF1A9%{!9=^3`>v4PU1aJkoiovtc>9LYJra+i5)-=6 z;latQqfi$^ucD%g5`PobCf^GM0^Y~N<$`vc`;oy#D_CN=~m{E{2I4s z6C3f<>>adLdJ356u5)LlKaFZqS)h%b6r5%3KjEFa>&%fH7uc2d8#*mv5xWfEC!HDZ z;7RrC{8t@>59(@iTAD$wEiMte;hW4*i)bdhCWj7SeC~DPEyB0-Krk7yhshW)hc2p$ zXCNg=_kJO5{laDyO(%DS7lm!R&J9eW4YS5~GD=e025xr7(s>rKDbssG)%f3IO>H~e z9-K{4$2bL;tdvx%H$C5AH(EBs)`HT6SXk#@ zOV=8m2<8e8I#SA4%tThK)>|Sa1I8xs2axrl40xH^S|)#;n;}hfa>V!YRohO*)rt=@r%{ z0roncrB2OnQ|U2Pu*={Z=6K?a*<^4Q&n;pFhT0%cw4^&byVPt5-#tf8@ScO zy7YDQ2b}>LDPc4n*x@Ao#36w`W1Y&I>aB2fk}u2*-S;io`4r5P?o;pX>1c_DZ%}Lg z@l-)>0Pl<`1U+|i_#eWxflTuknVnh5L^FJZhEJ8WhvR9u=eCjEO}<0_lzf9fC1109 zUK3rYBMc6dTEURPJN7USI-dJcFfZ9dc3O(b+Ta;;NFIFa_!ah!k0XOAUeut{osO4N z&q>g08E2OGNw75jqYJ_|xxvhlkHO!#oLqSM5Jj~MH4@qlo+jH^-#Zudk}sjM8g0-# zNiNb<(&vUba5%TA2CCtpagbS%eUB`dEu<nZL4py_iO$IHr&iL;Ei$W`*4YdE9cu0V!({OGKE9Xt1LaKiW^XX@x+UF8-6lq( zS&(LAIH=@A;KH!PL>-g;b2!9;sP`NQj zh-mOeI{G{e2`SjjY99Q`=tsIur8|7fU+&4uD&kVBI7N;<270ARdS*ia+)KNO|L&(y z1z~Tnrnwi|rq_zjQ|5+_viVxN6CQvSmOIhvgD1NY%ZSV3F8#de6sJGye>g8EM-g4^ zIObaYV(yU3`&Q$pEn3mJp18UepaYlB|<^8FvB~7)D+%@E(AxC};!`s6_vg&;% zsX-r8Rc|!8h`B{Xqqk0ijlsM*FPB}s-`h_{YKgJwK##D~enCNb?*!+F24-dCJ?4AD zP_mPC0aVlVnd3GIR6$4)x=8bSKb&y}?Y1w$xp6N%H`>9i#qis_Ii(d1ldhyjYVLxe zNdZ2%cA;Hcjz1z7$&${rL<>dSM6oC6IvpH^{;o>TLa^i}9Fb*le>>1G>A#-rv))qRH*1}C-7gX5F)0d`FBppT6xCnBdM zutNJj^%%DeC16^I=jk`_JSJP3^;(!ecvqPu;~pt62mLnT2|6&%n*OP}M>msS1;n!G5gjmu;zgcvb)X0g4?FRJ$O}HH-jqm+% z`fESY%t6{t)>S84gwgcbU-)TJ_81&s70I?8;fHrPeP(r18u)`DN;@V$ zoLy8r&5nw9LVshbHou!PQ@Cia>|bY1+kcbkuDlsu5#PqNZnBeMp@LqSuVPURfYh>x zo6vUdpRtbCUZWSWcy4D!W2KvS&Wl73P%}jv<9qMs=3+Kgsb`;S)t>?=4+oCT z3$JO0F$yd}%;h)VmqT=k7(uT-rLAt)A zkHLSaMFh>T1r~Q`J~9N&VUmcfIA4flMm)Fx>xb~&j?*Ou^&il>_Y4}7-IJZFYWQ>I zXZBRZn4Y7NOo56cJU5s$&$$MOP;i&{FNGKB&j4jEeUZ-80f)%A)C;;>3rrVfGC8EB z-hlb#-%=&liFVpZk85tK>*+9=l~vt4HoqRpct!4+GBe+T*H;CLWDm<`WqC&is)% zA#!GA&5@|y9~~$*B7LtLMkKTv@sgKwES!{~dM)m`yolK*p5>}Ms;`}$*~xBDYg>$} znRoK7cOK?pGgNnJoauB)q@kTWt5?%+KW1Klgj`_WynX%#ieaIWs(+%i>RlnQ|Ypl_v4CfUE?fAv_r$W z%O0y?$9%jFzVi(WTjb57nx$9vgmVAdKm9XU@Y4$q8cMyf1VX}1H`?=-1Y)q`0m zq{8+;ukgMZ-AOApjK9dl&cy=6a&~Um1<^9YXbYLY>CcmBoJyz{enhA2nn4il*+)^D! zZG8P@IJ7-{%|_C2DFHb2~Qo=?w0kZIR##R>x6aQoxa@y{1V`)KbM7qp!R&-(*HAV=^$GkBHIQq<=lfmV{9@{ zU3-0#X~NYQbHj+Tz{-+oPsaMa=BaV}qB)Cp!9wi-NH4X z7QIO4009i;b}eg_nji2g0%R4k?BmFNS%NJ6ErUr??GdarDlg4ilSGyX11;m$nWKao z+j2{7a2y8UAheNAmbLEWWu1+|&149gPGOU-n|ZnXVil>7%1chSl0Wt{?J5Uz0y2ii zlJ9)8Zi5tblfU^aQ1(;3!|nxwzQTP{(aXFKY$+FmNHt0gL7p2*bvc`pTZOqT2KPcX ze~(dI{?{%z_-ZTE8)Ls5=XJplVHf`Y+6oJpUv|NU7xhT-W81N9k`1TJ9N`G2l+^c_ z{}2L%{n5pgj<>Lr>D9&hzvFyUZu9roI);a>`|fTjgZ1t5*{5~M?_lKiyybAb<93qI zX;Q+FD#_My0gLktjrwa^++9|y)2-yD@UXU)vU>%d(2Y?ITnJ;2w92E12Rz=z0L3!D ziGhJN{Fu#Ny{5)P?|$mlbk0woY!qRwb{7qPp^}jAf5AaY|au-0WV8t zL~FxY>Pq7iX3)dOYKjjo@i&cI)0)~?QvPq8FZ=7DScfZYjeFz#S{XN7WBT%IMi1kZ zT9}*^jDf_*TEh^8)4#bmoHsQHAgfM~xiujv@YZV4lLwQ3vgT3n#4rWfdBrZ+XG+TW zw8a&k5PN8|O1UEVe%M<9d=iW703N)h`D_Q#rDtPvGVo4fW z_hY?3Sbp)^8&Ib7=cfkWw{~fOORR%<(#(qJO=v`R2Jki*YLDA*kip3(~?91`ZpkL`%Q6cq2Msp~Sv zZ5?xcLng`Ahx8n(JL#_O$p>!5yWh3d9&_pjqf7^Nl4HR>En+!VeNbg69_k>TcER=Y z8$wX>*}`-rKKdIxyBasl*N%PY^$B?=x%HXO1#Q{y1!<NB`mENCexlO{8d^ie&3!~hR z+_uYWH~6fAtL77#b5gTJ>~Y@kU4SdsZC?e+gMp`ljYr~t`T}7&^0`*esLOJKNbNsY z%iLNlw0@`;p)dL!an|`opaJRGBCb)jr(Ywz2eOMDtsg-+$rmVBYZK`KLH+-}M|Y@^ zI#83z@;~}|OKH6mi>1G_)_Dm~%}(;GfgTK9#hiaJ(3E|hmO|1=HUAR7rKzV@}Na-tQofif^+8Zp4QVtaTP^ngZid+>;mB`Rq zvAyAPvZB>beZGSulBna5iHy(P~ZuNjsE^JhK8tHx$SW+=uM;2jC88UPM9NPwEx5$zGV5u)?QRBWm_PB+U=;@ zCT%Vqn@%dlRVo2hbc0r_a;u~c+6Rl5g0t%2*c0t;%y+8MJi4;niHEv=7UHu^RQ$og z=!}^9nvbtqh}L5$)n>HuLi1M2mhf|&+}r8HVTq1`j=8~lO4^f$F`LW|r``=`KVmng zQOLC;#L6nz8r7sLzzV&D&)fJ!*)rqu_?l7oFC!ha+z^%*q)-l zkmr%>6EmAj99PrKQPnPz(MBh%i;Tff!`Kyg&sH2A1i!ShG|Qf@(IgPBng^0@Tj=P* zW#CA9#)45j@FfPNB20X4EBYWPRYYRY&6-V#(T|!NP!+Q-$+SszHjeNexI1w1;kgLn zOzT5*!Q{7jBUF!Ym#gB5`!309env>-Oqz*y<92g1=Kc!k1piv1FkUsk;y<}XHon(T zS=x~7WjR9l?d}(5M?soz6)iC<9|`B)(1G%5!?k?3Dr|P28Ky}tX=gqmPvbg+v!VI5 z+}mS6Es+E{&6L!uvb$cn<=_hs4QXVO%_sNp5n`jEmmBvF_s0BV=BLJQucS)+X^y+g zt>j*|2S+bbbHR2CVK8mfG=0v@Su*MB2$O}8Y$ehMpLbmID~zgH^0~8cpZBlmMskB& z7H5~0Y@WT9V9%PbQO`z|v$=7Rksb2*GakftBGRwfh1o{HA*|;Plc@N5#Zg4Wb*;d! zdtz>HQg)T<%&oZzMDAkYAE!(`BM4SEU-+ z!UYPuyf=3|))D6otovUtEYfTPGYFSw;x1D@7GQME>rz2q00CXWIrt2cxG)&Uzj1VqYb7Sv`H>p(RO7@w+((1kvNXv20l=cOJXaSvwfhOmZB5QTi@lD3_8a*9uAF^pQn!G=H;opYb@W^AJU!+Z@9E@uPs?C zVEVQj$DVd=B_1EU4HTS*l<%7{Pgw0BJS<;`Db@!uQjKAsS}Kb%ZBctha!VGGDzYUl z7}{es*PAP`5+5>Z3z+j19!#d;qPUJ~S$nrKJK+wXYp4tklUb&J1XhMHn59ZzKJ1M6 zwrQ=R|IR>Wt*h8A6MK|1k3eTMQ4+&f4L;@c+Ndk;s?Y3kiu@6opzEJc)s$$hj?e}4 z32A|ufETg&hEK%!OzHqL5S~_o*(@fV>g*2hwH}wJW@fT2y1+q3*Sw=E&^{dEjZhL@ z7$T<;@Mm1g2ZzQgMhMV4L~7cvAM2~33xdX7_U*bSb4l+?^F8r&BY4sT@Tl1dClMYJ zj(zwYutiLLKly=|{p=4On?15#Ipj%B8=B@S00}o;(LUkVN~OEw7SM^%1KR{lwu~n+ zoZGLxtd{Q=<7#EY*d$fz*(TIut^RB{M_war3FS%iE}UGI^rhkF&cdrfxhlosKN@*S zar3j!BvH-cOoDD)PSwRp+F`r@hcn~t*){C!R(Wu=)X#QA`K z!9rC-#qb&3=uFtmAuZU9cd5m4Pujp(`v^iH1oBrND|rNQu$V>Yi4{-h_y~r_;z%Al z=1ZFKS&V4diC(OQ7U$sgPz0~o9z~mxzGzq=*td^SF1V1ZY74S3`Uz~LOY~Nd@zH^3 zafLXhKsW=8Us)?&g?8=KG*_FJ+WG~Ix9c^yPuXta+gq+a-V8yFaFU->O|#lfZ3#SI z-(U>4kO@S&D9V;c&a#$Egp0RnX6r$zV4!}o1BY#VGPxdiTp{Z0H8yc+z?kEtQUPg? zIVN_+jd;PM_8cz~3POSHXopY$+9;fc5$0Hy;^24s40S9QcdB)(abNR98cRra zgsoOb=1cAg*pX_2d5#;jdLrTDG4`5T=kgREtghmA%VL#FoY&H>c?oW;?uZ3F*9(kh z)rkv5+1t0^%Oh3n17ri~UH3U}+D$fvq6!5mb5FOw1ub1)6?8uo?PkQFk(Kq_zgN^G z#cMkeE$N7C>GI5Y_~J#-BS+2}w3#sV57Wev26mtpHeU0-M8)21*&W1JE_aR4JTw!u zEn|sR`ZTEpE>9-kiWXPwWr%IDeCTqqCx`@vcw1z+Oqi~^A2|exonkVCH_6y3@}Pss ztCSjMUHYrt;Rc&Ci$@bnWGb}tix#c5e+a^H94(qW)*bdvXG%~YdEk5wWl`eb5(Bb5 zIP@JC(MJ#bLfi#BBZkk}0ogMYC-nPne*rJ(*BTsKPCqF>hKo9e`Hr%cSCE3^y zN?ky@I+9%C#YHLzNMIAL^Dv}g38(B$5A|=>sUa?>*t85q*LnllU3tnbU}&BRM+>>|F^}_t zbRS>r>I}r{a^=S1>d*+#U|7b*V zra_-$>0@T95;A?6tGVy2a7xxO0iaI_@cG2)4hm2?=z=Gf>h9|})EUi)by@pQOS%O` zyUW4I4kvTTCZT+bDZI!gw1{J9OS(q`oZJ%ZM%5HhZj96dEgJ|dV&n&paiqG`cbUwx z7{*R3acTLSK_pLPb{6N*p3b`O=92nGW^Ra-k1|6Zsxu7)o; zmpP}p8@6$<1#jP=x~ssE+WH_2oo&OW1$#!88(7Y8WF`9z(RGI{Qw>kP!jD6$-d6_9Hx+`FJ5%D;z+&~m|z3)cKESuHpr;ngo#Ve=IQrXN79d?3srRwe$+3r z!s`Z(wn`GevbLbawp>q-D7G0}lk*Mw^}PkQtBED>0Ssmj*V4+`L`M4IsiNV9Log5h ztGZ&dv*YE?LiD}2lL!#9xcjKb78oYtnI?3GJm@QhS}f%D`9 zsPgEGN7a)Z6}AnSHh7a|2k%LBZZQfj9E)H+MX%PU4yOwFX2bpe!>n+z| zHU2<2182`?warm&M&SMR#ckz$Aoz||(?y+$a6jMA{zXe+ajLp$BPvjuLya$BH~}Z3 zqtehmAEWxvInTIVGy%*IZ_FU{{l2LPW%;ZXLw_h7*?(BEYoP~m@6m^>7D^fJCXU}~ z|69W>pQf=8J&@Q`WdZn7HmqH(>sRx+QS}MTh)~vDW4hoj1?W}DQB@#k}ieNh7c~WZ3L5!@Xx-{og%81ugo-gkn zu76JU=5|8xzBAu#y`gp2D*o$!D&xG(7eP4}+<`Zq>$eb=2ZeB3O76korst^I2(m{} zWLp__A8%Px<>>d%z9iQwM-WL(4Bg7h$1}srpj?-7b2Oi^9b64JVBf5%yUgi`Q|o14 zumj@}*CBhyvQ#~Hz4HG%H#&j$gV#Xdx>~cKY{z|%>?(Rxx%AZUCvm~u;M5NuDOWt^ z{xLVw#Dt^HE)NY%m*7?N#yXsMc+ON+dGXjRL9xlOm+mGd{_Cr%YcyGs@ihO4*{;jk z*E8I`73v7nQ1{4IMqYX}*(?N#kLPTQ-bCc5O>Hwi!=Ln#*q>Lhf#&9GX^2|Kn)~iz@YRzVEox5yCpi|@}`d0g{;Z^{LcJtAf{X{niIsib$*p~d-UJVxuS3s z`@xaQEi3Xg&{?xCGw&tIM-|gqR!C{DAC#=t1 zcbhO!zw2SIQ?%D4vFK6)daj=5hcXPub^U&&FYpvR1aQdBU!2xP(wlCIMi5Ba#QWar z45`KEnj>>?S(x2h-H`YN6&5=O#o1xDw>RZw77KwE6nyw|T>PWKxHAJ((pl25`<1rTeW=#SIm~%&Pnf7)(({{rYDLgRT`vPk{{X?n^0SnHDPrAz+bqoF+|Z}R+gT9apoCZJ$IZ|nlu+jh_& zVx3fs!EeS*k@s0Cbp__U+PCYJ?=NMhL$y{PlblAwWYc7dvXzpgEe0vg zEUtEFT*?7abwAKXKJWk+CaQixvXr!$5cf=p@&&5y*;o`+l{=~#>MVJ)nfOM&=l@0X z%AjSvL6tCGF8@x|CsQUY%pLD*2wqEQM;!~9t-*G5wXxneFZO3kZtav`8m=+}S$;(F zFUCL$WF(cw#9{eTHRgsL!>%<((;TKku3Bt4fi&d^b}@mZ<*~guHITeU{lAViyV0rQ zX9A1_-&s@xzNRQcDnrmnnxED%kY;tI;dNnr-b%9UTcyrW zPEd%)(Ep;6bNjv#?dgEJI)qp}o3x#z_94)9Y8bc@e9}|sfvY{KKZ!_Q;sl8QLQdnZ zf6w^hz6)irPi`d$IS3JDeSc?j4|W?h??EgP>$qvy`fJf1l&-EINvWn7`hnC<8**^_ zzn;V3|5g;<_q!d4uiaIZ8%k|rI@{ODCA2(hCpWX%?lh4_&U4w@&dP_cj zGl?UnZfES!=^2)Y(Rg4wmUo*tVOcc30Fsg^1oqP@^rJ@CGU4@Z8R)i!YTQznL!n1bUfl+CYHwQKLP1pyHABB>RYpse*#)fkYIEwy_0=%QtJ)|bpj?WnvpVD z2dT}4d>N+g_$6{hFq?sc#YVTED;u+(BtW7Nh_Dl&m zPfk|^X92eFN{4d|kZ_&1(6Bz$aUm6M@MoUP{=*jhR4f-)=q zv2??mbe12qeBH8zSHn>tOJKgv=tmCH8Q?RN7^ULs6krps^Ay!X*-#L^C*S4bYK!rk zEx3tF`5DsbqX7mMgPVn@E%v}0_V3+;P9(=QzEhAFDFnQTfoX;c5gYe)(@LYUC11%YV#po zps7X+EwS!1f zmeR+yDgDOCb3)_bi|Ln|-G2&+>&WU9m&je}hM@NZa^YWb4sz!#*FKCOAiHHn9>Glhz zg6d@QU#1J&>qv1#4o`=^;tcuv4@w31Oh$u}%{ z;WHk0EP4Pk3Jv{yHd8Oa(&DP_iIX)>s&}pskPHbRDcER-jTp&P6|3Ch9Y2#7p-izM z&&H9h*|z7(KCv1~j4D`#ZcAn`ew0;2oR{l~3TI`@__!c`U?ai86AYlFeD6k-Bgw}c zd-o+LsECOGQ=ehi!^LXneBy_C`-OH~wj~@ZAQ}3%O4@ z0{D-f-dRGuoQet5(l85=UjNEEMF#(!Z9?A)myRxO{lg zTPp94q3WL!`hJxs97pl3xB$j*r$Sk?U5F$;RsXm7Wfjp|WFaoMcs#B6w!iSJ9-ZbR zgl`jyk#sM?o#Zh}Sh74{#0Vsvzza)e<&hVZVqP-CJ=d=^tE~VO0WCt`NDh?jUeHy{ zBg`cCe?mT7Bp63)VY5~TzofmWQ=l6`rSrrz4Q^bV?gm}|S#(jZoB%wpOpRK5tQHs- zS%3Umk*%FLGtrb}-j|$UblwJO}j1`B?S4vvy~3pymjm467wXMcN2D z_ms=h*gD>k1(+?)OYO60Bh3ULP?^va_Nv~=zl)?Dh>DC6@fziR!e~6uU-T3(u<6zY z`kaDZ!+U5lHAzL4lUxh(c{i2EV5pC2%W?{M)H5Qg?4eleytA=)+M$d=@ael6ZOEbh ziDs(0+`ByeR;oHnUuhgoyEzQJQr3WO%e(h*g8Hq_z}U6m!jU_J{>^kzv6>rB)c!GY z7*8Kbze;?HG1!0@Nd27r(}9br_@`#ANEZE(k=Np$t=r$0V>qsv?Fhv%8Y->JTuL`5 z#^4HkUv;B=#y_|GL5U^4&T>Y-@4q;&lhnN#;ISLzwbFN#=vk2Y7;16q!>U$=NoxuN zZg0`>lyR2jshKyv16g@W>XzdB_ZcaO!_VO6`HtxN7$)HcD14aENNj5tgAA4SxNXOzb<&g@m4k9Av|*25^Hl& zNE8}5JwnkOP8;=V@Ak+|58A3D0OTf>-6EfC=gSZWo>GhzRiBPI$6x6%eguXigjA1yj6 zKjFtm`-pm`xR@xpYJ3#;))R&q@D}%$*a7SR$)J;ck8WFZ%_}MVJx;a~FmL$*bx@u1 zO>Ni)$|j;c#`D`cG5Odg)~%o-oAp;aj-Y8+ybdYZ?3su&Rn`ypQsUISOYAWYttcZE zHU9zXOG`MgJqS+mON_YHcc$fxy_aw4iJaCh0F!H1*M-rTT~uLf?dwVMZx+1NXJ?`x zU856r*kgE1Nji`phxeDJE-Sl15UjolGL)D8Y94nsM4?`?x_xX3*B?b;P!rp&*%tq| z{!c$0&OlUO^<<=S>T+*pE;ypd2CAbTH8;ijQAtv@Q}?3hrL3ZE?hYouYW0=MFGD$F zL^r%+mGF*Pm9)+2g7o*_4R}PXGuH3Zvey_MYFNN0^)qlTh%X9FSvCo$zMv)6KqK|% zo2?bNjtb(kV|**r?JxU6x1t-%&j-^4eVPbu0ue6S{uOOcclJ{=qzA+z9qb`XlaYwr zWEdHe-%*Ja?=lP4-C-a}pYbGp$Ft05LKpTT`(o1RR{m60o+sNN>KUv%$vFPE7(Qgc zn8cNM7jXD#B1}2KKA#c52{6NdlkrJbH;H?beo3A9Va0h#<(a0{J5uyO&^>xXPT*NZ zW0myWpv6tP-|_<4<+750<#{@yBZs28pXfCX{;-|5nZok{P^KSt&dvC}Qk>m&e#Z6q z>Wb#G|1_O#QMJYv5F;0n(C2QCH@qEs=43~1JT;M@vVEgFSfK**^&g&#)Y@4w*=j40 z9#1_-5tAFI`>8hw@AlCcPh*31uPD4H$3hX1hGtIAIK;r-JoLdGvMykWyf1!ysbL5t zs_mT6@+0wc{MGuQz&>*5-tHbxQti{2f4Mw2Y#- z*3ws6TM+l88?hfTdk0n$YvLsl3Au~yiR_R_W1f0XD)j}~m?uCat$#IAE9DKy$=ge_ zhN6`d17XCp^vCR7yoR%G+!Pk|v-j<$s9!Fb+j<>LIbq+a8>gI|VzY zzIQEXMe2}P4s6cx$lcQCx3V@RUiuvx-P3v0pbRJXyVJi?j4A8~ibCn{GhPMJ1jJ(U z2AK}owZM|eu$c|IGDCJQy#O!i`Gg_Ino5_Ll_^k1VbHjq4H$U8W zFv+v52alG@2F8oZIoT&t{~UZy-DKw&8IKcq9^I8pZ)lfjw|Zd!V_wod>)ulIbPN0X zzLC2pJ7aN#JLOvU+l$c?U9!l2!_U8U!S$&Bux(sAg?>g1yVLz4I-g03Q+6P`e{rPB;qAY~lVEHk zbJcV9+Mbf!?3adQDRV^xeR$-b8zVVEhK;z_(tZ;_R@(mir7p+G~WK8r7t<6bTht~g1-U#IENkkCrXH!+BO)ds`HV>v^q|57?v z4|(BQpBGnoOmtg*yE*PS5R%C~%UIgqD`)lNp6-2e+vV(RBlIL4&u4JdemASwxm<;| z)UEFw1=0f9xM6VfYs`q~@AQE{o536Fi*hc4fPx8n^Af6YNiW9@_Iz}#_&HS~z&;PH zTtkb%JNkEFQ^6p@DiEFF_PWu9_3+Pm(9v@~@zsyvz*0_bbiX8=;`CA<(`hs8)Bdn+ zuLWAF&q0W6!!|T|ROf2S=Hj)V?$rT==_^OVF2o%u?%}qhOfTZ=hjJX+O*MU)_j2=P zUZv@g&czAk`f~Tr)oS@`SqxtnqhQdCf0p3yBsbTV=}i*N>doBmpTTe^GxO1&(8MKH zT@9=$DJRcT;3?2saCQp}>rsLC^=7!uFYbuc~50tbos6gj0j zt5>@AfQ)>7mT21OD1Bks6jGOqZ{!EHmNdrX7C)wg5#(abjod9G9-#wfbDAbb#lPP- zgzF>>)4W)yVEw|_H@>>H#rTE9`Ey95ctv0spO3c@b)?$oD3+KJ!GbAUOj7s*AEr>Z z){pA|kHL*7=^Cfh`Cg&^O$j%!b$p0#aht}JirghGL?~@(#4v-<&gvPVJ84$SqR0sSZhY-(QtV^Y?x{_tMsokLp+0NQ^%Ld|E?&vlq9e zo3T;j`o+}8#G#5k7Lud~=iw8kqs%cJ?_uG#%gjf9;gOvuDiLYDr=~KWu&%rds+W=G zY1(^zV&76^?6UxIhoVBDp=rA>)EhMRyXFr)mxgSn>Gm*q#jKAtQb#Y7{D@u_=|xh?Gh1WJGH8}Ew}ue6J%wfqM#+opnwnJR7FQyb(fCXxq(}28wchKwGJ9AA3^8{68r*nJCw1dDMojW z=!`jE917&moUgfsm$>{8_+^JscY{9^MI>}n^pC=Y-f|Mig?nHTTbFul*W*+GHj6kxo*7qnP$(`aJowj;iH-5xqU*qhu)sP_sq26dLy2c9L zSrjAvYaj4zyUP8_;2oNz)1zQhyBak)L>zy22y@(6Q5;uDu#?b?T`>V)DD=|rn4yn5 zLSa&NFk-5CSKhCv5-QnoUz*CkyhX@9@r$0fxIirHZ`4ScEty9S$0Ps(e;YE&*S~^? zbWYLdv)B!~P<8ANHP4^I>OilMvC`C5=G#IMB!j2Jp7W0LAs8msl(y9Z9e#nqc5D~Q zDbq(8F6E3bwq>2Ee3X3R4R@~tBa9x$qrc|hYf;(YAI^s{rO?ux!K*(!rO%LArB6a~ zhOE`5EPkjW%Qni9Bw^tZ3+j~<8zMNB=N@Rkl_xxUUF+2H-sPHmcki^xk8wcO zSb*7%-M!MrtVk!nifs6d-hdOC{KMv08q3a5*QSmMgfUz4j#=I~P7S;G3nC44hy|TV zLC?Xx;ZB526Sv;{c@Zm9t3lTZ;Tk=s;~?6lS`qt9ETwJc`M_1HY|SjFCQ*0M6T+yF zjVbhsyQVewV4t7|E0TR5?G=DS?i1EoM5m-h-B_Pw=f2#>#p{UIn#L2(D?}&5bWn1M zf=6A79E?PECNr$b@i$lHTHdvQwd46--czhQrPPetq6cPh=`Ecy+t@aiEbV2(V1&Vn z#>p=`+*YP_1y=%Blc0m|k$KvpOy=DnVf5xmdkdjG%`yw+wElvIIW$8^FZO15%(mfIJqP)*| zNKL7yO53+Um8Fj230_AV2|C{fcXC*y=}cGynP5YKhc4X~v)SpChz&ddb9>XF|Bg{^ z?)3YWtmBCb+P5XT2!WCA*}9v@_N>%2$kp)dJ{hZkd$)Nbu)d={3AYH-ukEY(W!DfA zWfVFKgPSXh0|fS88mtPT0qeaiEZc&oL*ltou(+i#-38#YC&sp=iXNn#+e~^6S?%6k zGH-gKUDw|dq@(!v==$O zFCTcQtUq8!mGa`C0Z^B+-)Sp7xlo^qT7JwJ{_?fcjftD+b=nlUyszTp5Tvg0?3 z?6&)rADREHIK)f$X^K)#zglZ93Qc8oZ#(A;@Bx=~{Q}_|U{vsO~78;PKe4Z zj|@Xg+r$e(iymMzVdd6_v<={arDla`Td>>9-(%}~puhDO-g`~zyy72x(&Te3OvfOX zwp$O~^QMe44z9n3&o1$9Q*))G^$RB6uM8zy{qM7Uv>b1WGycH2*IG$K(YC1epBs{e zs29G(B*l7xNdUj(@+ZPHqXmC9HNqLnk`F)fLkccrl!7b96S@-4u=ULB+4^t~HN;!p z0|f;OjBc5hRDc%a3+Wr9fPeO$EjeoQ;JDg}_kmUW7)xH8%!wo#F!paM(7>vZwOSNG z6f?_6IR!z0raM)&q?`USXxwu<^yGJb5BNNjM&9+|} ze@X#9T2C07z)>j?TIej!0P&WQkcSZ#|3FHo|jTMyfaGHrS~n8w&z4SVDo#>MxO$A*RK73z)du~R!Q-oa>YFryZBL}y{=-& zhl*AC$a>4YX~yf=it<=VM_Ze=StizC+}8b4+nA2enL4dyYR*Js9N=7H3ifw1J9bzt zB_>?i=pa*iMfQm3x;^o;$ZhV>!{8J1?M`9;GW0`_o$SP(){y>KWS`V1qZ(BY@nKu+?1^ZG zCF=N_R?PsWs9`V4=|E{pN6u~d0PJL8LDHE$ zBGif93Tb9dP$8Gb_2z9y9Q#sPo7U#nuK;3Ahz9!vyUEw&BJ`b3HN9)H4guj!UC zJaf|hIbmWi4g6YLB@tkvl587U8xYfRx01`PyX!_dCNeX|F#R< z|A(hA$7#mZew^`h^naiVmz@bDiSg)|m{NJc=sGOlJ88nDBEr$d#FW$+;NiQZFuoPh z&_U+Ilab};g1mecyDr}?syjKSoF>Nw9@hYeHqK3rja^MMIE%xyieT{jAn2qUNNJ;9 zl~YUG0v8;kw@*>YR{axvWWgW8@!qKmCYv)h1_(u9EDl+A)~Yu7V!9p}o`bAvh%h(@ zJrin=owDI8=I+XBv%EGen!1L*a=%>gr!CWo_SK)1HcA)#u7(epwGA$9!CCeRKhE|T zyAgChe?`?EnDLvW+%?rcTR#zLz82T=bb5slyL*4d+N)+j#kLNEgT9oN19Yx4o#C1z z``#C=XN;BhLi7|l0Nh2Nt&ZAyD1_YW?1EYkUAd5~4GoIQU(&2v`Zwr{4qG0-F4Knr z0mhXN{B_aYiF7s-cuF@NYt%Nil*8QBFJjSKLk5gRk0Cx#vkwdDr1!*O;4Wu>vZtsPuC8~jd!W5#AMVI2hl8t~F)=BYP2*An zXr=iAtC?kAJ^ys*Gk?H)ao&J>EYsEbjjD?nD7V3v~!XoH|I2XqzYNTdL z1-$dP1g*sOoY_$$gi0E_zPEmisfkuUrW$n@l4TZ_gAs(Q^*KM}-lc`dm@#$L+w%=J z)`jYI@frAidBW--%qVvMbt~y52-YjM6A?$He|7Ip`kT)&x?>art}1 zi-wt{fY92|4imLf8r6U8=t$raxvsl1yu`>&PIpZ!=uRsqDz(90Ji25yvT2B` zC7yBUTRSr}J~X=w4Vc+P%*cX2L?P}~!Z;K*pxVvYR&YPXA9%q>8cBFwz2bC*z(?js zq`uv3!RD+1kTkmGQuwYu8?ZhB3>X!HO!{io%-%8Ry?WrRIq0^p*`D_7^kD2fCOW)6 z1&qnZB35>@n3t`|-3OI;bpUVG@EMLP$T>ogEqBEI8O>t`V74)ehnq0Ky-ayuQ96zJ z8aI-saTm+byO%@8Ag?Qu0oA2h?RxFEsUhb<#M9rcfm09trGBw*IAqE?s$pQRiFn?o zkq{GgQMc=6sI{SO{rcVUX2x{F3UczisgG|(HPlJy3q!t;qrzB$J zf{qJj>_tleQMgh;>AGL8uTNB%2U~8={pmGe^LwOAGe7#`{2eOaU|`jUbUbnf^kG49 z9zy!u*{UOyf_bbsTs@!}p0Cxhy|zf315DTknv@X@Ls~Ho=^XQM;FG#cyJ6swsSVZi zZ0TKm*3BR075Oz=1G2s35&iBINUi#Snedv?5UhSU+u%r9Rjqo{u|sPm^C=l0y{zCY z*tQg$j0UQU^%XF#a=r2%v3kkT+We_V~Lr^7o+{02*ZW8MZpV zOZUd*2fhF*fTk6KCVY^(cUvUL%p@)AQ`Gr9K{*3yimb`0m0O64K3Sr>!15al#;A*iP zx|yaX{|%#B1#Zeq85~;GPf{D>T{H!xuB;}|fPNQNtkmN)03!&pOdFwdma7`e4x5~G zcA*R|cE#c>tQ55L+AASBS$F+NDWkFNLApErnp7X&MSeMUA9Cw`fM~6q@RmRQ;W44g85lP!fnBC@xmABCL??t95DjVbu-^T}jl# z(v|Z;*76s;AZ2L%iL7?sGDB&5R;ocSua-e~*dNakdSYv{VhQgc6v{f^Ar{y7ojd+azlUfH; zGAmuc6mkXy)F-WZJXTND7G%KpVZk&K0>CY6V!b!Q?g;8WIaD6gmKGCWjnG>0h8p=W zaNKLAPykf+I(LDR5btuTyNKW=D)p>R!m&EjD3up6=%zTLhfKC(9&^ zIjoN^?Wm^U33;>i_-wvME-Jie<}uARWnuHG&i_1n?&HVGM-hx+&V!2Fka4g(gN`vF zFwU^I!Qrg=gGUwjMqW8~Ph?DN$P=RpMVyNZ?}^dJLSYMo|3^o>b4X!mm8YA5f-i>% zRFN-ayh4&~*@i%^nxfQdmU>M=bB<_3l#5o*3%tr8^_SQ{Sgd9L^XPe`cun~X zL*9U?nEz%j)t&Q1^>oK=EN`6*!udOMT)A!zl9R5B=}TK(hbo&pQd@KrkP?b<2aCC4 zj^u9A?x2Vt3=l!Ke^m!%d)=;h&M>S;ISV>*KJ!VIamG_#%)PO$dEbk+Op5#cwA88FFwZx)YW}u zA8v`R80YcsrLG|?QtP%j#WO4_;}S)(0BSMDZ1-uekaP8h1lYLuJnvlQ7z}LF79qKO zroChM_rqy&C9%(0zO366*2Sb|CySO6r;Zm`3HL)4y~g}jMk}p`;ALB*v@R%b%-9*q zWGlL*Y6uO}&HaW=J_orAxG-vVS=BmRWE-JQWv!u_2zDRL)>z%Zu%^YM(BvOFCk+f4 zKB;{h8-~D%6WKJ^^w$yh9%6-m3F}WemLgThfAh?7;)X4k=F~L)VW+uOTKqq@&MB~t zXl=tyV>d}-Cyi}ejcqr!t;UUQ+g4*cd&fIwW9-=abIyNz=3*}9YHnuD%=fOt|mJ(TP{`5sKC}|s1X@VU(`X(AnAn9q)l%ub=-Z$fo+}*O%K(~2c z6ypR&6rPS?E4uHIC8?9Fd*!M(xnki}9??RL3^*+0(c{! zr>0g6w3bd|Dy9iEB=M>RshJM?U+=QM$*Ph zqx_Cx`E1|{1_{z_l4F-vl&21sEx47r+RGF4wbhrE|9OmJE;x|&rPiaU0FRPj2u&q< zCyiGxnjdU#)3Q&fPIOMigN=HTG}<9^0SEsc(Wx z*EnAlKYVUH%^CIZmSGN1updWus5HRU1s%F|S3Emj_Bgi)#-5=R%FTJ=x{Y!q+E)K@x<(fYEI%8+1zE$-1-~G zEAxo4c=s+#YFmoQW^ny5_qrn5Wp*LkrLtq8Puu2B$$5J`srwYe2=F;Br*nW`JL00@5LflOu54q|1_6nC2j#aj+30!*{|c zeiaJkp`n{aexX|bVu^lC53{z@m4oNk^cNE433;!uJB9v;Q|Xuox1Qo2!@9Q-neuQw zBnf29Jn@FbB?}~0_K@LH8#{wlWWoGN(N{TyVv-hgt(#EuVJpZ#RT^wIFo7HS_I~f zk5^yHJjnf>f2RJV2xPA1M0I@0$tCK}uwbiCVlgtC6h)u*Zs`J#{MsA4HH5o0lM;K} zj6Jp@f9(2b7_d-zG2=72kxt6!$^oXF>RMFY-9D3T0D=fvK+cT=q1d zLVzVGyPGC*Va@M;xm>z{v%`YELnru0q0U?{f4T2k&;nmSL@-Wd1JKtkpR>9wYv-UV z1-~X8=o_iD22TfDci8nr+OxVd;>14XR=T1}y-h`zymY9zE<3PV{zzx#Dc&eDSAUr; znW3t_pbReIJycSVCqK(EKw%xf5vjX@dtW;bVnEw+`P=B3pzDEuOw?&A%$ua)GX2lz zOA4dF;9K-Q-FEifA1Xp%^*?FFXU`O*`;*Yi7*#ZW+g`Cn3#+0SH0Y@>v4(RT)=k1i zzO;%bx|f>0FRpoD2%u}O16vd=krFmcn32IfmK-qdQ=}I@x8jTLfriHEDCrXJC)+wO z*NQ+}LXwtM<=dMwaF78`)9UwPrbJLjGVG_>1Yl}JZ*l>xNrP#m{tVa(*s{BnZTK_=^xeMLSmF6@@5=E=82rMp|E!F6wlUuX~lgN!Agp z4bw=!V4E|WAzBnh;;C)8&wHG`pmijt$k!r&)nf3bA1l+^ttmlmA-=(&>ffaYr}5fe zt_6_CT7E} zA!*Cm(sK*h!d3_1eJ%yAjDodXDQ_%XwbgW5>t6UOI>}&t>%8PmeBm0+Wki-A9hoLh{#ik z$&!Xi#?~&?VUZAmD z>`*Ffwy8XFeG|i_>8osYwiy)4J%Bn_SLiglGa~l*VOKqp2M-fZe4$34zsPf_$5`^V zt`Dv8fOvF|6U+f>Zs|Lm-SW<)wr9AL?@1&49P_@Tna%r!$#CHnV%rBJlNaU?_2Xq>}q#3J@I>*`KPbWP8!V- zEt2Ip7*%*PKAdCN9xa5uQD@CTbFK5yuGx{= zp6Ph&0m3f}?(w`bJOaNWPVZthpGhpJGr>6do+0`F zyVgUvhZAfLFGu{<^QoKL8FvzILb}4q_Dx7X!5m)7*J5QxqpSO#Um;uXuL!f`fqeB%!96bhHEaaJg@e>j2*I(Dln@ zCeEN3zG8i6<+3^X?F(K=KhDjcn=y0ST3Q4X+HMKUNuRT*EoB@q8;Tzt;a*0mM|seZ z(B;hgSI>j)k;8sh`?D5JR(11*q7ZN0<;g={{;TqTdjFm&eADS+ce1Uf%2Q?Hop}3Q zPuy|d8hkM~ZW;?X0c#oiNJO({=&j1LscyGYY;?X+xTGpYzur$sH=s)G^6PMa)R!KU zb;)Ol^b)PgW9~h$nyBUAP^yCJ$J~CaAkCH0SveGDxhhJABE&g&vI^p3c?FF1(AcYm z0C=y>E@`K2*2g5~NNCeCgkhtzCtumTY1y?!JltPUG}ac$6<0k(`ZnS>ZqSc7Qf-7s z*3-JpyG8wmUnlAqB5l?oN4rqO^ys5YUozRxu;33#QnCbe%$nOo#`tgQke$#UMvOzrW{=k+~}0vjSHeo2&f9QWLGQvveGpep z9Lf@X@yiciXbrQEorO}(K)ml6*Gtb9!rb1JPZMinF~786eK_tY;PG!6)G>}>*t)^L zqi>WqIRY}Qyl$hndkGXHf#E^Zcv@ULl$hT?{zS%}NX5Ov#dR?v<4FCjcKJ-<21 z2!G}5^^T7f-(4zs<+%8*bUL0mhi1(X7DFETHy((u%zF zlFc4QU{^fCj0ug5eg_&GhWp{(`(ffjXn#lhj;7R4=mgKPB*)g~JWfwH*|OlR&$`BO zdMD42)ac9w(3GWn+1Lu-S`#`PJj}2F`hidN4tWoIU`;aQTGuvyYHfAB4TRNgkl8l> zGce8?-9qC$aVV)jxVk{dBFv(xHgJ>G(md@P&=JxnwLK{u9*EW1{&7~APASN6hX%S= z&z<(ONv;pn@TIGLf;B!B{HUvU&r~v=Cprp6?%@3@M3svT`e%oqOV80S7AdmKGQn?* z@ys78e`Sm2ZZ%E2+ts1CoStkqFhTw`1UaG+i}T$8ThW;do^RvdpZY~QZT%0|A3p1R zBFnMFwFp%e^?kX-3}?cvMfKWfted`exSP+M2wAhXl(yi{JjpHlI-!0(@{iD^09jk( zaPX)ReqKlSr+{|O~mDIV^2Fh;qsA zG)ia0QKhAoR%$X-p^R9yt~Z**+?eeM3zfVdI&c6m9{xk8d*v9#&np%538B4DNp+8S zh50QxHVBog)~N9m?gJ@bnj74OW$FKy6!JMhKiZ` zK90h-UC<=8ZC!^Cn~AT4U_-jwx=E{Nb@`7sZ@wn5C(57uFDE)icU)E;W3~S1$~)w& zCc;;i;q=iKIHO26Zx$Z(h#7&;%FqHZK6mXn(vI!9O_7D&4-P2ku;+0bRzR?vjt^A& z?2gqj+@-XqDxN2zBJdr2V^I{yQEZl(VLOHu==82fCd`H(fUZVAYZqJIq3W+J-%YMh zsZ0w&#*uoBOcA#%7)WjF4=;|HHFdntuh%cB&U%FIto3v1sHgRrsj3f>rPS>MNj{a; z|AQIt30H;T?kdA*1=|#=1Rx9~O_n<)8UB%9lFPA9{cW+0f;pA(xd)X^DiI(&!-njW zG8Qw|IIh3hyp>FSR!=)K|6cvA)f$R}h(E2;RS#Y!wZia59MR;wwJ+~FimjwOYj7}( z_TCvRTR5jgNR+csG5tX5t8;{c$ApaCdPQovXVH*G%aiV^FXbhQhN_*94i$DT>>}55 z;N?ks!Q@FK^l4r#0&Lsf-$BzB;33mlx6p;SRpijxf|>TnF0xQV>@>dhf|7L7-BXq$ z4P?Vfg?{G-VDqUEgvEeLNeJG*438lj4@WPTEt|;lST5P2wHnW3+))rq+hC5} z2#b-}I(5=is{|^pv5^=x1^@i28CVm+=jpxqyaDYuEgWyH| z#F^p|<*X#(Kx=Ye$3m#KX??Es-91GU#5AL^2XR5X(?i9~pB4E5$(&=VTx~U*pJdjC zz7W3FK>?Vp)dvangfB@U4ADjxlAYrUmNi0h+=@hqd1WwbtpZq}^GWD|iHF7x z8_L&e1n9N=aO!{q?({r91)T?OmZR_nRViIAZq?2(y)M|wK&Jc-38vYL<~Jj5DM^1Z zyQ5zx?IZ^+fu4E_W$-rKU#H77bB!-lUxb;y`kM>o$5KM|Low;PcJ89eI-`ZOB{!__ zPm2hdWJ3sen19#PzrvRL6NLt14HE=6I%B{1W#C=S)IU684DtM;(9!XYk&*6&O}tS| zf>xiI?&>9osga>HOr2a#n80C|GvGsJvk?3dHyxB{!_}dzF8h$+KndLbu0%Uq;AvlV ztkLgZosRzXgF!F%zzh9#y85$&pd{@~CB)mJu9bt(ilnahy5%}eiyD80SYSP&x?>tf{$l<7FTOq!MosN13iZqz z6S%kcsiFn-m#i>UW9p2@CMR8LxMW#21^ggCVUB0-jq^S{!56-U@)z|eSROo2{a5yF zc#@eaj&w~+*(1pTP*dbE!Sn_plT(Mt=Z33YuMo%WmO#fV89}OJRbA4`9zW9)4_^>;KX}96zI2hEBF-i(vD| z(;qh$kFS@=8gFdHR!HrU1o3OK$ftluo9-C3PI@v{7I)xi?d%jD*adE%Kqcp_U- zT!E^yZDDS8f@0IsW~MzaBAHxCsNOtsinR>>!()V{zHVVuYbmB-CNS9pOYvfEd;@0P zfJV)4zGLd^{sDP&cqB+W3I6hO$7)`N2Ci1WWsynLk(X=^D9mmR_g?-7UCFoa!z#F* zsIM4+#3f^1&$RS(X7+b}bRd;0Y5vkPeB(WhHZc9-*C3G0)S{c|sL3_5v3j65zFzeEkXB#EtiYum$i*iF`3=FLq^LaS zjB=sHmF6^(tY8yWF z(#zn%{^^1n$@xb8t8S&`U6-knilQGQ3dzAEUYod?_b*8m$$eKtXY7#~w5>sy zkPJtzPqX6Sf-)EFq2b^U{3#(32j)eboctGR+-FnCm-N@Hmt|T1ImNwOw01c8T`ze- zdiC5A@Lt9C_EM;IC{F?D;Qb7VDq}w93hJ_3Nn-SpReG$B5XHZL?lT^9rG@BWDwN1; zTM_f!QxaN9(?%7$PLD!9njwvK?m|j;)a?Uc(C^=a9KW)zdAQZ`isQKT`h#Ei79?UK zby|PGH&$qFCW6ec0kX3t*@Hvo9dU7sYw4i~BRA{K_=M%&fve4xrJRtz4*TU8_+1Sm z$pi#qyf)({pjWtlyNr*>IBLNs$)olg$8E3)Z&Oy^r_OWTofb3RqpK#Z;_h|UIvJEV z4aHqBJkH#c6XbS;sqc31z(V5~{oMI7g{0TdUh5H7wa9f0_s{35&fR#R;VEqD;J9@B z-GlSYI~8||HbPHM?Nl##0_g+SWUtO6`C=v;KXZ^yPFdBchiNk_ts_88SoRc zN+Yr7Sx-J$QN1o%N$CmgfKnV>a3$7J?8KIW!;!aAN8%R` z_pZ4I0dx6#D zZXlNXC#;%*dECC&ZRv=TbAnS$H_k>e939HH0obv&W2Zkv;7~ z;b($el~UTNcS~voKfO`d0H0(eVW|E|9=TcWbSC<29>x+eJ|}xiM^Z4sFETUaXWN}3 zQ9HIIJEgS`9kc$#qE}Rncn3ZH9XbE!5Rv8AiY*1xP;b1iO_g^uC%1wiUyBEcwolS8Z|YsB#X=-g9A2VBWtV zXwEjG-yEZCp;$p}!AWTAy2k8o1Ti}ui7(x2U2e?0G}cmT8GZ39M^4FG352mrV#aB9 zHupnG7ch$xai>d~nK~v-0jiZ(4cH$0lijUc15Dg`WhE_T3elBVbNW>L zg(f5Brcl<}zp%&qB3KfmD2Bf3Eu>x<=(zd9%%);VkI@N=5GZBZTQ|NT&4r{>cy022 z!hM!?V;CzYHo^wk9LdN)iIsYa65-R&SlAJ0Laww7<*!R1$F9KtLYcm7n?3YAdJUSm z`w3}xk)syaP@JcYx>NHZ!)>J8@5Qi7zcqK<9>IIfkfd|q)rnY6~ei8xpO&p0idUfD8iJOEpuEaGfDZ@5Jj zHD4(J=uv-#kmA`Ipr#KOk_=-%hiuZD0(!?X6mLhuelQIwKOd&x&GK`IcEyOe<`uJu z>t&uSW5gCF_ok!kKnk)> zfZTA5#l>*Gga2#%Kug4G2orwhwytnUFP_&uc(h>EJHXWf-R`nB9P7);tU1&+2U}v& zL4UMmI=?UBx%--s%fFg98WeSAHl=eyJ!{Xu32vyn2P|~Xnwa!(pz*s1iow$bqP2g*U~omz$qfQ=Q%T7G3oxQ^d3 zjjDtEi_q`LvB4j)Qx{lPaldZZ!e}~-L|mEwxawrJl)V%*A7~2{_~tIqzs}cP7;y+V z6Lrcqgm4y6?j6&1C4RG8=Juu?E2})#3mlPCee2Cvrs1>w@#AeDPI2eTW=C}Y?>>MjptXL8%zE&P>?@E=)m)~#I;YX-&j(Q-k@(g;!7q!9PHqmnQD ztqfmoyl=1>wLQ>n$I?*iidwaFPxN@P_pUwQU*cAnVP3Roq+~O+Zv^ zVRs0ZMc5vPs)|6w0JWmlrpmtQ2jum3y5}E~dcFFy-{MCa3lSO`{~lx`cGhu<3nk?V zUif#W=jiCRMrJz&WlmH#KE?1H?&|nyq~CcypMu0RMC%11wYc^mqTXyRlSvF@;gZL} z+qmE-)8{#wkKdAKiD7b$=%1iTexmq#yS7p%FKlRt2g;)n>hanl%J&r0u@Yl%#I>Oy z{U&!LD~j*Q-TQpAxRft49WpSGq=Z=;F*U@o0)@eN3(GyB-I~iqJ_ZH*AD8`$F>+og zGcEE9YyUy|@ehx#q`xVhB;k(ByK7&RaeWlNB z?1LjVpuM#xXlX6IVQ1)XDJ$_1^fe0iN2Xh%#%$#H$9z;VkZw)ql!ze0KfMwBy@N3x zwOn11s1%}x98^U|SRL*@L`+^!CN|VR6N%c#w5P)K(c4 z1O(O|q2=8m4aUj2gFAD>NbTLTho6N~6fBZ<3d(NjB3N-HGL00=?t4O}mznyR%X4Cr z+#td2=Vq-QcP1$#NY6Q9occw_3nbJu#k!*}kLq(k;R~I|I;f*LnZI*YL)7{1kwxthta@1?wJ_ zl|<5)DIS7KmU9%6c_XdXIU*6%eM5TMGwP!Xaa|ox%^WFtLvq}guJOesh_hW$cI0p1 zenmM1g~N;F=b-lkK9*Z)tc_rnYuSN61NX{H3jyv@#6BnW&WYEB6mCaF#M(9 zhj~DUjV?Cz)75;>pL^z|z+jnxkM~R|1sWhezmSzNUeh`lc)T=|5rXH*@dQ6@V<6IY zr>Yc}c~1RXFxmUIxy>SlxluKFBka6i0@=6l1-oI;EZtfa<7tKQJa(@lCv?9ApH0+*!cN&Tcv39dWp8Q4 zf0?@$jWFJ2AMG<_t3Zno2DV*UC@Be2m{|+KxogVLqGu|<99LpBsyOqhIR0r5%R=6$78!Tm zVK$e$Fm)kp9R!D~e5Yy8_-Q*bF!zfi4^+2H#ND9M5u+AA%8&2Zqzfzds>tFrTyv-^ z5v*0FFRqnO@{*z2r&m&@684@{F#iRX@@-aeFX|wEl&ErzT=Kdr{qx}w$KueW^YD9lJHt>ocQL_}K;Hl*GAPK$+oi$GFSbJtf1d--F=8 zkc0-k$Pau`Q0*VRWp`Pc4=9+T4FK{v5k#1FU{#>U75$VHSPz&$tVo_~I-~9klq=zv zFqb{oX6b>}NKa+OU;Ci0u_2MN)3_A!OTDdn^YRXV$|EuAfXSs(BQXb0hg>AfZ6-}9 z;e?+iPG|8;SBb$jW>uu6D*cM33tk(P&0v_k-(fgs)x+DA^3#TCGSuYa4VC4=7?^o# zCW)528&2YALIEeZ+N5_>Vfymk24KTboC=x&Yfphq-UHnx13Ft~*qmufDLlk3!3#G; z9D=t#8lSBnADNTAYBfS(PLlxg%8tCxMp}hsW8Dze=nq~sQPiaNYaeS3w$#;V{;3X0 z#Y-%=(3Z&Q8;|+P#9fNZ&fkmoTssEeJ`fWq0aKe!t_Fi@Cxw1gZ!ES>wA9@ZuzafC z*`T~BO2@v$mQd+R-9ERlC4GA|3{DoOjE}GZ_Ue+XW@&5;^QGgZo~+M)j8KKsKW5rV zGFUYRf(pMlcK<*G!Kn|#3J}YVR{6{~8*=XLdpP|G@9_Df$IYU>eOT9JJHN#bh_mYo zd^E~0UogL5Zu((@l`tRQ2cjSdFdx5vUV*`c`|^c2>htuU14H0Hp5;Z*h$>}_v?683 zE$^piS(-J8_1J9gCk1ISY!K8}Jmj#-d=kCy?P+9RBjKn?{>Y#yf3u>Ebd4|nsa_2C z2m9hZhkxr~+MCh>c-9T-+`Ox8@rXu7LL%bDL*U~Sqq7~%cpAXU`}ByI7*bVN?a~<2 z1#~mm-b|Pcc#po(RQq2^4S>=Zu{l40bpt1*;&8eyuT00|0>)yl$O9wK7QdG2ugi%k zuh)+g>%C+JLIFoF%36mLD#mYx*l=0ymeBY6cLQNUd_St=PMHSGwY3nlGXIKeg`1Ka zt9!c`j~d=%wO))#Zf4xk_nrt*+ls6PgtlnCHrGo$vl`o>Vx&FoVhn_BHe&h6Ko9fWegA^5CoZ0P?>De&))h$ws|DEdVzR7rhCYweW^=26P@a0kuO4~+Z5^z# z?4-U&-TZ9P#j79=QCCBTIM385TENEw{(AV;#r>fJABL6cN~*%B<&#eq+S1V|xtz#h z`|tJpY5J)r`c`b-yUm|w|4Zb1JeZ8lBvXE$$h`W-YqDR4@T}Tmk3#n3y(;XqA$RK= zat0_IyK9eWA%KQp)WEhsbT7|019vI8gQ2;rX? z4A8OQI%mc)+&f>yu(x1_ypR!jHEp@ra`7rtXv?2JfhL>(LXan*&$~_`w|S!OQfQdj zc7}v}-*K}!-Y>h1z2obNG~*EH0#dZrFO>K;+p9~vztPzK9YlRpfXC19Y!?*KbW#VJ22Y19c(umh@EH^!j@qGV6vPlhZe*BRX9z z9kB6=A|R)Z}Nk9GR$j1OK$fhprZD58$;X6rH6|YfS^O+*bxC za%^Ud@!2X3K2R*H-oa*DIv|f4J7@|Ee~yVOemN~Ei9?!xr5jTWn^w!QC9`jncVq}@ zen)XP*#SvgO6eH2V)L$2ZE(7Wf3x^Gbpq$4G2A~+n(H$>j!`yy2X}HKkuIYz_n$P& zcSD3LTVAoO<1Y=B{OnQF*B`BQ}sf?`F zVSE`?W~KPix1-+#cVnt3RPHc*8a47lrb90fY}B^spQaFi5FhC7t7X1m9&tikeNXXP zKE`-Jkq<@d?=buPEH{JVhl(;Z*>}Qch3agN(p+=GvY4~PMl2MlDax!_4`r(dbFGDU zwCvl=VAeeX0cnMR@bRm%qX9;~=TJXEU!f<$DYGfIc9`u6wFyZZoS&Yr8a zU86Ocb>Q6Yy3)R`{A85b@Ju|Gue)6ia!9o)??0h3%5A}gvdx9*E)OVtJnL_>e|n|; zfg2{ZZd&)aG!`Up`KR7I(oJicpMPuUFn`M^=z79qU}W!i+-xORXbO$C{GvQ)!D8h4 zy;*_Z3$m7Tt_LYQ_{b@gPD|t*nzi*|yUP&wS?N7j&BMREZ4rnot`-9WS{-~viPAIz zVx4eZx}~38L>&C>)%nsIO$#G&F~}YcW~=@{!CO4oVXz(VGz^}iIbpe=xwbb&3~P@X z^LOPkTn`Pt^{C6M!0VhQyxgH}QqB7`5(FT|T4Axqa96T>^5H_! zPPYZ_Y4Wo$g_uAyp}FM{Z;FYkhpWKO`~p^+h^hs_vrxupFd)24&(nRLOWzq|S(FfaS*70J?}t}2}dtsVOp6S5UNC&Vf8tO>YP_NmXj+5R7nGCh=_g}7>kOxC3*ccCOjQ3i_?F3Y%PLX@`bu{nNPe-`>J65ooAUy~IJXvIK%C;k?^ z`gl-i>G^8c2Wb)Bgw=LXfxa09?6SSGi=+N=;eHjK>0QZpO?Kst z@roi9%r6jh+?NV^990>cBXphIg>XHRV17U4rPzS0*tuv2J}%BcJ7y$;2TE0u=+srK zHE8^sD&_hKO!a}(rQZRT3~zqpg=Mlo zCQ9AkBmiEp2g?@!ymP@+8&laR_D3p}SUJ-EXfUZB zgHxGnb6qSE<$liGWv+fw=jDw3mklF#%Q@7JYC;IxEJ>1yZ#X4$U@5xFy73UjXEx%?kMe)TvSBCO`fiuv`LP z|09>`?9n0IXjPd~tjgB>>{8*tdg(i5c#xfHgi}_rg997B;yuK4qnBy%Tyk`}Wkq$g z)s%w<_D!gpy9}i@A4rZ%_;jl5Tht9yCH<9&^ZDvv2AOPYIHsT(3{Jel9}%VmDI3I6 zsTH)2=xX9U07%**DST+xF_tj6^XiB3w`v`aeE1~60T(HUzh%G#^IX!u`ff1icyH)F zUcLQtxwJV6e7W1Y`zv{uLpk(^b-Ov$-BUVui_A?eI4Il;XtF07tm$1ly7vqv0Ro3G zTc5Hvbd!#wrWK)tgg&^}>h*C118?&z!6qo0rMd6TiHv875Mqf~1=*Evc!jlL4cC~i zg(MUW>EZb>C>G^3X|mZGI;r(h_Lx%@k2ZPr4T9w*YtGN2t97mh;cljYYkglJLQ#;8 zSh_F4K3^!Zc6h9n!APE~khOMG&Z{+3g43O7dR=wlW*!rHS*6Whhcxeb=H~CG0x6|p zX=m$p)*Of6Sd8Q1wM+gB&gXyo5{7X4!qMJhjku%AgMGi4GOKmtBA@kw&PUQ#COL@h z)yA&)Swg@BPQ;8W`h|_Yhb-`ufUEgu%~I%$1+qZc;4fp#)8xF`2yi&sPFZMnP#D?wgD%=Ecb3PZHK7Xw-3d1d=S=1D z9zO9Jw;;cT@9s{+-upcasLc=a^X!b`~eHP(g zj#Ia7G~(|P>mqLclvl#@t1m7{Tv>bnG9uF^;(5U)I%w@bw`Q}=fTY~KVB_0gz8bP+ zpy%+;!IQ>CWJ3h9B~a4stl_`}1lMAjDzG3`y{SEJvtPhmc+TvL5j%ct=SJCE<{`mn zos7ayj-m)bMpk5^y)7vglu808K}}vwm*2}G)}4RRF9%ER>&BY_a7)QgM?KTe;Bv3M z$uf;(OI0wAHEuLn@gD=k2#58Hu>YdCz#OYM1}op+yLCAq_#5qkt4gsH@Gi)+?nXrv zg|JcS7|JV(Iazh{wvlCqv#x{?M$GjJCPX!F_&=DfRYCAvI$ZHgIk|^+F>~uasT#(- z)MhnNGSZ9j31{PT>+k5QzYaU6d{$rb&)rn#x^;`UM)B3ixsp5Cfc=Wt(e00EH3>#S#89dIyXKGX_>nfj8rH% zE5&XzIA}{@tS;gQ;?iF5zsb7T5oezTGnC>Bn&$wrZEm;F>eetEQ98sQ+T0+;=mJ1- z%Y%I6hZQ6YD-CNRcl&!c=VPw=jv>$Tu_iKBpWv|~fulPXt{s)13|>oSY`9p@s(xr? zD|8m$@DHY6>uSD+M~-GoNovnFWfLiL%<8a{9v83~_znFq(TAYY?=5LkqfmAVU2`d( z_I;UBQDfi7zRuc?>j-+mDvn_exe-c`{Pp2=tSq`iUW<$;4!O>Ud8=liF#!h z@2Z;F6XAh%>k`@Uo~WmzcMs^XS7#>_5O3q-KzYw%@O3^Z!U?4@~2fhNDWnB@fk0ljwEy557{>g{*-cg+N<6ntDnoePv!E@ zc;4MzD0jdPMF);NK+=__$4*4%vWu+AP7SOF^zlB4>Nn}TMS8LmWviA_MAWO z5n8X>D!0(wY|fv}oUyi(tViwFbi<6>_#%4P6Y930d5sfH%H<4(uH&W%?7wkNvke;)tnlNR{ZR((|b8jh5Z*?4Rl#b-?%tpaqV(ivaAGs1m z4slIerAJIpZKDppGZyvC9bza?$dz89sWZhl=J|g?a#QSQ$4~`&V>NBJ6X7y5uQ6`m z2LP;i*+Y>I>_ar2K2k6y(F#@hP~!k6?XEb-~+>t zN5j>BKH6aXn3!bLaQwQKiQlPWnc7Lgqm>A9RsIYTV?y zO*yY*hT?Cudk;DNRuaK@A)Cuy8@5=WJsRR>4V(A<_yh5|Gg-!)(;_~(SM=&c%%21J zC1iiJgvNt+vvRE;l4vHHxeK$lbN*hw40GhrO&DyxksHj-<4 z&Hf4-LIpM07isZW{iMe$vM`5hf->W$qM&?-)_!E10!yFL4PeJfj1U6zJ*gK5yH;Tg z#{{nCJb}!9V<=7ecyhRE{#r6zs2+E-Za;y@sk(f%PIt+hPaSo-CnO-5E5qlO4)vS( zz&M?|0(ru8A>W7QO?(H3vvCHcuORR%)=G-$fr7%qGoKS>Ek;D~t%u~E9V%+j=*Bif z8gd`D6a7frnNd?d8N_oRMSM?o!LH=iK1iiz-Tf|u_`Sg=(1Ngvp$$c-oA1PrYh;oR z#Qz;Af|4LG4=^xqmp99x2r#Gq2k69aRX4VX`J3=88R<8z{<8pqrt;ZO#@0_w6k-k2 zi96@+aToLV_mzVW7hlk!g`Xejrm5%f0quJGe0YWHblf`tB#{30_8SEG7;_i0d-)O) z4y!V@2K0aG;_+!9>ic;$4#c0bC{L^PR+=+D3_=s`AALl7mRp?%xtyH|fBXQdYSYuka6J_IBc5Ag5{0mO3#Cr8K7RUcmB{KmMMns3lID2e*|Jz^r ztIt#wTIJ&Q0#{tspy}n;S-5sq@6?)_4>N5V%IA^ZnwUVX?nm7BWmNhDa>YE`Fu>X)|l93P|J$J3JhD8eK0H4Xdqf^&S89_)Nvb10s` zGa<+i0f*LAWdu82A>OZ6U?nU8QJq{nkR{5TAQj)PuZ0wn9R&w=jz1i4zVanqo#$fb8{oM<(+SPq1DFCdb4Z3Qaw8+FDING*i1Z2v{&<(^=~1a zznLgC@}5VywN$+dAYVnW$`j2I;Z@L6^iXrM1B%&PN>JbM=F5?HLtpGEy^GKxB(YgM zU*Fe$^9Y*t7e~EDYvQ)CszHv%F)vvZx7_zTr-;!B?ZPTsxOwQR^Uv(>4qXl4Gb7E~ z&l5SAskQx~t^EYN>i*d&_LahGCNy9{~7po4?cc9X~W|3n2d%M&WxZ$x%vS z(@^tgihS^5r0?0*=ha=(x*AkI(roNyp#C){+|~KC+aoNW!jl}bOK=TW!h(GFEP1&N z{Gc1nq25al*OV!N`U05(^1Np{FKcF;b_Z$Zw#55q;q2)Q-#;H$iX|t3Omft8t2{s* znLb>KG)vR-UF$2#hnPD4Z23;ked|mfH7(w8RS_Z|s7i_swtB;h5V~}V|Kxf_xJ-2R zt*hP&o1Z1;2~64XEswP^P8PL_^n47e^6zIQ>-}kt_6%us0JIm@V1a(ndd<5gE78h> z0+q#@&&f+wruYT(R~yY3_`5gr)YZj@<;^GeUR4jWRT`zTbse`)cyUW1Fo+jpUi4nL zw)p`;&Zl9w`d|%u%+5mNExhju6>_~;|MbUySHsc_2+WD^n8RsFgYX2HL;qu|$LDHT z`?(rQ%#uNU%@COmpcSas*m|YkUC$O9$@bOfonL`wR|&0Lq)e$y>;!szs~TzXZdCV2)osTqwm!>J8VDa{z)I!Bbd2I zX_2Y^F4NAp&f|e7rSh^{H9b}DpSLwl^sG2S)p=g9el_8_M^BMY_}sN(Y!S0wA=Ogf zGzgUOxS><52JtHs-I1GJ12uC!ooidm>@HU3Rc~pws<6$pVve+*6Evt1+nBr;IA8sw!%2OGQgjGzqaysVxyHYDoywX-h4sQKGd)?IaRK zBDOG7OKq_xmLO`cq>)-e#rF9A=6N~)!RLN+?sKkl&UL*|Td!?lH2#uqRPb&QmLWuA z9640&K7!?!ZRo@5Q{v_m{rPtyREv*roY(${jV@BeZQzl@*X)*F{3_8Myb8&h`G6Kb z-rGpDRA(y2>h5XGvp-xLxHvze(3+u4KnFZcJ+;O1VVarHoxRu6eZ=iJaTVM)uBdl3 z31aOia40>Ug?;#3*?`Z5&v%v_gACYi-Oyd!v=Tk`B zrjKbaO*zblsNuii8h&XbU7BB5A_s_5xN}!~3`2>UU+D7rEb?EQUbqW=dgI1NeC`d25RH!Ryy1KU>3T0oW{51CIZebJjlzwDT->|gEjNg32hXS)G zR?Kp|_xu`Cv!-jB1#LKfnA$!J4S)koEKYpsCap#l*2z>oRLB^(SVYoYA{;j85!*ih z53QnE&$=QkZ7#=SzDVZae^ms{^5-wblN=Jy!#tMfind^pg(mg}BcHM&A-y7hqZWpK ztEw;5`K~WX)YI^0q$2A%W;1bE2z4{6Z=vPed}1eO>b`|z$L^|y^Lwtxa!D=siAfuu zdKBe+S(CTStFMvPQ!2m;3!cndb$H?tc1UT7-+tGpoo^XJXux*oYXAj#yh#QUYR~8@ zyWTVN)Z`)Znli=lp#s&+InOFm5({n{g<2P}dr^^r{KKb>tnEUWVVVhj4m=tjkZ^7+ zwwhV?w8jB$136POGAW{^*!SULJ?(X}!)|2FQY&h?W;9Dp{3|c3TJbu4+;Ts+SRuf~ zHWAT^w3x8H8_FJ68h9TLn6GqO*A(!7S~}WQZEQHT7jN%wSc$bIk^P2yzm!7Y{BzQLk13WCVEacMx2dd?PZ zpEI+tD;2%`;6XUjHmaKU*nJCd$orBS$DT8XI+Yvz{dY!Nr1$)l`OC5%2KLdd2SxXK zq7U+W<9mTkI8Z0z^q*f zwDXxmy&Jp61zq$#qjq-9=(-6eE_fGjNWh+kx85>YOc!>6w+GWLUa|**6R+D1qzk16 zQlrKz-}jnD;Rsouw~=fUthyoSc56DAfOg=fwJET}YilH%~ILq!)7nAA7obX+pH z{N=Fiyd`gALPUPPiN1?mw3`2wMy@z`OmOs7?vN}1;9F#$%~Q>^^!MhlUj+n&VvTu~ z*k)y$n+0vGrXGD}t#MYUD+(e_MPmxG#Y*l67V4gnWcF%Etoff+SG;4r>#&<_I9LJx zu_cK~`3GGGb0j{zHuFzKk^f^QTYadm2jb3cW<(RYk_opFw>kA)5g&n$(%rYv0G&z%P;WT};g z*Zs2PcC&m01{EjTq#$w{cNsYlt~v$!8zt^DJyu2>uo#W_ z^tl6KuV4N!%ZjQn0E%mc`LRB^2-k$a4C>qYwIWFau21IyNwR!<%G&G_myRoD5huUq-;%jIFE_@7h?yvN2e~ zZbM$o6*gsjnFBNtIl;v(b2uMBkB0vYz#k3UG`G%t$r zqg}1qK_(#IQcp0L0X2`!AXr%f?~mBG`QqUfs;LH0Mca>{QB^8TCbn^o7LK zuHfM4U|<$L>R==V=$Jl3HQ7|0Ui`Dk{Xvox1f8FzMmrs@{uast!l+dfjS#UJ*T{Q} zh8?a1%oA9CumxI;%kOSU)QJq6ca|jOcV!`kyE6}N4sIU{wAfQL6sl7-@KmAI8np?4 zJr+$JjNg$9r3!G9iAcH|R;mT73H*KKa@@g+%o5$iEqxD}BH@i7)gR}HMY5UH(xW!`1m?j4dzgmNCv7=_4^+xFTL^wu;L{b7+^w6 z0uy}qMsU;EovBLTwSuyiX*$(X_g5s)%|6eUmAw9PVH21I!W@XS^gC%_xnV)umM3sM zQ-#?QqRH&a%c$?TjtAuIsrEI&8N+j=3Ml=2ZqbC5IpCd4TXisM8@I+Ku?m6_&PWT| z#}D5p*2tSwUeq#f4xA&0IHtIMqew6L^wZ3`q#kssdxdVCl(vtp((wG$8fk1oSGuEx zpvKC|M8!!_*$EcEy*mXJ#E*uBwWGYFdJBgiGV&l{ZF4yl?C?Yf;pN~~+>Rf0Vg1BS z;uWe}UZ{_eRSZJ`=9Kf!WxSw_4>q-chUB8*n*JJ~no%9UdV^I$ftnCmnVV;6;Z_9;E?%^J zIDKI3Wi}+gcg!$ug#)xOsob@%HrII-)FfWM@_{wEmezoRz1LdI*_6$JN|kH0D>vqt z`8o>IrVI7;&Pw|8udz+2ApsA?bCrj>7@cn--evnM`rEL&@+#|ts1ic3dRUmK099i$ z0K3+2aL=oQkahl2`9(!+F1JpmF}2Pmyvk-qVaJpNKz{r*v2H=4J0(+@03mLs-J z<8*|rk7)ZpFBt_F#hd|nHjK3%tTPRab6e|vQndvsb9t%f!7MSnAhmIrPeaj8}tRNN#E1vR@JVN{{PQ#$(=JTmnAdHlY;!5&fdi_AY@HrtEXQ znR}Xm@<5b!?wMb&8~EO@IJe~y+p7q1bnT0*Vx>(H7WW5c#ojdYb|L>RQK&yyqCk<9 z>WI(kJ}fU)^&iSQl<7VT{#^%WA+*Ad856U|()d0C*{N6WR0yEWPP{qyIl5Q4d z_kVjvI6fsEa)om|l9NsR4O7pZDOW3Lbe^y4n(NqPJaus8|Fm8)t0q9 zmOq@Dn#qR)2LzIiB^dF~1iZTX&oRK2KWAxDCoU|uYmo^wm&-q#+U zjM9h+g`7(B@Q)5ksDIE6JTw5@R=*gvO1$gRhn)gK#b@5x+kJ)2;Xbuj8I{~z zB^^uLXz9_JT`mA>_=da-=lGlj8ZH7PNcrcAiZfNz`3pW!w<+b(4WxEebv{qq4~+JbGz@o&Ml z*g&IF5!5T!SfkI%E=zm)Zb{vPp!*f#b_v$LKRd9&Qr)`j!@U|M@n9GAx0eW5ZC-EHQQwLaVuw3`pYuOj(V9kr8t0Y;aI z3(+opB{Vh9$Tfkcz|HkwY|WIT#_^D379b%zZPvsg@L>Ez6A#k|&pJl6`L!ZB(<4&QI=q zFyz-(0$U%%&ewH{jZB0SQ_gEJ_LR=1n)4K#T%X#xkY-6V{1$~5u_t9{{v-?NC_VOt8s;n z3fT7N2dRS$aGPja`k3uYQ*VS*qeCxAf1L4!6z6(=R#tG4BCH{XK3xNBUWV!V0E6shjw9UY=<$3;yUj2yd324^(CZ*Gad{uof=YHnt#GuF$vZ0mUn_Z6Gja$6fCNSouPn-m*KY0f#7#*AMY~;Z^Q&B={W$%ZE zjjtopywU>lQnQs49+x*be0H;3{p{1dUgtY%Q`G#NO84^3vQxVQ7}nBrdn0@4#XsGN zk^dy)Bg|v|$paIj?tFOUPG5J!uGhVXphY=qLdozwA<@s-KhYY(!NvYG)d4b{geVPE zP@g>u{?09xegz2*zRC^v{GQQl+@Nid48OhxY(;Q~2qBXFCmuOJy z>T;06$o&R=2xF!)w_Jt;F9MhIgum!->X@ ziV1o`S$c~Qcb^yl0T7tajb9Ec=Q8TYc@JtBYbGzzl3Q}Klq*4Hx_|1;UdhKMzk<^B z$48FI3lqr|*SQ{8S-vFI>}~PnRn>|dW@1+E`Pn(=&g*aVV6V(1Y>gYPGx{?tYu^s% zQ>_iA8_$=zLxq&VwCHOBXSBOYu`3laWNU``fFJLTRW#`*mVvdGFP*&Su`gvE7mkAS zhxhJ#!Fd(!>jx?Ipk}>dF{9ypc1z{k(mus4jk8bX!>siJ-2gJgdZH|a^6jsFe z)|L(Avwbmy^PGujobS15)}N!gQM#EOd29UN6Vt7z=~Lv)??dkMkBcXN2!&4NMhUeF zZZ(8-4I4XejYub`p)DJZ6ElfF{Yu}sc-@0}tppzq9Z!grP8kld}@^dS=qKyQ% zX56I4$xD4Ke)3*}cQMtD?%+b80<`N`Z{|rR=ZYRbM()>5E^MsB=z zaN_o>X*#Yds7AcxuFKq#CWm_k$S@?RLnL%uHg$g_q=dmE?SfvmODrw%CuQa%9mnv^ zL11_Q269g?^ZF?+=4qrWqh&lb!a`EIw-A~_`7Pzp&HYG$k zCf?^G2{CTqU-TgX!*3I&=Xo&t**MIcn_RyDdf}Xl)mT#7Ps%iaz}@@#>KL>=oLT_# z$Sa>nFgE4ZXRb=_5?!S-sHlO4bmtv$V%BaA@k2c-Xp?QO61n|sp@=~vrl36ps;t)l z-V!O$*dM;{<1tmT%Uy_MD6fCwfarUN3X|1_eg9R^!+#x;N|KPOem3bXFO~G4(07mg zf0M-YaRT)Axg3k*@oH+d{hNPZeH>qSX|-~|RHr$Hz90zMiA3&m)C7qui`%HCq??j~ zO}SL1rm2jOTZ@Oi7o)Z~PO3J(?n$+<^L3vi{;|)%SWiE`lXG|RkCFI*sU#zh6G$Ll zpl}ii#(XD6hc!*UeG~P*#^U3uY$@rwL3yx1?dj__DD{H0_mkG@jdAMXw>eaNtw5i+ z{A4QcA_j45Irr|~Nrk?dvmVsK<}cxLL`njF0T4r#*l)^MJF_WTHTQiNvgnd2O>E7E zRn^dLGj?Iml#|X?t>&n#5oroCV8gNMhR#j7<=h^vi++(a&qc;&V>t;IS~N_|CapXk zrxyedZ^u%h<;cx>my^)MHyymkB=o3L(Qgv3xepkdPYJ1spus68)=xs?nZ}0xQwh)V zUobDq%e{qL&s5_t=~6Y^Wez3QM=roK$g)6Y4Cc0r6cHldc2c*vFP*Is2s=CzKQPyO z-@gm93OO~PmyjVG1|B4lHGNZo_e%nG&fv(w5?=E=2ojq!lixopj7mE;DI7xwx`|RN~CpF90NQ3 zCXKlbW0fp^L2cc&ndE9Qz8oBbO{Su#7+(;a3_gxdy*pE;J8}?Lq9^{~nYgy=Vsbh% zXaq2R{tHF)fZ!7kNdC*LXjNWrd;Nw^5grK;w`@Z3wp$`d#5&7aGr;Y>!1rUBlc~DO z`adR!v3(C(O|Fo^Mg>ko!pn%B=uG@Xs3Qn zac{#FYI$0RS_Hxlqi>_wF{L$-rirf@mO(}mC1&vKeWM>NH0(%q-UFXe%R5rkI~SC8 zZFeCpM06w0t8%@!2inw_$;z4^UL99;%w@>SswPG#gln|{h2@{l5PQrMjxp}SGul$a zevwsYoA9F5qFmyo{j+XTN=8hbOz06sz2g!9S=xp($kzeB2;RA_ucr@?r8M#Au?PDM zOcED<;U8WnTQVpmlV|hPBIl8q+b2x{t}s$bwCp6_vRnIOm?gG`@fMoASz~P&3|XUU ztH1N6mN4t7eu7#lxOf5kuZr{IihDWDlaah-=*v-6+5N8q#%89$^+=B`uceOgqG=$7 zHMNk-c53*U|8B{HuH-*ezImA>$u26xW$H@VTf_tNl}o~AmJ>Aix%#KLIWNP=bhoYL zt#c%6Z-N?Q;##HNBI(YW!Dj*uAmLoOFnhVnsQ^ zx514u{F!l_E#*2{7QX|#b#l@|o6z(?auvD8A9mzSQSE98p0IKL^@9(S#161w$dycQ z>WrU{Ul_yyP<`;~fFl9n^$|{3sx*zTo+T%3?*gajc%0bpp5FiAba$}};9q=ueSmNI z5>gEm#`-aNsXzV`;GBEMubu|lUko}HdDJcNt^4(TbFnA?h;Co~Eo0_e(9gT?GNzgX z*EZ{#W_d=6Q!6{K1%mI@i>CYd#2iyWk8#s%T!zt<``DC>UESht+-hRAdy<-9-UAf_ z&YhkkT%_ep3_S0^MmqHp^I)k*~Y4RbBVXAv_!$LN)NDsPce|BFRe z#=Fsi!^{HmZI$G1qeAw9a6;_sx{MzjfqYQUJ5& z61N_?_pPps!y?~73^(6~=&rSN>TL7!x`MA&Xr_H(IJutjy?0xZRe*iAye>&Mk1i~h z+5snuHP%5f8ZRZ!f&tw|0`|k?^S!^&d#ybH)|UI|t80%+c9>cTmD6sdE>@emr~o}& z!u;ST|J^?c5q0%rMEaIlE=Z0+p!(=gCjv7{oonls{)}Fm&8T2*rO)px=?MDE+NiH( zKJ8khxw0~6`uz&NLi4m8C%ED5of4Vzx7`d9;-*~?c`@;P-;Jy(2D`nFoj=75C;iO5 z{W^G5RI)`B2az+|lB|7F+m&^}TGj3jhN!^Tavb4(Gi7iO)qDN`LJL+@bQV^g zRYiXNUNfM($fT$|Mq<(g33nVPln;H8>VZ*o@Do8g&p2S{Ype^j_RQzb>ot}tO%Z;Ua z8ZgJc!7A8$*2|SVA&{!yp{#3Sb}y@rjrchdM-h^uxNRgUS}nm&3gTj`1U@d3h&Of% zNX#7$Zh!IYbu_anJ;f&Hd}7ULK8ckq`=of&QS%^m(4MZTcs4E9Srd7fv0=`Ht-XF1 z5PHE|w_K_#O@!v`?e+ef^D!hMJR%TN*Y$oyeurxzfYuNgS$MgFan_7Nh(1;o>laf8TViY3ae( zylWY4yp)#2cmKL{(B>4jBhrY`Nkza9e7%vt#!&UNS{aYd?3M?u6JUOhyI`Lj4-TFBKTk z_E=&2Bqi$2GSavL;bKKkLw%@UTa7X8=Il^lvk~ny%dpWc1$@+E=@u&S1mEZ$Y5y|R zQfTUsjhrk5=Q#`;p<|}%&6UB&ZR0>SB(Fn#;B(xgAM0-a8u)x5Y&J85Rxt zs|`aXRLA|D;VP@j$vgGvIXe9nzj(~ugIER3YMkeWybH5(w zST*5YRh2JWVnoX!uFIy^J(m|^FMje)QaV1)^!mS;VN64`sLw8DFlKRTeJbP8g7x%2 zBLbvY6Pf+?<623~KP|WlHJ#PE%8}HH!I;YS47J@h^|^m4NdS+e@Zo?bk(IOmSb5Kn z71ij3T$}yQ_}-EP9i0Yqdb2+Y?Ml@vrPV@#qP>kA;p@1J<2qfBx*JQtBY0c??q>8r zu%0Lf6>4uXOg`2X>2uN8uTKsBy1#DmTM7=E*`$*+zdsJM56(A%7(toeh>(aI?~4N> zuh^SSl-dwp9yelhrq|w&NZ+}@cji#^F{pQMgAu?SjYAmM=ARj^>QL?}xiyQzXM4|4 zmAyk!NFin<@t4;o1;(d!lw%tr&yntq|9n_WB(>L(Trc5k`t|!WGZTBV>&^1@jfZd( zYJ-yV+V4QVj{$m9$ey6tL3;M-4fc`GTJKVbza;{;Hpu>fB0Zv!+os)uS zC+XcxqULf+JpBzeuRH-pH)gHN>gErp)p7DvC37pHq}1rXi7aePTH{Zb7d@$WJsoSX z?+G?`nqq==vPUPwZM{i?vtH#nfJdcR+WJ(!m6g_xt$f9H0r6w1fLmWqZF^$1W7EgJ zu*W}$C%U3e#uRNFH^lOB`~!dG@6bjo)U z^+HJGJtrmS>DbZI9Z1cHv8`KoHJhfD-i1qFLSe^XsX$9&z(~lF%Cgf7anPM49R!=U4^p|xo!`dWA)D#yi?8G8 zX?8m5eKS@CJh0TwCMK&X=;U=?v%ln21jGReP;&O0oq1z`?YEDc{f-Ncb5p5))E z11$qDwwpm;-RMV`8R<#;C5?P-7-(8QXuq8HLb^wqSO-GH|2)Tnx)P7z?J>GA?ikL+ z%a&e@0+XQS#0uRIX;rR~FtJG$lXZUZRXylaS2NA257hJUW7{Ij=(Qwaswq)d3E4RC zJr4F8{8^PoT)u}T*CLaOM znqF*xm^kR~0L`}2CcWWf6O|cvRdS#05j>qmjGyKRE3xoniqPYMq2!>3Yx|GsnFwJ+ zQ!x!@O~`Rlv^scp>SlL37nT-TV@WrN#4S3loE`yVd#OakBE4C}*7wB1b4!G#9DF zZBfYDE0Aho=r-^tj7xw;z5Xm0hkCz&oUhX!#I%(Ug1Z^snHCADRt-Y(FsPN%zQu@b zjL-Cb<%Q|~IRx)K1~XTI6-^S_jCe(#;{)i@jV3XeEv5LZyHBM4wL@MX1P${grDoDg z>JJd{@BgqvFl>AB0sFgSf3e2P$mq+S{$P<51JHli_0rzeBll9&gKrd;2I#D9k9h5w zepoa%r*2?^cv?~8EH2k zp}qAxW*%j^m-~qxt|iSsgW6b^?lWCxh6Vz~&=a#>7y3FYuAtMtdD`gicim5Ue-qY# z{_^rgSJ^_YKkxQ3?H8`XagQ-&+;`&?gk4{X8?!1u$f?=&AzUL~lqSa?b6$M>8h!q^ z@SJP+cOl4M;>ap}ulTDV=iUqpHmr#lFF#+wkFD{W(6LOEys+BXG4h-l>mh@lJJ&oT z5`6|BywZ1-Z3Bs0{BbX9#^%~ZR z=BV{xP$)k(bU46T+@JDFPozaz|$=8;R3 z-!m*xveF4hLO!}p4JFjS>rd(dw3ZO_;01e~Wk;(EA{#;$kH zSoPB#E1))A%aY|`W0NCC;7ube;oD9KdiGZ^uSEIwV{+9EO@|YafRVMk$WS9Tx9EGh zu}O2Smd;i*FLGUQWxX88!NrgQ54YL%-4F%z;n78_X{g5mnk=h`zCip-wUDp=q4_up zr;BPRT1(3Ti#H1M*|bm`8olq|-$Rm$ePjzzBb8o%z1jQGq$a_r!>eJF+0qs4#aqJ$ zF--`YNRlWIn^DwPw@1?PG^Ok4^a|Im8vHCo{spd+r?yfd2 z2gIoUs@IqQIKdGEiai)9x=AQ9zqf3^$W;i`if(=nxY<>$W^z~{0vLDHeW$u0mVONO zDgv*xt@WagNAO9%)IQt)+~ef48-$gt zsX@?1D6@C_(D<$0&zgny-Kks!JFsz%@7Q#hSZK4J8H~uviL%1rIZ<;xNu)vw`nK4d1x7xwzDi5$*=^mtfXhXir!FD7i@7N6xR%zN5Aam-y_|vzebzX*OjTk6he=2 zEGw9f8ma}kV9g}GsmwdhxELMlYIoi_uhkz6oGVm`DPa=)A6vD=+Apm*#np~Deo^mg zRi7SACa}mkT#uZTr^4qSa zKLQ^2M<09Fd!OlXdB%JrRmJ4 zZLdAu3q<;D)cuJ3?V%;<=J;`$PlX_CDZFT4&$b?BX%qmO;Ivjpa1WIr5oJLYV>PTM z-N&lV!Yv1)!p>>#w^$*6Fw&If zu*=-)Hkt`O*hsXe+6y|sRp^O>QOEpIB{QTUaCiYfg!cEy8{9qQk<3KQ-eN3~eL0s7 zfw7n;N!kyh4J+WF^6)RXk5B1qe=WQoj<#ZrdcFuavyY)%ny;&G?r)%;L2YmL)D10w zk|&}|KwUqmEH1N**tfqVoAQ5l(GN5Iz*T_0tclM0o0jYFhSApE1AJq9m5ZUn&zZwN zx9`{vWX%_u{vHOIAKLQ_-F&#$=5M2{sCk3OT0H_2{GT zzVMNJWb~mgpCZom{mUem4+?xt2`OgQa?U}prvE>+_5TmnbgjN$KMC6h5L9*9&PvB? z-wQ9Czo5}N9dtTZXKk}jd*35`d2w;?`$TT;wib4zb11iOcek$Dskt)vHp(k*dF}IZ zZ`1aRVh@7|>!mA~Cby0;a!TyJcua4km(cDmy6oovGP#BMwtUBd`gEm1UbOsAgS&n- zQB@1U4eW~iMqWoo6XV!WtU~!sa}`;oCs*j_>b{ebR(E=b@Grma4xFs% zv}F$5n||*Ol_0Y*FthOEY*;};Xkues;8R`c>(5`!0n#L7CaYzvZoY9ByJjwR!lixP z1@`?kW1$;+7+MnTh73Ip56~{_t5FP07k@^A%s1nz8~w$F2K}|ktFWUgN$y^%jhITD z-3U(s`VCjY*aLf3mv_|h(3H3D4o0Ij8gbAk=keJjIOCr@MP}A^he^=vPAYF~k zLKr<2hIlMPt2cDJDH`V>`2+HEAb{UW7>bIF1c|#a&TCT=&%IrY^ke#}#2UW}BbKeI za9c}i5%Z{*#TZwB(gF4=Ey`ee7iSh%M9ReBs{ z^WFdGcm_~7_Rg^te3JCRrY|j=XU>XEc|=^fC|m~G~q^qVQdvOiT3Y%^AWwIg_=?olY45(#>zhVi}PTZ>0s)6 zyelP)<=b8Dpx?exemgQ$c>ooyjKikq@0|MceW@oyg39O-cvNg{%TKMWDdZ*z1QAt>j2#>Xfz;h#hk&-NMSJ&|;FF+mo>ha8xI=W(o8)~a#O`g_ZlSyV+4~Etje0YRMIJYuPbmJLe-K0*jJi}lC?*}Ur0E?X8T>30NTmqg&=GEtgg;1H{nnzHLG4rJrYn5+xyZyyDi zwWjUak%ENkp9$0Gc(%Q8sC2;Cic85^&C95Zk~=sMh3z0i!rFgYs=uc6dqOhwU`xZs zX{4Op@eYzDCH#@$TV&4LKC^?pUtnZ;=#=w_8LPhifzUbEzg!$zbk%_9oci#KlIS|@ zArjukMFA`;(Z0UAjuXs5pFi;9ZO|B zx?2;y)ahEjUJ5hgmx_kC4e0!3T#IQ}y}@G>b1yV%eyOMwms54V3GsHJ2SRM5g?cv} z>0;+}J^hoEy<*=Y)Daq!d`+zJv|_*%6Ie$g*Yvuver<8>C0bCeRD4337b039r29p^ zsM%RU)sPB~w)_?1fO#Ca<8Z|c3=Fp8J>bRS@K?m#?3HAB)R$oq3Dv>=J5UB=EE6l~ zQ;<}n^&;5(>?Mr4>;^4AsL0;`ZvsY736do1K-vT^2Xjo&nuz9{o(=cREyyS6(H%Q} zMI$+h6*oM>*HqdfU^>D)6Mc!RpyVPt!*fuhfNoR$9_paC;-kx?Ia0!FBiB!vn7}aeHUI-(t+>~kaS_)_8er7C7A_`Yr!Bj z@?UDZ+@a!MRKX9+ZwwnZZ%VS=3f7Ex4D9AtBzZhAZ%{zTo1$2iZxh1!XfJ9#O2$@Z zm`B6~6x>3Jz|P_PdTnwPPxD=}p}Gab1z%nPOZ7QLv@K9EZ0+n8W&!CNiDVES~mQ}8!w^@_Ve={N66h5Ok2@lGOZ<7LrLTeZB`P@c*xxj1=VV35RZ; zHyA|r>_pv+5n-aopQq$(<%Zqnzxdb*UCK@RI<``L(>e6=2Ij>_Jbj0ia(p+AZS*vD zvU~g}6$={7{`Sp%T?;&!1kk3IWg~%sK22Zj&~>0wdML-e@Jj(tqP=mxDFAO(#q{u78dG&_#EjZao+U`7?VbE#y;sOVm4cnqiu3b2>2c!%`jl z%}<}}?gXs-#m$9D-KB8>5^XW(xJP1Bm;g#Zk;QmOwV*axuWf1quwWKpf^28 z#2v?g>5{0k3jh{=xA(Qa>A(Km>r8Onp!%2t&5Pq`+7I`Cv0wi&&bM+a#?szwhH}Jg_IX&9&*O9v<&;Mailg=iC@RpO>Cx zGq!$1n?KZZBn=o6tog)A_ko`2IXUhgj0~>nOmYAmgzxKl#%1!UZ>C+WHD;Id4kWq2lMx;n7E?&M@SLVN2G^ z2z~CKVz;FMpF4P4)2zx@hZ&15r{J2fPg}6MP^*-+-U^a_;y2PPv+6E!UvIoDY8_Mk zt3k@*BiEBw_ttrut=}|(x|S@hIg&7%XKI3W8s90Qz7dfS-7yA`YR|W;0hDWM0n6;D z&#`@vrrL{VKqgQU{n~z4wgWMS1nZmu5zXvMa-*Swu(ae2@I(LClSagaB8xvb3pP!V zO%1?4%ks{fLFC@AH@TG=qZ-BQ8^JipCH}%;dd>mkt$x{VS-Ba>9MWf{iDWNb4*|&9 z-H>UbW`L4ltW=cklxkcG7`6JjLb%Q`Ro}NNPYHM8sUH)Ogd(#Mn8i&H9%I1Id5kPd zP-6pE3XwS$cS)Y#)2o7A+O(>v&;U9|qSGAtEy15SqAc!XR!0rn_FWKD3C_ppjcB}C z=TsbAzZwFYcnQqr$1V-mH_?>ZgH$K&w|CPC7ZTWYcE*D-7e|5W^K5XU@lKc<;v%^1*`S z!&LCf>QbF(>P;^bjE#p8XfAEsCX)XC63=&jYTx*@uYZ_!&!&rY7{cQW8a@ECmgJss z2kZiO7XB9Pt%xDW_rGKRRl~#oqNJ$&S*F-QZ16u&5=5epd7a%wK z@lmPr;3ObMz9AkBglV9=67Hg2QKvP;*ZqXC*m}2PTTo$RRc9CdKYdeSwv*ZRej6 zK$K|WV;vf@jkFaPGqvmG^EngChjJ^**IV_|EOP22DtU_J*74_?kATs45(^|JBHcJ1 zN$j_1$;cLUyTC&Jt6Z!vsl~Eu*#VzLhr?ln^d^6gJ+1u@?g$M~iXbUFF?ITir_)wM z`Y2#94KW|}Ov0oGyJcm5-4@JExziBg_@f-y#NsQ;bD-3cuB{S>meamCNPa?g8pP(eSjKkocXH?4%UC;yMF^A1Ql@&Etb zd1c%4Oiiu4%PVu|&Ou&7Oij%N3XaQxiYqfU!Es$fQ^`!xT(~C!IRF=Wm$|_WDk@fP z5Y2%kIDUM-_xtnsgFpC-Kj8I#jpy_6^v=s`omgmz{xBF$8^6>V*4?`NDT7sM4>iKh zb_MRXzE~3IQ65te;lC2gGdUC2Vn0&Kx(Aw z(bjy@N=QTl3HDXj9hGwbqPSrFc=(-+97o)1U zbw2d7%IgjkWFzW#9>_OAbeqYfNh{BskpM=89(!TzKqh(WMdhKkZq>R2lV-kvoBYI3 z*pOK?Lw!P+Qi{{e56V_Gwu>Rz(*}JF$z^LKM7K#CpZmz=F`dz%6l-tdZ8hSkTFV}r z7|-D}QxiKDKO=tDzp;w83oZ3sAzy$hnHXf@Unjv;#s~7L(4|-&K7?zymHRv+r7NQfHKY~e(RF=SPnu3u5>jEnEO0?Z}M&vd2e$8c+ zBVG|ky6Y5Qxu2wWc~bMSE;QWC4t!-`Okj{33=aPW*B{Gao&p_C*ASfjYD+ViM}lX| z=}(`$*_5eZQZmZ-3=_5k1C3XZqoRdm^3G)@RO&MgrhK;HW))Gk^fgm|rZrHl4tA1O z78nJ^SF$)pQ*+$I`xI=TydouQI6C<(*Pqz7*8YxZdzeF1oj!Pc^$?g~5gEMyqxNLn zi!^L>;lWMQCT z44fqgT3W8Pd51@rx7gj*vJ)Ec3{W>lrkfXNPVG_icVL{21GSZSz36b8mSqepW~An} zN-Te?Z#Me?C2cKbj1Sc`{5luYB7+)sU|hthGilchg&r#7W@Kfv8g&EM=lAqHOw5x| z@$#9s3?|QZB(!Zj-g$A5dqE$_Z+}o?*V4yTV_1_t?bnOUOzYcfUDhG3RVGI31sT8u z;7?&|Dqn&*sB`?Xp{pec0U!6&()SeQgHn~T|5i!%zh=DIkNb(DUb6YWswC4rWa{@* z<`4QZHM&LD=cr-2H0tEG*clf+39$#JU*2RLI<4t$ZFOt6A9~DkxX9Qzf-I-^pm#T& zclw2BkmeVCZCa?^L(GkfZ>QE6G6qpm{(ZaY&iK;!o1f=1WtSu{p3@w6{`6%Dtu^Uh zLuE7ask1Mm*7MAYb{2PgpdBq%zaqS@o?&nIWh>}O?3hpN56-4v7{30Em*}RP5m@F~ zBOCgPjgFdKtxU5SFOI!sfmPTs2b%pH&s!#R)JOHc%5jhh_CQ^&^7p*|=HQ_NiZf?! zr?Q>+EWPvQ`a^o|(HywrWI_~j1j?Em4&k{;W~ymVzX|ne{IDb@r(k665)9`$v@LWU zR|XG6RUpRC@NQ^VMP(_@!S(5l!5KSEGA=W{e&Gqb<{AoH`_7iA=a4jqcNQ^$w+_+d z)>^sysgA{;!IYBW^4)>`@y(jxcLdZR9*CAYLtp*s){dM{$(&-2Kaz9%Vgxr z@JAIk?zDOQu_|r2qLw9twa6WZsJOPojmc9_*yK^j=z2YOa;uh&E|KSfwTUH#RqX_6`o}_TOQ)N$`hyQW@;a^7LrR?; z9j2;Pv;(+cjpWwQ(ak$dlti-3nZT(b1R|A7A$gZ zdRz`pb=+r%;l<3D@xxMOPjAKA+}loD11ot=)dag$m!f`#z&L+LmU9VE^7zk8&vS>g zVcK=tFEF+J;pHL&zoJ6L(|KfNVnv`qfTxNTH*rLRcE(o;e0!+-cqpXVn_GueCpYJIhKC})WW%#Z5GQTo*RSD?eikdG&DrA3u4P3KA z0`B?L1+^BEvQ_#P&`4VwPM#jYHhou-Xf5M@Y01XjXZK*pSWVp zWdH8^+(c4PpxNu0Jn???L}Vdp*Vcr?pwL$H!9kHMc-xBu=|J~~Ag9~ZtdUEV}Q zx4t6>f>?|<2rHFbPFeX2XaYai=`hEW>dH+_ZZ(*dC+-(s91jV3VvW3S@`X?;2&#Zv z`7zE7o&?DPU-dx?0QIUw?RZXIK{R&ull<{eqse>rC>KuT;;1gc zH0zd{1MkuqZffZeyk6I1*1>V1XjVQ$;zO^DIKGTR0@rs+cbVttMsMFa3B4u`6|YV0 zBxCj-TMac<-`Ur}x8;1V(6JDm>&~BQ*lTFx>NR?vGs!lTf81Ae6z9$Brg6wunWmRM z>^lSjh6uO5%oSW57M;1e6m9l@*l|4{qyO(2x=nOw`6MM>_-Xo?=+M&hA3Mg_N$CH~ zcz5gtlHI*0V29Xj-_uuim&M#F_kTTCr*7|4d)L03y0>3XJ6z%g4B835H3;`iXE7*k zac**C$)T`y#YlIta0%w#Zp%Q#F_SURh%brL;{&R)9KratF2Z^ZUc_}s*xlzZVmB3g z%72TaWY5w=5f>ZW``y=59?R~Sy6!&>Vy=0?B80^i%_9z0ZNF~O-m&N9EbkOHI#)yP zKQ?NAyPnoAbVkaquh|1y<#bzHyojYFIy%0}5_xCb5*tg*e;3w%pr$a!P|)F2bRdn2 z8lB^!0IPTNE1#~}-l@7g?f|Q;xZrO6qp`!F2qjQCrwn+MQ+eM^r=m@r%{I9N`Y~zg z!GOtrdK|BYivM;uwFb+hJ4^yoROss)We|FQ((ki8-@};zLuR@*akwk#2HYW2L6Gms z&5sUs(h%b`0Z&H*U&m@c>c7jyYOGfMhJwmem!P1epXj=;bON70Vo(l+U84xQrM_wJ|QCickn(HcS>C zbWX*wsibQ;igiJBlPHo=!N-i()iuO@E;*9#y36tP`uz@iHjj4Q!SkBTSU3;ltJ__9 z;JyKA6@qo?7|qW2)GB>iap%i=ePGXLSjwfMV;Na->*Fm7Rp!Z}*eZbTPHQ27=QAGU z*IC;)RAbiyj*F^xf_uyu#NXGYd(@xwMf#QQ>r`V}%JdZ)ELy(xe24I~ySfGjeADv_ zDIFD0h;GQLj4YK0-Lhas^F8a2l>Mx`x%D+`;@oe;K*(&66_sDAn-WU1ps8`^-A^V> z2>yzJ-l#d~SEe_mj86z+@8mSO=~iMwVA`-Z)%poVes+PYfeY2hfKEb1QQbe;aY`@V zHW<&Vl5p3vk!Q_Cqw~APBmf=%ZnAy4Ka+qxzu5=YcDwlI0hOOL#c%@i{{D>J%yL&) zm_ZH-0i(FOmd!nWfD9NjtQjSshwobLHxy6oRZJ3Vr()4%ozB=xw7+tm;l6yUzA|YA zQyRGDaJu1OPL&pu;wi|-3O$gMN(cQ<1a*}mFBvUIqnPaCVC+?ZO5zfvDH{q99ZPEM zZw#-en9w3}FGC3a?&e5Q&`$LW5-|92;7hZV0D42tN)iIJ9tC%e;D9$6>3;-bZm^r^ z?-+8GxYz$`6N4eyEqXQF`OM4C4b3KHcGu=h8 zHeNyKQTa}Un!S*X!|~53(Mb~_cA+2pdI2Rw2lAH_fHGnsi;a&Q*1d~$!3tGko1*95 zceCWsia9gxyG#T)Fa?mRC*%gix=`#JPZ?})|2A@UvOaQfoQ5a5hY|#u_4-GN#!PBc zGo1K-pJ{G6-vLt1*@#?x@RB9RaVpbG@bYeEYcw*UF_? z);Estg2&~6;P*ro+^FzMS@7yZ&-TEH{OSad8?vhgoaEr0uA>&r`FaDS>ob74Gi|*N z2kX-p#jrhUgYP897#;6KzKtfM(qo}{#s}XYmQGd9{=19Jd@F)ENJ$S);HV<+O3i)pL^}>pPz3Zd)hCe+l0xw>dT27`bO}hnZDHM_TK-Q^MbeddzRKP3|ALd~1|1V!J+9UFT?ZCCDNl zRLfHA1PYf8{CDoW6UQ3p=2cdyh3aqiB#%`~7P%YQn|v9mFLksDuy^2{AeuEfy0O70 zcYmKUA^@;a#IN;*j#N;>xc?V=b)Y@lw=VSK)Kev57Xa6g z_ORb3dU$R+*(OM!RhF~HxIe?Vl!O~sAT-2y z+n4=H{r<10z5t~r`Pq8;j+sC@B4M+(Sw`69=!?(m#J}!K4(&}IGx#EB-FaDWVN`X% z*anfJ${=@T18+r-32Q!9CxbvE;pzi4N#5Cgdyi7mu6uzr5{Hd@;1ekF9QPhm*VF&f z!VOC~3g1HdLUAjdxE6V7=%8%fSypm-qyzfz=UE7>c6UnEj4Btjh<-Ttdl=ndz9HWU zJLLSB?m1@7*<-K;0VQiz^;QH|pkT^|gT4`ruUeaLX2AjX;}zE$&IFwTEA`n>wyWL- zycA`ny>?l{n-U>vM6dI|=viI;z8-CFt&ReBp)Mu3(s7J!R%k=6HY`s z46{Tx>8;&mrH*J~`4P4( zS5US~rR{*2G2WkQuM=vFl}3u0Hsme&#-BuO=JTw`dwryOBFN$Z?hbDPRrtcOwCh%d zfD9ui4f5u;8@C}fa9fi`tH2sHd606>x+-y_!QZ^j@9^>wR_z~_welfO1FYef;&2fT z<;In3{o`Np&QWS)D`0foVc$^p?Qjsb#rE%`*RgHH{X`vmS)38rvk+dpa5o28zs0t_ z*cQb>v(y6Hqqc=i3>oB($q6o7YDj!!YhP5qr^*A9vILhI2i%`irKI02j0#1o#hw2sda)vc&Hp))5m!xi zPShQmFPhwq`t$s!TN@TP-vosHw0qY+f=VS6OzlOl?egu$K^awJRyY;v6b(5dqaB9ixugI;*?eSl}p z!`RE)!x9UXQO=Mh)XBM!)CNR}N71P+!o@P>N+h3iIx4m$tl4j|cVg*tl=OP%X-VhT zv?C)?M}ZbwMKiWoj&4pn2^ z06xd^R-)lAD2uaeRcX*t`7XgkzMSOk=#Qrwje6z*t5s{Kt*p7*GwG1-=A1?mO1rkuuz*!0 zcfC68;&NN^wGNm0aT3te`~JzRw=WaGiT1c;X4qS>MnIY0OXDrlF5;bR#FResR_Nkp zgQiZPqA#Xq=w5Xc(e{1$$bbNdR9}0s3&9#l1?;j%n7>;hjF+qGCvjmjR)5HBHh>pn z!bj!$Kq72-ctc#}5wGF)&3VkeO3LLbE4#-~6vk~P#);99mEy4eN(cIheh1b(a10X;|w<_h@OMSBxE_xGI`ghIP>Ko%c2-Csf`BMsL-%l(YfFvqjWV=0`s|MT8+^LYgm*NToNwo8%>6 zHmNVWV8b=D2IV>3$;VMq0nH{9lfZT`R3*HnkZYDPHxG^G9ax(fDyXI&D2?EaA(Z3v zbTI#jAobfQ>xuAhG2c)K@lSv6)gCe+SS_2SAk#tXq+`jq09iKQR4vttf#i#A&DsI8 z1nClEc86Fj*I;Pei_-b7$8)kYAgi3*O9^<3Y_Y_9#=GSIl2U_Nn0djtwlDFlseF@c zHJ0@<8!>mnZtU@!n`XPu+bH=1=X^WJZMaawK9K%x4s}QBPLSP?Fcjt!M5940AX05n zVve=7VfN4!fN=9SU!M?;P;&9`Qh=%+5P2J^i-l@8jpC zEB|AU7|XOiGiNLsiH#3_+c<+bO!rvqus$05=gG@r9dCS-ihI1$25#ua9|mn5u4^C0 z%?g*61P5-ZgAsmv$oN>if^HOE^hr`)I!D3C9lUD?WchPOKd;c(;!5Df%63HCyOQVW zlKMt(JZbGMQ}r#m6$*3AuRVE&@;90kVaA8(s08PWt_%O%vpZaLkJ9cjlQSB=FT6TS zw#Br$M8&n~-;IodM-;uf(XPGK*@&s(u|Er5HrxDG+0ofD;PhI_4&>A(W1x!GO&KE- z2NePowhD&tsI5Fjlw)c_`o_G9-_(}GL~T%Ghkrloa69&%+s~IxTe>pB*IL!u0?3dN z$NO=yw*kB|6D35sa^RR-bn_;4J<-1K<-oX^x$V);Bq_;Dh#Z6A(T`kjZO7!Q`Wy-4CppM2cZx8=ZE(DR-?{hdh9G{wbfYZcoUC~O zz$#clg(hS&YAQmP?NQrstgqz3<v_RVcVlfeQti<`i zlTrKS;+N7JLAC)g`alSnZA2_``{2HP4+-yj5(~BN)7(kvse%B|l=iL1=!)&&Sm(Te za!t({xd4OnjFHjiqOk~PG9_q*)Kd+Ul&+?Sh0=MeU?q`}u z$#&h^s*-#Flchi%C0f+3NfN>hQWkl$A{P1i;@--l*^!_IPR*Ml@~8W^SQ%$8^$Xh+>g`alC+t_AeuXXsqgz zFi;j!peMoBIGE0AG@K?6-Y=iYDL(a2HUvn`%06eZfXZJd0$7{h3(s{L2Rk3fh0U9q zM<_Lzj!LO_WF2of3BECVRdY?Hs7J{d=R$j*d{b(!m?G~T{-nBr@fJgD-0q3duTstk zQ)xoqt@mlkM-_g#XQJ8}sYIr>GyCwlHjr?h4{t|fj!Og;Jlb3?5@3L`guf{!?&E#%z@T9 z-$m{T147kpPF*CJdZUAK%pi@STJGtTs|`lKa4qqSZ@@(EkOi8gYRjpc)x=nA6TveJ z;yWJCd~xo##}QF8(x|^4Ek^D#m-bMfeGw4iq|c?Mt7A8seuiK(rwHGc(An65m|YiP z9w3<>FWnbjox6vd*`&wR-?M9?c$KuxUV%V~3M&N9c;SpVI?vbg$( zFuS3cVJp(cBg|xvS}jG1HyT-|$b+gR$Si&-vrLx0PaJf^!174CC1K${=N@u>AnHLSrg8%FtEk$-eQLeyS z^`>D_@8}CV`?2rSFS2_VkNsdNjg6+K#Zg5P#i_mh{Mn`b1Hldc`WCHcKQ6XFJa8}e z39H=#wobqdMe7_q$dVnDSQpomfGPGosskP@+`qUsD-&mMwSDo>vx*YkUbp$zX6_96 z;kTaEXZjh&x@Dh%1Fu*Ntm65ck%<*4KLEtX=Y@ls_JO1wz&VQ=F|s+eVGzSb`>?=+6AB*sV3{XaG8 zBpHWiX1=Yv`t#vH()p5fV=(px&m_RWb}ieP1w0ovG2vn-`xJB?D1G_uZS6^j?Wq=m z(j$-O=aTOCGG<#lKhYtrD-aw$tzaHp!++!s^g zM1pA@>~aL(nFV?|NUhe63IJo>x}G~58c*6MpChg^AuGLh^*+Sfk>7>t*cxZN{ULun zy+ZDvGv=?3Vis&pEst}<25Lezf}K`Y-h7kgexVVVM$IL4{PW(#E>}SD^-!OX{aEQP@go|g*{*MMYTYwudpbG2=t=i=qPi9(-+uWf8Sv$PM_mCR_1m)!8% z#{iVwN+vd|$s76nsZ_KpZ3?rdo(c20oow*Hb?$`v%nl+ub#a@%sJ<;u2XUL5MapA#G0~J%VUv1 zieKzj3&taLZU2_=1yF25YYD%SxFd?e=&PRCnmT?&>Ceh#qxX`xXTvB+dKB{{c|aS! zVgphUSQDM*EA7Hq>5+3TcE5Qy6AkKt%0_1?Viy7+5g7%L>V2pBdmPkpGR5`& zl31uH#-^9wiuJ?Mm^kg1Xutyo2TJSXsxQl0i|LoRdH zMvy=4{ws=6uq@+sOYBp7Tdg&eC}CxPrW#gwzY2;wZMr>=tN239LEl3*W4x2Q zFv8PwC_3e)j4nzQWeQMOcMV&&14sNZJQ7!rrn2L;c7}1!b2}R#htLt|J5VbnUxkb= zW6eOoob7>Pzw0yDuokiy<;8he!X#jBbGk=2JE9wbh<$IrpG|n_Dmk?k8I85UEgXbx zihG~t6-WGACl^HxWzAXX#iFOVr_M{4{#QdOlJAJDe&+U9;(psXWqf$1!9T=Irn%@_ z$4@@5e(vqKZN-9#u-V;vu^z4Q7~vMnY(4lOo*5s<8eiHkKeSj($7pOOG?{qDdc**p zwCim8tv%aGqfpM(*LEadjB3&5-}>-s4|M@^_x--lp6%ekRK)Dvhn*H(V_r;n33m&a zb9W-Z>|E1u{8wJDkk*0#C~du(PsXH^48R|j;OZ8*XMJGpZ3vNtS8ERS9lb5&y?yeIw zovX^{ZK}?Vk?_7}D__Wv{Fc{Q4HMU}`HeL^gw8B z#Vl&yc}Ghp`Hk~g2~OXst{0G=z4ZK^095j;ALU%K1$fENQ*;I4Vxp``5O=tnTBdd9 zuk8K_d#xyZY7TU!$hB~{Y&G^;+it^M$Add25h^1&O`40%hPcx50eP6T8SdUo5-SX} z+fAJRi8~bI-*jGR+s+8NFrf|##n+*AP;U0h-lx1FWJ9g1-2nb{5o4`%1KXgh zv>p>Y)VLFny8uWHUbhgwDBJ*oXIwMj88#xEsu&C(SFeyx0lKS}-43|Cd2 z4&cw51c(x*C_w@9Oif4J!%96)Ja_cO`!K_X1;2V{yJ&zUzjfVKWC%0_>lwdv?>#lx z;BG#pdh-rZXFLSg0M!W0UhH1fFS#dF+!atF(X&49StnJRvWw%i%&w1H)$ng^zH07A z`E~-WwyJ7>VI@%+5QhDxv)wXfRt=$SGM4LAvvkbWn(88no@}th-rOkPCpNW(^Lj7J zH~9t-!xNaUvHi`(l}i?sA^1=P-I|&;sKE2pwa3}qv9*(f<06p=gkf8OutcZz6EgNV}4S^W3Tn5b^5q?2DIsCeWkjho|1NtiPSj#ywrQ$NvR3MmdcA zcsqJzM|aM>ENJMIm-8Q)VamkP)}mnas%^tylH9y8e>4elszKnsQR=6-51p3x#kG)P z^y~Q-OZ>;*z7~ONryglcwwkTZ-?NKu^BZB!pW0Li#C@C&H}c%O*M?_5rN1IJu`I5v zs<^b1Gk*f38h5MR?P>&-lkcO>Al#no1zsO-)HN1Z+&gNbgFTGRqbDwFaPG8V0E1Na zVd6pjilSXO`P_28=-6ZP0?EOZ=z!y$5spl=VcIv^nYH?9dhP@6Z0_KL7`tfDv(DfM z^&gVAx&rihD;I3B0wyU8u)Xcs!A}Z{*ecl<%2vgH^jR?p_3LF(k{`w#+MJr>t)jIl z|LkbbI%k3Ye5PmQ82{>Tk@>qJ0-PN)@tKV;S6-vz>O@bR9+I1~ve5ZCcG!ov_*a*C zpn>kAb>05w`~5>!o5J|ZlJt~mh4+{N1a|U-pG#Rq|AC#{Z;8!CxYR!BF4)G`K~ns8 z)bH0Nw*D=cU82TvPf8m1>06!1__F<9jisny&R;ubZqG(r`Thtv)ZY0H63g+kgigoY zNqeTf820x!ZP5Tey?D5_7i|{D3Iz@n@RQdi^g>~=v*O_kLuu>b+MJ_@5_8HKebEVn z#}mW^5*ZR*T3xdbuNTHF#BKoYS4%~Oi5gG5ykoj}p8G-v`R5yQ8?nzi0JE}s5S#s; z;EDd{hf)LZPyg_b)6$(WUxyvU=Q{XD{Pbf6Axv<=Slu%!9vSH!^HFcuVSOaB^G9u56A<9hx_lNt%6-VEmsn;J@xu_YPahv zFrZ%Ot9?8)9CiMsBset2();v~<2(C!7>IILp-J$(%})_q<%Qh8nS6GNUL^0&U7Az3 z7>n1&O^!MzQE`W;Bf*fAN9%se&g9;vv!n`KLoGe@(DB_zvoj!eD)iu5kJelu5X}> zO+I?yYzuIc2s#Ay;=1wo4O*lp5rPRe+iT_TaL#YZ)*Xr1K$~#NF~(ja@02;%LWK5e z^%K{hlqV{WYOEVN?!}!5p^^yKA5IONrQtTuUlo^w1e3g+ya8a6YM@0w^L7<8)Se{p z;jOE7jXiIZZj97Axcd2do8b}OSH4c*o9b+cC24J%c%KyLUK3)w4){w?$}$I+(M~5$ z&~3pjeNg4@O8e?aifENbm?%ag8G!{&fR3GyrB(Q(Tb-BY+||IUbJjuAb#OHxpKeR-h$vN(|8CL7Gm{B0pj5;N}sjSF@&XQAKcyy)~ZBG0Nk~;j z5&0Ul!umjvdUpgVuGAd4o@CWFW71GRb0)aO&-|9}s06Qh$78-+gGnja%deT(au?wU zZt`#35=36%8Fo*B2~oW`vNc1O*Hvsacy@{c+z^2xh2tXO)n%Q^?;|FrPV9qBS6J>8 z0G*M?)j9phII1v+RzA*az4Aj(cRN7k>@GP59v7f7cJUFC$%Soybdm^yZ8R;lEai7O z5kKP6)ds^{sE>tW`&{_rHblg9te$kdj6f*A|LaldS(Lj7(Sj)LjuY zqHpsTKL3FH<9Wuc!W(flvAqk3jQQhx z2@w&Wg}vd0H|5%lY&5Jp_!P+X+r9q?{@H?jy_mQAV4>~4@Q1n>%MMg}LIm-bam9hG zK>f{mJ)=ka`4N}PSz0V{`HT0j2E*%Rgg(B1jy?PM=so8QFc!3L`_*bm7U+rGz@NYP z$N0FZCyS^@abN+sa;sJ@@rujCe(wDJ)6RWXD7#9!SAnY}Fy#xn`{~P(tEQ4{@Cx=<3^=)p@k&uM&vDXtVAy;&MX2fNX7$F~r3~WD%tn$OD z4VJ;sT*E8==3d#KMk-PUiSvRqQ5FF_{!j26+ZJ2-!mQpJ^<*=mIn)^geXi|wvV^rz z*5r+CfwZ4qxBC08e~SL>jF_&GC!zHdHrqW{+cQYA>l{{G1-m}j$Yi!SPb*)uD&*bU zfdATMX9Q0fi6_K3AK*plaW`LYXMMkUtTpUWau-Z90%8oX$cGeJqK65!Ggp7>g0Vhh zFIH%dC7R7u@Aaj%&epMd3Xf!RnBnnUZ8Q#~DvPKzXyQqh(SojCw)&&ql0yNhvzZaG zHXOR@SWron?jPZtulRcg7hc&5cxU2?r#XY5GweX@p3wRDawoTF+w)hty~%Q<1HPea z2aEoFD+o||5rUw0B7Bj?bMKhL_|n>WcMf%x)f2f%P_$#bM>YClx z^EkZz$a^w+=|?0sUoiGvRYrpp(Xss}f==i?25X+I>l`~O9Yt(w-ekpfCyt#TJHYF= zIs~r;614wu_WVj^+(Mc5)qh1H*XX$!aKC!myL|J&-OZ4-b8T>Ua!r)+uO1@3Vh~CU z+U?UyXG2z2Os1wwNaM|S-TW9Pp3E!m_-}#Fg9d$_R#%(1mIfN2FW9mlPI-zhV)&zo zl%)JO$VbZGA)Y@L=q5*C3XdxANyo}fO(MYk`HDN3N~wtU90F7^iIPnYAMPp`;810` zN(fKKXnDTz5$;&n^_#I(`5*kP=>y-DIIC^3zKuzt?dnT8xTDCdv2W^KJ0Bp-?`E6( zzXfPbYZhKC{kZ+l!1oP4gX@6T&Hiz#L9QNb8`Ni5FAGLO;(+oSS2ChPJdOAR4No{0 zf>5=QfkP-RIBp!@&R2)(Pf+Qv@^-@}E_VUnM#ekPF}}&(#13GO21%mDm>fEd-81Q! zX7_^T@~aH|h$sEIGso+{QnW+Giv~$uYwqD#f$dJlA*f!82KP!Tewdy+yH(+>HteSY!PmgXnhvpck1ql$uXe)+rCh4A>V9W{|2R0+@krCoXcee@08DC zbLam@i|{=-jGdYr^)Tbkjbc*T3Y&fpFFUl}I%qI#OK=?^9_H*E^^3W8eui7z{OYE+@y+itKxAcjH5PA+1O)cgXucGIz<6kTtIX=6hxa=7-dwaKV z7GycU3w(q$RpJZF0rkmTJ^Flh1bzN1_tixF`%LcNHkKiGU3O!QeqAI-LFv zknXxU90-@bomz2tVnNrm1>aU>%l5PiNflB?2v^XcRknw1qm4gCjyIj!Q~rD>F#rv+ z9!hcpN2?y@yCSkP!+~!N!-;Zsj{V?U!xOWMLP2?8RR=K7+TS$F?jPrdV1RuKU}gL^ zy}DxS6a_i_%%gw#8d%}n)xngcskO2v9`}$RM!yNq(B`33!hwQfpv0HdILf3SuY4f% zVdKrJ7max3%U@neJPRU@blnl=?p}qKkhngw9hEtcqXZdgcl$TC0a_6b4ce@#+O8;j z>s`f$15-Xswlg}|;}D%2PJAHbog}K-UOZVQaMCqzN5m4Bnn_lytA#dju=#IFPQE`; z&@pwO@RTo>z6LU~X2jmpQwmYjCE2(=PwI?qYa`AHLclm*sD}D}%Rl)_8jo~^P@GL9 z&G*;x@5&`@L4-0E4O7e%hXC_0^_4Eufik$@v1Wu_O?OopE%o(KGMH@>ZDp3?aocRW z|K`-1yUUMjlltjt4rYP_wmxAfP{X|Sfs@_vqrH_d^Y+D~n7A$Ree>M*7bB-iv)_KZ zZ{}>aIK?bg##@9=e##E@_MOXlH{_i0AlbKT6!aa!POc4Vl>K!bV4EN1KHPSra!~|w zF+v@K5T;3ct&Pm7q`2XfV!4RnwnRYkb0#oYZj`3|yAfFkc-Q z$%20G-#OTH>Grx8ExH<@B+55|cKtnS9|4UIFo$phB-q{SH%WNBMzBsJ0mkb{vYN8W z@BqcHPD5TG1Lmup&qb$h8|E0!hKLfjH4m<{*Vc-Ll^|t%`Hs^6+!2eKfft$NP~m_g zqmH4`@U)7sqNb<_kLOyi-E{){^MsB|Y|?N$-B#4!zoV~^HwR%XGm zby`OjldL!@dgA^EN{p(ff=3&Bcc-XL`u8M*ZsWdS&Lh+UC?Ru`;8$GIH(_iw`McnQ zW0akmnd&K)8>xqGlAJk8#<3pG5CE4RVAa^A`p;SDffzo1xOvp1e(c_YcK?Xd(ExEw z9N!@JI4UNmBdYy&y#8$K2i>2s==TICL80V(!XF*#0QcVbc5*WTVm&L4G^WwbE4r3J z?()gR=Aw2{fXH<(k9f<-LrD@1 z!-A4KJ2Ts!>6z)h47pmg-q7qH&nZT!6CW(p3>u{JkS_KC%7Ct8x|&-lJD(3Qm0g>=uM!p<@@a*tl$scbjySN4}j|EU?L)9RSHUy3RoMH-36hjm$xm9 z##+!efY!|6#V5S-gTV@Zh($7TWL#DdGC?^U9f`k)J$+a5U#&MQR(ks<>H24hLn5j7 zx3kjUNgtIy_T6`i$)dOay`)f+miT{jS5ON+d;InRGj%5&qrYFASa|wJ5B%!pMq2T@ z2+kNzo%M@lZjH7r@7`Fq6Ye#Lugd~vT_qMZ1+y`f+VNgZ{LY1FzqL-dIOiMDfq3YX ziOjEw_U4;-<5#bz&3C>|Zn7wqZ>!Em&TpLc^Krx|d3&@HY7A=LPuN#A%;`Q#5)adf zaZa#`t8IIthJU?%Di9-Aav|ZutkImw++fevC%3h=ri~Agdkt~8l72bWPY~4_>^2I_ zSCi^9@O^?K(EC_Bi89fjveVW~jI4Pp>fX)XSL&P}1#2iL9;&UwTf#hd!ZRz=6BZH# z$u(YX!ZvJnZw{tsgP2I24w_GE+VNi~@}#c5tCl2XzsqU$z1QR(;`BkyJCA-3CaVY- zzgF(2{W7O)Im*lOLej$~$n&cYHVR3AbuX>mzISC7iTGlB&XY^pp0DBRZ|J@Un>%RI zK}-Ms%UB-hvV8&Spx7zstU0X>#7L9Kwd|wN`f?I_3LtL#hKn9@N~$jp>hK5tDQaSR zpvL0@FttJ}-n{y%)* z=p78<_Fx*nE<~3KkI<>(-x$PppAO7S?t8;ivRmrWt^Wwr_6)3kdLh&2<%J}lgI9a6 zx9(0z`OUco0YA|Vo0GyQZ@hbC@ymzahZ z@x7|!%ji;3QFK-e`o5WaG$8_ldpiYvYJU_wxOJk3Ue7cf!y3POZ5z8=WN|c-fF?O-En4H~cC1|4=-HqhVoSvs5G_)MGgjjh zAnmO2T;ZB1PzCjj4d;vBp(f6EwJ8B8Y23|;nOM@nL{5r-Q1H+w_kgsFYq>rJ^(pzY zVpNm;9-YKW=7eFK7FXLbGd4|2GqtX<@563}g!yW$8T;O9uf~ehZ5|a)xK!?{8i#~n zT%R)g0X~sNGt5b-*(SgA1Zr%xgqFB)z!+P{$I8{L*yT6_*uHSl-l?Gd1>l@ zO{9~-19`?GfP)0}JL~0x@lkWXr%$EXyFe{( z_94Ek8U-g9aK8<{HvAbxE;D-)8sm|j|22Ea=^)oJPxJ?M+~=<_l}$qjoCoNy4r=J^ zrxKg|v-Eqx?v`t-&i!rY;$0$uu>ERCH3`)QDMBb*&uT*ka1tiF^5Z6s<KZB}JAlk5P&6B{qe)-!mrad+!eN>?XwXuBoOzw29c4j20rUPW4N zgWci;?UNWahb|o6-|#LsNc#nbYHIfMvARn^2PL3f>)F;^cW6Ao+#8oYYw8FHYV!-$ zP?YTHoO%+p8YH7_fVJH6!0FztWbLgr^;NX^+ib;@MP!KyZqQwJosh)g1ZS`MM=}dL zt3=*uN<_u>xK1EuoO1=+Q234}DS0=r77bQQ{=Mc%eO`Ye!=tQ$F%GBQ0{uM#e!IqD zLRz;OqkT=E51DvUh}(EVKW&e3wK6m}nY0SFd$&5m_6F+M8H~^r0HN618V zp~$E&;gWrSGf;3Tsz&wAy6RRKmILocwHGNy+ApG>O%9%_B9MSF{5;&&FD(3F@}RjZ z53kWJ?~1c325n&MmAqgN{sHoQNI6(N*w}whuZ-ln)GIy4Tt{RKY&GAzwXF z%LSa6yX#g_z~R!v?cjXpQ%XOa{dM{H&PP^C|LBxkKuQKq7IGRnE+aj0d(D-!W)T~3 zYsH)1_>MRlSn!Xq<;hgb)=Mhxae8R2=x6=Jq9J2<)aBdP9f=nJ%2U>~hZe2?*-c*8 z9s+j|UmTl;zMoO$5M2A$qSRjZxIta!Fgyrk3ls)H8EalH)iX zP;y^OFwt*dTF`Ldxdp75A#0K+Syp~mCXJ=b#yy~e58vG6cNjS8Ibs@KxoC4WTyq7U)a(B?Hz2RUz{`p$S<8S<%rM0}i)#7tSk@l)f*BpYM z^lOR>Ym3(wKXccgh4~bG^V8GOCY&KVcS_$P2z%M?F(BotXXtc{^I>ZVIjl+7BUiaTs*U}`7c=bAYu6R% z4Bl~9Xp85Kq^I44jXT9rd|-@@nY7wC^Vh?zC0HS4Gsf4Zy*V>#!ESXuXn)$Ys>@I< zw}6+cTID5_;`k`SGAHn*k`qHqR7zqA>u+$k(> z1QQGjS~!TtL4|(9jAi#v7yPD?sgFI2L0b=W90GVi{?yed&wbuhU+4|l%krMCcL1xe z$0qLbz@{q9F>t$|`Bu9J+j5KvH`8V9<@KaV``15euG7|$nMq#XC&??E)DHIQ?F2t+ zVhw-k@F5}BT`Mz@(ec`CpE2*QV5=@0ayT6j4xY1=&PSPz7ih6&YY3Xz4;Ez2#EtT$ zMO5+HK(7j;Nhz?2=v%tPO)cMpAz6kpzb$Q)``=UMENsn`7-mE=Fl%K9^9Z?QzY5`7 z-Nv@v$;mH&uA0Z^l+m)eKD@lXH7XYv-!TC!GMb|~Q+T~Tt+jZ3*9=wNqc`}aq{hl< z%r)qpa{$jfZ(~pI=#5_4(Ssts|FmXCfR*se8Jmi!EASxm$ zB9*C$Qx4#OLY9JpNU3Pzbb0T&|F!OibJzVSi}m|}zqQx1_p_h9_5D)pi7b3&(!4CB5 zA}4P$#Mk<~Xa2y?(nEzGXL_@AI_gwfSim0KS>OI}?<4uuCAH^;YLnO<6~w{G*YPfo zqSDo?ejC*{1cl-|X~24|eT_gTMCx0jU-@_)W17%*oImFVo*#LEB3dEbV~M|M=%DWx zT4iiy{UVj);q5t`3bCMC6b>~t#WZg+io@yUF{tSL157~}PSZd2*Aflgq=ueb^^cyN z6o#9b41g8hT%=ml>a8h%_B4=Q?Wi`C6!kl-IWYQ+*D5L3C!6Rwrw)VG7$)p(qlw$$ zG3h@D(Y=9y(jdycFSWWdErlT%`M@&9Gv*{IgMfH74~P}}@f$)5L(c5~ec#sdsYd$W zC9WoE7LI1yl-hx_WEsW}k>DH4f>zX}Qt^kdi zq{1q|q)BN!tlb<2&h6f1BITBzOz6MW}N<(j6*7r&FYH5JN`kl+5yRjg`$z}A!DmstW@uT zqRSb3?>+uB5~DTr6&PpVjOWGEZ?T-Ccn))mU7s(%( z#|7@x><}h&s%MrpzQ0R!>zP;AvzWZ1y}1ioWC}{twyee&Dv~ljl>!r`WCNN^sM1dp z^v3n$FZ%0y6&q_J5~I}i{(HPEror_ex;G{1*~~j8`=AmxD-p%Y|7A2wpZtG%HoF~` z&XtX`wct_nT)872XOm7OpL0I`q~|hM0&=T)dqW~oB6laY#8;^H^?J9JP+mCHf8EBE zN}m3-^g}D^SIKZ@i5Ya_Z=he830GpTR7YGS2z!5+ZToJPVCxRsZ8m-bg?ioD{a92e z1TN%~R@Bt1TqiA8DZK+ki{WB#+iJa9H}-b%C!IZ)`qxm3Od?bXea8g9-AB)tq+^vG zr%A$c<{)rx&0}7(=frl+*cux(@Q^xGv<4Y`>$q1}%&VHpUs`B2r3TmTxY^DFijnY` zaq!V@cG@`KjBq3Rn|COTWQ$nwzIE|RnU8OxvQvGMRPZAC>Dm|aHjs*ryxUZ?CTwW& za_5}VQP0Ysp(HvZ>b6BYyum#8UQFspEt1B~H~Erh)V?X4Bb$GGCZ4L1s@6KhcToD4V9m?gts*aH)b;7x`N|^xL*loU%{ja? z-aMwqlao9_uLxOZpg{OM2t}S z>w``DAWskgI62ob^6wQ_<~>~(O#NEQDjIw`V2Pnw$98%Cq{MlvHtA+-+r=DdICtADM1k#hIc#UJM*=dDBS$Y+c=}8Q^q1ZI^gU8_9PL zQ4&dWvpY*DMNw$?_Dv`})a*rwmnE@g5WgR8;xQB?3-RLnSOnnadr3QI1DuAj>!IuZ zZgs?y{rOn>7IsncJ}FUKQbrtZl+ptGhPFJqJT9lMdedFqe39I1AD6*W1>eI@U3lT@ zxLCK@bslnVCG1>-);;}IPso&1jeB82e!j<{g>f*P*>e(arg`9~d9*JJgP`0WaQ#{ElgwhCM)12n8`FAWvkamP3BPJQVp zmWdN`>XA9vW#iPkql%8!$=cMQgs^`V1zHP8oAw2}v zYI0rVCe!)`kDH_)XYHSq-5U@*1bGoP>Y?fkT=z81%QT&{kBG9})ESV%3~UQdqQHUS z4Kwa=UC(KZw><~3K$)qQ=pUI#chJr2c4rYb0_~08wWhmFQDpg=!0dH~i9tcC{;6@i zK-W_v4-PF&Xj$sBq%*7(IGFk6Aft|%>WS2beHDkFnD`4BRr19C_4Ho=FO-T^pGj9| zQi%33w#1Im76{4rB!4v$|5l8;(z9g?^NXv7fSB6P5*#9}B^_&z1nYhT7j4?PkA>$F zg85%#Gxaz57LW4-hr?2n!8_m*61Fqr3n`4NqH)(M75>jKRxdGoOdXT$6<6m5;v`TNx_7R{^HAzb%G8LH+H#k#8PjdN^H z(^SWgDemiDrAgH2J{r`fu;dVFf3X6adP6HLw;wd=wG)=LHDg=p9U`!{d%pS_YB+Fw00sBN8NH@a zU#fuchGdem<*d^H@Jh4?jn|-;9lXKNSGrDj6w>c4f@Dg%Qp#;bzFJ{(=Eud-duQK_V4L>^9xead>}Q zIeIHR2qwD`m@7>R6-H(fM0dNgy4^M$7PS0y<1vR*ZzGWOg7j1TZ_KtS0oY?y-=pr$%*l2>od@`@9Cyp-sPt)Zv$9NX3vJ+_ zljDSVTD2E)C}`aaMy-FUREhxpOt{);M-<-S%?Dl*M(|)#h%?6|!8wnvRz)-gN+BrC z(H>dR++!A1H$p@m^mITS9lgsjcZw$%8@q;qdUtC`^_KK)&QPIzD2yBGfb04uP^rfO zqJE;%t&v&|POt%7wVYD%>{Ea9-eo zjbl9Sdqe_^s>v31LZT)6j9_Y*H)iW$h0Q5?2p4ju;RgekRImoB*wUjNQ#a8g=dqsj z45nM9G4X2SOJh%C1r)zfHj?pLaz7@vNfOnw?B>`UBgpS&m+y8uTqhrWIkvxu;fWWk zOaiPtD21Y;Wb{>ByFC*-dCi=_>lGV49_3nJskR}yAp!xK^9)4F1%Ujn8DSA56hEW1 z93T*VtC~g{B3xT|G{sAQ>BdbhnspTvUEG{1UB$4QpE!oc# z?uhVPr#=MD20F0Ydu7(%=N;2{^<(qx|7y)+2ln_oaeJGTQtF0dNnAqca=hW4=R+~M z0cH8-rA`jW_RiIh6tc?X_rb{d)~jq1drP(iP1QojdT2ahsUwm;gVJebEvC^`br4IF?Dcs1|F8ssP3;PI^y^%J;=4HUSsOUmu%Sg`J!3 zqv)|`O*UKFfI2(hk~_3St0QYzVwtqXrsR>R8`Qe4w!Uj6D7OmH!0MAY6JT!l>tjCV z{Ct}npIIQCq{YeU#+p^FjrKOXgjziV%`R*7hbiEpQmn5; z{~IuFd0IbiAJPx0+BomPj1N!+2gI+q!$T4=r5@loX#G=i|GjzhN$2qNg7OZ_TreRh zM(0U=-gllhCn*8aRL+g5TFa8X``Gy$iy0+-7k;nPwywBzweYT+m5>O+x5n^~Uwhci zNiHrRb!po)PCX0d>wi&Et1$zV9H2}hawu&+GsduHONPF6GB{z8L_6nCg4K*iQ#N`2 z-scQrbAp46vfS81SEwS_d1~2X-s$(!1MBB#?1kSc#v3ViT7`F;Z%Qi900V8z4$lrC z!#JlX& z_j3jba@tSds`HtZeFuFGF-wtnh&uPLnS?^c=pNq0NsrRwt(dZeyYMEvXc_r`;BG+Ft=l|ioP-hsEQlC^_Nu$`UT%Y zl^c%2At6DNnvhVVrMe>3tB#j{R^6RM8925lnmMHuG=pOcdeyZcQ@Eif>YX#Uhxo^b z<5!9h#MXtjgcR=lUPdoTJYr}oZ6MhTKQTaoKZ#ZR&FAmf2A07N46q zx7knYWex!|@0e|_w@xD#j{ayT>~2DzQ_1F^0h7Om8=D_p=$rKneA7uFm&UtW%CAx8 z)Sm341uyuP2{pzdwasQD19hT(Xe>+?a9Q3Up`Own)1%5gXx}PDmd`pm=uW9^^v8?w zhk>jTCpjhIG(TY;P}|%zWd@QDP}ibgg?l43xiph9okYzfAgic+9hV-yY!RxVK1A;< zPEaSemo3ruiEa8042j485cJXlbJaFz?5BZCxG-z$P+l1kk>F~@yD8b!R~3nR{K>=j zULCjFX?c0rOpf&BvKw%)e*Ajff4>$omNG<5KHbf{mIXnFI5go->VGT(O38X<_fs1 z?p`magt!esb?Y48fI7z_VWaA6^lEqCuYND4N!?&IzeexfFSbVjqIE^EQLf`6y37|e=>S-| zEHQbuPOvnaX|In0I_L4m37O7-GSR1ALgRuzC6PjxoD`X?H5q+!0e6j%yca=BKdoVH zUEscbOmVJvnt{?DE;97>8<56NT^<$b^hG(6br_yEd%H`Ss$ef{c~JLP78i=oWl2hS zqNA5nWV62qdNh`d_u!n>$*H%ro?B8qR2%|-KHt5hK+3ma=WsVb*LTJ&$G5BS3V+&B zqutl_I>mB(qSuLF?^YqB$IvnljwAKO@*fu~=e4HU(UTm9O)KNESul-K_4qS-CrF|V z)Xq7BbQ8cuNt1Dt*(WyLg&M(8v$VdE(DTeI*7V7=Uce$T+oG7lJ7-XDX4Y0VtV%CK z$5u2;BMRLNdf@BCGmV-Az2-wdEGHYzM?(^?fX4Ex*RC0c5~-Wka7~UD4#0Yjh+*fw z4kyJoN5AphGz0K`i)CwuaFEDjK~&*~iZ{eeq(ex*!C?RU))ULNi=@H6b85fVX~F%D zPqkT9JZY(~Dtf8#t}V4?*=N`HQ-{Mu(NFGA2LD6Ke<{N$d9P894*dRgZ78F*@$P6Q zfYpTS2$*5z1)PG#e`4=!1xPJs_qRID-#uswy4wcE@e09-ld9btNpLj7wB1G4T9lDU zw7w$UG{4qeY9srC8ry7jX2eMG;cUp$)m#dVYK z$|T(N*yRl1Hlk2V+n>bmCWrqh^C5KU1`varS2d0LuXi{4IJ6342Nt_M_QKYiXvYG; zDvk6S8C!lZ-&kFf@t;o8{%o5{)#{U)nX(mogZ+7c80yQteYk_(F~R@PM~0*%LxN8t zX8ESX5}Wof_$0|j&M3Nc__1His6FA)In&e(z10;*ONGb#AEmsX@B5ApfF$mr`q_P3 zTl~bJ`OWCPhTSiMjnxAKLVQL$MG`oIAag%z081Pl&H)i6j?0oljEg2$H`6Q8JH&6( z2u$hG>f9;yp@)?TiRZM4#h#iw8{2VoDE)0dUTkiH0W3#Lx;|e#QHxRr;UUn07)I$* zQlyZMKeAkSjrblNcY0)*4mHtGq~CP*HWj@)8fSZ5T;=s_EU-1|J>O89r@bj4R#zf<#+C(ux zBP}b_d;`AOdZny-ZB0$0`gRnzYadB3?WWT_Che}@%1P7)hZhzmiJsjhw=05CGo6553($)va5#6N5UDK@Q8(hL7qO0V#JF+nyMw9Gf znfkT}U#l^jtl+QxIl7oesm3f)mk;ibpMuL=#iH5{`1ILhK1(`vFrwH^Bb9`LQC(8D zrXp6{aS&v^Z&a|zcW&Ir+F9f};C9~8pfIo4_#mJ)9a!ZhJh6~>2wPDf6#RYlw)OT2 zbw1AeLG=4aQ*x07Qr2Xmtz*oLc}PF|T$Qm1OLOUWlhY@*zHWmmld_t~wIcPnmM~_B zUd#@^nsbduj+xg_0)}MaEk)bvG zZJPE@+@2RpVEZsNNj?Z|AiMe4`IglCB@wJ;XF0>LYcmmTH30p3kH+kdoxV$g4ir|l zXrG5S7(AuUbDhWKMP+&?e`h>}kE zXX@?4yY#{LUI=1>J%R=qQlodCkgIX>7F$hxl~pp4|;Oq=2C3&JLp~ArJsj< zu-hm8Eba!9))UZUCzhXO6?bK34e{cicRj0dO;Ar4cjQK%pfqhavvO3bQyrWH#>QSV z+^1hA2c6SonUI>2vk1mM&)_X}y*qk0)U7PK^6MzoJMSOuCagE(slt{ z7rdDyNK$RMalRD!g7K#)0T%FG$bO3}F&>(VZDs%9m?N)@wsY7&W14Vaol;K7>PG-f zN{jQ0T3!jW4@Ale*imZt^!O{XW`m$xz1(*W?=45V_K#9==n%nUMMya>M4~Y;0;G z@(c2`_19#JEia7DDtQzE3)s6Hq85CYac^Vhe(Y$!`%U$&fiqSd^`!qS1Sh=L`Ujsf zB>1!}E!)-qU|2#6|Na+zN|=;b_Ag1&L_A-f*7>t;FQ_RlSM4DO1>{#%N^l-vn>e@)QTFK`aWwTadtQ}qv4`aSLpZD7+`Vi z`2Lntbrjn;&Kj>3S({^8Jv4x)cvHS-74x|>1Qdo8Lvl>Cm@GqGT=6b-9?aNS%lbam z?CffB)U2jcUCpMHQERnPYiL}{>+LER`#6SO8&lD2Z8B_AuRctE1_6(aC7HVVx?72| zR9Wx({a>v|8>=;wMw#i;B|iKUee_GEVx*`xx^9)`RC`80uD)Yz6!sWW`o%`z&c+nc z3txa^&%jbcPvL=UYf#oXSFI>PsxM~$YOc>rzKtz+?1|a#Ah^ZJ#KQSeSk96qu|PpE zOEz6R0=DEWma|jcq1X^t@^LZ>Pg9GmNnZn3yop<9fRDIzuAE+NIy`W%cAEukajgx^ zql4$MB^ZU_SNdI>=`xCFRU0+T@Sj=}7ur$J&7wIRlcwW@TS59RftK`q z5T*8(&6zWrjL~aU`(4`RsZG&#Z~V~9tf%ThmFUV3Aee$n> z*oZ$BiqqPL;W)`!HaX=_o6B$mEZP|9m|r~{{Od3VWAOscKW1?7j3iBgkr(j;`bJ@t z_j=a}k;_cI(fKUP_1X$(J$Jl!qd7vTYBU9(h;R*(d7{I4s%#S;S#L-_bztj1=^M0w zZBI0!K$>FwYtw8Q^2O>{Dz>=m{w%mMIzt&|b0MdzjTdaT0YNKwlr^I}oFO9kuS)`K zm*RCo5lcGY!^yI0cnHD}m{_yU!p{7DE8I7}p@~qT3Yogv&@swyQ66LD%2}nyXc@*S z8p8@_lE0&#(l4up=uu-x+(|_1W%id6iL}&z-Rq_GN13IG6PIV$il5FWl;U7BZzfD{ ztb0BYO8gxIr>BfltTUzfntLq|Z%jyRTqnDyJ%#A&wXr>DnsQT9@9*34{Yk-V2ftT_ z+?JZ~SV(VKZZxz@av#JQVN1F!q~CbBnZSzuIH@L_UtYupP{DHbMp!4K!YLq0x$#gL zmkA$rY%5GWyWHE<9#AD4SFC?yWrmS)t9#xVT{g2TdkTAO4PTp{H8?U0Jd9G&V{_iJztujhmAsk_8 z!0w)p@A&m)A)enw%+7QwiCsz9OXi}vRXX!hrTZ-L=6kPW2Z!+4JKjYev68dpmFzu* zaQgSIgwxbW+^IqH)<~L?_nGMuFZGkQql9`VUaq`kGn+6RxdnG@jdkf$i3QvF#Kv)B z*$n@xbBUUr`iRUsnDLa}%N6@nv7U551)Q8bY+2H1!?sn6% zMapKSvb_5ICvzu-{^kT9^RS$-_7IY%doPhb*UPzkx5@@m8IjY`5rS@Za8DEj_?Z5n zO|+K*uk%B9d@OZXashOTWz)-o)c!2abNeg0YNY)JUAT?(0@XS(fbU7H){5O>A_n^R zaYx3(#iVte_XEoFUZ;p)fqsam<<#YD#lCcGIYzH=6?Px0dk1zTH2g(U6;ET{WENC7 z?c_-iw-_K6ti3JD%^4xEHt)OAbX(_1TJ`ooJ!IE@kZ0H`d}f`yZji+Ii3b)Q&$sY) zbicAF2vjsfa*#m+D?V}R`V4GH)i+e@=3Sz_aTT5|U@YARtJR4Zo75k)+RCST%yA9` zSKgL)x8rwxNTQiR^{%9M$0T zQX-!yxWtDS3|?^!2S=&IyDM&l?X=>WNKw_!+j+)Cy2UXC>=YHU26s)nEiPr}C3-5& z`w<2aZBo|kO0)`P%sA}X8-Fnh>*F$!7@tD(#vWuL^f)3C#0JzcK*3;=2XXJ3d<=|v z&ezVgljS9-$PVV`4Aif;fn%l?8jfPm5AHJS@$o4)`M1APPDmc0dZLdwd*@5C3&ANm z2Jb#tIL9=)Md5*9Ig=BV%ldgh^@`bS93f(qOjlGt<2-}EHVkX^1f5Y2FK(T2)_(U*?gM4hsWD$jj>v z-3692$a4uJVYjUbeZd=kIPOraC*aPxauO*p~6?7aPFn)sy^XTEM8Wxz-KW&aV^GbE0!2a=d_>e~j1W9#j|i0l9Ju2kL;bzIjo@92@geaH+ltQ*PT;l=nO*SxG1Rp8{$%U? zT>X2yo5HpSlD4ofi|I)bUs_|lCpXq*&Yus7PAtQ z7qJEzMdQ0eJ&{zNcb6}0tcNenWcE_fP~7szU8#88amTpCJ1&ZK_Kn9}=UFA345W>Eqgpmd7<3c?m+CMXzn19SA&){grP|?4GSWY3L%tnY2m2 z_Ep#){>RaEhW6CARm$U9DWGdsqU)uj$VKpyg!XS;ZhW#JIU^A zoA2<<=x{nn9xs78cb$_L59WM@Ztk0V2hFJWIq2$;{tbOo``Y$EGN*vo7y~*g@94+V zkVo_Z;l!y))Fa$7u9^PoZU=&1L7WG;1Ac<^itTo494*}N#vS(G+)5bV7}iLeDuzSd z#L`Tg4pGM3COYmuw3W>`lg>iN?=*YX^be?^Zw47|B3F7CL@ck5VfH%QMr|>ija&>FAm&&y4CwhZ!FE3MTGLP14(!H~ zE+*#d-z#IT9gg##!~8HWQ2xuzCcmnwIGjrVZ!-N3_xh6`17oJ1O(B}I55dp}+V58I zjf^MeyqFg(C!#&I@9Evg`#_oNM+k4Nu3K#q{f8eKl}0EVmQ98`V`_8`lirvb6JSWW zG6)e#-ly+f75tj@Z2{(R8KXmuj`06k>oo0CXdXIk^GYVBLvoF|@N?%A%H6M99sx?H z`wLfG;t~m#tKjrJ@*%5tkQtVHeevF{b!JiPwr7Q@K%;VWhN}^-eWwscpGv2rM~raM zO^wmwZX}`MLi$mMl%hSvsCs0TY<6T9{7hKU-4t+vJIresU$jGj_xBok^1fi5D(Z%K2(yCq-rRU+>1{p|hGnh1 z8o98AIMpVXxa&{!B#_x+e|{A%q9L*7WYI+v1p*lFd(pYkY&7%N!%E}mcE#EcMn1NL z(XVh-XeLA;F$ASS)@m$r$=!{Dy%(!mdq++2%72i2dTl?)7Bq`~V{~_`n4MBjq}K9^ z-R$<}FtbPkdT(%T8t1C7WGdSnu1iA?cBg2sqR6ASiHPg1)|pBtcG{-VJ+`b9P4+2k z6}%c{uIV&vWt%BeQ=y z&KDFL6jYj;4@Ay@R%r!^l&hP`S-_6niq+&Hv(;c^^bV}dr0m=MD<F`N!+`%YELb7UX-3|J?hJaj+Oc~+EtTD##ntF5sF+P3XiU3I@QP-6>cEs z+nWTfL~ewKDZ094qgOoT5!@~ej8Armt)E7*EoYc_OIh_%nsm=akqWH2kp+t=|ElUD z1bZ*Uc)J!%%_d(#J64Ttr978>V?6N)Yn~8QH|hcyG$w2M1%b^KknVskUEPLNW1&lZ zHmiR;QSzwULFdaER84w`;dIR!Mn}yVG4saNTj}x`kb+0ayQk^czCteJ5DcK@mrE8k zA+9D*Z=n6Ef=@1ryI-tS48);g&mU=5L5$<+-ALe3EBxfzh9@D{|Wc+o<~}u zW5_;Rx^HOUzh}lioYrmk6+N~M^_T=N2BG*7v70$Vjpa_>x~zFplk2{{XpFHRZ{Tur({9NWN3cB{6O;^7^XK(AyGSi zB4}Frrlz0sjra?8%_AgeOD;VQzLRqwGA?gyTAxG4j6oYk(tO$!^PK*=RLkM^lT*O_ zC0xIC9nR;%7bTqSyD%EhJzL+n5Nk{(RKux2iMcfOS0sAA+PH4~={>Zg#ryEgL?|ch zl;8SDbq?~I`x=4j31jmvYv1{Nrw>Ttd29(sBL&<%yA!QT`;pzyApS5tC#{S#Qgyor z+eB11cfwe$dyxy)BhTm$p}ngrk=Oo=Yf6&7qzWB$-TGuyJ{G)TfTU;fRaNqFX` zf908W;a~r^Ii&q#>~hW-TL_F8-f!T!?_G^%;}5Qyq_%KlTsNtdJG8YOOXae;8~tds znHI1^vlz&VVH{nsT7j2pkz>L&)OTq5I^pqiBy(ycM`~}y{1YDHa8~jr5Q@xK61BbD z*VTkt26nUk;W(rY`TF3=qFTc^yf~@J<%oKG4o9EMUUFcl6uo#dg7-yV)flU2{?VJy z+s==Pt^bvFlRY1Q<$8%ammf#LM2Ob2W_Z`}3>)<#H>J*A6ap9KbrL&96BXIm2UJ^v zEV7??Iu&Z#A+W&g0y2H57lFXaLPuTdqqr;ZP2J^!(!vAtm1HzV*&PN@AE`YxaE~6i zL-iS-P`Jm^YltNp7rdas^}#0iYNA6e)8^Bi0PW~i=2=X5p%f1RX`ux{CeAtzdKGDu zroV!CN8-QxmtZ-4o5ALv^hLc>AE&_vCx4jU^Ie5*qp@=6Mav#i77C zE=~Y*=T-P>bV)0xDZ=t4ZK-sR0N-=XWv|7-t2P!bP1zlW&MNic{5{kP1V*S=0GHxR zw6?f6qz&x{Ju0lt==8U<$tr6@y(uA;)@iX@)XPq7f1aV_Y_|2)iH2OW3CR6VZUWt zy`T67G@OCobv+j-&!`5?BBixV(L7gsTG`Vm z%#13R9n4qkFM^!l-`FH|HXW4(js;+}U^{bxrDAkK4SHN++ygBZ{I_TebhWq|o>c`& z(=Pb%$RWudwmO{x2VM^}7J^w`UF;!g!PjlAkgjI~0DE47)= zR@J$Eey0tMiuajqk|Mm@aep(h%xS|(5sMZixYOi(LBt~{-wL=Y};5i>HFDYd4#${RaupIKPlHxB^@}p-vIqMnMNBoCey8jRi3;Rcg1(-C?9F;Uk$}l z1wRx|DP()sp2&DO<8{=B+Dv9akmkvp^Ul29PE&rJAoPbzlm zzb5ii5Zi~xA!QM&#Ho+ZVm2qs7nAf_pEd?#8<4TP6>f0(??>WI2O^J9cZ6zCy+bC^ zY|dfkBRF3=yFPItZOWEn|C^FdDbD1Miz&ynqQb{dndQ`W_PvtbI}g@^mgxKb;Ue?p zjeoSU*AlgUOWYC~&d2-F$b=naaU-#s;CHq^7-fda(H zp@+JV`#a~d^?gNaYQgSDWacO_-ahq3?jTAN^%y2ci_#k@@(3&OanNysM<^|f*GpVs zPW29{+Xr@NphVwzQ9m<=vE`Lin=Z@j+Kq8=^Q|%}Q%jzT71A1tEHAk;&$g&y`;Dal zLSl0vp*hFjRgDrC-l`?60x0A}=(-hFd9mnW8$f`_30`qSf`oryr^DCJ>QaX8`Rr-S z6CWiSeqP+=ghw^H9nX6PFt=*n@YAV}PP74ybZ_+Epy-w9i=Z&{ zTq|}gCmT7O)APmcVb>=+ig~21MSa`(_q59D4UFYRHm$DFIn27={Z7CFuK_JJGc~+YU^?8X7!XV6?kN&2~;rF%tU3;$1aswPV`$AUHsIj zqA_T5Ld_*;VuI|h!6sKum900>)4Ok}3Fg*%QH=^&>g@}6A^GB`&X`>K7UMMNR49d5 z?qA<*S60weQXAO+kv5tZM)Vba`;djNdkmTTVl`=U*+(ExFzkP5QEvEP^3V^QVc`yB zr+)nvBc-f5>_pw=)s9$#E;VlH9M-_L$EUk)H!f&mN0goZz%Wk+%`2Z)Ac+>B_I=H( zL}(piY6tG_6cA=1zFoa4*@WFMQ1I(q`fxRAF-RCDm^*njv3T}Ij|0JA;60SVYB%yK z7>r%2bzBGjgl?5~Q4zM?SSsvyvdP31Vmg-R&llThZs|E|^v+_SPQFH8!{jc;^0%*V zO|M|Tv(LRY5~O{AQ1OrXIEt7KAQ*P@d_6zgMDoid_1-gHVU84QqsaRmh%v>pUQ1EO zJnHGiSvleY!<1Rn8tqz|jFypi*8t3}x&{-(&aklap|ep@vMb2C=(WIrtht8hw>IWBSf#s`6CI z-f&B8MPuXePzPQw|8|Ws$s{@QXjO1V-cmE{0^nlpSzm{trOJ+>rhX@My9s8Z=otSL zM}wO+*jVggR3E;_DWZa8!RaBh4E^<&%(V^!k?+RgiMC~1k^lABZ-C5+whOR0Lf+n? zmV;~%#$1%Ot*yo=)e6}h$F?flX~EftLa>CdG-UsddVWNKvo0i^;- z{d4y**#*k>g49Xb2meYC|4)7_(%-RcRPZs?XlLT>PY?N-Z=Y*CLn!M>3i@_M5`s7& z+!Su)qk#ZRiFOA>QZ`Y^oXZFPOg zow&QZbIrzj$ek{Vq&%)*$ojlC80PZ{VPsmS!R_=GPdwSX&e9S}qo_H1z2h`6I82PK z(hL~KFqfex={>x3C^??B)HWH-Y3|^26Es72-kp$mAyN;~rZ#A{(LRelOE`P2P+?ud z6Q#KKW;(^RiVdoxe@hN#_D5{v}&ZJpt@ZkNXbrL>qg){De<}JHecg`+zyg4DGK=Y zhOh~h^JF>Nni=|z_pS{zqj;*DEX&RsK!C5+1V?+|!!ru*xZsWK#u4d3kvE3M_Niy1 zL%x=6154?x(qo5R*@>%Q$4?KY{8!F%8Z-DuAtw3{iEnNHnu`4v?Xf86Ofd(?%(zhfq3VWEmP5;rH>R8Fa@R~eWxUtdsnx= zar#k{T~ub@!czGyN$;T_lt~~jS(3V1pv37Hk z1n?Rgtk*GmXyh)tuRC#EcbXWw*rh}#czb-c4h)jN5G(Hsk9HNV)I)BN5nBV|N`==2 zq$K>9MfJ#JbjoYZ#pEwyv!dG_f^@JCe7%8;ZRsmqZ?X;l?OPyFo4MLGRE{#!(0365 zybaAn>nk8ysRy-SnrEFr&wqF?@%iwQDTsX`k5U!_U2z|@)2N>?AkZ&2hHoU6l-{EE ztD^z}Mrp)>*F}h19iEq4Ze9_}EG_TK;WSCVO#ai_v-IH7{k{s(<4vM?iou0xCb;jH_3hQ zpU$;0Ic+ipmiwvYhqOn2(;IR-$fb)?piG5|i}sEeXe}p5xw-`{tjH^^_RUMW{QSYx z=3K)zQ?^si#4Yo%7*`bnjY)XwDH;FlXtvk+fm|%viu3o!TWXQf!;;yNl>+HhXa{ zZGxnn-js!ptR4mIWa!pLbxd!e>}Bw+g<`*y?3VoA7_b{jkX-apR)6e4_tOa`=pDn) zoiupE&ZLYT(WKASR^sQc+Y^_JaxZk6nMgCot-8#a&B3cVg3@0oArbF4Ky8%FQDAHd zR*4|_7RA^&Cs^xE!iPPNt$iU~PCNgv^7@HmvL%@_(yn;BYwJPhN$eCxsSr&W)Qz2SOC(MV%|J5)y2$*NW(=Ln0ads}|FB0-!j4)zPAIO|(83vo{wN3BiG+nvWGbok!@{U>?3WkLa$T zBU2yq+SdxXnERf-@nAdnQ=mnmUGSvIw^1+WMcU@uy}%?jJ5&10yy?S``H}P76?9%p zo8hvlZ(J{P4rA#bmhk5IlI7Bov@=>yx8^?-8G&SLk;K^E_V2fM-vLGS;^c&D8euG0 z^=mHpEA|vInrj#Q+*;!dJ4H*;>PCEt#^H)LnkI` zlZ~gf3S7#04VJD}^31SPjiPc#{acF@+6*z>XYpxi7q7g+=6R!&eahJase;Y8a0)+C z-6z4Q!t=UjC=SA6kOT#otOHM2SJrerwY+1Qb{~4*l27L}EMo| z&OC|jF0?RvDoE_*Nos)|)|vZ3LA+I#C!%pW?Yn(?-ktIxM`Yc8j88%||B`hAco)A* zN(+80VM;M)AaVSUNTqnalza1^pf!{<->nRByUz@Q(zk+)8l77o?>$np8Mu{6=!#x+ zD?d=$N$P&{Blc^{5T!pCA}~8NKV)=_J7M;t=NcZiG-fIzRw|dPHuQn+AT&jf9w@eq zp8G`_`^GJ)@Ee1>T6#pQ_Qq?Mdt0W5xC3+DO|fs8o~wl}VaB0ZbOAN_x5^SHH^K}V zVCVmLFG(Vi9lMrHA4ztmJZviQjO8;+%>d*nzt;-=DH2D$QbNm~9&a|X{VA=PTySou zgXxXxIcv+-d#NuTdGuhie=nT#Gn^8A$v+vDIuY?asL5037=Nm`cCS~N|K>N^VMdQ_ zpp-~>RWus*X-C}4pxy#d;BuYDNAK;cNETeh9j|(wP*~A!QuJwDYpL%;K1WK(6QGxF zqNfzsZrg~1oj$Po|7w^2 zTSj(~y75fH4F9!VeiC%#pAi3y;;^zUwlxsgyt$x#BxFqG!Jn?jx@PZw)77_1GUsfp zZtHjGS+4B%_s6vFrUi$vRyQ_B;{W%zmax-vBflv6-PK^*QE+@y z%|(sUPbWd{i4yY?@1u5yZ)wpz4t50SYBAmuS=#S*1idOPzMmD-ttCXeSLx>CwR^AP znmy}poxhmh?`2{KisrHw@;6srh4sVaT0`o-eSX~i;l%vjU38!T2Q#ba zh5<#7X@dS%;)v&1vQomemk&2+!Fr=+KK;3S>M{BKK((o0$K(4=E0Uf;&|q9mOg3g} z=uYmT0@U&&tsN3+>w*J-@*WN+nhFNd+~ir+xt2X73ISAFwucG8A8L_xWMd4}nG&}l zj3=7J!=sE`fPG*5Hn67{-8?m-!7x4%b-}d|=+8o3@eO1({U(iXNmyaw>G$SW{W~n( z2LRWJ&E;$FLc*Q!&f#6~uWr{Xz-)|bK*Hw6d9Z=JGe7ebKLLE6G}~$->A}KBtLWbh zy4XH+r9wGQ#}Kye5gw;jk<4ICyC`7-X=sJl3bi>J-3|*%3IlHko)5f**|8`Q`adBZ zoq@GETG9bcnd?qw2>KrkQLX$5LVvxTj$+gpnQz}sUU6j$T~q50tCQ^=h9gOBy?NwW zW=_@^M{BbLJb6S&bl)jIQJ%niSjggQaCjV9B1v0kYP&B5qDSqT)~-mrfP%pe>e~Lb zX0v{>u2xuaG{ufiCc4ABUDN(n$Mg%(l@qLu1{F-(_ge?!qVbzVLI#7ZbpTi`)4c5$;R^W7%VK%)!XZPzB>1 z8p@HifK9MX!&hyq#D4CJ>iWp^xrUDEEx?ftBJ+;UT^9ELqU*iG*Ds%c+%N+Kk=R>Y|+_k=7<>+eW` zl4}D~GbRx!#}Io8X{cj$YzCTQ8K$j%c|4@)h?t~lcgo>mbLR)QWphc`Qze*y^z8+e zdmgRM&X%O|aE#uj>z=97BvfEC-~%#jL+LyQVG%uDeQJD9%Cg9cF+K%wY#(p>DQ9d@ z90Wgkg)(-WLnBc;;B@@Ap{?{P4^{Jn?jN98eDQQEZno*b-b3q7m*@_qDt!E@jBDlg z;Gs9xyP#qBwb2a(LiicVjbc8%wt5VLHTo9}fRVO!iN^a^zTJb%zWHSv!DklynOADz z+sSfR<$%pAnRr0$vH#d*QtYZRCNhVv#%bL`49qu*WWC7)K4L8aLqSofG)IZ)Yh@C^ zqxwUf>7EDKYhAZZ3S8AMoo~i$DI4_oG>PTt>WlbV+k zX`9z_p3hesIZCuO*yy|Ct{SYy9BC9D*)DaerY~>9t?;%)(l+0K#pE5WqEDnxSGkw(M39RZ110P?4_{MHBi6({a)5gsABH_*+#dJPkj?%><84GTHt!y#Jw zFCAL~trK(Wf3A6tYkEHH7d=+BHnvW~iTp%Q%QY*CWb!?=A(met%cCZejffuLni^VE zU$HIay!$dawa+3a<^l+@otHFJwdT={{+knE_&r3ZOT-LQSa?l@;%JWpwYGC-)`i%$QDxk4z&My!AgJYF>`!aMU z!aG4C=Nu|df z&OZSG1cI3BOc8+DgcdN8W^rE=o!is&aGt7zre|1y8#xQ(J3s|LC1f~-1!n9}(y60E z_t)FfTQs8a-yJJgO49TIAlTlsd!fV{e0&Vhz08whMSYWOPjEGt%0L0g&Sb#OTVO-Z zjmCH}O~Fw-gN>+gWS{I)FW8G5*DFqM`!38{^=#;_!XNriJQ*%$PA;+bs(@yHGKEbvsUa|qN|~0y6FWs zQK|9L&F9^NU=SWT99Cn#$Yv;3;-BT9iVrw^EXHokG?#dppz@x?K zp=tu!w>C33!P05hleHWuSy}3osW$m!uGyb&y~88d%f*+N#T^f0vz#BO$V^hY{E#)H^6| z68TsiQdDMo>$wq_Du3fcoxR&l{XQayTvwvQ6}g0n+r3MZ*3&x|-|07es08GVNpQ`i z{F=|;e5>&8*lvAfgOVP7m{)+?2su<9lehsEaFTZQn7DZP37-0AQ8)tHid zmAgHq-0Fhzu|MaBmsI}dhh?@64{!dqI~vMs`}uzRfG;UwK10t2lil%MVe5gzD^7Y_NoYF3V+QBV}zyk$E*xeFC#f}ARZMpk92EEdrpo0D4*ND-4 zA>TGeazyg_)(0VE$WkhPr2_|AWC}x3YrOU(TqR5B_r|>k3h-^HloSG)npzTY0(33^ zwbnGN=Vh1~p)}M!>|N-KITnGf6_rf9(=oO!-AG;jS`K`COz8@p+`G@7x%O+I{Eif+ z_{2z6qyy-?c{o&x)!UddR-C7Fzr{7OA@*b~Om(E%U+O!UsT}-OZ7U!x=KwamzP5P` zWZAR9l6%c@&eMdADM0TU>4OiJK-3vT-M6qkPOL$qcR zQSkAD!_@kSQ6+3sswUa|20}BAJ49Q|ap#*h3g9WtK5X*NSwH!ilR3U`oeM4EHz}K# zCg+e$z378bw99-&z;e?4kb#8Bz6b*eXCI(bhjVXl_ZaY=1B%OBz_WoXp1bUMq?ek@d%dY8ZOLp z#S95vFVlSh%B|?&MWdwL+(^8voEtH0yYdPc6)QhoD!K6K=|^dFStc^gOSaf{May~0 zog2oAjXEH4nGy2~v<oUtgV&8ffT z$Bi9pGf&Q{ps^`T^Df*K1P_3r34_87p?!_!McJu%#e(b!tpROc`vl1o6Z@T>2$lN~ zz;Kme=TuE+t#f@Y|2^>HwV)3OnI!moyqT3tbV=*T?^!qf7(8V%UvJAGg+g4a=Yl@WA){^*pGlFPh!R0oW{S%o)!JWktq#@%MBT&$)W+w3}!JdB@w4muzFXr51& zQ3Ng4Hw~|S{PG&WGbi*^#x-TmPV%R17V`CP)uloFC?AM+&r)^*nG}DY6pH)y`#ni^ z%gv*n=uY=JPG~tKI%ka0$tCynDxhx8FMi7MN`H8RH3LyeVg{Q$<@@y>I?FU*qKnbF zmIXd630^lq!{>pSsHoEJ4J)xv$(>L^lqKY*uhA+-w)R=`wI5KnpL@&wZBKDo89wQz z7!*_t1_OAXc#L57Z8d|(ZldcCGHmTPY^-{iR0FhB~x zkg^X2hV&50>hp4z+|zw&4Mm1Oj%Ydc_$dD|TW`6{nFo)Kre1~p$!wj zPoW>K%--u`Od&^zUmu|!#c%3!Wi2XOXqUxZhe+s!HY@8!{UHQkSfip>;kYk?9h9R~ zlGbXrUW#dgQ(RoB46jDtcJ6~$)!_J|Tnl%A=BT+4zC~=fYwb-jIDTt8V5^H!Ev=~` z%NLlA)RUHE7@0uHQWNZX`-yx#>sM&T!YvFwv;qRG?;vG zN3YFvkQXxel2F%56&P#&fy9>&p$c=2D&>T)Bg{0(#z?L_OUIx`wya`x{dW408hDTt zTy_j^VAFy*$ePS@Di2ohpqudkwpCQbar27}I#2g%`@5z-2;fw6UhFfSCQ=W^RU|aQ zjZjBU5R}nsFng&hFPB6)!c8dOzMkMVcn?z!*lKLb%yxTOJ_K6?{8g~)I42zV;OT`i&ODc3m{xkti|lg8Za+KQ z%7lU|uw5M-;$V4BXD&pzbjG%5DV#}w- zwk*qFZV>Svsc`UT7CK65P@CWaKkjQhkg%sT*6q|5RGC-5pZ#GjQ8qcnHlv{EP*o`O zjy@Opjbw=l;4e?FtC+oZOEzJ^S--|e!HgHi`=e%g@-;21ig8|M_Qy?lRjzHP7oWNr zNEbrykV)=vjszZc)yR%jch`Du)SBKwlM>gzL42!RJgMGJoB6ew){Rw7-17>{-qlyV zmkQUjpYYreeqYu8d>zp(3YKw_71b>Q@B+VPg8H{}ojXgdhIX^rg(pSd9n?*Uq^ozO z|GWoAE}}ukQXl>9noU5DeI9SpO?lv5HxbRFFNmnYcLn_|b%TDS{sH%p+LWf;S@*2t z%;|Wh(}>;B(~B=F>%0{gL?@2CY@JG3LSUtO2C%1qm#{6uym`4kRfK(~nzO-*#nup>Vh!@oN>L)0Uv$Zz z-7ZAG157CvcHc*R@)yoIeYuuDvpa=fd(kB{{-!(g>q+2Y2Y=l<4)Gu-RS(eTxAU6q zc_ZohpbTvo(&O|;w&OMyN{@;U9?5tXT$es2y|s8>Qm0Lq=a}A5HAjN$)gW?D2y1Ow zZ(25ZufS&;D~y(vOaepAC(_boaxWa2*(iW#Yt}@DPbNj)<@fVy< z8n?Z6t|{$qpfY7H60czKCl;UMUuu@99r(L=0o)G~M?K`1qpBxmv*udBa9M%*gJ_JJ zlU6+VKbGjbmpQ+VzMY_Uh<~y~hfV8H6W1lpQOT(&F~`#sQK`7oURYuu6YZPK0wUQT zE${M=fhBrq`1oKJ3c%+bN>L8|pjBgJ5%3MtgONOo;N_E=;bf}a@5{M4J?!vP(@dA3 zxsl=Ojys2w_DQ6B6eVfYySj3dD`VJ;0>f$L>CokM5s|=#=dHrJVTQI>Sg(wMis)0j z#v)Bn>&Xy*q3eBy7gBxOi(27F?2Jjuld5U^X_cT?cZSiFwLZ@2xNQ64fJ>|@k62yH z9mEPt$_)+CZr0rA?~`AInW44!{Y)vhb(2hTeS@oPHlGIwR*nU=hP2frT`eu?)E5%K z1sj2SeK!@>gf_}n_l1{qESRsp*DiAcMYqbRiUhZb&Es^luHKTnE9g>Xy^+=ynrE=r zsJb>|L)JqYX1xa|TPQtWIRrfCg2)XIM%0ADvz^Tg0fm%@Y6#-{+>v-PRotr_yYzZJ z;r;uPVucEc@=%fdQv9c}8r0f3r{LI!WUBa$WMhzf=Y8k%)(L8`#{yth0OFQ7U_dFY zt@N{M>SVfZd5*u^a3sc!AJHXjDziS6MQ&;<6}+Niqa^ZVRb>vqqmd}z7{rAE7FX-y z*UC;;&kc|WPDvk&$$F@C-^bJP28`|)P5)uTKuC?zHLNWv0{tY|4O0=b)@RV-KaAWa zIa#Z?E(p1Dh}ol_6uz}6l!PX|BFG2JmZxgg?gkToB2;RhSK5Q877m}0ya026bFf7f zp*7PoP_Mn^Fqa!DAj&OVya4%gE|;QqrZeJt{AtK1hs^we9E4#PtDd?gBb9jsvbE$IV%O3Efkh z?B$wj0v@P`s}N8~{m;Xp(d*EHctiINWMh!ILSy_FPn<(NRjW3V-2G*;%njxGTq+L7 zVgZ@Zo=GjVwUGLNqz)J#>YbZIZa>dme?Rye&@#lb7UnYA{cLy%TR1%@kJ(q{=Hk^u z-;>^Bww%YAo3z4;*u_8@A@jmNekWDp*5lsTbmjYr5np+N(EgS)Cc*KiOTQSk(8lPv zVI;c`&KZC@t#0$%YdDth@#Sw<5z<`MZASLjiNkAWPH*0cg5Ic`o{&|^eQM;Gc_n+& z+Q5swt`%vcH$^&2iXh2@=nH27hn^6eKTs-bmDH2%&Mk-75x8`JW=tP1|C->gznTxwKZM5!Zq=MRfZ9ciwD||{3bl8*? zSuwp(+P~%52nopMO@&n6t_f5E)I9T9xi~ZbP>RIGlKp|a=#??ryMT4j6YLM`7md)| z_SH9Zd_Mk8&vjjYeXzcFU=KP{-W-~930UaG_vY#6pMJxhnV?U?P3J=-JY?kK@6NQ~ za@z+`x>m?t;S%&R_MAuXw~(>clcj0Dg_hd7YUT&+I&Qhbdm$T5HDf%}br$u0h9a8F z*)s{hZde}E6bUg6{Kvi=f0=XQujE)NKlM-c<*;ep#Gii)cG(M5B|V_1Eg>|=(ZNL} zu{h@ba)XD4Tfb8NQS2HxT|Nj=Yr}6=QsOChq$>@=+cpvNq=O}>a}nf5OFwiw?7pH& z{pRw+A5|-Wp$gmSnLucchRd)GRmZ!lv3&XK0oza-WEmN`NeU^z@M)7nUCgMLQ)U!= ze&rC#M&HC!$C$+(wq{V-P|(YJU^mI#r;75FL8LWNzEy zxnb2hH(G<^VJLl1o3PyAyb+F^we_$B6n@*K5jKC@bhOZ@+;mcas0C2}-rJYHx+X?- zuq893^k!=>N5&0kFxEuu1y}b!cd>dd2)UC2-bqa7kP>n4 zpTO1qwwQ}b1I<7<1`#*Zw+Fpo96rD8-zyX)VJ4&Qc7ddH_~U>f{+{kg|Elf3TJ_QO zc$>Q!p>pBKzD_(2`Wq8h1+zM7H; zci>RNIndWo>x-3+#U%V_u~Vn&M|gY__jsS zr0O6W3fb6N>g62z*aBK{YAy#XQTbi5i`C;j``lGM&DSs8lzB?WixZ&&R)8dt-*iuW zX$+K@ZuY;{B`x(a=!Z4fU*mX>rsx-OVC1*t=A;9;>YydAU`ipn_LOsI{As+SdLRgDutsf`-vd^tD2BZ=`2Lsuv=Pj2jOzTC9 z=e@?I{6?bCmi7F3Cm?@d?R2$1KM-RkH`<%!d)v_6Oyn?GwYxzo6pq>!bunz;n0gjJ zUV|)=1UZ(rG_C(eeYj}-yQw4zJf5lHUe8e>N01HW2}qb&!n^Klu`5m zodE)QfUvG3pk{6D6)aVm32Smpy~51$uIVmame0rCo@l81nR~Q0(Qh+6?eC-@ep!Gu zVAW9XYJx=x%S_uH_TfdWi@R^)u5%h%dP)o>NP9S?|TpKK#w^cd2 z%@HjST4x?`_IpB4%*YAIUffC-@r?8=hoDIB7v-Es+?`r&B3m|6_2CTgOK*IMtpb<3 zJWJlA#lsvv*4l`{OiO(NKYOM7q~DjE(S4tFD!0Fx78W*auj0@WZ&)N$5$Aw%B!uVM zcdY z1-E<;u=m{Ptlq2A{{>CQ=TMA>>*MnfyE7XoxgW~2SLdC{A=N2k!!8TIo8b_LYA!7^n7?{~C;U2VBTz~x)#*-X7*@bVd-+1n#*N_m;lSK}A~#dg(IHdhNk}V^lgUM} zr5$}xb~jMaD&JyXWI-u4&%07N?KJj{Tub>fs5L;LVKN?CM7`ZV?^$k^+G=A0pEFSU z-e9p80gH`|1rrrL-qptU*%is9c%N!N z-?5l+NWU2vfi6cgqSki4CMHiMe5wQt8I=wTnjVj@87IP4G+l*0%n+ly9mKr6zyUdg zL~{1h)Il(LBqfXbOsxWT%IFF8Yt5`xETxhtoZwy>=lXV3I?Qc_Eeju-d{kBeqh;?m zIb0OF&m#z)dgiHSjwIQ)90Z3ALv;0Iuc%CM-(}?0-_igre+B(%*9qwph1~-M<|y;_PXSwm|Y+an9GwP%wa{ zO}@#@Ei9S{pb$Hj2DVX56x%x$k};+e=4g{dlpX<*e!zZf_c^l8pWZ&oA}Cg%tAZ>@ z6+%GbZIX~=4KK`dw>X4w3HO)Qhe_rBA^0_q)7yB#$gC9b9wy3WvKi9NWaDALytYs< zUx6J7L9I+)G`kB{7Ou&iEze(1Xl0R$`+^Mn0SC^SZ|LP^rI3$uC;P&Pi=oInS|YDa z&0p4&-Lv=xrv^JT-*SSUv;=&$oFM9?ELY5A_azqZ&*th3h`s}3W)!`Y&i-0>lc=(E zv(2U{|Dq4jY3sTFk68p4VnD-bXueMMB~difv(DzlPLZ|@^EtM%50CB|WG<;VXhFxv z)!WBr70>d&JUCcfl-0lV1V{8_unKh*AV_z&0E2m$=>?FsVfCET;(VG0$Aj68nCZ1| z2ljlMdSMN|JhAgtuoM-Mr6~TWWij_yV0}hZaB=J13it&qVZqVgKKwKNMCOn}bh9)Q zh)*i2)l4zq3A^Ibb2euJTXUwsx?P)tR83wDTZ!P1G=d|%S;81SmA6%`bY1XeNt@i^ zQk>^IV^$8vct)bmmF2cVJ_yuJFL!+iaj!c*23=nhQBx_6C3W?U2KQ${H5#$%QkbZf z5N0K27azU#?5kN0>nA5_kod^DSf_PDXrWc7TS3{Y!^MP?8oQ@$q!00(%JyW@)58aF>oFOQ^L?SpR|j^_miIs&q=$e5Z-#NHK=-*od7RD}mS! z8RhE2B-nnT&q$m1Fi>2}$E+=W`u*)$WEdQiuZn=RB(vwY5@(-p;y@|m=S!HnUyon# zOP-tE<1B9MnvR#0XE8YGZDsQWn`a1s_xZjllV)csfmCUflQ+dxEo8}XzuNoPpl7<> zI8y&(08VARzF9xw{lEOb-OCphFGL#hoBF@IHM^hm>*B`+lPA#621j@3zNI^<)TdJY zIZo##PP>Ve;A7R{`DJ~4CF13Y&hpS?AiixU&!9naCuO{ua*^O0w75G)S{)N&t!xQA z;p@|K#!ve{TKIm<#F}VSaV;c%Ogv{Buy3Z+VI$+a#6hrSN_*$4WNypnPoR!UrlnNO z>t#Ec5mx&^pWJPf-KF}X_V~ zwxlY6MX6f4&)0a63w2ReGjF%MDnefF#Bi6(Hfod`$!|;Ju2Mj! z=uc5+{>ksT>DVnUC8t_H$WTgoK*NLGDwN&_BP8^{Ai@~%`WEWOA> zdS=6B6$3??j%uuCmB9yJJ@KfHdhkge7i9i5g%=ndmajTB7IQmWzaDg$E_;9tA( zy^|KdD+jtb3+}9Y#z(7+S`}9)$$m1Nsd!Jl_;0>$2$9lJ9>)L}neI?60^aEW0^NtXqtI&(B*z4-KY0H5h*imstEAqkE>%y?HO5~aonTYdV z4g7EeNvUT5>}RRC?3X_$>6qtFI%}6yOGc&U4OcsN2n;KP5ii8#AUF{D1EW|2JpA&d zXM1zQy6 zXrX!4i7Qri+tu3nzF*N?iEs6EnyTP-l{ngNvw)}O-Sjt`>BU`Kmi!}SXRwvB1Lq5Z z`vl6^wS8Chrda)vR$UlZ=e8zH-p6(E;HudK1=s}!& z5#QkVnHwTIdf?h)Huv2cqtXEN)f09$Osbz}Wjg7!kR>?Ny>2K8PoG6+eYDz&eoN7ROu8#BbC$+t|Lwc+N0V-9 zH8v|xrBgH=mGV4ZQv001zfy5#`pel$HM&Q|{Bgri4*qrN-rP^5oc>fe&D96s^MgM& zO14e!=$!U>RgO?nH+F`CzI3=xpY|O6KnVOCJkggI%CkQ8K1i;)2>7&lki5T`>g&I} zo|Xp6*VhlN_VtO6H*DlfNP}+rx6h>Q*NlYb$u+YmnM7<@&8&EhnH*kG4!r@k7C~-= z0K!U+D6Obriw*Lt>gC*oj@-08rJXCgb8P{Z>+d@bR&(mXEABdk{IOCS8-qOrsN`sZ zUZmr|VVMwIZNvzVr;cZbJK8W#?Dg3Z{Jb{?XS7$(>4JfW^Q5|W;!f+Ia*OWjNQJt$ zPALPAc~1N7P-@xc7G;FBf}M@=MpZo3U-3@9D!y6YwQk0=rYM8d=;Jlp@SbzMMsZzi z-~?8%*0fzV(BH=02A0!-Kd&9wusBhvSK{sHlGV&-O-0qWau!Mdz=)u3J0u1@yN0qz zEG_oId)Q?5o}gU#>$+}?PEyRMb1=7IDtFl!gi zJED&~u6~pqsXD>b+_UxcX|^dOT;)bwQMXmoa`17;3vrTozW#~W?fnahHL7SbD&s-R zarit22e``CdXygQr=WrB8@MB&wKA(p|8|&tI z)qNGp^&!Z;#;){UuN41A6$?3A57UG?>Di~~n1_p6Vpr`3KA|CQpy;KSRT-`pcinXD zoF&93jdvLgb&*BNsVOW1F|AEdC&>lv8S31N%6+j%gW(@*>x?dj$-c2=oRQnIY?Sfc zTX%+IP*mx6_L&c1i_7R1K)a>n~AjhLx+yBs$0 z0=b-GJ&<1~SXMts1gc7`QfMJugcShg6L9fNYxS|=@MQ6-|8!lreN#-k z?YM1g9>xbg1;z3s^2^5$KJtY-6t7GsS_RG=)Mm|PQfCO3EumoFjVMHdLx4ImZGs%d zD#G5}{9`PS4K*8gUCwq-AM0Jg`Sw^-DrRK@k#7PSM+(Shyidt(#YAb_RMrP%k*qQ( zRuCZ-A9#zpmh-96!E(D-$*I>-rQr~ac5fD>X31ISWJlD7$KpO18s>n&zNLIP&g5=I zSK{4WwiNJTJf$|SH_qB#zv`aMy%`m(SG`^`pTN)Dp3h0)()%DHH_+BY#c%Z#*nWdg zYiIL6_SxSrbAGH^R{XPjHuWNGXyzfqUv}B@Rh1)$n~?5u+oqr*Ep>7`kKMq7qYmZ2 z+ee2JRqCAf@eY;Q`mh3F`^xI;kwK8-@&3V4@ccgIuXgSMsSmV^EQ?G_$C#xloqP=a*to05riv+dYHDelq8OaEwjfYBQK$!hf0kC zoGe%^A=2@fJw$Vy+J1kj{o<@75dBQ4(rK*`b^90irlqdw5LtJ};TGoiav5q@VvMT* z#(0{iZK}t^>+XPJA)98c8i!5Wx}Ns6$!|-&z&2U&u0z(4#VDYZB^ilXb%zOCTAP$* z9_!4<^(2mkm;avgY1eNQ%Q>Vc4-~;y4Ax*ORZN7kzq`M}ng47veZ}W4)&j&YDa`c` z^*2;K>rl4$bd#RCcNOqx&S@KUS^h8HCv_m`S-E=@qU;ktXPIfV%ljcN?$iXoyN8L$ zP1&Rm&vIN9R74UbjGvNSFSyNCAP_eOS_T=4usoYz&F+}%*E+$wMedlNiY;Tos4Al= zQr(AcsAor?@yTGFB?WX{A`mR~>ecBQZpaWc)R(ZPr#&b%R)0qd52D%{;qcf=+lRu> zL>{`#rF|MLX2YmXPUrakR=qVbIUqAvBp=`hQfCYu=D~)i8Lf&8Z~n^X3VPi*hcq`{ zCC_yMyMhX?wf1{f%SvThfcA|(^UU63oPk9t4C(1+)rSWZ*OHmjJ|%zU#=UVa91Kt+ zpNA5gk^V(>8s(df#l9;w{s%K}TngP-W})@D1&T&VGt))Tvw;AQBG8~Q;|1IJw&^LX z?W72q?@s8cWT8K0y|sEaMDEl1dQoR2VPJgiaUQDV!xyqvBdTIA3VykUsk(5I*uUd_ zd-9-ADi2qeuJ(0v712#-yp}nM$T?rZlat*w9VS@inOmbPSucbRXr7zcg;T}1J`VyD z{L{vk%b*sAx>x{Pz)D|@ZO#LTagDA6!jqJR=xh;^Rl~`6_|ZC;tbh8YtD19S5v&NV zlx}(d5^tX-^VJZy^usf5tTKgfoo9e;ZK5}V%YJqzyj*rkbgtFoW3e&@g^83K#fN$P z3V1wN=`2F=EeGEfIxT9nz@@v?Gj%q9hUGEp`Wcr6yO%kFU;zzjU({w!Xw5%bivIr= z*FRV}6H%{sS^sQt9Y*DWQAzrmhB&I_*~dd$m*{f`=h0m>DX;8C-u*s}b*%VDN(b6D zIls(GU_!g&N1%c(0V?|;5BDvgPxfzibnLsV9OI-!Ecwlra(8-n1`f1s_=cYzUJQG? zALCK!9sF2)UPR#KW@I+zM^`40{9-c)omRIzF_~gbp6loXS+}D6RQ#O?Y0Hd7lEli^ zNu0H-6*JD0jfjI6 znc1Q$G=G@;S7v8%MO+{k2buE`#?aOl_X!wwXA4{I7IK^eK1tI=s%Se%JWCu?VS~3X zb_HnvMEc;f#ZH1MO?`*mTJ}{(vYeKhjWDWdC9Vp|}R2mEBP>&25@veVimXw7O;hRE8sRQ4P#(1xTXewy$HwqPL@%sgZbl{ zjmvVR6}dO{lXc7G8?YVSD9^eVgctgDPO1vy)2}~#oFx4a5J8tjjpL;DTCWDh24RB$ zB_Xp`Ga(%zjR7JX(@)YW1w3|&oNFyUw$}8_qMc{4t8(uKU)#?`U8cV2pT-$tA6qLV zThZ+W;FrP$^pgXKB4`@9{^!y>*q56(d*8g50B|SxxqQObtJr0U03oT(!y%2C*rl0J zXSn$QVtr{%e;uJ$&xlO>&%SuIrc&_~ECL{IDINkqo5mhNscXYO}QF zcM~TVpT~iE{ED9A4(?|*-#hooZpu{h>g-KmhcRK_y^^Not62RXqfEW@#6{Auxm&(* z_=fVtA@7#R90j>tj1zISlOMuDG}?5(Go6WzBSI1^_EX2=Dn{LPR}j6BKE@;2L9Y3e zLvb?eXoh9iKhD&2oYy&3IeUJQwiuMDHurS@wYhh2eDQ+8p6+tu&34u^iaPSWtY`l5 zvfiTQ%%$V(?t8n+Oua?tl+e7$sJS7> z6naUgXIAzLgHG?Ku72rHT1DsXfb|ZgB@%>+%ay$F#e~@|iuER8FXcV=iq`@N*j3ro z-JTEC3vaKMtytB+eYGgRU|1PtPTUqOk#82o)})era>)ghvLpY9S6%&9 z`}p%Qow%gmkw-(bmQg>T4zm6j&c<`ADNmd7FVSgDHajJezMGUOKi168y|i~JLHgAsJN?VA?Sr~@ z>^DAW8--*(T;s%?K!ZE3xW-GEhRU;%^gN;`HsZD5pYN+)QgSqaljS{5jrHoe z{H8wrS<%+H@J*hXn=;p4D8!TU7Xv*ro1j8+92ChaK!Y4pfA7u5ifi%|MwW%}=KB1a zPcJz#xmA?|AGGGFf8tOgR z;r3GI+9;-U@wP;J?b(tLOlo29a)KI^{*e17jr>h7cG^kH1rbWFC2E={S|8eYU39g*_c=8kMj8NaDST(dTW1!mXqTQuietNX9TiFC<446 z9dYRlRNz7ND4mYJdezUj`phOyzZ}G_ELBMQ6FGjg!;+`xUOm!z=>-`J%|l6l9%QgQ zyYvGeI~+0GF&Fs8VkA7B@Fad{#jZ{F;74{qNelhul!*#@sJz1(<8jfHiQja{RT7P4 zYj@kpzi92!au(2WE|S(LR3G_jU1v z?kG*BfFz{|4MYO>H|@pNE|cRm+w%;-f|kx_1ZKHcnwwNX$H(m}gZ|%*`k^}vO%$mI z1jDVE_N!+cy67yv+<(ioqUdwd2{pPX9q(=?jt4)8dyPiG11EmW13Eehmw50w^Sh!9 zqHVuLr7xXfsY+N|X}i+laI5>^U#PZeIX}{IF5qIK7_V{0{0XZ4Gw1(-DuxP`+f;O( z*d7x1J!9o09z9uFb`;7?vU=g%BnNJD~vZ4Z8M#2-oA9k zg(7aJt%(YbJglPok>DwLeLai$Y=y!r+!t!=e;l*i;n%^>eDC2w6(#*`D%aMPSgY=c zOEmPI9-P#`39fx1z+v+tN7x@=j{0W%i{n@0cflI1jnLQ8 zi@lB6^9$|i(%Y`7hrk;-wH-C75bZ11v*%P9B)zs=fu3&A;kPf%F0s`D;llp@kFIxI zjlO?RSg(uwrYfz?9+SQml1cpRIv(DxhQ3rtTX3dv-Fj#LU!)FbIZNp{``*PyQoW0o z{)E)=nX~^Owe)vTxqay@{Av0Z^zsiWS~Ht`-RNF-xwiz&9482aU(R@FQ?Q-#@m!vL z59^5xXH4@?%;)K71v}jWn`F-n{PkZuP}AVY8bh>d9@T2kjGpM_j^s-?z^~$`9yA@O z;u?ondL)x#xiny`uM&=uIr$w+=+)wS=-0GHsuI}%tUL-mPb66DMPtMUHmSx|R*`=A zhxBy=$rdWSA%!lUBNh94u+d92(2%4l~`4Ruz#=}TX|&H zCcD5#-5fUvl;tXZ<;~#^!{^7Of;-BefWXDd!hzz&Tl&M#C&o#RvAaEBrK?RinyY@g zxZ{7(ilXDJrQ>{CU!6hSf^paXaQ@NK9G(1q{>5FVp|L=SXuRBaei(f_F>_nHmh z(S6_*x)J=kki!_}%b^-dYnQy#{MfDb5)I3jcU3dChnH!fz`{z*HG_z!0Abx7cXF1M z<_>Z1p|l*lVR=pUSt#9K-%luc>sK(%-?_fCxu{ROeizGN@L8ne_Eg`3?bd>{`l@JB zx~IbL`8t^2+{6_a5SPziJG(>`7O+Epmn~?vs|QD z^k=S{8KZKXgM?KI1`GGW2ok(E7$&j z+1?q(ejt8G8Pq< zna;-d;C%d_sIQMORq;-S)*GiXbtw ztoO@5Kk^a`VP|ewGio-MOtnOHhvB#D8(;1GJW8r>Ww<@vxFAQD#XL1`a6>J798+0DKOd`XZ3x|W zKL>gJ_AT(Dces^OSn2lN(J!3__s06el7(Vyt>*Dq_lS0Ye&P5++~-b#JSESKqsOAk z{7Ot_B_5Us<=HNzk!OF=P*8@tuNK5Tat-`c-lX)crL@dON>qNnA$6zrMgVSl_O)K6 zzhVjpu~N5vE|pduvb}xuFIa!kaaPcAHXYV=Q}cV~pY|cOaQJ5*W>WW|R(M(5@PdEx z1CZvTx8tO~m^I0zIYFGRKJ!Y2L4*5OhHNJS&lyiBv$|;UML6x1YKN`*10WuJoBpEV zK-fmYIPic}+9;Uy_5i;Das8=)mpwPJ_uSj_>H6k1T52xeANR5z7o*>h38=+hW(GK- zGC46lDY&qO$L}A=(x|&Ie!fO$FznMKthn_(esPicZE|GR5smuhiP-E;(z{>Kll6|T zkAPcL{$~v5GjB%!jPGPEGpC9hyk#k6rstR-d@td;(2Y>-$K7@Z+?$oAJ)M#qh4>E- zS<)*SvqPi)Bf6$&fxezI}+BjF@=F_ryjz9adf#1`zNav6DERGt2Dec~2ZeIeT z2FplSZBP6iQs7t7@$Hpot(%s#@5$YuBwoVXiBY@(XZHmTZaw1VzkyK{%yvV@-HsFH zIKvn=hVJP@3jSwH!)J5dmCY#bW`X=S)#hN>eoSiRI~QhrwN zU3xZx*8W#P+55AYpDj};UW(X-8&RXrHAu;yEpL^@a8BaFBufw$|5LcXrsMof$9eo$ z^(s}vp!`pfMMHD)Ph^!-BMbV#WN2I3i)7_IQk}`h_Gmr%)2FY{DqsrQ-dq~mxhD`K zDd@7^@UH&B4Y1f13mYbu6|GMY4M;_W3(Mf^`?j>g)blEr$JLbcroH1rXM;ycpR+Wj zvrOJIXERmqK1f(xa#MHtwK0o{^j@@lGwFs=IH{Jo=Z;2Mj<)PFSMqoI*iNYFrQ{4p z5snkaOU9SHim`XDOiBNc-FsmCEgO7opzycpg2H2;OV_{jU&>`)I8MTGU``Mw>+>m> zk|*p!Q=C2B2<$ln=N{?#+xCVPEJPYiPIsEkmnMFHLEmyN^i5`a@5dqNG5;d3PY8vH zO9yVr_ZP^A{i-}$9L>L>_5ZXwTYt*$|J&+p{hi+-g>Ea!*6!&k(NK=^b}NQnewsMg zj^f*yuC95h%bSLp*FE-1Y&l^iJq^RTO?%2}7i+yDbi}L|mpo|mJ#%#u%IP))A08g!q<#yw zEK<&s0N;3aY%!+FGKD+gjn9|Atn+UQ@0E{fUAgiyaE2+fXXoz2v89Voq4s=BOhvEs zr$svb@_x>^tgzZzb|i?3Dfn2$NiP@ca_kn{`583rp7_F}zy8Wj4MCY)PxY00ag&2` znNv5x>e6Kz=BM6E^Gwv34E?TyJj99*@$Pde%I_GmkEy%eiobLASG6E9;buFL_OF`9 zd4x6j%W71h{DSIAN+I39q4ob1r+)

RwaHnL@z@2o#b3t#wK=Ng_B=IF6i=!Z4v&7h$z3YbcGkM zbC^=Rq}U1_@HW!I3Hv;sk~8%t+UcttyBxdVpDV{em9_GQWRotp;VP&&CE+pe2WVka zoM+^BVZ1_9+RZ=Be9`VFJ`Fo6&b4(Q>mp}{`bYTNnya?{?3@@h-a%?!7}r{K++nvo zxnaWsAav`$;h9Dpvn67Wf{%Cp90Wwik@rT(DUHazro%PPzN#(Ysncb{885cB=-P%B zJsY4)+j4iOD7TuCVPS3YCnf`)bq$vXHHqBoQ!(cSWRsQR=t+J=eunNOZOrGkT~T6P z0l-lz6IvRKjn{fRQVPiLd;g9I86nqnwWuAjWe2uohHlb~LhZ(sGKF8Q?~J=eK7fx` za7X-NIvk_0GX)o1FWYaAH5;w#Y*KO)hA<6(IrMLH`@U%1cSmvdrV0LT8`*`GhZU&K zw^T2`chJJgibItN>8=s;U-{AOyNanB!?(*F3@!@nDnf5Z#HRE`tkYM-;fXyB;n-yS z#~-S$&mMdFBp9!NLw1?{iSca1yCt=YPRWNLt`5G`M2gj&kG%mWNMDYGCKc?5_OA5p z8LJyi2p6^M{RFq0_HZ^1d8KI+8f-r2YqvIq;CTCO>k*c^_h$tx(TxWw9k7$Cg1G03 z@Nyd3W~TC%$BxYxcckw?S*p@*+ob*WXHKzmgggAo?&#~(hSooit4mhGZl)&Y7#9LD zov`AF%6y`S(V3ekg`6K?8RYP9xXBOWE2!)qW@MV=?YwVhqIO~N-Q=`&kbdk=*6ow$ z>z>%Z{~b{V#qL9BSAvWFJtPDGX%uhn7*^0TV>Hok6-4p zzg#x1SKS4NbboETxP8Fmk%!ecr&hcQ@zpfgdE_U1gBu6!qY5E}_;W8~ll^ zv3YzfO7Gj4hogx+&jHOgv0!I?@lI|AOD!Mgc~vf)b}|bab#zhbJ(Jfj#SB&cMY*#F zMj-z=GCB+uf68W`E2rGJn-=f->ep6N!Is&FXe)QCfrsJA1=i_5!-r!IfFuP_LY&|3 zy55#qo_oekf(UmJ*+{WP9q47VZR#F#TTW9R&?JYDMz&T+RZFFSTU zN8J*5koB}JHaqjGAh_hf8Bn%Evf1}kkNsk_lUuNR31Vu<-`;_euYQtn^)C|H1jeT; z%43tby>S@Wz)u(AdBpFpOl@0z%;>JL_VMvvXAjkRd~z&JJlUp0f3omnTt{1eUcri* z&ic{)JHD~_Ms+*xK#jx3&$N8aFOWo-A=_W~>YFpkftbmOv&cRZkILIIeI+E*sFq1u zT#*_oz7Vj8YOj=Z_I#>UjTo$dNch3<=RsOALEy1{*3H%F2K^R!pu4iF=QiqM_>GIs zK}Y$rK*(;~^=FJMV9`10Bh@=Z`)+CjYAl_MQ=f4qprd5tE!8}A2 z3efu3CUljWch^dJiujCEpixH0^^*MBjbVCjCo4Te> z#h7(@(gBHtbMxVoaDR_SjR@*1w*aU)714-GTI2qaa`t~qI%||O{k-ls@ws<)3q{8q zsn%yScePD=ol$hDIlV5UtQB@H`M6~5S`l6)tiW%cdzaqVhdCSLf1q{@bySNV9GH|6 z${mdKp1$H;@;!)xV)C2rEI=;Lc+h_8$o5Tgtb8_qbeLkJf86HD%`Hg|?V~MR`)eXS z@++e(()r5D&vcLVwV;E230|IsIQ!r*d`?Mw;^{<;`L^NNi2$+=O5l81#BT`G9ec6q z2Z!d_h9i_$o5klre};0t3qxr8RLi2Wx_5Rb@uMNr;2p`hPOo>+2-2_ods=_fxOV<* zf+?)xqGt2G0mnUAJ2VZklRp@>hp5uN!rX(&FDuPHRD_$1p}ZRf#*%Mj@52H&m^AJ9 zbUQFYd}|;H3@hiN8vq#9{eWgi=iyo4POc0 zccv||RI;j(?MBc`aG^u-keJTg^kOC<>TFCgM6j5C*4;w2vnLciB3wv(K#S&WLy=PK zel(y@7z6z0bq3XKnb5hd8eYZ7*E@$tPu(qf=-b)-UF(h2IRZoU&DnXn6>Ja<;HdICsQ!Lmxsnn--V%&z0n;++Cn!vW*GM|VPg{T=H= zH|+i`yCvpyApD4tx(+t-QS*>jV#*+V8pA-*q>nMTiG*_t^TD5zlBaC^N^H|czx(FP zyk`f4b=@KITw~Sc_Q6SutgDfb{R@9uP^fS>UbY8DE^m%4PN$+{BPNQ~Vmr|0?#o8Sp>?zKpUu-zx>#~KB% zYi5|_!N(ddw;;DzdVZ<#wZ|V)BHx(p1o`+`-3&W^^BRpia&U}+jHW)rU`FWi z+_SNt;%;JXogKirH}MGSY+s2ndhgTe*jawSgyWpvZrfeQYR+uSZR(wQ7q{}s5BU%nE;qU0ah52n*!@*$jiD#D{ zUKMYCL`;Um{0t}jX}@P)b=eu=QuKElhEb}2pjpRJe3*TerB|1{V=;Y;)yjN+KfpRv zz2ciwnq53!t%@qXlP+KVbaoCnAT@xY`vQB-znKxfF7g|Yni$4n!aQdO49|oGC5ROlua&LRhV&2YzA0%CSRZ7!F zNM|Lp2MqQZX-8h`2}as?)kW0F69Vs-m3Vc%@w@Wau+tM7K}qs!&UdjMjz}muOpsN| zfZ22R}0BU9NzGNlso$^r|YU__f2f zy(s|KYkPO*q@DzHGbv^NuIOLQ|6JFKG{~pYK>&Z_U*Z2#(rGh6-!VZSzk2@>q2_lOqJ=l}1!=42BE|rQtecs)eN^E)002)7pyCNr)06M#y%Ad~JR-nE+5Fja zW&?KEP8*NUCi$*S*GoxH4rMq|UdX}c*oNnQyhHMbE}uakF|*)}rOQ9%=Q`b0 zNjh&Idz}f2k>}Ync6}B=&nE zo^ADUS7KU6`8spAt5)f}N694<-O`gm#8ROqsA#)*9C^FjcWJ5jh7%f0Z@qpqaV1f=GZ|`Se74x+1wZOaR>dB*;H#a`=DX|TI9tDfv$w9xtp2!GTyUK= z2afLXo~qF%TACKWl)O55{`&RF@w1G&Fh-}|)Q^^UC*w>j8wcNAlk2nQzz#Ofq<3-O zZ#xebXWI6ABX)f77GbVUgX&!aO}%S2c(4d(aPv>!*{Ig#b)z<51;u4M?%N)i_RK$n zQ{@6}JTV-9sV-2@pl(zHxf*?}RDF9YbN@o>n`XNeY3GIwQ;g?X7b#rvxG`wUT?+7b zb^46KmnUskDZ2#Q6trtCVU_m%CZNsNSIr=CuhUz>BD?-8BCBJoi(wfM|EED;Ot7Gk z=!WOBRzP7P5h>l64NVV8z05d->GGk( zwT^C^*5DtptpdB^aD$CIVP1DATadqaBhU9>mTiKbJ$I}A<(`?zT1m4I>foTa^?1~c zUh)-oSi|xsByaV6cL@Ac?qfyIqO9AeOcEG>i|MICp1lLV4#*c1g@j@~4_=aXU`?5nbVyNW=E7JSY1MjrB@wb;Qo+rUOTs)2X-YmU;6@XgL{v9^%Xd;TB- z=GAYIqwCPJf|cVZPlDM=o8#9VlQ*hBz$1DOGL$wLV8FBR>~?#8+pexv%M&2=vMK*y0c{ zrfC|xF07;F+>6`trXX{r)2*;l^X;UVvsY7bcIze`jpDa36KwUdb2;}kU(A|{Om@4 zP5-(}pSX8pz!O*;DnZgHWr<_2N-qF4?;uCZCu4W`9r%&-FbeSu`$OOCw(Ff)_933n z0q)j5@e_XZ{2!RA+SQzPI_&WH@g4bMx+1UV#;1Vki=~&x0?Z#hz7~sO-I-1wIUBS6 zx0?y(HRTzVr$U{^kEKo|9{SlgY#Z0ZDD_*Ci$C5avW=md>mH)j8lzUuTE*P$KAbEVJ+;9iS7R z%B&(#gDx!FK#73(@#lWEYQ6D81CefXNUgH8|JEsHp3{y?`H zc>Kgql1xtktTBoB*>2t5Vdv+?ELywAZu-p^3F1u@<5CvUFd%7hVR+82%F1VzFh?R)`4d`xw*LXDfN*y7q>J~c{*}Nxx1D!&w8OGHvO`zHRrS* z;#7}(3IwoU{q(zJO^9eP+5;VgUieU6sJ|I){J&8!vGII99~_f|Uo+HXd^|A59S;TE zY%INLwWi=T?7=G|be%DJ=5(iWts%CJwDv40daY>lZOQ*sZcKqWdPV<{dm9O;mnV}} zKpqgEgvM~6t6^hMepMfsF^>n;26-DKPsG3{B@h+DNdz^4VfCh+(PL`hWK1I-YR|}l z$tJiwN?t?A}CgeU8HAp*>P9efB|x&Mh3!WC_(ESC(Tbh?w9poE|HNY@{}6t z$Fp7S;tV84ILGEWd;E%t;0nQLLXv02&TZsH^{{$hl@$mc*F{{1II}vY zv}Gy+m2M5=!6)Diuo_jPs*EI#e+TOG-WZK-=-jQccx zEc9UVqzj)(<1|(ng^`&!K6PU`LKdg&D-|X|pNKQ!Ks4N?S;TBaP!}CSt4*kzg3n_j zO$mq+w`fnO4--A0&5jJP4J<>ZiU#tyiU>}46#(Fs&k;IqUbJA-f_gkwLi51^@pzPl zp^9o^8XrO1|0dz|IxNbl%!1=Du|1J5@Fw{=Xd-8X>{)06Oq@|cPxLT@!=?5Msg|_1x@1om zo=^v>cBw8?i3xYo?PKlNl1{dXz~p|F{b=tyh1ocAA7LywP)EeEsR(}s5Qbs9gW6md zB|2*UWe|rdY|r_(zyavv#^?!SblLBU`n9O6vHb^`mg{BwN2VIvHJL*Cs~IGc70NXp zR^WfsEpb~V1|#fB%EZMEAymXv6qH`6$YO)N!y1AZGy+UwR6!#E&vP4Nq_uMlfJL$r z0k=yhCt^|=8m@k*Q>2Y3u&b>opM@$GEJBTV$dxf9)5k-Cf@U$5)}A?$VebCpd`_6u zaX`{6q5$jN4usLzNbkr_r;uU6K~*ce>>V4TSK*v7BgQcaRUyV-a5VUA<&D2 zXoGY(j`+Bopv+c4mIS!PXpeiG{~n@e55$j7L|-%I|gRS+X?1E zj;Mj!m2W%-NVTYe-!Z^5)rN)*h#7oZyR#M3T^cr}QNY9J8WL1=c$9M*5>-~fh<}J6 zMQSV={l0!p95%Jpz5pM7W(a^1m>+<@0OyC8fn8CAaVOZ~l>MR^4hN0|Od0xCmDb9E zkMOcNk__&uu5%}K7Lt}~6L2(gzO>eyR92fE`>@n!tRu0+pibMDM}lisbakA7i7ah- z`}!*67G*xp)VESmBwaXO>)95diS>0BV4QQtG`JU4!K2NKT+F4sE;NUV=j5S5?a6~2=j2KJ0 zUp7@aDH<85qG6hv>+b;*rPAv%3vX+AAVlb(gf~x4dp;7#Hn)|#aO#=iV6@o2i=3oNY;?E9~H*+BLeH>I7c!; zd5$9j=hR|c^fm!?BKiZd3pz}XHAgxveDy$=HFVK1DYM-SnxZIVM4HutkX42bJev`~ z&Zb1R1W`>)b?pg*O+;5pQfk29ooZk~DZYT5=&xw4r{iH$;nCE+CcehL4Cx*T@_7;x z0#b7#V!-Cuv1#@@jYMPQyT@rJkLD)`{Im2RuT- z01{Z!NZh>Kr{QLdJ}w%I01uN=`XzzUUd9AXk$;14jDTKbTaO4Y_@c8rkCqr695E+CLQrE|F3-I#^`It=)287&93dv7XOocWqJqyBVX~q zQ6K8gx#}A*qj<3EmDV-sH3bIb~%b z#><9MS(*Y}U;`#x4JnLzt^Y)(foz5_0?NX2Qy55C63T_r{}^G3iyDIW@r^_D1PX-- zhh*C!Mg&|kkj5Fv7+kR@p$TGBWEO@M+gz69v5LB5=z zgzdnb34w2STPvr+s%u}4*u@fImuZMTdV-6)9GGSJLf~`}-WK<(lIkO^)A)`C~S$yR*@d+#OdDV50s>~ME+xM1%IJDG2aco zNx=^mkEw!PaQx_{SwbY8jXO&4o^_gNNmbw}AY?296A2)YYKb|n)i6DB*`UI3e1=;D z6=EwxPPjyFBL=s-e3(m*;s49x$$LNVX!Goth$nF1%lJX|WEhF+8gh z&B3OF{kz#J)Ut=1TH63NZMTV}1WpVEE(s~q&>{}F%QYj+mB=YcNxU2bnAQX#%Hh-A zjdG92om^wljFCneCyHl8tzCFaeNEb|?qrTzl?(eD^g1d6`8i;eKH$=fpZ^Aqhm*Yt z*?v^YEX}gtC2K&rYyH4B?J)MNZ6Z>IfKjv)C^e(P$#f{7uY-Sk3gKN7H`0ss(mFkd z>=Gnsp2|eUlO0-%^^{oT^@|7V>Nara|^I4YH$YYe;xvg#UN4{|fhi+2i*AmpyL(w>>7q*YXGk z%#vnyCut47{onz=2Rp>UNpK>>fGA}`D-Roq;O#0k8P7Ub5qHW|NMN=4a}$WbRFRfh0c>@c~2SPLC~0S2HCEU{>r;_9fwxs=bXR+&32vxaI#1h z!ho7mTazZ+kOayL1@$xvxGahE#`2>OEtakTOQf$-0n{r9=oyM+tmoiRGAs-)6Cn`s zakZM{c{w&;BVeqR`MC%Bfr#D6v@wfQwe4K?m|&&K794CNj>E%q839FAiw*clZ&Mwf ziqkDSpa&l+hLj{i4Y=Ef7N8GkTsk)`n@N_*89JgJs}nGw+YJ%}Fo0EBL7q@2&(QM} z@IZFE^a-jITT^LHpQsYxqWtyAkXg4x5fyIX4v3@z&8eRiYEb~$P9C2K zlcuuCOjXrd@&o`UuV~BTN;)};mLgB*Xayzya#d=56`$MW+}I=8K~h`pO9(aAi+6`r z;o9br#%AJ}A}BjTB8iaz+)+@Nl-o6z5)mQLQkZmN>hmoAEs0SbF(Ontd#7}YhfKJe zL+}E`f=J*x6KF*llWhUliSkH1T@Ei>xHRR{B!og+fot79o~H#5fzdOp01NVptbu7L z;fAt&f|~?WsPYaK(+6LQLm4XMLXjd4Moa~Dr%)GITt`wD7~&#fqt4?|>DK;OA0;{c zH87szlB%t;MmcAVdJY51Gg38Rd!6fGVo}0;>LlLE1jXqG1WdF6O&kJI;OM7vz$(Pn zo%(T6DmdOAq|yvOY-d%LX6$p;DxmP&g@qAb_DN%gujEWPEykJ_&XjL2@-f2_Cv>68 zlmP0apQL`53jpK@oO6TKW;TqAxs}jyZ;?+@jj!T*w?CM_TrsB<3j{Ml|2W+AMPD&c z?8uG07xY zocC7_+xm2dgmiM<-3`4Z_C5ftM@;D>K%A035K41J4sjCucTH3zdCvrj0eMc9nQ-rR zAppbm^T_bpXB5W!1QiSBG>si21eyiMvj@S1s=en^>JV@?`>91`V5odg{dufP2Z5u>3*GpHusz_szY*4DGlxEB0w5fZ6y$B@~x{4=L!kd}ieLX-#K zC1agW*^3s6>%anGZ8x>nDbimAhH0n7Q}{uoZ{?K0suNaMMW|GU7hKii0|Y`{47eK(u`Zt0(mX= zmh_1cuh==I046zMHXNyHi!;qiMUeu>+?Q3N0ph-_FuSb93W6>d4o9ZoiH()*r4>*? zDAYyXzzsc5ooL5~#joKT66XN+-jzji@~TK1xSKc$D~OIQf3KTyscNw8>ia}ii=arm zxn^7i)y_K}JPYSe8atXmu@wTqe04R9Xd|OV%g;E*6yRlJyB~s8p2&~_QXZo=GvTbF zxN=SaGUno-0rTJlSb5>R#t{^6hywE`Og?%KQ2I(+#)cR^r~!r0QkUs&+TW-gvSa2m zv^uaCNK}JbU@L|Y=D-@Ei;PN_6OZr8va3Z>;++$G7E%^T+;%Fbh>Ykpj;~1_ZfJol zxipt(^lC+EeR3FFXQHR$cNH0_0Eo_FSRlEv5@Ee5GljQc>q-h)%(lis5mq#(uu?mp zE=i2DKu+9siMnR9xJ|log~=`k;52ZM<}^rz)k;!aXqNgY+z5%I(lmxacl%MnChA=n z>=aT1KOOg)243T^b7NhXF`YO)vb`xlruPj30%A`r;KwZdL(&gKx`)UWET1OZFiCp9 zBvMf&;l|_g3w;%e*70rvT+JxTf$KZi2Wztc0N|j6GH;6*L5?>d6QS5`H}9-H?mze1 zzm3ps#_0ST$Nn_7LeKqEe9hAn{l^&3{X4z_XcmF73J5o|NTNu+N(N8c!YaV6ee$8u zQBFXu(W|$phzdE=T zS=LAf`c+}a#D)RQadq<6da?0cGSlK{DO6)$LQ6C+G)CGAxHCgM)MPM;Q*jYrR|QMV zyeV#P+KEYlr9H2sWsKA|*2@{#EJqXfL_bF;6N;`C0Tqh~eKKRN%ovlI(ndTtPa(m) z@&YFc6SJ# zhD+J~4AKcOT&u{xJ(nMb$LSkqb(ST{4nc7=e61z`>ykPzIVBoSTzgkzJj+86MMG)BEG4kAkt*ati7eHG!^A~zFX*1j}NHvD96$ql+wrkQpBPlprRDZ;nQo0KzX;Z z{@NbN*S%a`OQ3uz0HokuzGyo>YlN3Ba}juuhQMGAp{U{cB*e}tU>G;&IY8j5)_dKF z@RnqDLMIj7EM6yko5Y~6jiSmT*Jgn?q6D=lasWO6 zT|?B(7^5Ga`MY4PNPX>}0&Rs}+JE$V?cagcA0EdXi!&N+Y8k6wN8Tzmg~rUtDNzjw zrV`F6B@`L^-l8*NJaQIX0xj@qT4P>iMWCyDWCxhguW@k%rOZ?VkWi)pokRtREOP;- z@h-uo$slJo^+M%rp>0!KR-X{TJ5xMG8DJup z;gr5~kwcnoi0TR3owJB+18TtiQ-hM#A$KTO}m`>PTrYwt;0EU^&2X9~7 zoyWwrr|8Ip7+Z`otcs7gQy%AE<$J~}z1rPg zMLK)DJ#mb++aj}T*$b{CLn_Ver$vx3l250cgp)G(C?hvOPn%RTT}uB#;c7`-43p6)7}ou)${d(;t&S}!SGc4Al`*` zU`Qdt_hdwgB_;_-r8#X10J06nr)ZLJs7VAh zk5pM6SBZ=~t_8FEQ%!076POg$Y%&y|(ua@qB@>c}3Dk{60hQ`;C2vu}GDXC4j#>ByLtkmjkj zw*WB;h;wU7xPZirF^J_9bm-X-giW=J5`k}D)q{l7Hk~5hctE+y0sC<+>5(LEr!ul$ z5uhdSkx34?%Hwoy@ZAdDVxen=u&87vF%lUD+Tb#XVg!?84bH{U-O1DBNFUqOfFXeC zX;ybWgekTvn1ZB|oa6v-2_0NRbstmXiKqkfW-wD!Ykg(_#*ARo5Wd+9o=Sv|&?_qn zU7b!vB1AGJ;BM64TPpsm;9rf<4aVrjww|T6`J4WKinxdMr2kQH|Gy)yj|bVXq%hK7 zqK9e}blED#+gVo;GJ7$%0PS0Vx7lPw7~BCjrzar2n9^!6&z!P?Z_Lk3U~BMFHA&Ce zsHB^JC+!ux6Kax?UQ`8%P`b^gCB6XL5!{Uesv<|!mM;8MR4{X!^BVFXxPY~;Dn-k4 zy{>aH@;+*2syeWx{Bwi`Qq+8^z~UxINopbq%#Z;*DqFPhV1fbWi8+Sq<#-gSa=_n^ z#*tIO(I8a3hu#<8|kT6-(L_dDF zN{AnDVC#`%Ppc?y(zFV-k9@$evI%h!5X-Cs;_kpIY^PxlXB?h*F-wo*6Gw0zX@O1Xs^hyhdY?`xDWUlL_uFL_#eH3{`94c2QCDlt= zm1LH@ymY#}2nR?oRlq7sxpJ&@+zqhS{3D{pE@;*+yY4NW<8_{ZrHr#xVkXFgGf9bD z7LA8X2%*}zViKzk#1Cm|?Iby#YB&WqwdNe6+-=z{Dkn)Lkj5BzFqGq`bD7ZD>dE6Z zF354Mm8Uwk((|!O8GI+7X|j)7ZxOfHP-dg_wsIAik|`%h_8{*f7ck`|4S;Yqn_|Hy zLReTisfJ4O0j8Y8XHO9FD5DYvjY@^9hZpm6X7R{*1fdiVIn!{CfZ}la{X!&xQA)T> zhzK}MS0VlUyAzV7>Uk1NC8Up2`Ga($xvCGCp($P!*Z+o#UL8RNxmO2?R)M+2S;Ea~ zl0{9!0EU05I$17C@D>$>#V~7JC#e2n;)Hct zfF(JQ1hAhE_jX0&2XGL5)Wr~G78A)49co0G8THOsyEO442W^QHAwnYFT2KGwV*6t% zX>2)zb#yLLt8p_g7mnG{OC)feEw)B2n@7f80RUX-0SbU|1s&l?8=C264+5aKoQ1NB z_{l1^j*=KpV~Ee#)JXsXG{iXK0ZfSZ6zQ2jDO_hPSH3bvUI8^Ej^I;Ts@aOF7I#2l z)(O<%|0-1rM85%|-&g-Ow3alU())*+494Z88icBDka_1tG;k3Yl>e zSmDHw4r!W)X|5^B7FM`W4tUxE64X-Ytt08wwo=but4mkqF9X5Q&a*6BD$|fMwf+%9kpskX^YaAkC|(QO-}vXb8w}On ziig7w>36o)J~NxYgn6L-nu+F_uJMc>z_(WpDImfA+FcD z_``kav`60tmG}hVdIg?SHr7?=_If;CXkYgO)ATGL?|RvN56Um>{BH-lw&dPFP;mlx z9Rz%iKk;T{oBS8G69Wk1|o9h?Ks7<#iD0#d@g0+Gkoov$oHTbiU?(bN_wx5ZKk86ucFZ=$o(WU2yyKd#t z4HZ}W_b#HNJD`24U+!A*8tqfGv$(CboRr_AhMHQhh6W6Ud~|tsz3uB$jn?vu@sAJT zUjp)XU#or&Ip=$9@W&ZGW#2ow>POzz4uhrYzdXN+;2tE+@sB(0x=IfBK3TeUtIt%{ zK|8q8;z2#;WGh~-BKJssydC#P?f4~hiRqJDM~HWTf;QNY{O)b?%Z{A%ksrTSAE|m3 z(|kjIE1?UuEH_*CYJbO;eUl6JFDp!visHZ; zc;WHl=`AH!KR+9La>XSh{hDjh*w?ZjR$9Fv=W64Vn$Km%>;?9WcVD_J?TXI0j2X=I znEdpLtKNEE{SF?%JoJ1r#NOoo#l@&6knO#l5TZRyyDKy0=A;IC^tw!L{rB#-L!j-G z`>ss*-Jwk`Vh42s|dVQ1M`me_{%O0F8=*oI_wsrf3i9RQTGWETkQn;eE zWV2 z>rqVZ*|+W6XV7<*bx}J`pEFz+y|)11dff17>&r_4wQq%!XMXLif46^c)!%c>)yto` zcOBl=jJ14~?TOwlAiS5GZ`l}eTBR$x{?Y1*r};0U8-1&W&B_#EYeLuL&5jS1Q}VxMwfMkJenCyf<|h-mCku-HQ%^ zYuKM;3 znZDbllrK@c%`ad2X8-j86t4OH#~+9LQe&k@J5Z0q zcKUa{S@yyd?8|IC_~c({7m8|y^)=PRoaQ&kFCcKQ$s5UfLodgzs}1G*@6%?2O0m&` zBQsx0^e)+43yOTgz5cn?Y{G|o%kx8XnSX@; z5xcW(yX_yEU%g`YO}!MoB3z1JWc0uBNp8*UaoC+1zB+bdsjbi9yS}iYFJEG7eh**S zx0sWA*T*l7UUeeGC-rHdqOQSI&A?MWNfO*U3*=`}V$hDHiMrjhH+(djMi~buur?IpKFF?T)U!!np$x*Q#Lb z?yHnbhkxMNU2^SfPUiLBub+SJeDz4$XT;bOt9rKwfbZV?yml63ZRzY?VC!#(9f`|) z)p#PyHyZu7UGDd%y|Adh7o2b7FTge>rTf1K{&a1*l4I<;x9*L@*S|+hjn}=)e-quj zq5J&RTfXD9>g3|D;{%*GHzG5FY7SjH-oQNA{eYC)D7}k&=5Q_6_4?)$mCuZ#LIw|0 zzWz}?vvq!t$=m#Ip{Cz^dzI~cGJkLUW?8J==U)&X?*>S|{jsp_8C%xZ#J{CX?Re>e3No-Xnf}LrbipCsr0z#y%!#`SMqHB4_n_E&Su=c zUDRGhQCrQH+KSj}tI^ipd$ihGv4a}5#cHS(s#Vn9BgEb$)KskZ?&IiNyJ6NAD{E`vrtQxi?g5Q{> zYx$HET?!1JxqV(if{Sk@uJ|oN+hC~Rgakd-`3*yKGc1XP)Zz4XhV^O6*}^FhmF#Es zh_zX{u$PfSFp?VU-t7j9fx+Fo3^#1;w$mJ zt%u=%11gB=<46ycRhvNw`L3pT* zFm?Pxj!;G6VbG@kbO2l9dMOOyi@5d2$Y1TXB3oN4D-RE>8*bU?Z?VpdA=XT7l!YGe zJs8aIIR~Au?7xZjLGj}Se=j<#n_DJ!=`em4{@~nBZj+yC z)iqfqI4P7OZJobf8JbsXj^64QT`}aA8+$%R=PHwtyL+?)R8KwT&g6G-cQsf6o#6aw$^aN$Mvnpj&U(N~8~# zY|fVSfIHJRXbl+5^d2H9&H4&}S$OZ8>?WM36Xf(`;IjL0bubaX>iy@Wr@V#FT?s_7 zpYpe0;B56$0v4yryw&dq_*drDbx(;U)=S);wU8!wHb8ySYdjjiQ!GXcZh^C#>W&gT z(o(t0fjkY9y58zS!cm?pi>2QZo-lj0wQXwwarT#*=20fX8Nv+MfI1!u8&2Jl8kSNr z-$xqYAB+(dKkiCT2CAigyzU_nV#!_Q&5a|QQo)$)`~oHk?-EjEzmVdn*(I7{fBJn1 zutn-SEfd3PFYkM5=Iz_Fy!UwFf>F>qmg_rm6R-SHP@7Dch;+S_n0mU(YUn3)MNT== za$rOT;pC}cD2bR&VrwHmF1(2y2C4qZE?vDZW!*7*?tD=MtEB%M=J2QOCio?f&AK%A zMP<@M5e@=iwl#MbYA16-Dy~P2>SM+6jC~ml$vH8aS>q%$av>)uKM;04NS~lc3^-y& zwPl+oDY*hqzrR??gIVZs#eyra1Sz}?Zzt2P95teSmB?8D^axuMyo#l^@|QL5bPe97 zbIx{RNp7b-kMF0%0bBLGFxtE&qU^1JH?o#7OC^V4q9;kJJ01so*W>xq-C- z47>Jznuf;0H@q!SW_u@|ss!+=m(uw4>=>mWLVBJPchaR$CjFVMYku*SM3aA11o}r) z;+ibLcl>K_>&jk0-=z4IlF$ME-Kn0_WWR5bnBJOgMUA}iH;okelx8)A*V0x6yA36= zj;h!mbZ}QcxkGv;8Fg+o&3WA$=J)>3kJP_OxKn1=NT~bcYZ22--2hj zW~lT}-XfZwi071qe;3jGEQb9ZS9Hz5qEBq~cvGcq)AC@CF%W>B{UEPDAnn7iSL$OZ zy{!f{rWWt1m}l`sZ11$+3Zry%)KBxp+~XmWRlW}Hx2dw5_Ks@0zK)wB4m`K+W>#;M zUpZS3!w}VLer|Si{@lYlzr-kR4?90>zqWwbOWWhwVm^HLMe#HsXLg}JXbNvmOWI~u z!hffaf(`nB*FJp8q8Fin`a{>ZA85`$AVBoMY)Yg}6P zt*1~N288rk?cnjUJ&<$!wY_btv) zTf*9iDnNM=2hvx~Lu!O<4C+bsKT5c_mS&Dn9G98~) zXmP#0k@Q~ox!~#4XDD|4Mn5p3rK?@Tb_w;yOtP^0CjDh#YAE#bL{XLFKk>>Z>~nCe zerdtn@wd#gFu-&^B+dA^+bM+m1B|kpD?!M8)T|Oxob|~vKXlHzc z$;JAF`8}}A`UgDU^{!<=es@bhoJ%8X?xINgw|=2n`(3f(?OKt-;22=@n)KsJrFGx0 zuXilVWPBCECH6v?YQFJ#;9!`=iD=@mznHfgaLn=o6DDfnL_I0QPi(d{8QgGYZYcG8 z@q)6DK+pMt`~PIe_}U=Fg_Gt`JE1 zO=z3Uw6T%V?d9rR+wNfN!Fa&_zuLu|#7*__fZbi}*-`Cc)7BN!!If+vzz3c&mD?8Y z9b^{r4?0!1`rh(FRzM+$85yEGbU;~nc3yLI-OMkOyo@ddjjZ3sy}xm6*rkOZ{64!} zHj(Uwjvu$e6w2D<8NRWdj*Qf;xZ4*3lQQI7%^?i_OTn9Yr^a`zDJ^_j7zM*u&@s~4 zC0wJ!!7N`@O7LMA4{}dk=9x|LKQ=nnD1%x=1s%OfY|XcqFG@o z+4={4b?B4tUlgv4bJK7cZH2ZtnS44V^N8`Uo)(hbAFX$s3gSna@G$ys9T9HHu@#Cj z;}6wOduS@z=Rc{tn5pMJ_ETabPA1Q^@xH0PIwFx>p4zCj<7_-EX~oZc^FWT1U~Z}M z{3qjV?vY3f*ezMuD(LO|F~M`jYTF=e_4qAPEVy4Rr3*jKM*Y#MF;E+L`R?-kWC`@w zdCT5)^b3VqxR|p5W2?z8g3d!+L)p-ALTD*F-5oP8_(^{-h*K)pP(qicKAN)ad4)~z zm?YTJg~ZMUN8M$`o_(8Wz<953Wrhl=WOiONC+Qb{_XsZ>n$Mic0dP_7xE*K!^A6~r z9Bb(^)U%78(9SE#+_p{M__2#8lN(g^L$-pZa$g0&iyLA#S|Lbbe%CIj#IpCp4c08-@D{Q-C zZV>e{Mg5+Kt~iS^r*VFVBKstm?_o=hS4Z$i6*3iq;PulUY2L}BkW)tMW!f1kfl<K5a<#dFp5>kvn9zkZCs+n{@@n`Nl~p9rP!OcEO`8@dL|N5 zCX*wS7e1*7k<6}8*-Hn>)@CmxO}7o(v~Gk; zu1%PCIiHFW{#$-(qEPj@**GN z*w1zx+hQAX&w}i>OAjo?!hQJ4!$0iLK>2R+JoDpR@%qk}PU@pWy)LT@pgp|lH#V&B zKY4#~a?-*|)EyN0)V~{i?zWvgK^zpPN6M-Y!129a)PZ!v_gjwL`C{<0sajZEdwu>G z9Ob?VwAg?5uRql*6H4E8Y6mMI4w~l8BS~?=aRf#f2ej|2?zhWAHXBqy7L$soSq0PZ z#k4sq|B=~QhKKLRw-bq$PAKT1`bT{5o`$FfcdJoShnYn0zX!W2?bC)4LDT|CblQ6f z{Y&!Ftj`V{6MtIpb)9jn+O}{e2^MKB+spz^~M2h8pq3WeiTMCTyP<*(R0JujvtFRj@TMmff%XlPc z2t98wup4AVBNQVyPZ_grVy!Jhe~EreY#LXI1g3Wn+oB9eT?;kn9xElnuAh_pW1t2fzh&&L?+gdCQrbF47I;cr(-7oj!P& z=~UqtMIM)WZj>*Vp6X}q8}SttM7=8vr>nQ;8F1q_yd1YR-9&2}V9%A-hevv%Ei)GO zPfI5JV$U-q_6#u-Rj?q04&ht78SYfYkyqp9aza8-ITlP1%=Rt9%UhOjvfs!XQ`YPplkxV=B5^)QqM}y*UC=hgM@P`C>4;?S%i?CZQF~>vc8JV-G#QuVHDqTDq9d{ zY!Gy_K9uMG*54?tFGCYKe`~Pd*2@3#nKDjL@*}|?cl9zZI5!33$ExaFZ5{@h{=Q|R zdD}S^%8}`ljF&T{(tBTa)(nleP1`p zs{f-~M<#ohrcYkH@3KJ0PUGfsr}@%=|1*24j|p5Z^jF}UD5f5o1jYEm>^>5k9)GDY zO6HDk>dvo0wFHJPi=emKqu+{_$^7zJEc{5_qU4Rbcvt_`{MBC54PYfZ2oe%ivAezq zym%!hrTypSd@^(G(SSLUK+8JzWX+jdbC|;6P)_PqMyke;TMPZdYRPKDjVzVOVy9re zr7nVx_ez5l1^d@S4tF7>82G3%x~tXcwlzc3{;&<7P`^Fo5a;8Fo9x3Y4q6$FA#-f_ zPe>}*70RK&9Ao4j`RaJ>?HNAzqX*mwt`G1u#(*QiqE|F`IE0``-ydobuh_c%Vslr; zR0x491-v)pks4vf?ogBV+_A^QF|%*(?O>ZNLeT!S6Kj}S-=t@#{9|l_+v>p5gGiG{ zaB%Vb=RQ^H#2)jcc}c2ZaIN2~&%;kCq2XL>lENy`?!803zK~ShP=ufCXTUWlwx{$wo)Rn4F3@WWP z5f1h<8K{_dDV&sqbS2^JuJvBL&2+Cd$fF;2UMH1z*EIau-2XP_uAA_;YIq@B2>VG$ zmA+Ql*P)kA7C+;S{DIcvb~#^6>C6i8Jx!+=!{SV~N90?YxId%$f^y$;m1wiS$zt&Q z*$ThujW^0MH)p=-55|w4=?AG_{@c%L3u8@eQDigxnAJ+EDuRtskBtcVDGx)2%f9KC zuTTbdS90MWi?t=th?4rFUi-nk(7h-PqePgf_mSSxSy^8`rj(4t0$N5nf*a+h^d^bQSbmt~ z1_i@bR3(g8o1Xjd8}8ur_2;FT+ccyk@dA(^_A4i71&35A?{Fk*Uk3#mSe5(z$h|x< zr($RQ)HflZj96+zyg}|NN{U3 z8{%tCBDvEJ!PH-!Kf?a5wk!iX;3bL7z0pvplUp>~DolE4%|WLqAdAf_(>SZj^la43>&x$@ca>jAQK$Y@5l zucU>A>Jn4wsE2B7ABUmrCS=W{t^RHC-?0+h)9!Ybq>^?1$xh9xPPj^FNzNABkOZNz zMvj7%Tj~crPhG-$L$Hkf_2B^!m_Uv{O{PJcnLgS7QhNgsOVvWL`ztWhP^hud8Ciy@ z0aZ-ohl63dzRpcQC1V91SGw&Kjo(diw8rp}KjWe^;wPs5O?PPl&af$>^xROdpO=A6 zAE~@Ty*y`r)-?^_yTJw+BWUmsJ7Fnqm;j!egHaJkm*abC(u`>xSnj~g5@a!TUJ69* zrqVePdhnXpa8%ckJ?hZtHwmZS3IH0lR*otY?6&_r@UuK)=o7?Pbj|cROmg6ts&}-r zK4PD}cYNYHYP!JtU=Pus^VM-x0H~)5G}X>KK>vk+oLFQa^I|wixg}gXmmFEZ7MJmh z32&^d%g>$_AmXHbSP;z5t0&M_M!%k7T#YIxK>=`Q!AI@|CNN-*6 zcH#lh%OM%I4$C*+V&8pEIATJbf5fTKxHVP?yH+}IzPJRR_|oX%D={%=11jd#y9Ucc zlH5JCoG6hF?_vCx)X$-RPaao1NLW)wxEbk}Zd6c~2h->fb~^uLP8K_{H4N_^y%=v5 z+myq&^Wf+~e@nCQTRFH5BNStX@hU(Vz%y&qXSC4L%D=(06d54W$+EcFb4b?IuwZj? zTU}5q<7Su`6KDSB9mbid=cmu53@uh_yJ^cV>hp&dNb}oy{amu{Lv4<)s5h^3%LC-8 z5_EEnJ12(O2Jr_G_A}tDPL!5t77IK_h$tNHriN%N_xcPJ2(5+wwJ~*;kQK{tH+pY* za-|y~etA(ITPBjJ704xFrrQlV#|Kw40df<+oaZ}ilRAa*RpL09-ltqWzk^cCKcHD_{@+=jXcY?c#1`RWcW%pwi*Ph#qnC6Y89 zrB1o4=YwWG#G$lrbO8DzCm;IYWx29qM!xSZj{Pjk2HuYj+%Xg63~%P1ab1fEM==*DV3GTO}gw`rI>KrTAP-+Jd4i)rrj5XuM+v4L!tbM#MA0k7Ml zd*q*u%Q})_or3)g9*RDlEg9}r&`*X8)@&JxW7U<%~k4?6fiaFt`` zQ=p;xqewhr@i#8kPxJr7aYaY_iQ>j6s(BjY0eDmS*qG0dgWW&v9SGttRFpgrKrr_zL0dB!-s7?%a%z< zIC;6m;7mnsZ=+lSe?{dG-UgA$5aaSs{JHY7cFTwj=8NN;Fo zV@Zr9WVMN589Q3WpecA|?eUESE7EbKoP}!bSy_|LmlE%9rSQ8c5Nf#us(<%hg9u@( zFMBdpQ*}_e@b}*_0d*+>HHezchE?ED@bC+c?uuA?PmJVoyj3N6>Yg2{@h;z`N_5Wg z2D8$aS-NcbgveM~zcj-dWU!z3tWPQe zjnizP&Qtb|4*68hY)~43$L4 zhYxp3oNB;srViqgtuP1+%+pD4h-rp-Jo@zvquXX#e};Di|C?pEZuehDMk_O>+*s`n zN-nLn6b-4OtyzHo{1zHfr!EV4=T$M%iebe(@q zou_D~_e*c3?jlr}5#;^6*U5lswa@lv8$5iZ#OK3&SDrWda2IL~MqocIhGQrg*qy8&69&bvOXEl2IejAX6|4+A1Q(ctu?OL_$ z&-*&*;OZvBG26^*&%78nsrXfUMweM?o6K`qcdiUD12H%nvwI_BvdR$DPQg zti4Sa9D->CoYN#@M-zi9ukD~rzgDwID?O$C`D8=g(!!%g2t>`0jg#;8dHHK43ue}X zy@^{sCaS_)-C-k}_0;mz^JWY7+;n9WpHc;vGvLQdq}9Ppe~q3TGP%Xnhz8};{L^iJ zjVwwPo#9}?$-%9kV~i_{lFustY0DIxL`gYA8sY_s63U<{#mimZY!<^9Mr`zcKQA+<4c@oMDoDPi}Q-OtXZ zXq|%kOk;ZWv(uaMKlrl01g}gC*gbCdT<|0i#(Xk{9DCozN`l5SWNDCxAy&QB2W2(c zQ(v~CbAG^VPh7bk zPqA$385n?liUbH{Id$OTxjAqj^mTVIU|`788e(T`tY|Z@G}3ZC(Uf5 z$ZQ6g8XB^%{h{>+!HELA)1CE6?h3hFynY+CwMEw4x=Yy>y*)X}uOQuwDMSmimR8vK6)6hfmokf!$t{l$52u35n|XlUBY} z@EN(_0`p1vUUc!TgOc(3MJIRLt7_NtiC=?aUaU=ol7H}JJgt)A6{HeXIp1*eM3;b? zOn}xHvzt~PYlBHIFL9!j>7a8-BHwK{%znIE;gItz9+Tc~S#>%B1vcAoN4`S6AQVPMIBF!kiO0e;^);3 zN}T^8KMiyIi(>RUpY?rw+`4)|T0UV>gO$(@c?Ynvo2ap?@6q=c;N!h$S9`lo!C1Oc z(jfizR*kuWGg0oD8HVoKjW9$&CwIR^hDSkGh*HBHMZ3F}bztT19<)>b^CuzyJx4w? z`y!jmYo^rU(dDl5ZI4Qj$2l82g}K7||KY9tdmd|ti?(24tdD@1w)}tMt$U?W^U+tl z2+LqPB2qim?tA)*iYn$_Sfu)|=15Hi;P}ty9)NkCvvb~;YCNJ!zE>X|9^PMzKJ`U5 zp@zT9dtIKDz`aYFoBu4%o0*zkTvagLCGxj@8tJW@6oHQPDoi=A?zhhRVRC1&S2?qO zUV##eb8LFKJ7t*FvN(B1r?cJfxrko%+}`n_Su7Sci(NBSKfgTrBr=bE$z%-2lx3lM zGskDKhkYyKfyg+YGw-AMTT`-9kmo1_qt|Pt4nbo5etM!mo|pCYot-A;rIoA`fDuNd8~1%kX09PBb+CA3RNV;sHwMuGPlPG@#sa1kD zHZ-Fj#fNto?{MA;t7N3Er0|S>wMCli5^5~Ga*_SG^h@FLg@%(5w3?bfk1A!BDz{^* zYH5kBLRmCE!}*$=7^*JwB(6|rQ1d&(M=6pj)-@A`-EKB3Z~7-+L`s=2a!c?%EAg*S z1o@S|=CZTBm+b*(+c_}$11EGkwC*3jG6kKA{szrvlw~xRJ5iS~n*y6&m1yAC5@$zF z*5C;$oiWmcrC7*Og023?Zv91XgHRkFLP}mlgW}4X)%4FCd-?#6;0|Q*av64DCRq$0 z&s|kJ+VVWEgBSl_hp_X*3_qdU)P9iJfXx?8anFZN5h6asuiGv^e~v)*EO{mPn4~4Q zE-4&&mFQKybY0<zCzdbNzU8Z0~>nSfWzV{0V>_>yXc3n(^V$cV4R_I$5!>)F)Zva-*I*F$RaL*(4wRJUI?! z2IzS^&dWq?pJe-mxoMWB9d)h1N{8iw%_4)&)8*(YoLFlS8GxWAU-<{GxXfLh%NU|S z&iz$arXr8xmeL@?U9hHP<16uueKS*Bf&Pv!^Z>T=_s5nDLYJm#ql{TAff^b*a#DIm zr7id$U7~2)n?Ue>Lgs?h3NQr0Yr5U|kP-_*&mMjL194qeD6?w$D!%45l^xf~l(}-^ z{?+oY^Y=8IY4zn;KE(#wl#V5)zN2o_lA7*`N_t16Da~ze>Gq#CeSKy_+%~U179)M& z+TIsFD{=P~Vsuqrx)(2^F(`}5Y46{;h|du@1a#{fVsDK)L+5_UWiZ~_3rp`LHgZx2 z`lg|?3H3A_%y+F~9^3<9Tb9(SkAlZvzl` zBup2h?LyY)|YNH*GByEdEyL zIBeKbVvHg0oSEg|EoVqwAH;Z;R}6MpuPf~_sySY)2#)ROo5%6o*UH2^iw_MEPAYmV!So9l26;Nk6|q{MbvYybp|!N$e+ zlabp8WMPh;g#1Qo57i_so<9CZZ|)aO=Bfq_^aWkQvRq9i)jCi!nOd7gLt^-{7FUp^ zBm1^SQTob91P1E@Tfzx5h$9<1TP82C7oSB|0}-RtC~zzev7gj;ebPNrCew&ZvzRJtx;h^< zLQm)Lb7kMw!{&*Ae=L?6_tyII;Zrw4UEiu~&ZD$84psVPCEcC$6VLVX$Bb1_29#+` zZ`NdwukVNa)xl-)4lBg8DYt#{1gU!o8Hv$?zGvUdrums37H@G}2#mwa+5fuNJ~{1k zeZU_LJlp&&r-CN1P2^bNlil#Aa-pequM-{poG$$a2fPk)jnymt$x{|OPFOZ|Ur1#* z4^LMuAeK@QwLxuSc8(!zn)q_!J!&!8OpqiYs@E^=k4@H}UPF};;Lt>Qj&*^R`gRx?3=oS$E!2 zdwynDA9HeYqdKLoe-dduVyKp?WkVj&)VDs*6O^K9EdAPc(c_;r9!v84%TCx~?CCdy zAC2!$5?0-OnhLi(w{(4jvrNQibd4QsG(a*H@Zt@hIFMlByJp3#bp;xmz(=gC>q)dudhGHxb5C0xGJz|ukNjiFUmoC6XM=q zpSj#KR)B-977#aV%z3k6I6tABZi#(l0x!gXHzgZI8=LWM`Y zPVW~l-&yV1aes4@XW`#ep6d}WZ{3+b>l?=|Szbi$9XSZosf}Wrto97aD=~lB|f>2QYgeEHu>yYt)4J3Vh4j>*#%X+ z6A5v1)~GW~7^MgN_`++G{)Y9~;?7JQee+~>S$y%Xre)?U9hNw-iv3&1tSFPeyP^b_ zy2QtT)sa!hF;v6(Yr=>s{M#=`MK*xagA3C{x)~}3aZxNWEN|HFPE|xns3jWG(7^w6 zYO_>{+-g*^J}&~Uq({_FFIcLg{vWq=oZA=3r`UqLM0h=gSU1jI^zXy3uS)NJiXPIU zzR27XRCH&kt~(GgR}LlAtaNEPwEsII+2|l=`XRFjUh5^F4GTwDavkhPNuLnhNC%k>y3{}QsF@~g@(w|pTIrNF7y!}}a-JS0lz zei1pSL8CW{Qq`;CaV3RR4YU=c>Z$gtGfcxyA%bE)nFQRQ=2i6~Xv!;1NeF{bGVY8! z>t`PBW7OqN!rhIyA^YU6)GUv0JtN>=rBvDd2 z#Y_6oa5XV|R7v-|n$FDD?g$qmyus7 z9Uo+r`AN65_XE5~>g@1L01tnP7nhQ`5|{Bp*Ov9$$|U7$Uch%0G7`Q>`{WqC)cSH7fH)Y+Q{UgEfVjeYj?H~S`;WEUrnmv9*q z%8eM}E6Y_Gj<;|9?C?R5lXV#u=1PhQ6THYK!ze31r_q1KYsu$9(|Iy5ldcv$^2cGz z=Z-8n0>Y`0Uj!u>@!U z^VV~_wY>JF<;!ko`ETh+c|RUWywvdbz|T(qlyXmW#d-L6q+53n?)(2(Q#UOh|1(fl zpTzy>m8eiA+myp*T<^DyZKYOi(4wi05yL`u-%K~b`#Mlf&+e%us^dqrEc}NY4(4$`Ztf`1(uPm}^pA6m@z z{QMQSD|!{V0E_G09vE9CR%HL)TGN*dLY_iRmJ079t$L<3Z zAfp)7b};t=B8W%kJ*wqJ!#_#K+S5CC7(S8sn(kIG5g#}ylF?SES8>( z(egdjanw*#newmzBhrj_R>tjmv|gl;yq%>eeBoXGSzNc^pcI0>EYU*qJ-JBb9dhg7 z@((*!taJ?eY@4$_;iwLI%KQ2y#ouPV6+R||@d5^TGDYOd2flO>JDpk^H_^kRNz0M> zLVCzltK%e$7n@1*<4-I@?&U zm)2)?Xc6gD734tnr{nu;*sTYW$E{H++T{TmrXcxEQa*cW#E%H zik*<#ze#LsF{%UPPO9(MX=4UoS@x|l>B;2A`<{pj8QFE5N?4vD|KO*{9B3K)&)_3H z(t_*4m;VSrzFkJ_sp+;mhK!hUy2c}o16ur^s(#C@wjW92bt;=yl+9%QVZ zl^=wLM>HD}yr}+o@qJ9nA&S6 zm^mCvCfetQf%P_!Rfp2T=HFt<_G<>V8g%b=*}oa)5`;c2MEy8K&bRC|Y?cF_cb0k- zY*^ewTK?4n<>OOk0#u7@u6K~hzXLY}31! z9jn&)A79Hu0tnY2o2$#iSCa%X8g_KWjAf`P3wbs{v!=cUOradp3o4&W!&;M`2 z%fRSA8OLkjq3@HI+U%eX8}>&$)Y11LN1OYQqZ)3qBKspE0*($BVMa0nyigmiNB1VK z4#n?H|4lez2Tgr*0@3n;ZC98*FE>E}ft)+9)jN;(>p(^~H{I3awsCCV(EFA%2gW;T zXm8ek)ZL{%2;dT2 zw6}M>Wyt+%6&h;3hMU@_Cxh3p^{oD=CYxQTD2#RYp!Eq1R0h--)Y^N9FvSblLYOI^ zn+#z*DHN2JFf)!*{w{W}OL63WDPNf+mw{O_oE?N5?px#%0}GG=Qp@~UTY`fney6-% ztx{5lN(X3$a|CkcUNFk28VToe34ek*rgK$U%^Ng-5qdTwEAb_m4N9wA8rOhj{~IcR zt%-D77Vy}3y=`BT7x6faUg&{;Bd+sC$7nK6mMyupt^FRgDVPumvqz0{JZO01^YJNo zwq}%(^Lr(=Zr&GQKNu+6MLR(a=+i1CdblJYz6qfq%ZQ@+_iHt4mG!iW(1&=g8pkP` zcP~tcoz#t`B;&-ht;X5*PwBAD!tRX6SI94*-dR`BFp$0J{{UHebM_F+LDK( z5P!JJhwtj3?yT&v3@{B|icGWzOmV);?x7EcowJy!=byD&*Jl;#~bpJHqGF!6z z30W_}Du>#o?OBy9HMkvX|b#WP}ZE5m7$(Gwej~|Z8lEGSyHQ(2!|X%au!oPo#aaB*BY=es26fBP+4Crkx3>0C50pf1iBNrP zn#jAa3ZcT9`0lM=#|gIg8A`*XBC!!@vK8%E=K{eVy{O6M@Hf%i_MvQ-Y0eJ38{IPC zOWKqExEG*%1w+>%b^Gp`&u!+(yH%FaP#!UKK_i?b;Kc5`Zk(Oe`L@fpAN7Gr`Go0R zX*2-UQFbHE0tr>Pv}0xXNoDv#e&~xH-$0GlCl+o`DChraJMADHkMLWdIeezCx zNddBKq&^i6se5p21C_n1f&CLr?ty1p?Wz@od*$o@vCf}=TYxudz~WqRfKc1f%Q*`} z4jq#Py?E5x$ob(i&ahT*bPM6!yrpWWj^L>n5|G(=anc<4V3X7^6%xV<`9p{kGy;Bta&yJVT%1hzbD-Ev6PpmamtewSvFpAY}3>)q%vplLqTgcL(nImagkPc zM-I{1h0#ygn#d(ttG#Mg-~TfZ5V4tULD9@}Gr16o)IXAN$}SwGYAFa8+3@l-pzx$5 zZ*2#B3nKB^Pl#t<(#IPcmZQrZ->;G9$SQkT97+`UnaVM2S6WeIKntG&e-i@^*mNZ8S= ztdhm~$Bw!B%sl7GytP$7+`o_Gy9V&+zc~V@$a2U4Y^wAzH(b<;0@NLRh27n(go@wS zOw;(R{4z<8B)i-fk_=Y8o#ttTc43QWOxqv-@{p6$n7qfI2E9Ix$_z zG9CRdFdV%cwwWHm3})xcVO;)HDa%K`M^v0?E3sL!Z;qdEV+yp+>Gt;~Q_|bmLDI)w z{4|e-r0UVG^eJa9nH9ICE`gc8h`cKf2GVp(XMZ0b=Mym3O^FT`Hi~xY4RnSv7e&6# zMRNLco#89W+8u3lvK(yg6M`G$+HDLuCiyWL1?%so&E3MF1wXn1&!@L|=^TEHc`ZGu z7mz(`oIi|I_vN{)NHXQJms083PU+Bf$2yk!nWcSvQ-A7{-7aH;Bem-CV8RRuFPc<& zH~ArTX+-=3^lwyTI-9ujP8jQ5&-e;JI;62t_Yxb*@Z?S-nO1Il%htcU3T0`j#fP&mufIOX7wF$gX!?0*Vj zEdL2%1AR3Jt5+-{;~J6`khv-%IU*S|9uuO3Q9^QDY?SVykb@8=*Yiw7oxQ-?f*Y3+u%!tTPQ6bukW|Sos62 z{D)t;?)_I9JaKp4cJ{n(#N$`}TZzd8h;IO>=|qyf#H{2wynnboTgP^*0-ZcJmoDgc z*{|UhFMvtqC)|Ub;d{I557*u7y1&TI7N=VEXkPnxt7q5yBp+h9%UwmSnFaw4*l+YBOJ-G=i`}-$) z;FW^gP|@q}1q!BD0KL*{Pva*h-$2-L2?4iTBr!|Vg^E>5v=4Kd1n*$nMUvOZJa8M! z@Yd%OC$1~%OJf!e|v+h`{{RiNUs_0C9A6W=J`6WP%$2z@`_H3;`A!udGgl}5u~?<4&Mv9d=-3SRpMu_9TzEitxkUTC4bGB# zG+`pHm47|^c|f~BNN5r@HYJ3JH^;5hGw=ZpLG#%1S32_o{h;gDQAHEe#Gd79cnOdaAa3Jn>QPq3Gu|^X0vTM_P@T1HXtc|AUbU7O^Q4bU zr7K7*vnVxN7y(HDUNB9=;mSDV<_O1G!OSCm%zR`o*LHc~ zx3yNaZO*NHeWT=7oo{4JJh?GuG5DZT;wNlw_{2(wxX(sw73Oi7;(oymSwHY2I@B}C z19^vaYVhu6JzCt)kbIrC+h-5fbi&h>>vK}94s0r=^>cbP%#SsLenQeN0^uW)4VNVG z)}^C_>7_p($}r~%@>0E5t1e8Aqf>x4uGBWUo6)J+er~xZy3cyHG5XUC6 zdM900D&I^P11nFMkR#3^xa2hW68#Rn8Vq;Qhj_ zAlIm@Y0xpWA5KR?hnfZ&#yFDhm%4>s52Xo_sE@u+m~5O=uFmd;27XP~@Gsl)FMkQbn#VRs}V8-kcWbRJuHMVpNy3mSx+Q(XpMw%G{QZ$}U}=Yj`xGp;azo z@y0BBOD*5u2Z`<#&9yH&osv5z(sq2&_DK$0`S$%m{ttyOH0xzrJ0N&X56Sc_lhd@7 zC)vB+o5a8q;ytM!sOCz1N><4>MAT)mYilNB=Vaw7t$qj=G#X&dgJy;_KfMO=*3^ra za(}G)n99y6)jrjqXk<=!hybVe{n6eQsDOwVA-eN1>L$Rf(VA~h71oG*6HRYUl3)&E z^_AT-@4?)6jqa%0t>2v93w@(*p=tEu+&0^b?fL{=Z>v)~F7V0x{>AJDn49jP#T|Du z7Qa5uY%2PMtCZDYqU;NM;kzAYX-#T`IITALp>olLb&!Xt^Bz{neAso;D%7e7^D_$Q_KP05y*m{zKQJ=96wKQBlS zT$X04+tw`jkCyW9`C>-Y@)YN)K7TIz-I)5bgm&F0U>za;@L8W>pOnPT>i&(XOw)nu zLASVzusa<#(^DhcSrxgn<`H_buh45oyW4Xe$57v_+f1c9D2Iog!DZpcVOoVtM^=R! zj=Ij*{n(k1vsDw1!iy@c@%|GCI&3ZcEKD&H-m9O@9*Xa+2m~U%gd@3Ub#v?`eW;LCq9U8J=B4{vA#2??Ig(;ihqr)%YCK zku~oYnY`3`lL9<`aS38_aaA14rSf3A{La1>E4gG`)J41(1@`fl7Z7H^TBpUxa*OGz z$c?Acur=Q^0C^c;xOlk4##&juA13^^nZn$V5%$m>f6o!IZH>C#pV?yB^g5FBv@Z8) zA*yznnHaTdmw(c$?UanKsCHx1q?3-GZ{2{T(oPb($ir8>hx}?1SbF}Hi#1EC4b{7? zgZOA0`Yhv%$NX0{-t)FxO>!auF&gjB=@-?{gIiTdGMB;(t0zk6>;g<0pTJKfx@1Gny2_6WoqU=bc?>-y1+94D=*bmJw26g8#F;}u0diN<= zdl=a}!$;^8;L5AFSQ3Dh!bMcud*WpdE<5Tk*0((GLh3M}RC0Ob)nbi4_btu!;)aA5z_v z+LfGM8mJfAV8OV2+I7>#d(TMGvMX$?=5+YzSiR0XxNNUCbjW#I(jyHZ73ezBI}B!W z>1ZWs&(`8QBr*vIe!T&-CT_5JM!_d7Az9e}${v37D}4Ng<0EZ&9>m=&rurzoZtc+_ zv&^*^c{&{AQ7zaE_hc>cAa%bp-9wo3rulq{nyn?BOh`~eRuU+n+D-dyqEQ}7K?LcH zd|@WrX27M}>S|$nm(e3eKg*~2I}fog48Y5>L9!tpWN7XnDaM-;wxCbMK@?4wTt40lZPnl1ph&*eZ7d%!SHl66u5jt% zVu-xaX~>CvSH95-h!$HUH4(JraV2Gp5zsZ)%H2$oVevQc00W>#9H z;ptK2*UZ9k;AP!1?GG5`$(VxQ_x%Kib=Wz*L&3+3!JL$#oC7-N1G>!&>pshB+H7D^ zB#7vEQ^;Qc!(cC@vR&|!8=Rfb&-P_CNSM2qHm#2$C6T9bA^YkiFqSF0X;OM2(x4TO zr8;}%L^N3K^pBuSop!|yCfjoCCK|FfId>3}qd{~xQ<}uW@2nbT(_wGrEB^J48KS9} zpVRUEh&>tRj*)=8_#zi&7pvDAV1}iP~o&AvaXXBCmYD5Pv&0fc2c%(2D>AA|%69Eb~nz$h^ zgJMT4Zl>MfhOt*5C@R6KK-YUS#~+=Ed+sTEs+a~iE02%G>yN@XprKKsH=w8FGz10l zbid)%@4k`eY|qko)d1eYj>O7YXeDEyuIkZfH>lQPJ;mT_GT0w$^HSrSBHD{9r8iMN+-&xll2=L}&6Lpsf{KqFx zskQxTSQxDLsBpk@vf`C~T(}z(y}nm@_HGb!bJXuBN9(=}YVuumY<2K_zzo5e`J2dR zyBf1TBZ`VVHK0OwNdywj%mzJ5e2Ozse2|-P(n*5wfw0)QM=@qq(F~Ig%%`&{vD@;5 zpG%rD#CUG<)P`J#G^GE`2K1p%u{8Y>R)Ovy#{|by)*&v1dCO0f+8^5X6R+MD@(GMNc!UWGL~^ZwEThIAkn9-FPi^ppg*x5+=>)f_Rhj@2B( zmN6&TeNVos>`~zPOOc&`uC`?CZv*4ri*e>Ju2^JBZWoq?H%T;{FdFkKl-Oii7Exb- z9xn2H+jZ+Pr|Ah3QR~^I4ROv{ZY((pS1noQ>({w;1G{!wY0s?Ug9oHD;h7TiQlY-}Lr*72eTS&hy<={7~_G2jzzAv_4+xU&Dq8?Sl zDwk)8>JRPn2U*4HC1V1VbL9Ki&l1%$S0t&7D8v3}3#HXze-%YSPVXkZW$fbJ`*w!7 z$*XO-Z&utEJnqiP4MLURrOv*nuoAz?LreplL)VN)am)w9x;m&Wk>n*qHaRWPIO!x;}3Zpf4hqm0gV7F>V=W*s#m?ovlH zRWJN4;)Ekf2{wxzUJ!=v&eIARgJwvm7yS0Cy!!4x-`(5ZQRrcpE~?<5gTfIOe*c(` z0oU{gbffzRyrJD$)G>cjlvQS$$ol?5ZkWEh^%GeZuDB&oHL=!@NwB)r@ZpGS`%%S< z0mHLi*}?BfE0mYYYy`2AOFYm3v0oiM>ck~8>siPOYwLe?SM5eoO#DGMgR2f1Fd1|Gd9o2;TH_1bp5WJPgDt#G)aLfQN=KjXtAEz)U#> z2eP_?Z$Lg|f3w^doFuow9V8O`SkOxC5gzPe*?}=GrRo9H0m^n+JYRewnrR zJ^h7~0`q3fS^r1SEPWCx8sTMUjb)EuMT%+>oIRG;@BP@~tFlzXeZGeay7{s`N#5Fr z)g9+w0j(qg4_uNqbP$zN%7AW_va=R;1JTPLo+gR^s1+6?P>)CSh37UGMPA?DO)L~u z)I`%-4P9?RvAu+W0|m1uog);!eOC&(Q4Pr0&uz6p$ zTg9cmNdVVIht+nWtu*t)zdV|m<}(t2yff0cb_cGri<=pEon;Q>x~;o~2?LeQmEq$X zR_No$@xULV}$EA8^Q}-YPuP|9J(J{)@SyS`W}y z$a?7c-3EEBeCc$am&3m!yi^Iwig@FspK9Pz_JCafavi8eGrYxN@L8yEreX%e*y^)^ZYOwx-Ef zlt(~U>*%5jFFl`Uh_~yNxHcN#_4dnm2}I&b>HHJpBEi(&N3ztwe3U}nom=j?>kcdb zv)$iSvB7(p$ZV_Zp_>WVpg|XbR4J$$9eWd5lPHHwbHot)Qt2Mw-ZfoZN{!n($ffD4 zb!t+sYK+l1=f$mZQb3D+X5>aY56MS@>^ex~orDJ7XB3HX^}yznHbo4<=!S93_PX!h z$N1X{@yz=L+C8!PiZ0AoepO9WrFH*qj4xZ7BdZT6E-^HZ6uKTtX(>jQtvX0RHOgOm z)e_KHzEZW~H>L{6Ph3{6LX^3y+S#DvfD2^*XGRNgzeyb3WAGestBC>k;mbYdqlw~0 zi{WTb2%+B9d`sB3U4?A#VatsMQne?H3j39F63Ve=5`%u_A@p|?&HEsQl@$8?pX4+6 zMYzS1Y%}x|PFay6;1<%C5QM|al{w5L5?TGK))$#`p5wC`b4f0b1nPycUNg8X8*ZazPyB#1R(dI)JEdyPp*IuA~9zJV+^I`(s~**ueg3oZ9jp4!m-mlwL+x{vzm zGds#JU-&~@FR(k}gEkhWd~$-XBLy!xTpvX#T-!Gae=+eci_>!Mb_-~FE^V9^%V)?8 zD(_8hKx?zh@-tushJi#77@ z4yGdm%X~ztw_GBrsJ|PdQ)xsfSZj|yyI0f`0#H}iO5L*2^ZD>h9!_W22n!=sgUi;G zDhP{l4Jppfx)q?GZ&+%Z{PHEjrY0njGFanlt$(GP6h{3UTu9=hdgn$oJJ7rpa8UXZ zq5gvCKQ1xi07ehj}4t@ch`QFFzJM_1t)Ry-|uD;K=DBSec^1mpdnLWmo!9<#rH# ztAE4QAd{EfGJx}FXxuzdGHR+$e3F<18z4Fj@dff~S3 zyqi9kv169H{Z$+k{u&dV?4MXai)b@af@s5mRjm%vne*8@vMEB*aKpTr}KiV({|X_UIKSz&4DF4cQy zHvs@}RnWU(m3!eA$XL}Zn zz5r#sjzcSRvq~%WThD)fTpYc_1tsSgvHV|b>m0&`_)U$|u(=9^CdQfkA2OCj2Frir z*?@_J#h2^@nY903Mhn_G$T?`rIL#R17((J(kYlJUgn!_7U}1V#x8U-69-*fo6fVp4 zx3h-(3(==}z0{ZNw*P>r05VTaO-<%^?YEB>HXS_y?saCMvL`OMEi>w%;~?rFc&1W7 z$WUORIO^a6zyd4>4+7T)ZbsH0Prx-Fz800hyDz)KEu>hWmzR+jx&40|Le?Ma6N0fq zNLvmIPyQD>+fOGa12^x_k8cgFzd{5z0gtuQG~rPP*`DJ20bqlUGG?NinuFSmTF~h7 z0kEw`BR#e&`Y!4ihj+z4D2EnR__-YGob2wwTr+*jp{cdvRqu82{7BZb7iW4C7G##{ zd>%};hNSx#s1cCQ;+Mw>un67ICg5eYSzo*vOTF^b5U|F~1m1J|ibP0}n+S7>(Ohpq z>23H^p{l8AeXQ>=Ih~xW!G+Z*XyY;_Kbrd)dtnNJ156SB@cOJgKgwK!uFXL zp*^i!jyhF9KHmK{Q#;D*)mh&*m~@Zl#g+2%Qv93>u?58%876DkNN1?2hKHCEA^oqS z0)Bkf3zm|{7}3FU5$mU>%@AtVuUY|&35AT_^D`tHUMT62;LNvdZ65>s7SBYvPdGw3 zFYX$=tiQSneFzY*l$)W6S~r<0Ww8*9RSCko28&ur77+(sTxkcoTdX_O-4ET|q=l9- z^Q$pEhVmx|gpjlrxvzFhN-lrKnS&$fFLU(k=CI?(9h+T%lKekR?EE6ze&JTGPGS;m z)ud#t_-m@4O0(G2B*{Lsk_p37JTQ1luRG*~qX9P|%c5k9)zB%>Fs+LjS)pfs8wMj^ zi`(giy%2FZh^zxiCYTzX_T^HD5eFUPpUfs}P8zUmG--lDzg{(lyz4 z{GrIn++T4C3~`%|R~GS)Ddr7cSz_2!quzb~4S9G??%?foL8y>*%Qc%nq<8M?!9{ z1|8K7;D>*1o!#@~{mPrT@7e!cUf{LlJ0JuTTN7J-VnGHS$MTb5XBJe)nzhXy16vx@zRv$JrMpZAo&vCIYRaS< zguatWYTb#EzspD#rc3*hxq}bR6kj6`%C>5XZlH^OO?Q^z_p$RmMsshVPZqzBUb-#@ zdk>kUJtz!v!oOaw%;4*#G-Y9~Hu3QlKBQ~0xTXQ^bQS+`+BeHzOQ3|Y`#&#QQP%N* zIc(-}7A(D@_#JYRWzhwvl6UQbBxr*4D$P)=ssCIIu;zppeOU6XtWdXqnS#xLGrfMCI(%> zy1BtC^s6!>?N|vWU9vM^?iQD(%x`Y$T&n}s2zqVUSN*X# zR_QBC^EfP?v<7_^)K*G|i#$prej0E2f;~&PNJsG+7(+y{3{a>tF~w)Mx3l<0W%*X< zF#!;&11o2o_^ZGYW;P`X`R}hLuwP% zxycd%ceC27H;dzX9Cwte&OSv_kezAwAOT)iubObF)gKG@KucpCX|8p5{o03CyD@G& z`ZLV<%f<2Ip~%wOd!I^%?_kNqiK~w{%8{=(;$Sg!u5F0`-?DL_TSLZ(RpUp;35cZn z>_VbcdA=>&a|&-`>{LC`1El!MTQSaB-Qi_v$#dG7w8Szd{`}^Q{GKIv;Q&0;JF46; zaup!Z({O-7=xLvHps&@D68%2f-fQzD23&?)?(6X2bBR2 z`fz+V(^qy$S2upB;`lK%D(rIjqL!5^71004K5{J8W}gYOA&A*PZ(9X^@m59feJZ-F zvtEPa`DmpY7R-2H;uz&l3Y;Jb>?#eUz2UgCIO>gw;%cCtvTOp};Zgd=?<5pYrfb^V zBf>8%HcEc({j>l3$%FUwq-r{b7MNv*Y2_3^8`N>37y0QqCWWQ(npk@GKOnFwU2krzbLdRQlZ$D2m7j9gQi~XBI#SK^ z?OXZL_l4i9?}?*7+wd22T6#oIkb87k77g!_gn`ID>k0NeCw#>FM0v;I3Kro^ms_t* zeNk+Yu;;q3m$hkv`Hgdv3s2d1=HNu2%ZsmrPaXx+2UX3w;5%PfQ$Z^~Bhk~kB3=>i zpNonE71kqfnv$(G3oRgN>Mb`J00Uv1Wrc?)?ua zN0Yd)hyRnmE}?Mppm3hvxC=Q7)<3WRx3%%z5GS`} z@IT_j|7vX%%<$Gvn!_7hQVH`9F|qp9V`9~9LUtc$vcWBii-`&SUDBEEfhl#AZ2`BT zUVoZ{oq=-%&U36`wLz7AhS8Od3Gnc8{qzKW8n^P=@CAE=T!F6VBnr`g{=~H6qm(aK z;ok329fZ0^-eNVL$6ob5ceUwgLoAYB%Z+AQ`%TB#P!e}uUG1@${NDY!j-Et!%U(V- zIm~&7D;N`g%~uNhNjN+c*MJX=MVSW|lb>f8$adck*iC{LI))#xi;##e_aAy`(lt=@ zcv%e>BaU|}Hd-_m)9@-n??_gq7q3}Tk@ZXh#1|{5B>l(|hYqkCdwm$QDeR}N!+SGM zNqYH&Ar8_ZNsOe&*CK=PydyPx$uh z7^y0eR{&4Cou%fXWoOtS|44s*K$W&@l-S@k-flA5M**9FwK4yIU>bJsL1`p}W8v>O zjo?|fA8!ar#|PaZ*S`Rpk?Dq`F)3YAb(V7tsxDUCGTH{&3CZB#1n+r3uaEBqHrGz} zNH3Sv{oc!7e{y^!=DbrT&a73>WN|3t;`^TucLt0eaL#$$UY5b7trZdfy*8qqtJx<+ zmO-n#InzmgDD1FyF|J^x&AMwBs%D*Tz7OfhUYeYxkR0w{+{>Fcf5^*9PyD?c-kDlFe58_bn8O`gJ3}{*e#$a>)`diq zf!XUaO)_Ta%BJajk(cNmv)l16+i%K$veMd9eBZAO>1LgUggZk7IP0#vJsG#J~KYq|bt9E8NJXsAJbr)Rdnq zsP8GESO8!`r~)|Vi`_Tk*(E`ycK9dF?C_3lCNU5SlH>C(({*Q*`zxSk=zyzsSI2}#&LK!*?REhXhNGyM+o_Zv_C9@>oRmcZXEJI7;+P-9S2qimCFan>fXP<63nc9 z+UNXhupCSBlU8*B&Y6{V4u@nO-T;4!6_lq@p+AsJ*%6t(Z1n+L;Cm~iOzQ`a8fh7j}SNwZh z--s{-X6!-h3Yb?L9;l(~`9Se8-lP^I(q;aDD^eFXe^SIh28{MOHYE-NzxK>ueKkRy zGVprAi6Ilnk~4CRb@IKW^Rt9?F@Ro)c?f(Sv-rm~rICwyay694L@^~6`(%2M_wF(m zs-qw(t3IAQm<>t%zzjDbUX;qj>j_t21Tn<@*`=1?neegPs#`fWfc0w~vgBPW{7qsd z|Mrbf61`9gXfI0{en>b#W07XS&E%}{+RvS!lc1BQr?brxnZNr_S6S62SR{xQIV25w zC9{{_ae_i^WjJXEc5*DvR-kPgCT$YZJ#JJJoY-_YmQ9h+Bs@cRdTJupb=Gh9!Hudu z!Vr=@*ekV!AcQkN7*t}A8vih=Ql00*uf!AH5TpmiQv$QZy*o;!d6)}NV&#j%f6e7T z05P9?&&sHCzn5EWi=Q1Zcz1pQ_s#dzr@aZVyO<(g36YMU z@81%FL&^6cL00md*ruUMNVjbDn!}tU>#-1z`SE741+iLOR~9&%=FmzWPv@pAL}luo zoY+uIqV|q;X*I|v#|fq)h%g8FgcV&V$`gIpuDNcGv z@G)QQZ%LrtqP#m9yymwjV%KMB6P(mb`mzMb<8Hin>2j=sDGJxIJp3^D@pmvoz>S>C zlLV}tD9w$+djk02EJ25cYGL7E}s3pSy zKqG@cHO(s#l#&tvv_;UesMe9H9=KzSVtPyh66n$MI(z-nc>Vd)qD%t{+APX@ChCs- zoj}@@9RtYk>F(f`1xG(FFWp4`rZulh)}yU|i64;fGT48cZRm!BpJU=0(fcZk`}bkd z^-?oZHr;J#j>n2JlAQq@AnD2R+TqT(H-oOE7J5duJ#*AZK5udnQ|r)&Vxs?>UT@4g ziV9I?a>C$fy6tafR^eGbhR_Kov*~K#-v>-UwM(NwrOtZN38dyjMpVsaOPW5+uf3FTx(HSU4~6=`S13#N3UF~?W0<=Zj)_EH68M6 z6Hg|Ky&L5Q4|7p)acZ#krXXvD=ck++y~s63SbbX}vC0>K;=;UO;aK}DqO#m-59^ze zSMq1Y!Vr?~k9xD)t4-n3{6QsC*>@YR4ZpEnE@${e)eBh46I0J9qbY_5glcQ2o0GrA z+iNbRmME6reYLX+UaUJQrQSYiA|mB=l>?|zEDq9RGG3Po$F1|L9cs-@G6mC564S@m{vZDUMazv@(n}W)0qA6w)r)*saF3ZEXYbfL==GL%VCcv)8zS zly=`=LDK>Q4mM@o^BDHmtmrt>WY3WB#VzF zd}sA1IudO{86%>?*fK z0E*SAi{Ln6Nt&mP8yT|aVGq7br3#tANAz)cAKx9l+te|H!{)1;hLY~jeMpC2xgc;#B zlLZ^$_UCx7xhNB{3xjaxKoS(ko0Jz*8uKbDC%uOhb)6S!6rvA|hSgVwgXeMX(*9LD z1v?h9BC-Jw`GW?_eetFzbJju(oOmbT`V=1u`s`WonM$#3&j*dHV(wM}SFlohR|SkIf?44rIgjbUiv%14)-5S!clUn*Lz|% zY;qERpkY^km!GV(hbPD{AfFvEXC_o#7gvG_i*p_lXmt(dq)i%r!3*JZw?vnP6e<}a z)0Curjw296xQDOW9Zz`w#a#7~&KSO$a_^5^{x9qoM8A37;l-7__Es32!I-LB1x-$L z7#=*8aqf2Lhz@uIM$dZyn#g)rgp6`}{jyJmxvV|4>8M4(uP+nyY_Ag$OKA0gBsSDDPDDRR@f47VMYi!fB899i& zoBdYf&0X>=gHeR&c;vUsEuVo#`oqWuyqS50Z^=}wn6)!%8tJ)71<8DL^#GJ+ts;%5Ao}jKr^kD*YVmU3}zD$nE3Q zvHF?@xR5G)FheoxdyQQCmv0R%AwR;vLOR?s=2hQ+`6l-=}%LP~GC zv?iFQ)09wTuC70%!zgz`WgnRGTr1d(iro^)c(MUyGE<|_+b2AbqhyoOfCvF{H>n5@ zp}PtBW0ocG;K9>01=)g>e9Bi;&WTS+T^K0&$=6s6jQ5gE-*vNOmE?UCE^Qf0mM;<< z7HCz9F~fP|y|Z`f)UdOsOwd~zUyzb2h*f~MM8C$5Eo)f3|V{CjE#Y)UL7$efD<}cgB?BP3xr|B0NXk1F*v^&i_Ta z_q^0k4LX=QFo`+FTB;Em%`SF8TbCM~YCF?SUr)Al3nxTVlM5gaxB`co5rir{wkmh4 zbpsA%$!)5;RVpoYp7!NcQ zZEb9EL^H|wNh65`-aEW7OU>s(wsnw|_r)MQ+@@1Mzu!$XsvWY~$sVbnnr-zqo7uj% zDdl;;(RY*2vAYbN&Vf7?tT!3uR(^~#Ta}ALs+%L#mw2(&R}f6s3lH39YIAKGzSakF z06~9t-H=k21u3Psl*|>mcnlViFnEK9HUF-6Vw!ur>d*uZD-`)?5SD~_Vn!+Aug%*F zK7Xz!1IP&lo56HUt>j^(8;W84Dt@6)ntxDEGj2>y7ZvOZX8eFpH0z=qd)xoJxPQ^Z zT(^+6x>`V_LZnw~xz~@H^MrBGX>Y}6_D=JN9VZIsZGQdAgXac@{q*P5&>a0p1Jbfo zV@?}6=eDB4yEB1BgJV9^8^P!s4V|dzXvuyjQhQVrtp=F!OfnnTk-Y+uTVb_Y|NX$ zw*6!rlHn;%O&0y(nGki`q-yG$ljWg*5#i4-8fv^Y)UWMf(K@jywtZ6}es~^^USbRE zPk8Pwb518bxrspjy#zPM8%!Yo1O=7h%N45^_j^6jPOLI}E1$7EOgWAdipm$+oI8ac zhB{pvtQUeD?5f^46RJ(y5(dMO1v%^slUAvFOV%dcU(d-o!8jQ|>~F01VXf?vdcY}} zt*6{Auv0ALbtXQa@AX0ITxLwV>X!w2+H>giB_9;+oT+Md%wto0PoLRJ-VuqRFM5t6 zPEPG~Sk`!DxbFv>(%x3{)3fvLG0mQI;VT9$Zc|f%GBZ_wQ0c|HtA+46ri=TCQ`Jh& z4|8wd@H&k1HP|}XoWI#SE=rs-3*JV^*}zZou|0(_Z+d=zSlLVi-?sGDg}m%Me{V2WfL&XsW?ffo?x35wFOPT=r}DQm%BH#NPXns(`LwYVacf(UHmi|wV`ins zH6X_|(A8*9-D#}RytXzC#7za-)@WrY<9}t}U;pD}3G=dK9k3;6KMwT#3i@kcXh7BT z1n;k@r)YIsDdo=O`MpJw#UjHt<46XCGkEkXB=UsWO z;NRt~(TxjftOKR$83&jR{@{b#5VEfdo5`6#W;K;wmM$nQZ-Nu;n9CqkeZgo# z*#H9Nq6WDx5KD0@XL-RWXONksb30K7(JNE&XU7G9fpMOwCZvO5<8E;!KNkGUd2ZPZ zKF!{Gs1>h;WAZZmI>}?iSe>31GXc-njJEt%d+a4F%=lI6>3f_LUr(INYmY--{_FUi60ulj1bKVS3$)k!=?UwRW-9iUFisy23 z{YB2E8tQoh=tIWwm7fGLp5W?VI?!%b%-zPZI1ge9X!b`=jfg%FTz8B}PyWwCxQ4XW zf=qCh%DUzeh;fSkM}$@|!&6^Vr68#bos`dU*N~ZDw(^kqW*$w)52?b%b1^=F zxm{gmGM_oJ%F^x%YI+zAMfLDzEI4LfUcMEkHrLdRjY9)8qYzBLmd72-B6A3QR~>$I z_6$`|3;0&>zRy=rLM|R2+#n&;mS71Cro z-X1oGvy{JKSLQI*heX-ppF0~_bLDrQ-yoVfE%(^}M2(OI%dj`&9?!LYGlYC|G7*e_ z3qMDWImteg9tmlGEKA6e=I27T9sY~IGP*8g?rOBEd=i5W@5iClc4Aj{p6CR6vnC)P zf9ikNfR|o@tJWtx8Snx`S!HpG{%R~Tb0dO9HJxDN0r&~*cwI=j6LuNJfL^WpHc4mv zGx`y~qf>~i{mbD}^grQw2$79mbyG;cM4sy&qt%f0ZG<1OlP}B=GPD~Xx|E@(cNbntp9OkV zpI}kr^9w@C4h|I3Kb@6cOAZPdGfl$J&K*LgoaA}cqaY!Deh0T^^N$J{y!9m-+Dgx_ z5ar-7ZwU{T=n%CPZ>-|nX##|D*?KG1wcy7ld!>-szhFi3GM;FY@K}h~VgBmV`(SA~ z-qQ|WM7$Z|93-L*b%oA_dl4YZh#S2+>NwQti#JYogu{M1VtB>v`$`^>cUk{#F+GBOhB2bk0 zssjI?MW6Up3x%v*4+9q!a&Cdfg0r?@b)hdXp)0eRA!YHPz3vb!?Q6XR1ctGH#kCDW+Kz%T z0u0Hfmeuf;kpLbFT?G{4Aa^xXM{7W3n&wjuo*Bm(J1>$BZAb z(IxOBFcW}X&Mw$HU;$NlP)`Z5xC@6T4Sqw&VcF?Imgd49)#0Ehshrbu$NZb;VICl@ zIv-6+;R?kSGClMUPB&Q7gU6;wbbOqv*GCB2k#!ybIT0!-ky9uOO=Yw8fJ+!h~ z<4>fP^IPrW%Wg-15YpI2hnG%MDXi_!3YmYOe6(H0S&<&b=%7IV=5!d<_c7TtJ+F-D zZ^UYaVXCcEdh3pw-xnP|}sdL8P>amCZUAh3G^9y9xsJ*~mK`$4#R zs5RN)Iuo5eJ|+tBImA)-l)7e+%PDVd1TEno5ciyuM9t?obvgYxw)eXXx62;yV*N=t z#AXUwDV@l{kefs#Xbx`&j+`%K;ax0pdrm&OTm47KmiR4rZ!z_yJOMV-?k11i!ts2m z7L;ZVz;B8Fp#uzEi*F0(?-x>~24iLJbTWZg$(mGsOCjZ_s_|F_c8!^jwwa%8Ckl73 zu+MgEiC;l+)^N9vDDiP%7UGu737zq zmHlIX1@oB>*qFMWbLa1em&TWX-r^-NPqY$dFuTD1@D&&cbLugQ@& zTiL-#zK^J>sKSw@{7!8HMQtnTTNKUsq4sP1AHIUK8aL3l7&)VxzZuwzd>d?sUm!zk zf9G%0#?BM6V4QS%?`a|PXAt+yIYhi9)(Zqa1tCr!(Z`N=!#~R3@U5pcSZ%pSA86T0 z?Q0LGMg$Kai+wCWno9<`Y5Wd#Q1MEt9U68sJIMYy>0q_?AL@&xofN65V8F^1Z4IV0 z3E629#ovY@?bk@$`;=TW@Zxv+AbRGoQ}`+;dX$iH-vsjZT`Y=QhrKcbYEZH-s4%kO z+^F01Fr|7ht-#qX0e#Wpt!76G{h8=!$p9V?? zenLlGYYtN-`!QtCSjA3=1JgPkX6f4?$S$P;MyeT_+ z@@-g`s81aa(oruQ9^y$iIV|CE91b6`dB5Z<{BJ&oH4RtjXG+dsh1cTq`2RUQ-jFa* zI`j{GrPaX68Np;zUVkFeSKSQs(s(i(o(VimdI1E-yCUmgTWG)k-0vHdxVTxj|XJw?4VeL=GG(Ov8a9Ipk@r+W>_;w zn6rx~@r-wR5k5&m`cB_NyrS@xfea7y_a7`|(kkYiz;SDd<4%~$a236@Wi}Z0T?C`r ztztsSuc0>>a}}-n`UlMcDn%;epq_6SRmN}~|0r`o+rI8%(Aqp9LqEblQR#FP)&-pC zvJV~dmhgOPBkL`$QqlZpj{cu-y{^%#)a`-2U3xgZqJ+04nvaDnZG#!6ybI3#mr^@y z4d{8ak3ekDaQMM~0*Lb61iB1v63rn}jJ$U9)_)H3pn571G(RUzTQk9SfDzp2(4V}s zDu*S?Yw>*b$I=dW={=GaOn+%jRPETW$kr@7yipYoijr=!-uXd2)+t#THya(oYv`h@ zKZg%ZOo@p65}G)jgRzYk;s1bLu(8oNCaI+T^uW5EAbEn9HP+b>wSIpxr>7^a#;y*s zzHB)t>RL&>($mOai-GuaObZd$mJ>DitIU`#o8e?nN2!bW2@Nenc$=jH^E#4MAQ`VF z$mzcN&!Rdzw87G0j;zcXg2xJcIMMbQ_P(`*nv$#p^QINBf#)vlN;iBgd4?G+We(?d z-06p($FhUtNg}IpptJS5%l$ygW-4s^HO`%~fsAlpNyPnr#R~s<>|m&4uF=0iwRP;z zxe1HO%GWBfDExDLQh$l{u?y%AbIzklcQ2@m3gbl2>oBB`JFzbNP`W<}f`=z2vx83& z91=LsfhahJV?_X!CF3`At^izmZYSv{GvmqA_DW{ze%}%Oa$mZF2rb;M$%HL)7vs>1 zIM81HfN4~7AI(coU{}pOY){L=cDrAsoNai!5Nmoim}>p;}^RH(#ItQZ+dosD-QuRZmodTmd38=Ao$MZWA)G#`F0 zIs)>2&fw3c!Dy(P!tGP`9x!RYiHMsirE`yS7o=%}?adwOx?}HAJ4{bui+8wC(CmbQOQWTyM|-3l)D!j$14tHw(L^zNL&1M+aZ_Bd4gV&{0{Om*LW1<7A7W3GDQUe$W z>7w@8-DB=6{+_HaFD8R@dZ4TRcQCBzWF`uHiBFe5;ONF3vwnu347R*%}|F z51H@{f7|F!c0_&84S-b$-X4|$kA*&GpQx8|A;We{mDHPB*0_fW)WeD!A&W!sYTp!D zG99hPUH~&CZfILjMbFdqGg)jm1w5Nb)geX3{GL_{s~DdpQq@!Zf35mMDiKalR0q3>I3*WN(&<;Z^Ikm?&z!@#Lmek#3J$uySbeJOO3h2*CAv-u~2hv@+@*n z_W~;hzE0NE7~)$qOFH?mQFzt=Z!%7{7>{+_VnR>s;O*!&?B6$G1qIW|_zb0{TiM{x zj6=w>06ags0pwf9!pYiNEOK`O^QmSST+vH+C;Of`LOFs=luf!$MDpqPWQEvU{0RS~ zFJ^*{dyhVUYBxGpT?NnCI&|=Q3biFI=k3CqV7_M>r_8>`?+h<|qMHWKB$(8*rxq5< zX`jqr9irA7Tf(BwOX&>^E|Xs|-Q-=p0^hp+NM!TJNfU7n9h%z%5LLW`=aylYg^WH3 zGYrg>?pVU`WW}!_TxU)f7o9>6pS}`qkN6k1$V%vMi|hESByXuRuK`P=l%O=@C+W`T zclHT3!is=sJX5upX+k~^b~Qdpg|d@o-gxJALN;?brUWQLW^+*?oFc(}xeF=bRX^gRUv00ndB1qDiI~tfkn;TQ`dB|4+o;AHco& zE8zUV1NhB)9=4Q7y|HE?aqP?Pno{1@J;7JCmUI+0Da1?pC01C^;z`{W{I7o+2J_WP zcce#Qp@>|5J6Vw3x^J*Tw~p8)mod2nu7thyXEOEoeFacWhM$9vgG2up)P>|}>~C^M zn%uM4ZP2a#SY$zuXS@&{;*055tG1G}8U;O@@fmigoj}hs;56LpFkHzR8$2Ajidi^l zQKb|+=7NF#M9SLA^?wn!Eu_}eCDZ2^Y^S?OuVU7y&w}lTsm>6=K_IaI5#s3GA780R z1-;o{W8+9aY_xNxc4T?OLP6Ka`sy!vQZ$*aY0^k)t85JUqMAld)%`XEpEuJ+SHF)| z`hPMjRi^!!>UYJrD6%wThy81?m>$urp}Xxnum-Hmy!oGbM$Tj&g@pF#&q9XXYJANG#Z zQja>isM?|brUy_i=Jbq@yv|m8(Ir?|!MPSxQ%6*+?&%x&Xc! zUd~+Kch8fzpD1Y3n1vN}U_NljA^z&&H1P0{`;*XRbOjmjGVhplVnx0?XXWi83sThF zH+EKo5LbI2A$_7ioi+*En|6@xd$%yiP|iW0U*bju!gyTbn}Ew0rSq$z1r~W)AmS1ExX~@)p+k zUZ!>wUWAQ1lkxfZTWD2yvL0{rk9A_3m>YBU_#|_P#Q6uUPAsAtQE9Dx}mDU zt0m@G)OQnQIF|{+UAmL;IYlLP@rK#r`*gMgKMu8IpV!ay1HR>Wwe3C=O^O$6sr;4b z2lh&L)Jb%1m5=!T7|~0==k)*Fdi_R|P3l{8o`I26*cb&vj4!`3L{guau=p0{(|7DlJb3-lB zG2R6lgC5h(*7C7VugaM^HL8>7_@AOLvUKEL%f}FYbKgh?ImE*x8K>~4_a$nvtG{$J zoD6HpQ_Jd<@|1FVZC`a9hd(k=H;Ak$PvE+@IQJHITA7(pX4i(gzV8^wdE-y zXu6enDaS}LT1s{lxxwxI6(ISo8+0src4!jbMLxP}$Q^%Qddq@rZUeFofN;Ae>W(jj zs+TU^hS^Re;)P*E%|4x})AtCwsY-Yor9zYFSJHjUM%XX3SNcYLn27g%#nvQ96Q3L` zwH1*C{dAy0-o#wc#~Z1C(Cs+-A+vA5guw^6`|?cSek0b3C;nZ;G<+82-1x;x~e`1;daN2K*GJ z?YrBlJ_b(8!--eMKJ+yChsxE-XJNhYBkXU~%{P7ZLGDcS*GhMZpCF5+uXPd5aw4({ z>l82ev-;imq3Q)sY+_-*9DUf;cC6GN{zGmC{mJy&dYQ`TybI0=-b+SUhhbNX)Nhzs z(&3C+cU{Qb+n{m6>)@@j39YKXg8}N)Ftc|CxN18O&huv1OS>^Q^-JV;`%}8~JAq+poxtQ~&{FsjSjrtv zTzqFxDKh_rrSrGbcXjYnFLx((LH-th+Yp6r&bv8LW+7$McW{xr)R`2j@tFBtFqYiH z#2vZ>OY5xBvvwPG+#?h=CB-Nu8eF>nn&G%Ls6!3#m1NehU|yux2)zfVO@?xl(;m!eO@ zNV=n7Q@$a(oZxGSPc1&_uckJY-e4b-U&vY0;oRyprQwZ^idSJn@>zfuA-Va?qFxd0X&@k zm-K&%R`ZR_MymLpMR<770(_M)5vwbY(E-{#A->Jtpw9h1QFE%Gb8fF>jrMhZ*M3AD z>8oPK(zn;b@2;bd2Y0!pc3ewYl)#h2&clQb^?ZFWtffBAok87{jUj@m;gGoM2vMuq zfj>jfN&mYbzw!S7So*;f`6U1V6#y0fmjM;>VHLcs-1ZF|6{r~%02Kxm2mk;diwFP! z000006Fr)DcvRKa#wWw1OnUES(k5ln%cM_w@4b+O1V|?d9Rz}*ic%B>6anc~K|w`| zAaE%c5EVrc;YvV6il}%IBnaI7J9)nEpO-u{=j^@qTJL(-T6-8KlgYL@=?Sdcv_qiT zY9E1F`SAh`_x~tRz3erC<4#8jtZMm>K%)&_0zHz7c;?MZ0^2V{2#mC_5a|0TNua&{ zrvf`SlneBKWGqnsjDbMgoW%m=Mpp$c@LeP@c-;wsrD071b2a1n_8hCVE6fGzei#s_ z4Vxvf4A5Yv_T)-V8n26?h?(%<(48ayCI zb^c2PnpkNB7Oh()&?x$6fyM7}7UMNX1!l)>5jebII+nMwn*ADf?9an*2^!8@pZ$@* z#^2`(%t@LkuwXBd^_br!(06i-K-Yu4@Zk@##<>T62K4V07^i!W*!8pW$P=k>L-(z~ z@}2nE`Im5kX;Tu2`+NAR;C>?ZbbXHxD_3LTIT)KAe_dekQcmif2ggR{a9)=zxY0bo z=U#o!1$MM!o7YMQIAF*p)+^!!4r{{C>Wzm5PHTWabH0Jq7J=|Be;>yuEB5kpDT%|z&@toISB}!uEh_4Eab;RO6*;i)WBe2z_ zKwzZN=K{xSIbEX1d`>j_AkTDTNy?J1@L>l$Y&wSLy*$uE!_%C)$DdtnW=P(%YvOq& zyDPT9w1_RN9QYO*>+$eae7>4*ReOkd>k?LQ>te^0F?cE97994zMy#VZ!V_C4BMY+Q zpO7YOsL)|U!a<@L_7+;*TZ=}ujv}%ey*IJnku#P*w-?x`Ca;H0BKOB;qUY|DP*Lwk zd>%)5WqJcoiJ)O1Cz}5Qj2$G8g3O1*jUVyt1bG|2dd8aV-{W)7CHSY}5t*(%K(-dW zjEZ_J!P?Ya6%YEU@giyX!DzHTH2G-h+gL@{A2RQW>&eE$RkF4zQn1u&w={Eas;({Tv zgg^0Ez}0^QMnyj%`hD;-yo&SY?;wwiCb6>7Vm$AZ4fo7GTq&^7l@%)1qq;R3w4!bj zxe%L*PjX7gN9|KI)8itw)vD}w_HrQB`9G6is=}ApYgrj_bma3f8R$>49`+|KM1|UZ z>lQexl26=JWK-o96tcSyUlpA|4N_9ck@zte1ZGTzU6~dx0_S+1#;PLLanU;>&^H-B z#9w?S(C#z5I%0JSD;LAYVa8O|Aba+*{h26u1;fc69URU#&%#SDQlX;sSxs>k-SD4H ztW}#?$D^0Zm@yG%I{O;ppYCHkSvLot>?L?<VRLdKj7XYKK~I;raz%N6ke8eDS>EC_yaAHcfmcq%k)Bq(PYOs zXSCYci6`~02MEk?icWCQgRb}8KX8~MRZMi zon*SToD8bIO{7B8$YPuA{Owatrj%*y;L_Wi-Qna?IQAtyKyHha`|~8N8xIGn&l$1i zE0O{)gUP1TV6TAzJgmuP_g9{hHHPa8VQJ6n0*hC1mfHEe8~Wf3s~wiu=z11k>FA3} zCa_mdGoFmR3OD3mQ=Q8?2JpZqQeSrr_8QzEJB%;lZS4j8VDmm__nLss8K2!3s3|8p zDf(1NZ$o;3;yk(tSKJ`8)caUq!#|Rq3@3uBL)dAsfptvxYjtLLflPxzuH zfn!4ph(aFE7^(l3m~(`6Tv;s{0h7)4e?wrrjQ+N426>$^K%boT4V($kvX{>aIPbqO z5;jH>L9+`)s^luGS>2+?p7lM`S@vsmX!3Ye+C7^%%1g1?HiW-j-h-5p8uYO31aC!6 z#^&4zeBP?X8;xt61lE#3vdJ!}iY9}sv=70yoOiYh^k^Y3?6jOQVL#n$i!m{cOW~O) z4b><644>Q1L4WLwm^c!?86hzL6IS-PigvlgX$5x7g}16aYEZl_d~lvh|6pN7?j-)Q z4$kC}7ZVEkHs&H#(xr>2`L`0Uq&4RxpIj7}cAQyqN-15UY60q9SWZ{w_ZKU~zm0#a zEnxM;FPQ|Byivla=S+?{SEQb2HkRwEIg!Nym?O9UQ=sJ?&YRwcC-W@uq$-6T(qb6T z3=6@YocV47n|H$1z}@u!x)XG{y^(Mr^DgRPu@;*fL&=VUGoha*o}n$ZPcsrlRV*Poc|~|ne;m8VdBQ`(B)mGs_B;&P zPlJ{H5_K)N51v~b%7U zqv>-dINTICt`n9P6&(>+l8eu~kE8#_-;w(k$Nt4z1t|i(R$*J$DbAaa{TWQqVM$gA zlHI%=+nnxUkLzAmaN3R~{>Zj0G7=95KZJWTm!Khf2bnmAGRWyYvdl_>$80U|f9(f& z$$Bh2vG+uIlI2Y6=3AtSzl-YZkjuNn%RL1eeU6IiU!yMD2cwPgRaA=LN#DV!U+BK< zX2QnsM)V}-7CpS}>rC!Gb6F zkopF_hjAh^SIK=iE=}MXeCpCdM17|`vkQK z|A|b=GJh;Ea5E9n+fUaN{w;I8TPl-Rgf~&v)x-7z`zC?)H^^Tv8IF!mr(5t|&V9uQ zb833(oA9>SgY|1Xu`u{2&N#xFT&VdJ3oV!6RnM(h(tDlr7WAO!3ABmb?;iuUFvdpxZy@L9Ph-EGIB+=B!?A^&V#XO?F;4eHDX-Y4r z_Xo<5diNTZ+wy!?4;4JUlX=8sAM8qALYETs5odS&0L4%|z>=uh^u|{GRLO>qvD4Co z&znN1@AmJ)z4ANoq4#TiSi7IqMoD)94IAOMdjfgnrl!y7`2tT4O+)c7uujdQQN&AW z^#9K!JcG&89-ym%2goU>lXQlbE%YR58O)f5+lZ8{ zMK&dEMlr_dqX7D|Q4*U&sFQO5HGdwQ3sBcVsg0wY&gXRuMa>2iZ1Wg2oP@G@PT&$r*OS2DhG z-h`6ZJR~|sbzq%>z9r)*{kd^4Y*dF~RdfaRMB4OW?QJ6EmoL?+lSDx=lEUd>&)sI} zeEep!f{Z9V_?PVeZezUE61#4#6IgVM%ARwENk2A(b-LQ1X6_+&_X$BO9A?2B=j$+} z_N@Vd1%7zdcnkiFaKaC9Ib?3w8htz_^?0dYz|YhtJa3kU+Uw1O&(?F$Qk^;8unohW z3aPG@z5>I2ucG{|ci^P0iuKc;!%3s{=(*O3{EE(FmY=?cmA$8;)v6}$lyo1!E0^ue zfM(CRk&c{7Cy-fxd$rVutR%CP+vSpN zD3X;qHe}4EWAuNQI-<^n4>rG&1tr-;-|}wAod6OJGV=T=Y0o`;F8Rv;<$`)>-!dKB{IS#^{#_zc8H~SZM zX57H4QSMl)+=I0X+Bmy*Bi(QMOGGv!j93@%Lh+I;a*1p|Z>2XfDOakn&}JuHXh8E3 z{Om{e*-nHl#`9Rsb3Qp6VU5R9Mxa7rnPhj(RMwBIq5sTsV;x6R`jeDdOfD^2{E*Ph zolHkA9bN1xJl0`Irib`R(LWBVzmq1ueAX-v;wGWxYrI-~kJ#z$pqq^vprfnVLHeyPxNEP`t5n7j>-dHEaO7{~h2}7CO;|)VwA@KQ5PE6@ zXVjy+iTAF1Xs*QF|UiJDlUB}d$K=<~2Zo>5ltcbqZ> zW`2y)c3#J;aW~PZ&{=dKngwumvOm=%WCu~UX@emq7vbknw;W2+$O48_JKXkRmE%J^ zV6unF#Ywsy4c*CYt!Fcx|Il3HbCtL!tz+y7sblJNv*dGSKY3CAks0eG6E(9mSnahQ zP8#i?9#y@-owJ!4QR|Sw>eOjc|M?ecZ|TUKm9vpfPj?C5rm46qi#9|#mA7G$k%QDr zypR2+Gtr-bhj>i$31{>_#2IZZ*~M@vKJo5=4CVCT3R!G!jh))H#5AU!j8LSaV`CHO4g4LX`^ocYi|w0C))xL` zt|Aj14t+p93iyhP&(E)kIG&WJE6OZ3NOIIE2?qyMa#ixS!lqJg?Wv?BH$Rxp`Hyb}B1r|Kx5#GfJ- zBW3u%VKtpe%zWb5WlcS8|FBXYpRj`+RF~1x~tpF?YoDq9g%I>VkZ~^lOn98K=81&5XS$ zeDD*p#$y`2MC33$IaIY96YyJ%2OV{6Ew{>9(R^FF6c2=TpnjgEaMI;DY#Fu|Z;Y73 z{2uxdmMFW3T2Kc$8^$0a(=Q=*KFiRq^f^R1tsF1;{eo3lv(Uzb@$k^=uJn7~wG@Hc z0Lk;uSi(??#}?X4XHSHC_LAaNKOt&zkwgfr#r9jQ!ZN+5q^w+ujO*F;Aq2Caj=a93`=WVb#3ag@jV$DcjGRyET{?}wP zFF2*+nP_(yrG8AemiHQcOH)4F({3UPRT1QD;vMuq{2F>|vW*VYvI8b$`7-%d55Pd% zb$HeEEABM1q+j}SSHPe0b#w^MGn?4`3AdE0$7G7hN;*dUAg1`J-O}v#531|mM!(Vh z2D&Hj!M4h{CZ&iRF{$A`M7bId(g^qDiMi&~Vs(*Nd+HY=oC zpyOzleJ`CuW)y1EyogN6>!+t`OC!okNuvr2IB94su`BwRY8a+~fl)tD@GNigZA&82 zPw}Kmx+T(wbZjLT^p|r3ALYyLW0Z6nFZ{+jasOejdN&k4XCjkSh@?d`uENz3=cO9= zDjWzsjJoQtg?r|IqDxL2(U#6{uwUy(zZ4usj1)Hf>Xe#Ad~+(P{-b-yg#|l^azGay zRHS1larrM@T(OcUSWUnl%_weDlRG$zT1nM&TF)~ndx%l`KGdct14Swbm!_u}y7#pE z#Jctdyvn{z7R=0my}5{rOc%>Nm7^2genf*xpi2X>i26#JsU?QDPh(A!zxaDxJ1bPU z!i11&Y>Ug0{{KKN%FvcXr!+pFtWTXzt@QbxT0GRrSF`GH~D)a>L z@*2Q1L1*a)0zQ=TMUQuthSLAlSxl{bc{j|dyiJFebBMm(=N@`%T?2>Jcjyeg>)=4> z8+5rZb+Ajw_M`s;ULKk{|0Mtb6#y0Lk^vROUKRA%vgiRE6}lJ|02Kxm2mk;diwFP! z000006Fr)FT$JV7$A@JYV1}Jx7lvK-eTRX8fnjIZWB@@HMMZEyMGzMxm-1LzS=pkJ zTbgNRdOEq4T4rt?v&_^=&8;^}JDsvkeedtX`}WW8GoOd&xu0wKUBBzP?*|K+Ot#O% zOkj|wqrlK3R|V>pJuk3)@CSiz$A3FXz2k|HXzUeiS(R%qoEm_pva{u3unPK$XC#ThsXM9K;)KIs~RY z+b7U|+BmM{MhmPxUCBKqqqy?%0f9xk1jxLuJSi|^csnt1!Jg<<#R3cUV+A_&`U+ec z$*patLhwb?i9L7bVcQO(G@zY|{qidU(^q;6wD{s{eCd@uZGVlGCynR+ZPNuhCcPuj z?w1;#T0JZ9{dNJyTb7|o%Mav|3REb%_0jw-{XzfKNOhY zIYXfJ4)T(>v4CtowV4b$5W}2v0jxiUbv$~=RQ7)-3Dmf-X3AleKxH_-k7>Zd+@D(n znioe3%uMR${^NfMY+U+CVEjd*7w?5_c>|*b#vOZ+?0iN%4fpp5w0RL99d|v;J-Mtl zIr=w&P8T8snoXS|u(F-|7iW=ckMAxDbp9|$U{XI;<=o&YZMlhfCTtQo@9;OourVKQ zT!9I(-)s@+r^YrbM|*+Y*2FE~60E2i{vWb={SF>2a28nB4;#Zx9z;Wv07Ld-ZITTU z8@mOb_eX)_F8rHZ-@uDmE&SEt=4DoP!NQtTtpZDK!0OD+M9=HyOW5$i+pL*v6j--^ z3^9CWNZ^op3Emw@5g7T&18hhkV)bgGt?1$YdER(E>IRG|vm(N}T57YngO&5#9RyDB zco&b{xijvUWa7D$9O+jLU`ZaadVC2Mc3e6kF!(cAmv@5o14k3?CUuX%g15Gk!H_|L zsS|y;9(5Z|K1s~wxx77a7+->Rr*qx<-%@^Ghcn%Wc>Bs@tk6f5IX*@FmFZM&XdxL1 zvLu_KuM&Z%mxy`Bt|tVB740K0SF>P<^Km%POSL9_8z!)Q!8J15WX-env1IL8I977) zh`_WvtdsN{mDoA)1MJ*E913kf_r1h3u9UoM|3sH!qRDJb@Avq-P>&^RP`v3j_%e1m zn$YZr*H*{*Ypv4-cvwsQ#=e5u7`|K~P*p>gRWsnZ!?)yOP6>+C_y=)Tm10kF1Xs+@ z)Cx=(A~F+eKM|OsFC8y`1L}CTaV5+wcqH+}6(5_`Fr=p(8(hxu7Iz!slfDnW zDZYbgnroQ?2M_!c-#>>5k>Ah}vc|yeF{|Nc>-Xe6tA&bKaGgq2q)XGnG3r`ghriy9 zbhMC(-x2?vtUS`kZBy^!yE}T=v4Hp5v|0)rINBmG*@s^2{xV&_Tm6SX-4+zv+_X`k zwiDYTTTtn~D)L+F1C#3>xZ{@}on5;Z#p^Vurb4$#bI5ffU$uvd4yyj^?+8qrO6OJN z!>$RDyd^1-^(~rUVCpR5IoY=jPdo5BV>L{Uy641uPtsZP5~Yl+$F^h-I>1i7WCGXngD;_VjRn z^mL6tw_`60^!yr6?UvCgg2K@M*rjAG?-?RFuaAoGc^a!!TGXsZ>Sel%{C3?&zLLHr zo6|2)1u<#ZJVi-%Qu@hDNjjE{NybN?3ttJ$Uo7>h2Ii&glXTmZLRcM~c~;=KYYJ9g zOYgH#Q`i0r;j{4)T~cm_+8Dc-JoEnpf9CX~?O9qPQ1cWOs{0R>Rkg}XV8ABW67(85 z>i-g7+~@N=*Y-hd7*BN<9;W)#-Taon5D!03%_c8aR88bcveuUNCr>3) z(J@=7&4gX>GiB#{+!MG`;Gz^5-nbAx$6b~zyodfM-bED?O0cuW4F#}jB)|H#=)dMT zGe(ULMNMeJs*3x={Fc-4l{cAebs0>F5nExD@^x%0ccbd_)9C`U>0^O? z@pONu0(dg+7**Er;EF)Yt;|=&F~q0tvMav)$nV%s$fn0M_@Jtg_+SGQ997&e{~zA9 zbS<^1IEZ&smN3Vfx|6tbH<2$iQxhL6cqP9;MOdDsf}%G`_urQ0Hv`^v+@p@%4--|V zgS@f$K06D~vrLBb3*mp&CHNeD2(4?5W!h|rCxeM^!;s)h)VSYLu3K!_4NDJEDMg#{ z)Kf#oJ?9dWk{4NDGfYl3ck!ZPGhWDBd6(}(?AQHCj_TuBbKGlZ1lo?{ujaFvev40I zf8sAxLGu9XNBsm7>Z8bA_V#>2j9nr zFNtmX7IrlHdh+6Og_xvOFv~^0OI<6XsDk{p>Zf>pJ#QJJibwaz$>1Y+Xt9@$ zW`BzMO?VG$M{Q&^&zrofb0w2WVbM{6NxPN_OjTe{!F6UWyVK05T|P{1=Fbz)z>QRE zWi~q+V-xm_iXw)U8&DOWU*T>32h`vs2jbbgp4IBVrxz#h%n{hyPMx|mpM$G2;e7rr zw8$+Qerlg&O3C!Wl8`2e39)E$-L7GQ&UF%gG*|ZjiZ{}cvv$CT%BjTMwg<*ej*=$Z zuc)lzJJdzne%_wu4AX4F`R#R)rxZi<`pV73s=1oKj^4<#!TYh({|h{g4*=DD^pT(# zp0%mRa-)n%BkCO2T_VY~a|Ja%J{>!q5_xL&5DL(AN$SxSWU4icO!auvoAMS3kXb*8 z)6ZVj0TUT^UdjgKtj$@Juc@w>@mL^HpW5hi381qbd=45;%gtE50 zJf2;SoeCFpsUQoh+;7uK!#9(iupwq|vn$-A`T>PWd`P|6k43Tb7f88ifZ0^8~Q9cnjr1mn+mBC%;&zbcshPg?BCoQx0?>-=gX&4E$d*n7tR9Yr2VmzxNG1eTQCI77hOuG$mPO4XU_!8{E)eL5BlOOSs?2u0Q`G zSx{I_5V%mrv=g&~sch68vKhILx1_JZs)+~aeae@auH%N8kTpdp+n9@Z)O8l`l!eR& zZtLMy^$9Xq?<++>Chf2&eOc{0#8%%LC9q200yiG6BkyHT6Up%t1=f*vnb#Snml7SC zpX<~p(CH`gn`OmR+^#d{7qO1&>sd{-+i#$6ozkqRKa9?I6mp7^v_aZ~6p*{h zM%J`*VBe(8BEpW+zOP)%`tq0HgXdbZ6Z{9U&0b6VUHf5a)=jwNA57L7tJ&kkl*3Gu z4>cGPiY4c1EonFG>PbCGrk1n5<(v3k_&mKJauIhv_B-6j3&zd} z1yyPPP|AKjlUvOyo+|QV=bNXYb2%MFE6fkFJE%HKCr#|D=K8lpz)Q)b6qrXwTw-A9 zC>>r$zDx(vPR=Jc5COAu>;*;?viBNU z_F93)#s#5@Ug=Mf!Iw}HWfie9`D2NPf%}c4a-;|#vIARKIpJAyt)7kLv&&)BjLp(X zU^gn{UqjEGFC#l{VVodkcT?|{JJhQEcmf zjwl|Ax-1%w0@@3qC~+^tZ@<${;JaO~>Ep zCER0~js4ypC`tTE@~%BY*7QH%k=0rHPC|H{z**a{YJLE3aY^LNr@6`k#y(=Tj$W!M zDG?sl9w6qWcj1g;549O%BzhCSk~&uteKV+B((PdOR;u&ZX4DgBWjbdyaZ}leO+jc{;DFE$mg>!j{xF zSP}Ca{4}&P3DlaQW3i8;q6L{$Vv?R%jdNoE;=5`ltg|Jmx`QyJu$8lg8YR_Th7#U_yXzK>*%g&O$SkbGTC+qj}yWj%rcLY$6{tvOi?gh^DT58#?be(g1 zTo2T)e~oRkjg!W9+OW~2u`#i28;xx{nb?}xwr$%wIp_WNJ)dD;v-hlLJ?p+H9`rt^ z{!9E4+dv~GW2Tw~fI2}4R!Z8ulv5_`jgg+|DLlzsRKYo}Qm3y=Ya$&>18lgolWYtr z8xlNh7r7?~5`WXNu&0M~$8;f4dzXBOf@}aVL=N1O83tF|9l=*GAqI9GsnNfgCX5`!0+zB-6c#rgnim)oc$)9E~2=%A8o!#ZXmQ~(g$Zb zBe*(xKc~uNg@&lZR+d05ZsS?#wM%Jg4EyT!s8&RsQ3}pu{x+odXj`XL>UDJv30}5@ ziENRwXY-tu=#~2UE~2`M8PzgaQzdWO<+&MSBT1f&6r%$4)t%(OB;yi>44*J%^B>&p z|1^XL$Ko(q%59tF4MsEm)+Zkwf9@=|Kf(;#5RT1}=99G>qxI&*5#<^7iWc8}dFdF) z7b$hucB&A=+*19p#{^AWX5FK({c&qN%ARqTBWA0GpHKI=URiCfi4dER>GIYcQ_CDb zzNBaj#e9ZmMJEz~#sFPE`7DcPj5)&`VT;R|LR& z1n}bKt9S6jX-OSWp0NBll}n~ujF>`nG?y6^2l2$_5FE(Hu}l~i{P-Ol2$gF;AvnX8 zaBQ3Jis0aXuTnPVP0c2=zlDU>SZ9qTmGdh8qK{G{BF*=$q^gt1oL`1oowl-$c-2;DcE z?L@yXEWt&% zb|nVNn0wy2KcJM=T{)Y=y=u4OsSK=%z-Ew0>Ee%3g3m)zgSJhu@q{~Dv&yro%PV6Q~t#m=&f^7AQYFp zBs3&;se>1{K5?8n6ZsKG(1kF8?yA%rE8y|5YTJAPCpWG!{yq>rwG=0L7VafH9J(z= zB^Qnh)Si-ePpUu{m^}a9n3`N&xA*g*B`mYO={!)0nAg6{JJ9N5!Uo=w7jYAGXG_bP zDEKzX_U^%*5p^vmr>OaBKauWfbNZWFks=^;aEz zqYPF=od2~=g~T-&ZeFne#1h=PW0I8BO}T>U_A?&;AVAd7VPz@z?;nf)R4F_ zt=QuKVUC2P$|?CV(N9NhLod@LHl7D)f&=-4cc!h2)v%GLLghJ*^;SI7IJ7o74tON+ zF!z$e9H3OMR-}rJr7vKHJG|E5A6<~80(rd(wcS2_z{Z|zqoK@GRsUgR{{s6B!S z99Enwt5Lf)$2d}yd#dhZ|XiQ{m<`WPAOdSb_OHUl!KNjocDNd@au!# zZe606f3n4WQr}s?O(k2-PaZRmk6ve^8FNdQF8GiBsM{X%I&|!B^imkEi2L=<2`@Jl zNuSUgkj9q{|ARCd`O;2>>SEVE1s;6K3P%1v;eF!#!$5Qwkb8_R1Rv#=p9IA=1?4tk z>>u3F_wNYq^JwTu;~60*X=Z6W1v-RxMP}v`X)7Ed^dU22V=S{HbM-AL^un zmu;IJi{1-$Sq%?@N27+dm;jMLWhM;@%Vs&>YNpc<2GH`tau1!!I>th&dv^72>AXNE zZZ_?u6vUaQJZ~UdS|fr5vrTk$g#c>M{f9$Bb-oY&Ps$;ye!s+d zxxD?ekIjK~uV75h6+M_+HA*QuDiP8TAj6HLAU}uxP5y575xljT_8lf=(o@|rG}csysNw{nF9@W+ zK**Zu4&!5Anl3zXabjs#)-qt4X8P4|A%eHVgHNze1O6rndk|KlK2&*{4Nv%b ztJ{;LfrhxRu%SU$i~o#rOTG52rWVZ(U9$lkc*@=Ss~}f^fSF@39C7HmI!Tj7=lkon?m^bSNSTdW}JJOG`aSG1ac6DQV%^b0W-;C$jP2Pl!|N7JT za|D-DkAxmsC}N%@;o;^dNu%IM&!k)@&O&E6lEQ@Hde~r>7P@d0*XO`k>d)VkMVa%8 z#Mcrez6Vc)g594>z0B2yh1I}lJoOWB)Dx#}+`HVnxepni)`ee+wKO!J({~HjjpZIo zGYk_@Y9|j8zpRBGU<7GnC8@WtrvsgTS`f#1MoQ&zv`?;ug_n5+wQ{8l9B6hpTono_ z+HzrFT)r}~VTFpJdwSRu6j_%|1zN%EgXrw$Dz+23F3xxg-prWrW>Vg90LNFWp9Tan zxa)OUk_hy?pw#q*L=RacZw}YSPE=RUe8>U408N-}{ZGb8!?#h_7^GmaeSIcHa-zx< zjLhA!V^x%FE6!b!o6_R3MnqD6LBC05BW?amBV3)^zY%MNpp<9AAqi_kLk<46(2Xx& zgkYD+XpKuCa&54OAg$hMMIzb#PXuN&eC&b~IGG2xRQl?-C>lfJ0!~`SYNp+)Yaxdx zIv^$=Y?n|9HHoG?;b38eP4@4|spf5wBgAHwHWVR=g`U8P%<7dR6Hn;~lY7T=(Y)%A z4hb32^UT8K9i($ zhL_FJc9orJ+yAcOmR$ncC)jSOFtBlhuvO-rRh9y?m-79HP%99(OOB(%!>E+I zmnNS*!#nbqc};nEi);03X2uIh?|Zm@41iGG*52V)N2ELI=fE{g>cP&IAj&GhrHj;|6r za%9E-_O}0Ty7c1vj^JR|h`8dfwe^%AEPF5x7ce1bkHVqHVR>D z+VYDOlza^*Ax&0(eT=_m*zY$>R7RxitHJ+y+2XJE0XVksxkwxAyf0g;5wL9(vtulv z-Kr@U5fO!lu-$1WpvPn{AsH7z{RYaJ_TMzA_7ev1Cm@G^U5A;| ziF6t3G2)~8&;m=a80K7-q@V8Qn|W~bdBTHJz3p98e97HHr*iywM#Xd4B&mIC2HJj&tD6C;QR~# ztO^EVTV0%+bY8}jmx;I6K8FJa?uhMk&*5brO7%G7X;@hfVQ(hR75STz(SxF(YOC%r z9LnUTp93vpc058y@>U+As=b-E0-PBI3nof&8%yxO>I{Lsf~%pIt_-fcq`OLap2&p_ zf}#K(oo^ylrf4Zg|9KFvPT5A&^PY=3*|gNpiFtD>4^w(-jjHPbEvv9{d(}x zNX|CAV57xnT*FB+$B-wvs&x}8d&vDE%ZFs&<(5iLfDzMTy+s(PNfKj}OPa$kV&muA zkMLHX4lg9@HOx0TZ1}t1!1ZS*Vp$mN-k*NDcH%jxrqVMSPke+@s{q1b#3@+W@Atgr zq%ge10h2tF_`7tOH?50)&emN*L6QIT#PW+MfNX-Re`MzSMeD+3ssMhg8@wuK6;yQz zgM=xwzsg*4CqUjA+NQ$8eV*hSyl?(01#*7FR3@d4kp_72)mB1v6)(i~#Wj3w>lt#? zq!7wRngSEL}6nC`@hEd#vN=^v2t=bX2#ocR?coW!a2Qz=g+-;rVCk0hblTpvx7!&GRUendLJ_&0N_AH7XEvVDnMB5 zrwihcAv2i;^pq^@N?)H#r)0zhWV@qw3ysIjx?JHTFyB@?uE_S2?qi&o-O-~}(&q3X zeCaojv!&TYd(}><^fSF;t8R31AUQTN(#I^eTfoQ&oI6?+;7rlsUd>RCSX>+JfAYqU zq<>YTWA*l55n6EYB`;MQUlH-*Vh}=1Vq{MYt);OXfC+<|Po#~JP2wGAj7?}HE9=w} zpgkp`wk{hAxh-Iyhr*#-4BJF=)6DfUQKGeGptx94uGsS;w=UMXVdI2N3XdZM98|V{ zq#0**`5n(A%yZOuk4Bb(;WiR$|0z-;jrAS#Ku8|LzJz*>H@tyX#MQz*zB}QvC~}5nVL*;`1##rVWw+OhVnp8y06% zP&VNmYdi&av;)53Qof|}>6$=6{cU}zh$5qWHaDS|PLk8@`6Id9gbn%X$TV3St~*VMcxxrTmpNvjdv^b0md5yNoJhK$AT!SsDUHy!Ti!a}y>2EOp{RivhK z7orC58{dnG@=M7%9ahzvU=s2?)fe69GCijjcKb~4%@;B?pf-ioYDWDp3N2~Z0n=7= z($9)-sPF$;W1eB~Sq!MkX1s$_F^lqt9W;gz%PW+=?K1M0(*F?81s{pbVp}`KyG5b! z@gIJwV-}66Ieux1uav|v&UEukE> z`iFBzwR5Mffr>B19DbrMcK4%XIlmucCQ||UunuV z@`uY?MWN=n)0t8o5fRlZTkR1Nx=gfpDxPHd;K4xeaB>OXz)!Vl`zl9-_!stduY@{d zUX%L1#T4ozs@Sq|JKTi|qF_;EvG+Wt!k!qnj2`+GJ?R|yB^)rl(G9JaJ8lQHS4(nw^GHjo8HMOAbfh71oB^$W+G9Z+9YgW<3 zO2ba{p=yNurHIqdsIP@Ny(_m1bw+6xst-n`g5>ODS9~BGwk416gZr1`CFn4<-W@fL=*n$(fV2BqK*WG6ZiVASe zGDWqQXkEO1r8E*{Q%~@)HevgVG(b9{Sexgxc(0jqpn>WLZCxp)k9CFi{y_wE&TwCEOUnA4seBx7Srq+<7 zatKt|V2&JY^=DX{iLz}&^VOE_5v3Us(}5mIu9QX|Mn4uYRmSeI7OIk?>Q@V*IwQM$ zp4u`f2^G6Ik9Z$lRf>Unl|N2l9%t?DMWx?Q%pAc@ndz8|_ZTQ4rRX>ASEUK9IrZNi zmp%nIl8(?z6WDDPttpbd7!{2$_0J=uiY5F~a^aEYih%cH({k&lG%fpzFEQqd$6&@k3Ki?7{@wCfZF82sJkeWf z<2p|<-btBh7NG`D=r57*LTH29A%8|kq`V}n!0;-cYXy=MgT~YrVWIk97d-XK*cq?p1AKR-( z?~s&N=Vkn~=1y8&Xui9N@IY=xL2i&J*&7H?UT67W>!S2KZ)nC8Y>4uBoaw*0WmQvp z7d}sO%C96GRpnzUzlQX}!J z38ob7J4MTk)`uU1kF(C@m>wf-{14zB6(VNlBM&=z9ttQx{y5d8z-6tsxpN}# z?7_|7{;|mx(9HryNaG-`dI?KDp`5a?6)Q}4QOwjVyfx_8U}kFl7JfInByCN(M6i#> z;la~#%mb4^-#&yW@?{Tf32tj1%8a-M2M7{s4}LZ`bpiLZ%4AY9QoX?csjT zv9SpnX;*3I7&>KaB4mF0`Y~`GNnT53EvDu(2XG+)^7XV6StgNH;-YoF&MJ2C)3%Hz ztvb)0vXDBSamo+#olvr;o=u3zI`w6dg)3qg9*?$=yY{4e*fUIF9}}W| zS5$9u9J_BA_k#D+yccfYl#4)2$DpX;iNa&>Q=N6r|er1CRgUO$bmy|QL=Q%rQWxR zOqai-zB=w0qtMJMpX}EP(^NwFt>)~wqd+0;(mt7JEwa6J+^oSx3)CqO+2oy8jtXyNOiR! zr!bei8`c1WFl26zr+80CR1*K7{IPFNb|3h`{|-^ioNc#O@7R3%xK*Y7aQoM0SEtr> z4CZFk8kz_|^84e!E3^!&k0pBFl{a3Cc9CBS82*6mY~GflU0(R<#+>GBl_FYMO3t0W zN+ga-47;RyNC1}Agw7}9ZJE5&g|%k{o%N0b=>$y*5}rd*aq&9Wi4d~ zq6Rtq68DM(|M_$(%iVV)6MXU!{Ou!8DGlf{&+k{**Z5PUx|xLn`MG7y@WR+BYT|ni zP-Zd%z%dERgP|0BOUo`NQ&_I+{;|d6tg-Z^xrL$ANB^K=89y<~YNyRu7uH{r*TTD1 znh-zju{Zv}Er-@s*#Dn+KkuuX|Duh32<~b7>gHGeukpUi|KkPCQm4+P#;~R)41ZyR zQtyUibpD|JD2V$7ho*7Fp;ZtH6H5m5`!C~nh*(4^d8h&!8h`d{e__Tvy>AJKG-S4q zmwT^ohh7J%8Fi0zU(BEt4}EKW{h3p1x~bcIU1*%eJ2?6d4H{;Bh4`+nb_R%93~w~N zIvzmIISjQQqc#r&2k|{b$eZnpOu3Y zblN0$fWW#99DV3l-(IjpEyujq+E3kT*%cTz9?*&ex*AZ0I8-CTz%vZRPEp9mGQC&8 zNt*PO%@KLy@FPYwhQFKgq+w=%m9Tt$3GJdyR|9r|4It{sU!bV8+|o@NcQVbltyC(r ze@<}((~RrrQcPLAu>0Z1-&RwD;{QY~iCJcbMY!|S+&`UDZ5A`)lz2tBh_r8h^Xa!A z9K3}(c&*N_bD~6F<&e^oh4z_mt@Ec(H}$XyL&Fm|`tBwl+a1iX*RzYn05g(wmJY)j=ktl<_C{PjqFFs9n0S<0iD*>C2U#woF7z% zm}TgJ3LbT%sy5(}fAJk!(gy6^BEmu~2#^(zuarYbsD6kqfCrLi+yyQNHqei%$VMey zgfQ{kuvh;zqHB&W_d;n@RiK-;H)UrO)$ATg|AxQ?(W9xYzaxVjDt#Gl*PvG{%X?>^ z>uR>irU`vMb)WPTg#$N(<5PDB5hI?~5LB8wWda4P^bEpbQ!{n^DW8{Oj^qFpTzZ-Z zs4D=j!6ZuK#*>5d=Z5!?4tdA&I}J9ksX*H+)9C)X z$`j8~3F6zDBve^uSo*m2lBPYp1s}pt(Y+>fg+4Fn&x)|&TQW7A<9m#0GihYc#4Pjp zG{4AJ*E(OIiA2mBVVw9zw-u{dl$}g`ONG3 zCZ|sQeGfNF=v@~~+taqiGpD^IM`zED{P9yT==?(U0>N^=64D%RF%Cnw2|=ul6DloYSI)mV zw`vxCI9@Ewv+*5L?xAYM$ac(YS6Fzgmll5bAv)Aj5xOf5G=5Bb?jT+B#H=2E#H4EK+z;Eh`5?v)^`11c8S|_4n&vq_l zU9QRob<;InlP_gDmKbRR;Ae+pcL`d3#b{0`Tro&@MNbybQPPAyST@{b@}hdyx&~NP zR1U7*LO-K=S9oufIbhDEj?FMltGIi>X)F*L#DAF63|Sxli+3 zM_W@Y0TY}UGH-^izgdwQ2JyJ#HWv|J$N05{E4CB~3;4I*s>D_26V*SZ=64U*W+bZU zyPQF6P$pzDeRuP*W9yP-YvVaOZ^9?S!^rX*8~AjhXTB*cOItO*>jy$gw!JS{U8NnB zWa9e-lS-FK$f+Uc5yd@57O$AzEs8nkLH3@tJXL!lYz{LCIQlv->$2EO8UXp217E}R zH%ZRtG zBw;$L(%h7ywIoDIRqM4Uf7)w_<`pxBQyH`EM4>g$(=O-iIDN*4g*=@I9Jb}KIFU&O zStghoOq3sX%hhn2Ef|wsL91JT1|?R^stq=!o9yvo)_RgqhNb!`sEc)V)F#(#sz;)5 z>-?4{Cbg!J=vCEkAxb|7kST8jE>JvUp_`t5nQgP<6|G2H9?~WPdCu8f@j4>L#EtSv zkT++cF%tTKXVHI5_I`SKIv`DOy)b?!cpzrp!?IJ{$bUs0GlI@8HWizKY&H@4FUqX! z2=~Ke*f<~J-c)}W5&hVf@}t^vV3P| zPmMm~oiE<=1}d2A_Zu(Uh$llj`6G+-Dz2!#0(;nJYfCt5|lXeT=+ z4e$XS4)eJ3+J*e|5r8N+c#u!;_n0mRBD(^)(pp?b^p{Lo{7_0-p@p!Ua&1yFTwxY- z1p!GPSS^JL!xX3mi{I(x)j9EVJ-}>1H>YD07r<*+~or$!C1_nf!2rVValRQ9z^kp=I|!pgs{&dUn9yu z%aOd;viZiDNq;&lNy=L$^6CPmEGG1}IKnyr;{J7oEq7&o9ithJWCqvM2gLj{nFS=; z*m25`rlcL+9_>H}CWb3_n_hV7i>#j}9V79TVtpzG-Jpxs8TBoCi&l{v z$hU@;9`+q~v9h+6$SW%fgrqkFB5xr2<{&Yd3c4;^q3CKi{n}40I%N$(K6lA_3(k0x&D^D5r&EBLNwvS{)M5uMeWPZ|K zM{qC~4hb2Yz=22i&N>sLHgD+UG;pQ0;?PAdU60r9C{6xFBxWFri`A)?z|*NVWH}{d zl{ohDIu1AC`Z$2~%j^ilcaSX$3mD$7>%V4t74FoiDfWmajPp|K?8lPL96HbFS(an< z_Yt#cUBotp2ev^J{vPmrc?nh`$!0wNHDv)h>ACE-cnx2YCT|^JVf)#WF_%*rahj{j z#CFCtZ9?BE){L&5;I-G~CdrwJe;DBBH(NOS+k>zz8T7rU@^QPkf;^SMG^ySWlQY`$ z4|N=MK1IdhN1W@>G6fq60FW**VD)f^K5RR24}>=~x#27#e>U2gD%*IU@tHXgmqzX%HFu1vqCK4p6aVH%Q@JHp` zYCEG8u#bF(?8;W#P&#XU#4;gb7~^kHhpR+<9Y+faIw`zGoD6*iGh(a-u*$9L(u^S# zwQjOA3|AelOI+&;2Kv=7a(@dsh|8x33B_?lGQ1A>gqkaRM;uV#ZrL1gaO<@riQE*v zG(6@X!noeLO(`r)J4ZH%jn+`>Z^oDE;b->u#;enK&_CN7 zt#QJy!xga7Wg2c3nGO4WyGt3qSP3k)e5!VG<_Sv-}PlBd+qKi&(r7rshzTKb!pVFDke6@wwV# zxg)t7+`UnA-0QhVR_Tbl&Qr+f%6JdUanG*l|ZCA*c5*t+mp?laZTuo+_z z5GJO4TEK`=QS^Nfr1%iip5i7Y)rKp`$D3Pt5kW@<2y@+Y3T*l}ndcY4_?I?GlZQto zBksbDgypjd2IR{i&BG;oMB(zRvy_n?O=0vrbss_QTB|FD$+YP7)q!_=J6&dO)-Xw9y^eW@W&`pOW!kUGUKfhhVCi6l&8H_aia zNx9f4OtuFN{4!2Qow)`mx)C2k6ZskMF7EpRN==Wv0Of|we1nthX5A4-aR+__S8p{$ z>iprT_IZ7g^ZNF_JC1HhPV+;(y6BAA$YqPQOge<&PlSFQk?^oz(>nF0S`5nht%HZ$Q1g9fFb$Z;y_#hPGw&xX z2TL;hxSn;fo{=Zx?9ZR-T>DIbBIJ%cs#E(jkgklOz4_7TZ|uGG)^FBrmPHnXh%Gis zlcLy(5~Wg~#U4-qo;xzj(!(+R^8MFu*S57MXr0QD$mFA6XIf#Igz~QmNGeh8;>8w% z!(ml=5QXztpJiPrh#zS_n&ea7{@*~kJ7kMJ8)J<0AT{Vs$MUU+PkBMuv^<{SC_^^x zco*t-f66e1F?%;A2e}l&i8I4Tw2qn1KdAFeVhuz(A@DUqq>1`say3mm+RA7T2@n`WxxWEO7wrhs*8_Z?r2dfsD32jdOT+9XGmc`0ou*n>2?2?dLx z@IofKOwwp)1|p=k7dcYD!)>`SffIHiEaS9FiS77o#lWpGD`$X7Kv2>>OkT5%ii2#D7Q=nq|A_}kSm7@K&M`A1Cea48D zd*q*oY(q9|rKG{OF=^w?_~$f%_Y%xKG4$X{@v`CYS^Wn(f$&8?IXzc*5)ELR zIkAV+b<2$lS5WAjFCWJ@so*K@q?jPFF7|_XZ$4Ny^_(Vwrn+W=Sf;&H+!!R z6|y=(Vq$%B*b)f@WWhhv&Jlim+i-XI7p6MXC0nZAdWE~$ZFUap#suA@0CzyOGzI#V z9zToZabMQeW^?Y08ryHD3FG~bS&4h>lbRxAs+7*3vJe(}6J2<%zwS8>Bq#SBl@U2w z9F_z3)kR*Qf+<*ROk}tBlpXw9EJk3m?j-ja%eObgpKh_iZH_tEFGkhV4`d`pI{9@J z>&67p8Il?4J*jLVOuPIu?U7@2q*Liw%4+*cgHf$!Yq-nUvBaI>SIa=|=Q5B9Mq{!y z9t38prLqAU7L(*HW=~Bw8zj|}x4o#|FxOia&8$6wbF6pq!`xJO0M`uVtDO08w_4&m z6^M}YtKfw9Pxv8Q1aea}RoRQV_=f9j?1T!DW!G#bcyvu2`vpDzfV6rL?T?tNto{R> zM+B}slRPvux+qm!3WtG6N*aG~_jXr>=;Sle`yqZ>bHZaVhld#98@Dno?_gR62U{nr!1%Bl&AB3L8({(!=g_h z!I*5mc%DTni;{9*H`sUGYPYR=IU`J?c;Fz#OlFtmz!jLAyKs;VbhiZey>}jeJGM3< zUwcTAW2!sTmomHVzu}g4^e7;nq?U5wu;abiUkQ4jkf%|8QkomEC8$g~I<|f|D#_DZ_U@Q~Iy#F6VJy zO|)mKO{}|7(B>x5=()ak`s#P5+e|}#F>#3~!3g1^^Wd7AZP1im48lqF-5vaqTX*); z(N#>_tZ##jxB`RwW zOA@Kn2=|jRLs04PNF}Mp@GnPF3z;YFth`SZtv6wiB$Eh3uXM@xbbV*ZW(g>m^w@o0 z(iBa{A5YeAao&4&zB;A0WH-4@HL?#uj!BlefQpH(y;{|0qR0Oj?_HAWQ|DBNJGYf1 znU;}+W&(>h)OMb6TYvcAr%M-(6>$_w@%6+nSsrvWYbBH!B4@~~NGf@GxQhD_X%l+} zDk^^p7#i*YJX8HBS$}$yH@>R15~oFCn0urGPP4iNMoA8y4+xoC8lWAsCK2Q%P36yc zTivdxF1YC4kFiEvRsIG7fndk-V~Eu z>K8LdA3L`vz49xUANXH5{$govY6;4(gCWFU!#t3*5lH$VW{el?g^gv19W#!MhO4C1 z=k=3;gTsSuoS|Y)Y7|SvepE_?VSHv|ZDynpv@3Wb`*w{S)%xsT0jCDlx^2;i6#+G$ zLz)m3RPUO4fKfWXG`?8RIwncI`bRkk*%w6U#>EBw!%Mn_V?nB^VPEZt5GM2#RW3=r zwVhXp@Dmh-u{D7->S%3~K?_Z&k>m{=(^VcRJ=;B*w-Y0dXQt2YwpB%_1snVC?ictF z#JkSh{WA3O#9%*Ns$mFB?`w3m`A=f+c?KM{bXS<@R*Kivu;SfE(Ao48g#5c|$gfMj zGdzMja90phyOgECaAskC#jpLUC%i-%{x+xC8uQoxroB_h{t6+y5<9;~NC@yi3=XT0 z3!ZV;yu8+T-CTw_fcr^7yIpO@W$+N%GM(QcHhcZ#NwE>fK3XL7#L0QaKm420w1 z)P#rN%D`nkt~sv5y|!CY z(XWM$y4O0UT9Fub65T|1CZ#}0wa_jg!tecut6sOW0qIvd*d5iDxPnYEHQe_bGREZ94K>^yYO`7mET8M%I<*eYDcx(XC-; z5e%aA_WT3&nR^5-KCTL!<>xuc_3@kN*4nJ- z{PllM>?b_jIpc$~)zRudPG!%S0DXW&9t1s|(Y8XTcaVyQ+SVBk=w@mGj_BGp@R!jC zEW~{g=-|Dq_*e1fbZRrUP|R|y?S?HTKN3kNT)O5pi~al~1mXf3G8?X%z<4M<{dFW7 zUivSi`Hdp~w+cNyW&Ae6q&%JnpF9f5cN#fU37pd-VDka3x{6TOjp03qpZx4)&(gAC zfy*oQ&NI%>XddwdVY2JWW2|M#sk|D$2c`F+nx!PLOKMke@sWZn6{n2Qe~}nEV;iZJ#l3egX&LfdfZV@q!3>M*KV|n@YW` z*g06ws(7GrmiAR>a7zO0jkwX-PB=CvaRpWw!&Xmvp zYjumukCPXzdP0!7Z5GZ<8`C2xQ3L`D-i5H^}2-I?)4QyP-js}(8?x5J0nl+f&jbWnV;XYU;lc% z`ziSyUruu%LH>POGMP4? zCHq0l@i0z4AzAmAf8X(hw@4srJ^d-;&PLFJElejL-ZBghC@2}zJ79VAolQg9$jvhZ zmr;B!L3^A3sU=7Qjx-R`IJ?GRc~KW}&Z_98w$TD=WeHOx9-}$o33IkTa9znIsDQ0f z{mL+dV19I$0rYFoq^FuCMn2KW(j+wROn4ZE7Qr?U~df9ewxhIPKu17_LwKcx4{lrS> z$6-~0@wxKVAP1ffZCOgyjE+dP{bdYOmGv7kX*U-N;tsSpADbd&0cPp}?zYU8n|_Hk zH51JEF$4pOLa`*rq`|7`nd42)h)l-#l|J%oFo(?E@a2_2$8BEM!kQ11%H@x`VH?)x zK0)!`c;Rhd;CF%bT?TWZ>Egr=&fSJu2ro{Lkfi||-d{!fuHtR5PB-2tgyI**(7VWp zP#P8aRc^$7YK_Syps!tWHy=Wwvg%gswK{L+Hl=;p+GJbq;Z%vd+mS;Wa#DR#ox%Ax z3$hFm0pm%(NvFDD?VfCzOBTYC=7_KoEq5&bJKZ2_wR7_HFc{H+xl~KecfT8~C;907 zL*_{+fnWo(hY~wK?U*hiA#=|?ulBm=gpJIgagkf)u0&!8$f~?Re72gkgKs!VC}`Uw z@-#Ye{j=vBQ9m(5#pIb}?pXUXjkO8yz?Zr}?W5qKKBZv_D zgKEGt*&?{fY`FNU?DDR-_de|%k0Lo=+8Fd{}#R|WoF9_U}aLzsUoc)Dbyh0uEH^rj763%5YTh=E+`nRBu669Z!88E}3!2}q4 z7NYYw8~q(*O<{qRF*o|Z4t`p$(I%U$8j)irKa=oGstpFQQw$n?EuGf_e$;%kEz%>T z2EX}uV{)6|x4~9Xrhh|uVm9Ho@IQCWYYTS&9mamy< znT#Ge`I>LUahQ{h--9uNt$rx_d4w6h>YMdMcLg7R=4~X#ovcvRX{@7G^HRZ8Olgs` z{um6?8PP1Tx|3FV^k2`(O9u5LZD~8BEr@3pD?)7+(bbXc{vWo^vAYtci?&bDNyoN2 zw(X8Pwt2#iZQHgxwr%^wI#9wldA-O)`gKZ<>8_efv6DquRRe_pX^>yVTQp!Ux?lT&vMkj@M<;;0U&kyLUao z6mMq+O;^Hc=~@z5p;LaRxZUcqCG|4-2DSilOsI8njX;dKb?t#5WfgE+XTdyH4$m?* z3FSjy!5_@A45GPv)sAxacs*(cGt%dVII_O>D?jB~a*BjWt!A^P(f}+T(7p(W!Gh6i z*MC~XDVcgYejB`_?Ov7cWNud(Pnu)lyX4Bae3z*P9Fm>Rl7rJXoryiI4{XbfS(UFD z(z>)m_2SD`IjL#=hU$8n+0&U~jn=SSp#?b-zE~H)vOLXT;pdvMSKypic;MkV=yD;@ zF9W{TwXbI$RSnnTqbSbcen&G}ip%xagz@heeL`wsH9}U&X^HtDATk>z&Ep$KkI8@c z7sA(c1&h14E9Q-ds?-^xSy-V3gp}swQDRcp(cg%9xOz>Kc60VN+kbz5mqqKI3+}kT zgjj;pip(&ttsMV$Bbb)>BF*MB+qLa?Hr;OZsdR`b*%Se~FV9BnZPx}?S8rK<`yJF*!6wDG;i0n6>GBhNf{&FZwKncxjkU3O0`w=qcaZ?d^oZgI#HV~O5(7u|6 z&zIY;y(Y=Tw7)z(*3Z5a%h^Ao<4J<=7WFQ^cEkPR4N7p0o0obMIJ~I#Ryvppd0|hP zi^dd9F`3-2I(&`QK;RkOt=ev$d^^W*y^ra|s4;LnAg@Gt%UnJbfGEh=o86>#Y@IKC zMOSO|>Gl4zZC6fV2NjRclMLz9cNa1tDzL-*S$_17lvQt-o7M?eU{x9|MwSam{h6SL zUzaAurXegSrJg|AqMz;vJnHw)+tl?zhRSGS@+L>G_1T9veU_m-m;AZgnYC&_J@WwJ zk>v#|Z@MWoxxSh3I`!))jJa!Yq&mlY9+8#VgSF)T7XRTchbr(gOdm>2fwrk5{v%b?Wzg>>9 zM4;^0IOxaw(^RU*idwy3RWx41f9j^;$D$tjLU8=c+wv5ENV~(L=HG6m+R`e{1!)Jk z`P|{gnD*+Je1u(1=eM7rXS8%-H0v{00-j>H5>wO<-Ys;0U2Xc@O z`-@8OV_K$nYvBB5dZd|XV+Ir9VE)wP+SeXKdj27rbKzI0ob43zik&GEbC;X|;VDzA zsdp%o$QD1v>TD{zeYwW|=4WL?cA1QHIPNPfR4Yvqu$1Wa(>FmI^vZmR(&hfgeQ40t z{-_GFnwTqI-nQC^JKEOS>$kWLz8={d5)byu!4$upm9ZuWf{+pqGUhW57UA^B=a(^S zg!%*ftfQdNv}{BXESnAR{Z4(}J0sanUJ-R28QNtx#1MoDR*xM%?C)_K-o%mbLK_R4hGZ$qOEN*3yuTbZ zSJ9clz1*DCHK?@5f2e*zCD&JpQmg2yqZbt-Li+1Bu%e$wR&G%aGK7ww_ROS-(VsGN z6tSSJhulhBE!FuTu)=GXoRIY+zKz|ko%XQEn*lpa%O%zq+A837k(KG8pas;|<5ckM zKmR=*l&N3=q?|b$W$&)7*JyPO5-5&RyxJxv)#PQ4MO`)jZ1jd~W)$P^*Gm8WUazro z^QX?3*evLE{;$?C;%UBloQ)+ekelidDuvfap7FZ;Q1nV5=0TOU>?ui?2irMxzUYu^ zWAa?UyYCk5bG0%UB5z`QOaZ~PI)67wg1aEPV}jN6EKEd$@1vAFDrU(ewz~$#-_r!Q z?A2#)YWyD-V?XIJi`oNU?seS8+IZSiCuLZ86Xf2LFH#GFY2y@)?{_2X3rOpp+7d8x9?|^fRhpU z??}7QUG=Je{FCwDn3^Kf$P+$Erdo(v=&&;Bj)GFLzT`ap?Jj-fon5h$;4E!g^dyBA zcEz$IDVw(cRfK>5Z;><(HDT`*!--E>s73TuP-21lK=kv#Ah!h)O07lkCCA*wlVgVh z{K1)u_a`^S&U)E&Ukg-OOcYljq<7yr=WWmet?W@>ih9@>6WpOv1sWgP;phpkL);c5 zd*&=;F#W3)X_?_YYG>_h%2b+fW=DDu}R~@sgrk@##3ZSPhc%0-Ux*9knRJ zY&5JBjFF}YCXY3hFXJo1RWld)>w3?QdLf0YW9bgg0Eps{aR3~5_T+|xER1J7iq&)+ ztXL3EV5dDU9#GnH*i3G@kr~FYYG8Ix#V0T_X+k-tncFiO^Q5xVd=ok68{&+>GFa#@ z%VHK;u63xR%oY|1>j!1rM6KyM%{?G(R((MR-_Kvmtg3)2bpO+Ks!}nuM}5Ef2s?)a z?wl)=g?b_m7(^QVN$ZWfSp#v#NJ=J7+GczdBdHiVz1+%6u z?euuM^h~n-9-Kf6lclATm{x>a)AyI?qih03xa_E$p{)mP1=`nRj>s8hPr9l?N71_J-_qE6ryHDS$-@trJ>Q7ZGh>&A5VsSP&EExzMasot)6 zQ?TB4eTzSEyv>%HAZaR%lv@bI8UGX^0Ynm)x*y?z9j1UsT^m+j=>EbfUYe`0qh>%L zK>S1wF@}~zd!Sqo615gjS6%`H9`Ui2&PK_s zOzPaJz9;%QJ=c~HJ(I5B`>e*1NEg)&J@0_n2R5)^3GQi=aE57y{);py3-LoxqxwyYwndVs-F-wn?* zq5P@B$bwxkxEZe}qKB|~&Q@ka=0}7BS9%LFIgCZ>C+$Dzt$FY|yHdxZ7RRmQTEeb+ zRb}No{axF8mo#LrxXku2^xebc2}Y^->AcN@G7a+Sy%!MA6akd29g;0;_)DKB-OZc=8sVDeZQG!OpzUD@qSLn)4IN!_yW8ywC$ab5~3?%g5f$ zYmxoBUCgH3j#!NIh4*}d3Zw5wLRqR`Fsn6mN^|x~TDgMl?6hUae9qC>_n1MtDN_%v z;ZJ)Xm*lk6){l+=M}@pj7oN{|Z{YCi4<^y0X;_Y!`+$AVx%6F$B&HX994A*|!xh(^ z&6%tem0r%Bkar~2dR&b9>!KLLo?x+0`L;DA>x5RtNX9HUNc!L1AM<3O!n*L;k>GgT!KCnOI;-U6-2B(VK4>DiWhdpr>{8mZqXb zn9vz7)N68}-gCVVR@Yz&;S`;MfQt-8X>z>zCi%2 zzupu;e*a+laaSvXAFS3Wtv=F|`!kvzU^Dl)$*k@#%!0EMhB`E51!3`83#B#SMq+~} zYjI5vBA0^~=`<;=7HBG0KDOLg$aFy&qPBqO)LN~D5zk{>rYDveS;AbTiCM$4}Fcywb-EKF9&svP?Sy9-|tJ`JiYj z5M$-l_{cSJp|-|K8X;gUwi%XJSL~db+4d*z8>;0}&d^t3w1weT1vo$D?nSqA20`c{ z>{%m{z+g^l;EPhdZA*+ZD#YmOppo%7TPw26=MR>p+{Cjed_H{_6hUPMZ+fd43KI5@ z+LX?uz~3K-7ySYU@k4hKE96Ffq5W$2cQrABsKHe*)7Uk+@;V>;MQ0!1TBg{b03~A@ z`=P${bN0G_C5ir$T)t}vI|HI$>Bi%@wk14tSk0~ll%LoJo|cF?yoZN|wC@1ndIU{Y z8sz435GGkofQ5rGMUOG{_FAvU($;Nr?PQ`5#W}IpU%i*;T5sr!o$20|eugkIxWkdC zQKMh2sy;k1GVIfaPr9`v#xtrK4SUZo^(l43- zQli~(TNbYw(RZ5j&g7zgnvz<%eIY}nB}2HgQFEbLzuIa&^x<= zmSNHvhxfZ6%HnFZVu;7;>dW0lM8QDX^AMWMv7fw7>EfS z_YZdL*9{IiH$z_mF^ulFN9eZ+P+Pd!8pONKF83X>^lbP9m`g6fN}%w!&;MF5YOCGo zvKDA&U1~0j`6R+!R=9TowO1T~yBzUUeA9rOmtw}SQE-`r;h}-t3AnyTsILs0;1ts= zaUyi2#{P_|A|5S0esZf(Po=A$Cc#6_e`i}g1>rJHoAKT$Ym29MCZBcw^iZY29q~`j z1O-k0br|MH=71w^$jiH)B+aM{MpBf`*kCH5M_$5VqbeET$$vaG9T|T4hyL-{qrnQX zehWn!ek_?&7fW(=*nx;TL`tHxF~l+WPJe!m$ilUOvr!9b_|KFuWUSt4ll~K@v5Dae zljEghrJu*MSFdHdlJMTsvxmL9>`N+Gi@w))nlmd{RwhtU zGqo(ET-)uyK3eb471=c7L-za!ged>WFu`LA(SbUedBi-St~a-q6TIX1lHNK<5=u+RbG238`BKe_lP65ZEE z|BT4-DNc(&z2okbOOsd_9rcdCd@K8Wi2Coo8`y$$YyBk$9N?vLTyw(u7fbckhA{!! z72B~Zy8-}+&>iDRl*d#4vDc3V16zAx-X8gF*O9Aqs(g>BxRHTU{YBhe=V~$CwUEBS zQ`D?F{(FY_;>_)75A2GqRT;uN7pd;L)cRB@`~%<37&{+d|E^B*gR+Gg(oWpj{PmdF zAo>@y+`G#@$cO*lDDL%9IBZCI-jaGNf{4_11^}Cj=N{nqO9lQLCzVEv!pmV^v{Il( zy`{3)02o&5>!l(mG(t>;s%m9}i9)lHfyP2uw+BLH*$dE{(-j&3ju29dT|1g@p4zp_I zw{Z`vEnWk4HD$U&T*b@I-^s($F6Tv)&m+QR(oaUOFtQ9Re2O1ABbmxR$Mnw}QomL_5uF%=NDg&6Mr-4cqLy6(RGOqEvA z+?QBO`?vLow^ZU*?J(lda?sKWx`jfWB}%P60^dxj?WtpZrtBNe_l2AKyy~ z6D@d8sND8Yx08j?;D>XgpxdTF`(Kqf%L=?>r5$Il=AXT2lOyNH9FFwQTP76v*}%OK z#br36osuU#zIf02`srL5I2K>Fau(dd@j|TzJpVdbiDz;>C5{7gFGoFg5fCmB+X}_0 zp!k;{*kM&NI_WiXdWIa)O}fOzLLn=$wl!m*x*1wG6s|SM6SsP^ZZOu#F>g^ z5RAt`PvB;WXS4^7=WJBz(d=^4m=g&vlA*ae6L0#WB0-Mjie($DEQ1$n)g2LWwInpa z&5zcK+AD^;cI??6P!ant?vPwba1~+I-9^P)irg(GhkXty)+4wZ17Ez3R0T*GPT5Q2 zCE&c{6Jt|-7JGPwrSNB^1G(Cn{-%Age{n(bUa00Mor=y|sQVTTeXk8&?zbFGyWKxr zd>w|k>_K*eik>+#tCT40K%2Yj$)9FH^=7YbXXqUo!Dc%zn&r0R+rEH*eexi!%>S6$ zz|A~L>KtBE*fd(gb*OUE2VfEG#61JEFwz4K5N$9^!j>Bol|x9NAz|h0cp^3ZC{>ZjO^5U9`ocmLN9}7P+CvD3k>azz<1tLU zBT~kP)mme zjVZJHT{e&=IV7seKwJ4h5#;rm3rg5%`iN7H3?1Os&;BA2;9PKQX?CrFoXp`CD^l#E zs5<8qsTN$^d(X-n;sfsuMbEy8E=g`>!X zCucgau3Iqf3Dot$bZ5pyo#&0ylpJ_!M8s6YW@X^r0RxodaG25@7o*CfgBD{U6gBdc z8vfZqhWo+!h3XQVHZ#KNFpF?To)v7}tgu&=3h%ZCtW1tIz{8mylE1P_E7p*wC!^wW zZJi9gXjT{M*hW1=XCT#6#j%n>08bCoG??o~BFb8tWjy`LcTFYy9W$>gBg$$M-#1F} zjLC)23_adkTOUmb_<|_MaP>c?SU1W4b2coZN$(YB;#eP+Bb{vWAw+NzBULi%E%PJ8 zk^4MT3P&XwAu;l=M}#lwE0ITl*37w4ebXl*ya}`dy*Ksda8aP|1hjE^o4-cBY63~H zeewL2FmV^JfI*E4I;N!nWj8HCaIZJT}{$|J1%Is-| z1_6m0mVv&>nn;}mc1O7rogcRa-fc8tj+QD^e|Zm>Eo4!-(f&;R7jl4V61tMR+}6E9bKZyICZ+4A zt>a@uYInfrL0cNe_z??dvG|Vs#`m&s1LgDH2Kw%KlLX-S#QL%jOTIk+^!r0b9aQZM zDX%Z#hOw_BdHK$5Z+6)idTcxU(rJ{Hu+A=O`~c@m8zQN_mg_{@aqw4c2SCAamRV_w z8&OZ)qH~z4wq&0wS+aV;hhrZKvu=XIDlbMhUZDRUF4)$dF@JxlSZv zXr(bjG_`23l3p_3$v&nS<`miC`G4~3J|xRFI19p)o{&GxGsmsNzlmZT8&UMx)Dwc_H^m+x17F>xG3Gd~@R9W@S0JTY z9(9cw%@hk%(fL_m|0zSi^@Nr=>T?$1zNIX}>Qr${$kv`7$vij$vk$V!aE(-k zf#?U6`=0LztO_$-4sIoXxO%g?fv^k&ifk@|nUiOlr!+zHPhRI3CqkWZbDK>_68hC% z?ai004Vm2?FFyz=B|FXb={h48$#|Z4EwKJ3{aJUcRTk$pAb4FlMg6%Ka`%f~55vWu zMH!~){-ghxRBZEI&mv!n+VSyP-@M(P8M!nYe-0nw&WzpI_MOh)&1vraJcqRfC+DOt z$s<>~K;MZ~$7vC#z%gEDOBfsUfi15l7?Bn!b0tVq$zQ6;tqlT z@4w^}XObFa%Yv45`!Pz8Vn+J~Y-yzf5(~DOG)LmqOmYCc6aR>jjHP1TKDhw_U!`Zz zCJn;h`bz&lv@I-qFYd^Ss;K;>*FUC0`sPO+{GEykE#;kCTLn!`g8N};Cu!>AT<5mF$zfOPO3C*#O4F1ACyq*P#_C36Xuk0?=8FvBU5MsI0A*Xb|5WlFola>gV2hVgK4DyS~A zCzXy2YgS8+`R>bLsj1u_21S>t@L9K>b+w~Q$9>M~hch}$rj^IjcI+jCm$KYmEJx6mJ+~7cj4|=0~&d*-jAHkHGpm#H=1HQse3Lu6m2KT zSmY`hQp=O# zQH-F!YYv27XCTmWYV!Sdp5Al4N|$Fiq+iaqwq?nNi9ecbCKZ9x3+j{C|W+UvR+H|fu`MhbRA z7HEpMOwhTqI~iB*lq7tky0o!S5WE{`}d;rUZXu2S{8QvLA+f zkk=HY<@bx{$w_c7lR9NKF=NJ_Q~vCW6h*2W18n2$1CeR+OO^4u4I%u(Wns^RzNkD! zisVs1#r^mwkJIm^=?%G685=DxTmJ?_4zH!tnIm`kK1^DKCIynoF2%X0?AGMxGVhvL z3_TLdJ;Iq1a2jq_p6)(Zq9<4ts?Ux!w$UK@Jun%n(>|gHk?Te;RDw~@AMXT? znyeM^ik{-AuHGmk=>a;*$6OpGVbv`OQc-%mR68(F=a+^T`UlebOIlu$JhE6m6e;d> zPV6f+>i!}od;j6PVe)n9YRnq|Szr4BfXG;fV(bs48Ee_4ewtQVs)*r=x`|Su%rUz~ z+VAZ#8B9tB!w=sYAsdHsLZ##k@U*Kg1b){`gQxnc$`f6%Lyo*c}Ef_uem-yX69d z-5FaepVJ*J}ggk}q@FWAqkk-0_U|i(vmUUSTU%&>iKdr31PSxRdNt8}5@W9u;?Q z_r@LxfMZ1+iv8ObhEH3xUA(yjlW)&7dF+6i{G#t)b@-(hO%!uEn7S6o0fbmH76ViG z(^Zba@f2<+%+RgEE2mIR?l-1v9`3BG?L_(7{;XU#mUA<EgEymoLTnQppiH%1AH$=1npYHOK^y>K-Q>lYNjH}NkA#chbmxv| zRy8_%m?7l*p@$&PdE(FVwp@E52HtBAzc2_lH|V<{cescjuIa&Ow62^l?E#Nr_vFd zIqiW=1D_e4Qrrt~oAgdb zJCaAD$e6JJZrkQY$JZI{Z9pou+1RNG6d?Bt#j6-g@(A%{X;~&#(1Q~^s zAR1ZllXGYnnts8ESau%c$QQ)gQLfmIZc$>4I_o%x2#-;DE_6jB8`pNO)PHqr$1G^y zmYxgEc56FMT|L(rIsO+0XR{2jkTy<0y$!Sgh-{WWkZhWnj!|?) z6;VhhAq^`piC-am&+pbMHYf zx1dLcB6Dw5{@+tx&yQjqTn47=y3pQdLr(v^HgW%IXAgh-ikI`ar-_dZkq2qVMgu zCok3f&DUJvfJ*czp!$0}|KiCa@JKcLB|k04oQ_pkt9AuSrg^NYPNPJ~Q!VwI!H@9X zI`%WjL!SvPv&lTkF+un9-N=~n0~Av)`J1;OfSQb_ns z&i{_)^B{@I7?D3LsMOOPd{HqN+z4uqJoWSeJlaV8)@zl+|8NuQ2PWdsT65@(TF)(# zRrt`{!c#BT7oGpo6#-i5y*@qDjl@2eAO(-T<>w$?D#VX+)9k}a&1c`+vP^K&*xjy6 z*9b`Qno4vo&%NbizxsOd130EZq_2ew9FYY9Vx?v;)g{9o7#L0)qX`pND?V?jZAf64mu+Q#6Gy_ImEM72%cgmm^Gq5fT@d8Yn= zI`6fWQ#KGbWS&(_K|9F*uhTg9lC1XqQSX__r}HOz<9NehyQZ+C#DF>($EoR_vUG;%I z-4SA9srPob*>|ANO)Y;E-fPEtM^4<$gN_VKH;OSN+i9wMR4!+79*}(Sh2q!1@Uz(; z+BeJO-g{3d|IJeguO}|&URQW8mz>a$V%|D$;Z+oW(`e&3qs3SaTq&2o==+Lunynk{ z9wEIE^n-E&?hrC-DZlTvubW>cI!~Cv5H70Ys9#d{vmL;$QnP;qdY1jEvjG{tUsLYZ zqi-Y#Q10yx8#c8ga&u2_p{E%>5@uu$6iMBs#%GDdz`Ny$-DZ90IxRbjYtSVgUJ3 ziu{n${MGbMqF>%Sd+PLUAQRsgJ$RN=z&Fc^pUVzr+dJH24GHBVs<)uF-9=?qIOHXq za)LAv?c@7SZl~Z|uDrt0ZPEl*$*no5X)+3Yg=vO_?<1-Ct?yqTup>FjlArs);ZwaD z{W)A#gBD4%8PyXa|2i2YC?*oN45D`%7zxR;<`pra+SH+|?Ff7FWB%olDU7WlDov6Yg-KWP3zTg_5$Q;5x3G}ns2B*FjBd)ws>ugP%gNu=?(Adw zJ6G-d6y0(8qa^>Knw?XoVcd(}4!);;J8_Wf@fXOs+ZhO&HdpkK9$l~H+YIj#9+G~~ zq9ITSM;xZ($-adsFDmCk5u|(izSr1Zf4+1nYrnv`6ilRkOl@_-nO}@bZluezwV*LB zBV8LG^101{`tIth76uBHSfmMspz9QSjX(ZY#=Is}tMfwCtWFNiMF~Ie@e_^VsE0W} z)0H=V(7*5Tgm~;O$xIECFzB+3oS*>&3Z+ig-XFpQ-Y?y!lspf~?<{LihR zr?*jc&}o}j#Vd?{Fb;l_Lo0vjAwjYn>S7TYid?R&mT_t`Tpo)?m=cxPUeV=XT*}P zYn)AKf9G~777`Mtwq6hy3j@%ZLf|U+P8F{jg_NH~mSK~4cSj{HN-y7BNnBns^&5Dk zcN^Xt7WV&E$wR+A2$H@qIgZZUXsEpI3*dgfkXT8k(a*f%d>HnUQ>;Bd><7D)seqaG zy(gh|E>v$5?A5q|9MOR{Z&-+HUZFWL{KiFla&!)>60$LF7_sc<{dzsJ@;iRiD@UpS zQ4+`*|GbYNcTwI*+oC(IH0XDcvu{NkYD+hcJ|e{an-T~;K6mZQw-7h9!iRARU{ul-50QI@Mw@p)t{$h^QQSkF* zT{IY`Hi-vid^|sv<)S57`onM@%%2(lgr$jFz{x(yzl`9ThQtqNx&_+03OuZ42l7Dy z?*@B4c&opn5&WaYE^#(S^sGU_O;|}EaX=zU%{R~ zhXS3kC6?5mf_CrvYylH)hir*OGPr8ccFPT2-A^}xab3i^JMJ1vsG zn8*xFbsZA_vZFXBr5^hSg;6>bIyS5$@!GjEoSWa`Ri0^!jdEkONnbFCzql-trs*S8Zs|;$_bX)bZtczyyrX}bI-!~z z8Huj@qHd;4n}*DOA_+=d0QB!NCV7jW1V*Oyjm*C&pUa1p%300AGrUl_lX+k+gVUro z9(gJPB#?m%`^vgqExUE~2KfcegP}sRd}FUcwktmHNc8PS+iMWr2d)bAGxqF!FoKhX z`$fMA=uT_@DCUCGehy$3@EXesj=bqAF+4f{UX#1+ANl0<9}*dzZ0!L^doet$ z+;AvT+3rrxndajgWmDHyM^G+Fl>X_iyKY|Tb(O8|yY4Nxf8D3nm<0)+1AUI}4d9CD zqEpJsgrh0E9g~##+gOjc*$rDlmFHH4=7U8!icAiE z2sK_bOjA}AlcApkjJ}&akGqg)JLa*nZ|zZXNcCVWw3q9nXZiSrQyEdGWUL4L8#*hc zXE=20?tx;&;Y(|w7cj8Fx6hrzb_|iyPKDAUXE71$8r&o}f|54U+iz=tD_L8Kn^n}c zGk=t^8e7Srd%;KC4{m|W!U49uYI&t?4mrN@txr zWDY+gQXz|%94zDSp>t%O6E}C3rN6RU<7H+y6UwuQbQb@Hg{{%i)UF)?)4%B=<6%Iv zbH!v9>NfNpy=h1OMT=d>Ipb^g1%+WAuUITq=2VMPO@c%o;UCt<7))#zFd~cL8=VHN z3{NcN$OUH^#&7My{>h)D$QtF5eBb;xGJb~b$)j^28$p&X(WidL30d~RSGo(bGCWz- zV!v|F1-23W(&)=j14S3CgoXjhhd2&C^G9eNbvmOCx+#BW?$|QLbmbyy@(wUt<)0=L z+$zl;aD2#o5dzqbU{&w9E3~1Vkwj2eWJ|AW=j<1`0QETy$$pM@nPU(bINW{h><1PN zsL(Z?)-73d6tCmhw7^Lr?EXZO;B z>yMb&%Tg7~hKsX^8FG8|#L#e8MJrL$Zm2aV%reSHhon{O)?d;le6cjVp?$|4ckJez z3{^g#rE{l}TO_6^D|LX~2h3t7u6$Ec-oa;^E&8`p@W zQcO`gfKMPNcy6W!MY$_!?MT6<4r~uP7loBM-5d99u;=Y}M)v)K03~Yibz*egU%m~2 zCj!YtN?-~4zcL%|w)1Fy5Vep0Zt+?0eK9!=w9iq|rN0?#T~s;wPn$m~Lu&a?^F&-@ zktOeBw_lc?>9e2nvvn4@r}IIU>iJFGZ(3dr71aT2@xlfD0G}cO=oUqP>-o}j9#Un^ z)e^g3b%<){SqE)z3{wQDz~Zjd@w}vz$gkDt3$)Rk<~Jk(g(>_?;~XOmhpjWDU8h`)3}gq&$T}AV|v?* zqpxqP?%BNuhI2KRHM5O6{P}vKR@(PRcBZ)fE}ZP&v%OII2TOO#Q+9{>V`2XDj_isH zvYHLwkw+^evJuN_m?&?*oEU(ytSXt>z4d&Zs#o<2*~XfVh>s7_mPO7Zzf9&-TQy@| zo0eha2pfgV0hS6)gQwNMYRfdt;^@K>jjWKw_7X2ncaHQ+kJR<^wH1t`v`Y)JDYBRm z*oHNy#3U#4H*E%M1`l)nQJdcZu;wmzg@~=Vo*!RBO4<-z^2vkx1-ILQPrP(hz`Hi!fqM%0fUl54MjS#9cWH%0Fh;p)21Io6rFXk{6fzSFaQl_sP(#w)Um)^U!k zCr!9W_iMs3%HTJ~iUKSHh9Ei?+<2pIKmNvdm<+CB6bVcHOU1=5(Ji(1A{d*YwVlut z8^tq#0Z5G{-x*d$4F_bctXJe%7IM4Nl^jmOjjC4!%+p3tu&*d+{wMLa5RXM6^{%jV zi=5u2)FU9QM@zSJ_uf-SVoNg@&|>F!;OqvAi|J^1uVR?78*vyz+g2C8B1@-~+VIvG zCDM|`Zfx>qnx8HZSW$ejC0eur+ ziy^K&tuBWnz;j%k6HQD~)8hst{KYa?-V*eTc%-=H4x%mruMzE&#_VU9_+cpn0|uU6 zrCYEC_Z72wx6(W@?v&hme zt7sI21T2=MANC1BGgBYox-8CImSD=Vff3zm&d9l;kj!F9-LWGiBLkt^Z|^nzI%q|stAiIXYrJ%GU=Su@Os0FQJyM!ID?Li6aCN#dzPUucAF(TOnF7Jh!2n2Fju zJhEGe?TK1nTwHeT!~xu)g*2yWc(;XB4=U2%T!j}|{4|`#t?R)%YF4FB8jj*#T(DO(d<=15HxK+Yv2Q23_2)#$#RaQJ1dwlBrdze(0tUS`&Px}T! zztRt0X+Jp;g}YVB+eo+I&UU+xY9O3coSjLvRC)FsOP0ASY>}EQ48a@Qk!yp=w(p#@O~@&SAM5SSJ|n&`viG&@;o428 zq5FPQe(UEb%py7Rd=mwMR%7I`@Rl>gSD4;QbjaeeKABoO)Qf97zEUj()lBQ#JxI2p z4pKH`C3v}pWp5U*NpiA!L0G>=W???~izRUp19kmfpDSO&sYjbq+O-%K2L>%{!dX?0 zO?kq!n^Mq3OF&4m$v%VYm)A*NV-o<&(KTqa_h!}>q+8>_nZH6M55}4k=3a}gBkV5_ zHhEP(X1I(k{&g2~I=F8%ZE97A*$a4VLz3ikS8xb!bY8Ksy$OsjS!M>p^Ys5B)HY0z zZnfzar}Y8C2Xr(dxK-kZkL-kYo8Okjp3t#&HHwO~%zcA#bh#*-ku}2P^;@xX@<;*> zg^ON6kLG;z9-xmr<*vF>!T6XI!UkZ45@2dSnY?r#D9W>jTy0+rri1?l14V1JP97=n!bxFJjF72!Z7 z^x5gtzvP1wH;ruZl(M7kqdDCyu5*mn(Q97gbYC2ngRO=gOj}}+ioF?4PHb_Uh{sT^4WR`lk~El zpSnGSfmT0X@%X}@lVwO&Y>>LH_l1GOi0Y~Mlz@#u^vS{|jHX-ffvz0emdHrml*n`n z#zIQG&_cVgIg~;+rLS%*afug}OR|*&^vJx^qSgwNe~8}WWHpCNe%^?I*ba_H(laXR zT)?yC0{ruObpVnUqYa_3`~V0@SH2X+(3~vEV}l-uX9K8-$t3qpSNe@v&7K~xLK*{Mn3RcPA;L09L zh}|Zt@|V1Nlnq%NsF8k{&uGLC2N{B896l=xxi8QFxrqBSfF&ZK3*(D)eM~3Sl9IMz z&%SoV?5JdpsJNTHznHh{b2#UV&8Nt7mM08D+t938w(s!WxQVf>H&4Px;g&|%QoiIm zIo}=tq66M?r6<4ID3vAnZ$=WUc`1?P*AepAX8c0$)Ldt#dF>VE-(#w&+ZQNAndaU5 z-nPG%ZGKv*nKLjDL;z{D@$t%NnNPJjpPl62NCX>GS^Cb1hx@)#7mL(mm7t+>QEzYJ zIm|JJw;RViW9OcR=gEa+eKx+%!jBss<%&nXMN%O#2DTA2d(RVf#i0NvuInHgbbw?_ z^crl-JZ`DEWb=JxwZkJFBa-3wAyA6s+*CBqVQwPc_D&y&t$#ici3c;kq$Bdq$uSJ2 zhIxARMI|*P57iv=@Hbj;Pg@s#{b9B(yd3Hd`5ebG&MFw;!CxWFz0-K z-RZ3q@6IdNuf92NhX?-RnBxc0HcN8V{9_XjZ$-);F$<(wuCADdi4hwLi2$hnxYdzk zR%1LX6@lSuVCcEj>dSML8!>_LphmrC@k7hC8H(5F#J>m*5idUUE|hmbWLZwPcf#E#@Ci&L1Iy@Wb*lw(82*l}UzI^ATBx zopwR@s1C|AMF@mA;f;`Qbsnd{ObkClt$N~Fe~dK!G`cwCp{SWO_AGo?|q@S zpvR{F{{=!E7{)Zf-?--ahORmY5wYlKQhh6Oft;&Z4 zf_|v2Pc;oOjgngbD_NB0wu|=K=%=b6=lRoM*^j@`wI*g)Kk*(X{sOSlI~ghycXR=X z4^{o`E>~C_m+G72n~j4)6tA^!7ZT4d4Z<29O{(X-CLOiGRW(2HP$MY;%V*bl zlhPo~TWr9^B~|cZ)4RF`qjgH{%b6Y;hts0i`*{^=b1Q?ShQ4cbtx@D_aJI`anV#&k z3Mia+^2W;`r*@BX8Londlce%>LkKXL*&jr3yhdkbHe%MF;WNsb_~n4bF@2O%%Q_*X z>SE2q{As74agqB^cb`49_1(FD>A4Yx_Y?}d9r%tixLJ#`lrN8smN}0V_F?Oi&fJhj z7c>ZSrkG7*^>yzp>a^Eg+l1Fk<6=negL&G5?`Q%)n3t5lz>0vz6+FM_f8aa_ z=AqlEBUMc!IqJgy3Hn-L74P7QUCrCDD|IfPPr$3X{qcCX5TB#ZqY1&L;FH^g4@qOt zjr^cw*pd!Qn+#OhL>t(W_AWlBuAw%aBS6x8I}Fj@f@1;OSRp?e4^uC|iteQ#kn=LE zwr}}P*v8dVbi_M+Yc+!M1Si9gv_&pF`3|*HmJh47wa10c+{at$LF!alO4dTQ5sQzq zh#XbE0hjV&X5g96$<(G(!nU2fCah%xD(AWo9v0fL*SNypgl(7u!m+QxkiKuws4-vI z!O}E5v3Q>qWZETs?$|+=-vpltbMwjFMPkW(mo=>~2ZQh%=Y_4Tr;4NdK-*i=im((og=xg->_A_re})c{_GuYW9I%B~ThPh&5n^nPBIEKCqsY{17+8ILBWoT* z|AV?l_6KY(!Hpb_dHl<-as$&+5pa_$KZ_TOYCL$E7tQ_vJiL)g$a8|?nbpG z!i@+yyLYid_B`Th%jgEB3$Q&xfu2X3(_f4_Q?v!P!7oE=(&o$0=nQvRWZBF zS~r7zt*3&(Nmv)YAGQZwW*4^!RHFG=x_`ivL}ZptERk|&;)*AV$i-+{?fYm^+y{6Y zJjBG3u!$_h&7yu)3vUYRpGMxx53x(;sV{_0wgdT?PE=a+karvH>0lXKL|$bVLtPj0NXWyQuV0z4)wu72Mjdk?Y{c>4x4TRQi|{ zcFfwxj!p}IVHZ2rDISA|5f2W*Mrm@exe0F*V!~laCZ5FKWnZNOxEVFD;yb*xy@NIe zf5OZe6G8`!+)ZCoreuJ|Avj;x35FiARFlz^IznxfP6CU_#RjpK6{J|G8mdD?M?{mn_P`o zdEnz2#(W-;OvTk+C6*?LX5zh^jHgW^`(a0jbkq@)YKlKGS{3lT!v**j|J!Q(QLwVj z3i2Xb13#@lBf7H70Yo}Pk8XMh-+D|@o~T238)IOdT=_fI5u>6yypx%u+?Js*5kbvsro3Cy z3o@RTL?pro^7O*?Bo zx2b-S?3=(1`Tb48cD_YklAk6oMf}ltsvtyZd9)~c$v1p*kz7~&mpm%HIG6c-#5$IX$dt1!>!jY}Z_Oau<#Zawi>yF5 zivLDijI*)V5qKCD56=sZz_Ee`I<@9o7~7mjE$bYaJ^bIs!!$oo@~)@KGJC1G(VehJ zE1kPF5qw*1NiQ&L#kcA-^4o7uh+!GzD4D&srQdHr-W zO|u3++t2ch`7V?yIt~nNH{+p08%(oTP@8_5{|FoO584}fLGmHydvg6TOqM->u?aFf zbh`(Cirsrz=O+G-ZKKLUFTgR6$C(>D?ZG4{2sJQMOM3MSTJ5wRge_(>nG`)52!`9p zc*J3FPFPE3wS8&AW z)6lNODb!u?QhHO&4|w9Zoqh99aYl%5#()1|G(2(=xpp6-1M16}QY;qJ7lvZ+ByTMf zlsuMc#QP<(JZMKQN@B@TK{(hpYRQyYF!kPO!wGk6JZv{z3mPqLRMYHX_D!5Lj|@7a zCuSC$xq?oU1^dHPmP1!F-}YgvzJ?FIhdC3;^R+x*fnwDD%_*R9sdT#EMSt(w&-&#J zoEdFS!M!m9C}Hyk7?Ku56mfa*(B&@dGOD!mW$ud_JYi|X%MXN|d=KB!KA{q0)-hc? zvJG{rmpYf9)5QiU-dE6y zm{;I`#t^mUc9G{(j&P&TM zOcl3Ed3034rK$6Gt}lcqr@Sn3vvk|=M1b6uTOUrpSf_}``Lg}ZPIk$g-#V=BSZ@CsenvEr_?*A63@a$E(DkCTQPh%y)J1j(+TvA7 z|H_XAz1)5Dj=KM$Q8GQJ`{E769^g)qFX-h=I=7CD*jnRr^eZqyvxu(IdxU-z_bmFU zoXvbyy4H_<50WqYLr=oe->Hk9h1834DW{V0Tj*$cu5=Bjo$z+lLpp3sK3Ms!W8d0Z zw99NP%pWzMcLUxgYV8BEUvLRVHN61yC(PppuY5Q4?sFA?t_E+o^%Rcf)bWoK$on~qP;fmL&Be+QljHZ$n0LQmAy%JhPU!o zg(Xo$PvLGYu$Nje&QUEMoVY!|9D-ew&k7syIheR^r4~xgN$XU=8C4`1cg`hVWi6U- zD>2G(kW;y44Yek>24}@rRGI2;PEV!5Fux%S=I9TMu<|x|qTOi_cFH;F?61X>^2aX! zi;_n-{m5w;9vV+=s!>wxIuKU;&gY6)>M_F-G<>&nBkDE@4SBqPTO+Fh*3bBj4rTr! zecWO<5$P;ovhJF6|8@)R*$izKHob?>t*p?5;d|((@$^*izQ3^9(lnB9!;>CodD31d z)!Ml;V0)P!5$Hu-M=ztx$kxH1gm>uXt~K!F@f*~LQL#-Yz@I8JD&2A>h~@SXo%2I7 z7^PyhtfS<}!G(#sW-&}CIl;dAi{w;xG(lLqNq8Hygz3(;0p6NE!cC^Z3O;D>qyLJ# zbZz$p&Yrrb=*V`n*~_5ETjSK7x01a3{J}J0*v-nBn|Mb34Emp6{t}*Kuqx# z{6nIAR!Hsu=dMpV zZB^Z&x}z${RNxmVv-1PEq4gsoxhoT|+kL$9pG3Sh62rT%(m~v%TgUOHR76w?nF{XZ z$-=5_;HKqWr|~?Q90KxYYpKe(d?IbVPgj~_L643}LRXFVljY5LZr;J{VKtc>#jfqJ z)UT5}y1YcH)qI4E8_)g<4KrTp7u0^vZ9H_^OuXe^lGF4Nsy;rNX|3x73YO_d9ciVW z7@`IT#gFhUV;WV{IgftfkjRYZd4!rOTaOz1zeZf;Z;-W#(rlBnm)S{olbe>}`P5@@ z6c}dChd$6fHuA|>2}#E#biF{2M}=)3 z;3h;H{1c!1(MiiMbI&iFi59+pQ2}kRc)g06v&A}m5a}Q8O``J!dq_7(xF_W`O|kAiZpF26RfI*ZsK&B z`ki8hCmtu6d}PDSb;_%(uk>fS7`+FS5`RE#Y;2iMwZ+V(qkUK{SO!Zi52CTwIw~b{ zCE2(6p63&<;BD(_@Qgbx^@srJSCZ2pX<>~H7}V6Jc`Luu`Ik^NdZ};bzE20VS-_QR z!AtBiu7w_2_A|3b%0WC2kZu@^r{Jl@Oad7kJx%&=NnLdUy{2L^{v`j0PgFlhc^t;K z%y^Wr%O0_9X}6v?ylj<))Z@Dt1hxv!+A25HWQ9H?Bmy;#k4pRzuS0KH<@RA zKBc1F9!kF&_(=F%WUt5qP8mfzq+2QJhNS&xswVp^Z>3*n%FTI%mE+IM6IOAJy<84c zUrsL6QPijCUg#Zez2di{_LiOR@$PE+i+MZ#Boy(y*%?@BTYXaE7?s}nBid5;6SG;< zDvm~uU&5S}&zTa9+7i`DMn*r$)LK0U&!gSAm#*liuX*m0Zo=;ou=&S4qtTF;bRDd4 zie>#K>35T$cF@*zQ%%!ukW=+0{#K08JA#igJ&xXi{%6*aBir%ZLTL5$i`)=Yw@-&P zp0G;X;dA*46xDdIXgAv*-^#$*SmQxc`7Oh|4V8BKn7KmH$pqE*6PZC)UDq$B->6bN^-DD)dAAE+lf?-xR zjP-6p!OG;UY3T&dqa;3izYI6VJ8>p;ypB$Kro&Rr5)>vq4ps3@;2xl}l3jvYnS@nm zq!Zsac;#)u@7MAm6t!H^H1#-A4@f-GsSQ8CdUWTe^BVb!SV!W z-tL+3YVs9W6uTOo3C=)))1^tfC=vF?o?>Q*>48A8%ji4tE8sx(0cvF2YJ8AyMtKY` z(`8DB;DeJbT0PwzeNf-!hAQC?)I*ae{l0b^1}b}Dk=q$&r{=F1~Kszd@FbX zpN%)D_MvpQic54})eUBkkXxv?%#SG4?d00ql8jh9$!w4|juUH3Eq4x623f~=hZ*?< z*=ab$MA2)>Y!(*Iw-)(yL*uU-6U$JBAURL^4#WAO3KS`!j|^Jmaw01_$W5JvbR*p} zp1Iv@IT9YE zY6QLXVX~u^6W1u|e*kjJn6j(~q@FUwT#-DBP8xrot`w@1^z-#{VJ8*Qvr6_eCC1$3 zmPQ*2R(-Yj8J~(4>D>4KA9=^sXc;5`02Kfg-k1RuoMIK4yoa6z92K}36#x|m6$k(T zABzY800000024i$cX(9Q*2X8j7c%L+_uePTB$J*@dM`kF0-=NyAR#~uU3$?gB1k|$ zFQ8IIK-xu7>L(Wz!B8To6cMFfP{eQlPM&-JSde^(w-Ult2Ot#-nLs;t% zt_vGx`L?i?^A-qedDlVMyvNqU`dzIN)-8`GYfAPC>oH>|->>>W*byF^g{_J<6gFz{ zAHt3+7e;2`K0?^3t%brG9B3D|cEDWNDYHfjYgNN)3Ge(UtmdYD!j83D$LFFbVZ9@G zUR$w&=Yt}JRaedyHtx|fVS|6N0aQ(S%tTAUMvx|dfS`qTXRrY<+yr&b>c&wncXjJVeV4!^aq)# zCrgFZ-91WJ*Iz3^=FJtt8ii{LJ7XdHW}SH;tXUA4=r{gH*q)9SVFQmpXU(;e|G*;~ z`}r7fmLCC)(gN~mCKtA7sVTor2FdajL&9dy{*@g)iQ1@zi2NJR;(5fo>~(d&u;cT| zb;fyianjLcuRPFbivou{eX=$x7C*DH>UgHA111~^6gKl7*rwiO{ff7Uee!4bg-xEx z%6fk$3L8Bc*X`C4OU}o(!q$Z)kToB?Z8-aX!nVHD6ShgYM_ALykA;ohL}tqmkl7LM z{*%nQgHp>Oc-1{GR9KyZ4Z?=LVEx_*q7G4mYE8dCm^=n7&sg#mQluj9xmK)sRLaiS z*YTldzOWvtYlIDxSZDGqh3EH@S+B2F3u`{1jnA`w!|rT^561>!*Sch3EjFA02f4Gb zHeV}*?fUJeu-fzPz>W2w6ct~B&uhuPz8r*I9N|*aC46wY@E3^H!3viH#ARWMw{;CZ ztg{3B6PI2AtNHv@`-ic_vVhNZt<{rbXAYC|zl%;gsnZX72jULx>UPM~ef_F48d ziUh;5Tvq7#84iqo31S^z90#RUFw@4D3epM47uMktxh_b8IVJ=D7FOj%J&tYN#cH0! zWx15~M<}T{_3bJtI+!ry2DM|`d;la5QokDSM6-(}Y9BY?LdHKp_lyTvr{o8480k-S zm*Tk0;~`4B^xHRtZQo4o`&a6Kc?p;l-6tYNC=9Wb<3n3F`ZMhd)UtF3`Z?O56Xtk^ z2y3Nc<;A1njJ-E?p;!;|!#5NQ>zoLZZ9ZgtSUE!)sLHgTa)j;M*~9bBL=pS+G|W-d z@vDb0)5nmN9ly$kw|~K+_|L%BZU@ojO(Yh(SNvo8%?21U9t_={FBUfc5=}1lCf}$l z=@bbUS;1x(eb82mESNmk7FJ2ZWxnmZg)ROC>6nMqa{Fa6n7y+I|Hs0pfLCP1sdE#UYmppx(c z?4__xnqaPAIUEWNIx6PYjsdFj5>CU2NT|-#Sd@cImrVW#8^4G(w z^YABe-!Txoi#9g&lEEn-;cf6H`kGfcemWmO6I{oElI~2j)W((FbMlE!=L4!B@;Iz( zXy*C29yGz=*89SGT2lb&TEQTx!!vo4SjYVk%=iC{zOT6agS4_2J03*wTwh5l(EwIT zU3#mZCCv8^qBB-+rs`XE(Z{QkSXt{7s+j7H=SI#}1 z@{*B_SE}<=g!WXT^UXpHqE}GA!TmFYbu5M_?uWr7tgVlD6Zk&Nf{N=kK*!Qu_*La8 z;?0;f8DuJ9kz)s+(+xd@O=)yvuNDwR>G^~n|RV7&C!IL|r&YMgqU7 z6}GAs)h(!p4|an*ueS(o%-G2{=Ifaag1?2&I(JZ&;i^e1rJ}tZ_-1Mn@TP%~X`c{UNL{{tm^n_&5GrUFMmn;A6rTt>niw zH$k#d&R=5|eLl!l9n5Ck(XGPjZ(^_31S-_$44=L&5 zJdBMyKy~>0;(1Lm6OozvHE@vAwF6B-+eCw?3#QUNYp3Fm-fm`?F(LGV{B5W~$a<7H z`3n?aq`t(E=isBi1pIwYrV1N-6&5uIQW1Sw@H665ywX@ii!$%WAJv_UuO z7NeQ=$yBT3b~ILAff{G@p%uP2K)ZZTILtZ7jFE3UL|%HReUEO~rJpBty(m6g93}5= z^O!xH9ht!t(@`FIAyboAhBSrVf)Alps~@e)J1~c4Oo{!jKyD4(J9xi2|Mxwy0E?G5W9T* ziLj1mz&6yAjK|)izqm{Wa~oQdEI5kJVkez70y^MB>@Tn}%#x`!KS;{XTzoU0PQBQM zgRpWDbrJtD9-5w@UObn>pBfS_%WQloY+W`y^j$`E`+A@r^*+oYsn+aOrU7$?b>lTZ zlrwe$Rjgb0x3J^((${nbskW8k@d1(i~}pQsZ*B2#Hi@W<&}GVAY1u06hG zK6e|XB9Hdq;AKEwiZ{Y`jl;|dR*fKR(#LEN^qjMgZXEiMbOmIp=2H>zm*`N+A^Jl2 zd(yOK!TNev(D@L5@X24p^SSk4o4*i-x2cJ-x{chWN~i3w`CzO45ojzw&zZ|*5_%s0 z65VijL7kL8gL6R@$QZxDJfpve399oUcSHrVi6!U$U2smP3Q9IoS;`NY!z~)=#dF*E zrmd2dM;Oyznnr&P(~`{a`53=-D#7!hW2l__N+vG9KCFI zfA#JoM_Ff>zFH$uTC2@W#fm}d#iW8tvsDMW!qGPLs21_^&ABbP1E3^E#f50ybaURq}_98aXS&Q7z}mO@MKuE zbbjWJQBNBrJ=@SC&jD_kbcd*m-iv69-S2Q%yW@rIzc+!MlD1ELAFSfu=TvT_4;qDY zSk2!Bd<+;0WKBOvH&pxZFiZ1&Dc5|HSu6FIJkXPOvQFivXq|dKH;O?iXhQHlG$G&u zw*(1qvSZl{&ikr-8=@}4H_d@_Ab)}xyIvQpto>PMe}fYOBR5M{2?8G_>3lhAN8~C!XLYLQ5EIasCU~T z)^Fbr6Es%vSKA|CGF(}jfpGY>K9o7502bNn$|XF>QRYbyvv|q+#rvqKX`gUqpE8@y zSjWI63mC^)L*o*Yr{7obb5vcJu;Uy-)#A$m88(rb?&qw`>6L-K(cJba! z&nk9iU-=oPpej3N5Y4-5GBWgd~$d;iqX|t$Y$2d;AhW#L5x|aHKc?{@r zeoRWb$LVWcGS+W&2e+)}aLl`yDadUlyDP52nFdcfomD4YCNYBEQE&*xW~LK$Q7QK} zZa!2{@pb$+`2dxO9!FLB-QyW`?M~@?koR7|E-6o$ue48c-buSey@cH7RE3xS1foz?U?&8)#zQ|`AkWFj4g@Xy_lY*u^#^oj+0s2 zSo*216U^y(5C3ER_^i2;Ij6&)UY{O|CKvzMPmY{;3!rH`hulTd{gr3w^|1+5mb{*4 z%H!zlMukM-{)*`(+Ken~T!US%Tev%ldkT_{QLr~_Ja_m-@6x9d((o!^4ikvgkv$UA zsHPwdw65bZ80N}Q508&|!(u6QOq~wtPOOj)qH3+<$=$>kuno~ z8@|sE^x&@v6)-3AZFV2^oM+s(qEwZLuFP^Wo>+Imkn$htzv+L#ZKF!*baM&j$2~(o zv##=_sRk&i&ho~`yo!wEy3sdH*Dz@))zo{>BD!06I|@)3kGJv1;mnv-+-FYslMZON zj0slz8uxqg#$>!IorsDe;9kBVzcQEVYg7+URx42eEg!Ir{E(a~_R>=<+gUSt4Yk&E zgevy_hook|!+-x2Ff24@9!|Z5uI5&98xtbEn@HM?o_j6f+>~#^+1BYO=P$Ql)ZSM? zht0bQL-Mw;d)Z>L6BJHwQmUw$LMQ3%i64lmPGFaIJOjf$^zOb2ro^-uRLE#Hoj2o0 z_6`3RJ=f$Ilf2t(CUe`H=wZWLew*7*jPiNZn)em%<+NPDA?*gYA`?P6pSG&OQ{SC? zeupC{Va_%@$ym!2s?p&qMNO`Km&4)Av1o5*KKO*`v!-G;$j4UEL5B6DS9<^J|2>mf zei}?`Zj#>0rGkJ{0M%sigjm{DAUu5EGGYUq@ipO%w&PLwruP?jCP7Qdn%!pRK$eit2re zJXTqOn9VL$3ttZ#Z9Zj!vY)d>*zP;rfR(*uZge&vy3zOG0 zi%GKj5YG(1!LVz4KYE$Anw%P+ zKvhaNz=83%nHjVeQTyXhF^5>Z&4l1`lG^b-PL-K>qeZb7STkWMy{T~xoJrcw3Zws^ zT75RFgw2){MVJ@Qmuu2%64ua(GS9$9&2PB_H(Nm-jgNq&dK&j~$>&&e<{&qOdJE`E z9y6$pY;*iqMU%V0)2MWbg1kGQVfU_5CL-0l)P7Pr+$&#)DrQ@NK4D=#=GFfp?=%{& z>fge{xV11L_9u2U1O{2wCsc}AJKembg4u8QHnOLguIDqvd9$Gaf4ZjA?<@?Yd3y~u zw2vs#D?oDoLgvz_c)ZH7LmyIlu)`;f+)000#L74DM( z6{cSm%Hhg492L+Q6#x|m6$k(TABzY800000024i#d0dp`7RT8cR#}E^h8dP&8)O;w zZ5Z}N_C-)cb_7%e0R?w3L(Q~qS}s>hw9M2jwVRfCja;%cucldADQ;z^S+;n~w0ocL z%jeTS=QE#op7(jq@;kqC&hwfY85x~*Hx_c1K1s+4uXqbN-acDM=ObA{YFDNTX=rH` zQoh|<$Z>NEgbWCE60-Au3L!1Z?g|<6;!Yv!X4VVoXDbuZwX{&k!khut*q8}9^W3eYw*{WBrWk8EM* zHJgM~@0%%P!?#bOf8|d?CT-a$WO?xcA#GNa3MtcZ;siTvap~3zsp>QE@467Y^ynT| zM6+9jIXk2e*$Y|qGyi7%eqG4mMVwo3IQWi`PPgz?!0DwzrY&15WP4j2 zR{X&Z30E$%=W7{47G>oMIi(7_s!rh_|AUY4{Oo;Ls@yN6>5~t6ZhKispLs`wEL}}p z!gAjea_q-@g!DXTDr9QN1R*1Fuu+2&@3pJ(e7qALPRz<93ckfcdcBW-BLCuFza{*Z zFXQyy%gcmxGxkG&{aIq_c^u?u*sW#@EA@L`A>uU=MD2-5XjM6v-3qY9?*1&KHGa^g zCF7Z7)~G)X05jVjW6Qr@7t-oGkkj!Y=g!dJ6YooCp8q><*;k!p=c^9bIE8qbH?uyd zV=5YM0b@atoY>(aD z&g`d`gsgbm1FTk}c})T5wxttUr5`x)3Ik(vv(P+R5sTG%CZI96T}auNzCupChu`G8 z@k!K=tjyUok@Yu*vG)*q`m`SsG9_g^ky>^X+qdSh&n{9lgv2$HZGFV&0y_5^3*LZi#4waDUi{Q5>e}JCKCS&8| z5i;F<<9Q*wthXAGHG@L7{7Ss+A3hK=(dsrVH)dkZeD?BFdgHG7792*MQ>Xi^ZoP{`cp z$nk+QUkd5%_!ZHA3ab;|C0-M4;Og0C10ZcC7J0q|!{^T?QVor8V`Mf_FptDP$?oK1 zL>?upPW)JYc(6Cnd9&LfVuVS~gko)OZtHJu#}jIc(v zNXU|heL^~@v}dE7SeZwx{$mk4gEhNgOYl%C~7ad#ihT7uXK1)y@}|@E%m|M z$9G72Mhm&{J{b|E!}gg2AA$tZ5w{bzd?aXIkOFKB!Lb8W=ZpT)Ttg@{Asr7 zXMEy>U46#*wrT0-SlR*``0PB0SJ$F#sKz4u61tSdOR&gK z%}TT0A$I$bY*KlG$yk4^8>~%5w>#$SToZXrdbV3oF)vuI3&P zRdCIjSl`3j`Rh*z*>&e03l zZ#!-UZ;`pMjr|b&n7mKMS^LBAX`AT~G@&rI>+t*N{1o1{pZ+87?p*~Q8d&MSnw_I3 zVO_u!_B6c&W~Ldlv-}9sWG&u~+A>8*^RKBUjg4S61eY64eg&O1o$zFw0z8L)Ot0wP zl#eZ|$(@33`hm0s)GzBAx;T&a0c14Kb9REdl1216g_a=4$rnaS&b=yR@o}nc`Ykl+ zJnQ|pujDGUt?u(gw>f04=Q0o&afG)<88~4NSW^8HhIAZ&osw=63;k2P<$D^>$KNCR zf$Kqq%_8#MX=^9%K1UvD<-|In1YgM$kcrpmDqH28?!JP&3o9kBLj&ofOg<+1(=zdV zl^MCyA51L^fAxE8+3O`_*?xSbIRR#L-@rc^JAc7fSHIwt$yB$79q3tda=MV^WfCtf z!#|5R!kkIN__-#HNM&uI7qVLlx)y#xERwanTPam-=K-*k`5DY#TSU*})(yLQ6-yj4 z9%i2WIvyMQE|D!iLiK4ojJG2PH}lp$ertEZW^twb9H{hoXKB`Kz86}}RF z^Kw9cyIWM`aZXq<#u1*6t--pWZ^>DW5qzE$F7*R)DYCR+Mt09L2NfP83%aDaCTlCF z$5v?Y;bf`z>m@opm+6IC9mx7nX>u9+fE+JNr}kM-hKuA`p9+~X75_I5(3{oDK!w%U zaI9)4Na%hZohSE)3TZZ;h~&-v7_aUJhcVCn+lYP}Z^&QZU8gp>%^7cUx?a=&_vz_h zz#-;4{s%rc&`U>cV~32jXi{b)^|Zb9VDfe5m#NFC%X(w7%!J!Yw0xFIn*0~2OD&@lmM2*VX%R)njn;0vJJ{IPMoi73z)$UMPKnR(=EO2|2tPyI zeLcXZr3}xTm{H4G-2=g`)YFDp5WDhEu_DBpEFO3OZoS*6wy6$yBVsxADti1qSE@s#G`+?zjS;e-p+U$x1Kv>d)9tvV z5`F&|be>~E<|e)*&DT#-Th#|@|2`$Wj+}BHBvR?8II;O%Ix&NW>_~nKe|B!6K02!5 zmZ(J5B>s3;dIIaFo~5Su-8v_v+af$RY1be4#|;)uwE&r_LcEb?%5(E5yyX8ICFJQH z>{BDd^QpbuW0W+7@$MsdFD8pU4G)8b%>0_&QXahhxBfG3)B8=_3mP+?02S_~Smbq@ zEYKe%(*q5#Xj&+lv92VJx+1r>^A$fQ_ZHXJ@?&kh_W0H zq)lRCb-4p?XP4lyhIT5QmG@#H2Sd7qEY76+^14YsW4RxFEbo%tGd_i9e18MOhUeIS zv_>?qqX!#rfHw?#$c|VmI<)L2yj1=fwMns&-<|QGrEEDU^4UyBXHgG7B-fHDZa?#` zO$pP`d|N7ooemzJeUy1E=OMVta;1k)d4xa1EMRZXCEOB}bivX|kMK;_0yK2G3v0y= z!fBc!Y!ANzU)fGX&)!CAN7QP1iD{dt_ysvsYOgE9=wnWlJ3QbV-6mMe={z2eKM77+ z#-P9ZZFXxJVCO1R{9}3rq{V*2y_Urr)B^cRb~B%01k#epQ2kmu3j43&joQ=fFy<8X zBK{ZNZB3wW_uYg4eP00;p0nu*Yh~yh<;g^^s3zAmKhY7jE&;3oMHNvOLVrg_jWDHV_`?Y*J?p!4a7BY3oFC%G6kL0YJ}U1LTvxaY?&^EQ zlhSXwVJJF8e9f+qgNns)b-aRFP}fTo78b$wqqhVZf3Wk^oAg)u1=L^Pt5m%b%{Bg= zMFbVz+@Co3;JT02aurdYuNVToJLw9v?O4>(#LD7u={|5XY+vXF z(#q#box@{zzCMyn53g8*y&r*9tNY~Fszy9xJ5QQlA9AA`qGi3Z?7wI`gFB5Prv$vMz*i~PsK0}SFrM2* zG%VJgLfb!C8G3;{%8-FB+cPj&d>c9}x=h#Vw^Qoak8>}P8UDD@UPivP*>uIU1gW> zhV3aXRMO?+hoSQges4YV$53UtHh76arzv1Mn6Jdn!FH8<@so< z9e{O4)0#Pyd=q^mB;S+Th zaX0Cb5K_QYTPD>bgE1Ucb=aFd$4k@ba=NeBlSJSAFl$=PIVIyBd736Cj^6#8-gA6d z$a#C|h5TFK-l8I&Jb4x`#a)wbt7hTt05d#O_9~cf4%KbF%oRouxW*MlzOx#U;# zc03Tu#ha074s7o@1rPh&U{6awus3-Pp10h)o#(#9H1Z7D;d2lanYMwk?k;jXVt}c{ zdx+|j^*LQxp2QxGOR2mj#`JquXTQZ>%9K&iztPIBgN$%I4`+J5P6e^c#jdCs;MwdF zJwxD&ps2)Q2&73n>kq-10o$-T%#r8XNN|{S5v{U6qpoEeLq~u%u6*@!ha&${_26ai9WK)POPjaMhWSV1i0uj=; z!s3fW$A1uBQl*kX>A!H|=#5$EEWSjfy0O3RG*MQ}0bRB+*phjjorARaFwhZwVjf_r z(@~zXvY!6GWgq$2xQcXrS5#B`_Vw}Tkz+v)B3(g2r3t8XLa|U3rAvU&q(cHk zKw4-=QBbOs&^v^Z5F&&SDFGF!(jfr?BvKMe43QF$nvXl~%m3ki`yp@pw?{@Y=3Z;g zxz=1dt8gZ9LvHsYS|cg4-Dvcv~%jc4WA%t=cGTt^y{FV1*xqm2Sg@ytNrh zv0t8kzgmztWz_0+&EIGE;!Vt!>Ak41i&4Xr$_4KHzoV0**pD*X(rmbu?2^J$R0an} zlg3W3)po4inV8`I67Cp1kGd0d+Y<;#_x{TN+wh9vV6*RY;}<+C_%Cr_v8)THA1I_* zW;Q;%*JvXWx&8PUP_l)?!_PEFUF5&MXDigtaecAY#M959cknc7JRcge+S_2CVtd7J@nevp zhm}SltS@z;)nL_6^+SI``2lmkB32kSYgYA`nfDQoGh@GGk7uFM`x;Ni`uxb@ze;E~ zR%*i~qr8&4to37^O{jzE$}6IVzrW86XD*FN*@b*wRCT@o$oj&&jT(t8hp4P(hVf)x z#E17?ha!^-!JmhYZq z^qZIYp9Mc;xco-at`d!@#s;R5F^V!`t~Rdq?*>acjrVe~e8IjobOnV?N%}eK71b`b zW{70jj;OPyX8aG~#3|&J<(1YG!K;%z(WhJc3+{dUy8!HY$=16)Cd|;P*LIe1Pk-yU zOw!eBeX3hoCKr9$G2qVL>Bv@%NI&7a%$*Sp8@cuEEoM!=XmXqiu_yWA3~{_XCaHB_ zPxj#EtoH{I%uTU{@1W(BnFumMDW#>=m}z2>2H2gLTgu_;ZF%R{nky+Ona}gOYwz^k zPSz!jH5=*KH<(w~q--s|Fe^$@hWrm^KJob|3-cZ>yLvH#;`rc@_4mhi$!i1r{l<@L zElB4tVp`w3g}~&9*PO4x<#ycHP?EUO*0R`F`?EhjmkUJ@8HvzYKY!r(326(C9JbL% zl~+++?bP<#57QxmdArTOi#)?!pltpP>1e?=v{T}RIW?~kExNyp0hd(rpjUcO#K4x# z+377xXK5zl#q0gB-m^v~+%XI$)x`7z_X)$Ew91GBzH-0Vh3_G*!`=_z=ke0iD ziRRBCh8K!mR((xOrD^Efy7b}xYpa-@u-hxM=Z^;@=8}4X+~WPk`#mNfS^3Tybc?!t zxg}6~g4iVX!^rPx+(J`!>)-%kJ6f3ZLu}{2QwhIL@Xw#%Z+uhK%zc^v=5b2!Y5u?e z_(MG5_~SnppR3FB=Vq>_E`F_ke{ZyV;oOJunUfM79JdpXld=N;b$c+QbhWp@;m_;x zkj&R^O1M|$-779vy`%NG%B_`7JQ4_ZI+^xfSoq%EaS!1Si=2QzAMIT+JDq0WZE)}I z-u=^mq^Rz9gdRWV_`#%O*P=Y3YN>F?nvD7VkDOkZiX`6Up)$CjPaBpm`mEv@_S-HX{OrN zPz&pMUx~NDf}$xuF-fa}0n10Q!!Iq%$P|=m*oURw_p8RJ-aPXM7g!PdXB}EZqsNJz zmSHqf&()ivx_Rg=_Z1ky-uqnLHIUvHS#_zy4mT1y6>D z#lk3ke}g_`56y(Gkru|WsutlLIvW^fV?q0_kv0$DvaPKTgx7RD8O^mghiw%VWmFy_ z-@)Q2>h>}<^WS2GgeaojTNm2FFYl&60C&dD%W(o+GAt#0JtO#rBB6u6Du5EaqwL*0 zOy2#~eXt|Tt3goIgWLRNeMP9*=1>Hp5M7-ZFx?bjmt^}^1k-=PAf>C$17YAW(b^Uqgs<;4*;KFRgEu(OwMQmEK1Q?cQm~f2^gRmSZU!!eD<@m3pNlj?S>q8 zln>yqG{?tz(Mi|90mbj~$yJu!9&xWIeNICvF%a4-g`0R86nRWw?`N!mJR)&^>XFUm zJzqH#Y4=bDmx}dLgKp~uO=pjVy){P!=Jn>z*)g~mY$R>UK8!YDZDu`My9g%XFu~!b zA`P;%6c4q=XJWXz^9{!&W_mV3V8n5S4E0m8F()eoBpeu_2T@FqMap1Hir*awXXE=MVPpvk?mx4{lFFc77c}PA9>dpR3i%_>S z*hj`IDXh=rSb3^@sMXC~5Y@@Mk;e#6ADs$zzT;-Ps(K0v1gf26jmsv93n zD2p7%nTZs9d%>y!JcL99ySw6?aIevkLkH7^Rcap#3@&6)UZ{BDYQ;iH^95^un~vrq zcAcbt;x38Jsaw%C`vJJj_%9l7WS!tIy~I!cGXML;W&Vo)F(Vc^Wt+Tk`xpbs{`1aU zgy`alspavZ6AI6c{yEQmNnbk|HqcMioj%}DsM~B`-3DFip>Gj1e7GIETB%(UydHfo zFpfTizdu({m<|;C*;K-D$&%aG`MKX`tCJI0m$d}n=#t%^)i2cEHw+)ZrOJXM9(C;pug#O;W_hrPTV_KLF?bizE zl0fGt+`!QXIr=FV0mT}Lg^7DIMR6RS7OgGmI<@0dBHWX02*R^o@MF-%ooAWxX>c(+ zkthB1KaGxAgn?;gi(Hz0dQq(*P+LDlG0`fd;mLg8?nbr3C!;fRN#Gpm4STZlI_ojf z5S>)d4AcMQsT)|DbOg$i*8+er*E0wECnUUch_{$nW&IygzCpVkAX9($L8-&F`iEca z#Ju7khK!%Bo~4ul*E}(B;uX4e0%{iYXehM2vd!w21_-in5Mu?dIy4Ge7R1Bkd^>+} z+?-tha3B4kA!@#VQ|mik zK4VW?OWGtbY=?b@a)SBuR%Z3yPBP?`YlrFY;-$3N?QFB8i=Er>Bl`WM)3lF{QQ6f3 z%1Cp+La#|{%?+8qcK8Vh+dL=M^AsRH>P0~5nWE#_D;V*S`Ds+`o?VjhCF-MkW18EnNP7e^*~FeoO6#f6~U zNSP`HWvFjb9n=N~N-}?fzT*is0}rH$I5eDXMxL}*{N|?eBtdg{0;2*EHP99Iji#P! zrCIUUn%nM9C~oO{xQo(^OhLNpTNqib0W6X?Ns89wUtd{Ov?>*sRHffpxSNFNGziS> zD(!TXi$h-sVuFukV+5=z1xj3C5WFqSdud;Ua)r<`*M8@{nEg3hw??E))auCPg2cvT zH&dEVmlQ48s6-KhyWO|B1B%2pN|ZoPNL0cp@V^)di?S`%89>YBd&}JlUrxNL&L;JX=k}#$+3Q2Y{_V<+}8UO~| zoH&1k6YF+&|GSv-=ol+MT;kuVz%L#rTUGrptR&kdPv1WO)>(65rZr}bgZO^yw#+ck z;g|IoXX#7&=8JRsySX{k_T=52d@5_GXJThxg`+#M!H{S13}`QekkW>fP~wTv6rN?% zMGc|oFLQM@8y|ax$BTdnrOF5}r5%#pMRb9s7JX6ML*h?3@;Kt7e`sJy}!jHET?pwQR zPsN1Dk0}M|qZxyrwr}?bw1cXpQ2j83fQ_-^i_Oa|eNBlx2SvlnDKPv}P^_bG3&4HY zh$?I@7P-`x=yh@xIog(E_jn5f8%Tv}hx}-C9+5l1MZ1&8v@u>By7D{~>NaeWY$m&V zyHpo^t#Ds5V|8Y|w0V-I&`@=*Z!gOcSioROMQpj0l^5zHGD=R0M>>Bn`~K`nY5d+u zw0Ky%WC?ZEkUG+hJH`9>yt!BqVvraE;HNftYJYDkY(ah6699mrWYZu-evB5`IkFL( z0n8IN&^vwOnJyT^Yx!*ar~tByGD=CA#+wSduMFa5sA@u-=JobCGnz#qu}zp^Vsp6TW5wXsg@zw5}A{%-Xy(6TL>??fl! zl_TpPPneTx&+Qz$92^r!Lw{_rz_wA=WuhbPeMT%nBNKdACc3s1CpK2(V`0M=K{(-& zTxZ#KUS~!U{=prRa?DN6dTk&%?dKG1guvy zF73P|$Qu1%{Gt>9Rar5Br%qFTx}~5Ch^>+_=b5M-(xjmiUo}~9)gjSyT&!Max^NZO zs-HrsUV+bBv8=)^I#Ci1M8ja?!JmZJLPF=?jE~Wo;Z4 zZNPPeemiY9)~J?$M~qt%^bTO5H1JtJ@jjCB;_%Au$x+kfUCw_Yay9J)KZ%dO4_)=^ z+-3ge|J_p89z*1DIr;udx8(9c+{g%jkNogoKGIdXryf>F$EQ4ukLvGdIZYpgixsW! z3P;nO`uBCrC`JC9M5xE$Q0Uvugx!Q)DO6O~Nk7x=*}E4q`hxWW!eCwLA)aDAyUFL< zSi|3UpBsePqoPdOnio2)ab!!cFWfMB+KQt_s+0DFsGlL-A;^tBm}ub}F*{d`54G?m zytT+-=NGKU9`zn12)nEQC^IeWDb`QUcNegj4QrpYF6;oQ=(e^P&B`)Q5R}Pl@2+>P zob+SW%syCg)SU^IW=p$|DRhvXiDx{~D~=#D=tOXU*%NlN=ow-P1(L55SSnWrN%BM6 zUb2!kEn$vRQmx45#;ZJqcy*-&!k$owe$bHIklKb%f6th?4rn+XCShLtTe04|8$5Ic zXN};m)_V}?Jy{17;0~5fP~8}JWv|W=qEKR|j|!jVdI*Q;Vu3Y66q=UVD*t@il-^-W zOIu}T;<=Csi_)UBOON9pfQdFrjt#|{BbaZ}>m;09yA_Xq=+Jy31;KTJMaQhouNg^; zwBB7~>Z=4TATQx_CzB*cH4addnZ1|nx-ki?^fbaWTY2z>6_6%Dkk7Ly) zOue?iG6N-r%^G#Y=R3GTLELzB*oDo#m>H3CI_Uc3M3M<&x5#p9J0#8`cNLuj@&kDm zE7k%p29=6swA(Z^+f@&Fyn59Z&JCz_DfhzY#Nl9N>lP`*3V2hLpSI;Q-Y}7rL22}F zvl-P1JE?}XG*5=G#`zX6-ifwU((iz%A5$%@z;shoy@#?n3g2#3juJ1!;B^lTb;aqT z*h2H(0YUwgJhg>`Q%Pv1Z4hmTuRR;6@vUsK5DUfNx}5rzlCJ0yK1WZX9v zm1^n=e7em16lSkDDqe0qWK9{s0&UUCpp(j{i60yJ;#k8wgEsb8LRYV8Cv4ON*ZKK$ zdt{6M-qIPBb<^;`%^;0AtAUIhPk}>Q~ursrVj4A zbLQnx5~jv3(Lb_+V+(AScga3S_I4@(a+Jd~wO7Lhn^g{S>WH0sssJs5IT+Hdq&)x8 zVz)}11xKISa+H%8@s_YMhuPrBc>X7wq2VD>SG>p!KB9PuAf}(S0qt%(JSB;5tc9vK zE`ZxnYUg(9E%@Wu1mIVy(i8LXz{lQi%}zl$?J5>K78~27R^7&j)8RRlmV-d2|7p}MulUeXTfjxM$G+gb3xw|tPIpqv z|0X@f$CqvIHGE%H^koTpY?5ww_S190{UN;7oXEtRA%X)&n8m~HJ0;K3wkIPK6IjtZ zTL}7^HfzA_5U=v+26 zjBfG#KuUs4#~yCg(yM{)%-Wjz7o1Nj6GHoI#Bl$j_P@8edecPIasO?q`;LjlZ+!g3 zSFe|k3&p zowc)bsB||mpSttq_s~w~&f)%11U02UY+cMG!een>ubox0d6wHk{hT7=SHYEhrNqm^ zx|^trqS8Bu58-#ie#z6M7Vrmq=LalG-^_B{X2z((sboOl%Gf%2BaBM)i&loEAYLN4F&PE9{59RyS`~Q&ED=4?2>db>=O@ zqPm-mk1-cR-K`?)w{Ni6OCcnqlHf{$y5+81nD#-8Hm#OoKt(Z^qZ?WRdXrD}f*Rd~t5|sku%U7E(``aJz0LGh%-epV2Em>sw6mROSQIY@HNE+H2 z8nl&kewIb_-*FUPze5Li!kKl$H}C3W0LwdC6Vf$!t*o3qO>H#v@bN-@s+Z9EOG5m* z2_wqGJ{)RNc-S1@jM&OR?4}Rkny8zjk$?qp;6j4wY3E zU9~h`F9L|@|1{%-sy?6e?<&I9IMAdie93yk#7oo!nmH;odtk3HUcD55R-9e+HeOh} zbz@|H6(#zR9p!OzLp>1%2$BVDNuehi`%oiBtzpdqQi2;2pu{x&eR(C~zCpB#g)$fb z6vma&&r9gpJ~m~%9@;={ylh(r_iJxDh^om}uXE3J8mwGy8>kfz!{<&PSOxM&zMe{G z;g%fu0@6hqV=poF^CRkknBRjDht5jPUAd)oDvuUg{8!-`-5Y+}D4eF}Z!4yCVL+7z9sV#bikZ_O@Bkc`4G3uVfj-HTlTXGhi+5Vi9J;UGgK>!S^_F^ z=VhC~NL9HbvfHSoX{^FnGBrO}TXmWWc2q%iZ_7`h?@b%8g3zuH;6lr;{b19F3Ba4M zpDtyD3%KDmCn$H681s=x;!nHwuazUw->-?PC47*;3cvI6xnSE~atag_%=AD~_VAV3#SFT;vhQrB=@w*Zdstv)l67k1H-}ziy zT&B;sA9ow=I^SM_?Loq}L3DS5Gn&Ykw?aId5x6p?`HZQ*(}kYJQfMpAAglM&a+3Py z?h*q7b)vrb={S$89iQ%A>Hs;lyv9|KQqb8oHFlP2+Zxd*$v%BoGnOZ7V(DQu~3%;rm}?kIh(3$=+!4q7G>=+r(^fvtQr2@s>UvY{iSd zqJ(eL8|uLhnB>OgnU*GsWY6p``2esAQ$jVX>SYU(zExAq^sml5xAJyHV9O@QDo*yuGQ+7{_trYAA~Js8Xy~=8$yh$%0f?* zjF2H1W?cgrEH(-#+rxKdp`9j-jODTuvmjwFnY-d6pdu3Ozu;GTTmlaA@o!*j$;Sdu z?f(G3$m1^gm{HU-?;5%;w|uBrJfWVwxb%nMYJ^bit&vN%*Pp!H&e=T->*r7zfsp}+ z$xcO#qb5obBX5_w&kWQK;6%bCn0m<CqXC}npt|6VtpRr8 z;QTk~v3)k?w_;bnQU_>Z0`GOl`u!4MbGEF|f=gxM$gr?2wYbn@|kiqFf0Qz zDj_motVFHZtbDRl9>$FMZp2q%^g?8d*HnP-(TL?5Az#}trdd)*;c1@|2hP} z!Iioo)>*zT;m7fSd1UkXvN0R(=jaC;%91GD!z;kMQlkyK#U-IHDhJYQEdqn&O1Ge8 zIP+@Y-V+UjS5*RM(0`(vO7Tzo*Eizp_h87UxXCEb3LDv4Smwx|xY#sNI6?8D-JMgo zG;Gd}h~xkQ{inw>_e}4EqItcV%>CZmY)sFHh(RXBIyXutZal9~IiwmLtc3FFy`OFs ztC(O@Fx!{46+UG$_4>f6CO(c-UpV0w3BM(K6Xj;P_u!06_4#h2RjU-6se$}~UJyq6 z&BhJFuO2e})Q*JsFY7m6fjj$${G1L#Tj1j0*j+oqk8u48D?7-}Q?O3;v9bB$t5xXv z#)qZ-8wN@b!(}L0gm?Xg5A7CU7uEVe#-Nf|-squzZ^OYE1>kq9KJU17;?&G#$HXuv z1-HMH*W&q9e3riD!0rQG9saEyXye z(vA(5XczO=ipM)Uxx<+HcI(sPURH2oHe&YeBQG{Ir1St;Ky*gKOL6_VR;N*LJ$>QV z**+Zft?y1P!XEN$??@;AC)zrXvLE?fT7J7hbS^(YQ)L>SQb$dCJ~pxuP71&_rTVpg zTUuS=+wTN+S*l~yjf*1gjs?^sq?k3QZ~}P#;}VUS6Dg#2PGw?OfH_=sSj zih~Jy`lesBAHs)}F^wtsRn1KNIF?QbzL}>s*q_0R%ObW#%C*6^pHZ}ip~^;qEVK$! zd;6PPf3V;oxMHGDAYanYuAA*KwO{NzG{Ll$2_$ygL=M;&AjeB!7f@Tt{7qNBMiH5p zi-26~725lUPq@G5lDWLN+RQC;?miB2K3=Js$CHF}Y01Yrsr3hdFq_)0jg=+zqzg+m zZMPbzrEj*mfK$Z<3cifw2n z^>60e;1Kl3f%71X{AJuWiug^-4EVg2v3~c}hH%)RTf26OTHI3#Zm)urYbD3ly$Cbf zuNf?7=3`7+m{-^>SVz9_fitCu{*Q+??wLgyLBF>Sd9FVHH#GIX2w0zUg1`I{f5zYz z`s`)?uKy2DSa$oLdm~2we)t$O#c_l=$OlCw)BAqC`e3Oru}||++T3K_;pi5%@9geG zxgYlSo#>BofL?89$*a!>oxs*He#~(9D+V)bqHP4}+^W(1oL(6KmFM<(hV`=GW;$Hz z1D9Oin8RA+(w25;2KQ3Q3q z$Q&oR$O*ekC|au{J7ukc(4^XXh53j#@PE^)tsYgjRo86Y_w8`Pw3PaLnel+FwJQgGX1Q=|3h?@kq|=7aMIJWfuP%RSsfR-*L^V3g+{%rT z%PYq(PzNrZEQ|?>+}*EAc=grv`3#EKTL5d-#b@lR2FjpdPH1z5j3%-1AbD&61=R`ZSjcXZZ=hPD$#+azx$3V()Bl#qy&+krsLEyyj;WE za7iA$9?CS#xAB57%lGKJvwygDWeL*gm;rxwq$!lnUuo{JQhofBj)rdzDrG!s!hI2b zgmCZ~y%h_Nwh@+g>SP)`$ChMLn;%49{w4i*u2ZIwWKN1gEtf=W0p*83J(GWSmwuN} zHr^QXfpNkPoA;~<1xIK4P-;3=ze)>-snH=&a+BPs_p>{?d*ihrHZIev)z?o1xTamj znD@SPWgGOD%njEv;G$!gvWv8xD<-E0Ywyig`H9O_*>C;rrv<&_d+-dhAwl05t$jhe zLjN=D*&sR-rMUSitUajW8E#?t;eZZvcox#=8|7`-zBCa$(`FZkR}lH=%~?e>*=Tj_ z9KW=s=O~3%9E%N*H9%1A1?}qFy(f~E_IC2kg2YM-PwxoG1f zL!_RZG(qb>Vmf7S8}5vqL6k(%%=4-RCrmW;orptyU&auIn3bm$V)#{yB=dq_J($~Q z$MMultA>~w=~u2NsJ8aUU&$=DbL`AyWcz}n+$L~j6vGu%3&=-w2FD{NyilW(R(x9E zlE_Sr9v1yB?T0P69WPe!M+35kJJe5V=`iT)NS~h^(}cf9Y`RFqe7a~aJ1Z$1tvUcf zZ)s9%xp}A_uSlcpkRk(km$)~!&)3avfEy-S#VyUvY~;o9Rv)&QPxQM0FWNDpQ6jrm z`-P4eCn6CQ{+V13t1+xnTvtBpty>$?ruif5bbIHNkaW9UFisa-1aYg~2?`i(vjc~3 zj;`W&BUL(9e2dpB)r>)y2pMg;c&ke4;Da7#LNGD!oc1FyAj?mGV*hTnZ-}cslkn>h z?IbnC%s@ncrZxdl%fVqd$3w&wsvBYt{7uzEWC_g;R5P-eY(X2`+~>F9eQrC@5xx_ zED)&$a7Fn$ASU< zk4?U>2n4G8al!UW@Zs)Xm9I%Xw*Q^|ZOk~qzskq|`NLMhu?c79f0c_< z{)5xE|BiDemp@MD`+2lPcCNq`H5kHZY#vm=?XF)ZPf|SSJDBc+!^rMUB^}|E-JSg- zs_p_?*r{lJ;5TCGN7yg62YTU{_FHWFF!yFd*od}uOVMD(>0DT2{?{>P#H=uJx=R7- zQELX|Ks}Osw%^bJfl4OVUmo##!vkYB+|a@P@HE!&P#?dyXaM|thZ`~i! z4|RV==_6w>ii_*x7L*sE_1~59wJ|pQCkq|KM-)Ua$H&j1Ld`g}qhqKc7oU3IV=dPp zximF8`_kc)>86DorRIr~8VT{vRtpd1tNM*3WE10z>f_qhJ+rHmGVIR~19ASI6?e*e zGEDvN_AAFS1d~?vzxMS>08jNf$*Bc9{L*+BeGM)a>1kyi>1n{7aG70pnqmf%IDArs zWWBq0!~1~ujEhzEni(lX3wJTTN=_=^Xg~pD2Ajxs_-Q7K-VRVSUu`4iQR?c->b~v@ zZh}K;tWh~Y9;bIXj$l+VmCWJE`MxKl&|P>MU0oz@O(*W&Cg`@#m8Qto>^46zK4@e-70t1rQu zcFSQ(e?nwjy*HiFQ&m-qdu38qO0Z^*cB%y~*GrWVHjCe}o&ZZCd%@xG z9gyWji(S}zfinDoHStxCtv+^jY|L9wskEz~vOz>n<=*9)*}?%(f?5Ax}tMaq@2V5t)J>rfH^HrmD6*4`d&&^4>pbHyfjoI}&Xzt=P?NSK|2oot@C zjcryV{(I~i={H?~~ zo%|=Rv{u6z5YFBT26P51HGm)I!ZEv6KFS!&!M&(3k!YlE$O^Lp-$y-beL+9rErKHKy6 zR3F~U`>zrz8_^-Hof`355${o`{|Glyn<^B@Otk`i;bWtbIHWY#sZK3fXS8iBvre1K zRI|p{=cS;(T#*Di+q;06^$8e=E=S&xtlYymiE_ttvu~-b7fWpiRm@tK8AJ0){^%5P z6WsASekaHsty<8dG&kHgQG|Q%U;PNOHXd&4ElS|(TeW=LI$+lsx=m7H_HMbG`F^i_ zTZ5Htf}%62(>&e&3^AJ0aC9SE3{m#BAxK zgoiNL9`~6~lr2=K!)we|_AJ;+P|XFu>NM-8p9|Be4<_D$IE<;awS_b&&FomBy>*J6 zyb4I!SCWd-aVWv6Z6E7*z6!IHngf0EJ!(w_>v(b#E!nWYs(LWO6nB7jy%qhhf>NJ( zf{v$Cr z?(ScvuJ0xT+MTHDN6FAG1|TYQ|6$ly-4}7@1r-3 zjz@ezsDg-bhd}@o?)GCL{^Vhyl}@4sSlBaTbB+nnEKKC{BU|P{a;{7%blp!2@!F~t z%mBj_xk3;RhEO?L5Sd(zlOa(3#_8;kv_g-$3N?X$Y?VQ_}0D-X}3 zBzpJVnGLOCjX6x)pqj}}&&|8);K#leHItiGhn5&eXEHyGChkRPCANIi+*L0-Kl*7e zCRu#zqV`(<)8EwomJ_{8Vf( z@boSteJxkL{mz&JV;X!NCb)#lz9^Ei*O*x|v~y`qfib7;bXtSx@&UN@t9)1~>0HPc zp^VCw;n{Jdoma+3&u>?*3~(V^Dv_Y`N`_urkJquh;QR~FB!6+&xg~RmZ|Sl1c#QNe z^nq)>KvL2t?K09oe9bKOosd=`)M72ox=9T12X>6}H;Nbu-Dp3vuyF2GZ(ZX)rg@CJ;S`g1z3a zr=dz;X}lNk3c8v$AwouazL}I0sjA*5l7|nBpm#{XoD#Bd6I!ny0k@3)XVdPA*AVHd z8e=tX@2PfcsKmAiJW1t_-gjllC@{e6;)sPCpH(x&qhxlf&}h#_5}&%ZI-q>8-D`aX zJ~eqqv!6SW_R7E>L>%|{3j4INoP>TK{{W3$9hxt!LPR#iP-qIvyX5vmVYbQ|ko}CU z8H#$tysx5&zK*_Qx7#e($7){FN6EI!>J;tHZkn`?x`35yq`8K!mU)CS@Bz#3S&r}G znavGHX_TLMotuBYyUo0xQ@3nAP-LyGAn6C^??DTD2d)5LgUZ%z057@pQ;I5Wr;w;A zuTfnG{c&x<4DHSc^1jOPHa*XyNb>iMM6ak?D5m)`84cH$Gj6{ER0K83I@wNI+UbGY zMlRP2XJuoN0(lOCn0VK&S!CAaV2XfhE;(V0t`l`*33v}3+mLb>z)A$%9qxbKMdh#z zRL91ob`Jb#^R)#fiiJXk12({>oV1n&xeOck1=Iy1@+3*@dCM9zsG-lVyCM>#5h`Zv z1>~p6^87Tm42Pk!Pm5@8_F01-`lPnk-MqOK|~ zyZS%3UuO)=`e^G6H~d-9vocM}W@cu$vlZ&ah(9AY8Xwv9;>uMv+&Zm}r54ou)BQ|X z%|S~Z^ZLwgqg2pPg+$Xnf~3tpeL?$9Qig{i0d7ZIB1C?0%wt^ZGjsTsPQ3CB`NY1+ zn?&e)l<;UT4X}o1N-0A*(e$y}D8Me;picjY5V*2CdbxPEozmoUuJ4r{(`gFEqdqqU zSoZiEF%jPR@P=A$hMAplx%?q=pA5z~x$ozo>ko~tFkDxsThPIlkP|r4*RIfTLuQaD z8f{yi5iLnpLgAl~+Y3peFVN3sHdhM8a@Kt;$P{U|-x{{qhoW9Hxmtt2h}d}sHbkkV z)sAvbD|@av#E!^Pz{D<}GSg_Ump;*dFpAH<{XYq@RebyjW$gD{|2rWziR_SktTG#z zlTGp`<(A`H)=D3XUwBb+fAgx4So}Q8Fuu^$BZ@<&vsi4~ogvsS`hIvci7%Jsgc*W8 zYye|k{_Tn5 z<@igG`zy8}yPvr|+l!H9&opppBHYjjfik^6BK`GRJP;@y-!=@g{|y#YG=4_Mi9Z1he%qAUBN(U^!l>HQ=#BAy3S zragKOt0+yJoy^OZi)2*ztD6Z(v9k&TSiQ&soTv4!P;V(B#}4M--&)npbi6rq$oTB! z^Ad_`v>N>_?56P5%Xeh*=k3BI;RUCuZvsPCJ`rL1K9WX-dk-MXPIrKCb|3HTZ2^KV@@s8%@R!&dj1Gqg^B=yXjWIC?w?bX7!`jND}82b3CpP$0=x$yIZ z=k_=V7+O=n4sS8u7q0vI$$^a`@WWbCAC6e~(T00iz7CJqTJfVR=Oit?ardA*`by?i zkrkISET^^x&q4=kwxtMiD_?>y8MLLAX@MsqWVVsv`*D<-9j*4w8e>?V8KYNb%?gCY zRLaGWytrJx)$WOhOA#zD3;mvag5O6wcGj8j!(!GB9o|5BYzGZCc07|a>L)FR~`aRX0-=2@-)h$>G4V<>P z5l~bY@SXx1edK6drRlk5{UgfBZr~58rK1DwktO=4sq05O;btv;z!wCP-5eORrEiW? zAPa>TZuB_;1i`FXArg>KhJ)7R@8Oq%6`WJ954|#FsS^r<#70?0^N)F-6A?YeT0Ylz zbd@z(U^NRgyd=a^UiDY?8xOsu&!yd9&B{aT5t}z^I@96!5c#xmoe)x#EAv_V)VgB( z(H~DIW$pe?j>j>dn2<5H_5aE7h@A4S%{Mw9>wMBUq~k(8$Ji~*+FGwql2F~j^9KB) zNh?B3{XZ?T?@@H&j^~jMb$4geW1GG&tXs0_vfUZ(NfS6b^b-y*488B`$^y*jKrSr* zOf2e;3KefvGim)Xx0s>bQr=Xz@n*<+t+YnR3~1!!0rilRtDc>kv26XWcEFm`o8@M} znFV=mCDk{)^fzwC2|^T1YCLF`eRgYX=f-BU1$atku>d84M@m*0G3R&#_+6{GI?LFD z)e12ux2uiDWMG5C9~w`=NTO#8tF%Iv^K;lW5dXg3M}(UEOWYZ~o$o@j+SIFS5i&GW z(#)@FCbwMs3!40R{dTpQj45rw(e!1~q(u=tn&2ZO*uh>iIAv%rX*|Twp>6+-2NeL( zw5o2ah}7t^!hqqHga;j;pu6L>ZBr`w0d7zgIUW}e4eN~!Gflvy^6DX-6jjXIWkuZM zROfnB6j`X;g!l;>1RA<1+`4kRv=f%lE0dg{zn9#`N{Tn(J8BocQ?GVWCM@z}0LgN# zW<(s9+0{p`{QKrns34X8hUk;J-47!K{4jJ9U`kQp_xwNcw&xg3n5x$}yeCl1Ys~(n z^o$X!sYVq|dXZCA<(;b3U-r8{nHg5*AnycogB+Kv2w;rd^oxk!stV^@aeE&;B(whh zIW`G`+=<@zAQ_7OW4eg$snA#10iM%uQVNf=MUWq9_cuEhLWW%0bR^Qe_|d!(*L8Tzz`m7rq!>(yKn5!(uP0pQAriDNI zp$K>in5F+QMjb}lUu8VRm*{b*3?ERh3b}IpME#;iQ$brTbzk^rHR8kpMf2{i@c@` zqGGw(WjP&ut^m|&*KeZ1b~uyw@Cnad1!dElard zG8MXDS=@kpBd~F#MD4D12DuO)WZ#mE)*c8& zjQLr(>TUj_%O=s~8dh|xx4X$Wvki7NY1SpHjWk6kB>kj3tIZ7(6^ySt8A;O`)>YCe z8RM{FwkN0dZ4@tqjMI15pv@Fe=LgCzHrYmQyi6fjRkof8@yI0%M zQ^?Y+bpEA|cgkG3l=Y=#gurxuqhu0mzyNvZVFq&nXe7llpFjQmGWdJTa>(K=@+0Sl zsJq^VGe)YBuhf+2Qs7I!ogd|XjDNLyO6a=ZN{J0|ACU|~LYnuf9r;L8pY{orB*_j1 zUyB?y-OJKKcdR*SviP|gii4alaX1veYH4+sS%Nbi?VC@*ak#REdrtqKU4qAa;_IEt z^y7h+|Fujoa%w6`^v?NzoohX{tp8eRFU`2*cIB%0uZIk-FFzh#Vm+DeKR9BmL`P#- z>`hKSr+p!O_TQqP2Q>gxsNWCk7tNd>46cnAEbiE(0@m}Ef8Q+)G6jVw5!zFdl@@YV zJVJ$L0h{bjc|iaVsh1sZ5k@Qc ziLAv)+I@Dn!S6n!0)bMT?o>6idP)ut=>-+5Cuzango>Oc z+)4ED+BPTC4V5Z_TjF1>*7;Y6SZ|=08#E)4B5Hm$i`8r}qR;oWQz)bP1XhQZ#QKcp z9v_hnITm~pMxhPS?!?N5hmwwx&IFs}hI|jJb$l8WJsQ<*8Q`n(oBt668dOf}fl9hb z#(L)Y1NkxAR4tvEwIt6Y#&;L5H@gR0B+|7mIr&O{Z%Ici=boplw~}{Y_nRpBb_aCb zh-{Dglnx^3t#Y;<5sEE$ENfqOV(M(ypi7@%8&_@R+uKxsjs%Kwx4IJ_3R{7fl9uQ* zq(3I+Q5LxRw6U)AGZ8;b&XX(2$`FQs(lL|SWYZ)Z&St>&7Bf9Q80a$cYthT&J-?jf zh$*K(sl2C5$kvgjlOJk8lspG(>8j-S>&B+3QSPH!I`?3#f&!vB3PXHWj*Ii0rs6L0t(sw9Y5u?3LTA zz5oCK!q1m}#rJdB^(583aKJm0E40pO+Tg7ho|P|=-BOjBNxXU5Z-3hrS;r=&Gm#)# zZ3Nmc4pVhdm@a_pR|wmVQS~s$3q%cPAxb__#QebuAML6RB&~@LDd{< zvl0PcD=-%dCniMgxS_3cQxVw9!VeRP=%-s>JhVPi;Dl_u}X%8ZELiVESko?m! zFS`)^`*&3$h_zdNT~Bu|1^Q0^9^S=j=Kk_xneE}CSD?`&2gF+>&`4;NVvTIGmD)%l zzAmJ=a%^xgb?>ZY0q}0u=daV}lta#c7%Tt=ZM=zcpVSQdycxJvTN_?Bdiy&hyx@=X z>Rd(*kD)&k?gf^%dWClmXU&*s02LFvgt3#do@gYM{(1v2LlJ%ssSJKfegT;nT;D3_ z${CcNX_6-X=hI?z(Il-|{r5<^kc z+*{Gonh6m@O+kpX6cxhV(cD%EHAN{kBtb=_hEO$+Q6vbWW)edUiN<*He4q1M&wbW& zf6s5N-*27&PFDWQ+Uxp!_WI>MrE&_u+WgVhCD6YR3I+x3>j&nBh&Ba-!WkNsd zhK7w7w9ZuBpqrSx!~)WKV48`AFSVG~cNzJ>A2KA`;t0YEkCcb)=W2_QrT6zIb`CpE z^IZKe-1>>or%q1G|BklyIB-z-M*C5z;fAX!0%H2g&y4v_sfh?=2pPXT`r>7D;GqLQ zUxYaR4V$hI5Nb_N?&kk@*tBo)l*y@AmYgAD?U!2ntD}FaUOm0rpg-|DN=c|nuBft7 zLc?ozfBy|9#A#*w%I4s9gwZU1Eqpw3Exdk(hir#-;jlVt_8ih;bOn*i^S@$u_%+8t z3q*;;NROzK0O(3t-wHjh)>$Xl7KbsH7{;RqlL)IkHEKw7D8|7j2+8WwrG;IIR7f$~ zxWrx{6&7A9=3D{rtOVz8cEHa^K=yf*?Veoc3wz^54Ra<;a#&-u>on#PxipF88r1?( zJ)7SguXVBZTGWQJ(H5#a;VbE@h!E?MIj&A#xs!j_4e4Afq_+zkr`Z<*Ug(aw#4*}C z#)omjF;Ojk?xrn;o7G-^6@Qi zNraBCl8=7?>G#6A2QlDP{J>X<#o|3rJ)QCMN^z|4C?VAq+YRAohBjvC{nk8LFwG6SHE=?OFxq!s23%}wX(QJjMX9}|ZUDw==k_U zB^GK3xd5>T0 zYv~0sRvkB#+jwGBQ1N~tUfNkVE!^|nA|fP1+s+(c8}BNDx0Rb!WDOUWci3yOHoA2Q zW|_Wlm|yZ~p#7Ili3CxtLjWS$$I_-duJAOG)X~MWCYd8q8NsaD$iwn8BUR$Me8xR^ zEcwmZ+V6@gXOYZm`4_2PQtzAT)U@>BnIpa$8eY8}cohuL2#1}oep{2(*7k5@r@`UL z9Ut@FMR#3!ea4r`2HEc?y3O!RNXw)b)7H~Zr+&sQ=v}uq6K6dVfEnJ=hh#Lioicm{ z)JjXM^`sW=q14kiGQ>T#j?Nj1kSVywPWQJBIrZmNwr}!N=@FK-LJ2s=j8bShIytl$ zS0`W@(fOu4(RT(ugAclBMn7ikrCgu(Rwr@iO&a3C-p%o0t<6E$XV;Q}`(Wy=r}rzd z9wCL}@%=bjsyeon!4nqjE5T{<$s0?1?35QuGbxJYE)p3%nKJ{A%l$vot1}WV$5}};Ix^~|-a}lG z07fOPZ`fzbZd{Xh^m7d*aEJpR*g2@l)cpE4&iS{15Q!Ew#Z4K0{-?6au<@I`+ox_@ zioCmOH@|%!*#oqbXWTb86F#nSC~8iohpY22=ewu3va(6u)Tk$)ZCl^1I~3_TcNnd; zqJn7-{&-3*m<%Z+!_U(W(9G;-cd2SfmF5A;Xj|8j?dxu!Mztcl!z)iH2z>62ck1ro%;*I z?i+5kpL2ma`k}0F6^NBR6G)s2P?=)px##v)vv1Q^18sY1R@_$)V1=v&!gS?bUfr*^ zq#d_NyL8*RmMM;Emzr~U74-9U0nqiD-?36{T;qM?Rc?JHg&UZ#`8NDS!=I+owmvHr zpEbW!x?#e31XLP8NDrhFq_xIMvwSZ**vaPzoO!YK#jMcIpF4uL-mzJJ;n+3n42|DZ-TdHhKBiwO5%4-dlCHzHN5KWxTS(?{L^Z_@ zpMeVZ!FV8IgBDYYmR@zf(%qW`bHtQH6D!OCB>@g2Pk&3?cyPl;>eT>aF;5nZH_GRR zP0dQMu0`upg3+jvh7h|>yM;xxer1G9!g*fYJ>MWOp*H9NxYa>B_^ZP!!TM*`<1-}{ zAX8Lpwiojn@r_S8G)t}FXR=lHQdL4`&m~!P9pC2-MRBYBI{*5;w5j_=qBTykwLfv`-rS39WTV|T~*9eJ;(;n;yETw(5VLC!Ym zgzI)oKa)|aF-jq1~w}QRTz15Ct$qg@#+kbHpX3XK+;g z=98WTl8{wp2Y#s1*m3HSr7KI_%tuMRf^Sw)v@UqeDyXtIqakedDS zZR*JlD)~pmS}ERS1ym^$rKUD)Juc_cBFH=fFor`b3ODugBD>=Mh9t8Eged5(Xl^X_ zPe^jY@clOpRVQD%AkLJY-U~_+}Yn~0c?lCcgv`ybb5FQxqUAn zY*yF~eKLDyzoDj}8lRpWKG@p9QDI7#+w`Hc9)Y(AenqA|R4W$dFz8rMCrmZC%vqCS zTO}|3_L0?TyI{D;9nrfiif)1~(ms;9DJsqydS+H+%a6(jVw!yubtzUlmElxL${o*3 zzhw~TF4qM9tXIn#Rp@Z4rJ%psK?xC-Bu9#%!Jv_Wsw0-UM9=k%lil@~GY2*iVUgT^ z=FW-Nc8&6sx0VkbQ;AzvIFMs8AMxJJVD3m}l8nmv6%=N>K7%*+=|FMp_GQF(bDZUcVTrd!3s+RB1K(bgaxGNiD#f0D z2p>GbQaJlbjVs|H&N_(M2q@$SyU*`a-r6(vcs+yanz0)G0e07x`bSVs3yemJmrl+H z_c4NJD6<8}R^gH=D6^{K9-?0%`o3us=9ozi`k}#ty0FT+F~@OmF${3=Q^1{BTJF}l z%blno+BCLOZ2nR$P96u?0L$BV54oZuNVM}ryW%-JX`?9HnUokc`69sj&(~iPu2hvH zDpo8?Cd(c>)!4;CZUdo})1*rTV(!NL0_pA(MFYS_AiEk-x zoT%0d&)UyuFl=+M2)#eNXyFn>C(I;oEvXTo2xxIA>8-pdo#qB&8s@Epca*20WjeJw zcsP620EA8GMwagwY07Y%9`R-kS)}*F2b{uQ3k!Nvqzz-@o@Gtv*Hm=)RM{+|uOI z@XTnW0(M-A>Gu5iY@|Z>pOf*FsoUNAo@!*9l9uSlPJrJo?cWMW!32ja`+WHN87~X= zSt>cVNlrZ_>GbJj|91muw~U)9G-@^wgEDj68jSu6|cxJLH)KqU6I@J!+xYWbghX{VUZLrn8(dcj39iFkcd(b+y`t#o?;z9wT zU(f4GdH-K1V$Cm8r*hUW8QrzJDc_r*@N07G&JB;W+~2pt=QULHA}_9Qhay|*^z^2# zOzmDUpB-Oc3H6-qa>)-b-$SL{4c1deKME?Z+cu-W?Q)FY+}R(UU>u)Wi~QEEm09~i zPoML8ea*sO8gPjtYsVwa7-W=a7LVp>JhC5o>e;$%h3QQ}Xv);8IDh-x(-#H_6Vj1Edj|<2vo&G6yN*wnmszDeDulje z<+M^iOzR8P4o1fV@~u!sgqUYj3fdvz)F`k%Ut_|8*G7c$si3f_V!fl-tH>p_2Nh{P z8~2XCj?cwczWG-G*ha=A+3Z?rsjYuYd7t(N5sX%S&gwMN1Icn82~(UZ+jIN8k+3M) zm6G7CvUMMZO|i)MQLt4a+;xV*)$1XTc?7A9&p}KBQM^QmFRL3O$7-mHdy2~W;8-gw zI&iY-+Ly=WpOptXixXiY_~axOxMkbzNy4hWjgsbve#YAb&m>NJ?eIg+O}&b(GYwsl z#Y~#fQo)K_X$0}df_H+2f3v;0T|}~%$^|kYia{i&bjRv9r#X~z9+*RCA*yz9G;B%D zM&7I*;gWp+QwObq;?9hFb(F`bNZbH6EbgymZ(+t)iY~J0G~ljp_-ehhycO(x{;|%$ zC49lW3G=Kcm=`mSejJCQ#`Cx|ldQBrML6i?B-O3x$e0N*wU}T|?J95i5-z*M(05G7 z-h|qj%v#Pyl&mZ>^y91wf~6N#&Lgn{@rorf*(f^I{fmgSf1E>tRQJ8G>aBs#sc(#8 zOPL#1D)9X`N6pxxb^a1xf{rad2^!}mvn4BDcHjY&EVN{b!-~xdRDEsQB`5}1(=t=C znEtFhG`i|tU#~TS(UUCrDJCdwX!{|goSs=l2J-ty6AcvtI~iO6ECBNW+dl(cx2QQ2 ze0Hho0?*QvYIX(n2HUv5*0M%`5_N$&*@t~lS%`BQzB3fyu7FC6nNNqNGMyVsJo*vAOK)^O3}z(Gh@Bj>9jA$4AG# z?Vz=lSr&a078K>I-C6JSi7tH;H_#<3?Goiw(e986A`X!mcOYC*=82Fk68N*nCHD_!Bd5omM^h}%6SmKZHP5xz47s78@q3-^ ziYH+4n}5T7Qv`%wbmsj!{Li>Aa@e@!UqP6I>Ebny{m%B{=i%o5k8b892!b1SafkevVMPJ#BtoKSh~`ZWTOx!1m8^|Y~Zb3mGz-*- z^2^VU<-1=6MkTDa?`=hL7ECm_b9U=2;eK*<)slTY3l(;Wd3~nW;oLB8-cA=YN3%3P z1J)h2m3QitfY&}{iV@%%KQ1+mpyg-n?2u6LPmQ(Kd^2939H-CD<%3SZ**Ce&Hr-Qh zpg(?N!sbVS@Ck5=%ug{sE| zMY5=+EJ-|0C>7NH#=Vi`&~Q)hx2&X$&z1XQ{!bWlNX(K^>uo%sTMp6!H7>;`6&o;2 z`F{we0+ctK00oJL~qgi!|9i+l-68UTzz4RRBW%{Y^bc|)^i1u8qKgHG0$c~GrqueeXy*Q_La4Vo12a~r_Xiej9rnsrN!VTBlaS{Jhqn$f+(zw zbbheSy2{i8{zz!uDx-X~msWs@-&HB+LK~jn3)XYHYc?wA%U!`4k~DRg$`@YvtoRr2 z0BUT-X1dFwB=N*(l`l(e)T$&?`a=VyHt9@YQj*d7D@wApV|GHiP?7axyz!SOYhj&= zvqk&bbBbFPyFGv7a=B^yl-y_Q+`{NTa=HIe82$4yk*p=q9=0e)M0>zicXy~q%nIc>c_CxW(VzMZ#UGrQ#zNlwG+DQj)GU^N!%13D8EwRTlD zE?&5>mEF5lF*oYotm`DxE)k(gH+b- z4eYI(x0_R>ZkX15mfkS!Y>rDj*+H4B`|hzKz562K8c3m73>n82(y?~I(ZF%cCB*AG z8!}cM6MoF3gAhSsPqy?GnU_Qso+8xoW}&<|jE=s3l2c$oX?HO{Wv;{E6GoJ%mui{W7xp{QOdgSV?-x3m5v3L?upGXFATIt4Lejz6gdo{ z@{6f9F$jq5Mk zwn|kj+(&j>4YlQZ{TjniX50eGj~CCWGbr%rM@^^lE8nL&sNz-#hC5Mv4JC0-liwzG zcwbc>?lz%(@O7kWHg)B`c_UV@cJ8Kz3mH^uuVM2Vp}j7%ZUVzyjM+tO8@aBC8ddC? zJRAyBQHbr;@4g#9#;E+_o&@78VZ9B^uj>w;P^;)&4XE~cP9^3@{a6ffgmIBWIQ3$e z{}B9ReIqC0;5_|tpZfS`uXsv+pQS8t(!pHMU^Ze?2B(-G+VbeFBLPo?81AQ@W?ceK|#CzSe~VWfwTAdj!DB3`$b z;H*d!Iwb#@`sT@&qxivz3M+uwjalHKy^ML{#8Tb&*|6?Kg95O>l{me4d8=?q0i!eH z*$KkFRPS-v{I!A-wmH3+e!9lQcz4bS&-QKpv_%z>I~_oScP{Tly90~gqtX^VfXU8dg6{A(lSqQ+rHI+u4C%oP zPy=1rA76&BVU=wH(MFo-zz?plw68&yKWH5OLUjk!-%#3L0z%yW?pmN~CZhq@u z@}yFg?(iITeX6tFC(9Rg-wsaw`A4s^;p%qVMV0lrIn{kMH+P?Z5j%czw`(LoOQffs zX-=nekofg>5YNsfcB*qn@N(KXV8d>D^)qUTcMPvE@5C)i^^Su!u_`-Zaq8h*1?H#J zauVz1eq-tT&4ne|5y6f`9me+}54o zkPm%Ow7(qbX`YRy%zHl=TP$*t-7hN$kN-;Z?x(yoze1$$ctjF@SW9itzia~vs!FPh z#R1?ex#4DXzGRw%adCTxg^oFu>Nquc3uh*^Ep`Rj6-x>uZ8V!P{X9!%0gclmBFq4P zNWT%}JADiREdew?EgHi_1@t~g1+IVFiVOl5ZTxD9Q_W4xZ&jBcgGhB|(6CnHf_FiyxEBe)vs@JE7R*mK+_MK80ZoNKm<~a{hH6ZI z@s}tPTIRJnIl4*qt$kaE_g%Fdb@&J@=EPzrBVjgWX@oWpfxQmLQx$-hYR+FH@)w&m zwNIuj4{Me?x$2rhjQ*Gza~i&s zh-+rqlD>?`8j0d5AC$^3p9yQ+a>n^wBM5_Qh*iwZv>01*OyO?ZH0;t`$?OGbd6c!z zrrdh5N@KejAYhz5`fSo}c3l@3I}3&l5Wc!KQp9FM5b*Fc0-_Z0S?9ajQ9{Udb?IYN zf>A8V+o9 zxH&Tx%&euyHLX=8y9&*2$Og&?UKohycAR*(5T5UR* zpuAf!Y#doL?6SmBd}}!#*qt&tNAqp0NV48>A^+ahJ!^H086giwLm*+`uy^ev6HOh~ z{0nxo7zeKm{Wu4rM*UcID0YWQK{e3-JT+Xfy>w*<_;UE^0y4`dnwfd;*8SmX^y#B8 z-7;5qK>1VKjq9&PR#?JGaP?8<%kC4*>A>ym@5MjBr`E*I=DOhg)R%CCk1Co~lo6cE zNS;lp$}-~_J8#x?Tr*CM->+>rEk?CaqhYTHw54X5_6WF6d4Ml@0(fZt#LBV=t9#f` z=akh2aWmdbC@?BGob71lbL_&#I>JeZy>oKz>|M&=fY;9gLJc#dA#QT(pMh7WlFS?2 zZeEqHQ*t@mE_Nc;gt=LqM4ICt$wyjOSG)pyxcYdlJK~Dy>_{>|Wu-Q9edoemNXeoe zeoMul9eHPXwcTz!JPg8Wi_u}e%wF4R?`p3q4G(**KmPU?FufETJUc1uglvT6Bj-R> zL32VKGx%}mQ1)m*?&(wOs6a2MrPERjX**-?v~&nNB(oEgR4V%gZKd&GRxbzpK(jbC z315+IvORpVZn0_|cc;_l3d$(RE@J`Nc?X&2dH(mln9U4+_EBWB>YoD;08umHf_0RX7i0n={67F_}lf_a+=qQ z@2o6A9xj`JSPe__!Ip$1kDwzI)*Z}dfO?E7FSOtZM(#yXCq*=EXPT6myau6=gMGUO z12Lt2bxasWtkLOmB?^2+5x6RB(IR; zHs4tFpJX(%sAvTv+SUOFzfc9)vKOuiehO!JQiwfI-k$fl^B!u_DQ5@P@wYSWx?)yl zR3vNC!e6smeW!k-!9FFHpGXljK@y9KRHY4AORYd*AAB5_ z3X?Z>`P@@&U)w)nstn*4C+wYq-wNeN6AS#t%m8(aCzut@MWO2lr>c~O_l8nhm!5)o zHJh@5Cyd%oH!>B;I&^+p=8pYxC;=daGAY*Dc4W zSzdkN$FkDw$Z**SjNekVpLmr?U06jUt+MX(QU84{C{gke=J3kAODBD`iXZZZT4KOQ z@tN`C@AaD*f5*SQv=jZ%P-?H!T8i>~x1}d;oR1ws;jH;c5JNOG-S`bU6zgc!w!6VK zSvzq$T`@YHTRk0v_De2?FmM%&MNq}+7(FH+{j_*VS)6>L8Xrd7m(&u@6o);$Gn?Bh z|K9#&99FG;D2p!neoSE7hI*EiDlVJodhMZS&Pql@GrOhn^ff*&TdkG{m=|y4`9@aD zL}?5WfXSz7+h4D5bUP+dT**|G(2{}5y&(v1LJM$mY`o2VY-1c0dEjtIUeDif)Sm)E zi_f2J|2rIY*r`18HWx>|=rthk(Fh8Wos)vk$h4g7{4VVMQ7=NP^WcUggv zQbm~Xjs%3WCwU7U*O57Y-8< zc>-e3klVM1wfAD1jA#ubTw zHsoyF)@gFST_K3!`1)0weS~%71s{UzS^Fc*giaKqF})!u)^q1>F`vm78V+%$z=)6D?P%SpVJKd> zP%5E)88dHr^IhPQqyLs2LcZ2kangQp!#~8jiWR4$@7^d18FIq8Mx6B_`rr3kv75P4 zZVS-*+(Aek5N&xhIPiYbm^3O5;t>s^6(k$2gxC;i90iPUup&E;6x-DS4QGfDL@Sha zac5#VEHi=*Xv*Vb-;xf|Y)%Vdw~juk)LQ~drpu-n*#%~^71!(!kAxx2wqcvh(c5Hy zWJk)CO!)!dH4#HhBuPecxLyxiVXZ>%Y9ocE7;^te#dWK_NL7ZY2D%sbq67=0%|0l# z%H$s!91`P$*KaOP7FfEH-InrZ&8C|sTF7cVz6O+njG3BNG{~|-OF$>HD5ddRftN)L z`!5|-0zf6pUde(>6fMhbzEt?Ek(Kev*~Kr*X}7N1YJ-c|k$cKpRBR)Ix~Fkot@E;D zq#>eke^a3JJIOqu2BT!1o>bP9;6=2+X40T^>9$tn0cg&w_TuS2Pe~V ze&i0;x%&)$P&Cjh7=5vqAN_@r7lD11Z$3>sI#OU?l!D(f{|b7wRZz4QokG1&X($<{ z&2AgCowBVGqiI`IJJ+O{7UMwQb^r`byt0Uv`|3)(5RE<9!g$iztaFro9Go?Eu|4Kb zroz6jz4(Ru-L+D`XDC$o@2AHE<>4=3p9Ss$Go8!*PjR{)c!>zhVv@-IohB~CyODvwREUD{94 zSyaQ>QQBRk_S=r~jv%+srkM}nNf&(GrN0I5<^oyjv&7Ih(!Ohp6?d!9(a+poP5D)3 zN1t@5K|irDhLlA;cqn5poui zA~jZoPei-0u(re^tda{;gSg^cd?%Y2lVIDw%d>8s74SD|^re8%=Z(#%(j(tRx6XxjkB{$$)~(kD zvfFy3SMoV?yQs}}lZA^pikn_$V~K3%-_Dt&wFv$7j<7-Uz27|@X+knKWSuZYfQW!Z z+oPy}P>O>2H1_JukXMl@bsVHIyzWL?(E%5gEG~-YKbrQ-FGevh^MnnU@8$p{*yd)o z`>29FM^knkYW?7=fKo4s(3F}zkVYrz?BiFoGCYwTOnC&eQA8?ko3l6GK;6IRuZ`#6 zL7X&J$vGZq;+{gGZGh#Mo7>L3C{rif^@LX~e!G>9h;enT=JpQ7_XC={wh}Drwi*Kt zlpT3dI@Ui_nn{TM0QAWaVI=K3I8!38G@r6npoAlBy8(e$2IZszsMKy+i=~c%&C1yQ zdbU8AulPFl;(gdtxQ(c%VePXxx^nH{WJaV~dF(xgNhm`e&K_vwNo%(Eeg7xq@khk+ z-lV-!0f0mZktq#OoS6^~kn?Iy*+O=it&mlJf*$ z(;m4#bknCcV^s%i%-~PTf^ZzD(S>Yogm*H|9nBE$eh=wuN5=`VEZzh^LJN#{y#UsW z@i8q0J9Jcg6~%)*yFm$QyUa$bLQ*kc6E0MA0Z-T>EfQtvZngKmvZ{as)OSfWw~TMp zgG<>)tecUDdyy!%8+miD`keFP=sVZ%m>-tispaRvWDz)VuQS$x7spfaCef-LhapOI zD74G{9Hv*hNDoxLiC_j2C9GlaplG4wx`7%oR`3~?j+Sl#EWjp=`jBlxrM$3lD)Qp9 z(tw?jCd#>|N{4LoF;jSFaHU0dL*nOxmJrx)I1TB6uHQx>NaN}3Huy|!(5vo2b|=Bi zGtdpvJ?nROI;~BmvnPTQn0XErF;((};Y1%R`VM(4eUFwaJlfl6d_S;4OK7=Ce zKavH*i&u>(#ate-E**LIr*j7gc55Z5+UkgKP)+~7se!;;lwik;OtZ^siZ~myV+I*( zxDu&z165V@(hs!ny*p{nHDN2!kKXiNlyDRBRq*Pb`J#3-Ndw>@}8o?ty7r4F`nH1)GuY<=Kl@jiTZyUxg5$p zVVAeZ4N^p|>a-d1c|XxU6E6_gJh_>u=`}m|wsmJOD-vDG+1_@jFJI&9&T@KNEB?E- zan5)5=aS%_9lABLBlvZ}Xh%*+edFA5Bz*I)#zdk>h{ktL+X(t8xxOCC67mlLBDUg) z^xd?5f|KgH+u*aS{J_`!gx^WGh#T@!c1pZc= zDQ}lo90e~JP_M>h?z-$I;}L~?%)<$-rXz$P0^;^~P3=>vJ3)VG=s{mJ#wO(VzzO;Q zX%-{aVPb_w=U)QeXT@&4RkAlmgOobDWOY38kg3O=BiTq+b4dW^5<3{~Ezujvb`G-Q z;}#}QCufj1mS;(`6>+2lDW?12{YcOJvzGF(o4DgTGpCVP3NxAVR8{0}NAIq#?j@cY zQIPuDz?ReIGN}md_lBU%klza4q-*1J6a0M}j*>vtkCi4M$fCPW+J;ip@hzNCgNda- zs&Wq4*y6OJRb>hI$a0fx4|c&AliI%CXd!;p4fRKV7AF>~gM4-z6d@obUrO+#!l%-k ztY`Fj4-%n#b!eyT@|ZVWCoa_``{n=_#LtlkO5ChzGfEDpNU3`dmK+1123`6^mI zfzOR~dND=f-Ix&0roxOnx+0eq$+EaP6RS~66Z4p_Vp|YW>pUWEyuli{y_igcI)|m* z5zWXg@9-<}52hi`D0%p9yyjqBG4X21J3fYLb+YwNOS4hApBZ{H( zoI>*_mnDLM7D&ze*lhmy8KEXB8iPI_Oa50G*&_SIuO%;YK>&!Vby|0IaK~;owN?&4M_JYth>#rhRsR)sM@s;JOfsq8HV4<;&`phU9Ed0UkN}%-MTdLiwea89Ln7$N2#CaR zn8zUhD-{)F{W|M{rqdO>yO6Eg5B2(-qr1->I5OQTz7il%g!6^e{Up@gS@I&%%Z{wG z#%*j(d<=K;SFb|RRKhT;R&0eaTNrS*xiOIwviuwLJF>wxw4@6_T4pI^mF@s2=JV!H znIzd~eMLdF=k@MXU6<{CmJQg5d)ZM|BRgkhamcMmfcbR@NORb{;;6G;;>oJZU##lD zIRIWU?kzp3V`4U~A{3bfY@rex4XA7m?D-Csc@lwt^+HvLxL5q4 z>Vu+wY<3+06Y!M-<&-)@-)5CY0y{H^c56l{CZ!?aWg#DLa2QAc37X?*yG?SEd0=+) zgiAPOg@tu1{jdU>>=-Jx>7?AC_Cu#uJUyqJuDS0B2{i}9TrSbQwK$qNK)BklC)+a* zFqdy(Zl!HgITjWf7x$R6Q)ZM6Y!a5$8Idig_UX2+Y>fg%qIRl|lQ%~qV_Z4ngr#NiQ^FvUsLRu|=Z zZ5X_{Z1^d3pi-0RNu!IXn$0taEtA*CumV0&kH}Sh8jb_4sBQ5)pq$r_V9xyVQwex^FC@pe0_ugAqo<`_c^@6ih zFmXeTx=4EUp~7mU+#*aaVUIf7_*CDOndztTLZde8U4;BTd%C96@k!_I#t`IO26s&_E0H^}Wy49u$TwCvk3bW2gD-DngC%C_4+rw@i=LZ< zh|J_T+;AA1%LfnIB-8Gzo8zZ6A9+%}R+@Xzg!dGr(!&IIXFCHK@cx2d=zVyx`N-Z6 z`>Kq=;oT2i8K3-vHPec2c9o!9LzxSx#%V{4Q;p^xxT^Fy(fjFJ9Ss@&fWFd}XvAMG zkJ*?0pim?Ji=7M>T44?pANk)*s^zkiz^m_1$8`VI1mQ_~@bLXv>&~`g$@yq^z$v~q3Q zwK<%>*{l|CC3Ahyb@-ocZZ&psZ8nFy;s3Jv&tuj%gwH(j&&Q1C0QUK1@&9|w{_ipS z|98wjst7d+2z?*O zlt(Y651oxM@W?%b@-~n@rW9l7H+d%YX_NFpUSk8d_wSNDu1X)3F*bzuy-RtLCw=%r zy8)#99m>x`nn$DE(0}b+YWQ@)fuk}8uCW%$Zs!V)T$3^M>99x%NG>?UotfoXY=MH> z6dY46GK4WKQV|0M2T%JLxTp6fLv9uvmG?39rS+x+R}>r;Uoh~h?nU`ODd5prFobXQ zrbcXd9XNcU>EY`e$*!loj$FCW?A>xB<#D{%A;GjJkIy$yJ`cQ(siZag&E7~2?eaQ! z64d0DQkLu~=XDeSYKBtFQi8BvhsB1QAeCh(n4cGq)^M{wyDSyCICp?YqsjHnqh$As zb4R2#ntgg6r37Zq9TKT(@+^6T@^zd$re4(yTYi)pHad5ZKeWjm{VUl^d+w-WXtOW< zS4v3j+~G@{Ca=0*D0uiBj~=HPzV|CN^2pVb$A4eiKO6n+$mQz|$IXuG3&i%myev0y z-28~7Kunv#W!WFmW~bZ);)a?oE38JF9};*U-E{S`O#3~v6ISnI2l6hLR?*+j#RZpLe372B7bBR6y1{HUx|OovU5ETz!w zG}J1NHjtyhDl|VV+85n&Ge@S|+w7!6Uo5pEM}EoMoJX@SrsqixU~JxuAJG>_-^fwi zohKhSrX1aH%Jq8V`JUrO%CSGOI3NVq(@ zS+@WlTK2VdIfDT^tq=sk7dfY1S< zcZY@0hlP4(iwe0gIQHs)f2f{rW=bCV^W1@-xP%g-FHvcMhiLj*;d5^EM8E0 z(m@5M3C7@RvylLv5GM~YP*<4QuR8OOGI_Ut#qu#}5{Z`bM{cpNoBcIuol%WXW@*@e zz2_8gAuQ4mR`*70*(jI;^n8>`?|%{g620to_+s_hH=K4|qDvPhyfI|QB6~x~w2QdS zIUI#AC~W)Bco56oL41y3GCe35z#Sw-*~#Q=?^ z+95KKB8X4$0^)*6nga+r?62WaNEOkvGALt30feJxU0k#l|nW_Ro zILL@4dfF%fXbS8WmY2dIU}{Scag!{B)3Gs8onVq0Qgjz6_(00Rv^))~Zs=P)d=s}| zqH+-e?JEC(yV>epLeqZ1Id<}iu&8{WJ+`9Bzi3#wW=0fn5h#jN2IIcIwmtolf!s^?y3xh|tDip{d&Vx7^U~`9C<| zXv3gM)9v$r{a6Ws`8stTj-D6zE}(z@wd;9<@9$44hF|9v_?fh@71CzOT;5+LK;ye( z^Ae7VmX|(fKl*F&5BeGZ^D56TT$5;g^GUiaXEC$;ta&LVKzWrzg1V zI`VDUx3H_526HO2ujWzH3BSd6pK-2kzS=U{?Olu8zH>P-TH$?V@&`9w69bfqg;b@Z zjYB%%s*8*F5*11j$N%LCI~)LmGbpE*~p0D!lA?K`S2h^=zi3 z#I=3{_v0@vN#j58e*e&MHSx}a9BqHmv@_igt(xRYo@Y0Dw1pZ7k#1K;JLlS5aWbM7 zVc%ijmst9rDG`L9fmw~{!`y@4)p&$`KyDmlyqiBB_8{Q7UUPO%kEKZA{SLdAgICo* zfEgjl%ZdCS?rsQ9sIXuC5?VEy5a&GGySmBR?mxG)eYp7C#|pXsbPDx|(EB4oy}wwo z+?g$J{_&ttpojlFaQlbj)_}|LYcD;)-*3n@-gtRU&T}|QH@WX^QB6(W`|29*hv+g8 z*r%Ukp`7%j=0NGYkH_oJTRxX-mT6SHYxB<9IPZk5X~i+?INQ5ei64KBMf-gBm8v;9 z7VCY7a7F8vmZj6Y{}X}!7YoK@-*D|l(Pi+C@_t2KwkIs+jW#fZQHgcwr$(yj&0kvt-1Nmxxd`1Q+NFZyL!Frt?u6aJpC&2bqDV; zvByvKUx+S$qZa%|4N2b)__<8x|Hrw39bAzep#KFryQyWm<`t|cNl*G4Il!a}N(D{s z@8HgDF%9<5D)k!=JlXMwF2ldh73_REx|B5EC#m`=@f!Z>!CHY@Vs$``kjf!O0t8z|&_GU>!*F@JwaxZG-BI@QPt6 zM$8FG52R6!ew!I}NJIq9q5S(*Nu-*|`d_Gaf1{@TMs1IY3Hwop-T&h>!48Va4)Fh+ zrrXAY-(jGFC=(5$g>(8*m>~#)5I};bfR`Vjc=B<>i0mP0f$i=J;>8a;8}jHipOC+g zzo}-pcnw%c`Iix(9jxL*oN>nuFBBMzK?q|kg-PjU*d@j@C}a&ZAr{=L-EBr)17J!5 z=cyZzdm^RT4B&m~8e-M?Tz~P1(+9Chk3npcPh)(uI|6e_I6|>RXZ2+dP47)9)$@BJ zwg>e@sSDH^E+bro!EVZ2qFuB0!S&I4!nen=^Wz%Eh~pW_&X)c*GoG3I`RKB!`mH8m z&r=q0&%6df_7P7YUoam6JVj3GJPxt%+#bQhPhXfDjl9vSymj+ma@Zl~`nytTDsxje-AT&E*#n06Ibf zvIC`xSOyV5K}0|iKtcGeARi|+cY_>rfbuztSR3KYa+bKNl(w*n)z1`UCb5LHY%sl- zJF=W;+iQ3D3K|gkDx=I=s0E=l))L<%Y{IoR@f-J$Q^L*R5^7JyafN;b(pKNLz&5@s z8X&YPGb?0|u9dfsH>SJ)G6!$}K@Sg!w})OIJI39|OY#t<%=gY=`io2a8PqjwQ7S;K zZGc!$i6P(x%nyYn1dHq&jmJNGuuBL)#uWVhmruqQ?1sn^?E&(J$u(w`+Y#sk)DE#R z=mWJqs#EY1+Y|LIdlmi>>zebO>>T|X%oF(uyIo|B)iJa|XA!TSFX^Ai712KeT*O$2 z@HoLTkb=iNat`7{1A>BZ(69vc{ z9v~9f;gStsR7M%3_pEozH-i(p6U_PcOdCLR}5M;ItSyD$| zQ1qNc@))lunM&yhO15&g&cj*J`zKzZ5>}*FgscT^ZsS?f@;Q%asY}bwnzN=I-)Q8B z3vVuiDHz!AUj@$;+pzzK#&6U*7St{j)U}JO6L@aaD%k(8ghgFcO3Kps6+9R7qhHA~ zR$FYykT=r9|6egXH9era)86`h?pvmrOya@c+iDC1^dhdWMg&i>x_1q0aS4%B`rGer z%m)#+--xTbd%pAYkO>~rJ~zAC6$4gEW}EY5Vdp(^@EP8RU-6?{bPm}w{DGUGv~xjI z+3p-wa*d0n5R8;9xNMZ@C7zkI9#Yo5rgNqk^vyhUL0I@G>%oh0&4FSX_GIy0G0;+l zcU2A21e^$$B}^2k+w%(A&0SR3jl#n$bP?S%s>v0^Q9kB;kr1*Tq7QV&hpbU2Gogr( z5pqnT=!D7E(%Mnr58h`<#u@i_WGI zNS6@(o6CWFG$Kbf7sCJ1(H4(pga>K;=AsNan9?TnAR5Cx(q?!>>4Se%=_8M-%n+G0 z#j^nUlAZk6w}=ea&)tgs8?o=0N-TCTky?y?Fvw+CD~eeV@g}{q^v67jD*dNKpIM zi(GRb(&~N81OGbB1+4g5J>UB-h5rzoH?7B#=DUc16~zB(|1BfV#ho7}&8*u7U#b8h zWlo5cxGQ{=Y_@L+ypNboFHA@x5he%Y>&cojOn?bF#_!0O5HVOp0Guu%={lbn;9Nus zo<1sS?HHe0b$Cn{5f>|LoKNhR7}}3g@XLZiKww-kYG1j4P=!L0#!@NN@vxMUg+h*E zTs}8TIiH}oT$ZRpA$FKK50d3n5ISSdpLMaAqS`_X)N?L#+PM(b=PmU4u?XMCd|H2< zG0C9G_}3SvpxgHm9~fh>taB0g0p@^l2vc}`lp#W4`iM6gQ)aldK4RnPIK(hxVmY-D zmJ`N^qH2BCnYAI1)oQ=!53uc3rr=U(eZFW7i3d0aBvkCZoXZXV1DD6d7aXH_G4;_* zmnQgR>_Y_8j=?%v$KGN9xV@HX4;3OsqRBY({~{oU57mQ?&Ctd zcPW39d{ZB(@-J831ACu+zD9L)Z`0j^ZrAO6rr82G{w>B~W$C#ZNAtTEASPi15s=1< zkGjo(&G8Qs;^c=1&RhxyYa!qjp%|2rRK)OPA%U#95O4Ye_SLC0Y{y)Rs&fIE&1)#H^Df^zV^R%_ z>F=#GIXf`=Fxn_X2xN=_eouyAyr?5RQO5M}NfWRE3>m%>-V~Ls4*MSgF;0xSO@!JPH@ErYN&heT4B8hd zlmHI=>sHPqi#*7%!{-DW+J90>6t7!I4_?;-Hl zP3dZjlIBYWwT<)jW!p}k>ubIC9eA)`y{b|Y8o$UXg8L9h`I7R-sbvu|pUyJY zTerdfsjKbCYi)(q+7L1$$+5J?X>Lq+J=l7oz5x^^DV{Gs7Ww8l|8O4QG$syHf^@8& zaBP-z*s8+APn~?XAA|VI50U~+9r2(K0uWN>)R)}81B$A0?azfM*zM5>h@_67_&pqM zk@QF_+Pyu+IlWDOx4p+p&IbHp$V^gh7OgA{l;lYntYWIGGmpLlAU%`2+-k1`Z};rZ zs68Rn5=i9TR^!#*F&kUrF$m%NpU})E!&%{0!`|_l#jU%YH^~&9F(|JZz@3;#Kb`*C zHovF8Tw4zSv^+4`#!jXNNhVQeZrP8vzz_Y(0ybCkbT; z<`y?2bCBaJWNFO+P8+1iMe(nO2};G=^Q!Fkc-Bpbr&_aB*k%_H63cm#i1WwjKK(L| zGXLNi%hyM|`@>E|O(z$=d8!MXs;;rd6;JWfJ6@A?poPI0RaL5RHtu32W*FM%z8zi+ zs}UCmdzq`vyOvno(+ElYplg`1gY0E~lK==qTs5LYS5*I6Hj+TakaL*58&hQ?%U@d2 zlt6&3*;Br(REbgk&--MwjkT4BU_3 zghlOPC`Ltdf`fbDpV~qgJDQB%l`7{8ExV!uy=p844oyns3oJK_9wX#No^X*TY5d0kOuV^m<8-qko|H-H9r}zgmFaWEVvVTEX)g9jg`?hjs-tOym94A# z;8EI!5(ZZlpb>?;VgGiDJm2p`Us{>0Qrf`+4kryZn0vVHoeZr<85qXULe1sscP*dU z`PuuWrYPxCAMp!v%d4(d?9c`NVwR3fmqoc*E zF6B#?O6o_Tv>E`yqwa~s_wdod`SKwY(cC{tB6B-x%4!>aBb&s@E237*RpAVW*ta4Sp!eu zcUl)InRP%{tNm7Rcu(sXVUyJ*9>$;vC+A z=5lw~LX*yWdG9g1D}%TpmzAd=iVSkYyr?0dni*^%u@XU9vy*3Oc}4O84N{MT4pPzE zjKF|tItqUj$e9=03bYk8*{4J^{=P;Rv~w21nFc4$aR9om|1;EWhA$If4CE1|lMu&@Z4$a@Qcm z)R36b*}Z50*~^%lb%2(Q`YvNe1sF1R40LLZNpAX>f#_91<)pPdD>uq+X#}L{>O_*N z^!mLur61z4fhY1i-r4?!b`M6+;vfKZNq5LwlQ0dYw%-W_9EMC-2Kqk7H+(C-7)5$t zaAftt4C0BayK|6rXPZs$H75)ft@}wDT#k36I16&U6F6C?t;RS%GQQ{J z(Agiux9+$0Ukt}x6Z>g>Z(;5y8@uaFCg*d6e6o+byQmF!QzPmgnpBfrK}6QIF$n_M zP(eAzrSL}Sn^)+EmxaNX3u`8Zx)e+70-{)`jo8_>#NroxbLvu-GLB6q8IoOKTWNo` z!at`wyU-z6a7xQ3Z~%$Wtzu{AtJ8#ZeD!~j&%5C#+^*voy=oh6CK@@H*@L29HsQZ+ zd8k3TIYuoO?l_d;c`MVy_Ta;D+RE!peKBcdA+)EHXLkO!BD}n-Oh9`=qdve8w7GcG z)mgZ+cD4UIC^0_QYlYR#ktXVRIH*Lne!uO z>38uZ*74Zyn$rD7Dy`P}_^}#d((sW=40(2Mv#m8jsys>-Wn_usHnyY&e>c`|=T+$ihARpFKWyyyw+ zS_$AdyqkY0^~w^tnr9xryYJLP_p@)u6ZL(a@fl0?((4+|*4K^O`m9IZ%S3L<{oIjA zr4@b7Om~PfcD*Jyilo-+A-S>^7xp!cPx$WGd$CKn$erO$FE&jIJ1Be8fJu20s?VJa z)ly)|0PZ{);UuO!7{@AQFHmQ{XBLv9QgWA04*;b_ z%cdxYTQeLgTMVTbvzD<^G0I>yoDT~g|5Y#V`d8Q z$F1>$m8(S=^Nr3*h(^G++u;|a&xurN15*ajuDuRZD8U$44{rTYrCe*&aa2D%PafOk z&_3RTF$9fFYcwn-y#o%u4E1L%H>W8KlvZIQW34(wQZYhjaE`+8knA4~uf?ud_a-c! z(rnN-l@%yZg;J?fhvxN{Itk4}HjrSgS2U&BAT>5{jCV?zN=cS4Hzv0Z2R1CV^; zJTjjLmhbA%Vrr%pY8Pd|sIz2})g%NqT|FZdGhJq4Q^qjfSBcsp7QND~FC#6Drqiw7 zWEU_SPV6+@9R4yVla_84B*6f6GMUIE0-g+d%U|-xB9&Lhqvv1p2P#)k4gtkZEZ?j) z?-WhPGCEo91c#PnHyi=pW-yT8r7tLv)FL(5GW6(htyC#y&9RJ=aHJUk2W#vYi(ng) zu6GUBezbHs&?NC_*A9$w%}A|*2Y6nHYcxJzrjZ%4gm`eF zRP)zbn%IXMB>l#TnBEY@c9l&1Gs?eU zSthAmZ?VV+BZCqeliBQ4X2P zfi6%w%lI;>>vjXKV5{6py-!qHG8PG~e1(UJsgLJVta=A=u`7$wGEhz;Tvb#!Fgwfw zJs<45x6y8Y2Bq{ktBT@`n?|ag*4%E?=7T+=myaghg9DaqS zyzuZnn1xOYiFXw8)6i8?583avJ}}t%4jWt~AEe`pZ0bNDArq=$@yUGb@WnAMms6#Z#0JKd-x(E2 zROnQ;ZNMQ??l4)If6Lx)HA~`Butggap6=0&_Uh!XFxDM`YEjz+K#6He3EvTGgbqK5i z;Zze_$~W0`9v$O7!>l!%HOK$N^fOOtbip`|8`@Eu6 z$<@d_ZU=t^S?vLgUSyv-m1C2-cwx!lCLV1yggKXP>kD4Saw+v+BRCe8XOX*vrehFc zj-xVI9h0lt$D5)Tc_TpiJPy!ml+zh1+q?Tu|#$n z!g7&UZ*n-j?5hkEovgAN_s`EgINNUi@yKtGaUvnz8xjD`L?tG*BP4eOeruLGx%ktq z*GL=g#2SrO>4|~ssB{#iM`?{%Pl5wRolWIbKFGoaVULdKQ2PgNemN^?8i$r298gJN>LhZV{(M|><52WLrNc>@gV9PYk&tOm2de6ML~* zv)&ur$z}5h22I%D&&*y1dsIP>!%Z235SfGGsQp*V6%o43L7xZGLb0dmM1g=Wgn>Xt zDs2X}<>>bnmnto8XW#|3>4RC6E{p5hCQzqHWLkqW3>zM?nf$t-FKV*C(TM_Xjmrb; zkP>U15Vilda$m*a6d|YD0Q0yPIdlf64`4ja2*TE&)gn~bc7WW*xZh)nH2QaGY!QdU zr{UB_bNL>b5s*al!*U!pe`r(9oS@s}5~IiNXfiY8?M{AxDe*n08*FZm1lfekM95`^ zi24_+r?AO9GQCNM{IC})!d8`Y?x_N?D(nusY`?FDQ;W=Ijm;Bz%Z6Iq_K0p~nut^} zyfLLmN6J*)!IrXPZOGh~21YuTe(*D(!3AbnCTg5-zp9 z!|2SBD}>gt?IsT76q_`TKs>&T{gGG=ezj{n+Q`*t4Spk>2?MWo$V{?8dJa#l)>+lw zwA};_CpCnWNrMx2oegDKEf%+<6bb6p^XQX4w*6rD1!TOb#R?VP{-7f01*g@Z0=Bm`@fU8Vi_oQAPCI z$gh^mP0CEssM61}nXPGC`VAkVbU&-}fF70_+oy4)(%2F`9z!PgMs0vFX^BT-3fohT zD`b?$qH)-*&T3@5O|G>0vgTcBF2UolsbiiYe4}2gwZ?qVM#6>NopNdXF(E~>#1s$N zQ}5}=kxwcOn8UM1MVOYYH$_blJ5jk`>M*23kCm|WJov=jc(t0eR)sGHR?uZ7pA$X~nRH-yc)joH(Hd1YHs0@kQlo6Tca@q}!+$8r| z?$c|-ddvcp+-7nlf)11;9#Rl8_!IVdJSY{35!SS&fz z56Cz?f%qlH_4h%9JeJrC-{tpGxDs|R=CyF4pHTrZx7<+;?3bz0pXv z`YC9(@2(BQYKKV4cuc+e}exjm`mh>RDi^{W6Ih)x^Ix7w3lM|?!aAolvqNkcm9ZYwwY zQZXO&*6fKAhV1sJFW(ri)H(bXn>K5-jgaY#p_rQl&_T7hoR$z68`v+l#)854LqX5Y zle^3&yYo*0?-N_?W_O6SaAC{@^R9#g^Wj7G+JhFWYidd2w0WJL0u;n~7)(aP26yB$ z4x_Q@>_HRSH4Ckl`F-}ka|s+D>I-WPf0qUEvQ8gz6^yr^`wm9v4rtY9{t^V0Mx2UR z(Q|D#Va(_VrG9+t47}XsHfzF3l`ywQZwfn{$4Cl6A?^#AAE7)ur9S={2t-iXicE>u zZAtBdv;7QiAP}^HeyEmyTPRjEq4~Ira=Sa2w?*QCwZW*-9`gbcBh+T6`!nf=J(rBK z%DCC=2xlN^bPlK9o-n@YvcLwDC6*#~#v^R$tRa%kjM|T~s5%lnA z!$qOI)osB&G3AtkOeX1%nJo&vc2_=-D`Y_#lrcjzkca;R@2P#}Kop`y#%9Qr$zs6f zDaj-nLy>6q>NN`&RP}C4g2>xm#OuN)Pc9D7wzWr1sf0g;3s#`mWiCU!-=9v2saNH+ zMPoSVWhV;6t;HnbfAJqDG5I~-I}I+AZ40MzLATthzawf^+JhldyQMW|2>JXKGM_s` zoph!WF!_cvN0o4SrS%I zJgYGk%w7^4&qS1p#NsguLM0DmbeIG2Vn-B47;`4QA@unagR74O-O0@rve~N++RXW? zr~$+g7e{O{A2HWl?(o_p>EHcVkL>lw$~O2LnAVn)hbq}%O0Jb|^+pq!^bR?NZ&yof zj(7sw1CogZfUx)<@_$xAK}vKj^<)c?5PDpX)<{7{u<3-JnZ(BGjU&p6^*Dc zC&KwyUXELkiMt%RZ!O1z63*G3fs}SNPT-1X@)2y{NRuN-=52Nm@Se_`c7$AczYY{& z#~P@3Lm-8^qs->8xze~u(4sP!j#~2&O*)E3=T5i-wE)3Mlq?0@zN})DofL@sJmG8< zjyPvOO|?!;@XJh?%$f7CKoX?EsSGFFb)amXxa^7tQqC9s+l>?or*pYWaS^zqOuORV zM7Xjpkn{xO-uhY*S)Pg-+`fAFs@@~c#`1prJ~2fd@_SvGAb{nfGsUyjtUcE5%3%cZ zp%|Dg%*B>VEe7x3Cfh=6H)jKdg6Ih*qxXk%xq^4qMC{GhQzd&Pz9{F;2ct=>@w$m) z_d04B;&hV65Hr?;H3Y~hzboYoSPEZ>Q>@tS3Fj`mu0(3E%w)#nQr@;TDGg)>#FMgP|*a~rkmi)&rAs34eBc zWjJq7#EVrwqJO6)Z*^w!NeMbrCgpYhCDeI8fMg~WXfsMAbjFZB7EZ(4qUBDft5GY3 zppu+SmEV{D3+57mQsJP*p6ZA#3s{Z&)8a(8a8CunaJrc(We`9;5nHpADU>&Tvhw%QSrw<# z(Nr@X&4QAc-Q{GyR@NoLc?|JnsNx99Hn^g`TA-A()-!0F(U{K@sk}tVobggT6iy-spi21-QYD;`P6!~MSn}!4w#yUuhYzhf&5mH=uOA)2mt@?TOyDmb-7|41 zGOGbmPlEpLiZ zbV1~VT)AGZU}j8F#Z!5IDJ1lKH271kPNDS@K`>{%A$JiB50(i&`nOvTAdWEzF(pgW_Z=Vuo ziB~+a;s@~x6i>wi)glU&cV??)vc*dHBVLy^CIa2~VAw{Iida(R0yG9%WoQOc-3o1n zdv1+H%E=CGmi^85S@WI&J|{xiOm`y1W^KEQ>WG%p`69kEMCA^Zg24>MffO9ghAP8C zgt7jkF;wIASdLCw*^XEGjT%8ETkdiuBHb8n2~V8Nc`8|FtCQu;QH%=pBC=9hW~n%u zrF0S0Rv4It1I0=spb!+x=aRu;Rs&aNEF~J*$e5XXV=8BJi5SED6i`d%1EXv$t(4#{ zw@dvTCZPT!@f4G_av`zd=!ujffntq;h~oAXBF$J}wrS-oS6it%$0t)%ZB`nUU}UR; ztB&Pk)k?5F;AyA_(kXuf%w^`axhmy4-P(vx%{c{tBd8Wa6kf&SO|goLV| zz*rQdTC*i$^==LiktXc<-SI&e+1|NM*YZ&>02}o9ab3Z7Adxk)Mh4rJaVvmTZC?k= zrCPF2f$uo!BDqpN8Z9|tNizL(sgrbLTsfS@Y_iD|bh?o9QD3=6oTAc1qrGxRmTC!E=peCq%$wn?g65qG0ior%HpYfz2c)ZC_Khac%^4R?TN;*?> zL(q9sg<7i&FH%v33*DiwI6%h{aG_+o8*VBGwpEc-)tjq&qsgP*Xth2lw}@tO?h+ z-ArZRaRiK|l`B`{Y8B{=)gQ`L>yV(SV6K>%W=riYyHL0i&HSaA$neg5FcGU1{5X%Y zbf7+L|7m;zEmtIv%=P%m=ltinGFKGx{V!dOqd@JZK;3w|kNKgi|0}r)Bo5X3AG%sc z{6kk2oDx*ZmK`J~{baDfI-N^R2ORAJ!m^UA3Wh>;Mt#ry)FZVKdbR6*}m~G6e$WGwCC0TDY(;_%_ewU@*&jGZi zC{d+X8rSnJ4h$d9ni*y?p*-H7Bp&IfCnOm%Cd=;+=6Zx;Gm^EJPxRXOl6g(5mdjUi z(-$;JxE81;lGrPu0OSq#W*rn4^PriHcamWyXN&7$ve+wiq=hjW@+Mprou6K1T=B8R?X)XC`3+ar#5b6k`j;kT5Q(1yoo+@ zCSJ)m7;NDIJTVzI8go(2&|qAgFQ>fLY`bBUT&2qL+oy}A7cVa>O)Xw%xY(-|rcHNB zU^5?UbQ+V8Pln#m?)7H41_ZZ0QtWo8a#a?5Bj(I5+f4~pR4rI9EUV=#J+j) z)nb-2mracaeS?pKqcCa~lSp8hsy9-o&iWXH)@!7npLgqH5ZO%MlPIQUWJIusBQhC= z8`2>8t~MX(j_a*vo@u&PO-$#?jbvRcJ*Z~mWI98prqHgMygk(dnMuSv;yV!LqHFC{NaV#Z7aQ-$wKpMzislaL>E!gqoF3S+7<_W9) zbg!iUO4A0a#Y$&XO~DUt>ZM_?PBjpzIvb76q#>0juvt6xQ8zb<-~q;J)&6M0Yi2we zcg0bE$vLz(Me_4$r&tbr0B=gUY;8!xB|sKtaMbs9E=w$;c}(bQ6P2miJfV?2gm~F0(hX%oAeaU)hv27RVj{C zB7=8pDnF<66pWsE+HNkZnANzzJdT5TW3`BWxPEnV-ec;yC!RJh$} z_KOYiR(rUV9xgZ5b`RljP*@+Ox@je2KA)QxcmDkGJ+W+kwh`1=q7&0$V!dO$$>y**FxVQlZJXnp|7>VbZP6Y*1aAIWrjT!Qe8ul#M-H{a&Xw zSKORK6Pd}ee`*u^@r2!UcLf+^@vFg2s@<#_In!j-MX7SCwsb;9+x)OHZ*VL0!_&cH z-S4d;+ws-Xemb7@e?i0N;-WHaOWjyUvr=I;Zsm$Zcd2P(we4{v!0!gEdFn0ej)<~V zyFX|ZlmJz3W2#@w_5>1uJa2JOt?!k>SH@_s)*G$(MnmglWSpPR`cW7J`U!r(J90 zti@(lR{};k9z~ujIKJJFOQlMRP0){R~juEiUWD^a5)*>I;LMjjw!Qi zbX$6LPIRYN*fyIi*D}7WFpSARXDt$w(yxk8<-lgbD zsaI>Alwr4u0(hFwrV~4RT;Vz^5Bhw?E_f!nZZvv5*WOt+yKXK0;=N#I> zbhT1xcRH(Z&`iNe1tzPU6+iMadgD16^4SVhQ3PWT(9L>hIWR&zaHkdZsqvobnzFwQp za;iKgjyqgGteZ3aA8ci`YAomUd&E+B-dRlnhOk;-b{MYiZSwv`D$uE1_b1`a;Kx{* zRyJ#kkUgldTFti|=H(#ZzPQy+*U{LCtln>xj`eFpAozC6>~GCGJqILwoXp2tOIoR| zks6FA6H^i&gIt_#x66ZE*yuj1c6QT)MrNkGtqvFc$y>nF)9rO8D``$%Xt1bsc86{! z^TQvj-p9*bMh`67oOk*Y;S^-US8DbLv*Dm0KG&Yim(%5~bfq6?&ZZM5YJoYCJS@iB zVLo_z*6bJOH`6jNIn)`Pt4H%>V%ORox7%acomx{{=qcx=&|OfJurDjNK0 zfHu2hGXPYm*ay&NFkP#Hu6prmYu9es-njC`&Z#%hR}6wKz1`*1;@OFuL(}PCy4G0k zgg>cq=hPie<5vTh-gG#b-)RSy1Mty!8I8`R{pM}3z7NQ6wc$qpwtpz1(hA|~y0h2{ z!8HQ|cs6N_7e4^oO{Kh?Zr%BqnecEh+ARI$k^6?psIhw6H<$^9&3eCCZIt*J`pD^i znlMLIg0uc`zh0k^Q9`5ba&_vzP%t~;R=c`qA# zlOgdO*7)wcA&u2OgKWgdUcDuxza0MOdfk^~ap<)r@ceDxmq2Vm*>5|*yxZ9>FL1q< zIjiF93am7Xfs>dhx#~$v`Dp564)@V2`FLJ*qK!M4I)6jI3HdG!uY*(Csj+(n|1__1 zm04zs1WGK@PO9ET#W$BI)3ln-#C2|y?YV3%0;5prbAf(v$}QN)OF}3nozXx~SGC-H z?-RJ`)BCRurAC)O=?eL@nD$fQK|TKfK9>cp?}@Aj)+H#N75BGi(&*1BLu$xZ{!`MXo+lKpuhkvh%WErynw#VYO`l%HyV{V{}y zj$k`ecb<{XU-*>z%i@sh^S9Tyhh4WAYG6NtjN@vUkj5~U;XO?LSFzsNF}pkIi&Izw zYJcnqaNNZSp?j3+DRU9@w-HFphJ9Zx?5=kWea@dvLJh@=W{%cH_hTEPZclkavcF1m z%6{!t)1Mkh>nanZE1_6({6o@SP_odp3S@~b*1zjt^To+|9*{@U#gNjTWwCiSTonAL zsvg(1a4v|5tYcOQ6iZLqtzzO!U|SoY=(j{k+SOR}>G(#%qVp&= zTSzb3B$|CH$6C{L$BlfDpiRrtHriUEB5P6RJWB#7NGgrbofF`BGA+iLk=2z27Ct_p zRLgEj3OodivUa8ov@-Ofm#XIWV*8)rCmSp>8_3wrGhF0=e4FG&e2W`ZMt`K1{pnY>=oCR&b)nnneW0FgZa_?LBw$;2VKez zlgEU(?AWjbEmbiFkCSCC>8U=)q6!(k$?Ct9?Hmo`S=h*qwvuNH(U4a8{2@nkX$%a2 zKRm=cK8Hj!)ecZVQ^CF$8Fni!Z+iR8$IOVKs(-skT&KQ!V7MJP7HH3SqCpS{ z_^~f8uw$_zWR}>bdHyClWBpC0Zejm`f07@_;%eg9!5moVDp?zw4rp%lc;giCzW$^8 znL1M;uCk>V7RypzMVZQ5WdAdVw8PxX0UtBcCp;Hx45;E$Ajq9{1c6SKy#bMU@{bQI z-Od8JvTYVQm_Jj&e)!hFJ#LN{EK-*9Owx(pt`L zJ!-*}6gqH^K-En$#KV<%NTf)1s#MS(Y#NI~WPKPkWj;))x}g8V8P7W!DGnjOALW-7?+mlH z)!&SRu;lvo7l@$C^Uw0i_fs9}uqV5sAR4vO{%7>Tt-So>buEUvQ<~sXI2k#sGLNu{ zQxlgG|Du#30)`*wh~o6y4PuyDmoo^|;`>fWk)Mal6!78>MN>SAd1DAU*7)FFdMY_W z(QAe?UHRS{b65p$>B(=@nL+`;?goyQn^aAFG7*V@jmxb8)^;H0v1`{q6%7(8U zX|U&uO(HzbzJ?BLy-5h(@g^~r>c@UTmDb&NVDKW52enFt^~13OquFWB)EIUKo#6OA z0O~LLcI-8pHy|?8(fHIR^X7Kx@6ZUvEDci8e)I*1MHj4=wDC{~5YE|bV24N=`5adU zX&NGp>MgpPh<+CrI;?E@KX!36Uu>rJA(>xoJi8%90zNW$&?hzk4~2Y{OXdy#_rr5_ z?w6QtdCHwAACLW^TPY+vwh1LGGp+=nl2jhe;rBnPS9>DGjtcjArD4>qq0x={%B8T= zA0~c`lJ^gz6N~N|K*c681tH<+$*zQCdk4S`+;M;vox~2?v9jFx4TP)up7ro6uBZ~) z7PZ)A#{^z2NQfM|ISA?p53P_%t*X0-Qf|>)q-N_`3h%E-)bVn@7VJHMBPq_fH$n8{s(Rj=2Iip>AlY#}wYCUP zJ(+?e5E<$q+`k(c8Y?pT&USzh@b2WXf3-CrCjSyCKieZ5XYNZWgY^mwP+NJ~2eoT} z@(ij4@$%Aw4My;LNR&9mvdxU8HAEnM(_jyv!M9zL0ITE+m2_DKR$Lfk*o^)ZwPjXl`1Zj*2Q5o&Bx0QT6lg3!bp<`SOB* z=4C0Cb6$fXG{FoWm8k_;J^Tv6`$BI8oI=mA3&C+fn!rs9M6elMzU!MGj~yiuH2sLH zk%mX_%L$kB-;&k}bQ~F@97>+1nV96~rus>Q>U}xWJ})bY<|KQvLVeoZ(K`S_@O`;- zV{9Zr-7@VEl^S;!a8h;7=S$F7DA&*gEo1@b_Onm-(X6D~=%JX)Yhx~pvf5dx6s*rz zcXco2Eri*pS}G_BrkF1;gkpcH#n}N0Ux*V&UrV8C>1)l2W+-cDAt5+z>F@r|FjDJg z$zs;`%wQ2@vG>0YOqT}Y(QK}LFQf{D9!P7bduV@^h&E#>2K#&b_B_?kF*FW^v+4Az zxLOrB#8pUqy+_|<-TGB3_wgp*IZHUB4-`x?X-L*Rp#;C{}SFHKE-M@)#{eXXwMU(A~S+AIV^780FO|)~nkth##WgMAC+rbgR_<5thI0Q<#!_M~t#JbV z(%3jcif?aIctJEm3^Vg(`?Or<$^_necNPolsdmsQYY;_s)`#{Pj0+*mVm& znru49M3Le9`nQEJlsKaQ2JCanJ#Msvhy)fmATWSuD&WHX5gEV_<5ll`$Z*^CYmH73 z-Z5z#7szXrMPl^+cQmu=A7{`VGH1I66ie{GiS0%0Lug{bA$905# z%zU{o?bUG=zfl3%onY(*pO)q99!H=isv}i|%-nB%UZ%)Hzb{QUOP?}m$rXPx3-X+| z(QB8cy&JM(?Kd*i2(ZLQATD@kXDVL|KPy&`0cYrWsE{>OD&9BKg3|@RGD4`oKOO8p zjMy)F8vB69BV%cqqUZnN>>Y!93Hray*iL@2ZQHh!6XV3TabnxHZQHhO+d4_kll!@M zpW52`zuA2^RXsD+{i&&%p6UKx7Yu0GacNh+fC*`@vYmG-dKA46zvg}5zb8Xsd6vcV?kB$^OMOq-nOR^eMYo%9_2vFbX!687kN1|H{<--_y}H}7Z!g1oE65wa0uxI1G!sgc#!4xK`90 z8L^=bsGW)ocQ_@1%TKL;hLS&yi+!rdoeL};e^3VjilLtm0x0OO5zRkF|2(2q71ixfK{lQp4&>sr zC|>2h-;B_#+C=<~fLoODn@p=E1^kevy|y_E3k{x%$kJEi{u-DgwXe0DbRkp1xXj|H zf4IJv*t&Y*K;PCM{I6xmQDn zyi|7vhq!uJ7>(URhO)3sPxAFaGbe4@WXtZNva>@gq&+LJHh3A!P!|yMQc@j|U+RQt z$9MIIPrzG#Xe=tnY+-OipEtnfY*wvQY=y+w-Sz2fl<}WxEnW8V=O?t0in>}IF}^8p zKu+6B82}RD1?an;M($y3&NYV&Ux<&=>?$DxO-pWPSsVgQx1Rgi2NtE8oIViidC$jj z!?gfQXSF32Wzmuxa!)wR1_`o?CmUpO7!S!8W6}!s5)<6;WG7z&t~X!J;`Ci3x!ki% z7n;KS0wZxT1jh-c_t-3(5S?3u%63D7`J zKO>qh|7(Srk3;86FlOGYN9NSIIaz7vOaeCz$Q+s?_#|EhtyIvhl4XO_VK+f$l+ zo9g4?2Y&Q)OeSA{^U3rY`svzT0Zk$bOYNMo!IsDxu-PTe)HNV5>Vu{nS)<^nTy#nE z!C(Z;6*X=wtGjx(sJ$Zllw$%b^#Cp)a?pm4MQld16+*N&AWE5gRy^jsS?A~Rl_w2& zGoO)NG;-Sy&SYc%Z+ii$DxQiB)Qh=BrCuB@%gra3serWm)?CuC#E)O>v)f7%06#og z2%jT;d?Y@H^%U|8Uy$Mf)Y1XMRSs3)QwU^nsWndb%rI9HE*f|(@5TTEMhL$;d{psZ zUcO%uQHRc}6HT_iNs{X`Qi|QZwDahWy4S#VnY*+krEd0lS|0HEiK&~|Zx`NwcUFcW z;+^I*A`~hS!9=T78GnZU1@zB(%@z=yb?>X@emRGt)_#C%NoT3e4!hsT_*ZuwS~{VW z^7KNy;%K?z5@}DekdBDCB!|T~D;kpX7G>zbrjz<+t&d4`3y00y{bo;YHu$hMAUpl) z;PQtcZ_z)R)_zGyIDW2TsJp!FKO&xGb4AFs-GLP z7a+Oi-hj2?z9BLG(HFs@To>@V-bLF%WFb#$J$OPSwF-aiVHSQmO}8D{2Q`u*O~*|=G5il}`Vdx~WxX$AaS z?DIgS&8`O>G*(n)oWzNHOv!e~1Y3~{Fdz<<2XAwvocbiT&jY?@_^Ir=*Z6W9^(qn` zzDili3W_}u!dIdNq@b%4sX2G;3q+IN1Nep+pWY``;C#fCY}a=+6VRtIxV%{8)k@KB z6K^QqR3z|uzR-0klE6h^8M@Uy6z3wXAc|88d6m`33LS&ycNwv`Ze3v5ObwI5G;)oj z#d2UqQHAbSw~PD$XVLr7D3TopxY_}1!YFcvG%v@5%vVZkYT@Lb=JZj|jbE&@hG*nr za?bJ29EzGFT<*Lm^Y{6bGj1>RRkEE9?2rb^N`_Ed zxs2oa4lcmt%;OgIy!PWDv*+@0<%WoA(BRu{DG?yA&lJaH*A+H77#LGjlD3)CTrAQo z1BM?ws_OP#+J&^9K3a)L9#xR?dTFvk5rqQXx-?DoF!jox=moM+*&AW^bh-dVPR5O0 z@)z%|CWZ(m3(77JIN|W@(^TND`!&$?z08$mVy>Yhslh}J6DlTmcwL+p@6Hi(XVIjE z0S@1=+hbz4mh6DDW+V2}HK&$#HRf-+ij}XQNB>JYL{IqL2R7AOPy={tcMEVRu0(4AxAoI+fX>B49qw#%Nq+dux zA`*B6cMUHMWN$Y`216T#rBbxNDS!`?w=+IUyH`_d%)^!iAQsr#i|e|oP6<6{M@tzc zrIa5hHab@$nfE6J>&114{&9d7h6)XmMh0cp{ZX1-`n29D@f7`4N zPVG^=l}xP8K!*17mZ3mOEH@Yir8i$iiDidhPZp`OUWcY(L`KZu>A?^L<0b?>n0s#zdMmwL<4}ceG;-{4!anWR0aR6D1ZBK|^yBmlS{{#|EpcFQTGT z(8xeND8v_xo4YW>{2(rzNW8}g`5Sog3tT=oSKReu^x5H!sZ{=#%mZJM|2r0+OR@mh zay*phDF)g5SSjMcpB~^H;T6Yd=@d=g=CZn3MTfJ0&%O!b!e8P4on%Ka0A6STx#6KB zKSZei5p%SBt$_0i+;^WOWsR9E=MleRdI5`u?+vkGYT3j|vyc^0U1Fp~U2jOe5cm6v z2M$_ViV51di7b*R1e5X2I*MS{t+(;^(=YnZ2@weu^r*DjLdS3st5smXnFvYI+_38W z1ETk?x4yTJ!(N~N`T&Oeb~DPLCbdSoES&q%el#-ZD7AHStz>e{<5Gbz>hN1MNwrPL z;BUs4rXU|aHP~oN*Tk%DzSvSst0fz*DpJrSlG1gkl1^D<#{!O8bS4CLV|F;B41_gv zt((59Q^uUee^WG{x_rMzVa>l_$v{SQYcGnd>m2L7rvDa;{=l(c%Zjf*d!GU}L%xyiRVdY2ZX=8o)r9 z@@g-Nx*9acHAZOj8jZ5q6FF3X6h(az^(+)|H-X80f&*x#0yROhI<$~ey(-#URla71 z)>bqmv#5Fu;!Ki&mAjH)+(~7mSIYtXH&<$7g!$XMKxE;+P4wfPqEL)pCT%k0x@7z) zB{FF=TX$BQ%}aMu8(|*oj$Lqa5ciB2F+cco2C8hT9F;@M@BJ%Wio-Rx{38D3c~Oz2L#V zF3lZBI}6VE6@Lwr(P*M?H@S_gq{0>PTVIGA+|yOYe$vLgcm15vtc6M)!+A}>Rk`t) z%S`1EjoprtJwtm(UUMcjQT`uPrP`V?78WjP1Ljt(4KkW0Ph~6sD@tOl!Jf5K+N7%# z9rfmyE?t1WJ`zlMgra=7ONr2)z1UP#oMLsJab7Ehlsl`3eR%Dmq|XGqIVk759=iI$ zQi}ubABDe5VhFcb%I?}YnT|7NYKc$hFRiVi7gT-gnR##_mL>;Hc?Jy|{yB}Ih;rm~`Y#S~ny zkZ+?C4N0Hun5bE2UIf9MdQRFVH@ZcT%Em*;F*AD;oBD9=wa<_m4tMV#gHU%|_&FSi zDs54bj&-mf8KH!7W}*1}~o{h z68{BV=uF8LhnWVPz{z?(Nyod@%at&THgB0eZGs^4#_IuV*KAnVkD=H8PB1W5Ym|fV zBedvUBT3+?A!R0;v1KKGkiVuAe)~dF5YwrRB^H{XJ*ay&X>gzDs0q?(EfU0sp&y*y z*NakTXPNR-E@U7g7qk=US|YdN5 z@^+UlE)N)QPD?6$9m+C4p#TOsv(=dnEXAFD+Z=i}almT->ShY|O&b=t=Z;)8DMfIJ*vPi>_!T4vgOZRN-f zml$!b$pTjQ%SK0X_V#`vE&PJ>27YxYKKv2f7`hEm;lx8Ay)u!leQaK@OEF8|8p(x6qi_-X^m9kC}il z$9K1Wx6LKY!}Q?}yEx8wTP9>WeH1=8@L&iPxY3xH#DMl{%lxeZ8VP5dqW+u?HQvOe zAoL}D(xpOf(~@5}@`-N>zNy~DSkO-+8n_p>^=Z7NCkvs1n+`R*kt9M)b_pTr{!`Cw zzOYkQau;oqL)^If!mRpqi7gN4P|}24?-r9RKQtq+c^dUHp-a>3E`i~XcSh&2lb%1g z8+7ekH+SL}W2j<*aVwz6X{q9G9b(B{&5dBxyl1}M-sw>rNN(;u?=EHXJZttmZ$A^z zb`Ls~7t3Bn(NLfcbP6EbKf;r%n0qIHesJGFsN|s2^8cteYim$o#k&F~dpwk;G@(a%%!#rWneb6V%XLjGl%2TM6+U#x;5sIy;i0RT< zvD5^9o80|8wy)f+v7)qqt`lt;c#t8BIm^89zp4~y*o-p2dfHv(ut^DjCe9ko5pDEBF zdam3`Ep|nknU4R&C-u{-6xexL){+%ADJXR*Qx}Q3>L5mDWRBOy-i_UT{eb;q?yD>e z9ZwXs00?T|{nPd*lz2YmGF}6fV$&up>c7!k5ofXEH%C6)%IehPDq$enht+yxDM0oy z^g5h(II2OBUTm7vB6VsAEwZxyAQ0jhZ5`3e2sg5H-YfT-0MF$XUoKLg);bXsrKSG! z+>wxa7ttaK7nA(H=Co4qwn`W9grF3)s&E?Kzavo61Cgv9*h?tzGp?VqI;G3223 z`Fr924|G~8$6@%tlj;jPzyvLzdaSJL2P*MDQ8D3DU=>tCLHvF}P*)Sal>oLB6+a5- zEJ3!n0JmL*-JYc!^!cKwa;*7l3oGUz-3e{Rzh;7E}M7ZNa~{7o!Qc zC8}YCm1vzl=B6_$rGhphWG%*-*yQGs+feZ?@mW|YTt%;GF1opZ%vdZrt?}uuU@jT( z@8mPCuyN00brEO@DRBLEkS>|`8`ngZAY)4%^MH|&!uiB;7>6VWWd%!8DyDD(mG{3em&;^fu3qG5f%*H ztxV>)K;JaZCc%k>0pd(nT$Cs%t&^XX*tX$5C=U&O0|+Wxsi4zJnO zTN0W-DSd+g!mZTLxSbK^UcF*cHGiIlNh6vr z$1KsZJS`EFOe^KI-7s{%b!H96q51f$!d~RRYcjJMES9fbELqCuEu8{x=msT~ccqw0 zZQQ#-<9vi@f3|WPOLY$MC);mdjQ_x`GV3}F+nv5hQTDLZ5JQ5J31k~7i_;f0x|}Vyl3(6rE=R4>oS@ZTSq#`2 zNCg&;x%4@Lx?+MVHMha=tRh4c%i82l(-uW%w#nk1z=D(_$+J*~jjtvO97y@EmVa6( z&=hk++dy2(_^6!crrYU#k7(6VTwN1b{RJ*r3Cq-!l*c9)83$FJLue3gY@=N^R^(WL z3_h8OD)l}k>0g#`4=(nxA;j;$oB;;>8XW*18~VDj{Q2`iZR;gB3&M*%1vl>y&GD2U>LQ^3>&R=$aGnB$YoQ-qmKz=F@awY zcFkxdTr{}}3pe})Xycp__TVf{pZ24^*^Ls7FwttW0yH%1%)oqa0L5dE$d41;~7$S2!*B&zi1DU zt|q5u{dK&I8(N^0uagw|Ad+8=*7+tTg}&-myyu_SQjv2ax1-K;qi(RZpp9frGKII# zkAgDvT66tgF)BZMhKqHD+d2?YEq7Xw&vtb6UoSwc10dAZQ+L5azJBr6f2G5)zqRU= zGJmqDQ?1WU(cKhSu`fdinlyh#nNeljdaLei#%cotNEd2*=Xh#eQeFJ~6Iy>M+bdDV zaZ5>o{Q2BJkFm|WfJw&l@QGG#$}KZs>1)hD&&`eWoc_I^olRO38+Ei@o^_DS~yJF@P&ll#qj5)n{Og zUw$Sqt+ElzGBEpKmR_vJj+Mz9+Gj)<^{TTC;tdT~yjb-Pd1nwAD=;#XNxRAR{ zV8Q%T(8dsgQ@PB{=q-wko zcwAsM?a;uQ_JPP?V~Q9*gs)Q!6kTD#4mte2r=9MTTgKDXdb ziKsR9{XH`OK=V_I{H1bJydM$B+=oS#ZR_|fw?&8}n^}8@-Mtui#Hzf3t^cHkt87V- z%)*#Lr?f!iik@N+&*qYr8~*tKKy0$(c)+j@oiOrCrrT3AKtC=4rXxs*ymo0`%wsSU zOn23jO>tlmn6-*m6L9GdWOFOBsi)*cq$@}Bv-{T`1E;#coc)4oB`_qHxr_<1ylXqo z?QRNw=7UE~yTQ-?h`xPWJ=8aLgz7)m>y!`>dZG0%$K%aQH??2I?o>zg1?2#U8}*o0 zBgN}2b+5e9A-uCkJ*5SM#g&;Ca-tWNh-T6rz5Gp>N<6Ujp?nTs3^%N*G_G-Fn-coY z@x!zDY(#B>Z1dYmvXG^Di#T}20Xf5KnU9NaswSL+>kcQWWM3}GBEz?XV#bvKnA3W5 z_lBDJu+m7!HOw)e5L(5mqhl-l(Q!m_m{UCXrD;O2X#tNT180eJ9P=GEV8}oBc6v>( znT+fk-uvNOP~~#E>Uvt?hbAOPK*Xo#f(tR)B0myT7WZ&%V2&y2 z5UG4gQ|ye1YZp6jKCeEVRWkYv3r6*10mmXrbHTyLd)!KuO6zq=0*mSerBz>?AL|Ki z*)e0r!!fAZeJzGH@)U1raz47w@GfrkGmq+T{eVN{AZV}A@$QPJ8O;B1q`izfMMH5r zF$sNhIFx2)QFhFnfKlrPNaD3mIyy8dAwdRohCDb^yeRkiskbMd&mG`d!f5s+>)?LT zL^zYo=8moV$lC~g#yvC*pHZ?7lEqQ^|yRcyAx*-dwvvaLBoIZd1;vUlp za}h}xDyX~FRdtR&!b*-SfgvJuSYU&;3sHl5x`>d6qv0t>e0q`pCG%%SRwAHPwba;W zj`uXMe%8^ktTZD`joYS?4y@%0XMdlB3QjuAm69)X^C2t|TOs|sIP;O{@W=Tti|QGn zLyM-il*D@sRO14ubfM{xiP3OP-*N%+7Y>0- zq}bMv@vHFa-LmHzcJO*0<+eeCf@UT#QK!J6C_7N11+;Q^^)wPHQie6RtrQkKwowLl zeqP=fFHmlVR{;G`$6E>$c9mlTVAJWY*_Dy#V_To+qth5UZ=q$Pg#~R7^f|V z5uRRe)?x9;D#r=pSMB>r2q&HU36jPx#(8E1@7gJB5<2E#_(Glfe#OGBx>4AF){fKs zW>t@45WR2>19)CrwWByrI*;RszFiJed2V)BTfunB_p+ewZYK{Q>^oj-l=UZvzAvpo|spi2_(UeLR5X1w{P6VCnEn zgDsR0K%jk_{Qpnc*Uxcd_Hfn_WCE!aC`bZI^j#1MGgY`K?*NKF*bo_ieYm`80%~Ls zDLJxIn53+7eAH4AA%caZ47Xui4xBM5YWldaT0?v`?h!c#*SLt@gOZ;xS`p(rm9SET zQWi>55t<35441S@PK0V6b8?yRuL`BW4AoM0mohQF3*}r0)*@s%b6J&)`GAwABDQLC z5$%md|CpviRNXUKj*i8^9Oq(;?sFM}4<XeCUESW|}Fjp>6rC-r{sw1!NuniI?^byQ`Y<`21JqbBQc@}@$seg& zd;%IN1zWkK2=_u_%6~G}9FzUYIPmI-6r=q=87IaDpye?JQ~sZfi>X93CzNtvREtoO z%Z1r0lmjo8iqKukv=+EdNyW zpNs`Q7~_DS%qRn~h8RN_1LEP#{+lsN%nyqkCNXV9)s!)DT-FTTDQ#Nkgee!C))+Zb zb3i$zF)F#r5WG@zP-~^$|Jm9YHFJGHt))Kj`pOuibA6Ei#nJBz_kaTOfMYMcVCwvNzS+Xr?3uZ$Jkr&w|x0#bQT0ks~3+CLe`dhem? zzfJvQoSpl*htc~!sQ*nu*iNvg4}zRV2u8vw1T6(6j2faCM8b;yxB z06r!n1fvpGfR>LrOeI7mtrF6ZRLBdUloF#>N~cyW#%3#%m$OvLX#H} z%3z8zqdiO;GbU#Yr;;_nUQF+oHDwB2pEALR)fn?euTLwc{ein{%rL7pMzB?x;9aiw ziL=+IS6dt4Y_9jyyEbI8w@&z9-TOrF?}J8pkFgls$DQ=#`8 zz>4a8-&1{`e4#~u2OuT=4k881hmx}&Kn|%3q98^N``=c;Gf|QsE1(?hj}`EZloU8z zN|ZMtA?W{D0TV)=ibzPX#zfH@64JAdh#}X<V~A zsFj12RZBtG-Uz6d$wtxs3o{QQ1T+u=Ds!$jeoA}(Czv@f2-Mf>f8G~Wq`;8Aety3N zDKSAHMTP5sGot%L^zFNo;zsoU&J(01L>>`g_9w+niV$N=-4lx+iq(#CR9+-0w1d4||$`1oViiR+p+Y@Dn2Qu8Zut)G86$@ZA20@G* zJ$M6xiwF%BHc*9u4eGB4661^uD>nQM#*H3UEX2gfn4@?KJ$2~-iL#Sa4~2f@T*EebX2598!Vg&20y z0z%J)IrOIrr-T|QRSLIej*%V{W%lDQS~M5IW&@&xsaS*x69qeM?O%^CuS zL)!zv&lUtC*0}v`|vI-_i{YDW?<}ZdQ>}PU- zql}OQrLL#WTBJtRRHY^b4PMf6(DJQF4j1807ROx(8W0ZWEn6D>rDUSzk4&W)QKwf3 z71@a0twot8iFP-nST|wH#*7J?S&dZmYQ-Kxcl^s;o-7)@fsvMBBP=u7;#2Ia3y za^-Ym{9jRIUdw7M*woF~)jG}7=S~Ch#<(k%V2^#Ov}yh++R>66fi@5z>ly^BR!^SU zdUT5C!HfImOzNi3mgUwBTG1(v?cyCPl;DrvqE)L1p(xAzB*@b#(%F(+h?1w$?e3fk zQzZr6(#7l5L0b8S^s!cumG!21nbG9j$>y!evg*Qr>^nEDl0V91+p@0*LHk`B;;kjT z+8xXBu39|HrJ2wtU7Wf-yESRSWv<1^P_36_-j%6SBgCxke69NL0Tk>lEz;}s@;pxW}iVD96R=;f-~!FeQWWn zk{`QO+1I5gc#|DDS8gB@*PSbL<&&kWZCTc6AV7PxgfZ`uBdcvY)+f|Zzf}hft|?G! zjOn#vLvX%52hk@=Nm>G^lNC9RfV~(D9VA&t{KQkH0hRa0oOw2*FyC5C$x33pA5lW; zSC~Q2k#rbWV;S{6ohkHbheYbV#5QWA*mgB|^QOG`zP0ENl8f7x?OP6RNACgESvC|| z%a$EFHZME}8eI5sX;;m9-Mi7LPs^4($Bgc{~ZL&9xqIk5FGzfo~ z3*ae>-U9}561aw_ewak@R5-+et-Fd++9=5_yHlY{_lCo{Ra{3z7WGB86rk9VQ3aD zqm=O{k6ixUrKmp$fzaknlv7cJWd1@#$Y-(0@l*K?XZp}K!#hr6bPoZ7I}Wy!xS6vi z4ghg@KZznH+KsnZ1>(o#b=X)j8k7!mHkVEUMe$3N3@Ia2m0J9?0R!gNeuTIwYTMFv zjv!%T8IAOxQF6yL5{P21oLk6d=}6xmf`?F@?~K_a=Lv7&azVX1No$bOo}&dWG<2`g zgXaim{Q6akS&BdWp#lU+($BipO4Q65;LN`3gpcXgFJb)WfDM%oA2CbxGjFx>wPR(A zrjIoFBg6|>^3dKRri;(7G&wVrN*n4*{v&3OY~lLR>Nt;N;`A+2ln7(t>KSv@byb*l zrE=A>ZOce86DBS>*VL&Rbgm)s?qOmKt04Nq9Sf&5YyM~{>IRD>+uBJRmA^Xt&BB$e z9nI*uG9rl2}zeQafi(@E#E(FI=8{^-#B;Uo)I3OIaKO=4#?Lu zL+2e)i{CayF9*=HFJCPFuptH)OPs2vQku0(nWdN2a(616ud-6ISLl+ggSDo&D%+?{ zlr!~@ooMOR%U2)OWp*5Um8#OXi`548${Z`KLyApUR{k#iY#6eBYOl^I*0OG=s!k}} zYFxa)(=In;K+kUWDLkZDodVY`T`OtPSypR5rc8q=7VbZ>OY&srm^Y&5j9wOQS5Rl; z*Z_zWA3Jwsh8Xs%RkT#-dS%O3x@n7-wrVvtPwuFd9#;K=%@HrxV|0aro-oMVw6;<6ae8T~lWOwQ}uyNZy?wG%|Zi9<8YQD5<-^R1*HoEQXJTcwfa(}-j zPps6l0>H*PjaoKneY?6XpVtAdo3$1-?Ot#YwVSmY+-^z=+f6Rl=yJ^3bsg4WF&fv; zpEY4II=6SPv3l?qPd7eRWKR|*jX${Q_`No5J5zS#LWh6ba^uD(j$5{B@t$TJ?{sEY z^LGxPG2{J$&5uofTC*;TO_CI020rBgu1u*C#mFl}skxw$u z6-2+vnZ=E=0AS!0Xjwr0&}QJn(2%061r8DOq--x-vx_f!R=Zb`w|`$h{=MI9^X%{b zd&79Rzc};~wEH)^{MGvExtOZWVgG!6ax>U{eR=BD{dfp(51PThHr?{tfyj;H4yohI z1WG{Xx4r*)H~%o~?Q62654?wweSF!ypEwAkVZ&ZJX!5+ZBj|ZINbxsY!p;q-hNt^@ z!*UsC`@`xDd&&0$PH)rMHG3K)ZR$fC5^Tpqw(fkw{QJV6f0xIOHt$l}?U-Hn(yq(3 z_3~rHtbx-dW%6Q+ymj>gRCZN76|x10d(F^I7$SDk=EJ+%=`U^TZqFKWEF?s}-3Sg6 zP_Jg~?Qf2NfwEbf$9AFYO?w;hX?eVhS59CG5Uyvp6zyHMi?&t>+BzXb>>upvDKocH zVt+;F^Vw&s8s6FS4(6wP=(Gi}%ZV3k6=IqW_mU+$NZg=B&>TgwTKXJ8pH zOgbFKQ1#>i$4+~9i{RSnJ=k{It)m-=XDb#q2ryaKUT7!-K}O8*1OD7ul;@CXWJI73 zrBHa-0-_~!L0bO39i!H_0!AnukPlGBJ^WT;PTUu-t{F;%H{G;N==Ce1TAWa*xvN3O zTt1Y@&_6AlKz%^KYq*#&L41m9hYf%EdNB9zLy#FSB=~>?^8`uH;5LL15<>M$LM;09H$N^yk?r|Y3 z@fxh6?(3rHaz;=&@TmND2=b(XKgCz#1wQOxr{lc-5|Ye`_ke-`%fP}Ebd`5MXJEZA(T-pZ2&LLxF8dXTKzb1_n#*gY*lj(=ryh9`bkt?VapXNRd zS45Oq8H9d0j7y1*(PeXrhHSVu#dwi2h?om^wP7aeW0!^<^RabzioEGYwic7by=-Qs z`mklU+C;o?VZRG(x;VZ1SaG`+zshAov08pobwkmb#ly30zE;U)vV7%MUOK+^d2zZA zmC8j)`xc>!Ir|5A&F;klpk~dM4W`>7xb{GeRi-zZ!?@lZf{ z^l&X)yN;!Xy?y_)3O(XjuW{SrVY^UqDwDuTB`+b@Gk@zM?Om%w*Ag5;$>FtZ+P!2Q zRv0toHaeWeTmAgAo4nBZJ*U#OLRbPnsZb?^j5`S{y}k%N8w6VmP5f z>G4V%8_sKmve#1X{h(f=9sEXt)*;2Gs_Mj7Jh#P9c_8ihqpM8T3y)*5c5ZZshzC1X zip2DZU@~bje9P(HCG zpU+lW%}SuS1ai=@P6lhx@sk68T9rhkts9d{a&y;)HTR>C?>zovV0%UN@weIr?dCuO ziA?3ZInhJI6Q@=cx|IXRRn6(-i{9l8d7z$I>jgc)hSv)-mej^K?FGLy``x`U@$W?i z3#QA;hH;N)2h`BJJ-<3V=5>9edh+*6UPNaxH?{yBsl%HT;52aCy?dh~(Z5SWrgHS& znoS@IGIQIYT@mu}(4DM&aoV8gQoshkWp?tgqgBN$!#g2#cJ@Isp8W5r<<- zqZHS76}-_3ozIV#7Vpk;Jp^tWrffbt*=}|MKKRLBK0JKczEr*PkGZ9-i>7X#>ph>| zZ#}tqsiCK>ql5d`qnA&sj~@JA_m9{6&L3YNUY*~MW{)CX{IwVtuiL{uD>-@yrMC>7 z?JZtkkKCEve*`{1Z^_p0S98A#3%ahF7biX21!_$<7axvm7hiV;d~$c#HwP0vzyEsk zm-%-8{r8F3^lE5p@?WvY|3@zKLcz5qgscXO$*d2P5qcFSqv2SL7>=o|2J`RH8Y!L- zxYZhy&IqPKF?fwyi-n9Y7Ijnlat{lEiaO(^vv%VzF zPPM7^dea#ncoo+uOm_2yGGLsSslBdaUW!CHnL_tIcwi?xvcx zR)g(C3fH1I9S)npdVx^oO7#zXf24>hv83w9(Ycj{d`WJj#b!3%A*WI@y~b=d)gd>C zE|<%*t z4wENx+RQc^IX0E9HrSk|b4eD5C085u4*LmQJnVGXoQ^9Qbd#=fnH*N5SzMd!HR-KJ zV_g=3u5=shc9V6QjpPWp>_0+uF0U4AEN0t-B(lPgYV8*Dv4LbKH8~t+r=5~$BlZ~` zF0`O)iUzx%9*q`VCIIoi2+)^cyV>=c5V}6$<$6cIVZFcpizHt#+HkAvqF?=q*;q z^B!H5Qd>L@+m#h17B~6KCdG@|)a_8%xEF5D2=>c3Ug;BxXM+yDqO)8@xl?tWj@Z3QW>!((^Tk*@LzpVi^G)zTbDfw0Bqab6csQYx3z=Cr$z zJ=yZZ*77**=r)q5#cQ=c9~swvwbkWyx?9NC=7`m4bGq)xM=hn_=5cvC$Yyz!*W++L zoAM{&sNLjrJ~)Z4G>ggMbU8aPQ8|sl z*=e=AJL-07t+)F7$MW*ziAeERoLZnxn2UcrnuGTart)we$|S&!|(I4u#9?kL%{FxaB;4JahKES{(N(e zV{WJ0<^KH4upN=!{;M>*d=xXd8VVkH^_sHmLb6Zm$c- zd%pQ9Pzu=s z9$?4FPd`mO9{(1aJ74Yg1l=A#mPw!#ce?}bU{Ib~Z*w}m!GKQ&IqY)2r94zHY`FTVwaSJ-t)Opcl$gczlLeX zX7vQTz7Pgf%5L?0y+DyaK;LR|c!D3m$eS=2^Z0xqZ?R<47;^YMKflS~#$xok-R}|T ztK1X__yX<@o}@OUx9Hk=LmT{K3?#SAdz-DJwe{dhA?9DgxtZgKDa|ONM zzo%g_8T5JofLzaKy4~t^`@S7q^Ss^e3J3lLgP5YIB^2;}1;b~>totnz3Iz$VRC=2$ zpx+z#g$7a8dY3EY1w^zWVs1|$^5^4) zYzl)ZSJ)2(gZOhbE>|cJ35%K+Q;|R@=nD}Y3m`#oVOz|v0`^@Kxy@PLOyjya-%a2Ql_V@{?5p#TtQ`lHvnpMgP;ff%pN z?G1#1JaYw&*&hr9qrel(mc{*y0v?B&b*z@SFA#+8Mul@eU(^qT2m$QOo>(vf^&R-i z4f#MY444W1lfxl*FaQypn6)f!UpNq$Ng9pv=RST=Ure}fwZy`~Fm$H0m~jSTL76b>hUXIgT(vh84#ogbccsM_34YH}3P)nn_*9KK z{1%VFz{18YyE76C#wWoyKASrn3df>T+cKX!6b{BE6ZY1q+ZPT%V$llTuV8L5%mEG?TM&h#sptZ*pjs|IC zA$m5wAr}n9lF+&8K-H5-h7+@yZiOKoh{t0g(>JZv<&TEr(eWEgJ{XCF!Qe$4mEGqL zM8h#K<3b}B4n+gR9Ieyp3Ps~^`CO|o>y3vK5U^sZ-kA0WqscH*aMSNd#A9*D_-QTb z_eWz0@bo_4><>f|QN@I5+vf|VVkl^GaNF;V#UgN-w0JEN3Wnk+Nqm~t<_(5|@o?ye z#^&=!q7anW(Xbp2L?h4$3Y;n94Mw8y@p5=Ik&8s5FywN0EtrT0;|RD6Wi94^VkMLD z6s5X1lng@?N%;JEwj|;ZxYV9BlnO*+P#9dN(j15-BT)%*NPqe>7>UBhFH5ClG!c!> z;fXfxU^o#%Nny!LHGe1`LqHOf1m$=r6@~RHVyLabSR#=`E`?MxpVae^ zl9a@9=*+=HB8iA2=BO>6R3@C7$%hqM$!Ii;TB+o7ZQ*Dl2~#L$%iOVOG67wwgj8zY zL?#LOm+~hiGpS5GfW&Z1Ly2@Ul8~#0)q~k+G7R(o<}#mbAf){N<1*ujh<}bPkV!=l zQn)U)-4~3c(y^ojwId%+#NsH#z?GwrN=D-_xdw4H6pkkY!VI(QJGJX-?%^coyuY6^lr&~G?hYbGA*lxQZ|yr#xs54(ReT(N1)Kzt>t7o znZm|1pnKX-ESW;1HLi+%e>9dtDs;rEwQw|+MJ=+2wuWpbn@ObC3U|s(Ihn-i+$##5 zTsoRW=UO@>+XOeb+z1cgX7l_(_BYdqNil8&a5 zsjSC`w7#FV(>RrS$p9+m(n$iYX_GHahvS*dfu+Ts%;s_#f@Zb4Yb;cX=|Z-x2V=?P zGP#@vd$MayB{La9Mj&#T^A@v7%-(7BH^V>g$~aazzTE^Kja%$CG(d0Ac(}=Q71CfzVe|FO>@AQi=RM*~(;# znS7p9oY3)POO1VJNx6_;xN(%@<1B--XaT zTFMv81k%{zxBwRO_{%U3rBtaGe)1*m!9}T1&y*+yQkZqBRwx$9gyWRZELP0ts<_e| zQa4j86^rI8A9}g~Dl!|!k_{F3bD-{YQ{ZZ)G@2Tesq*D3$Xjm(jeyS9Kz|%aGDO4*qN#{nn z02IrGqxsRJ9x9hAG*U6ZxmC~SDy0j>KcLl4=4&Jq<;|p*Xp~EoVy(#67im_CwQDf8 zI$h5ft8@!V*kDpBmW$L&5T>-ztQLNXc44q(rP8WaiN%F#=FZ@&v=g)n8s8BCg$W~u^b*WUT zH>_dc>qMhot=FrPZ=pe_TdL6Nuqx?fy_l=fsNja5XrWeb(C(0)>U0IDR_GV$FVQ?! zYLpnqikIME05ocJ`?dLS*eX>jG_&JAD%Ai>RYv`OdX;RI8V!17Nul4Y`1xv?24+`j z_KU?jxeo2nb+ih#daW+`793Zar4o&@Fun0dfO?5+6N#%8+xIZky7*(y(9bpNB~!VF zty-)!O0~zL4$qoTm#VZf+0~?7=~f!#hJ~kaT5L2c3{#c2#uow0WpWiX=+?`$E0qSr zX~So!)~gNrWnLvX?B}ax3aQ$|QEXMp6~+y2OY2ML%T=2F>geZ-)!&zwai?;l(`r;1 zwD~=*FP&;M8CQf|YPVcyH3O=g#In)p)M^b2OX0s;)~fVZoynT|F65zIsn-xRW#vY< z(jwnwRZ_`*x!9ywm$_b>?e!W?kxb-9*RmbjEhM*=%$=O-$!TFJEc2o19i&daclIws}na{IGhmUF~$b z_WL+%J=d!<`BRF2o_4j~;WktG`1)N@ro*$Z*4G-NZoAX!dOh^5^>VGt^?aR;cExVB z(fKq999P3mv&U#KZYz3)ZmrYiIBcn^jc%vQd}{r}3!_e})pcvShsT|Mr`PK?3LW;z zR;%9Ou+nz}!%nBw?>Ma`+Ra9<*)d`((S(W^tb< z7QIrd-|K$)G{wVlz1`yYbtks5LciYTIa3N=-FmUn=yRI&5zSG%)$DRv3(lKvv)1ap z|BYP6wBP8qJYK)ldA`k%}(#FaXB8$@7#+e_yl+ihrM=}*Yh;|yE@}mzr|(mlr^Tk zPJ0M+lNirm27`gX$353<_p99wkJCJIJS@hYw%|X#B@^y9JFS6BvKiAzv^Uo2|B61CCqkOB3o08^4?$Kfz@^ZuEOyU{S9&H0-x~J)WOxWZbWJyMho= z^P)Mgx4LW(x6$#im~{Kx;UUAU-LLff>|pyIv;Y1u?(ziRAMRSQ+ii;izgFwUy4vb< ze|trbNx$9gbH}`Iq4i+;)Af>~weue?^LL&!jRw7_>#XzptlU3xnbWKlQ6dP2t#fR$ z9+~tg*GL{jnz&+1Qptm&npUXGm~7p0SVheA2u;UDkd+9Wp1oq-RCcp=YuH_+!B7j6p14gi@N4Kth`eibzQ&zn`54Z4+yv-GwaoDeM1<7=f zaDEC}C*N6LOVc|4dfK}KXhf|$ANt~Vb+8IH)WWTKh&@VrUSAoJu^Z)^fL_zb{(L06 ztA$C(W`9T7)$w=z;oy1ElAnIC#p~9QfA!#(tdiQs{ICbJhuh8V?wR38A1BYySR137 zFWimo4pxiM2Q2QCh31(_S~rcKnbo_af9{W+k>;rs_~BVR9^LFsX#C>&a0u{_R$#NU zy~*TZGuwZ=j7dz2I9bgZ*ep>X=OAGjji#%}pQ!umG!}1X4%b*m{!2C9X+c8MBB0AI z5?>~HMgcSr|8B!&zZv)U>e~_r^C*p9%&vZj&3SkQ+;f4xn%ed&5h`BPbl`!7sGNFo_r4(CG`@`e0rg{!j zVAeD|o=%~+ep>9A;`W}UjDHhhN|nowF1{N?3{ZO)yxg}alwhwK_hdlf9}j%pD{14+ zECZ@WKxw9jUlRPQ!49+fHFMqBAq}_rer(deE+Cjv_|8QIy_Zb$Zlm?F|IC}M7OkA# z_vu3+<}aD#Mpip7WB;g@rA~gy8JWJWB0k58_^QS{NCwR2LdVxf0=|NN zeQ~^6Sl^!lSp3vz`04=@h6Jq<@JXly4Hj?s1%c3C$~xCQ@_l%b#7dvK)wZzP%; z&l`zNv7c-1xp4@Cx{rgLS_?Xxud4<`t+X(m_b%1MwMHBlw&{M&dz!3V+{+SYMSr%5 z`1J+e+(~!|iY+CitqB@gS!$jo?Q?>1V=LBq;AfHu_nXQx0mAlWgO%yMeG=<@A$yJH*?UmckwbA8Ye&s- znv46nWl>K&^jU8M?`0~Badf7)U_+h!WAcN#y{+~-2D|(=E^PEJ*l3%C6%A90wetNj z08x(oYv!>G0QoIyWNX=Y?M)icLeLVcKE511g8=^zYYCe3OvUq++asu#CMRV^<>B{= zxg1n49_vCDkug`qmck=VDB%ES;vJ%z_{GM7=it7|vpsd*9D94d+?V_9YHP0BG@)Cj z)aH$1I4_bvqS)7ABEe0j*VmR>Xhg&^Yf$M;n~{dicQaI04X5;<=PtBD9&K8h0ZER` z>UkZg?s?_fO|PcYqiu`}@D)F_#@N6q%$)CR9Jv_(R_JCd88K*VucyE`!Ved~Zq`{L zNPB9j4zek*l_3m87w`sG6O@ET?*4@b?DKh3i2d?r-ojzSkX2f5im$2{mIy_ey?ch1 zz&q}xw~l+J?b7!}vRQr_9PWe;++tk8>A_BT=fo=d`?p{?r`vuetrY`%t~}VJj86^G z8KfiQMLz|F0>$O1Twt(<+5K;v(0cAxzp&oP!1Qza!;6Dth+Hbv((xam?G5F76z#n_ zYFKuQY=YWF1*(qh@v9wnK26xJ(vCRJ0QQ&fZhb~PUjPKxc9ZTU_#M*1A~Df;?}FWc zz;x2f(R}d$FBm3eqs+UzHc%e(%xoTWc?b+Usc^?$sKkVDHGdM|x6L1Sa_ju4il&@bkiBRBIxUZ2way*4=*|r=31m-AuaMP zK`}>3^FfdBJcD?mh_zn5_`sGSZV3b%SiVb(PYWNiv=7t-<5*e*`v~VGPF?;Ht&H)R z9tS|$9aywyv>ayDL&90}%rJ#yAL<4?3`&>=8FUk92DOS#CIX)*qUY2f|HEWb z<~J-_LMG&vb}+?YOqf9GucU==)Zi7uUZf1RSo&~rNIwHb_#maa1wL|m6cJ*~MpzX- z<1d*_cSWoPKWM93!-Z$eG5(_OrH<&Lz_F@O9s^Ihu0A}q8?i0Yf3=)mTOMXV9lU9# z#nnPg9+}bAvx-F~UWLbbuPbI*px|Wr8Kw$#>`3*tymUCi%$G%7UmdR|ZZ|N1559ql z4^7XQQSht$U@id%x$hSzF$I!lc%1OPGTZ785K%Il@|Sd=>;r->yHc+DAH!oiEmR?9 zScGy)b7`#%QJ4K_tLkRFWThMz^}l!x!WaEj?j6soE-bUc7ldXN+KNx-s70&P_ybJ6 zIbXzd=3Do7EC@7R?-~$$C9ePeol+urWYr-h(pUPx1QXISET4j^-F-0|| z9w6CJhA#|dplLikvaXQ8)65v;WKkDRP<(P!hv%_VCLy`Tc9}n+f~Xob%6+*J+@^)0 zWyQsrh)bXCj}u!82Q}0@=&>zABbE<@cBhYy!pRFkv0{-rpZDQ25)PN+j^z3|7y&l8 zmySq3jCR*6R2uUjqF7Lj`UUxgWSRP(?C3X&IED^=?4?BS|AR!V>??~3APB2$4p=N9 zp{KfZiBhE5i-H1ou5PS-=;ldg;nI=2C~H_ckX^o*KXvCRxpa9AwUeI>QI2ZEK$T4c zqo%L}iByzjO*94GDp>%a!w^7GH5cqnk#qQWXC8d;8T1`*r6ndZDP?7vX#Wk}uE@s9 zKs`U|3dJlW&z%1nyv}_d>df45W})}rr1-qq?wC$Xhk#T;{di0H3g^i3CeMPN>M7@b z;V3lV@aIQg`n1b9SN-v@aL;4k2IE77fR^5xhVkCoXgoNj60s}SEx z2bZihOE_rS_p}wzDB9#CHnrG?OW0i=iL+>Y`5@->FluJOCB!&O7na5#?|N1SF;2ez z+y^yG&(dQe`!WB+buR6BhbKevA>R}l&6kN%-T$Nt)!N*jcGN{?D_CjEX@{@R%`anG z^_!CPxLx|b~*tW*R%P3SH0IZUlUoqvmL8a;*Z8S zfUYK*bTe&_30Cv$MjT-02TJ{4lW;6;0xJyMmC+^H1>IvJ*eqNERBM#`f%04UMCIfw z7tumb%0SJBBMUHl>z0REcE2G;!WEb`X}SYx|67D}cX zAN59;YsjpwW^7m$rTtNTHkGzbo>1(aDOPttTKFmO!nj9N2|1tgD}44$itI<3r}5QR zlTfMWqob#wy>=F1G>(P$M%M%Ek1M02MO-IY3GK2`*-R}o*{_#1B6C?jCGl8m5z{o{ z*yNH^DZ`UnN72kV|-6qZ{~BIL%NAFF>XUiGs-SFgF7*%1^=T{<@;9X&cz!$ zOGO)xuJVg{f9$*gSwfaUz1JBL6BLnUQ5KnJy8t>i+w(HeqeYp*p*u(V&`b{fG|*RB z9D~ZY`7$`o<$LS^fUiUnK#NeGX^KZZG=tbVFrve^BG+u~sQ{0~{ttD-p)E9r;hK+6 zJa{q;Px0&01ytz<(_EYue^1W9L$tMY8kct}g64AGY{~1Q4}QT8j)qBE4jzR0cbUa{ z9dcs9d~MZ6SwJ_hxzE=mFcwvJ^H5Tj5(ouzENx5m5j4o_VapTN`%5HBeeaJyGLsdl z)R9K6{*VyIlQ1S}_%o>}zTy}f#?!ejl2bexP!RTeQ*`&!B~BG=&Q*M+JAZk}8 z$sGs~qx!jYs4EVuw1AD=??%KwaJF%*XXR!!wr=%Ct(m8yP1OHn5_!#un>~ zU7h1;wC={J~;pEnKXQ+9Cz6>8w6Vb`&El{;M&X#yHzuPZG4$b{J%{a~NYc zjgM7&va!VkhsB~Aam1!;Mv!UaX*~`|RzH8B`o#$ZmzJn_1K8TGCcw5dM{R7kJ4~{G?YUBLUE3va!0kXi6yzyLTa81OA0MK+tK7)V zTHgrEp4{L_X3Bepgb833-WJAn14c5)?z~ooq{4&}KB+ayirS`K zF*|1MNf&nXB6JCmnQJcUkRe^Z2HQ_&NK0n3>aKshSHrbJZ4;u6mo~uTxd4p2Bw2sJ z5d!Re70QUa4e1W_lw=LaB)x_!!ri8(M9@}2X@B`Zz8*n~`- zM0*5*-(i7GG*Fov*K*ZJ=_qkU?Ki8fI}*R~YXYs+S;0ixoW2=#qEsW?!CRjmwAo4C zvFmaz9eQg^Zoy_l!%Vl86!+?vkv6@-5nY$O$F^%6vGYOBev_^Dbg`M_sT_ZHM7+y} zAR36Rr@nDA&=0hLHF_jFP&;BL9w^ivrXbN+8&Wb6P0S=Fo+=?C=UE(TC({`W6JuaPg9O@Z+S+- z`m*4-JsqUDdTlJnXCr#Y{yPc5V)Kw}hikT|J-h9t;!QO+>4U?lUmC~GueDHtbD(4tClz2!YyyO0U zX*j43-KGq6z!aN;M0TCOLZ@XFl#h-DFBD{RMLQ_U3#9K?bSGcAY48T2SY~Pr3(>q0 zci?(}J1ROjX_8CDrYyVkQ8Yv(o$}RD~%FW3feRx2c1CjM~{YP!R7Qjr<%yigdEK zMvPP=CQ{I~S5XQ1(wRTMU+IG*&xy!z6D)712-G?8o`6?_C5BeZea$+$b|Qh+J6=}V zUgzY3BYK`6eki{!0x_y4Fm@>v|4QokT~!XohNqo;&m&$;4BFk6b-)o{{}RgUyvPV# zn`5H={B1OF31RlW3=-;utCIl4myHx8H2#?vMYJOuFzarrmg<1NE(@`NV}Z>_!2ihG z?++QOTiQkvUL>_s(4eC|8YFhu!a!#{1?Bj`oGN|OAXvjoS8n*&pokK4!0FlD(5<2_ zjJx)R0E`xm;WI+|&r6nS;!PgaMnqdK7P_93?;ER}hn59dcX;F3b!WWAN9KM-_92ar zTr*5u*{53h=+;ewf7|7OmE<4(ZU*VU@;5n=xUp($QQ4GLXFs;yVV5c#2uaO$YQsK* z{6ciqy>9PGZF%heE6&C#KX3AK3Q|5@c&o}6pW%@Me`nu*UH1((%_{#7M7Wd|J!xdW z;v3lTPY_{rbv}m$OPuYADbr;JD&>EDHGU?;9#OEQ>v+3QC2qm43TncUHKb+i60R{P zgd%~kuhoYj)kDS6Pc)K!ov+MKE7|nN=-CMMSL@j%262@X0O@>ED;FD1GccD~v+h(q zdlT}_-1oj7y`5X+`#yEu{#cjr=i$9SoX-4%X;9kh*JbFz<$IGOG$S`wqj&rU&i?YT zjY+uh1dW*7ZWznLWAS+VOJ+2(fAk)M%fDGakL%CFbsm=o!LQFx;&-Oo=Rep2 z<{NLD^*k+@2*nrIpBusoJ?DOzpXQ5$#Wpbg1;>>8eh&}O3P5k8{hbuUZjb0c8Q?@$}G%wLJXX7jFn z(PX^$6Ti7D#4oQ-=BS&Z>?k~GKIquSN^v7}(Yts#AxY*vu)mMJ1YeB~t&~DU>u086 zUyRRTW4tZvy=6F65nOOza5FgL#I!QsyEUY~UzJZ7Hia8B&%)++);#q21dQ98BmH9V zdqqrsb;DvW)#J})a<;SjRs4CH07@Otha)?!AQpStfy@*@KRE)wN^QFtk(VFsM16b{ z4|Jq7IGIZNGj^Islz~M+7)QBDJr|ir00OICW*^7oXh1SYPA{vn;ja|oSc!##hc>o@ z7JW88^4h_Y*{_UTz2w+66&5x&>!jE*XA?*E`x!PpqOKDM9Htq~%>*uMbe#-$AH!dm z60SC{)e|O@Ik_o1nbD{YwBB=-OnWT*4z!0`5l!u*w2+MgvfS_$!-wU_dc6FMrV1O7 zy36}j2`VPa@yPrh5=-HR z&e7<2i$39C?I=UMiIk-bbfdGNJVflu`~wcyk(&o?=}_(kHNoa@PhZUDMD-*eVz8Rk z_oW2(pp9V{T8O~yd5EpXYDf&EhBi*C3Q$RY{Ix+scj7?1Tef#d(2N3t`7{}TK-O=o zPH^svRCM!GXGi9*|Q`&>b zn1IZ4ybb%<;mam|=Cx3p(fQOhh0>nhnrNW!;-INFBzdsMaxo>aq+NJZ)GV|;F`#2o zp_%4py|nrO#1(nqINF%!nY0-C@Q4(L#SgxsrO62T$VsMK`wy*E2wV(n%0=uHP730R z_hk-YdhiPu4&$+}y$f%{V#vng__;nK;jY>NxTMX#0DF-*Z#}Tc!Cn;P0A?DSxz#N? z(m2D2*Cp>o6ZPLMzT66+-usL@q0J)G;BpF4hPORkH~3qZn-&g6g{@sk+jmM)yJHO$ z`A{DhK8)i?Yh_T9>)teCr_|Kh)3vLy<9O>Su$=SZ?lW7uP;fafgl{2`Nw}@`0d`W@GF4F76gZhuim7FBPxWZzF4g|jfjlTgjskd#C zU|dc5C>8tLepaA)2rwRBUvu$5opCd8opmU;8URhaxO}hUf$Tc^?!oD@5|5b#ton5h=)%bc)a>ED;`l+#^cyUwno- z31-?%vGV<;+)Tj-njPi`E=A%byC>oZq=wl|EIf+8*4=#}WsZO8D&0ex+m5oAkKwth`fGrqW`lI$dykTB z%Z(2R6N`exBk}9)M!TXgMTtDXwffV)$V}$ts^P;5aEN70&C5Tpnwjp8KzDeYj9i$) z8#FbXJ}BPyAoyL#$GteTV>#TQ>uD4va{gz~AdJM{i+jGWZb(O}Nnhj7>)n)cllt)>!?TQ<0y!iRn-a>Dwj6m~w*CD)bwTwi( zOAqr={gb6A{8}z@vCcX+p<5L7xnJAEB)&@G)(69zM=fsx)aFFTb#BrQh^(`uQ0VTb@VDDbZfq= z^Dw{WjhGu&yi(Uk+VwK?j4YR#cxnDgc0oB#0S?@20U4Nch*FCNy9%FyQbJD;Azx$w zkIN|ZWr#`Y0L3l6Bq+YqYxByqZ`__<0?5cYr*i(cH0n|Qx=?`pYNAyK5F0(Z)qom$ zD-r!9#G^w*RR7$eMwWqV;Gj-7)^E{=*zxKV=F=R!kQ5DgY!cRiBX6Al_~ONKFMJSvFW(G0dN^L-c;{ zBX!+v`T{O_ueSIfxNi~-`tn*%=l9X^e}ensr`g+}9^;RS2#Z)ZHN)zyuc2n23@!3} zL3i(p=_*mDYEz5Lo)^nTA*bGvpu4x4WDLPtfKdOqBN39!7VTV1NX8O^(Mk{3R?*J&i(hZw^}VD!pPruH#r5-VhEHE!XNcWn zpvDj_mh4|<=HSOIjt(QfW?oQqTp*cK)*91oTdM;2%$BF)?H~bW*$-|@a~PXnynG~d zdN19v?YeYGX*U2#2mM^&pyW;>pNO^NXp6L8CkfureLx*s>|vpd9QBIL%5L^j4>6lTBg?%F}X87v^^I+9GF{+{th2h`azuZqpg zfH=e^3jI1wu{1Da{&o7hWy_El6l^n7<5$R3<~KTg8Z&Ui5kb%{TQ2GKj1UI7?NFTP z-MZKVqHHQ?w|T7^NJqGMED|q-egl@-C39>83@<+rDJMf*v8+oIY`RX!8`T*tl*Nvj z5e%3nQnH}!24*;&{U(9f#ao05(W0_MXp*v%-2~#(VV)3MWDCMmvP| zAM>_C1vg=`Vid)+K&wx5vRyVvQ4*N8SO^%vVZ)W|5qq8(CB{i=G;6Jq9)d`WyFWkz zN?xT&#wISHpfohGieSY;3Zj!TJq(~rb){$vez$@YwfpgihAfaK`HY0k)po;ytC9yJ zZ!brT@1HaDYi8gvdkqA#%f5I@aZ^aDqeh|ZNuh^XX_Bx6nanZ&yd(4G0+zX2SIn~i z&1mB)s6R*!A$x~j74;KgfQ?z%(1>@Q7L(e*sr9H3Argr|YD1J}TPMiU9%#VQoQPJL zc8O$93(%|tV!vj%XO7d)hix<@N#i}h~+O7I4@O!t)RvXh5bX)6T&*8l}m8YEs`KL$;*A zJ4yIs_d=Z1%b8`~s>7dAQD)5I_T>VwEVp*Z!)rf*yo(LhL`?-N?!v}R7;|l;KAqr; z#drAWqGAx4C8su+TPU>%MEE$oxl)f`9pOstn!)Gx3G0jCbZnwU>whvL(~6bL*W!sO zF;kGsn^j_OX5%5L#SjL`O;%18+=(HcsrJ?jYgp_hK)u=FrS5O9GE%&f1gV)aFa4Ax zFL^tTRtHZFSDXA}S`VkX`1QPPgXu^Cd3x?*jLXbKEQI9C0qG+C#j#;aiZ2lNXM0jA z!;oP_!YGwvy^2NIQ?jhI9hgV{mej4NY<9f7{JnmDsF`*}}AV(!)i(9DL}QcyoaMGpi%MAmnQrka$z0AiLz#CUME>Y=Of9Zs*T!afkL= z8q+`T3HN*;L2hb<#S-R(il16nK8Dk?W0m4M^LkvL)#12mc(=o(P!YV7znJ~bA;D8> z2tZ>4@58@Z^hFF$!8v*>vfO%(Rg(ZCj5}Ktk|ID}$My6I<%gW3udE#eTE#{Sru ziWqS?#ah1+;HAI`wk}h0Jqe${A28serYD>-m;2O-R&l)+Y0880b>3D)^mCs#M6jhB zKkwyP&QH9DaQ67%t$y0V51zUh#BRT$ox)=6Z&8{(BNL_^DQ2GhyFmfvIJIKy8yZkGyd|lIQcu&nH=RO=A&buTJt=0N97LCg)^XlU){ntv%#W zymA7m=rYA8eh$b=q529>R}2aec{vxK2#(#n&fE}E?u!5IkgNE!oY*1mfxmigQ&X^5 zkIR)3WhYy7G)wX8x1g(b1w5bbVs-~4aB!yY)i2gWb?ogZJ4*Oi!8u@suu3k~K@K2s zQNpPcDz~axrTZE@*#?F2aq+)vHe+ZX70hM`{Kmzowzv3llA35!{n~k?YnWVHrhi9= zxlWZbNQ+XP!XtPoF4yacnA-2?O5p`4U5_j~xXU6_jyq~*RZvR&#Mwy@TN}jx_66B7 zt8a-<83iJ@4g3IgOY$B5wLY7imJ`)Gxk~wBrC%4TMyrg(M#()U@MB}jXQMDb=8CgP zr$GbI6=uzL5Zr(n?FFdbQzcz5$10b^q_p}w)vAnJd(54zyK-4g(V9%l_Sa3urk)2O zn0;Q}4X45Unwn<&ak4h{^43ZF6Hu}cVPEE&oJjeRj-zY6Q5);RC*Jm#1A$I45`%O_ z+zkwL>Was(xP9shb!teSNJh-6gARL5_AJZi8S;%jv33JO=3=-XH8|4Z5&?UMVFr*~ zhp^PVIZNrPzLG8?>$UZ+^jt0_u-~Q1gpnJ2%F) z4^2<9WlDK1lr9NOI$Aw{&4Q#F5gLWF?~1wWQO|x(k$caVgOD$7*C4qbZ!D>l4NX-( zSr;!RLrETO7KLMdFJqBS4Zg3aX$QT1yJ@MQ+NtEXm7xTYA7?Y|?Brjj!AH+j{?25# z%b4kLLr*yDt$X%kU(33l&&jzp=Y8zE5-8U)6CPjZapaE_opZ8JKHoJ?^|J+jE;RP# zH5Na0yYERV-Z_!U!5q93NdfxnsEIWOsR12=^3Hv@`@fEn9 zxbzv%k>wcrr0jecJHu~P3W?w&O8HFd&Rek~C!PiU;yuzty_TNXR0WEOIbFIf?moSp z@GL2?`2}jL;)DJF51>mZW8*kL!hv@;f~CWIH;A76?l^$KLdP^n=-kcpn>3ov@i&x( zO~VjU@_WM|v4CvD0J2Jq<1o23Z{sLgwhi+TNpkh=Z+M=`+aY8H-n$_@3B1N(%Jdwj zacqs8yI~9oyoPZ!jTpyqLi?P%aSD4preV6aYNt`UD!9gRVhbDQ-=xVtjpJDUISr!> zeQ?f`*x}uGBP4CIjw84RK8|Aq&boIa%-J+hoH$McWnQJWbO@*EB_UEPFRWo&DZ0&5@7CGRajr*^TquP<=1xu^q}|;(_Hs-53{0(vG=pAD{w$S`QvU7{{iR* z0HYTGqmPyyWr6XcL!pBK0TD)jGo}9ZK}9t+G<0x$yoGaySH5{rfpUO{d3r_#f(hgF z>Z0M~;`XKm>j(>hcxeC{et-Ya?Fejc>Wh!Ba*haU>VpNc4p%c2{B8yv2>k>EJq;9n z^L8^E@tc4QgARrcf&Sge|MUdh3?%dQZvrw5Fo=JG^&#?g0zY}}D0W7p1a}5hrDK{_ zycVy;vovl1R8ce3%*(+6cU1__vK=oxXIA_?+0cdkQRfv~WpNdXZB_NzZSWjJx&{hM zDvaB+K)qYsQqYfTpN*vzMT3e(uCX5MTt)v&R?wvs+18UI3H4^=k=m^ky_o$L!!*CK zDSCe!Q(Xg{PGWPy*hFOI3aW|LVmh2&Ih8XeF)^8FMK}=oiB1ZW>(JlLB#01)Wk}8c zCGr3SJqZ*&uYDQ?554i9G&Tas{aa(>zcgM(!#ON8t~m!U3UJKV$TaW)?f|$AGY|Wj z&%_vS3iPc^751^~tn75Y9tSj!U~zh|?HByPeVD3TNo;psU(0cgv>5bI<72%tF0U(w zd=1`cj%roU3#cKYj-1d7lBQv|n^)OU8ZW*dV}NP(kXCEamg7B`#?HNaowipF+TTuW zJ9q%I{jF-YD)Py|S9nJXwrgJULxxtfUvDtP^(1|HYkt-8z83fzZ5Qlpa+4X z=d@%+LgS%7K>ve{l|X9$miPeuEwQ3dfkrwb;bu`%3CL{!5VFbr#mnmP@Obp~vKq-{ z>xI|0xiPX|#w)p%=tp2-Bt*xm&K`gyPOlq2M8ut&ukqEBipoz3YTuhxmk}ljj)J0R z0v!|^)$k0>w8=Qm`BzeZp0UQjjQZ4XA=R|KOH4u?6{7NBaCRDbtPDPT(r#EEpWrVr zCSMJaQf)|fHD{og0-giL_#qE9fJ(h)G(3=4DynAXe@WW}L5~7O?|n>o`Yz6%^iS^9 z0xA7lTGGGVOPbVES;Ko|sTOyTF)_Rc;WmDxzvFlh>o>OXFkzJF1cz)(+Bir-lu=WRmJ)}HkSAEH+JveXe8dTu}uI492YB125 z@WLJkTZawyy$#55Ku1vNoVcKXNIro_f7vfkX}Ve39xRhAjt3geBFPn&g_G?iN2X=Yc@hj^dA-B@6{OGx z1^s!kP(CC%NEuV~7tC)|S8&eR3hO!=PROlv2yVV^j{`=AdWatR$h4}R4AW97pPPB1 z2OiY%HJXuDfP4Iv{f6dzDLWUkS%gGz=|?l6ooFD?oo>;t`lbKsu)awucoBCY7toBh zY21Hea}@+V3>1B6W}g%fJ>s7fRs*U0+rNl^DKsB6+no(EVOee&%nZ}VCn|Bvdpm-h z$d~co6d5{@7H4HMzuS{=pOvddeo!+mdI=-}adjfzoOf>{$LlFey#95%{HMe>uVR{w zfwydznY*S*CkV1gu$+*Gv`BxLEQ8?Av05P=Pd(uWOQ1nRj2B)#KRY*VA&XE6bwD-< zIhR;oLm9cx|eP zfFwCp7p9Rga!YBNXSnk|Bb7eODm)alwY{-zkp=^PgI^^uBYhPud2vHN>*0ce`Dmw!bvN?0 zY|OLQ7?L)c3e{`p4IeNNsI{=meKledfUp<3<4f{>p$K6egf(JJ3d?g zb@OZC*B;~ZpNmmUk5%L*_nYwFgea?eWExmuEmEnQN)TQ*i{o38WK`Qzbg#%Tth$E9 z#x&DJsDO8=R(vZ+nHi8bvk4(inp`v|<&N$6WYypbssoFWI9kmpovS(Zk{!^lB6gkn z;YCZnRFFGmBF8@Yy35h`IihC z5c)%t-Z#sj{poKa5sO^r5W?|D_d9UI0(%aiH2l?V-o*Zvl+TCy>ey9GP}Y>2neId^ z^Bif~f>>@-kz~{`bU>xdQ*-72+q(bzTGAc*fx`FGcJ;SXpyVv)Sndbyd`O??ixqp3 zqI%0|)o2Y{E$3rB7u6OJqCA@{A$@yLPDN5B@QfdXloF?+ApmEy%@;-Iec)rtV4^+W zBMi;a3c9KYF>{s6Eyr5yb2Dk1e4oJjxz~yiHashr2;hOIIIXi9hLSK^@HtIM%PKdY ze*4+ayZ3(gs|!i`rTUGWOGAkK zds>V(w19x;zF_D(1?BCqRZnYn&G?a>&_)?OExYc*9x(K-0C_sy`t7e=ucZ zg5T@|zA{mQx2wxum5e#)czd>|S#Ar5a$w62eM!#RC#V)|HxceOf1RWEQsp*hculB4 zjNpN|ps}B4@K3nECcRDELQcOT`|m3_R~$-#7RdN9FX_;T(urJp>ks1?!uSF2X6+@b zt&&o~&q;Dh?n|e2o?4%KFk2BLLiwU+<&YtojW@1D<$>J|@>@X-=5G{k0iH>b)PBCR zaaZx@$Y}rA=MZLq{AgL|9}u?vp9H4ux4^L4Yc!Lx0dq`L%=Dp>nMT|v2i!_FH#WBp z9#ruQO}!w?^6frciQJCb$!&baL_$(3 z6Z?`;`>juWUcCHM$~2OFbc3N`IY(I3o_wBBicQskJYm`BC+fAkj|{V_%F{B z;(v}r;lCjUhv=C)Ed+_Oqo0!CrmyFWG~%QV!!4 z`!>MUtGzYBCBLkhf>w8Xx8nXi68#sn%Rg!7f1q9d)=r(Ym)16KjU4iF+N1M%j|3=$ z{AX8~?Ai|Ezn?Q^^>-`&Vp7Q!I2C%t%!~i{(uE#NwNTKy)SfhOGOwsVbF+MZGqat8 z#pBkXP)UK=VAFhKk&=QJBn5fH=Js=!_)}G|+naR>t9Mc0RV|A_cfMr)RI?AU9h)pr zSqeVld2SNpp|+!kR^t5#Yq8_>>g;Cw@Vtv!T1GYgdxIg0>=n4D3R<8p59+DhjLF6QtQ{_*M-qfS#tvFB3SxZ?$X% zZ74$p6H}G%6h3!LwXeZ&!bgW zIhTrjQ2R?FKxg&L+P~1+2*KNgV0`xU$W=Qx;ZM-=0M37ByoBG-^!@V!GlIzJO#1rQJKue!@wdx-jk={Y~<9 zLY9bfjJ)aQViqH~?uEoNo%RO|;T!n{W8BEurdA1}?sCWMynnJ>HzD}w>MMz>qhJM> z{s~MW;O^gGN`HfK&I;<4F`RiLY3$$;kwyPeIFPVQC-p0_rje~(QBVrf!gV~BH(5t8 zY4a7+>85ReJZr|-Q*~cs_nsWbv3jqps@<3QtXCVlysR=DR2&Ra(_JMSQxre>FXT#? z26a`6TQ%g1FsYyz26^|g#AgA`EQin7UlWQ2)F5~s$L%s9vQpO>b;5s+x8gx^*Jdtj zf3nj(mZAjQ&-A4({0Go~`H|y40s4<0IsOgEJLB6)>wbRPxcg*6HZOaYG5e4l_mEEJ z_rdK#(t<$=-L;8?<4f&yGuI~m@5vU~RIa$9jF2x{0Y{!Z8TUOLhqSCee#_!Kiwb)a z`guHWb_I4Yx4v7V%bqm*CV=JB-W%+!Vq)Ftj3!##XM#3CTSAD|oWgu}Azm^1fzV%^ zE}7eJSvDjB`8rhKZ%IY=ZdIyzBN}|Q39iq1yVKT}hZjjr{_98nKgGgODim0+28?{g z*bsTh_Wq5JPm|)lY)kFy&JPE?2aJ&;xuP;H<-p!}N3j-v@6Vnn9&epz-YKH5AHfVy zr7CbzVW>>vvOED@4VNCk&zj1q+XtMLoXR2Bgg;f`OLz$D(w&rc79rU&_O58f+3$p=TOD*UPG2)iuWzImJGF{BwOi ztUE>AX4;qF;q%%Z-Ia@)koreAlK|nq5uNrbTwqiPyLocBW&dX*K?XQVJ)t(U)+bOgf(6k-ytY24)% z-VI@-_PrbtLX+21b%wFAy=LW4oFfZf+Ldb#IXW~Gt{5uv`4BL>1F6&0Zl$;IFEVXDiO9Fhx9V7A7mW4kIe*vjb4i|YKPoID|xbW z4sH9s(0Xn=;@lNr!EG@3X42={8!vG}$7IjVyR^5#k8^)UWRB#3my9@d@ADlp2rj{c|i+gI^SO|A*VUURdv zO#o7!fW?1h|9_6S|9d6fTyFeuY(1bK9rp{SC{-pX>^*c_G2T& ztVfIKEwj9}Ugwq8nWtvWdA<5BHw<(aK1RejXcqY?pNap6^mE7M|o%DJ(M_? z`CI54gM)qdCD_mMh4POJoEG|@{SR(2-|VOA2LKone){}NS1$qhm;f9b^KpReYUqmO z50!LBK*8S;*Fi!+Xw~zO{psX9%U(q}dxhop)$^#DSOGDbjdR~w*{gREdy@LjqUoO( zARo(_Q+<^l7#kqwj+JddzG}CNAG(DQJ1?FB74S?84)umlx+tAWef%yLc1v#FXoaYR zWN^&!cZd*!1Ho6FprSV`5!8xcn*VMCIXgNy90z^=>_{-(!(KqZAx}2!4xCOYP(I+O zujwRAB?tMP-FG4}=2R}4cn7eX&b_K${@wOD27(Jn!K24}4Wz8#Bf3AJY$MS8`{BzY zIs&3FB?hRKbTHizla*uSxngm|2cv36wSmF*6})FcXnWjwa4zoHMd*spD3# z%Tb=5VTYI$wUp`OCoG)26@Pz;6=c{=tD2>Re++Dpo&Wv`;@V5E@&K`;+>z=uu%-c zQd8x2j2HWGkq@z28N@{d#n<)h&pMqbO+pQOT(~(#7?+%x-0W)R25G^^!kwV&1ZE9T za~)=Smh7u7YD5)Od{ss{N;Jo&$s7XaoZZ+ai<~uJ5wlSRjR*|@ppt6ko}QcH%u6`$ zG11sRY1M-|1CpwAEK1VHIpi6sx{)a9n1#dD;|zs9bmQ1#XY2T7{fw#eObI->feB{J z=mlw=4N*?93=TZlq%nnCnah%sXsDWBcCA4#(AZ&)qE=)$dsd4yC%Xgd+~UtD7{_4c zgRK^fNk7?3p5v%;=%G^ByC>9kwR-cazn4M^enASJx;&e`N+2&me;|5@K=yBgB3wZ+MFVZ4L$OBZ88CHWE zL`X3ZO=-tb8czRcis4fS8t6fk>q@p8cEWp#hN4r@GP~mOdOTi3plrmCL z^A`;h)8lH(x{Rl6NrU(GyyRVqdzAMi6t;nsAVME2tdUR5g7|gjxxvu7#%}O9q#<0R z)R2=mHZYKmk)qE~M8xO;UJ}ErjZx_zt~nHDL1>gW6z&r%!7FHX$X=_t6iRiKt4$F| ze;h!PI;>ibC&?m+$)AGPY7Q3`K4aWpeAa>Yac@Qx-0_mYBVr)zVs4ja9H;(e3= z_BZdnzj-I(Ayt-559}*Uv(qMX&L~%DX_BXeAo;~iign^rk)TpW9%6dC zsviBtxuWHG>s!HIN;81=aIXdsqor0K8m@|&!o|<#Dr#`&n9=TJMiz>w;^lns(IwjL z^-{1yX6Z>RAb4C)aG#5guhn~y&~%C!+AzHI0$1hA8?2BRu%0aF0#6m-Qdtd2h9>D7 z8{oBPnc_!rB5#s zxwfB85gJWM3hu+(X(|L7V>mx*s1hQWc%MD{XbVu^G;BbAXuC zxHye`69#cGGqsvVd^ka}sk&6)o}e=$Cq^zrbbKKSr+zP^G0HdD`hUYBG zHXJi*6DLzR(0hTd*NTuSDl%qhPE-oTrXPDC-u~G@=hB zHjsfAg;dQ{Y3hq$J|kXCPI>{M4~5RsF<__wy67d=k+Gx?`Dh%l#OPqT5N3sOWs0o| zPaBD{e!kP+P;XOsWl`Y9osBAQZAr9%E?_)82YF?^4|AXsrDnB&-d+IbNW-B? zq9s9WHK{w%vYJIIRW40}#iwIZ;r{K$RilqRk^AmUELMkrIl*#H-~KcPNv3a3f4b z#BgYvYFKL~M|cEON0~)g+09myxtG=>o;-m*H++Zl z0an0+3W>wmT!O*SfC;s00*K^1s7bs$Ern?{)0Up!T!JBev5woYHUg4fA^Z^U1$e6pCrb9v+7AtO=>8ETsY&#~w%V?jz*kdR{ zwJw?@71T(@a7OFbQ;n0R46d9_X*6`NnXQXXYWL-%h5@K170Yg*%;Ew79Qm1?yA9nf ztds=G5rT0`w6Zo~m=g7xu5?IBXq?Auq*wXi#G(jUW4+aq zEnjyyOrwi%<{GoS)tv2A7*r;4kd>pf#563f?m;%&LXl2p4$<3W5B)POfARS=KI2+j z<4F2&H!VIwT~&D-RytqcwHUIF3AKl~=_*_~fvBdq6i$GiPlb+eK&x6kESYZ9$rD&C zZ-JsVP3Vv_hbvcAs0WyrVw%tt^v|98-B9Gz1QL`Y;6 z+&VFzal8iEy=3{wo~wC%B%K}CM4_^_<+__+5>2s(7@r4)=c<{D*1eo)Ur=N*B_cMd zCuD0(PYONA)vy@j8e8DyT5HOTK-4E`-1DUtjjIC=Ru=~0!%e2>tm#L*)+9(uP|7|2 ze#3rgShpD7`gAFzIZ(H09)$gf)nzuP4tNwBiab z=ki^-8eb>jMsU)d9SW~q^&oy*ws$~q5-IrC;^kjgYMhj)@m>n8i5)p>J@xqk^ zLv_XKT=Ugnh%!a=z5|%fJAHt{&t%AjG#6rKB5c94A(lZ)QXMlC+MGX!C(pIvE|(yV zVsHj=XQ#6{FFIxw22xnj!bE%4D3)+s666vjW+ayUBiZ?)`f@|TxoPt&AZ=bIA8Dr~ z!w?Q}KodlDp@Fd*DQMXz)~ut4R(F=PlMQPUs->0;tu3d6$`~T~vKtZ91l|viJaMeE z5lHKs>f~>^n7BPvTLt0OX_lN-2`$aGy!f@AK{u2K-qL0sn+Us>5FKw1o2#u_G!jT- zcdIK-w?vRqW_NIyn7SB<)d;GHwYmTRg7cVn z&;Q}~zbtv%pYrp6EP31S{Jdp2kx1_9;1FmS7jA_PCCMHDQOe^gL}pDQN_cb_Tes*Uc&?_UC@onwCN@x$%xune{F-pv9r1 zOQup@63#@=OKkC}XhUK=gM*x8L8l`jif)qvl$2Tuq3p=RCNNCuCsyG68sx37dt{&* zi+C4pWz8svIedhyn+}b5iKGIgjT-<@RZ)djLud&H&PY1>?rAgZN|4|6?kxttuw@6& zVz`kfB~|!%olYWyHlUb7@LrGkGEoE|wf`zTt8WLmJN(ufcunLiCY*j#iGuAOCN5E@O& zfkxFt7U2baj;vkb>&P_c(oEW(?Sn;nZwR~!+Fx{*ATmj1Ru`%7h0lfBTOv=Af7@PLw!>*jzwZU zujW9&>e$!<4YG+Oz_n}|smdDOfiks}jO%?J=X*5B93;=F_kdt5 zDLC(>Z|SPCu=J<6ZYI$AJFb_0$2E5|6JJglk>xPii&19Ib|tt2cm&5&wKxeoIkzj@iZ$(WWEXn~XQ9fRSBi;Cw!}6UNf=mrUWBP zJGbW6mMK;F08etGFe3D^=4ve@43Tc7@))M`$n`6W#yizK{PD-T!gl$G@YS zb5eN0*rD*P-=PXAx;49RK`UILkbcn360FP_iP0vEaiCi)=wwIq69CxG);7chear1)HU_D(uP(zG{n}i0@ zlx;(zQJXS9)wt8Ojb4;yp$v&pTiNw>?v_+xYR!RJ}o)uaZ;i>POmJV69x|qdLS*!XxI+`Z56igDL;7LY_y(G+*jS_~s zRaS%$4yP#sF1I!_Vg@PiaSB$#Lk^pGIYBTCwn-}><=9V?W5yxar>@7`1bi4Ns49{x z=p$~CLTcl-=i@7u3h9vu=J$7j-^6axmFkvzHgN4ROjT<_>vZuzt&9k7F|2sFpa}(^ z9F3AYqqIehl9-^Rpa?-@cfIYDIY=*!L^utdEbLP?f^RI{)oLtd0?{{;9DuA!si)(x zoRta83#QRMQtcbHlMEumf@7n8T}qqs(knZuR_)AJx=i6W{or5s{a>ZEgFl7&e@bfy zzr)JaU(~~bRF4?JRZVcH9pRVMRcmnLJzf|^mtjaLb9lE zoxnO$E?t7BnXMh!F;+gFYHlK?$Atu&ZW`h;V$z*7o|2+r#5{OFlUMjzp~sLUh;dCX@u4HJ{Bg(SVD zsA7jwhU|=JvW2=aLf*p#jy{RVIG9pIPq;KHrV-#nPw&V>S>bVr3Ri2xlJVv$C{>bz zVd0jg-4B4uB-gIdT^`+SWL3fg9JX0j0M`|c8Evf&^N#c&!ImWH*T(4XL<@dH2QG|sM8FL7 z7GwV=v3M~gb+#=>-?}me?CO&i>@(~{ZQ{>vRtwWJO~*AjS+@C5?={aHhCkq3J%wLoKG9y2u=7apq!zH}Ng; zbo?nghXRZ|83WTx+aB2>x>nPs3DpbagcWpQ-FSc{5g&7sewrSvJ_@jlt8I)DWrx0I zPaPSb_6ROg6IAI8BxX`Kqr$Ot;+xEqMZl4gF)l3iJ#oW6=0jevsYHrc?=@yA6P^-n zsB2LnnVhi^=)MQFWsS3_GZYvvG*-!56TQ1s%I!_13aP^5%=vC|Zsgc|5+&QZSrT3k zPU=BOF)UVcNF#u>AB{!W9HRKhDit`&xT0$_;~GtIX0V!L!&thJ1|u*-hFPHy+)4R` zM5V%Fpz)HBD5XnEXRtQ9S%6<>T}LXs(RYq9v67UR1U{CrFn=1Q0=FHbDXnivmu=Qg z^l}|n;UHgfVp>#kY;rHP;Q=US{qpSpN8VAS;099gV1DGAD|3$i6X`t!xBn&`{hM@< zb*|yAM-wRhOoPeGwJBEyL`6bxq-VC^N#+POMz4k2r;*@fBjX!ucLu|4;PjXzBOeUh zdshxtg;0hdi&PEj&}c))d+8!bA(Dsw)!p{$dcaL(97EFKl4!Oj>3DC<5zl1F+UUuw zgvZis>NX( zq&Y5ugf1{8uZ4y=fLYw^Sq`49u|8>YQjNcxx4)XkL1WESCspt2xUbc_6vrekCw~2= zi*pp#HekK59g1!l^@E~Ir|L)y`C`!<#|vZcT=+MLSqw=ngFNMv zOlykU2IJ z_e|}gLwq=GZ87MUH?7s1d_B$Z0WlP*2D@zdRDw#Lx$~T&WeFdOj-j@^Vvsz#kt;5# zuzM~#wS^F_77kJ^SWYJG8^Rp*1EhS`qzKez zBm^6zFcbu}k$C6;0>4^qJraejF3OK~HI!rKgT2;Jl}{9yu}$Nv&P0_e*DVPqdQq$U zEQxnZRQ6MtRT;&zT^|yWbA-)@C|Q>}tVwBE=_pYfWrIWw7h)XfsZz>Fa$TwkDp2`? z53Pu^YfNoQ26=d21ddV=1v1B)dc^=$ff@VcYA9obPSI+ELBc$@i2G9Yb|z(A z#p<$4gEg4>5}|^_lf+JdYxfJ~5C2e3oBUHg|4%t>@^?Nj*qvoPR6eEk&R-RUGG8sc zA>oU3bfNRp8!N+e@x*Yv<+9?GHX`ch3)OELE#oAD@F+*XKsjuso=mN-sp-_l0#=c3 zY0l;~GF3tpq%DW-Qr*P6f)tFkMyz%6<5DiEu9dlZ<=n}Z!qDKtSTz??*U>g0pQQ}m zV;0M;taQDqz^+1z2<<#QKC#E7Kj{c6<#EORI1+$GO9Z&R-^DkHD;1dEoqKw{up z1L4!NEijERL)#8 zjE%&p54M-0m_a0#?R<8v^*)Ttg_VWbw`3>e3y&nKnxY#TV`Dg3OMUZr)DFpSG{v{j zLEFGkly$Mu6v&}`tR)cS#bpJTknd!Pu*$&TIqU)#J7Of8`!iK7B&bjH|1GCo0KuiC z;Ec%W)vHYY`A-yg5~%$x>Ysm8oLE+OyuoG6c^#bB&WsgTA>tqhclrsAZjphS-DiRt z%JtEDWwlmR;}5t>+Z&~%A%G5NPxT_bd&wSr(R&EO!}LULLQl;a0kyYFtQFx@{>gEf z#`rN)CL`qzhO1P*YMIdBU5KEIKh{h2U4$mrjWm0>sbxSu%&Myc0$dwu7X>;{+Y!^n z;!(NU*!ID>+J;(J=?x|405Q1ziV^o9lMU^)eGXBRMkUHzaU&)alx2h`+gh|$G>5Sz z43RCBZYW469iS@y00=7q7Da1Fssh>|rVyhm$#oz%Q*Xx-UOnbyLQ#-0E5O1h&@r-P zPD@*sCLrme@>+trkV1M4pPlSCxgBaH`4Bo*9}p+T;bxYwIiJ{(8f8@UZssODg&$h+_YBkFpsq>ztO`3+lS+J0%^KB|&Hgjc#yznfWBrKA?t4HZj^cGmDE zpaMEV8&KY=8P_vt0a4BIIfAv0w6eexr6o^(9kF~y-fR*1LRMTi1R81OA?)QI4$O_W zpw6bZAex%v*}<(jO)15}Y1dTr#X^(DVv>k&7F0sO0Kj>Sd2;7Ja*iVcuMvSecjA6g zT~)`gu2ueT7akV@e1G@%N6D_Zrf8QyFhUsC9}tjPp@-@i^Bf!`i_`f=w1-d8eGnq-PHow6_1ZW)|6W<`k6=J?xI zEjj>`+!utT9(b9*Er%9XlKEOWG6j#OWM@-tIz`lSnsa&;=IjdB6c6&!i3LCd&wwM; zsXA|jWfFkc2Tmg#E6mQbQh>dOgAR-T%m6H85rF%}!4usr<5#`mQ{t=A1 zbXkMu{K>MA`+j+FPQP}Zp?=<7&1;kP2Tg|4S{pH3;Gg||Hy%E7@B>f*RYRXIVK=FO zva+(>JJ0y}`G5V`x^Z*o&iUi18+{#u`%cS0sU91#FS%JzeQPMJaFm&MwLtbZ9@Z4y zWvuBa*8DsrE|puo=t%y+{83E~oGl?}G5XN*?U&cZVd@5lW;SJQLa(BWpMToF_W6+s za_O?|c}{^nq&HNc<-Bdx9IyG6j=9m($f3k9+)n-Lg9DZKGhuI?Z!A=P`HWvMr_SH7 z%u=l+N;;hVF8b((ssT&Z{<&9ZljqQFQ3J{r7bTbjr~O~dUgtq%nk^^ncbnOsIe09F zo7vnGJzUd$;)wM4VYYnHhl8+x;9L%=Wxn(Pk8a!MCErVQZvSb1)WEGM7VK#_*9>pg ze;rR}^O&advs$Z@f*jhGB_{|xTlIb@Y|3U^>2z39E$xF5x>>`V`!t39L9Efv&zX70 zsv*Yv>Gy6=KiupztNHosw)y_O<7b0C8y)*gZ+r`CO=>$%Nefl({v`YEN~M*w5^xW> z6n^bXOe42OKS0#`UhtgB6ZI|(Y9*SG@|x1#Y&?D(Mp``Ki}0~M-ELJsCy({H=2%IC zA7;=Nr5kiR2pf9+!r=QEZlTvZ&-f2?y;hUF)H zJE^7!Gp>W)eRZjQ6edP`?>y*XmeQA~4^MU~x;0vG0Ur=h{#YN|Yi7&cPoIid+*=dG_GmW*2`#OfwTn0OB#--3y< zoK~H#3Gtu9!j4*TFEkaXio9sQzYF&(c*W9~g4K6;75H%Nn)~Mz4ZT;S`uRhkS3jcE zzrWI>#At0uF5A5zOc8o(r)%T2Lw5kF>9f2x*I2Le_U8I?^jym5W_Vl)P_rPLpVIDz zpWf>SDZ-b2(V23EH@L)i{N}DPAt77)=@zr8YOnYUma~G!&xwTmsT;nIOBx{9p&5S} z7dy34j;AyfZynfrK-oa9g7={&k+}u(4o_os3cwGJUK94e%_&KNUs2zEQ^$8TcO!N6;QLX-8tjz0& za`%iY9)H>L83Id0((}uHbZBRLL01VmA56j8DoR`EA$zt?*=h=yy=a_~ zH@^51%Wata)@{>Sja-Vy3Ti_Sx8>V5li@}C+&k5a89(Ft_zG2dUEDZElO$+DbsJ3AN0*W8gw}8U{D7_`O2JH?vk?dZE<9;SI;MZU41YI z%SONuY>NwJY_Mit%&tK;U{y0%~Y@6PA{X7FFbGBOnJC@hDu*E zJf?3_E$JPh^iL&aN?J!8i9(_T!oOx+KILB$86C(2Yrnw!CujMN8`rdyb))l(Fy_l81MQ5D zzI1ztag_Ut0rc}FJ`qZkDPmZ>@=UIc^>m6Ik*G5bbT#>>Wf3S8&qo-Hs%ekw_B8tM zG;frdgL!4XMNzjqb%FQUlBNW0U|zwO?nmmeQMvLj(~H3wfeQg%ae4_q?Jwnd;qIMu zPrGRG);eW46!{egO;pPRW+8DaT0l-adi1feTkuZkt2^|b7TS5x(cXBFep;;`SfCgq zb~$conA{s9jrdkPutUOYo^&Vput*2}oo~ebj{TZ{tva#M4;_}3%+iQ&Sr4j`ZpSwn zMF~mVijjW@&LQ~P>Bb2(p$vGnM|{8`c&Ca(=CTvxnPt#6t*A3{xFmPeT)mY&tJyob z&QMf!an6T>C17t>f_x;##bb1ph@M&9Qnc@#1QCzD>P0AeC-SYNcMf;Q0oEP2AjCrO zQbPCyY`S8|_Ib?9&e*19ADfp>M*2r)fTYRTG~#h49yV` z_%bYYN%Z#9@av3;9y{%+FCSSOqzp4EiZj_vtN}N^Nt9MTkzSk#@_8ng^Z?UQKI!J| z^z}KvyjUWq+E@k$d~vfv$JZjGe%vVxT_Mmh+hSu@uc-{f!j0JC zW~C|JUM3Ffc6!yGn22b}&5)WjP?=aMwTQ(JKin>`ddOnewKwX?&A;6_;3{~C_511s zcr&O=im9Tk^Qwm9mO!zObs4~jH7Eb$zt{}?9bRg!Tf33xU$eb+5EA&@Muc6~+_S_` zZv!B*TH>#IaQWdNfxaCh&lL5zH|mBZyK8&ULEL09D-=fOQFo`f^ng$16`4fFal@z4 zCwD$PkY#^V9VEPJqQ66?yd`9L(ech|TUaedG^u!H=cZk!R!anChIj#|WRos*RPxB_ z6d=8xl@0HFiu=|N$c(TC+F4K+lL&9*k3Fg)bTA42`Z2JrXJ$!YN^4>HIO^TwoFMU^ z(%aNo)CJCiichb{cus@_ryu!!1-d>h!gmf{ZC>!a)Y;Y4xbyV*?8hV1M(D4foh@|$ zd)7p5jNhj~k63iThf$HIt9q~F2fsA^F#OJ5=8+j{|MJm^wkX zLt))u1Hi^J&sV6A3^OO*s0>!AP_2#|%El|$tuPcRmZk|Qm+%A2Ro0;q@PL;gUFPG< z{mq_-MIT-$oJNJbTT;D6_{vQn*5uR0XI!!+dYfnVtOz&f6rvGE=b#iWe`@h@s{QF! z7dkJhx^*5~s%o&EF|3IcKf|rk*aX?0nPvCACAs?!cO+QQ(ich98{Zb<^vyx!{y zW6r*%F@(nV)MWfsnvVE!)2dc>>OInfKC`|e-QT?|7gpzWa~5sny2ZUQn9P!P{-&g)Q{Z_WlscSyWFKnXj zTf9f;Vxxa5z#K82{aii^U-v3*xcQj1ntf=Q+h8R1kVlFu-jF%>`}%A8=@ao7lf#_V z1FF}@3!cgMKRTAM+~A|mctm~8-N&=Su{ww4?YHw;(=OIuFFVd9ZvnQW(>9hvnlBBm z>lNJ@UtsR~_3%_x%zP%AJNoS%+~LA79xC%JCf`PX--o#`g%Lc~S?xC&WY$Xbj8TJP z&|o=~PUCR%py%B(9I_|s81Q1oAhGdi>1tuC#>HVV&0zG*#7MIvwVaPs{BrP|(+6`b zl>O;1-=B_jtdeb=o)ram;h!7>o@-9YxO6(UzB)N0CcNBdo4`046UGHX0zFM18*ygQH63fHEx|{|Q(x~w<)t;@EDvkGF>2`=5 z5aYQGt%I#4Hg?g^0#y%)oAvQY`j&NmtPAEZ-mU)C_>$=RY__;ZZy|*O$bH!M5fAK| zG+C0vE>&rh2Xo8d1@$Cq=^pf3_#e$X{QJ7*iS@E6Yk?y*!(kV`PsbZw=qq4_lg85QLxhl^WFrP~7JI zd$U8ZdUH2@qeCC}_i1Wp+_moMB@d3ZIO?O03vnu){pTH|cN~14Xkqk7bMD`{mCsY; za3t%%5^<}GPFeZ`>2+qzcc~o9miMH+0Yt+w`sp0ZR7lQR4SD_y0q+T-9`qcQ*Y#{1 zQ~VY)i~5iPz7cU2(}s+ZXxWVk`L+9sALDBpb|ztW##hKadjz%G#d9o5J~2f)9-FdWv- zl{6OphAAt}9zW8?X>CMRsE_Qn<$gsWl^DADxnekx(?KgeKw1cPU?9*nEjjOQvf#Ph{@-RFf z?(00>ZDe-nht%|vmp}8sQ_Mhut?0zKtAwhr&-L1$CU%FXk*xOK*5EQtyr%ZWzUN+> z-j^VCYk`xN5NRKI-gyUe7t_^aPjyFKrkto4?F>+=xQU%ns2a+U@cE~Z)~e!Eol?Fb z-F_j@3U&{1PUjcnUTnIA+e&0^p#}&Y!oa)htKa$F$bAiDG3{<6*~>mSm|`{gmLz=J z_&DAwwL&heOa_z_qApmh1I2!4KCu)j*pguM;Ommr9!+Q$HoKbHDb9hsFS?C{6EMhS z6P@0l<`LJe+au*I(9`1Z=?RZpL`x`;-k+D7+RC|(HOsJLSq1NlKTP^wDXSq0CxwQ{ zdP&gX^4p)drDFSw(hSiT&ygw8ruRivVrxb$JA?wv&%B_TpI7lr+^6*2ZI@$8SmJ>S3O#4yC-jT=gP}c`*GnU^>wj_U&Up{je$k>Ar~8~e=!dBlVw9~& z8<(=>dbW~b&S*T>mvy3%oqHD#gPJHMsq4FF7YN-ZKUeX!cD^BeU2LYwBWXDLE_t2O z;Pf@+nXix4`6Z`xTC^&DCTL%aNhvp}_Rk?QvaBm;oNCr3)Oze(#{Pv$o3290m?Kp) zU{AZGy)611r6M@b`m4O8#K1LpDnfaw(caQq%Sy90%bKpyTD)D@_qLYRGHI2^Q@un^ zvIrkJp-v9JzS-|A8O;5I(A8VtcZ3cBfQPI>7y12I)V6PyG0=Ao6RGl@ZWFy_}o zBHg`5T|D-mgDt{uRkjZg6sre}GcBaZ6Tgy5G+fs2Ct&-jSaj{L zN%gYqWtptzb&>&l@dy&1X&!74nHqw8O>f+}CNNH*uUD2bR&Ji(=oxsLAGt1k$<+TilAK|n$YUot~5;XN78)X*9@rF zdZuL8n!>uIcUNi}R7LyH53}xwJy)E2)B}0-1ZFPLgMWi4e!(iYW$MwN&eO9tI%Jyd z-*+3k!4(rZ^0uAfZW}#Fh9VCg_;3&A;=QmY@am|B4y90{LhI3s;ZAJo$WVc8Xdo4T z<$-vgHjH?%_==}PZ>LM#01`A7ZRce zYV2k4D6>1jvn7Rs#0Pn|`M6oVL;f)T?rKo$nD*Qv-bS>A%Z~wlbyu$l{KJ7kPy-SV6 z&cN#xFLdiDN)QeEb{Hw?nA22D)&9#k@1jzYZ^fAb%KU9|v$}dHNMsDs~QM#y}q9yFJ{Y(L~QuU_K=|q0!rrL{D!wnU}VIMHa zzka1O>2jhfN^e#|z6y{mv~N5|rtZ9~x)04p6I722e*?l<7ipCg z_+M7plpr?jNLOpR`JcR+z#A6tb7no5mPUQoUB|oJxJ;xRvOH&c$@3YLJcQG(HC8spg8}?!VL#y^(kj0=HTD@l43qrh00(KG8Z(fwPP@MS=H`@#HqD$q@4lw}P%M zTT1`-y`!AlceQ>d-ATbtnZ0N=?uEOqWsC$cVGV$4UtfHnql~WDw$p>9RYMh$Un{3WNdwF7q|3TL| z|5f^b;X0d>ZQI6VPEEFLdsmY+*|v7JCbzR~yCz#_&gcC4J%7aWde-&6?{zJb`B}2l zWK9z%gC$b96MUYi9fg%JpPkvCHc3+(JRYfkVAJMq1INZOJWoBv2vfS&_BZA(7JoTV z8{W@b;$hnL9hpTGMnCCN(&z&H3+`IXvjl3%gYas!&lbNoKD=>BFJWr*rB?dQr%ubO zd8L(4_-!TQKS>&a*J}fE(t@c6g>lfe)CTM1waftu5GylB>qQ?r!hrtJGV=@>Q+Nl` zItex^3|qWZVB81`ZQ!T~6~hyL>ueN@J$$m)Wg^w0f>`-=1Lh3aBnBis>H7o%Q&^YV z%s^k>Tmn;Af#jP4)?hd4t?O%8=v3PFWAMa^u-dgW*z^a6#g`UgKIblsh^ja)bcyy> z>Q%`wi(A{o%n%Oxy8tayGb}VsEy{5;-Z78$OnRHR7j?5(nu>NNQkKMv^l{#yIr(Xq z!cM#U_GZ?o5?mIJopgn6MUKZ1@OE(^*%MH)x|LN5vlt|K6(pQWSF9zOBgRTN3DYW) z9><9Nq0N}mIAx(!@YR$&w170iMkO-{>FfTbNmQi?t0BHFAENh&j52>k(qL>-v@Cto z2r70yEvZ^WbH$HvtxXH8#vyq&){T!J0vOSuIj_CH1>;D=GOt%?R*{&SdEP%^F*ULv zemkxL;uBiXaeW1{Asm@j#1zf>4HSpztVIYkY)QM{ZVe;V{}49sh*ad8uLtmsj#n`q6FtwWIL9AZ1Z!nz zY8aR(^^J$K;`ZCH_k!@`9vCi?#ldG3MWQ&8lao{6ZVyiM(Dv8Mvg#|18n7kfvEUEz z?4g5%Td~zwz6aHqr}aUZ=b zip)xOG_HpD=xJP?1U#nN&oA1m$9eLESL9jkmS*)kl$Tv+JFGioT`d~AHh!*cF#+9$ zLM}JvRFB>4OQY{CpRxJ78n*A7EWAcQYs{(YUJ&D-%dHN!>mFOe`};js*$p?i)iG~;#E?qrV5NLGrcUod$1=k=5OwScW-G0-LAA?Tf+U* zXJCCQ_wk-Z{Bb-KooE*|VVQnAxoEVcJO5zfgEhqYK!8_@$-I{{w4f*p`A5?d0e2q7 zjN3lv#CgSOC$4|k9IU~@MmaS+Oh;~4bBQ}TDi~{ZO!8aIV!bgU-doz;X*KkC;!c!< zWe@#hp{8o86XptLAZ3Y<eS!O9NT*Ao$VPftJ*zD;uhj8$6n0S)e|2`pVtBz#a*06gB53Z8%qF4&!0L@- z+QQlmr+pkaLrzMiZt>D_kU2#}NT~{+XAV_{hTaNNOz-zG+5eaZ^Vj0h_K1P~JY1vM zv_uIjQ0OIKt5Z;@hzpcT*QO}vquM#Z6~|dwv}Ht@el}F)uJvgHf1cq_xRnnS9bM~6 zpj25GDx^xsv?W{nxk2r)AUcCu3r*sD7*Q}Xs{VWI+gH)`V%Pg`#4z@LcWqBW>RwqU z?!=x>*wV!=jR|6+-sNO?nLw4{d>~_#X*ujRPy#|3= zQ-dyu4H9-LFAsbUr9mE}hBT7?E_;V+l2Rzf-U{K_e`%F{bjQZVi&SuLo?y!sO5h7l zWI}s_Uh3({-*RH4^GC}x;0k9FcxX2Ru+fIyO;_)6e*`(N_iF2nCn=cnM1J;x9yC1e zG7>bGl|E*;?x|NQC*am=F=&H}&DX?bP2x;E?2d@)fm$A9A@olU;epv$>H0kh_dh|S zE7)P=qK;i+jas_jhX;DQAh;kXj#Tec=e9yi&vSYx9RdAm-zWz+;HlL9Fy^xmbC}jj zl(?xjPC*5q{qoTyaORF%RTR)<&G(xcR7&9Cn~xV`izb-zwlu+!@FIn$39jnUc#(_B zO8JaakW@L?|4}Xjx*yHh53!Z1@};B+SeB9EkNoA@AYc$w=50w=yT7Rk%^-dDy97LJ zu*@riZ$L`{Dw$yKm;z9elhiA=PZ617cVkh#{weW@H7+jXf23jsuD04s`BTbDJ2Gs7 zCkas1V0QEKXK{^DeBUNx?KgStT8tXKSg=q=Uez z2z4e;*d9x)907A83BKVOB_Cc3n@x6!!Jza=D%^Q1>-sy;d#B(k;r06inW0onK*c{k zt?@m0+5=i9XD-xOOJ;uXAU3HK-yzi?0mmeq*~$E$?ALks`(6g!CB?NS0X36#f&{8* zW6RruNJ5d8*bTkGOigckhbIEVfe=FnuPW}_qb@H!rtV|KY~C_l`sniyZR>$?^c`p? zmz4L2R@2(gP zkHr}tb#C%G*zVWQKkH0myI0hr_Q1yJY*XYd{Lsl7`*TfjE8`%>ciFuc{cK;chr;BJ zf;C~W^!F6wTT;Ek5ApK#tea^C3gI7#>&>f56x$u>M1+r8QF3fxFGdv$>s*sl=Hcbd7N$*r+F8#(gyBB%hz=S*iLYIE@qp1gr-))q)WX1>781JisCmH zJHn&;m6za4b^liUvd?8QKJ?WmzN6jJ=s~NdGJNP;TaS)Bw-Boyr?)(Q>*IeH|6S!z zw~)&P3`@pIAbjNdFo>t}idNiO5b;0g7c(f|i#C3ilRJ5sP5=1}ZzSDN8aet`@m?dr z3tnFE*6n3N9zEA-H+Q*l&8=RA>EYeBCPK9Lo4LRQbWk*U9gy{h^oJ=xE$KMmD{92V(@ip zKL6=~mzsb&C7A|Z@iV{jAg#Z*MBMCG0s?|!)wBJyib(1cc<@-x1Bccv8Bl6jD#i&F zWH! z5M#A%B6zsw1c*tK?^vr-e=^+03&t+7jjw?X(9Lyo@L~0J!=Hcth{i-xoU@q!f-~%v z2!olE$rLb{54<)?q7z&`v1H6WTA+=m(M|$`kJP#+tF7pG4d%?mbEHf_9&m%ERy9OY zwEcNc%(#KBKmW%9*Vj)9>Xg99nJ$_PyhtD7V^WaQ{L9me##oRzM+H+a)6YXDf{cFJ zQ7^LV;L*YFJQLy})T}S`*D{^s&vs4On~nLz$c`xT{V}?ZePe^L6{a|{6_JZB-{}+u z67UVl7BJ;12`;NLmGzBYUzj;P?H>P=-sGCas}C-Be<@wEoTlfsyn0=m?3YYvb`GyJkf=0hhzCn-;qv!FYKy6 zXVooBmdqY=b!l|)Y<^6J zUHFpDd5Mg>*HX+)F_mqkn|HS=Kth+^zdq2iynT_2WG=*KCt_F*K(2Y5*E(CXRsp>T zhKfZ23bW${BUn?3V+@9LAG6r8Sw^+Z&VcU8py-F-{zUPj-BWnC!4;6{{OCeC^38PL z+LtTOH`8WXUI|x!DjL&knq40BHAajchUeU^YsRs@XNrEuJaNDRoA?ZX4uG;g-k?Qm>P`)lZ&%!w2qqn1LrV4HC%758UJ?Skw+{xD#3-a?&{mXr`Pt+qxxpEOX~a8ie$}|KxUD%5J<|O7jnKG1%&5{Gj3%E z2=`uVJ$;MfFf8V(scn{|v+SAR&PR-W`7kzEBjfXU>q9sD*?BV8u zUBUcwD+$T@akwHv{2spAzj|PP=$>t~M42F<5P~&Xyu`C6nkXnb(gUmOn!0PTD>tfY z#3#NinzQQu3}RH1kW9j?Ux&x*csHl=xxNSfb`ooIe4sNO)ul$Z50JP!nyL3!s&IP~ z?vIP}(>BTXUr#RF9X==TWZJiW&iBwU=40|@Rt=@;h7`xjhY7jPWnzkeN_AQwDKlYt zghRP2|4A1RYPeU~@iGGIsxL%HFP%Gd96 z)O8eKZifAN1)#gQbWZ-fpXz_A&$+*ElE?E46+EP|w`pztaCld?7vmvO)o@i_zSsf3m@RnE#a(Ui!>ye0vmr z6MTb&Bcr0Dnvp7nHu?81AcOEF zc#-TQih z0YDv3F>yc{>np5`a=yQ%eBJm06r&D7XgRis(ZU)J_C0J`o6Bft8hGBG$KBtNMee4kC@K#w<37qV2bdGz2(;h-1g* z+Us<;`%w2$KB%ZkKi)4K$#+#>u#&A&tkQm|-$_y*ctAi#piNbF-PR8@dLF1LB#G;T`Jw)-_zfGi=d zE92bL)zV}X+;$F2q=o`U`C5A|V?#;z z_t*IOkY9vCLDbL0dZ9P*@j{c7(=hm%OcWG%qkK<27geUA??!Qmz7vb!L+5N=&_ieL z>Ol^;sv3HU5kB;6;p!Dohf>4anPLQrqHm%2Z$@Mye?vI)r*Pj}AX-40kflmsDpirZ zJWcvft!)lqFOdYKtiJ-m6INivgL{)}h9Iw(tM6c$yN7)5$?$akb=<>Kl%ot}&Qc~- zy$Pd`GwsEwqxAn>d&D}bKkrNZJz6&y(pNYhJ0C1#_fYEarzcy(yJ3VW<96DpuHZDc z)ae-sx*5%q)im4(%eGt!2Sc|SsP0MV|3$R-tPjEw1dJG^XM6->T~fw+P>qf8COUx% zyh1v-`bmD`0}pQ05@MDx!utwa#jdp!e0xy_^^Qp}%gD<{&rlt-U%2s1YqbD=p{%mxFBcbAAbBJh2 z=5KchmAheV$ExDcLF}AZICiHW$%~I|@Jb<%9%eW{Am{R3lC?id7t#PSPkmi?ZqS}o zT5@_GI}h_=&}IB=rTH9FWP!LgVjON9tum^Op*Yn#g(Q-kmU5RsBR(eDmFP9QSX4#D zj{`Vp>FmDDVq3+KpMj={;Ge%*2$>TJt1#jiu_|*JE>vh^GXv19@d$|O;|eEDVo@cg zp7N!QXXE<$+$NOwnW)f?yelFt21=K%{AZ?Gdl6RV03H2Rl zdf&Y!R>+LRE-ygHg(V3G5YgIuUBI0eN$xuSaEmm}@=!zzoUp(T{KDmBIjoEXc~PB{iDzbEBe zlKOmK=p}c|pW;5CWoWgDe;Vm{gLAW1-I+PD@KX}dIP~iOoo>maM?*{*Mg%dKW^|*8 zIS%j3FqQEku@x&|Z5)f;)X`dVgjo(w=0Y8)yNTvYIwVG>IQOE$59;!nTx_+ta~b^h zs?{vz`_$GW2EZj<0VidtKP0Td*z>3XY~uU{>*L>;R`&NDi6*>FZOv4QD4zA?!d^4# z#8G=7M5;4O3i)nPU*$=vR0k00n)Z<@l6T>^Ikx7uPs1K^eXQw@V4P!hKzH}axOc%_A$xWC|c(cRcja8&kR85vxJ`c(#Snp(|ypa?An0E~Wkh4JYK$F#NUc>XnqZ~~o zx@K^fVK1HARB4#r#6DA0U5p#!%;=AiLZl;qQg#SE#oPGAk`<-qridA>rafe?v2Agf z&sEXEJZ7YW2BlRtsR1MnS}o@Q()_J6o`j&mh&)x#68x1FU}hkZ7%l6DYVR++fFWac za~{tHy^ydis5$DNU=WD0E-|cSvr97}-FxgtSc^`u$aoRehyz35+}*PZIm(oqIqaH2 z%A0UH$3@s$Ur_E@Fv!NVdvD;?2UF5M3E@qbgY=hBc@R88z^uM*XV+rEdL7m^;C~h`&xd``wF9( zIr0T@-zy>+Up`b#6xrM8LhEzySr`Q;ofuzL_TGx96CORVy-!-G!^}3k+48JiiM|U! z#<=#wi!<1X8O|$7O_g-dzy%^-5eBd;V$hDi=ESHi)$)Oq^uwVjqxapA51dF{ki+|x zE#b0nG5yTcV6 zOPqCiE>PFU;o7*#Q7bj1UR45io|_+xS^By?Y&lka)NX{XtnQI^ir>Z7wV>aAxA!%j zIh;?Xm(tF{mDz)+^bdJ3cU2X?59KwPW@2{iIx!1b;&N>djkpvqrVW9CA>*i$FlOOM zdEq)nM0tM^osj9=-OR*1bXWYEcr}-5pa}mG{H6+nd|p~EImqK2#&69rFcbQ^JIs6v z6y&SYp{(fUn}FAKb-MEpMQKauI269P^Bp30L;lqcpcC^lRzQ&w6*Hpbe2n!O6>czA zDv^mU*PtI#`FO4kaXSFoByx=SzKHbblkazx~#^yQ>#Fbx8_O$a<(+un4bg>^p zM5$*jgb?ieW-YPvoXMSzHqJw65F0{2Q#rUHUxSa-T2Wo^g?^^QJQz>bk-Qyv+S0Xa zud_ax4AZG{9)Utamd?S6fW0H96nxM!awE-foP9a36s8=_`a;>Cpn};ct;_A$Uaf`s zG67Z{i+YIP=cg#H!?>RpS%r6{EAgXjc^As4i3i!fkqr(R?Y-ox=Sd?VczpGnWn&d& zS-ld%#R+5wxKX?uFTNsdBLM+r-4fXR35pS>TN|R*W zcxV>oXXi}#BmGPSy!(-8bpJ{%bJr@h({VHc>g-mB0?+aB0JCoeSvf{oVgM67C) z)ret-6hLMC;VV9kS_Zi>(*1nC=fKCjpH%ouLbG|Wr{fH=^&Np>tbzAudAqHZgKNLI z6&wTwT`?yOv`;nF*#?w4l|cGdWxNb_$g-fOcuW>`bUHjV^B2ZzVTFkGN~!VX_zv3K ztwPY6lY8$uBkAJ;;r#G`YZZMSzdg~3r8PqYR2}~C(o9kJ4tvXJQHqaXm^v`zTW(nU zYIiI>D{K@M3d>*k?El{E$54&|t=#95t_9LxtzmOc#5)qqi6a)`;68r;>wQss4udsa zI#`QwWSv_!&dg+PT_MvLRW(hu7%qCa9Y7;wbXI5^CfOG!SY#cPgk1vECWJdtgdYiB zLc=VHBW(g*x2h`^Zx2<}qFQ}u?@^!y!K`eV>;0@#2^AzxqB2{)HV5sK;aZDd zmUdhHN(pFyuL!*sVM}I3AA9kmtfP&}@Pj`R2jdU|AkU;x*a)`oHrZ%QTrkXKC*W}u z=0*=otyK)#HnrAo0pnFfutm_Ej)mySrNVj~>6{4J`dh6ie(V2PKI%PwlM6Eun!E}h z>08d#g3xCtEIj$1>yqp!iP2H>{_8f_#OYQtMCWB1$%I=|Orw9;_{{}M?GG2J=W^aF zW4J|P_A38!JoA$ah{XEF2!js?sJQP}ZSsCCqGj$U2h7*Gr{cPk~M2;+ZuA5g^c&Rl=i15r)Dm z{|T^$W*EJL-ltTL#f-%RPro_bwp@8?;p~p)5%tLWrW^YOexmu--_Pw(ZSQb6=&GN54=Bk z;;yEKN}aE>C(L<6o&NOXMCLGnn2_-Oqc(uBFJP_H#BOayaR{w@dKX>LrlbB@8-O@e znp3;;bzE?IJZlirqcotWq`3XtyxzK_@(*3+E}8*yR{IdnQ-MR}BYy6}cBMUTbme8I zJ=V=o>q9bcK4+_(eR@35<7Q!ku;iHe4Fb*5#2xJFGcrAD4!A?Ww(@r|)k0 zncdNl^IW+vZ=#CKZq&nzcGv${G=QM=2pc7=rIz4P3iXi_Ls0MGgZDIy zYCV`c=$Km-cSL5z+5xyL^<^v!W%y?>&63rSNDG@m%LnKBpp4Lp-PVu`m%AAS^5TLu zO-9e1#fxP3+g_!4y<^ApwM@U#R=cHZtLPVkp(XBxc=!)xLq*j^!CyTM!jjx& zv-{%ifgsZd>}|VcHZ^F80gu6)FbpW>zgY$DQcuBIF7Mi+?l^;YrC*`p=%&-?u);QX zv@>+jOG@xJR^*wmzP+g8n)|3gCVv0IV72|tOaC((2;}cqC7({Ft01c&aTDUx=q4wH z#DfjDkSD(`IANI7UouO>y+8d|{qbqx&KGci$^|`VEhNVF6JUMc!lDxcohSXpF`Lmj z_G;4YQf06Ep}WRFwPNeNE}^{I4atdk1=E>%|Avo9AmHyZwH=U#TRTgq;^c$AWsRvP`m1jyhw_>7gzSDse^|z2#19KJr5fVwfCVz!n(ydxT|D zYJ|x!^Kzaz+^VRn56iGyR;5mCh;t*F8KsInIe6>|+`ak>sd@Dgz+ z!0)gfQz^-O>}T5#1n4Pgla~JU8T5o#ob72DB(Moko`UqeLk9um>_$P=Nt1 z?m2wFTkZ=Ad#b}Kwd2AMq<|pT@S&h{6AhFe?1ZmgG7_}4ZXY8#6XSTrl8_p0*iGG7 zz~ErmvJ|UbixUvyU%@&H^L6r_&ZSMs_!!2IDG|mz!)lFvY-9l=X-!#^5^r=%!6jCJ_pqD-Ex51w1+kpNs$^AQTsH@LN`#PuG8BY_BfR)v z#D1}H6b^%}mXy6FtwpuKCZ9e!UB!RQ(Aoe;JkZ0G0?M4LIp(=i<&K!CnGrv zZ7B>h#*eonGVE!-N5r6xTmU9@nYO~|jm6a{kuEB=bnz^y8x<}AsAEm@yccwBe{EzGse40YY)mxOob0aFB#Sp->I;?;1yk=EBJew2^p3D zq1BMOIrkL?d_?vdxh+4?NlDMM#&Zlm1_3cbl_gC9?MH}s^$KT?gh>NAnQ*bgU|}7C z)~56}Z2?Es19~ZM!dS|Ms87tgVz_&2TN6k5_~~{o-16epcI5}TQs4M=#=q6W=m?v^ zNyLxKz8c^~{gex9gAG$~;2zu2MDdRxEQ8S(-Y~wqpGri?FD6J;;CEDy|Jcni$z*?V zS~r<6SYCb0$xn6!wZyoOFDAy-geAvAaMFjTAHmk7f;5vlby)-fE ziWKT9GWdOSrTWDhPcsXsH5OUVQUS%_z-yc&MFR6aFPRA{jcuC{W|(n5i|=y}ACA|3 z)fyY`iihnO6$Gcw1(CzqdR)$G@yNsnq2a$Y5;F2)GQKD!70%(@k1?QGTqxX-d@*D03()0A+h%`H zGR3;$DGKpbM*gD4D>=SvD(Y6&6vA$jEYIUzJU?27ME1{O?D!TsY(Tf08&|%aC~P|^ z=h*)Na$##qs~}%fk_3+9Rr?rkZsGtLkiu;*F_w)H9uBORDk}tiL0FA4ZfrZRUXd7q z9L+vYn5dB2jCo@bA#@P41bfnE-r>Ca_3gd`c2qK+)!}0w@-D6CvkCHJCbL5|ky|(e zr*G6vBg^YMvYS4ai^y zqeY6p+A7Hi2Wey;b0w^uPFa4?KK#()k!XnQ_T51)q%&8$ascaPE7afZ`ZN%)kB9+I z>rwp(-MWJ$j+g0|%6j{I1OBwchFeBlE_K*OI+vH%#{$Zb7z}wuG<-bODDFg-y6eKx zb|*orFxp~@25Ti;;IRmvV?Two!l*Q_(KV1^pC-4D7Jl@ED8lG|<{@G^$^a-Pr{(kh zB}HI~s-f}Tj}Yk%w>J0=+lPnCRfu^r0V8V2LCExgnkzWXSZBR9w4t{8L%4{6Cxf1; zRQ9-kc05;VOj(iM#+hw?D`C$Fe2S_4QGih6)b>Z4<~3l2Rb0}RL^+{88QW^1&EHb&p z^OU>Wi#Ol9(v~`q<#T>sXqlBP*By*fI9%BYZgG>Kx=mczIPyCkQCW$`x)QuGuw#Yg zx8P_hF&+Pr61+>sEfKlsgI7`I)qmEZ&xUR6><}QsWh>W9p>k zH@6O<5Dqct&FvRmz#A6zt`FIl*)_bKp3~TR4hr5W9L&TA<}ZDVgET9!_F#S*WGkL) z6Yx)2&4at59gg1EI4j7@5k`Jdbvr^pAZ#p+7ZM)fV zN}`J2mlTfXEjS41`0?L|0OW!9(xX7r;qr=xsa(}82dR$|hbjQ(GcSe|9ATtd#H#u{ zl0fB(AGWdI?~a7Bu4vy;vk)JjxO{YL@~4ywE*_jZM{Ss=QMel~6lMO9PKKnn1J`3F zX!m|!O3R$l7KnxC=|R^Ueul4d+OOXIvtIp7ZM~$A12?ZBljiKA#|3x$(p3oJg1pxP zTaX{KMABS?*Pyz92fyDo%Ld0~n9T+se1vnnGCNULNfk~S*%CtjI~|SBk+1p_SIT^_ z`5I?gYl}oSd0;i4{ILlym82bLDryM&a{~Cx>%?I%hXWU_q_`z*;~dEo=egQyxxjE( z1fI2|EE~0OA`;(EKjGQLKI{RcZ5Y-s_n#4Jm69xg-|cB}(3R#rh1Hk}$HReR3mt03 zWH-uqXROmM+HuM*_#NA`yyGZz;*hy(J792=>c_t0@>OKA%Hsb6 zrt{aRCU9Ii+!#-WC|7Sp1yuBeT($e!Ht$j7}p z;|Qz=t|j~^v#fl&Kdt;zq{^+{h(x5l8oH>p{Dz0OvW4B*K`8bCpGFH3#yl*v9YrG? zo(3?zdJu%!$m@pW70`L`MsQ#C`&5np;DPJef5whwU5|Fv=5;(Txxnh7!-@8=Bi@_Q zW~*i3ul^M6K~`Un7dg6|j%SRBP4N}Vc_xVKDVPq*Hn1Ce;IA&dGk(Bg!W}iXVPrZ# zNAnnpt4DyWmX!{}oo=8YNHdY%PGf@6GlNx>1aL7B7>Pfgk;DZ}B zm)TGn|A_ggQ2*xJLti)T4yXfxh!2#H@vDI`;G?jN#8~m9#aukF{3);AvL5N^!>r0j zw_w=QpG7#1R>nBSs(8iCvdOMAEyZYdfBX?gp~t+-U+Har*38-p8!Pmg381+FkwW;? zu;=d{|3tPsVMIu&>` zTvQemOOR4@TRL^;hbltIxct)PAkW>aG8dSFmEF@us`R3KJ$pn2Q|bpAQ7QEPkqP%7 zSi&d0N!VzUCU-m~@hYTUrqWl;PmI|M&E&cmWhQjaAJ`1KO|JDUQU16c5*mp{MjpBh z(~}eR3r{*Kpd^`s+&#BUFX@$v>jJrRR`GaL1%+b^mDUoeSxjeeQ6?o~I8Q&TwL^Ow z8}eE*2(a%tMES#+3)w;4yYr6

Ss(V=zJL4+^jcqOnDvG8*GwpMc-&);0COaL$Rz zZU8iQ5b)Inj+r{CGY9Y(R~3IHQY7d~QKV{k(8*Kt;|A7*#iQdsHcbrKr^7-YJwCL$ zwb>0>KdppTPxol90J7w;nS1=<^qdrTf<_nX>jF>(7QIb#^3hqYDE9#5pHA#>=y)jx6 zJwwLhuGLidq+#kpjEa6)tHnA~YkkjO;(by{VKXZ4WQb*XMhji*b|MR6F_OC#bSa=Pz&SY z0@9+sQ=zm(!tv)%!4%N0eS`2uD3iY_Z^;}eA35V53f@pJ9?UA=_9$V`^0R+I!Z@Uf z%`=Vp2+@oUIY^qWZxwDnU$b8fkM!$>FeEnhYloQEIhs;%mkA+1=cYPOM2mie>c9DZ zG1!Lzo9hnkNq0~P`?$3xMcEY8MF3mR$SCmVOF7B$I3a5NlLtAlJc6;h*S3sm~;m(`F;W(BZtYaDcFbpZt^Qe22Ch(dGGk0lI_{C)k;uM>e|f@+hSN zD#LU1_V--L#hI@}_PF1mN^Opsn2t3a_>)l5-!b_cb!6@Ru~1>}qjOo6zzA+_SWBJ# zWkr->veatbB;-Kr%0p+7>;{iSQk-1P*gljqxM-l4A~%j!)Q6RLZlnJJZFA_}2*wot zIg;jIdZqUD>kD2~WvI9EZG$>;Edk5uk38N>-$RszFifb;8<6t&uYah~|185kDfaU| z5c8M&TCDLyq(#ZY&Fa4OG`C9_%lb zwCQu#rE2x$9J-6!dEHa&GKhLlh4GZAfcA_AmY`bgg7V9L2F5tCPJ%4VGh;{`{7tPv zXv*PwhB2R;)|$&DA1b@Vg&Wwu6nWR!tK^)w;S53KA`+H`sFXzv924sOSuwggbMkD5oxp)P@$KOY3EY_X9ii zTKs+snGq^l*URDELskkz?o6?%(+9MbH-83Uj;)5Pw7D(jO(<<3Ud${{ssvYv#l3YZ?3m(}|@T;vR`OJS2vv zrF=5#wCOR`QlK+LgOuQJbcxW_XoFPymaF1ETqLAse1vFHvu-0~3Id5pWG2YygR;n_ z%KPswwlUPbZP+A29A$iZS_B=={z1c;a71GEk27jH{}sIvx<+e(-~nYR-gHMwnI%*U ze{<4{)&~!MY%{H9Rt@j8f9#qJQ53|TM zU_@@h=~JemH_TovP}{cj^3e?JE-EyDofbGd@{?hV(D|r~S~G@i=Oz7fuUnXd{MUM@ z`a{IIXENgZBNaWoz8fp6Nglv=s>FxCbf9A1|I7&V#YjEqLFswR`ER>eZ(^cM5r+q3 z6lU1YycF(yYlCFuf?+>@wpq@|iX5bu ziJ|4CRHk#MeM}|lY*VUgt%#}jCb2z0sV;{E2Ore5saMmKidN6PkZ8M*h1pPjrDB<~ zOtBeZISCnU>aSaVtNywdp_K|!fB$Wx-KK7Z$u#K_|e&-4Rsi+>Jo{|MIwV>=VM6OrzGy!;Aa@Sm2M^TcIg&q>54I@YsCCOrF5 ze4lz8%RZsRlFT^MJ?lG^L$y_XcCwE8Q$tnP0F;IKoF*NfF* z{hBt?Z2sOnP_eR`NI{fzwx+~tlSbABnoi=CeRRsY^(#W4mRC!RioJ^h6riVN6>Z*9oP}bQRJPKoImg$b7ZY%WhM1D7olf?;=lD2y75nGDx2$Hmp6%%YAh&Ie z;jl8Nh093#BWH*WibfO^bGN*AK1oKr%SAgW0VDK^{InQPIi7LG+>{iMGdv3`&*v-H~()6^qTfjdua%I8YDv1Swf_UClvR$Xyl;eJUucIbx)3>{{7IXKw8xf#19Y7*J zJzX`%YzNBxtKKu4T~IkvpgO z4y0{0U%ZFx(ef(=V!b$MI<&fLl}f(xhx}Pb%YF&UORZJXAId{@-hWKd>>_Axyts0B z6c@Orrlo}&r|D4oMYodcpx1{b>Ae|iqEGzYV_<4&rAcJ;Z+&}pX0vZR_fz6#o#Xi8 z)M_o7QLx!V&(SmO;~@rjdd#I~!8`f_a#@$X$5AQ6)rgPRV`ZDeIoFHvgFp{OLgl&z z;yA@v)NT4-7r)Cfc4iKPTiL-hX~W^7bx^ z5!(1z~Q)5%o$T2q#vYGREhpH^ele=~#LDU6U2U^`tQv`}Nx<^8%FkQ(`d4dE(0nZAD$7=MYo#n^f=i>$>~L(Xs~lG|$W8Pz(GP+uIfCa za2}I6G<1kJn*QBf!p4$+j5)%qXJ&GA3xExxr&6AZrIR1ZisQN^E_EXKBfUKOH6f%; zHn`O36oq*SGS+mZrXa{BQ?Qx^TWv5WcsWDdeAreyE-5ztb^&y!T|;gUVL-ZF`z4#R zgO~uu?)0Otq`|?kwH3m(3An15Agx5CrnWSnx8RaH?Cq@5pd`O-P7=8txr(QYxv5Rg>HmZiz+~>7 zW4YrLcV85$#)I<#5NX(V86zADnXrTZJ08MBo&1?5+r&#l`DN&-cKXt&$8Lgh;4Ji{ z;7EWKQJRVRuT*nPm8NI~ESEMByfSp*9^Eg1Bi<*85^l(X#n=h zIGs{Fp^rp(1pOQBQG(>0_kpT$TCYLguNa|v{bp@}ypt!&O$lyFCXZN270XTGuG2lx zN~7raSwV29W9(17GUZ$zg=sIcb0h_+KXL*s4^#x9HY>y3@TjW#xGBt=hg6`hfgK?d z>6;=-G2d#9L_s=>Eas& z6LB0bv{PB~S8Vvauh*2O3A}eJ6*AR>HH+A+{e`%ai{d&~T?UVA z+}O|iz5oaVl8Wgcb&!aB>%a4#AK$DycT*Iqeg(c*lM}oHQm0L3-h4WR;_&wH9QG2R zcpPNN4Q6c0PA)qiOV2j*X_>VFqKhvC5cTf$An_+!-Pq*uBaV00w&cgk5+seG#j)4a z2NYPCfuz!F=s03;D5fiRTz+l`(QKh>;0aI=RqXYC*kq-vT8**FIpl|&FD)T!lt%!K`l=sFDitDSzP{y}S z{@Zmj2+K6^FJH!XG}u~n8CJis;g^h4D6*LyAy>np({Ggsb`>SqB6i@AL*W$dX?w|{ z8vK`Io0f?Bcd$x^JsO+k!mwacR+_WHOYf46Nu+M=Ou#k zemV-_4ddB%0M)pAjsp7FdE75}0uu3eo2UG00l|V$?auXdX%bg$&bzj6+^Nsm+_euP z?5GxAHD!mAh7={3^h;1}S0eHa6nHaNELfeABJqMm9-jZE3w+QiTz`imNb90abZ}J` zr@?w8#4c|3coLdG=Z9_X$jLNf4$8V93xmz?ETvib>|&id+0YtsPnO9ZiA~hl233E3 zi2e2d!`C^r2@+_@wrtzB(Peenwr$(CZQHhO+qP}po;mY)zWXOKGcxvC;dPP(HjZeH z3()@XL2qdU5GV#ToA*!6E2hpZP+pBZj|0eWMo_TK#E$s*2p9lx48X!rfLkben-^~J z9@b~tb=I0>{V+{*d2pK!%wHmwTVpnXLq#{FL~`iR;~B=fYzZl+o#H+DDZ=&lFbq;~ zzL7o2;n)c9VFLe%U6?z5CGC_&+;=Q6tFcCYF#-&^$81ws=2n&yv=nTe%l^!{G&_BA zl=#R(1UQ%_dOd%VN#P7t`}B#>m{aiXT|ktVu9j4^+cZ~Z!k+-lVIsqfPc(b!=(MMp zqDMZ*UUj;Jn{lxicBG)zQ1$NlvT7HkBBqqGX4fZ+7=i8u7^uBJ_m^6l9)*ey9Ig>A zjO`IXSkn2@m3(8mYshs0kUL;aMg~{*P$u=YE=bO<3#T~_JLw0sun2h6e6a@$k8S}Y z+WXxg9`@X8Pz&QZt9aS4v?%=p=zy2{r8vUdr@JS^Sz4HlRfOpZ*2<~?=o-{Y}U)SIzW49KQy3Xr4n`r4)JsK2WZSiF<8 zN$(f7qi^z@IFLU!-NxXzcpj<%IPsM z%f?qpo+4II>8S-J+gA+w7d$vAu_QOv05!RSNE#g}<)p)IB*5|W_AVXd6WSQ zQ_(*MO{zSzPlJht$;4oo7`BzstO$&@YS&X{{(=q{;m5OW4ch-Y(V&4s-~g^xhs{h2 zR7_+#cv2|&YxzF6%FXdH>W&~l*Yh`1pC2pmXJU5NOm0Q0>~Pl1Cy$|>oZNMwi7)NV zdKlR0l(RATRe101P~aYbPUKLMj|`Y&J1;E0+L3(}J{JpnD4+T^+lm`4x;0KPmB)p# zmOG~PD<8A@<# zq_gL_;?cbk*b~JVC%bNJ>-FDNV^d=6)empAshKJR+y`tJr5HCml zP>I&sk`UR`9brBh(wT<(qu}^-MlSBfI!7rY`V*UN9jlrp&-4cq6-`@81779VM^fQa zO@&5-@l-r)uzh+%(f^KrMa%r+cxeyL6A(I$^;0|v6gmcSwM+$vFxB^4`F^xihD<*= z!hRyx$yf>-?ZvzySU=Df(%+#)p^@Wbh)LzjrvnWS^uJc0UmEF`4>qWZcT1e~9jazb zJ`?RKFK30#r(|?ki8r(0cXA;T&!%L)!Wev$~$-k4`I9SfQ!HbI3Np`QBaRPN@*eNEgAz2dPMStDod`29UNKmBGJhBc?nlP zR7m>8cQ-M{jm%=TKT!1k=VI}v|AbA;uFaxvK!!+N?UD5MgI&)8qSa!}|M$!GQ36L~ zDr2iMO7#c)qb@eN_&=3O$v*_Efg3fi?0n&$HvhjPSnDpEVsOh@9Hw|yxd&nfs4J1se!n_2XuqGcy)x(cY%pxX zw=If_-o?K*k@lw%#)^x4)A?neHPx!aJsR68nrulwfgaZt@K2BWSEb8puGViCB^VbY z&4%?i8Q(QFq2y1E7R?tWE=Bn7iMflVEZTMHA03$?Zpybi^Qu(xx^sa05eGds83#Qb z&Z|#}Cvs?&6|R}(;j*8RP#*)k&tRv^*Jmx>ES!#p%TBaU!9g~=u+QvX^k>i?7TV8o zb31!JZ7pkTPC~@1=$77PX<&v-9DS2~H$8h!jxt}Ciml3uLYr5+g9T5+OEQB>ENuHI zhQH07dAj>04R4+ms;&18t!`OGSlbp#j875!cCGlG^E`RFQMjFIQ+6`=7|&;sa5mPf zEb1Xoj@KK_l~`<#@LRK}lw5P?rgE33J*98@f?oTRsWDnTC(vDX>JX@taBzgr0pA5g zUZTw(xP;6d3)lr~&Xt<3e$VhVIV5 z5rZVT=by@}jTshkQaZI?qFF(=>MeT@`Sv!pUhQ6ho=IdKckD5@+q(>0?(shvTYuqe z_stSH?$b!F2#ekrk8p8>!j)r-FhXzt0?IqtfoTU<70upu1R!g6_nvPakZY^k9ju3T~SbL%bolMh)i5 z$AKq+-LIz_K&W+XsMNtulBYWcn_*dn$h&>neMRyVdyDd`11gfiPTlEqX@MqD-RSO< zU*#x2@79C^_Ph9#2Aoeqs9eljEW0K_;E?mZ(y#9_dIy?>35OMwJ}tIcV)%LIMS`nq z*b>9pC^VIx`ubc}0z-otCPpoXSV^oB6>H!p14b zcZ=c?m%)}*=npM~$-JvETbXryrtuNkhd0i)%Z=?`C>gb7#2C}5Pn(5?9ZJHPAiq!z zb!^&fYjp8M-vXOa3anE++N@>$HG+*+ehwuA*es2juY#{2UDX5l9c@f-!n zaUY_>Jrq-qjO1)v6drzL9RG1I6VLD(RPI%V%kd+ux+C813~I&PVR7wGmXnA6QI%BT zn2c5~C+@}0y1j>0R9jBM8>ghRG&N1gyP=}l{>+#JRLlToCa>6J;9(jspL4L7r(#F( zMd%NF4n00O)OwMmw>m3;za};csElM>R$J&Q87->3;g?3P%)skZm{@>or68FjV5oT} z!tF^#I0?GAA)ElETO9X;1o&t^DCTHKP@0XP@Y_a76kr1*oqOhbxzwN)>-{59HHSJjtEtO?#8s|oi!pl+A45xc z-jDHlYC!v<6}6Mor2C7A6gMj)2JxLW)__9=1+a9lF+E9fZM-ZY~iegRksF_ z;c~zY^~*M1jl;N2;6ZVt66oAw%h!iA+^r~m)Jnv07Iv!0^JU2a+P)>-jex0G?^iw0 z59HMjFvpo|x83?bES0B>8f~)kAUrFl9&RywvZ>r=t(IkZK#fEMneEVb9l^)SdciagwOTLV4I3F@Z6G&}Af{6?8M>5Bwfpr*z_@>gHNtu)Y z6k&cgO$3`~kf#D>Xu^L>R&mhd|AM4X!9H!N)-V!I1VS-4#k=)6H}bZ)PvIWwAzGcq ztwpdd@W=oMc$6b!?S4LqX4Hy8%vyQ$v)NusUy;ahiF}Cmg(p!QPN#zil_F?KGAN6ikP(zM{gk)cAZ`SS{8*o0z<(%~T-@jH{dh zh0IV9c;c=_-M(r!Z=f!lk+B7$*xw{uJI*G;4p;w?X?n-#I0+40NX20*kOdl`60R33 z)nahck@fSnAm5$Jz(TnMjPV*C<~E0oGL=cJJEHPza+PT3+^x;yl#`r~>J+SE8*{*? zojl2g6tBMWTASN$ukQ&wkpju`-%vM2GIShdnKFsDhjC9w#oDO(99B8xP*~x*uo7J> zwRle+dNi3qz-_bg#4H zh|rQUa?V|WB?`34wd$i%c62mr9)hj1+@rx&Bd=GczL4Oi1nTl{JmY|3g37;HT}bXH z=`z&X!|l#9o=8+kQ;o#1X(Nk`9MY39h?IRki93u(L`>>{aQ5ZlTR;R@)HGNh>K?9;6JVW@NXF)fHFT#f$iTDO^RfBxR zM6gkhNoY`8g6)7Xra1t;xGWSST7`xcFFyHI_+UOqmP0EEX|5SX6vn{hLzmg-U*~p0 zMag@=snIliyzdl@tvYe(( zFI%bFcfxULojvOUQ!T3ja19x9ttB{eB3mJQ3`p3!n>)#$?7+kN9+ZEW4NivzL_=I~ z1IG5otJr7Qe(Z6Rqi*A}=cny~GH6SV)bN~wL(VPY>F-RW4mMj?bN;5FArDMkH5dD2 zSecju59}@S?)jv3IU*x(EFm2|#U%l2V~El6#h)n`Ta7wcDV^TCE#8mN{E>G$O4mnZ zzalQ2V+7Ju!C!*SCHa$}d8n77Hcb(Rn9Zv-j>~6%!90`fW#zy3)fQ)*2O3D&Cz%E3 zo$k+?Tg@XUhcm8LC(`QN38#H-#Xn3qa-TL^S?n_sj4ImkkZMKpDS*N^d!!%g?U zSECf*d;B&cJx?}#NcdBWL`MN;f6yWkVsDRso{=$n)@V)rIu-Q-zBudq5ah(vD+~ViF2PuHP7sDjBz2OLMw}x9$Z#V6+!BnuH@UuEH)u`jpyr zDtRF4i{G#))K?mpGJeoGbpo3y-(?O4??QqwN1f}HZ8qCjDK-MQzPE~vSEf_c?GF^? zbr%xImc36Vg%0U$EPpA{mS2TNQ?;2Bn+{1n^BPoC4BO!wfR2Wz?6`1F+7nZq7|@3p z_wv1BOJoEAdxBa7@36c8q1cq@Ecgo_ygf06Djt^iA$a!}&h7 zD%*?^n5OS-UvF=bq1KV($rrT3XoiMNFnFJIF0%h##k`;PvB(AEHGhK493^=zWiS)E z6H9!68YpAD^4CY+-%T(m6jdf25W>L35)0~i(;SU*8jL01VWEjj#%ydbgtXd{{cI4~Ok*e1vW6+;CH% zDO{ecTi@%SrA}PYUbQ!#2^>2qacWcyF_K8vGZk^C-mG}K|1 zttg$$1`vl{dlMs{l9yFpjIZ|bS35E7e$wuk^Ubi0*J9U}X|!0!%LBaziK{#q5&xKR zMXSoTKo)wh^z=Ni>(4j}N30%hH=p-p!m%mWlN?Bc#1?w?kHpkk?t5&4gmVC-8Y=uA z1{NZGa$TA$A-HD|{-knoPDez-#8-y}GV!@`S6Os@Mk~g2Q zwoOKN$=wk{!;3WqydyHv+6=v^o(eXT`YBo*vWwKU$&O&Q(`}&_z|g!$5YO8bGJ6*YGg;EU z3-Nn}df>Sb+a$REf?sRxs70?Kr?pAiNjJm^*bhsu@omL2oS1V^MUkSj44{gzZQ;A#gFpzV*y1bm!45{^utVp^LH=nVsQ-%@tp_&+u&n?v%?2&7G+w$V+#l zq#PWj54ELhTmIhpZ3Ws$vN3MqY}NQX%@L*fN0TLtLfBOpAoaWJjwL9K*rv#_4MKe3 zvdx3Lv9l9l(b;Pl*Nl_e>+pb=d(}9UBU&0)*IrhaxH%Q&*OQ69&M%}Oc=Xt3#i{B< z;ybTq5@u($yZ#uu`jVkO!np&$du6T1yB7~~4+hB>ABo~n?c#snt*3H8)GRjC{LbG- z_0P#dcYoY-E}qR&Q8yUKKC6jJq7lk)C_|!vlRKykGBt0CJ=v7 zD}<&V3`9pqC-?DcQ2+MH_xo}7JI-pXt<<`1ay4Oc)u`Rr;rLlq9npRHQFW%(LY4Cg zuH8JFqRer1#&^@doc@{o-do!!{W$sDUbELboS?2*brVrt=t`{$udcZwm`JNF|GGYV zaKrZN+N<_rNVC1C>v6!h>XY7cR2sSbk43Az*1Ek`tInv$`+EI(U(U922m}uDj-1}> zh(deL{W@*1yl_aa&aCi0%H1+G}PK>PjAN z$(vQ1|JI1^_7skNUu`M${Z@g~sL|d}J=+S#DYxCOc+zca!HIA=?bYPrs`JU|_WS)B z=k#Q1NeECx*S!6nu^x%YqU%BMeZ>5HY4okr+78j~294%(;ArFRq`q`M{)(`4nPOBI zQ-QJBL=AkL`Mmfgft#8(aS-WMZ*_)9E4)htPMHp5)&WCX19zIJd#zcs`Q916P@7Yv z$&Al#=M-j-D&|PZr*KxEzrEee$3)o9-H{m3hp-;XYa0Z=XB=i8+Whs0q4Wg>Xj^fI z*K_~5DoCVj#*t!CG@PRe3^+dj9I9dy@t>!8&Ipgt;qt+$dVw_G68?3Ney6zOhn@v&wXPQ||eR5(p_5gntuv|BW2 z1os?a{FnZBf$W|0DNjM>f_oZa*FPN#-94VBBP0KNQN+PGZFRJ<@XZz0NLK6)s)MNt zS+Uam_^dt)5Q;2xDXr5?`?+2|(~4l(-pf$7}c|DN-uSk!u?Qn(D~jpNE<5`dd2>*P61M zz~LOdPVL2KIWZjc@w1)+E#@Tsg{a?%CCVF?^rbpk1iR$=9ozPe^f(l1{}XhxqaeUi zE7W|KaH+bueexg842qxH zo-%6t9=3{AGExoqFp0Gtq0?e8x~rT@xZdeJPhk^C?lDjQh*IHNKN^#x*TB%zx=mtP z3Ob-13i(6Nu~K@#qLFi-Ob8X^2K2d*@Q5D#e@~gp1!F3=05=I^Hd@D+iYZ)>dBN`B zcRW2)r@p_|RT?YPE|+xCFEbOKlu%0k9`KD->v7aPYKYv!7p~hcloQ$-WHo;IE2!GT zD7gM4ZX)87TD?>o2G>qpVU0z3A9MKm;T?;3VM4Y8mMW#t`j%t)2qmCWYJT&tQL2&L zVRBl-We5A*;w8nExat+fltP)@I7<%bz=?wB8Uzczy9GxeFa#*^@rVW%=VzbZNgP4g z>SxRqIt#ZW9c@s>&Df#1&>are>C>3_4U!lK8=RW@_#FxXrd9)#4UCfIDpLfw>_F`% zAF)y83~E}RG9tPQ4elfp`P45Y;Nd*TBgz*MAjH3dd!}SUC^g&IV=^9GxetkMxf# zlDv5HUBSMO)kP&nrZUhW+70@?om^I3a^a34sKdeH|M(QIN$`OI>*ylP^q>Q*Fh&2r=_J0*fcb- zTUM`P3eOsg?Pm@*=z$O>lMPDs4HE-Jtg~3a=e|QaHt)ZNY=~mZPg#OJ?)C-fPj=Ml zzwQaL)0P^>d~jfL4m(v8@ac{dpUfxgs>;PWr{@)vJilgMPKhrUg+kl0!zxbX^OiUf zxMCl}a$+`NO)Mp4B#ur}md2;H~*s6n9uGD4L)O%NG3QZ*3+#&HLVCb{v}Rw zWgiOtH*~GVkAIp&0YipY4~^k6-2!f?9T7|BXDTmP{|=jM2odTP4OC?GD&=v!H#!luR_Ts=@)G1p~M_}!FCtIAO!gL8Tb#gq%ueG-NQrqiXQXKM;O{4;{o7(*D06y zjbs$Tudp;|sO1m2{;(AXUF5stcz=xf~wI=Nn-)>JrK@`;5%6&f-4;`8TQxj$vQ zaY74abqKSG2Z^l1S$OW8JjkRZRK&n0gd~hNfDbIbzQ!_`4K|0QFSR)#P(DJMdF%~iA|PRF(*}^P zuIF?F& z{s@%Nw}Tj3Al24Ns*FFUpBFzq^a{%sU89!`>0{qWi6LNjy~A3J1LAv1D96=LP6ri zWW>lsLBbYbR14U2orvxb*X(cWR01%?oYOPiCs7!)o_ZkVKpQfrWC(kE+uM~mtH(=G zCep9FK%UT8d#?#c<}o<>k)^i|%#0ekc~yLCp_^0v3b>jY&#G=&vs1OkNhS0L(j`Br%gW^A%B&x z$@zvV^EmzpfY%nWsOwrhUuJ#|{NzZL9yKax!wfdhXD4PdD|=)-VI z$MgPKf{Dx$=Ez7O1tn;=OIzvBdAx;|h!kP%PL$aI{J8if-}D~y_&+~oIv{EeBWmB{ zwm&2e>eqka&Ak5&Y5%iyT)(^2`T;;f10n?g1A`!a0PY2tmb<9BaGZRr;uJ!Ky)cJmW--Qsy>oTz1pkQc~A1FDUtN5R9uPC92JCW0B$;yr1K)tGm zb~P%nxHh1jDb8F%ZwAkByy%$~{;!>VDYTK(ORCo(DaE>W#BU(rX(I+*A+O7VO`jZJ zaJq*9*4=a+8ZwPe@G9_ogl@_$?x49L^|=%E7MXxs2!^GxhlO8g8b9GWN|t)W&QJdzgvuGgW+<1 z;kg&#_D+qWub$_%G7nm|48ip-=LG4V?#?F+}qu4z^#B0*LUIIUV)vT<=+>AUV#t)S}iaU(E92G%5DUgV?%|FkKt*$%o1k=u7F zxO9cuvWP}plw2M%T)B1qP25j@lJB2Up;IhTzQnNe6l#@(tP53%%W5#z-s}3^H*z~O z!6FNjTV`K7vL8|lK25T?Z~%-Ba1jVfPld3aPGNO}w8%(s|j9FD~yj&k`W9nUcJbV8$XUuz@YmR{Sk{|ciMXPC?i>M4QHz6sI&!e&{j(H3QisGDZ zh-l5X8*{Afmd8{-_y`SZcN;xtGH8JTWm>3(;DS1Xk@{X5gEBqqXGw~#0JA~`7=k0x zw;DgE!^|k)0!KSRE;(a{6V*9UxhAa71Tdu>k5UI3GCLLf9Y{M2IO$)-+&RT`6+7#M z{m#IR%`b;POWx14iBy~M{@MQb7^ty`tA|0}?CEaKo>TSavhzER+w;fM9%TO+Bb2c> z1dz{x{VOcoDR1S)Uz^}NrdPRnUlX)Qz1t9NhKjFgtpc$pQP^EKctfW;Xcrr^50Yr7 zgJ2YnK*D&|J5F4JqQ_M-ygUh=t|Ibi^wFLQkh&k>eKPbT^xx;hUr2eUJI0#p2S3%~!To_`Xg>TC z;0F-B$#P`DRPqAZ4EY#t9^rIFUaXTuPuNmSdcy;IR?>h!U?XR8T~i2DOkizaUSVev z?pI928QOf7SdG|p>a5Tx!5S>!3ALdN{;`?t5Xhz8g!i@`^|f4p%=)ne%U8txd$g?!{vqUgoa%gRt3?LUDH*fs6kED;sF$Gh0T>?%_5<$47jpNpKl;2AMP zPjDgX@M7q;jQAQ!A?2h|-?d%bzv3)gjSaUvQ^H~#>Yuz$^l5IV!w_d&%?c}3T+=g6 zf%6^^vGb+2PoLoZIfGkO!-o@cxJ9hQ)gB`1$(5ER2v5u&uStLK1K~(>6e6wsdtde2 zT)0dQ!iUs%elCA<(LX~#8u40y3~u)|r}V6MKt&5q%314do}UfTRORKTuW4Wm!#71>q4pU9l^{-I1Rhp|^7hq^0P6YZ0 zouQBE1_Va0P=Axz3$c-bC}4~WGpSZr%@7$O1^+EU<+FM1YFzIvYwcvj4<&T9`4 zH0604M}9jD@iluuW>X63mRZ?~2(I!5iIsKYp;ySvo=`y>Y=$V}*iG=v_=A|Rl)o8Y z<1TYHEIIK(i>&@#=3`;Aq9(k+q-aJ3L7oAXlGef#QO__-gDbUGolY({+pE(KFV}m)J%f3D3N_r^(?sRTviU z{-}UN#%`h?aqwBFwGxSfuz005lm30MFp`#x;4~dQp5{&c&dK=G)y3ejdT%I!+^6vQ zD2M|~ruPih3}`L4Yvzd?{tfc4LSVySrTrS>CA>Opkzqdi<0qbQJUg2FiH00s6X9Vu zO_6(klLZ&^W(7a?BZ!Wq6r>S8m4|(h@0Ohxosv$|%RuCk} zF)}piFoce`?E>3vcJ**S8x(R$`MBy*%qjs-%W6?TD%YOgXMM|Bk@CP@mS0I8OE^ER zx(>cFDmnGS7mm@%IgxhrHReTNvYQBtWzFoTL0ilpzfsA$yHD3^YIvln_6kp5(=^xt zO^FZgLHjfcENkCjCHTS=cawq9)5A&_0X9V*6>aU|918^HiM)5(HQvPnyX_RdLmo&0 zFW_2Z>MI*FHxQfC52K7Q2LI6G`vy+^ovGZZPJ!e-p?a~3)_icl=IMAFYj`U^{`2iG z$K5#4E+J;V>OL&tc9CwQ;3D#|(?=X)HWrWL3JTZA))is&5^!Y+uTA1ROrnj}$*W<} z7s&V4Jd&iq-ldE>PnK^#3`X(gSohZ!M{QLYX=ow9@sEgdOBUD~ni5DHi8U`!WBsb_ z_9r?Zap?~QcoU})f%&2F;h3v2gq-LzU5$BoAGL@Za>RUGg4@uuXHXLCk;K~=wDW?U zRnDgHmGN8lmFp&~(|6IdjNg90olc%33;K0|#FceYB3E(%aib^@o~(yR;pKS7kjtsw zHlInSAwKP=gfiR)rtr3{O>UCGEpP7cbdjb{@pMu?rjAuAQ}t6VI7x1O+&x_9t}(Ep zh2n%7&ZBQLy!>}b*Ym@!THn6wJ|1K)aSYtHGR}IiS@%8YVhK>RhU9Z9`uPCw@NoA$ zc2oeKCmuy+9l3r9E{$`1Zjz6R4;6AzV`aRqbekPBN;r`RG$NlhvV}o^p}Lm~p;Gz9NFnuvrsSj%-}T zHc3?_=J~$(g8B&T!BtyyA3KIrgeN+*wMr{aA&B3cU-`+V0@i-W<4_f~`>j%?EITnn zLKQcDl>1jqW+OJ&TJ)csT|xNnu!cYA%;S_>Kw9oWCT!*9ilGA3t|Uy3suM#dvK~gI zA zE4?kzX~TXrKU;J(CKio>PaT>xnM8&nVLH_C70Y8j5E?~}g6h#YlSXCV-QNF!?|+xi zK1T7Mh@S-O>HbG*LD6b>7HN-v#*g2ksLhAESwVSOU@>EEJ|DWRlAGH zS)#ER@b4h=S*>|mu4=;tBeKCJ6Zf{IqVhf@r6n`P2O8LWw{?(#3o^5}SXsOMN9V?D zxo7KItfFNzo8r@Z6Mlaw)Lpd)InjvUROZXEz_g3a+~7v{;SoiuLA+bNA}mF*ioyXD`VDak z5EwPnMI(Cr^sdI8lsjky2jaG6T7+5XWz|mAeSGp5WLFsc9`5{@d%d%kfrv(o6|OS} z#nBSs4vVoV%|%E<#54SOcMe#kP^&rdC^AUht*uaXx}%``(gi|taZq12q8HW0MYD-d zXE89%n33o3qOoWji;MVc&G=AFWt#BMAtX_ltN=vUQSaFGF3;67y?Qbc%w+>Ln|>SG zaQyd7Jp&-6ruZ2k=CnN$<~|^NroMT;&^^n|EJjpMRu_#g1Sz*7iR?l$Kj-5ou2Uiv zPdxK&8+2BNTc~!3$qS;tS4_(cuo#??i;>J0$Rl-l-r66Dj*JQ<6rfxCg;t1g3syP{ zXj)*3DODeGHLv!>&<**Y397=>Ep*W#6=a(V8;zsG6exHAA7QxQvwcOZk8HyLr8ooV zU>C6VRn?W@nCXjRRWgobq&dnlg9ZVX<|__jSaLxoq63lAb|+DS=CtXJNHMpgami)8 z1b#>%FNa1yhW!wNkVK9TmelmjHKi1Z{%I+4JmY8kju!IpCH3WqO?yJ$ zDs>DqQzUnBm~+AgCaq-Ro$Y=}$<7kIFk*)m}hy_1JTPgH2HK zz%@IVa9jQ6cY1Fb1e<-O9c> zNlh)a5h`hyJ$K&9RIT3HU=AePX$M--ZO7#;Men0T`*9pI)>SEThXla5`P<%oV54Xj z`5lJdL*cvV^wL*v05BG~V?*hgP9pXj4N0oxkq?4zC#c{zY&8dqh6fOoz%2%FCxw#$ zVK<-H3piJ+iQb@YJ9yD?teR1tpI50#}e(6j5gP3@D_z>^Bi9N0{E4Tg) z!T{~qX^(#<8CK82cG@nG>Un42qDAwR~I zs7l4hHUl1nEHFY*=vhTU$_PRbM9lTL1M)#^8K`>_1%X#kjIFiBLyIh$0bBs`b^ouh zbzIrluR6VQDMBU*LF93#n^s&tuNzJLAH3`;YNL9!vl?arL~xH-TvrAQ(=7rc`P!P! zDeM7`4K0tC;p4Gl)x{h-r>ViVuX(>FYS9A^`=#tz14u<%4q}`W(wl_n5I4VmJuF0i zo_E?BN#Pi~%V-cAR`r;?d-O7$Nhb}=Bpyol&T6V0PfRe{I{k*E(s6I4dQ0OsHcSyv!q#n6lQPTT3Y|9S_6X?{A^Ad4Y31+o^3DZmK8VDV z$iZAEnF3U-i!onElZRc2?8XV~|Gw_`b7E|n7R2eoV{NZ?7@#ag6K@Vm@;gu=#?D^7 zF7!;=51AizV&I#1oX!;h0dwj>01=eqA>mf-425OAdJQ#^p#l0nj}}w2%1Z9+&7%o` zDjmmb-4;eq^!^5ewmX@AQ@Kd{%UW8jJ8D92*2zi6^FxfpCN}%@Dp1v5*}felBs7)@G)Ueaj9n56p%;>S!JU z*$U}V_fe}o{7DC$D7>xC^NN848bCRKp+2hrg_z@ZNySCHEWzFEDOamPk7P%$ zyJ@53P;ZUKRUUWlR$Y$;R56#y_6EU4Q{JTRT02c2kh8^G`c* zPxnaG^CsQu>Jqy@lAOzae~n}|Abj67!BnMf51$YZ-Hj5y1E}pYQ2xP3NuQ2~r`)MZ zd4eQg%A$575_gdpsRlbGYJiFOa7d){XmV)4#utm<-IS)sTdob-gg^4WMy+h;36_p+ zsj~3fedm1nMJFs8cz!~eL=5d-ymlT6W8xKO5$E8SaCmjw_GT7K;HtLHKQ&TOm?oaz zaUhUQAd2uz6)!Wq{Sg$;lL-!ACDZrQH*Z zi#Jon98OS29JbRk6ZKh=r53xYy|Y{w-5$$zs3@1V;j)6)OTCB-C39`Ilqvobn9)Rt z4$(??IRf=U<62~QA`f@2lyRB-^tJd)Rj@PiS8?Cp>zjK(?Lzw(JZ9Ld)agHoq$d#& zHID)HzHH?M1PArvzjPnY>f8V0+I#luRNZF-fj|~8!vIPreFF@MbPsp;Z_#eCJa_W^ z07u4i&oOW=QYf#u>h@o|sZkIY6&0=Uoi=Io{rR30{J6axr|)HXU7l&C_Hx+&$a3n18dsJiBbRy)4ma*JgXO zf77Tg-El?k4&XW2*S>lW;oW3>QA-v4LA5r>y*O8NPt3LOyNyKwvAsylzP zT|rE}CAWO}Oc|8{#s(j0P4#h4++zf6d+6Ee@m{^duHn5cb=i!bSm)w)dk2WazELz; z>nXZ&duc1Fk2;76)X|aFdQdUC#u-)+q$b8G`aUayP(O#VTuh^~vk|l)erwA%_ z;7f5T;TP=NEw8LhePZZ2+ulvYK8x&dJ@CoG){IwnxhcWZK13)ms{fr!M5+nwtCM%5 z;FkQ=uH$3rTSkaMAQSGZG)wk(D-(eY@tFAq!Q=onFR8RN^6OTK^-{ zFA~cg`nfjD$k5bgTT{n}L;J2D1JQYPAsc2r6sxYHp-uatYWLTTF$nX`c#fw}q`j7+ zsb#C;1~TAjl=vz%d092~k&UwRt^to%*z2T|*8jZrjye;Nemba3laS)PEGD*b27mw)P5eUoeR2hV;a&RN#VLm*w0qYUsG z(VGpN9l>}=&3n!V6>+Vw?k6hOf#QKruY-`sW(E@(PoBb7<&=2vhm+KOGN1n4WX0U9pl5wW`E~VU#bx`2A7P4k9BU>LG^d%GdO^&DHOgOZoYS zpuzM%4UjcNoOUH@QeRM-dcEx!gQfw);StbF7NHwjjJDxP46P;29}3*)Eszg!u5rrv zX7Z>*c~T-&t0cNk(-S1g>_;Kb81xkLf(T$DL_2+ zq#`^IoLh}b?T}Bx@E|Y2bH!2QF3obI1pUAF%SW$cBQ-qJ^E4&_B?g>{7_WRM2ZX4s zRug4wtrHkuhKHE3s{GoLQLym8!F)PwgeTgWt^ze+vwtPicLt7{njr z;N@at{M_NQw|H`H2zl`fCXBgR$y3195?&U9bxVj2dd}PvB0&xzirnC*`eglyf5JKU%V|b_I9}EG zJnC~7AEV|i;)7wa+A1yfw{s~{E6mY)=A$_CKoY^mw!Q-`C*}FpdiC0N0i$RK#+ov! zk&Y5zLl&&`XQwK3#B-GxcC%ECk|LH9XXNHny2X<@INVql3d-phI*?D6!V^6|gITI+ zu2Udn3jDuMARIMcH5ja|g0*eyA zI!5P1p=>}_0Ivv14nBE5QH#wMfo%-DD2sJu$|ggmFn6Yhg+y)0odD$8E&{ z-RmQ8FwFPO7+u*^X*3NTJO&6Q41YZ}KUKHfgFnhq5Tm|{5h*r^$B|Zw|8lh*A5o6R zEbHp1?R3DtQVYpYI?D-Jia}B+l6rrHYh0ZK+NI4Gi80_53KuSD~TmNhIvsm1z=S+w!h*{ z8_kiXcf5La^84i1jgy1i2u+37io4DPNN)j13PE0C{>EbGeY+bWXA`b7OF;whLZT&c zKe00|las;N7MBi-*Q`Ah6sR$Ay`fsRllC4rQ1CWDFe^tz+u;`K7a0&1S%C~y?ya+` zUYakcFRol?q89|aX<9OLmLm=$48vG$bL??RGkODX!i6_3bPJbmG|wm_@V|4hVyfQ0 z7Zi5+F#QlC06Y-D7KAiZ^7Qx^LYmoAM$qmRHFAf*Tf=u}Zo3}4M&f^ZtPQBM@uQi$ zs!0VVP*t4#NaS#{4vcE^drcb!e|VZPFv+l8lNGjuG z_)o>9*6hrYG)CPrk)0>Rb3MR|7@SmSTGB0a@A z>|NJT-jjMtZM?KL4qo_$HiP`3K3vM|hLwUItbj*NAS@1J*<}D@wML{wK9*-Y)n% z+I$ufz*V=KaCzG6u%~*RJ6{6@&9D^rH|XZpA15AMo@O5$uO1KwQzz+iAC2y-lS`~7 zKT{UaP?!ztC5q@sUQGOx)>zl6ahCrrPg7Z}$6$ViDl%dVFpZZu4pO6{f($%i^z<-lS6Tel)y*UrK>vp?a_+Vg{^x+@SsqD>`ke z*kl+ytg#!z0nD_V#~0fvV`Pm-v{b>eVB?Sph&IYwQ&udUz^Ot=+|%Du4pr%(##J#3 zQsR)?W|C$@azymuFVQH0lv}o|g)=7%ecFh~k<3`eO@y!@jgx9e5(j9214jZ> zFT9DG6bE$GQ7jQ!{Rx2wyL|4!8_0X2 zhgdSiOkm-e%VyOUcfzP%;?gfl+e|OD^=)O?XH#7k)A~e3m5g}c)CuRvZ%h%4Egu@= zoJ0*E`g%ObMw3Hf$zy%3DJwE4N6C-E4GZP0KUj;DEHLJ!6t+Y*`Oe0(TaZx^jI3@{ ze_?JZTqzkD5g%v|N)P~nxm<}@MANDLg;?$Kq-x!|L~r_2a3>6T=ZS^ikrVmAXp};< zkO`1F^iMwE)m!6NR$eu|W|CNA4ba72GT%=X>iq&EjrqF&c(Jx5OF0NS%4qyS69@MZx~t7TR3R|v_5GS!q(>%Hwbrh-hs zA^f!U^w)xoE)YB88qaqebeppm=29Q)m(@mv$3ytOos%{bmt>O=(PFqRFpyY_Yvf#q*ocm%fcm z7s)pVNTN3D)O8<4aHdctG(g9hzMtk|9u*wgB4v(;^*?ahllgsiGLscgW(|>dsZFeq zsqh&XbDI_ttQ93|w+J0cYAHiW(Z3z7XR-+wn1;z`BJxj z!esgZJ(m>gNA~kOk?F>JK>Oz0@bR5QFCYf7>N^pN$Z;_082PAWA7z<~n?tPiEYYh_bR_!_DdiJEo{ zPcc51NdXpgH)NVIlg0`rTZ?pAup!;v{g7dLQcNUxkiSYS{t_H$HDq!f0MCGlmGnW! za$kt&!wtFT9RxAw)b-T3zqGWZaZzM|rc9X+_4iw$FFB+)q61Zj=9%sAHPaJf#Y$vY z(uyOWdj1lMyJQT&Q@vFv2^R~xX&^5D^jtTC+R+)*9Qi&C(90zH8*P~o)mN?e)x6>j z#nk00c!odGpiOSXm7aTjUA>MED9RG8Z-f&ONqF`dJ9P2*gl-l#k&1Y-4pL8h^@l%IE7xuaJI^P3;5mh{1B;FWZ z=<9q^Pi0ozMBkzezm^b(mA7)D>l@d~Y#$3IBYRBWb7Ib=Bu;@VWvR0ZfgB&L^Rs;> zfjN|aZ3`F>Q5EgSA4~B1PZ9OMt*fQ++_1*v0KDLb?UEO|r8)G%W^+oW;`yI!B&Tq+ns z!k@^KbLXZ-`CX2|Z$F5kC~xl$`LwVQ6FbMKT-7X}M+K>Tn8YEq7A>+$m8hjqF)^U(e)NcnqB6&z618EF>nGK9;X9R49>)9eYf~y zNVllN0(#T%L^>~USGIskh{HuD?WI!8Tag~bFGn$L0zhYZl^GdBczD+j$L>z`xJNzh z56ddv^Wid`MsAj6Ecbk{t8Ho7YDW1n^pUn_TC{gUi4>m1vXw2I-nOIs?$tuK85Rl3 zjqG0H#ZXvorSW1szNBEMXE|!QMGyW-5fvqwmaQlqStEDrgm|?bRF7jt7X62+!#!)$ zA2(#-Xdv1PH6pql~r_Iy-rFj2(lhv|$Wa}0J*JZ)v? zssZ+mk=-VAc)M9}kpjf|*H^-D+!-U#F6`MbzeVjBjtQ>3bJ>x(Z1HuJ49p~z<%gB} z*)WXaBJ~-DJ324@Q zD7vRJ>ezwL^rjY&S$)fTu-RAh*)U)f%t|wU?3!!@z1HdNu1l9dAS8vPjy=FA zwrJ?tB0*am zT(^IzWy(~ZU4#Lz%pxmU$|@p`ua{n!?w)lh2dbY;^D-p76HAw}1wOxN43FZEf1zA$ z|CB!Ufc@Px$NdS?ljRLBX$XrqA#A3ttxl0W2|u_%k4G0JBLB9bd(&6@fRWxeXsrpd zMRrqUUFeKZX9d3$i8;#s)`#m!gvVzDB<@bFl-xQ#2FBVm^)bjTL{4uN{qhrJ#np?%;cSLmrZ;42m$WDC z^~XN@qN=Euv--Ylu zC*deh@h4ZJtkFUniNHw6pP#gNOgb8GmPg0id(nv00j8e927={Zz}*Y6-D0Z$X|WZh z0HRjVqqbcH-Tb4pul=XYNWXI{uS1B|Fa1279|VXV zO+GR4guPr<(v_v}tB%Idqx^WeviSKmF){J?=g>CyoHw5b?XqPD?bu{dW||$v7%u|$ z+mdbh*&g-RO(ycQbthT2}@8oKEdPsTI^$_|K@OQgyXg&sf{(p~c zSessJDPUgfE3e-}owcJwsm7Yh>%7@_oD4Zb^4XIZ_9d^A};4Xm6;IjPCgT{|yS z8I+vOL?EgIw9%h_mrMQ?U;l}ctvooaZCYuzMLk3R2Ep-ah3A=sc&oDvj=f%0gn(RB zgxqL{Hs}*^7ahN8SqoCAV08eAR&A!dTwjC-NvMu&{}!?Q+E|(%u{{N=#Gwi{!cLb2 zC@psUtGWg0YU72tQu9V#5Xa`=v-pODy!aLvi*`7i4F%~OSYr&Dpb!n7>zdJ&pHykp zY>Gc;e^eNIu8YT-HD3oM*$KQx@KiM0Oyc!(roXJT?T7Y5BVbm<2hA*D!+UdJN91>N z6fd_V-sokHdBYi-8!7_X1u!q`BCx>Oo`)to9@)Ua>9Hg7xl@d#XKYrid&o6-jpP@f*JEL_pVgCFx5WP~YoiSoCIvw^~T7DL| zv%&ANt2u=IgMJWyV!UwEXYCF7TuAhUPp?!f@116%W!aSZp$~{`~^wII&0}B zzQBW#?~hPtE=-Na$VK`3Xg({wmrFFpZv6wxg*HIYg6Y5i#s79fOBw!h z-Nz&K;jD%#c){UO2!hwBYQdz1E1=Or!Td`8}VpK8p7S*|M-qo&AieWKqFj@NW_IuhN5 zi|K$nt?8(1^2PU3OMyB6>Nd(Vs{*e@|7;9bctZXNIgHE+_@}JJU?JBhdGC}1P9aV> zR}VfshATWz4x5IOB*57fUz-G`Pt8U;L9LOWtGlB;g5xRk(2@ZNnSTKC1`Nm#$^aW= zp9xxJl1`w!?a$Km4Gx^Kwqh6mVM@#hE={VQPcBOu5soC^yIM)TWL-K+7)(uqfFKBn zW~em6VUc^@aqb|{L}e6g2GdBD`tJP^@Q4ryG~CyKTj3)5mvPeTcJbS%12=T@%{bIl zXi8dJO;;pZ6`|1wd&EPdQttc$XGx&&3Z|m|RH~QC^tLn_+k6T3_Zy}X(xsXIjPrfY zV+fnWUkk<7BOj6hQ>5VtQ}iF@*uc<^Zt1%G3FH~wBm_k0JfSy22ui6;z1q zKg%k~MBp==f%%-Oq`9}DkPzwDill-FbA0u3O`JR6x?DyIu@SYU`(r7X+O+_`*OfT5 zVI-)U_Y{@4k#>vW$Q@;!WdkqXoYru}TBESA7DW}9T6jbqB}>j3lVNZ4x}RSoQMnf6 z+me1vfAH*O(v#t~fY}Mn$qs&68*gUFB9-(5np&PB8@y8hVW%o+ zsIx=Q`RY+8ru&O&_HYg`m`a2Ouvgd{GgnSx{8!_C7ye-y)2P{ty^Stl1SkGcsZd2U zQC!~;hi{^Y*r%KdvnGe?tPHgn^7@IB5x>%3Z58tE^|LQknXgp>l{qe_ak?{F82sqg z)hA}vq3PusyU3QICUm|1RUEoHBP6Md(STt`1Pm|)$NBTRwRIx3=tB~+4UBIrn^&=| zd}l5W$xk|jY41o8iNkPl$xLIN+LfE=iQeOnj%d^@WZse>kzS1sCiiD3(cfAwr0s(3W zX$FrFhxUJo|DEHW6X$DB|VZ!bq~`o%DIXkRH{}f|BFP)@rr{| zlFHjA?8Ad0Wy{29BNKgSoMXWuYA^7dX#L9NDSE2DCow;2H~Arh8E+Ucfql#I{Fp}* zzCd~z2%>#>Bn9`=CJi(jl2nLBb8PgSi*9E>{!mY{g%C}&-aR0;$0>&?z{t8hw0Hw^ zoFrrGkMz2+hVb7Z`IHg&MtP$G&<}v(-qJdBtP+!DlL{m2K2D}kiQfRMU|~9%zYG&K zj!wSl#5+>YW7m6xo+D@z4TaOh2uKUbq8$dNos6D4#taT7z64%M5F! zDC-#tj7++p6!4w17{VsSJE?hckvq)j`4R1m*}526-mxa=TZX&p7rOSCm%FG^rJ+&u z_R;p7V_%jN)2d|dzPa&FRZ*lId_|0BVUJo17aG-&V`{6{+?_+yoHLIx*b|%@Bg*$! zD0|AAF1@0RGZLJylz`0&SdZxNX-$o@64*w~NwIWUl8R4Lgs9#~IRHzO9+Y!&v&`C1 z;m7G~Pg+Q36(w#rBEL$oMEBxVJ&J@xC+)sSrtbykhBpn~n{!Qup0+26p<2Qgti=@# zC7#5}5zB-vmoH4IY$n%TWtv@o_#>$yXU!0WVu0$qAq|}c|32C6-k1dtkTw9bfS_`} zHv8{S0*(iGm)&w)x*O9P@Sa3)U51ixwQa3drDd7Fov>fI$JAtq^fgjS0duou9iSlC zEH5jc@w;&)Tr{*ru}q@xvi%jpTwnY5vb|A&RzAtHb?t(ujP`Lb{`Ar~9_|DDsB5(@ zeRf+N%+wrd98YOXa+F3|&e<+2^bz#;ZQG08LrI!n|l%t|JtN5gd zyzg*aM$Uut0p2F$&hAIjSYk7?7?6JYXuxIA1s%)LgscRDGzN{)Rz;XCL0>t{!I3jO zpnqZ^a==LDl;2B%1;VdWJ>YKa8s-bzcV7rOjKG`xM&Ihi70(|}(4oUqw4o4}g>^ro zMLqkTawSLqVOoymL*6?0Qxwf^dJmE?!OSvfIzI2(WYRJhyzP}2$+-&==cZ(!fc5Wb zysg}u{Z>l$K1twcHo_j=Xfva#{OfNClUbHd{ix2o20S;QW>0*?>h1~3p*Q}I(WgQe zZRoAVb$(Z#73q91K(Eac)xaKGJd@Q;Y4GYIqJi^RT0A;)jO&W}}0$WglU~+N?4|rTMsr{MC4S zXW_De;^WR6YLFczw1f+7x3uG91v^rk>Dq@eW9e3Scj4VLBmEZ<9{ z_Ew<2QgfrKs^W%-JFhsiW*sfG;X~y{y5)|j=^m2~dRUA%cA69>7 zvB95}oNXIky~o_*L$DX)*5Yd9V)c3rHB;_kR_wVhGXtI)oF=fr>FtUf&Do-s;7X%t z+nM*hC@06Z{_FE8WQQ|+U{(+>1W@D?2WqP5-eIkh=c|7-NWQ8!vw$O%A!*vpRY_k2 z@rUirnr@;&>)d*KF#oA!)1`?!lI(U2c{hvM_*oWEczvi^==Fy~(-JGfCbZOP2iYUR z=bh|3H@`0{f~bT5gYQkhw|aJsTN(nK!z?d*|M%#<)1lW^fT4Mjo};aLwn1@n8{>mY zp4hDkD)y8y}YHeAgq4r+Ii6`D2-A3p(N8LGJi?LfJq3X&?Ln9IwZU?@6Pu` zOAyhkqT-7%2lg$zAzR44?)`cN1f4r7AYE#HwY1M(@TCn{|;0}NEwgCS0>LX>u zKoHtO7EwFKKU(wnMrV!dUf78nJUGK0wNSwn;zM7UV6dhH5Pg}LhiJe)6E~v_y_9Lv zSX&97;hu?G{v#a9azOVYNb(t5=D5u4g}Q8|smWp&N*w zJYiNE6smvcna(uGA1JK1irkUTka(9>XZ^P!8)WjdIsqmgN5>|ra*3AWzTjLlgl7>SCvk%0spAD?fH zAeSmCk?`-ls*e~HmB=8ROcZw1>4Z}RLk}LDNw4vzl!h@>3?vRmtPnwNi$BO_L~!S& z;`wRJ1W=nAN(2p*(bjI$(GMy}9L9uIe~ksp4Kd!W+T=O#B^|9&f!bsh!ZgiJbB=F* zCgU9eZ49ccA+u@KegD}qB~lqq0=~MBFhcYia>lTd&pObx6Tu*~aGG@dcTmDicV$ZA+y z^kJ6Ik_k_91_}%SOE^_%TxqjRkVLFeF`|qblp!=rXgAh9$tE9%?Jg~9@@*id%Vjb* zz#CGGx>VN`*GL{%#fSV6w3nZ;+SoUkBJt7om$HWqxCAZunh8zVPqzygJE$C*L$l_2 z@}4&ad~{Z_KcocCdh4Isa&ifhbn|+W975g4-KG)WQ6Gp1*q>gFC07_R8WMN&ML0Ud zn2Q31Lr(q*J1PH^S_(XHImQvoMh-{N3}xxxd;;CwbvC$@#|{#Sn`jqZP2T9MMaHY& zk(dEMfL)V?Gw6_XQi`D0e}iuE&SoOc&oSd5``BIKp(7B-@AfqcyW>HRhGo%=t=YGy z%VUyLt&9HEd4c5e(}^ z6twcSw+M=qYY>HglK@SsP*Das=13kUF+QJ+1jpUl8#qwN2Fn_@Gv%W#fW!fL-X-;A zm5yq7=ytSv`5UWj_2EhtG&&!t>Xd)sM+pm4s}5QJ3mmZ(3$iM;=L@!o-P_8f`K^>- z?nbZ`U-07BipvkWy6_R_v-~3=)Rl(ag%>12wA8k3RuF<1q1#yTp-XwTivE~^p~v6t z!|I{%N4ZZ(}UW{ zXxmFB$(1R@i&k;-%jgA)2c#YEzAC&HAFmOrhYocF*LO<&R@wPSdA}hS>jtIV>Y8{4 z^tLmv$cVl~9NQhdcLe);7>K)fCOp=Xyorcp9k8AlEggb(L~Vt_<;TUn$}N(*#VZD4 zr*ibWeZt%QqZ}Ai`2CTnW7a09obSC^$J>a;=*kr%tHg3#w(8E|GPJ}U>2z15d*@DO zx{!0}G3KevLM2OsJ=PquVHkN2?K(7OuiM;)2pc8ARm?!Zok-nlc6R!bh@foQr8n_s z#nwYA#u3J1%uTzqzWUN~zdQh7;3ZNHdzO&mpmXFM(Q`#?ryd(bSF>SVkl+*RMG(f& z9E@8=Q-s@ezE`R*t_mjz%>ASPvQH+Lf<~VmCJZRSK6)AJB6_0yL~iVyy23ze(AXm8 zL#>2o?g8-)kC%taX0kH1%!=U{3!n*=3&mFCXwuvdMf4%@)V45m$ z4>UbQL_8hTFv-*=mrPcW$ptx^09!uQDduhFBX$>#CNP1mDsy-o7gg3rq|TA>M9r+j zg=V-nA`xwAy6U`S?s%iUutzlR9gGnhEzZ-ViJod;tewU7hg$Zk(Mm%^#rP&XnU!c` zJt^K;Z(Spp%*8jFs|ma9hwDtqP2S%$S=rGG+#{Fdz6ko<1vp@|Wu7ZNm2Fm`k$mP* zYr7T=7y>XgKkv?C8d{4cjLY()sus#nYAg?)sE}ngK*i%Xr#b$y-ON5O52uiOP7gZC zn*YM@4x+jMPdhE3JxjI@&X@~yx_zlt5z8>iaU8SnD#(X?9!>MR1i`GK2e7_0(b2{2 zHLlM<#z2pFgZ`i(6=oJ=CQn+>LfXVV;_HEDGV_FB^>L_Xm>oE zk#@tiB?|Kd6Q<&B)*1bZa=8XF#7Sh(+}PJ|&AF_03B^(|vp{1WDh|h#cBX0mq(6iL z(kSDMp)yE?;oJHY0W`wwx-Mhv&$3Y{(I4Iw0#RZL!F;VVYM5_k^)ysBKpy%+pxjiA z-b^;HZt?JY&RJ4L$1F^foE}0PwEyu_DbpD7|L$@ zAGpDv_b}Qg7+XR2%mPBx^@;5+^(YJ!_gc`HIGOU%PE_X+e?y;jVQ7g-?DfdRn_HBI zc1(=l`xNIt<7CU@5HPOS5$Ir)-N{?^+5e%g2qDpV0%ul=jDr)G!{zhQDI8SqDG~ZH z7a*wVY}jBv?d2U=&aAgsg8c+M7m?}f`~530Hs`5I_zjCW@#S z3`LsJCUs_Twr~7ReHpEwc`AqLTn8;XsPaiH)iIdy<<1voNUj z#ZFZolo4TFmf;^W|XDy+gD=c(YX$tP2d`ndud{kzw-6By*b*yJ275R;TfCl09(%RL1ED3E_v(0Cyo`LCP2mFjS_wcH_qj#<_*;H&(*|+vYXwarkoz7cje8Q=^DM6L96ni*-#P5)4GlA%$)t&Qy78Po#sq4ybgMeoZCfn*@4pUzR4_|vkwYC^qrhY zEIZ}^oqP0_;~HCuYLOha0juu8p!aQxce!)yJ5pZQc4RGG`wzh3(7Qg({8oJdXQzsd zwS`VI5%J3!V3{-89rLz)F)|q`HBZ9>nMqFD!-#+NYjms=7(cCMq%mfQkM|TMtM6%r z$zJ_UmLz$7@|8s&M?yz?a6#O7`#~CK)sXP16~V2eO%kKmwR^%IA;X0C#gw+s%8Ic&YjEf|Pj|P-lC)DJ@w! z@5sANv+l4M1N_3yl}~nVdg^L;{a5Ff?Sq&*tXpMSW!8Cd1q(aax?Jl!rkkK-O?%7b zPTN^RcR|q*5B;NiBV=UxXn+GP=vl?n{kJ2XM9?^HV;>6r;zemPF179UQc@N>t}M z{|&4*s9uLBbRqJc?3v9=UtFD5}F41Cbr$u@#@QR1^$2 zeZ5>jqkR~HtILeAjF3npG^KU!l`;?kpONm-$6NX{fyc`BXe?K#f^bcDwfCSn5 z>ORR6(>r=Nzktqq+=2Xm@hRvt&S=kpow2DRc4x}Ye)DW!L=?MrQsU%Z?ymE5ov$1Q z^EDW*G3(%d4QZ4d&fkq`YM{Yhz5J^%2q?hbPRUZ^SW#QYqcq?`<4UiJcG*#Xd-}on zw4tRqF_uHJ+7bV_R+WrgSrT`mVt*W&JrTz_S5Rf*rC~aGyMyzKl-d4r$6~J<0D~Bv zW1?RuCv$J28(&}uCX`?IfA@5ZBfk752tXALP)E~_$;dGCJ&&a$@-!3z1N*Mq(j9X z2@y!uzYmYzlo1aApU-Lh0E$NFCuO2*ycr^1JWi5u6-0HnZsiox(A6qh;rr4AS4q_A zG1e?!67YCj4E_>9j3(ZizhO<&F9H@IN{uhXKxniz<7Qla>zKR;)MD|H>-ge?Qq#{A z2J|$HOpH*lg5p`g!uAqR>f6oFh!{2YoL#dy^rGvnL^#&BYlGE8uE~{Pd8(!}GUm+^ z6yqZ%e9R%hNt#~N-ati(GfBrhg8g{nSR?=!N^W0Vh#fN z9mJxdojJyMRM2>$ens0HaZWu`@V5F6E`DIK2*X(N5WiRun*GOD+5>*0^hQl&6`Lcf z)k_b>5=Y4ZL}c{{K!(50Pc0BVVaW6hTgdgM7gQ<>gIgRA6d3pzq=mi)KV7Mm`LpOx z_1`fcnb@ETtjhUVq2ANW^r&Ev7-XWh^n8rm=7TVS(1Tpb#jZr>w;^a9u+yX~g++H5uzSAdno|?sd&N45(Rjmv z18(ZbM>(O$O4`vS#uB@mhT&l)D5eqw`}wPBaLoaHyTWG7*K;n?VmT2_o{FN++}?@2 zC9-xU(w&>`fH$g7mTT~9lM0>_9hxSX;nGo_lvRT~%5V_?tO*eV#6fe*CMP+f zy1V$cypupu&j$((VxEIqN_SaysGQh(&N$wk6UeQpIx}?DjHMvz9LHJO#$=%+(rww6Yt}F_ASNgp-?q(L&6J zseq0#N74PvDPo(sr%1aX_>3Rq;m&9%Rqm=;?0^LCSPCz&LW&#pau>GvZq+EN`gaOx za_tO|o@YveCkl)cLFkAh!{kuTNC4(D{p3fv?Yh05O zh8;W^oGjaT=^tGJhl!MJCnN23VZoJiiV3(3vu#`&YH{4X#uCtuq8UCRhS998lekkC zmgUp=G7i-95(c=V3fQjT&!YJ%Hn@PfB(PI@2$Lw2j1;+?K$2n~l`45rvi4pP(LFx2 z*_%gIk70LRq)5yCuC6uxUB#4))uNYqMRPv_U zMI?gluHfeOafTX?PlSFl9(^{^8T(oRIEn6NzQqW6msBT1`b<)xbCs@=QL*IjpO zY`=j`R|9Y7_C)oiO0=n`>SRBrv{o6zvAbZ{o1{7XFQs~)aji3=<#B=C#KzOzp5NV6 zD}HIw{sPX>3>t~Z$AhzxnGq0IrOnk_7=7j2O3JV`9%~X~@#bc7???xWGpq62WS%an z`9t?GI=#zdFu3XrQ>R?e*yiN5CYcEei(%Oa7ecII2M5s}bx06aYdkf7{KzlAsGv%7 zM3`X7IBKEDF}&~b`;jPnG`WF(GOFgI5Bn^F5rfaXTDt00p=%nUlf+pb&aiIE(6~NX zzBbL3>aFWaN%Du}@>5$>n5Dim;rCM9>ty6GKv{83<{}Gi>-sJaf1PT5EJy3M!i%hy z?Sy@u#mF%&jyu6n{qoXPnADIdK&AYh99^1nLzdy9ByDy}5m*w`O|wB;_CRk5oC%0kh{npy&F6@)_7}T!#CF~x zB=0zE$+g=F0wnTRm~p`fS_j-x9Q@}&pHD~=AF{G*@HMa_knIK_Bpg4<;?XF2N4yPB zQTf-xDYUx%4eOD9gRL$4j%qS8+A%dc^&(Fco$Rs<9yG&gI#$UO4fy8x%7RjoKsc+% z8TNE=f5ptIznsQ5F2MvocpO6wKENf|R%(;Rg88`Oj#zF=d$DF*)zeA`(^$B@9O9(u zTe6_lR{&1!>^wE|`shjX*E>Py#>fw*LT|C&eAmjon+PpJvFeT^);G$HP|Id4j&Nm` zJHcVUd=gz_r5eDdx(FS<@pMqR6(!s75&z>{+{~Lk z*XmkR_4Pd18H@v`Ev=zvUUJ6G8^j5eB*VZIo6Bz=F*h-f>xL-XujW z>L@0RGF2upB@ApOb3ki;$es}?%or%lSiEu4`>(Dq)twBHm+_SnoMZc)k6ID*$y601 zqmWeG-;$N0Pms-#5w7$)HFduEA5=ea$z*M$+>}|F>hk-ZnG4cZmLcU*WZuz9jqMP& zl@kvKUh5)=pDEyL&uh~HR~e&B*+?g{q%ad!jvn#~e<;Wq;pz8FOGcKR_4GCB!waTA zN;}=XDZTSyOB5PY82o%s%lp4%j!6uf{~Hf2)dHe!u%SjaY@PyQqqh9VeCX0+Qv`0= z=kq$3*=A!aiI zw=9>lci7tf_$=S^w3o`&crhCJYM=wUx2o(*v4!c%Jfz36qM(ZMheo|C2JI_A`k6WR zJ6DaT!^WcKoA+?zcGJl~;Bxw#rxS71?I867-oo{4loX>mJNG@-R!bFo!@;3?5X@?z z{g)79*wguY30@nS8w4x#&yMHK>Oz8`x^^o6U~1t!ONS^!Cirbr8M9*TRopt4|v?)@G^?aHKO> zADx~%-|Mkw23)w&QFSOcv#=UnQNfnL!01hD$I!>Vy_zi&e{A>~i>O}jp}i!$nle)$ z476bK-k04i)J>NcP7h$_Hy~QR0>#rP?cHCyBO=?W0T;g7TfN;QFxC@_yvtv*HX`;* z+23{_N_z@8X}0I#kz|}H5$s*VZ0sH25$ER1iV<IQpadkkVPvo9y>H|~1BI8HTEH`gHGaL2_KXbN(1bHB zBmh9G$LxYcKRZ6D?&IIz*-Hq zVjgLF;TL>eU5OYzFX7;xI zjw>$og_?V{Dg7KI35R!$nMbfuRh6hvGELfU23<(Ccrp~HuuT3pXIXV0v=m8NtiOnd z!?9#TwWN7s8xLbcU94l^14FKeG6#PGn`qw2eozH?SgT)!#6oxXauE<1PLl}p?-z)y z@eJI&cptMUz7`U)eVDig>#7+ zVjJQFTx`$pw%=n4`i_g((R&#LYNqn~G^K0S#Ta&AeY9@`LLd=KXz+w&UPP&w?wO^P zdPfR|o@snQk_2UgxjA{$oW075o+!A-)PJK?*p&GbyAG^czEe27x|3IX;MT)YIG{Sf zh~De8!_39aX<x)7lHTOHhq=;(IkP6ZPc7}H2wKm*XSri2 zue||x_9R*4Ho$J3z=M0psYNS6Ncl|MqXv}eE6DXIq*$HQ*O%kmH^h*gNarX*Lg%hO zpBm^<&MH|Bb-n8SWiU?j0+pVuU%(?jhSLL})e@;A%_o@>BbF`>SHdkkgX;}USXG~P z#}aCOodak*%}W1)_;Rf+61JfCl{86+6q^NQH2;*MDZhVejH^C*1RWN5K!y{P5gt~r z?~qW4<0h8E{H8!$Rhe5}?8qX7*%Dpn2G{pVFzWz;O8G3YDjRU+=7u@Nsa{zbkZ))bCNjdM z&f;Xm8^2)-wx_qzgJiNXwka2d5qnH7jG_)@UQGxs8)vezrZ;S2l1)};NzHhIOn>mhNkVI0 z31R#(JTbD|Z45p7R8sU2@>I;H`-b!1)1iAs8}ml)k(!;>T;kLBEcr^#2k7gpe()cX@)a@9hQ?>LL( z)S-U9MY?Q2K~fIo@=LwUR%>zf$zH060rD0}z}h*7P~6Mx19fgals53CXN}LQpa&r^ z0&=lAgr~#&q)1V!GtwNIJFpjBxwxw_fh080%dgEOY;`;1g%ge;mYpc$C?oofh#6@r z&0)fO>m4BEi@nmk&CH~)ab6R93?-x}hP{SBEd!wlIahzuAr)z)(wiv8W1|sHD!}KL z=nvDtn31)d?{vV&1&W<>TH0TQra;vA4aRT^{H@0XRwgmcuG$hRrK7zZub*7Decub7Skj< zo)Bn#1$WHiGRpEvIBQ~wr_Dm${;|Ij}C=!(u1 zac!z69b{XZp>EvvQ<*NjgkOvw)tnEO1HO)jZ0VsrDCC;QJ_jAIx=7))FNQWE3 z>C;QH{*hhHI8MljKdAs&{6w<<8RVUB-_XL6DQf*N!|Kj7fDh9-{*nL4s7tmlg|ELc!fUBlD^ub7A($%&>7*QkvJF@N*~}9+)w|^WBQcO24D1w12G_^#UWv z$1Xc%q-9j3<0BMeJfbUS7`J@RnDyvnfqrECzJW}TS<4m8(|e#pE9fSVV);?zB$@<~ zqOt}9Or8z7k_)s6rHx=!(LUZ2j>245lcMLYV;0J~d9MtI&x5`5r2u^Y4r$8_C3Pqj zNW-(#dir|4>f1?UC(xr5S!luLA6uRQPVC;-S4dOLxyx2sR+%5-D4ivUZ;jbq^VJRs zNHcm4LSdimHU;H>$KTGld~rX(hACa ztKY5Aj$UD{esV-a#4~ORBr@<6%281>G0PphdL*2W%Y5rH#olaj9QtZfP%ayWYankx zZcwuHwA~V-)kS*U<|e_M!h9Y+_l85Sq(>T*2zn(&aBOFQ;q-Cz zB1}U00K!#FU)6(as}33+yx=f+`==$Hx@z?ZkeKR8L*$A@Wd6EI(0Ddev{~O0Tj z{MEXDT3d8g)+BA)rfz*a--@_~(ZHP_I%0kD!MB8-f#eH$`y$8#PphUm?NHJ|lNaH^ zm@LxJegHB<<@tCYqcjdlP(HpmW6znAOWR99lyX00=LZ6$vj0v!zm0$V)dE-Q)M&T` zUj~=ygAy^C*~|#0q(pP>Fy&{_1S)8ppT46L?nISQLx)$oUiV{ znTB7mR?nOZ0i0N|T^wrhk6pQiS_fSG#PY+3(#>=?_}_8|+Od`#jlygT3ha7Jm%h;L zL~Y(QvZxeFX@+NZ@Z6;a_$^??djAN zy9VOQf?4v^%%C&ewAc`y1|%+3seZ$QwgLg(j`GN7n834Ygs!6_BD*LGYwKw+w&$?* zD1YytfvFqK>4+yN*)gZ*_qOiwgOqR7MO6iCRwi!9Sf<73trcPAz-RGfIFh8$6m_Eh zz^brX@(D@%$|s*mV0Y%m;(}h0)7f2`s3>IHpFXKbOF!6d_0-u$Gq0XgW^hg{w|Rse z1442Zg8E&Ik%SrNK~NV1ZELQrqA0*FB9fpg%n{3(+R_u`EUw}s{Q;@{mxnCr8Fije zf!_KG?4a6;@X!#Dl<22bH5IA;cJ^L{jOEnGs1l0SlTTNDs{B@=&^v+JDwmZ>V~O$T zwb;o==p8EiLo!^Km*8X_fuOV#8hhsG(e?M61P`0OyQkVyavt|F+GKVi+=qp(zHVvm z6L*3~cXvE6`%h4l>n31??Z*W>a7>cC|fs&CTl|* zqGXZzaS{nOMnZ6HOC$EY;$0Fm?Y1Q~4eI5Y<)*Ma9y?l^-{hv45;FfFvZ}Po=(g|b zWd4$u#2luf3DBdyp6QmL@fEKw5gRpPYlO-RmY_!-6zEjAn}E!B>7H5-X;A(boNSa` z<>G(crp9hS)GKb(fs~=^e<-?qFjN3kFjNQt0Q{(b$A3QL5Y0`;t#Q~K<;n+w8dc2+ z&Gl4krC!Na(|5p$p&t=nf94@fM!WXpk2vsRi7M_%i;&+0X{XV9Jn*b&!Um(3GSYCpQL+N(yr z_tWpiTDyMNzNKnGGi*GmexaDq8-BSpZ9|AXlKg)7ld&iK2KdS|bMswalCC3h zRVo_J&F_9`#;?5p9-UebOW<7Ohsw{btCsD$rK{&cby>)X!RhJF!%B5jx<|pLU~WwL z?c8_yrM_^;+{wE%SvFPb$wqBDb!&2ZV6e-|Ci+PW_T4v#dAg z?oAqppW`tgS|=ENCsfC%q{4VP(AYyE_Qn}l#M+EL*v}mdmhX}xMFu!5pL7b9SCp}eGrXt00(4yCP ztZoeyapwb80)R|4+epPJfjqX;+}S_J``WLA$LHytV4L8(VVX!ox(Dm)%THS#L1yNd zh&&pOnEf|h{|-GVnTJCg8f#0|kKI|#IJ7-U_-@(*1YpB>Lecn61g+1Dt^v`0QOA*d z0i3GAvC?)b0iR|&Lu z1TPyktQ%W9l!ibIK%%8X!9Zgz^pz93$u@XTM|uecKIq$aL8U$h^j9&NhS#Tt8@PJR z4mZ0v8z&FZROo4kN6?h76$I0sn2CeTlQc5^g=-{X7IBy~jqEoQzr#ADx8(@|Ko*!} z$qwthh6Y(WlKiXcUC9>bb4L`~EmqkGV&Opg=wf}Oi_>xp)Bi@!vt+Wb2Q0P%y9Tks zXJ$`{7V<$!^KRtW5J>4s5v;kkPKz<|Rl}ITAUqbnD4IpykifpiH@7suq|pSIzo4I+7$;GsT@^qiI>-5Q}ow4=qe?kaKqd`~7zdD~yGT^n{lE z3kQ+qKJ#i1WSzwQy~F(Tkio_th(t)uMg!=XClJVjlV47|$$je2JR@iutc~C!lO4Ip z;us1aVQCmFWImmn%y-0l5ld4CATc39<86O-_Y4{)wSi;-*o@UFpA}Xw=Bs|O1<-Q@ zbT>i}EOifJHCH(zpOjG%PWJ%*?^hA+%iS5UcL9=PVr6ff2n}=2hGn7=KxC~4)eTZ+ zx!6-khO-k-9{@u0oeZ1R5cLJ6&JRBX8GXDSCaIta2f0 zQ~zckKDm)`>Uzn{n`Pdq#rP8(PXLr#r9u3qM&LEWvlj$jKD*H4o5`1oazv{=Wr2QZ zjMS4jh1bO4hP&OQLb7~BJrG%1BG_CQgxKMUY`%%XKcN%hJia!O$*_1#Z$E8%wY*AntWPsMU{8tq zXjp%aO&S2)@xXaW96b>xOeM_Y`fUiQ_axC^dZqvu*<~ z*>1r@j8LMNm3o{fQa}C?DOh>Kf2vwjWp*6Kuf7@527E?i#(Fk1@BTu1u~BCZ)wwc@O?BhehP(niRVA(dWw(y=dwsi4Y*?1$+IBZW1yTvgDGf1 z_9*CoA}f05#K-=kmHffomcTAn>qj~>^9{4=(?H%}%Q08rIIQqg>(lWf1>B#{(Zw z0-eHQ58_gYbh!}0^KT~mN2D?&US)!`+f3XVoT^3t-i^S!4i9IdFg9!>&LA2m9Kx{y z>kju4{Ycxr_9}IxRy}2#6M_b;nj?=LL&|$n&Mtv8(7`*#%9GeAhyxT4l5vv7R~&z= zG`CqA1;dPD%F&5o=u@OJsAI!)hCi5vR4iZ-*Wzm7Wdk>x4hz`^$U@~WxnrGrH;Av# zLxj2UVZsVg8m7QsmIke2+Dt7Mp__IgUZfXRzr4iarDCik!D3_R75Sgb^6T+!ZMdti z+LCxY%6S|e{(AjPgZcw>i3_cNob#u}@%&9mh%ZWDZ>S%~Td7gmcjq$>!2lR^V%gjW z_S`}?de=CrbdI!E7EL}Au^Agc6CAciaY(E(sAfJx-Uu+}GiHE$q45aD8e&mHp==8-;h`z(pUt^b(KuKUd~^vNm>!K2{c>jI;f4$(9`y~aYw3|y+GsY zB4$*G3EYcUI?>sV(J3GG9KmT#bXy-lLG*mqpSwI%!qS8I!v1!sK)j39Q9+pc6p(f_ zIUK-LxbAVr%_dp?%0PK|s1Jt@r$7N`mQmeEfKQVXt%l`DB{v2zq76~KMY>7{8QtmX zd*jE~INEVTWtz%w55el1rD$roUtDrIiU)1WMjw&=^=X&X!Spi!Ai7VBwU^~hhX`Zn zg$t~hmm%2xauGzAutb-mn0n+4|3hv3wNyn9BEma=YaU4?$~-X?Ca7YOc@c&i=?9^F z4`DMRS1dlmcga5A!IEzIY-|ou4e-nIGVz&C`1pbra`Y=gH0vU?Kp?AKs=%8_olQ$Qlq*h#ppMzFa1(xt6{txg@Y})E)4u45jh#( zBq>2~ElM!k8n}8TsCoMZg^7fLIRgjVI2VuxZ9L#qpHH~`OfDPcsja}&(2{n|%Z)#@ zLAlA!om6r_KW|dRoca+-lf@WMfPzbroE`~+9FcwEOJ4AzT|Pu@JZ!)j&j}%LGXv?a zE=WmlY7mXni<`|*O?-!!>eVDUwh0ezjAbLj4GrPRcmywmFPXU^a?&n(4r>*W-qFEEl9|wJ9ib*ZGc=r?p^X5wI05#x$>l+XyzY}ea-)1u#Ig-{+RbsYn-*3U%xc3#vB~AcTSZWFloIs!E;rJY4p8HHW zr};|O>uo+zCr^Kfz16zqC;tkMo$|0gdLRfVIVIAtkR_2}B|MlxP@>O*SNalu?3&%s zh1lf0U&)-2BzHJcd2uxzT9)L1Qvw@d9lAf+@LNwE&DAv>6=z0I*gmDu$!T8*V~G+H zeo#xHNx|slSigYpOVW>(=j`IC&`ePJV$sxrn1ywquuF;6@WQ^RMVw7qjLd@KQ}w$= zuN0RIrtYT;DPatFgEnw=GTyoIBq>F=;orm>c;2+$1O%g%>j? z_&{E*&m@tti~!oE4M!=PbQsDaBx@~=)HTZ_tsA93p3y{+pnfpPKf}Cy{QOoJMZ2Rg z)u!*qTnO38=`nhamXv3g0LO|8%P)D&CJF&q-!Nk$$^KfcMS>Gn3pSaM$|QhLgyg zIllX^h&GarGdoM4EJ}+_U-T1A`a7zm?x#9>{sv!N8 z-mHO{w2D^bm3W?na$XXf4nYS=(h1QpLue1rN)|4dc2;-0N z1)ErI-G9kp0B4^Is~it8VHsUq~2S1UuQE3A<2a8VsKJK6jZ-0&k$}i;rnXXK9{3+DVmHVw6KVl&aF$ z-{HE@&iCMwLoFpew?%}$%OO|G$zJs^pD^C2MtITnFB-eBX_A=BF#fj)Iztco?ooJ<_zHKckf6>e|ba( zkQR97O8&DmG;d?T{Br6EsH_{XFp#71G53|4E_aSlersuajE&k)R6Ky(`5cw^-C9Wa z23zN<-e#IR+50@Mz)rd__j*PEbqmDE>_FW(-0g07m0{{Hcfc}=Ng+&pt|DVC$SLW` zYqHTLOa%0pAGm9y^ia`jkV|5@QyLZ3{Y03fr52Y&`lyPWP|8|hZrvm!SU41ajDu1f zVdMk66i#MQ)*IwjAMYVrDF~sT&SnEkfEK@{`uux6DRUu<|$SjKAR}yjqkO zy9fs3l}O(%ZQG%&_+OP6@iZLf8qYe-fO1DkbyIk;q{)8YlX^fggAi4<*rUml(uRT5 zjEZgEK;iGK_eimuOqxw1GTnFNhtppbWh6@W#ZSdssduDdeDC1g1WJaG>2Ddv;bg#s zoItw+%!w=1!+^Ol!|p?1eo=kpT5Jh}B534sihQMscmC-jaRG@YbgXxjUm?6;k+lhB zocSRayR+Qu_thA?al5hZsgvGMZB8mB$W#XsIjd&uh1*Tvn4DH79>$E zG7HW2`8Y(=w4duE0R|$=sUf+pLP6UjnWfis#|J(~FzzskqI$*lzDVbK8=8RRk9=Vr z`dQD+LX7b(vxOQ$PLf%TYxH)`k{EdZL>`W)S}AI7^=EL;K4uB0ph0oVHNMeBWSq*N z9#m$ELz%EP~OK~LDK2EOb6QL4V=p!&t~6K)yN^i}Vr31uw|p zA8GCtm4+@&Rbv(N_!Hlm>HV%IpdXl4}Ubqz7bOIm_bpe1j@7;%!>~Xli3_1i^zwFU8TR7!yE>B5zT%cKpS;3 z*OI`bWFM>>j9F*?C|ZZ@!W(Hiz;C5U4dg{+A0McR@~zi1rFnOO=e-|+G9KmP3LZ8{NjKPIxhmN*Kpf)iB(O0 z%;~148E45Nx|jZxMNi!ii32DMc^3!-h1jZcQ~{3?K%#A=H+3MhxuwL!a@+fZ)P<1B zF6CPxuS6HskSE z@v8+m#vPNOYD{KodXtDWBeKj3;i-cHP5LbP z8gT@qr~c>{hg_F$s1elk3$*lZI?udYgDq`^B5CDb7D_sS!R{*-#hG>F0G--8=CSBJ zYw^jAMhSCLFjf4V?8RNvDT#Shi?h;)<0wRG>(=e4F6fFTnlbB?i2(ZFH!)` z%Cm)Iw{=r!VfE(~i0l9xQm&Zl5bzZ2Es~QS&|PH?dXdzC2cUo^HAqBSclLzHAT}Cw z?>0ja$5h#YSdYlmV9Ac69n?B(g#lRWam9mp#(^yED>= zn3}&{WTciE_EP2Y;f6tS7C14fvosePNe_uoMQIXZ{mym=DW+R*z$$3$=;k4cE>a~E z;<&LrZLjF+{%z$G)&lfpdU^~xTskhRB9Q#>G-kkR4|9mTg1KH6-sjmpc}htQY#E;! z&eemjmuk6!mC|Oi@aU`Ufclvb-AkfPbOS3)vl$kP5KsJhyc z5SgdY`LNl?C)T>s>~@3xujW#iOgUVj4)Wx^302$Oi*Z@;p?ke+_L1HqH>L-K+h3#1 z-HWc1(GNgEK%q^Hj~r5xW5qi+!gNgKb&jwNj9iJ7Wpr+8qC4a>r|ahx@u7oZ<3;8C zw7$%F-8MI)X~j?47qtA1e)hY)^s9trESMh+uYW0GLd0rP*wgv}w@h~vA~FMa$A^?V zYpt5|SJ~W+gUS_6#?n*Vqeu#LOK~lqOj03VWunmnk3jf~b zKN{>U0x$ebvUjsI&t?+; z`Iky}d*zmfy4SpE*9+&7PQo-2>r|h()ne0;Pb2c7sLH{69&ZzHJ~Tcyr64z_;0Q0(u-9B~9|tV0-A-Gd;+;GQR2sSo3$zHOuItOi954d9%8&~UHXz<)Dd&gwE_Cm1wtdqtADB0Lxt$A*MROuh|ir!7I49;bS-zHKwzEdEn1^IYz? zj_WlzJ==)ywvCpODWESZvTfaR6gVe#ZpuFeCUPDJ@J2KNmTx$_DavLmpB?x@hZ~91 zH*+uFBh3Ok`yjWHcpqfk@C^k%V~mC#gRaa-?kAoWjt~0U8_t2|{nv1C6^cw7L7FG_ zvY^jBmr3QCT=ZwQ{OC5z`iR=klC29jg}6_5+{N^u;Jx}S;z?Y&l0`nCdTp`TXJRyY z$U4i!GTFbQnhSBeF&lEN{dDBuuV|hqIeyGVInyt6H#=VGIlMm;qKg<&sXXg(>yu{XejXk@XoH z7*`+*5>P4s+OV}i$zwwOYG1CHBHWF;XkTA>@aK2uZ(;&S%tbGOuc7Ag=@?IKp;~Jt@i^ytG4y^H(jlK zWw>afZ+~Qkx8{twqh|ThtLiU9J%=AHXh!Al&zYp!VM!+Abnp)f9R8TA{gW~Xq&hBH zvg#wNN?u56^b2@H^!K;DH>HHVwBqA;tI~akowh}LCa5j8=RvFYCOSSbb}w)t2DJyD zx5umPGs9E)iY{s}F31}V>TdkwKUw8#ty|Wa0!YC`xwoxf{U;YgddM=EHmGbqoMf*e zkMcC^9;FexiGwaIMHj6Mkb8OUtJGW*${Q)mtB%4a1D`4s6f|g(%PJVctYoa`{6}LA z*s^bhd80PSDdZ8>q*dBqi0EzGlO!%=m!zrPc{>X`E~qZy(3Kb@mlX~hk*S`9Z<^n{%qmH2}BY9ntY&6$gH%c%&BhwW6+p6IbW&Ku zKOelOzNAD*cs&832_p~EiGB__Eo^YNQ;4!G7`wOcI(SPxrINYHbTcS?z>BU{e(M3l zZJ0mq1V5LS>Jv+QHy9h1><*t|nvQ_1-iA3wnjxuIAY(F8rshID!F4Gt&pRVn{>cVr z2?eQmAO+0@o(y63)fOB_7w%*!Df&L;G*4(FQK6KcAxHYP+$YEJ#Yv{Ym%I!V>~oKZ zGyoivEX#NVOvQ|Z*oFq3JxXzu?I_noc%t-F9DjW!EzAT(*(6D!1f#&N1fJJbn4SYh z??rqqJH}uw9N)9owC}{M7jiQPcr?~iJND}uWjFzfsA7Pj&Y-=8auiMFfJ9%Xap8{s zR*qp^4^@rwYm6hIx={>G9NEp!hpRQ0m7Xj+IzTsV0!h0X6TryEZ_MkX%NmEG(6pwR z*qo0ROmy33xGA&WU<0J!lx83wqm&YkeWI_YC!I144@29=>KdpaiB<&R#K)*^2~C(s zj;KS_Jfwm$K^heD)=G@%hEAOW>iq_eD0p5MD%wguVi|=~Gppa)Kc|s9QHeDs#1ST} zAclN*yN4neMmpONjf@u$S4BoMA%vuJag&9rs9SD7FbkAGv%Im9O7viWirGk%C$U6O zuj7fQ9$8P2&1XDkEy8MR1}^jVL_S`(IVYB59i9OX%`|Qhb{5FbvJ4%ejDe?J5EtW;;XhaT*bTE{-(-$b2}0cY2tQMDp(^GNl{*+G#NlY5@&SF&2p< z^i!dE7I=60$6BE*zbDX3S*(g<<>%7IQJ@L(g+AqE8;selIVZZV3ClMNy)-e1djgN}U|r=0MR!Ku%1q{hmPC%*=;2%YgbG^T8P zYXX`fQ4`EcV}@uoAJc2DF_tfBa36d#w52yBOeA=}=A3sspouhAecWyf8!porb$TT} zx)zX_tL#znU3*`4j&Ob{bn;jjDzINQplFQ_z9!A{YfsX>p!KOg%Ryg1yjbzi90EXi zP)eGYGlEgXiy2l8v;U{i8eH#%oSAEaE!Jd?B)0qy+`*-E;l#GGg=5%fN-qr z01g4OP(q_?g-flhB2qL88g(3aUEh-Akb(24E6X8C3hk0<+ab~ubl)!797qd&6#oS8 zd8+#X|4NW94lOx{zZ>agUarP(vw{$W8jhBOX4wC|##vRyPQpznsKAVGY%HXmP0s)9H`?tuzd=B&CcZ-2*709Kgw zOj4@rvDm?CW|~igNl_f}nrbEv`$=}cu!vvH-kY2s)2B?-`NkjJ*`^($*{P`xvBT$J z-QMqgk|urEeK~C1%qGpWC2{5LNqtd_sR6RG=uPY$tDN3pFoHEbsu9-7Z34uq^zKKK zM&&)a*-}AKm<-EXfX)@q84&Z3*HWE*vw#A3VF{_i7erV)c4ESROK!s!M}?a6#*4#Z zlQOHp9Sym}cp*MV1HnR*&bFDAoDVZbbPXhdulSWv5XJi=WQOF&^^hcieLhtqpO=ph z@0aBzG#2}ZDKKyTu$kg$l+)pu^dc{?_4Q+IL_iBQ?vG;%$Q-BVx#JntG2J8UT1*Q8 ziWr&Lx;H&Ogmf~Ogtx&Ui|vGE#d^X-+Vikj8ybCd^*ZC^dBqOS+3Or0`IhL!7i8h}^}trik_@<&CDkttsqw=S8>Os0CJ`C}&A3vr?<5TJC)e=BJ|0qW7$s z2vzwOrE&VQ0gNx)gIk&E+Xq3z&b_^i{TPj@{xO%3ORA5!={u3r^rka_ z^au%_=y@@g<2^;Xi|vVFG+xlk4^3XPd9-A@nog%=58l|j&PQb|Jd|ehg&6Kh&v9X) z)jWkgJdVsgEXfF)V?FbBHY|rEW{d`!q}V`td;}!C5cuf}lG?n$c?RP;9AMhWOeexV zc7&A-nj5U#)0V!Yb9rQVqg+Y?7t3s~b)){)4|m${RN&P^z*0n)TS^=X%p)(pC3Fvxu3hu-;X)izxXx(8fuR+JP^qFSC&x>BMIVE-g3 z8oWWg=GSh^#KW@3RTPHdpdJ(q(qodD1y)H5z3J_M3Y5=R#&dm-D9DRK4Zr$2kJ z@6vBWP2^;`C@)`Kn6hz$Ci+Z?fwSs7)y(aTKKlBCi;HLDzH=96en#K<{^d6K;T!y@ zzSnj&_I=izn?h%Y9YHWR&X;7`^Ci8CdtWnKhX_z$o2qP{Dlv8Py-i%{6{2$D%uX=b zz%4P`T_O36IH>Aw0@GvTU^h(eEkr2NH!9->n8q}(Os-??Q~V^FY@i(Ig{>NA&xD0%V4B5=V? zvLxU6Qlps8tnu+I0#z#YXFEb;eODv@x}-4b;<9KHkmlFtP+7g~V2|)Ws(k#~@`aLk zhWAxMYd4fzl(+c^h4luNO1F03Q!_GiVuQOQ6>DK{Z&{%ySHtfp;Zcx``oM$7!5snM zdA2j_Z77lkp?H@t9^5wx;a2`6jk}T+O#S8)q=R3K-u?T|5N*|yKaj8*Bc|+OiUSX=oW6k9cvwgugCwQC0HY21Ofz$GX7gOy(+h@ z*;yqxBa9Sc?Y!!Exl#l6>*475ujgC=vH9xIbUWgC zacTP@BYoXpdEx2n&oo8?j{0 zU-SJ9@1t3L3-Idp*+Q>*lyo=(zd3)`s@SVSh}P{%FS`IgsY`(6kk1E9K4cL3NPF~+ ze^Rb(UjZeR^!|AiU*0dIwQ>j)rmee$Ix)S(H=sC+bhQTk_cU8C*amNTlK9P3FH=x2{2`+ta??Ytxcj+y)r zSF{-;y=A95KvS|2z>#XvNoO_p^@;;{YeM@wplYClJ=t46v#p}z1o*1e5FqY0T5S`t z1&~K3>q-f=kf8w6W~ei~G!2J%ak{_*h=EPW+ng=d8=btuxtW04y%HfHZ%Da3Q~ zhpmKqL;?zl(b?vmKyNs7pr_yMqLR#~_I647x(?L2+R;C&Q%H;m_a>P~>C=PV9qt!Xe%iwbnI0hXU6^)RJe}U&7M6yyD{S)K% zROj(2MUvevPuWG?2cG&r+l=*Y+Z)VELsDw5j3}FYP#Enu9dPFHg7Xp?LtmAB&6)CV zR9^G;JX{K__?cW5z8Nc>!rP-v|pCVGJ>@dFrstYh))6ETtS(0*AW@D~J; zYS1q!XfIZHg<^ZWU`fOS4?drAWUg`y0qUMG3|tndQt?qnT?}Qx;>9#sn#=!6@yBVW ziK1_piPupci#gr4?bP4QCIILFSOgm^IKRO?KdX?TOmO|x54r9N3vw#u*8tC?)hYCfVL6ewk!6F+)ret-TcEbST6BKt2LXjQVUvOoy#F}Is7&F!w^mePj-z~=B z<0~^I-C`Hw(koJQixh%Zgs`j;5Fi|c=$)f27J<6i!*sYKy<%-lW;XS((`Tqhr2z*q zExJ9aKya(Yf?#G>#lwMU$QBHGiJ4u6gbdF9grCNf91D2@)!_;p@%)%47Dj8sf5m0! z?NA~fppZUe%us}>)%&zo!KcF-Pk&tkadPeY{45R}YBXIkM;PI@?@a);lI2=(LQ5T4>)dy0h# z4HVw@aqxKiUUPa9Mi(*UFtIt-^wI;pM$$$qO0;q0d>N+-aYJVv?_uRccuFiyeH|=a z#$Q>N32SQI{MfUjg6wc5O)ruO5ijFm5_Q!^56t^x3Xhdt1Kd}GEY+s0^r|&jmaYYW z_E+8CKDjCK@>s!*b>-_?s0`MFC7cmY1XHTiMTJcOSb~bI_+vM<>QJ1Z_EKmHk_s2f z%p~T+lN;9EOv>YbuwiQXGN$vDAE2a%myhAyUkeFsYrV)COgHiFWqi>|t)@dpm!Men zx?%tt$}%$CHTT}tY1#ui%W~n(P7dUK02;7}dv-K>tre_MYH1K^qE zYqlBzqJlcf#KsqN_Jl2Z9e|?=Cub}D!$m~(>68pka`1;^fHZtJ=0I$z?MH%ur`n&f zgV#-YqI@cUIulm^_De>%B|vUyBzeq8d6u?YSKWfXZpl3LF!Hiwc@sedBo)z(3K>nQ zg-p5w)$xX8QCtOl4n8Y|PBCn?PyA?SW=X+-A&%2-XHy>#1J7!oc0o-{##9-1*2aPSR-Aw-@A5H<5=o#r=&#C*lc9+NZt5%BJ}^!0Pu^(cA5+JwQ;|y`^Q%b=&0p~t zkaR(Jl=9EdSv=QOQa&U7#t(~S4*pIRvEVOzj9f!&P z1y63nQRY|6fBo?Fa4~dKqoeX9qi)yGn-R+d-AqRfMjXxN@k6ktNXZsUKO4DYX9d*ZbIOV58;kO9zSeOFZvP8O)0b#)oFcA0H`66GdO zVftNEUV;hNH~ibzbaELA*5;@YIqHQmjjv26JPzsFE2S>^|nV25d+(sQDaYY!MJ|K7P^KL?pP7En!cJ%p9zp zI~G=Cy!VO{I9rs(KE}YRw{PBs9+0H_QnGM@b8mmH=T0-!d3jpb)6j}Q#pkPJwN{Mh zhZm9yD-)QQ3|T#XbG}L}w*O}6?7D8GBhu!T>NH5Dylv2b3&n@ca`3DfqPD>P&h6~u zLREMY5Fmt6;60LUhDWkrY>5kDVr6$CWEepnXP0Ey1=Q32L*B3VAR#(}1fRHd@C@iq z1t?F26We3I7@37Ta&%e^tk$8BrmcJDj?jkmYtRf5Xa^$uxp7O|o>O)J4ns!0Nz@zt z@-zr+?v0smlHJaM2CYWafqu&8W(8&6W5#Ctpf+|>2HAQy#tgtZB)Pkxl*b#) z$3`j3ec&@u!3HuGy;RNJ=;n8PW(g!X?xHlAm=bQtx`V?Y!f;FA(d}^2X2Bp!cK7#0sy0%bi}So^CdJ8tE@I86EQ%B8ob7E;QZ;L6#Xm>L!1v1t zc|TMA?%?Fi6U?XO{6-0PoHPf!(2&r%i{SS3Thu@KhxpZFvqs?XMw9=0Wn#B+;#+Bf z0iV=m1wAFp`RygG1fa9;u-SfDCpeWT?fx+&JA=smXVyh0f~aeemQ~ z-kGH}k=<;oT&QbeN+n4=9{L?Vm#yPV5uTq6kN|Dmr0Q}lGK(H**BSMsRuCy@{B~mU zLJBphdx!z}DELvv_fc}({v`@gPN;Ko|L>a@U%PQmuWW_*@ORu+tSx8fi=RW5{wFW_ ze_gKHdO*}*4%FX`)IeYy)Pw&+>G)`F{|Arq{11;JyRcM|#KJFYXqvR*a&-pk*+Abw zFCa}EVfAn7oGSLS{gVV5=L;rO`IU-M)RCcoU;r@GkDrtC{g(gdCHUuD$)(9^)MkTM z`8m$HK$G33r)uH#*g!&9RISrj_!rZa^18~jMS69)fENb5(<0?%d(J!G3vR06T(irn z;}Xo*Adf9fXIt&bQ@08?47A66x6GjYM-tuDIgX~SF0GA~wyAt2toIxa_P6%3l{&WJLqBD-83(=BC$^?n)jI#$`*&d#B(*D+ zt}*HJwL3ZJtdu*t9k2U|S==w5)fM(TCx`orZVm0@@PF7k$M(#kEdhodCmq{P-q^Nn z+qP}nwrzIoq+{E*o$0wVAMc+y&slq|s`6SD@3@%L9%)FSFo|BJiE975a#@CB)i78; z7XB^UPsBBj)Vi<&ZvEE3!_MFg9wOgz{aT?M7jola6j|?i0>4+{8`kn^)|%>{Jz}{* zV}~4KrN&pB3#(DbadZCHH0|anUHW(SdV_`|K`vp+d}-b25LAV~5qr#5x#+y2S~Mpd z9wA_tO^f+*p)o<)cSvg95~lhM)K2;#1J8VwK;MJM(w1^!Tqm!=ahubFW0cgIpYg|o zc_(2ONQ5H;1jm~YUgvh~6Kx&btA8FB${mZAxk`B<*LOeXoH(|DEtSrQ8Dvo?F7JWZ zgSu5l?k=mgEAT7S9bmG`L?_m25FK$h63*tcRXsL1*@I2w2#BrFVTIL}>o!=~68nMN z<+}^TL>){&-)i?s2itM&uaH^vy2`@r>{>sQSU4!LM@cG70dwNwLh}+^)-S@*bV;|V z#KXIOzPuH}60Of3&c-DGrd0G~sGu$CqUuwP6h)-m{0l2-#}D@YsIuH$XJ^BAnZ*wP z7C>TlTKxAPU?P$Xh}lLv!%Ei#5E0K?No7V7ZBMoM?%%YGGCRIKFsVj+v4UMY-V}j1k-8 z-}LDCgr>m@Apl)*XTe zR{mM6YunXuu@TyEr$CUf6hg0?Pm}CXn}KA%>P$LcaXl_n)fkyP4NAX2jmqVori{cE z^DxW(_pG4H?8g&K6-P=PuXTde{qOj*EO-j|zohQ;6g7?#l?V|64(ymp1U=|9Y@GSn zRO*bnP1tICv4><5ekox{J5>V2P#rogQn0m zj+kV87nnqU20sT)ey-`>JLBTh)Trx<@SDy6dl;F!UV9sq`7+L~Wa5M93R5vm!Kdox zzASc|wTK>qRvhc7$XjLt{p`6)TPR@wXh&nHGjz4tUXa(QGD@BYZ)RDEqFR%VaUA_J ziKcF~^mMagR_4WJ+D`GEUEuFYr#^X&zxcX>BVWeywP z--AAT2xEq8#r@+F5Sx;OCz|xd-&&92K0%U3*mSRA&=hK!f5sR*<^X3yKC>!aVSJ^* zgKc?&Hz){=QlSX_PscsqVd4zNzkhg+okRLDfxTB;o_R=Nx|~0w`p{{>7L%^}>)*Om z;TMSRfM?tojwZQom*Y*rM^+zw0NU%f##`eOV{YZ1X+!nlpfsP`OlNEbjZ4y!v2ODQ z9@Z&d-1xz+>vPgZ0!=Seu6i#UwPsLY)48^QW|XmDwcGYm_+a>CNZK!eRzsE>>~&POjhR1R~K^+-TTUJBpw^yW!NsM;>0 zbyTj?N_HnuviW=ob^JUc+IR|6@9RhIUe&y^ab1V(m~d9fE3_hozrlI`Il7AwL+QlTM3bOk z*H$^V$Ggi{lL(3a zBiR)4EsOXCC9?;Iq8lC_ax9g4le&-JZ#3+(n%kgVkUiO;5j{uSZo;j;CW97oZHBwR zgs4QoP-R0l@wnAOt@L}9CK~T4s)C{l->h<7(zq9~`*_0#OCEpZ5l22V#=h7JhVb+v z%&3TvF?nfV&`^=MR+2#L7a>m^bD(w)^V=UV?%?GN&!|N~=G%Qt=OlW=X$`Whx;xA_ zj57EPB(tm2XXq5S@CV}@B+rUIX{DqWr-Bo+rlQ7WvS*ru=1n7xJpQT$?hwq9f-+AU z=@3jk_ZHNtv_3OH9%vE=o2ay3XNiPRm3L%gnwWt4dMW-jLF~L-Vu0*MIF6X2hsJ3S z&M!zH0%7krI=xEPJ8r($zmykgvrL*IdXpaD7fE8ficUKZNG_a9>nnVeCR&Ak(Ave{ z^piO2RC>+8QD!XE2$L|__jKL_fxm-P$ZPNrgeGdQe0G7eG|tv1Xrj3uvfr$^IhjIS z;^ZPJ(VppqM*i8kR#^}W7pIMY09v1JLRo`-;E&EE#*dqH^37UJ?#4ob@Pa64oOVC5 z)D|4piCe&GH~qW?T}@fvCX-BF2}PIPlxo86Z!Umff%U@A-jt=XKQ+Z@Z-{2RqhwrU zfZ*W<8-)xHVXvDvtz={i5LO>Iy6Y2bb7gBYFwBzlusM+_s+v~dfe&qN{Ojhe;6^`? z4r_VxKyD#+2B()+vsI(`a!Xy%!9t@wM24+8)=ZqrZrJa3Ij z0AjGbi=Z_!TwFd{KtA2f4?;`TB%V~a&8CM7V-6*Dh*WZni~alFa?-?hS%5b3t)?9JS05*&)wWGVC0+3F z)XlGbpPelhp?P8r3H0d9U7e;3 zNvTf=Sulqd`xzk3(dcY3jG-Ag8g;;GpFf_l&Dt@QN23w;FES}&dHB+d@9{i|g}fy- zjra@)XgGJ~pY*ypXgd}4M_HlVP(e;#uEXL#VAdpdtrZgI5icc?&FY1{+eVVl8x6`l;m%qL~;3KD7O_Q4uyUwFd=))QO#Nl}o^Wq5rd~5J}reC_o{_ znRBvjGct@+yA%RL=QMvw(9ejA{B`DrCdQtA7UHyelR+qorY&WrIiYq*oOT(*VoT61ZQGIeC0l^$*dLveoDkP4r;fq&H$lO}D@p8*f~FcNj$_R8$!&~6!m`;u>&GqsMrPam zC}E<_+*#y;eLYd9@|?ZfYJiU^@?4<4y1QP{D9hN0`;lGmZpj9axa_FCU>0BVnBEZB z-B>8eH>)8-5}|^cL6(r{WAT@Ep$X>Kt#mB)`YLwKIi8r=DETVBG7AdGd(U-+z_vTg zJ3uifjheaTFNqT49|qiFZ?Y0`UPtjE3aB!rdx2nIp7KTaL#?dNuw0n?0PNg^kTxue zxOzT3*E^@)xAfG2xl1>ha|f`iGMQ}DK~{w~c#g5xMcjkXfF^6cLd2X!2x%57(Ws+f%Pxs6o49Qr>;N^Q@Np>cx zFBFGLS^u%XFJlL~2B)p?{bGQUV(n$ z;34RodV%88dr?eK8%HDPpj>p8P#9yxV)|D-#)q0NrSuBs>P6HOeNlVQZPJo3KaMu` z+(eabsedCP$@Dd@_TfJmsRDidX1ByN)@4$5EE`^lb2d54Z?$mis%J>w<+N?(GQ2(d z3xsA_T9A3g3Quf&yCzcyfhb(d4lqM0MAf{WFvAIL z-7aP$r1t-+bz9{cyi1Twu*oA8+FN#u`WV&g3mn{5QC7BBqZ!^9#W`$z3Ac``+&aJu)^b_SVDz=Z*{)B z(VrX1o$e6Ldk%E)h8v5vjqBL!Bge29BOgu!wdFf*~10Iqznp5*q zMdIA0lq}tR{$QdaH7(@?B1ny`*d2~qH}VVFgXHarC6H?C_)@ED%ZT3i^pc=7nElDc z#_!ibm#?(TrZvE|L$yMcr0f}IMFCI%u=}g5n(Gh1paRoEIR;XP-5(=i4pTW&DG#5V@PWe&W`}rXMz&{}eoRH-aZOEZ7Loaw z!JbkrqwgMv8x_EGSM=T|rJCMc+Y4zD#EU?H|7R(PAn;q1rgMM)xVXZ8Hpp+^0_kDV z#nWmPt=U#U`|e@w`utUuyqfY`EovDR0U(G<&5h7!A zHFB&qiF|l!fT*$!GQWVUaMC@u%M_Tx(`i18CkZk4`Ev-x9qwE3RTWEGLE*-sqN8@X``KWPM-Q3Cif*8(m*`NI&8 zE`*QOL~UFnGeWL{HIcWW72J*xgHvr0ZEMY(x%M2%-i2mK%7qo(wol&5ME`|tYNIqJ z+KFqIKXX%fv%LWLy4wiRT~LtVTij=2Z`Ioro|zA~ivN0EfsND3KJQeM3PV&O#jRa5 zt973{GpV#(56}=*wMv(|AVJLZ$c;Rl03AT$8#v@6xLWe7x7z5Ul)AS`Ei^3tLofhfDI<#3Q88|k3uX30Sii<@c<;`Vs)$D_yo(+kd*d#1 zBYb$Z`oNyV=pE~3%na9N0Cy2&L0=Yo@`c03A}MBNCaL~OVTf|;pMksESFO|FigL#e z#T#|Fnx)8_EG-i__X!=w15k!vv5}cAF=YdV&abPUBvEiQha|`(!=hl@*zp`08Agl% znH>PV?6kyKnt9uf=M_=cvM_`oFR<44shKPzYl(A;Fz@~FH(>apOR{h^<#GaLS>=IY zoIu5j9a_v%l)(GYJqb2v8CdI54}lJU!U%tyOGn&d*LUa$Clgm^VUssr1G#cf;j&oR zQQmp@Hf3h|UCXqAVX85Q6&C*0IDVzx~&w5pn(hN@`9&TvO98)W_t{*w!qBUgA&sAh$mX?&}N^n z{O=JrsfH3ZpFE3a%Gyo(}p zj=5T&{Pn$GsEoKKyy&d(cf?6x1aSq#FT2FEqbyf#IJ_=(mIFw!VoUxhb8fkps`B*i z7E>O1-Gb;BLI!?o>O^WD;P+bG_1X${pI(LoY2`uE8Qm)j8ssdLIx$AmybV*Y@Dt1o zM$~@!VrK_Mz^`WR>Spb(R9uugFy)sG>Me}wQf_OYNup8IqR z;>VP*_@!d0sdGDSy%2Zp`FJbemZt^+4p-8Eoi*R8BY92e`jdq6OdYActK<+s@{3!J(kK9AD~tRz`DeaIQs@!ErM@pt9n9 zhDcY}&gcats?hsDgZYqf_H}maNGkI28M0KggsoJCrs?1`1M#c#}*|fubc9Ha24q4vy2+hvNCGyE- z7){KMg%xX;3%GD6)+6Mt>bzRTO6#`ZR(+9GPIJDJPl$1JBvn130}m9hf1QF=QpB+2 z*@xy3j2(&co0V!o>ZWw8Ol%ze2ET`DIc>IXlX_{Wcjpw=e+Uh$L-fJU(md^p*^cr^ z{k|JWO%qvdp@MdG-J#`75kr{#7s70Y1!XOe1&Hl_fsyNp(w%?<6p3$8B4qDYfZfr? z_}YB-$&;~;NmfIuq&sDKJ4wP>+K4vf;1p}YljptNj;f-~tS?@4f0wE_<@s4@YKxAs zKQivnZL)u{l9A>~t=-3Uj2N9=_$ANSK;#L>pHQ{@U~=M zW4h4dUy3zTD1RUVN$VA@lj;78g=q;yby@$yHLGp6zO$}=x8p=VR~(-~32;acOB;Vp zHJl7N?Je_fnCIj|n?3@_HCEpZBH&Gl!hNOxO=w$Q=e-xUL0+(aj{&mR@=d@nhMnX_ z7f5PM&Lyw*_!pf_Sw-hVEfWR~zOzzkm44dsyHz3n;kSx!QTY(3we}T8tf*4k<~!5u z;gwZY_yIq`r1gg&`Ogn~c@8k@Gb8Fw`&!aJr^DlaNou)hu6Ztqqdj{wsaeZ32|+?a zLKQ6d`cH$;5N()f1rc?${N9}5L7+F|D=W4KZ;(_md;5o<&-ebv8b_$q5Ozme_n}^)qp?Qx$0_hnpXd+jvpmt z$*R&sJ@M`Cgt5NI3kb<`xuX2!fNzef?xuPd(p0;(PJ*}jfdKdut6jwnRZTp-IHu|U zJQA}!n>Enw#oK&Ar5-RU^|@Zm?--qjePiQ2f~Q5OO6fR)Te%wPxH_zCT{BhltP?MR z;w9p)0;%L=?lEemzq@!)u+`UEhBjYJ5Tg3UExmJ%DZ5;OYQp|lRI*f`&g|ON*VNwr zX;T56PrGp8V%1+ZYo{%5tKHy5JtLRw=t*26`+=NMbpiia93@RR+q8vIXK#U1$BNF& zbO{O7NkcG*dH2@&ZmY=~MChvhGaRO~9_vCA+2ar4xms!S(T~n3W+6e-MpmXl! zdamUDs}=}lQ}u7?QRVpo$64up;Q}~TZSC$a1!2`+D5*XHqGxsB4(CAZXSkofTE1aC z8=iw)(9zCfLv7t+iZ~j5(7@J#(R7HxB)smH{~0wPacPB4gz(YZ5M4}Gmgp8Jkuu#%G zl5+-%(CyTO#brzj3v(Dy1GHbfDSV0g>Rb7Q%LN1Om#{h`sHZtEmx=yV3AJ)k5qiN8 zU{+EC%zaWhSHX5(V^ir5@txR@9P}ClsVZp`PrAdy`pOyOAY)7-O4pG~kmj6J3ym)G zGc4k!RCRa<*Cg*LgbyVUSB><3@pq7gq4o~AH*}WX{NL$B@Fh@|wZBOn%gKP8u+KM; zaX1?TB&An_i)Q`Iy+I4iz&48dX9cf;H?=nq+x&~+JuppB2&9yEQn%{u+)UFPxAure zb1RK&&6ehtOVjqZ;U6(bU<*uR&DQ!ce3o+OoSM9+FGUt?heFe_gJ~)I@Elw3t#70E zA%|PrBE0F?gSn7oxBVX$^lu4zgl*(zh53>clsl*sHXcz?3~*PklfPr1-5}V4%-kpvVuP>y0sd^pg2=sM}wy2%{d z7WLDM%gmV%@P%P!{I?4eozhnHcL{W8v%djO6;+>%%B~`ouJs{?m*5Q`UQ4IlH3(jk z`|*d1sZs{2G&6>r?Yu82u-Gm20xVOJdBpoq1uJNPog1(fD4KlAa};i@%Je9GvR@I2 z#o){wzxDmA$5mq$B@Caby#MqN+^=xol!&P+*K_JM9*Sh8EkFxInF6)>7=6Blps)G9Z!AjaBmfFvHERnHnnu;wu7k1ACepi(iBn zZZhV(in+d(4`qBHb|uX6?@~pX@QGa(s0#C!@0v0iujO!a5mQWG;HAzfITu5e4X?WBTM%eNy=x>Q3RRlh zf>~N=Y_D>1S!{{~~}-3gZ9_wv8BxhElBPgItPm) zhFCEs4~DUpJ1`o_$wk_78^SNaA0s$+GzUh;CVWytM2fS+B@as59v(4kc zg}HT~%#n^z#wgkan)Ko!-c32n=V9!x)HU-ITf%f?e(B02hXuG8TlK$8w%L$jQZ~jL zF>GY}xKr`d4hEL}U*4+QAFv}ME{-a{vqYJiM}d2T5iye@QM1frn$#_g5A3niW_2;* zW4&s1Qy-vZuUzc*jH(Mi`7=}q$~QYL9vGO~QanzYt|p53NghqcO{=-vmf^#Ss#^^M z6{|f#NJTgr_g(Oxn-$}@81Fm z^#WYm(NFTY{vKW5(|fT0d>x&!;O6%@Q)i$cQ~1fTObOwwp7E&x3dxd(H`ZE*|6LU&|e3S`aFE;7RN! z!vnNm91Ff$ooad4UTwgZo$kekLP(i(1m;MZowGU5aW z2R-#|whJx)x~xe*LGV1{oZ9!%$MC?Q`+K?-PYCGWlW9soE+pO&I} zv*A=Silk_pnlL@7|E}>_cZ)ZB<}{hD?wA?F@eJ)}3blvb&EU=*8CiGFPQ^WHtZx(P zlk%N}>j(uSfvXm;sivOJK3&5cQm{{wl8DrZ(P}V=J37RjouLu^#28GG6;7#>L~tng90IV<6f#14ll6H&t6l16}PmJJh2msJ+hcv^Lq za3AF)5}~b;9t^ zUI>=^Fu8Ub6b^K!W%^RH!mHTl;mFVQl4VUo89fFN*J;}5mTnG15$m_GOJQm9Rjhkl z&r-95#?Q&hOq<`9wD50JsXcdAf5#4+sa=PE7nmV^f(S3@u5VUND>Z050JTfAkm?ke zJ{Ww5#_pHz7<}!dx^6F9Q{2P4Ja=sh;6t1|bgHyBK_Sn!AHbfxdk9~t6Y@Scr!@&h z1NbU0eU@gHv?$;#YotZ3ZN-Vms;(O8ZnrA-Xfx|f1#5__9k#uDp zd5Lxn>&cd})|H1(^xz$Y+FRSwQFPu7)_EYXbX22{9U)@x!_KRj1IbTs4lvce@znw; z0V)6AjoM=>Flr?O>QGf>!#^75{=ZH&8>$;)NG8i1a!~OfF~J0akR8AM`nspunvYs* zkO?t$K&H%Q(6c+;S`IzNpjT_I&ntDhTvJ|W`uqF&e!dRCzN19`^}X>f-V4e*TPr3E z(I@l_kn46gbS`ZLk+TZa+g=*zGV3W*U0AG%0XO@m)k1DM0Q|vuqVHIlhN{8e2o5`q z&`&INZ;-uwfY&Zgc%KIA4POe|(aKNokH@D$80~_Sy8YTKYIWA05uYxH)#ZkprCL0^ zaI;$(ipf?jmA|F`Ru(pO<| zcB88ZNl{^qUD@rjc?e&Fe{IdfAj#*=%)vL5Y7G75?jEiFoMS&S=((bPe|N=-^d2O+ z&FobH*Z!*xFm4}VdqY>@#yw43OCf5K)TutHJ@z%iEJ5=^&<&P#t;UZ|rQt+A zz2Ys$Ubuv;RB1g{IGKNb;ofWvhWxAy+#N%8gmzjGHV9GgrMb{>#r|E5I}I$w?`*)S zY&|2awJ$OG_<#UhSGr!#oo)@?{624&I_bx9rnoV6dsShfZm8^|ZsJQ+pp@QCUgaJD z-N>qh;B&oV@I>j+jOA%BvkZME^~AHC!E5qgEJJx%E9uL#(Hk=Tr=2u+_W#Ere?3j% zyq&=q;0dMwru)_jfe#EWP;WhuCs?(I@poiXr{LIGgm6AaZqCl8#(rWs(DpA31BNtI zQcX{5k>h&9Q3AO3?%lc78=^+J%Glde275S-E0E>EpTWEBp}Siw#t)R%&F-c5XU14= zGhpZE2h-#pleIkAY7uhGR;0t8PsitdtLK%h-47a14NzPxgbKW=t7h0ts zd}A$+x;IigEbI9L<=f!DO5_y#<^nG@XOLYb2o4oujop+ivUrL}9gGH2?E}~E-c*5% z8usL0YxWwDg~nuDy&*-!KGrs5sNu6A+&_r8F3={T81#3}lb9P{YZ!QBFMM9(9Crj< zCgilbF;f61G(}kvPiUg`P~t&_14i01w9T%iU{Snz+fZ9 z*Ge=4yYd+G-=f}xs?S>hz>&-@y}8dM3GIy@{z@s}RtWlQ$lXu2N>_=8kV~;zyqm!A z?h-zV;|`UEf7!%fe=S;H{#Po9bcv@RBI9cE!#Mp`(CB?XL zN|<}}{FrBWj#JnoNH9F3Ns?A$q!0i?B!CB+=ph~*`MZC5A4))a2_c2%uyU2<7z&mx zH{exRq}2U~tq?~yKA3Wlwz_bw5wBzaP{W;!o_;^mYz;h zi*R2_6(to+{cbd3k^GNkWnMYl zt#0@*m%rg?MY{Iz`JGeZMJy*-$FDZueTxK2d?EG=R-R!ad&H8@Cd#YW`^_kl{)88Q z1@6FD*;i_{f=x$(h))txjjk^E^5M3;QyNX(LOoVp*2+)V$n-sglTf*6P=FbABWsc~^G;VA{c>CFU^?sT;Wmc=C zgo#F5d?x8}=c^lq`e7>p3PtU z7u%49W$6~v)gjTvcAwa7aZSR~ zHOefQVkf&l0WX3HZXQutHbnbXA33q~(rQZn!lXh>(LUy=@Y^Ir!B-UUHlwNT z?QRC-!GGfD!r3oqO-vbCz14C=YgK1rN;mxy_iSO@J(A*67d9r*P}DVMgw$$ZrLleo z%x$))e}r_^#u!Kxwy+p$VF0@R6l0-Y4KJF&!NBrCg~%Br6-2yVj0k#DEkUX7Ij1HIBbm*iR?6< zOH|7AC-&xE#yNGS1g0QxwCwo$?dnmho2csjA<6l7NrZNnp6bPc=ODFq_vNSPMnIq=%C56N`^Hqs{o_b1pZXv4%*nxm* z*_7R}Y$bS@9XrPLE}Mrdp`5acLShcB_9|EVVZ2%J6JjYba<-4j8B@Q~YsDswOd|Q) zLz7#T^skhxI(vZb4;2f|>aZd=9bM$xrOd{(P3Ros=xPLR(8Uh`aAJd)(h}9E6M{%kDO*` z(;6@Ds*&VLUzIerebd0-0d{=&tCJmXT*zO>TXSIbxqpCWG7SKq3vXXYQCN9dwk&z( zXe2H#e^jOIv@E1W!X-+QzV!`nsE=SOr+jD5xDY)9ziSEgy|&+2J!!w;$Co4)b?s5p zV;h^i?djKBu(s2FOL>_$E2J+ziBPV{r)|b;WM6e zH217%D95qSqP;(dh4)&Rf2k~-CfKw^%&`K zfKFZBXuO6WTW59JqM^`IG*s6W{*JO81*LKpVwv>34n_^z z8-P)h*-?vQgBsv)P&fZ86SL;}|K>N4n7ewc1%LG^%X3gqyw`{~V3ic@5`k1`8vTcw z2Zp2 zJhxGSAD&fSBK&+Q?eM_a&VBF9y7l=-{r{nSKOa@*fkvowP#?Dj@K6;(}9B|B$z>UpLXN-F)1lM-eF5mvygy)4i@Npvbx=`9xKX6*Po0*Ko6W?Wl8A zQhI(ZMh^)3{VI$yH!83uH_L3Jri1^k4>8uCqgpv5;!wVRg`W(hSkv{-EO)YZV}3Jv z&G|K%F<2}Q48im_EPdx+#JeTSo5lURjiUI)vIveuvJvf ze+}?cVN+k_&|RZC`(8gPVP-B610(X$!MWwMw5;8URpl8VA7zdTeG*TNH0xhgeCwp& zP5&u*H@k}h{uvU~OybQqzFOI)dK*u?+S40QTJ|lEYJ(k4ZPw0dQ^H}o9THj3Fb- z(l)cTWO z@cwcaDRjOIAC3|9d29&QAO`$n58tqTZn4fFK|_El*JX4fB=-@Z9(}}|Y3n-kd)c75 z?-O@UG7K~BGT-^HIXVRkyaEFoA@YG55HdVP9pxQ2h1mMvR1yrMd7Fo5_Q~-nBcaj; z#AGa3{6RKJd-|&ZG<3HX_39gh_lZTqbp^J$ePhmz;O%Sgd3)n89;F4pgR@!rX+{ZP z5FZlG$5zSh$4u=E{?!7BlwqkO{H*)z9iXyESb)L}{>hu7m!3?q@(1MdSYSc)Q5|(v z%AS5B8ycsnt4a@rAAAohCY>T?eiJGWq;Kl09$~-mTU%iE^o0)hc>)$0tth zzGRqI^ZzM$QVRa=qnD)`PHU_bMeZh_+Dwg~lK?3VlSAU{y!m%qyD6w`E;_=ke`mxU zn#MAQNJjaPrwj9~d8F$(rip6s*YWo)>-F0Q^Ck&}Dyn(`xZWi?m^a?b zOG+)hr16^=m6p+Lx%fv&MN-;1riubcpWx3Na#FC;mCbfv7oy9QZo)k!)SQlPLe`rm z$HQSMw;G&urGXwR@VER}V24ihAY9_Q;&~pELH3Q(m{6)9H9Ilu`iToOC`lPrf%&o) zF&I=I4&IVXxv(1;$1wgAc~jka3PqvNR1Mh+=&w|6S+HXVlDyDd$ut2?BJ$?SB87zF zK#LM)l8Mw@$1bN|zQT~;BRG8RFJaM})ZeDDL+6igN+S*=j>+!)1wBNl*4oEB1wpB~ z7Lj}u>WzzK$psu@zz!?#=z|Y6bOzhP6{r;!p zG$67w%nixp!9vbOBahXvz>Vv`mF|o(q85KksWmqGue=nrXmm~R`{e-^gG*kPe`@wh zkYV;G9HJDEZx|CvHONw-6*Ni@(+OfNkrVxHU!0zb+y_O4!gRYLs@5W7Iq3?fxpTvQ z1dDg3`|v6Hamx#2$4`o*v(!l7coqV1`LcC%t zLow$gL=@kRJX26>RteM}173-bmCXJHXP3yR3Wghx;AK1MOqSU)?Ov5yx8!9{#i%SM z-%cZvZ^dO*9ULE2H)M-7@(~w2yHh#_zoFsxJ)iM;Pv!3M(*ZY_Wlhue!^sn3u18)) zb`N(#0RQk9!frU28zkirO~otED~=UtNe$H+XLq{)mGBTAKZq zmDFb_7Aje-#f#X>#9F%c3^Mnq;uN{!53+i1_Q0AyLPu6%@e07>9swH zUty*CKgdoL{q+~Ky6lKXXOGOV9w}tmOw0p9@>yW^rnT`rGbCFl7UWwi4nv74wx1pW z8yVxhM z%m^$+p<#Y7QJoi9*ti9%u`iUmH1gP-rjQ&k z^PLKjI%cqZ|Z(k!Dp> zm$#)GhPDOlIn*w_E)O7M0ab@o=V1yqrNf|0vpNu>EsI9G%Gg%OoYqBW zkLORW|EQ;Po6G!#SVm6WWOhqprOeCwtZdNDV$+N&_5LU$3C;E{K1WfC^Cq9BxD<)y z#`W-whiF8*JQ$Oe!5!F=zU>YqAa(u>j5Ge-FA=h0v+v1-~?fS#qu9crx7VbYM( zz$;pz;v&NiMr<6xU{Fo3;+7qjjo)$5g2MMLFL!8|ljCA(!S5NDf1K&B1f0np;;5tb zAs^HP(h|Q4I@>L2!U)%0Tlzb98B50#NE#SRnlJQMLNnEB`^CyrI<9z05{r+ zgNAxrz(O}^l;T($zN4p0R$;2F)24Jy*oBXcl&K1q+F@Ziba_fe5P6VB>&S5FSs+hi z_qX^+DhLIaW&nDYgI`f@M;<2wmn4T$X(A_v9Fziy*+XV>uUqVA#Kl3;VvrP%vevEh zTdJ&^?-WfYpMK|b1tp{IZnG^HGsWnk4X}<|m*wOg_Q!k33Co&}& zqN1Q=*ih!ry27@Wde9=_P+K9WNgIQTFGwX(fU#*e@!vTQs{+}wituHU6f`F(F%jo1 zrW~YWbYj@)RYc{Ut(+J+E4*}Dsv14UKFukmo@r2~fhaE@P!ZVnXz5}&EKoNi;9+X@atG_~{#l*h!#>K9q zuH$$;jDm@g;zP>Ap1`*i*xbe$8TA{g}R#laX+=>A*|)tt`$<19f4Acj zKEzq=iibF1^l@nmSL%8?=0@v$(uQ}#Zp+Rrr|}h<{U{E}{TE5uBss?T6;UgpoYc_N z)pcKgJefCXvsoh`$`Uf5EL)!okj5os{}u5luK_cvDgvZGZuHPM0-(;RJcT2_Os^%M zl6xW**<30>AtH|+y-4W|zTO-t312z~jX`{p!8=S9E zh9IRtT(S3h7(BqWgFc$RSz!r)t= z@i-5(Hm$9z;Z@OA&_8mWaZN6j&XnU*ahQQLQZ5`W=S1Hf>VWT;I=}7`{3^pOo{R!_ zz>Rc+^|^yprjCQ49hsKf0)OvftJOtiKHFBM)${3+je&1gD@E8=-lLYqUr$T(J*aP|GNn5OjZuj(aAI*B|_fFg{ zLyHMNqq2`$jo|~^C>N%h6&76XmIeAR0jbUkojdr)o;9Ch@^70p6cd%XkAzrQtd~+9 zf#k~47v1=cLg)zrqtosAkDhArLn|Dkjm*?>P(~w*!TviXD^?{?#4&jBP8rE#1%vw@x_69ucOSu#e`r*6LeAjzs{ zPxLv)c!YCn|HIZfZ3zM-TeNK3wr$(C?W!)@w#_cvwr$(Ct=n_wb@CxUAy-63oV~Zq zYzz*TW84lVp0*WAN9xjO*1h#xT7<=)L(l|Sd#7$y(j8T@5Rm}j!O^%%vS?^4CVrh` z-a#0@!9dzReXk@odL#4Muc#FPC&y3_IS`{RJ7DR2Ky{cw0%;~=ZFkx@5ELb#ZmAe4 zf&(;_PzT9&Vj#dd&90vY#G7NEEd(Bc8pD{fFAkkK^o z8tyw?ndc})TT51V^F99|X^usJ!p?^qXBYwVZtI1QDMUC&gC1}cY0b#?@h@FbbEK;r zhf^K)B>-R^Iu<2VFL16h#*5^1iSIy58)9uy>L4<(n3=*2f$@dqq;Xiws!0Jj`Kzbq7F!Zl zqduyKA|hFD`S%8@P|V*19L5VEW>ln!0_tcHjkOqUN7UY4C?h{2f=0!IEI>s(o-+(K z9%4v{0Vav&prmX#5R3L+)kRK}vEP%&8B7LJNgn1gGQxw~%?yS`jojc(OtnJCBaAIu z9_7u|_W?xO%zH=9f-gA^s6i41dRbR*-UB7liddH3G?VXV~=z* z@8{zOdmUWyTDQg2sa>s%k5W>uj~Ltu`?sg|xhx2Z2GFaw%}v4%{He?0wY_fwo&5=k zcX<6~CFR~XsZbKUyLO6kX}Gcv5np7-j_hY#7*gm(Qdg3f*@E$GK^;J07TatQO!fXT zE^a>^PsarLfS_2nM>uZy+mA{SPWq4-MWPajE3+hg!u^rO>mwt+y{<0QMh;|i_8DSL ziZs}-a!qmY!)n)4C3S#|b(%WTJ;cz`!5#qS2~TTD+23k)7;qs4T+B$2gdN5yyDwae z#53i6HrfQ^+iJhYh27|nBA3t-T!sU7sSq0vjFF*Z)2_~!$nyGwb5H-AdQc9{$=P3qOA_WU($O=huu|Df(KNS6a5iE@o7Z*kk}pw0Ec5*kj8pEC z4sA39UC`|v?qt4sBcRGiBi7aY6ps-Ap7Cp3OJMo;Q5N|21M-Jeb)o+Cj!>PY`9tdO zw}$=$pTeYV0LNIKI>fYE(*|RAMccjbjX!QMk+L_7UhD8a*LUl{nY9;9NtXPvQ(!d7 zk9$Q*rbD>D?EYaWJ*tm>g-@b8PQ@&v6V<|4WAxDb-1H#P*F3g_z1DlCwQum0jYBZn zBM*bLC&2Za;*2CHpXwu@!jbCL01-9!t!VMQ`H{UXYwMsZ$dYni{398SCLOF$msGf8 zSrQRT_TWdQbpm-^3XyXMne(1$>ii7l$r_=5q?kl4Ck zrHdg^!8q6<%4UDUHoI}rI3v3*Z3~2QOI&z^f^*j9tl##aj3=hgp+thaPrxK0a7)7& za&MFp5h4S-FRH4A{(*!DPMz?X8e2$()F4w9&#{VA+z+NG?`dL>M-?c5M%OqPz24!D zA|_h`2*(~4FD%4dY2E?};~EaFF!KFbXqFv_n9=9`ue?Vx&SXY#^&@o=(9%(a`ESe0 zU)@+|UF_KRw!4@&C&~WN57fp#5{Subd-3i3;sxx-#YU?ZXGH_m zH*;K}7z#UKhCfi+l0AI7W$hAbT=?`$6qB<-@qq{&Gj>ayvdgY3Y=>n$STjs?Qjv-3 zH4!pvmz^q@kxsdk?A}js(;gHngUJf7<$3LgigdCiMH5G(6Ju6F9P8{41=k(FVh{-m z6791x$m^4MSOFreRyZGUD!U+>`ED@xDVVaXp=gwn7gZBc3Ks2-IjpGk=G_FRX%ILy z1&fY8SnhvuN~7~MF`Bzs<{niZ9x$oBlT`9Zg1q(cblh>dO>68Vl1?V5|6+muLJ6=4ZGtrG|hHhB_pmzWZ&)s7B#suV@M;u)!i>~Vb~-uk;#c{ zj#^v@Ysjj{RRO4fT@s$38b5fW&m2gQL?mc`h->V&OX7;RU0oRrpB{5Qp*S@yf~Cc2sj9L5_}9k)J1(E0KM(+q zPUXcZE3u$tyVxLicF@F1$ki-ok+l!rB?!SC2Pmm|I98Wpz7iT9ny^U+0ywC__kN|| zYAC}>hz!Lo@LjY;y+PCQA0#KHjhr2*_Ii&;IkKr?4{G4A0XAxCC8tz}CwgwJ*fx5F z@=zS_o2Z)E8!*+l&a(>D68g`!#~wx@i2shGT&oDs%`V*mkM^2Vgxyd;T>pX&+m@4$ z$nnD_$wSs!S&Yt(Oq~!z zBf6`N5M_@PVV6nK?JWBXY%`EFQ|9^2d=JIpD`Ks$F#=*JY>c_PkT}UV5TopKTh@xV zDOIN1g~P*6ybCfKCX|SZb6ZN8lJK%8VH#D+ERiYaWjX8YDYte2Khi+b3}-RxQs>Yf zsr3~j+7`Kl{mkBV8+v+OXz{m?9P2B5cmzWjj)=Kzs=Aqhdv0L%oXL-*?Ua1{p+rb_ z(93gMiu2Y@`m%(W0t1Ui^;Jsv^b98)Y^<{g_Hj*{GCh|#0=bVPiMo;}8nA#F4OCC; zj{!QAq&P5dzS>kMHoa*in42>N5wXln`SgDx%fUEXGS6j-a{u^s+b%5e^BQy*XdF3< zuZ`;M#ElTV?HeeGIs|f;on~ksR5#iD!e=jvy(9Z?WTvhP5Ot3o^{g{%?w{u8e+_&7 z)$y5&Vo3i1py2~CCkdFf6=EVm7(4+SiRX&8Y__1qn2==jd#w1RS1YI8SXn+mi`luX zJ};G9eS6hweBJW@b`<%^4OdvR4Y$E>msnpg!VGtO+){s-y4h6tYT8`3SVi)AbB7w@ zM3!I+X+D9{a(tXVDXj(lhs6H+53YE57FN4DE8(|{hN_LPjJ|LPps4Ogq8$$Z8TI^5 z@m)C{&bRX$RXlfTohPiaE}dq+Qhd|>=sWQrGk-Gvr($v7UD4%6sr+HTnjA=$4C%Pt z@YIbL3GPF6?Ae4ZuJDAoF0peX zBe%)G9qoWG&hhcG8Tpu7%Eyn?Kl|5i+JGLiUxgV<%i&}qz#X9RnO{ZTj`fM zyUN^(%tLFyj*jH1>9R*-A_DD=U@#sPyk>ojyZTg>fHvJYTg!gLqQO_F-2S`5c{pc+U?(xWKEmH4E=Mf>ZDw#9?lVt z5Xrk(amrA!i$u#FOn)dD;w5Z%edS+ZmQiIWF(mS)%gmy;`}SIv(0KB3hH#auxKMV&JX55!!qk|jQqb-aPd~K! zt=#_dWn1$6{m9U<$Z9NbmJ&rx5E5?cFb?U=;}&jnqPZ~!l`DJ=491OtwQkSPIk(nlhT2q|Sav*y z8;XsKR9Z~r+z=C-IS^)T2TZjlYck0;FVhK|j^4%b*y^q(f=2K92RHFK0o41|CNCxe zp5H7ZTQI83zj}P(RrUC??}65@RejnfdCWSoc+(tn-FZ921rjAE6#FxDV@$NUVE7VM zC#+s*rSz-H> zAFU+0x{ebG;|U1iFl41wzrvcXqPkmYfCQ4KFO;E|FY>OF)U0lZBw1^b<&)0D36b4~ zpco{hN_9A$^l%u_{$TH?*LkuMH&!~}$#8~+Q8U3zK>?B{uiw;oP96)zB~&2f-0U4R zgZMOwarPezgU%xz)B`u4GKik%vmw_&NWo%sY%;90G90HQfP3+J>_sE&Urf@I|Hf6`FQRWN)7Z*u{#PFX!9X%0KB5(1ZLkp`VM7{fz!5CjjSdK zxUwpH3Z}(>hl13ur8L9lDQz@!I@f&7MqvPXAnnFDhi6{WBDRH zYRO~NHw8&M(?56s{R*iZjf_7m@;Kwqm(yhe6MLwo5-YkhDBqD<(-i^a4sAin`Dxm~B)Kle6Cgpm`MKgwloR%fZVj;Z*Z@s#NOtGH@ z_^X=9U5#uc$=@EM@YD{I>B5_#CTw2WBhbbbwvsc`C*zDDL$C7vmh6l}aynhcaX-nb zp^&K zb)?(JQxdfGOQ|W_&}LP?g!$c1Z`!1k>DrA{ps|D+Yjk;MASfR6BULw?7;{3q^Isp< zt5Y7A$t5402}=%n|HTHtixDaW;iKkBnM)NTuRd|2U}NAOBP7-w0a#dIYZAxUqO)9W zE0!>zh!Uf~N~K|VaQ+g1$Pnb=8VSvuViV^Na#NZ~m9a;lOJ@>?I-X=O0!?&7!m9Cr zvo_yRS;Ynr(?DkoWUwx-fj_xZ%o@ClMcF28I8ic)7>>Nm=4+2I_kSo00}{nD5m*9~ z=689v60t~avSJ>kOysQ6QrgKK}6D4netgW~jlV5JYRVy@5kTZi!IM%K4 z92LSIx8fF<22RYu`ykxvp!v=aTHaPY7$T02DMG_oA%i>GJWjAT;B*W`o~`YY#N7n6 z9T81A?8!{0F6s4WD3v-iWj*+t5#8v%CRt)tGN)bcl^qc8wh48MPc4f$+ZpfL4^MDJ zT9d9T(<4bQKsre203|>#k}gZ%R6UyY3(Nz^mTJb6FLP@BBs}9(Uts7PX&TEV`bfR- z=p=Zs8M^kX7(X5PF+t-_lt#33{g7x0fN~k052zjthz+b5eAyf|vcy&%N}Ou*dUD=c zR9wFZU8G)acO25oL}A2j7oqMs-W8g!SZjeFLenRIm?tSLKDY~LU1K9rufzV?lGk1P zmN)N?SdLCGM@)<<5W`RWQWS|z0QGbKtsaSU8eR;;T8)SNLpK}W2s zG^Xyt89Sx(@~%_%%)YFxaq*(Nw!py`XXF&vM+kpDz{gjzX?1=_EGib4X$I@ELwAJr@9koAv@gTg?V`O4KX;gj!0D(-*Oc%DiYIa3x7R zLQnm=;X5Ew{THvLUmxc>&2{u8Qp+ofYpm)p_dY&f#Z4tfc-j82=5Qb#j-~KWhCg>dz-<;>?4<31u3QM!RM5k$#=HXDx#idkMiqy zjXK=N4hS;ii+Ndm4TH|`rfa73J%h%4W(9~l=YcZXV{oR5&v5P{UeJ9Soj0Hpfldoi z!NT;~_QEz;4aPAhz(~qqCFzwac`!xXzo#s9EmkUBBcXQ28ur&UQ?}=CIvUwuNL2VX zr=1#0H{>|Ll4!Tsi@K_sGA5Z7i&ZYZhCnP~4*OyQMI;@(V?3-ksdUX@y;ZLXa!e1e z+Mf(??j|Y3F)8G*g>Z01xfNft5Re~7XE%PVt7Q4BQKWet942l)mOO{Kux>N_3P*Bp zyx)2eC4n9Qth0r)up;S!HV$`yRHs5Dq)!XLNoQ6hJZu>{=B=MZZnw@#xS} zCmHFWN|(wt&j5aOmd^C6IzY#meQgc-#n%F`_*qC(hQP>AZU0%xh3VT-a_gz~M)}m{ z6-joA5pR72A#7xhcE6e6)yN=2Uj0)D$WP+cpRDkqba+~m2Y6mL>R6#B>P`SFqR3%S z?)3ooN>l|LKL2KND!y^4U!sM%L&LU>RC8p^+%LZ z&50x$cn4(;8Uj}nn!l(b0p_J%ETRH zecTh`qBEc6>Q5`q`NdaZ@@q#7pOLgp$2OgfnYxO3Y^kK6Y4D@RK{s{}zY~2OsJ8*5 z-7zObX|Xhct-NS8I_G08xk#?GKTOI&a15o}k6}=zT$ZSep?4H(r79mQXSZ%m0tk;H z@9_0j7??fI$Y2faF=B0Jjg)K*m)Do&eyNF@K;QVkc-@h&jD)ZlDi8Eti8ZQR^rSaC z>p#=9We`(k8BZ}YMY_-1i-rSx`9$3mB*PXvp=ns-&}!6r8b%T)>@j+_3qA-2i+y=4 zi}IhE2fRxD7Dj+w)9aOcd;T@zi`|dx%-x-=;H}s`pb|5|^}dj7S7+^E5=ohmq}f5A`zP{B|k008i#{$2k$ z$Szu&u8UGg&sCb%c>V}>gpgtW{s<6|9{@AJyF{C~|L_Tzx)A%;C(5Nay4w#P>8v>7 z7d15-&DQG8Ki&VC=;Zl*gZ{SuGq2)AcHj<{)Q%`@u`Eu@Zf~LU=MCgVS*?FV4Y!5x z%lK@|&hC$X8li8vY}-Y8;#TBaR8D)nwTGmIc0gWhTvaT`Ss{lSseZRw3u(ek^L{-; z^Pt+5fc>=FSZSdAD8A}y*G5`3UzE7w?4Yu{XtNs3*@McbHPCj|{!$IpRk$u=snvRF zcv_VjR%BZ9#MoZ#UJjaBQGGe#hvu)b+E0;1-%ZVy>wOCtcbwq-=scT=VZtUq&ll9^ zLAl4mBm3T3biYN~y`$v@xOKXKTi0dmh3CraE%3|F>5D$Zb7JXkwZR_A$nTwI-LUgu zZNPr9i(SC@UhW}!+k!J*MVgjv?ibaApA_W3NSb;TGWW7My)_W9Xc+VsXu(JoRv>Tagq(%f@E)jA%1E1o* zwVuJ{m?R!TsG&LK3g_5s^frm^U_n$4wJq}{lv>kYot-1u*y0Akon}ycLvI7s@~Rvm zsOsZ9r$Yjj9&j6`p6J?RqWglf?N|6mG!V-cQiaE^h6aJ)6$GHAIJmTC^}|h{|Diiz zhS&;4r#UUrXlq$E3yNYc>mT?O2Vv7_(KFmyJ!8@AZL9Pp(X>4i{})AAmnbdsGo*aEg^UBPYh;23G_7wQY@w2kRO z@cc&ATbL)JM270~7ogdz7U3U$Bp_RpDpa_&Hrgf^5mUY8e1q^cZtUaUUZN&V(ZJCO z#7qm`rC#I2gFVMkHfrnLj>%E-6VVZvbU$8*rxI)=ufLkMoPX>`sxv%to3KnvFPfShh6%Ccr;beK5tiq%BS z)gnzi9=#0RN(CMiwX98s>*A&Y;PXbdsdz1kM0#wMjCB9h1Vf&e#y&cfju%+*=Vr%% zkj8vqHsirRhrEwN`Xy@2YBhlHJA{flTwZ=qVP{ZdcOXx-6|!k20)+^jbR4yKlC}bc z4MIKDo1r})kU$rdenzEWyKilv}0DuBD%i8{s z8ELw!Zo?#OIP-Z0MQ&jAZU&>mY%35Q0a8IB>xQmTJscJ=CbCwhLPoHcEN~lJY&_yG zyD7@QMdMaj0N#>)E|hIp_m4hB#+=8wf7*)PZW93y8fgLtY&!KurZX37?Hn(Ua?R2d ziFJ?_fygM5=bS;ODbIADEU2g^>5KmQ)^zzQy7I(~5@tZpF@R||Q;y)D@WLQWf{oriu zF)PIJDA(|!lFWZ}{*0}jET)zX3c-Z}FiaLi%LFt%H>L~6-*Idv> zlZ`SbBKFBC4U7r)2#=^*ML{*6Hmzefi=6tgZjS7T7TT~`e1v7rol8Q2^GUwVJKu+_ z`mfDT$In4B`emUnewu^8TQyUkj?_fj*(#Rv_+`U4E6Pk)+MGFNl-aLk38>&B%x(Se z;>WeCNxMdkHw;;LIh@5+`babplNQEb&zSRq6gkRt>-xk=lN@{3$VW!2cbUBHJTpD& z(0O5j+-KWvpG1W}=sA()TCuKywQdIg(DE%6scKxPMR|IjqjolTer$liN&uDdXgrDtsU_PoD1K_o*0DZWmedMu@XKaUER*UQ~J(;BL z&|YM4x!`!?APv^pl6V&fKaWFpeMYG&zFSRYQ`XHdilDO0$g~b-QOgOY96E`g8O)HzZ7&pm}H_-K+^UrZxC@01v z*`-}0C#j~SYllY!&qsRqEmaC!Vq?W!>|CCbDp8g_@V`$-iUQF{s0bM{<$G)=Tv;oh zYYs=3jB&M>p_l$l@f_);Ta5`Q_nQjeE!QlOhJlN4h=k;l6;ybpkdjB`K(%Xy8AZ}E zQFXFIx-{tBF`>TS)mqn10Ga?I)6x1ePrH&+-E%jlH`Y~ND+v#qNV+KP>;xZz`J3rS zF49v2R4H9BCRn2H73&2;iugLT~GkOX)CE6J2Zg@(Mxa(pa@dV20^ zI|oye0c?6Sp1~Z-FmDuYqmbR9dM&O>j1p3WMO0Hgw!ycsOrYL{Vo~p<>6vXBW30Yl zRv{sZ9T#?^%q@7*0jgJey0EcDfNOl#Lu4k|z9~f8gJGVV6kR&h?O|fVl_mR; za>>bu(+QLGOBRxU}17R=-T#50ybgX(N0K_iLzz6maibIF1H$x4gt;r=KWZC23|YcA4DSHP(VN zokXpRkfMT9bKEU8l%z#K271cj04BCl08iA6YR(wuBRyR z$yHTf6XsyzQAoukV@g0I~74?8`A z>9AJk!_-7RZhuneou1&oT@g0P_4bihG44y2!rhsV8(%32Nh&YSDz0!VDd87)?peK> zF@)gd57HpgA>Y98Q|K<5L=P{v)HLUn?*&*Sl$n5yd7R#!$s+BLb3Lv5dI~lwrl^=D zCY{P-62AW2QPiv*l+)sD+6AJvqT=;Z-{2ue%fisghsaD|^h=8Su|!M61&l&3YemRZ zkttjOTQ7LO-8MFD%{#oZVx-GRY4%h=f+6&b*~O|Ruo5V>+NNUEwKr50&W^ZvRx^EfTgKP8iNJ}9YN1-3`q#pU$4SLs8U@)l zt=a8nBdo^Q8+WQcEgoRG)>Wo}eDIxmw@rokWu~=XH>(H{zbE`^o_gx3>LxIH?IVJr z*u4&RZw*-}G*5O|RZkI&s)M!SBDGYdzGkr{ny)sIGK`7j?L2-0GEa(6O^B}fiLl^! z*H|p^Oy(nEv|b(w!Eoq{1EOmpA||dx{UdFP{QQe~J~@-qAOf5Cs=YxzP~OT2y?3Fl z0Bh=gsL!vsGS$0NOVdkyeizVek1v&?M}SXM3Z)@f2$m4#}RIOpJv;`3y(71;D% zNGqGkct$w1#4o)U0H)|SVvPgcftpNY^p}Q98Iy7s?>67{@i8gnmnKAbz>V#V5KZ?h zG5&d%g!zVF95P9V*|gOYg#x6IX|D^%zy$Gnya@rHTknn0Ug>(KEoV-Nh`ej%*4%ab zvTL`_<`_iS&8gDpbYM*6(!C+}v@dDHB>fNABi)Wzu5%1Q-PebUp6=o3;EytCrogQq z%GL;!cGa#Lz``%^{xmjf6aD{l1PA^l;O;n3I}$(B;BioQ{zn4tA4kv?dlXSrPEL

pB`_u~E2 zsQzvDpA;(eN5!x#RJY1*q{eQvhf_^=D^~Wb;x5p)L?6zfxBPXpV&_$uy(Ucm?9$W) z;&QVt#^)Kq?)==WtG)bszm~;lm%h@0|Has%f-hS5>V?G}y=9Y3t+J$qHR}*5sdwM> zO}HC>!wPk7B3?SAgX{OMg|__M9nZsV54+(K&ucg8=bdiA;u z-@*Z}Uw#P~wj&=`Wsn=IKkQL@9P;Svk0k({W-Eulm+zU;ZfW7_H8kHBm-D@u%~_XK zHvQ8x>}~!IO9yx3T*C#tgKpSKM@RWbq_{i=V@zviJY3Tc+6aTMSalOtc!|q894K9r_o>~ix zcC+N~Cl};b;O{m3y6iO*OY9CgfH!$rvnt^BeK{W&qO}F$FKGBA(T=xK46m!qYK6W0 z!sE=s>n407i_mRG5BKukiPW)dfI*GXetacc#y#WSA-oG&7^5WlC&-=XPZ!y;i!2QI zHOo%R^ZPLijX0k#@~eqbbNbzwK6(UH;nk;KIde68R}O7GuBXw?!8tff(B?iRA&8{O z!;S~@5234Ch@A{f8X49$DlCXr^DtTbGkA!cX!m!u!Czjy(o=fJH*5owJfEm;L@MNh z3u%A6a97_TSsDPxB?b-?E$oWL6Nw4zxYIS628DxERrDLmxY(3y_%-lYB81BKmrr4h094Jf?3YNBjdeE4SsIPR*6oAFqJ zUANjq=9=r|Z@NUwfB*+B9X7)m#GBO^ol;!VOkS3{_yxU}G7;wceRaNcrsR8dq7X?e zNc*H+DJ*k&Ei0%}o?F}Qf6-Ma@4w;fqznz+P&J6xqg~8gj?M6!U4Y#4m`Sj}XC=)R zR`?9iVPw4590eQma2D6eU2E05$=hFj!pO07PUlcp)zbMRb3DPuwY!WNUv@VQ)4G7! z7jIoAM*2~(dspBBL!YQ({ZgC>ul1dA%mZnbHNl73dBA|I_Z73veOQ2vWTw6z@q|yT#Wq8l$hbGl9I?v4(pZd(&4?*?u5o=IeVj@jgvca>1=t_S ztGkXDMe@J4*x*tQNY}VJ#Ly*Mxo3PIR}nrvd)|zP?T^`FPzFZWxGm*~#Q8_#)WuGz znt1S&K;odX)xZlUI!7r?u`{7Q;}H+OSJCkN4(XCSr1+t%m6(cRCmG3khSaSvoCAU~ zE!OWQv;Jnsz}zRyvlf{KzzssNhH%4Gnkj2D26=1SA-vYfyO_nRX4x&AR~OyV!UVS) z#+Y&VJUHFR=3Sl@l%~jBMkI=j3L$zp%p2drlV9iUiNT-m3zmwL0}N|Aem`R)22@j# zZj;`2C2SQ>Bu#J(?#`4W^)dbWrb2H0J@V(xPx0db;V!dZ08Efl!lDDw5hUx^5ut7N zu(<*7I6r-DpkCKNaO%}rXzo^;aFy~H2vecEByQA}X1Uf`pFp}&z<)S2p@Y4z%G&cm zOkl;1GA1tDQ8;zXng=^IXJD2WL&tWqLyjt^R$`|yRwRHKA`X0$bnY%|Q);6w0L$Gf zb{a6y{b75+yhh(skd(y>imTy*xAM!_0pgO27o&^$6m;QK6UwHbUzy3C#hcV-570tXz^88tNcPczv!(~r>q7Lgb(o_s^si0B9Z6SdZN zz{;GDFA`c|&n!1pwvcMeDG5PNB_fcC;(AZc=S(6rLoX&ZG6$JzQ3D0?d>6aPLmfzR z&X@tgqh%g3BX=&z(3zKnB|>nkJ5+rso3)xh9l9t+=awvt%;5C)nvkvC86sICoZUO8 zF*9W#%PH3t(_}p4UhRR$WDfOe;4Zh`5HSwmYT80*WGM|qZzHN@J&e_id@}VX*)dKs zwsM_Tv5>F};*c9l1ipb>UHp3qLuv4n04EzRp^*dCP-_741y!V%vEluZow#dr~2*fv`Z)Wn!!V z(I`U6i8k`aOSs|Ouc(xsT1p<$z|c3-NG0wIB29_d&RO9%)lI6Df2;@<5KB&uEz_jn zq_4>UKrW_U!;hEA7Peb48n@Y}*P;BJ@}7E1x}>`SaPqkQQ#xUhp5RU+jGE#AL&#kM2>JLY&2IfAMv#o6wjCt>GolP9Z~ zR0@!P%Ujn_aWz_1YJ!j6{JKXbcL&R^{>;J~!x`{^ ztDi#~wv8!?A&YQ|CFL$Lqm=$s*TMX|T%oFKr&NJCltVJF4?!la9BDnTd}xbHMw;aG zFa~EbLbbu&G_hI4o~EUkYKoht;=QhzY-75pCMNusflFedbRc8X38iei=K7<`3Qp&q z=LBw=^o&Pkv#okf7~?m)Rs+-1YiFjV%|O*OpHMs%gU zTLge@JoKg|{7hAjk)+p7=vZEZ&f_I>2O><8$VcQeNfD>;2BM{OoT zk=pf##ac8!2ioz=S#6ku0J|-7R=LU^!}1eRoB823Bvj^m5juGgK^s$yMI5Ou6~2Vi z(STRwS0TF^thT8T;e3iKf-wx!Sf}?D&o-6^-cJ6oJg49Y_i!>d(`K`VY}NthxBFdH z#2!3S)1<^)Fq)PXaWWxyJ++QDB9FWWPb+q}Bx-t)L>5^K2_BdpXRiZLf9IhGAOdy4 zB-RRBH+}q3anFXLv?L*X0#P}-_rE1=@-z!W@jRVEHCdr92JCN}PIiej%9m*&7xWR* z2MVqM%>Y}D`V*hUwSFkZGO6DqkXH$4iwmp>4Raidki-oq$^(ZynaZRRpS;HM=;*WF zfMSFC^hG%gf7An}yWiceKiC}q;u@Pn3VMm7r{>p`s)KjM9>Iz<1R%;^LQbP3D&CBk zmBGu#kP5j?VAFS^F!U@a0GVbOS*lP-mSo{D#)}U;unPh;5+8V=UHq*7ctJKTPkk%vbO|3k$tBk zF2*(sGW+U~m`0&?VLF$k&*Hpyu#PC5_8DB-5hg?-7<#}Z7@dYr9j$^&gR@cxeVZiX zlP-uhr$gx)dw8NrH&pWq{($(>5vgMoaHLCSr2|QEfESHjX&}1rp&pvwa{ND5~xM?Kbk-^t=%YF8;fX zSbQ}xk&?cUf--35II1&ozom9MhZ=QU@pvdygI-cw6KqI8S~Kj;-N0wREuOQLF}Umo z5NdKC+r~WYmQMyURiSVV%$rr(WPaY??utbncT9Ni?aqix$@%Ml#XW#dYi%3M&x0 zhxsU$i{^S>@J_1#dN-UI`MCA=m;O!49QA%;ys$po91y{3m0&d$#2gvIC5qaFt27EJ ziEniP$Tc;Mty4agJxWVA!f)}%^8Q`5!BeuHf^xx@hGO<<`c%{taG?(6RU1paE&gO* z!Co?E{$<_c58Zq7e%LqPd|5I@jXh7Pb{#RD>;oC#nUBcjdA?bW@i3+LoGZ1S0P*cd zf*gY_X4F>d9~H+(;cjo`$L`Ji>0AAWWBd1-;fH!I9K0#^x!qWj!a~VYqAoLgAyu z;?nA_4yT1Lc-cqwt!jJEZJqN6_P6kQwXP=k(nNKYO*I$yFs-ugu%kN4=JY|ki}mZU z1&kzRAVw*0Nd8 z%nbk^+}s84HNB(l&Z9Gox@Ob-ju!9sp$j*Q`OP|Sn78n86imGU)BD!i5nboCe|y74 zxzS*k9fR~wAhOz_!^hvvj*4?|E?0HdLFy)>)2srkmM6ii>Tj>EppS!E14oWaAbg_T zV0hW=#)WPIVm1G+Zh{NW_VijX=skv~iy729->VqQN*b{Pq?I~yw`Oeb>v@I$?f_e= zK%;lzoz{fhds^us4I;?N;}Jy3++FD=INpI04_8GPrcVkmQNEe>j{6ZnL$DXq2h_## z&KL$`f4IXJtCSQL0Qw;OCI$weob@Hvfs=d$ywad#S8s1!g}HM1$ow}M!ie-QBlcT%m;FM4?7?|uB zpwqIWE*Ad?Jt0ciEZA7rp?44bE!Fbxr!uZzESrBJY;ey>l}we<`;^`EK<3k$u}p$T z^$du)0>8b;-=aqabMK%eVpmb-)JDv?2Lgde8q2|-o8U|eWf+19mOx5jSRrx-nNvRs zP%U!X7V1K{c(qmoasZA!hLp-K!h49J5}9cdXs}G?xqN{(6xd@y=a2N7s+YKWLnkv> zMC$_^O=Qo%$zRi$xz8YmIH|E*R2(TpUwAWCv+v%2T}Ea>5`JkHu6e z4yHJ-%1UbnBwVK?Q@>wa=xIJ#jS9DtM1dDX1!{h{i)J;O+|<`c)62Fw17sUz#;eCn zyfgtZ4-1H(C9<0!y{D*bKQ-bGDkHeyNUX)CXlK)YVjjhcxdw?&C6v*$79mF2NWUvy zxFlVOV!ZK`Ng~4`YuLa1qO9~&pNzN)o<>wai2B+i?Ktg6jQb)YK(W=YX)=z8eoPSj zVG~5x_|iZc5H2T4DRPW<=B}L62%|_nAUj@4f!xqpk5{3q+n>_Np8eEFH5)>5Qtpjo z5A)hy1cWLlhYTN+EW%xS7ilx9I=S~VlQd&T`zijr7o^gKNSmZ*BI>DC(E%i11o?SJ za!+vGPhG3=G&;I;o~okKBr96YDbo>%-Vby=~Cd4pBV z51oW%D6=fWK%mN^gK*C2ObQ8jFwFIV^N@fgqMb18ZG=J?aRj*8k6KfyAt)|B^ed== z*c!pL{(dvVTEt_Z9h_x00|voDWj~cxWc46gf+_pfKL>TU2zQX_jmhJ9`Vw41P%vb~ z162OPh8G%0gz+$nPq^!Utyrb%29uz{jf6!xN1nu$^<3tosL@>_lEKhWn3TA&K6x$ykpC`b3`;jj z#LZMS4+sGAnNkxripljy1!cWl;3t)nG(KM}Qa!I^KT7T(byLH!VEB&};eaX`C#0;n z$KN}?OsT4c98!@~H1*8>qU#vUWP>VVXpjR$eG=V=4&KEknMv+yzSXB#qaC-`>{(1; z8&X;Emqe0?8R+!GyG)_Q1yJ$h{zn$uoRcHgX&w(ulamSyCZDrdTcw6v--A*B8g^w%b5iTz-SAiV+c5=-j09)%4sUFAxPTyb2~f-ut@ZVTe3DL~=LOfIC3TsEzMWD| zhzrby6a`%5UjfM6nBMx;FMf%9Y94spsl=S1K!J}tO@4SU%0XzdpHNX9I`et{6DkuE z;G}Cr#ahG8YqXR^rF&%`M714`35gUpXJ*t9cxdJm>J9yp=~QeZT#}q3Uw7-hwVsJuV2#;F!1ed(9{rt%5!?Rl&uAUu;QKLPLo+B)|@T$7PfSksX z`5XNz-?smTu?s?^gBMNp30jrmH=DCjO}a5|jmR!IpJ#XaB^7Ea&MamhC=-7BzTZ+@ znh#@0)87JRP{(6qUa~mRwInFIX#Vp%9xATgFFfX%Aqg9Sk%YO+8G3052I*x=Eak|D zIHLRM;+SzRrWddktb$($tf7fy!c)`Z2g@KXEQ0XB-re)0`)!tphw@%I)HF(Rb$?B$ z2>s{R@N^L|XqmXiyKE5kfn}sC7pPs81?I>Y%e~~WnoVWY0CuP+0wFDZWuV7QbfS^X z!nz1K<`TAUp>yE0W5P+j`Cu|+6Z**A-+QWW54k+m4U-VpwZ+@#W5cnfh4In9a%b@> zv6O7ui4KKa4S0e$44BHSyf`VhE*+dl1ph;L&Te&!{%1B=p?$T#hl$6=}WA?=P# zWq`QeXp-sgPvDcj4;+X$&6ASRi*${JbGT*9FP;z1Y?*Qo(@INL8`jzw^VQ@hm-`9$ zF$ORvfFFeC&dx++9n7hH;t{OzpcAc&3?*=oU^!8kmyAec_aha4hUmI>LoRyqK{{KD zU)uYh9z`*!EQieRmm*|CA(~z z#UgL8o0ImW#s}}F9vN;)47>Q1T{Zqh9Us=@ExXWC4f)nB#{HKmk5Q5K6M49==lrtA zU|Hk3^6qcqTuKL!%wX>dQC=wRSOTAk#Iu>_;eYUTEs>^)o#Em8h0;@RjWH2?3 zLaX*+1hE_wdO|#m4vum;4L^8ex&Z-b{DXR1HieGr>kljXV{+#z93Fc8|415k|H`XPDTL*0?ne~Hh_qIG-t&1a zN_$B{Ca*Jxg5tz!XmT3eD=q`}Jg#=lO2uS?P7_&EbeQ(`cA>Ytq=_T{pZ787FBL&d zJ+&}-1KX*$IA`?le8egNSMiqfp-TD>#p#Pmf2c5kI2Gr1vnGPwGMvNN3rBTmr|q@g zyFD&Plyy^~LhboSVRhC_Wfew&Whywm@+V&9{8DM&{j(pl%?0@N-=}M)+bZ)VzYq24 zDGs`@Oag87tj6EM3bRpWkFZm1YijUkcVhf>Y3N`5uy&UZe|m5KTutFFs&E~0bQd~U zzPus6w`9*{rC#|;=9P^6MFyxhSz2=$X~TKeRGa-6D}8_Kg`FF$1xGh5tsa%|0Pr)u zcS{i_1iVW78Mt7+o?W2u`MGQ|YZWMSFirt-h`SFi^$u(EE(&RDy)9U4!e02kPk1qo z>X_udslNb|;*;v%k(RO3ag?2mm96WlE=`!Qdm7#? zsa7uwLmFzCJ{XrD65FiWgF}Qi!1z4)=`X~vgNje&(h)NPx5$(3KUtlp?0A@KrDH_E zX^gJmzxx)CE6dO*Y3VgRUQSknd0E~7pk2mf9&K7K_%Upw?lI+}On4SWkxPBu#nk+TPWLgd@xn`X# zdTo+a@8)$FE*9E})-8$<0yDPr1X6~c7Uq>_(c&}brs zcQ6L{a+_+|h*c?g@|(jPYp_Eej%4Sw6l4n`kszlL_i}jf85x}(g78*_gF{pMhv2DE zT~9n%xsv$5MiymB{?Pkj@Mp7-A=ByRnut@|YHzXVHPX5S4--O>C5n0T=@*70M9F?r zVt{`gv4iB8JPC|~zR6+X;i~SHSRMHAmINb?#!8Wz6C=&k1DV9In36w{{e$oR&TEI# zY)BlXV-|&HD@vq1FM};{^x-6iwJ3CzdMjk^*c(0$=Y1DdXOXql)ibN>GvnE=ATZo8 zYRX^xgYy^Cniw)%Z!eHZieWsJv5927;o-=hcIFp13n`dey|l7`U-~hIe$>E~$OqOP z72R>BA#Wac^)J*6Y4C}FoMWIjaWlDJ_XaH>H_DZu3BK##o&iWj3r4L1uIxz|$I3$4 ze;zp(NHd}R%d+#6{P%MbCVxSsU*m0!pFqamQ_(n@p|J6)V#^fMb~$YZ7zNXqGOr+N znGSnr2hqhEY&T#ua5X8aXex$ejutb2huC1MvkZIk`&L7Y6G3PKdISn=woNsyA++Z{ zp(g!0Xk!%_jBBLF*;Fe=B%!gtM8!9YLPQ#Lz3U|~M%9#a4}VT0H>wy>l3+QL1e~f%zn3nTNm~z2}A5-?Hj<+N(2ELIOIhU`16q8tgEBEUZRG2s*d2cG8fP=8G5!(8?f z)$Ui1a`>g@c=N2R3D;352k-W{8IQ%Qx}lG(zN3gay-u0zgAlq!wAoJImY6dbnqghY zJ=C0y`!IH9aGIzD_o?_eW)#KkBTI|WdQ;xfr*L}&nIuDiM8$y)~3UDp5Tu#bvZ5 z-QIMVUrV3XRuC2q>L8SjE<>*YV0PS;9VAg;HI-=9tM-^m!xYIDjU5e?JubLwHe1y@OE3HECnjc_UccZ$gT_c|^v%OtLCFw*pnmwVGZ{o7UIdXz@ zm(&zZxY#qB3MH4a@?|^P&S<5V8XhTMMetbKFCs^-K+PQ2YN><+DWsA{*cdn)9KwE+ z%2c#Y*Ba2p$ty@=_T&~P{2^?=Q`Qu3+DiHoFI0F8WqG>(O;AGAlqIHG#4@TSDZvG< z{#?jka(Ws;Xu_R=D*Abyg$Go^7t7}AJd}i5rb8|QA|Y6?PtOc2t?E9s$i*q;0`VI1 zQ*Pq_tipo}SEFeYYie#??-J-fBpm%tWrZy*oNL$7EOQ9I!cdjXUGL9dAVpiuuv4MYi%!}J#3kho8HDRL!R{6wg zJ_FSxGrw$xIC2~BOjlnXf4>gpNI6bpRzck6%)nDg7|y(Xoz;t%u9`&dY1JW4Rezpt zjd-w0L)=5u^ExQFQVnH2iidDe#DmmPERKE{dyZg?k47Yo^sp#V_$jZZn`cK?`eyH& z6;DHajzFUbr_9N*Fb|{v_}jcC6ccbZ_WGBcBDD)dPt0LPRIs|A5|V~6Yh8ITH|f(k znsMa~Dyq@=;&*@bXtDMEn@CnwcWRnLp%co`qJIR;9eCBxQ{e;L6@}3VIuv6))$vCs z1@Z=bPgHQKD1SwO6I=JH%+|T<*(bVES%fDk0deAcb^U?TWc&mAkUSNq2wr}Yf2n)d zDFM9hMSU;>a?5#e$^2@}M!Hm-U@QJ=GQK~N*X730IH+Tke=oyPM>WZ-gzsaq^*TX9 zvM;clnWHgS72Q;=l5o{&mO1I5^|(NQ{bYQ=D1Jl1VM?%Thi(!fu^@5gz6ti!E+@~I zvj>*JX0qDN6nTn|*v=ZMf22PhLG~1$mJpWo`9RS{48R30f6ECP%gCGWj?jPQ?!-uR z?SHA-zO-PdxVyuJRf8ahJ!&+L$(>AtTjDY=Z>Q$wTHh@2FQ-QF&aCV0A+zF|_6jHN zNOM{I!`^jE(CGC%E}0z$%V_KFcz%4K0sYs0YpC^N>RNP|cRUpGX7n|+g+`s5&{HS^ z^(rfYZ_Dwrf#9NdV@1uf$jTT==#OpmT`ysYAM)YgAc+C(Q)Dyf6p;Krz^Cv4emhJa_RI{FIh}6$ST$uiSqbtj|0 z_tzWe4XT|&xSWkFK2$r*iiuz>z}iS`)!W}9TAax324|T-_1#uAb7c-*zeRZax2H-$ zo&Lct#5H~aNaLaC6zr{nAs z&o`ml??bpQ#xYM4r8-t)-w8SOP}md-geinxNp(8qjHpJncW?kQ)5nMr_49q`U;&t` z%nP&9Rj=}f%IdIlsFGh=wo|J+`^6dadialKuGdnfv3p^cZ?o#xR#RHF>e_=d_`)3x zI8y7`ZY{75?X^*VzC-3`X?xnk=)O8V$Je_Fa~qlhG%J@gY=(Xpe1+AIz-E10a2>}? zbl$o&y-U2f7Yy-U)@Rk$lV;^(#I>zNMnQ%C zj34eZ!X|BoZ}*jX=ZkA?g1T6AqZcJU003I6C_nf;gCx@LwpYfgyCC63SG0vGMUU_~ z9rv=_1>Mw7dME2pm*I@qg_tF#y3*3y?uloTuUl^ZiSO4X1GYJSbH3OozK{Y~0jMkf zQB+*&{PuE*NU}T`#VT{w>}kmfxseG^bZcj1e&?vREMJc{!b70dN-wx&b9Ev5rM!ht zCYA#%e6+^-e_oP*6x6iSnTsEQsiE#>H^yVKN_3k2=}f6Kt;-FUqmDO@$i>X?Uj5Lr zoeUPsNuI5W?m}%-mpy*52h;Z8k+1@S?|_x0qrt#8xo_sqNY-XSpA%z6Ru*J5yAb1! zN##ZMix7#Y*cOtKM*teT!p;zM;MARqc*EYSdy?o|tNHYVOe2qvxCm-S4Di^kFGy0g z=(59IC>RO(Yh|B&{TI;DB}~DlG7?Gxu%XVZ+%ux&U0HQ&^E~QMx5hJV2R94!Nc^kq zK{l3h{_0bmf!*{G>Aq#UX94U#Gk|F@4d{vGPaA*8xKK4^eZd7=t?6whDYJST4WZvJsUB86jSUBLL zNNxaUO7GSbA$E$t*%o#9GI+|XqvLZB#1>w?O~leBQ^bNCS2hE3hdMslD^*T*S5I;W zzktdKtyNcWRB&F>vT^geZy?1P4rs0v>NYp<@_4vMT#$33QGVk7~N z>LDpL<)`3s0>gN(svV+S#~4hh&a9Pq4Y_tDL_IVCE%oS37W|nfMS^KJG!%W(h+*O@ za$J7L)P}I!B7OLtUI!khH*DKsI(zgvG5O(ApBT4LnW_>6u@(|JD0bBNZQ)4YJX$2h zB?uZ=C|P}jeVN(%^tma%Gj|YD#%FXze^D#;GY;sK!y70r$J*pi0cy6^eu31zsb{Qi zi~=Lu?gP-Pr4mc@Kl(pW(Y(mtGF9Y|5nmQawc_hj%~9s8g(!!W0w`_k;%T09WZxTeo%7p;NHUZDz8 z*^93?W5crA@Ncj!?%H6*m+Pz2@G;l8!>*IR>@Km=FITmXUrLO0arFB~nR(%q>x2t< z4_GX9D$ovKU^!6Z3}IGP79HnF->P16h{FGY!skA8utmq*fw0!Fu@*Og8}>214_i$5 zb}_gfjB}_R?}!y21-14>y{$u6BQmYl}CaY8<8acv75`nOAGXXpZQR zLx%dSJXl_9QK^|3gV=8#TakIwVJYlk%X!JoSiXoqLZv)@c>=2Xal0PlBI%6CBz>4F zLT{c8KqhtrG3*=EBH{hcY9x$w=yIAO^hM#iW@(o94ChlrSU45dqf0lN zOrj$km$fNS=^3qx$OxQ(e@vGUdHw06by_|w{{1Kn4I`4;Xx4gj^HB)HJcNxTYX&!+ z+0l3*qWP$-4nzN&S~RR51ZAgF^nuZ;Dy-r2lWWxvnRs7C-{u##{-Y{1^;aq$|uU`b6gHwFdS70ZyMKz?lWh!8F zro%R28M$`tm+%Eq`U~v^0iUx9qQ)1|3_Hl+7+?mYV;{+w3_NVXGfax)S7<)|(C-B@ zS~ShBQ`qYq>Xt4$;bl6(5-kU3B{6cZ*Zmf9{+LSgtwV|FeLITzn!@8CHRL(X^e=SD zKN0d#R-cDtnh$I|^fpmY&;@|+P`MFEpgCQVF0I8K-B;HLK6euMo7>Y<$qcZFmvs!r zB7jpBaWG}+T5UQTdbre39efbQLmRaKrOCQm{i{o|pbX>(&A8c4SjgXkAECW=Ls=6EPCCp@ntMPx)5mBQm5Z&)=u| z0alj%H`YQZu^FNsBXTSKxny`#i%oK3V*w)2W`wa!XPs+4&JjTG#O}t|E|&WM%qhSCvLtgx(z>RDnW3wq1rKN94`CHv^*}4a~M%d5Dh#eQeDxeH}>+ zRx?d0rRDIM1~rpfeqj-->oq}shhQ=Ww13@_AH&P<1S?>RkcZXaZLUs*V`d(v0(zXP zYfUs{2ES-ikO2Ve^^Dy~Wxr96*E#M4L1-m`{*ZJjLDN3E!>w>FQjP23ClfJ3Z;3>3 zq=(n1l5xSvL~T+^RzAwdt*ElWZzku^BiZQr&%cv2yy1U@4y)2-pL4!uY?)iRouP7^ zv6Z^-XG~}sF``FH`1phBX8+uPs)w2uWci_PODULGVhN?#b?4M)KWq&NbJ5Sqtb6S7=b)pAON zgqXqZUaGKgkYkHp$ErMf}f7%)-)Zw~qIBR8>Kl>2-8z0NOS>Q^DHxJz67#qld0Qgmxy@*}l2A zj-1ppOadz!&n2eBB`UwM@#Ns2*sz4xWVHY~Gcu~dF(1wa3!O@&OTI?X(T(M2EUT~B zu#v=al9y4K(Lc7P9ClRnMuJ9IYH&6;6sJSt0u}v;AlWN#E(T*PSunLt8PcZ8TIbt? zzj4v}dMr84Gz^JTKZN4iQ2PcaXWXuP_mAtWp9K3#&%#^(>#th-SAe{-pntt@hC$$= zC;pED#7|r2KmMv3ZkH3trBNYV^Zqp!Lw$}Gpm!4vX54XQGc~LD5)s5KW9*)T2;77O zZ0e5FA_YPjWph)$n;zuz|0D&$pZm5`fM6T_mffapl+kHarA<~PUr8QsAHK~hx*4y# zr~#n2>v6<}4{NHx8u#7m^rExfW7lt=Bj`4AKbYWSMsqdH*mm@{t6PJAj?9eRuHi472N8|(2e3%mAPe9ipNo74)5sTT{FSurR;l&w$Ex7 zQ!r@1M=jl^z23g?)SQ5e4cokuTqDC1pk(e6;zz6Tu-$5FD*vt!rx zJ3DnllSm9&r9^L-t85FsOX?zw%!##W5zoX^o#Q6 z-wwGNHt+Qwq$#S035;&Sb@19AXZVs?l*6YCC_2JM_+xQqLIfwT9!En-rriL{s1} zp8svScNy={MmNhJaUJtZOS!%!;$I(x>HMe3i;4AFA{y9HOn+Q0Y^xCbo zWvFF3uJ8k5kzurA-xrNv6N!hCr5hu!tKP9s^_G5+cZWL!m=+-wcGNew71^#gJcpKF zZiL)9xDMnGD!vj_Wu-8FBCG@qg^eq$z&NiJc85BW21m7?d8^Bq3VFrt%{3yz2rVdg zRmTY~4M0Mj*y&2{;6@#K!xxyp*qAUr|7_*w2UMBDLZk~ld}_|6#7Ef08#lX7rgUO(EObmf3dbu*_x;9kCAR2fcn@w^^gZf?Et0vn4nWRY)6gQIBnca1 zSpYKM5>QmX01Z4ti61~Gwkh2h_lgU@OH#b^q-Dp(jABFD2?|n--WM$dM?RcoEV6)H5+O_uSI#J|abHtY!@uBo!B$AE z?rU#~xH{HF(3m-br^i83AcG{R8x-%bR0L^`=x>9)ZD zEw2o%^a1L`qyah0f6Ap08m; z{^30nI2XK3$;2xkj2L^k0mP!gT;)^=iCaH?FzE#d?1)uZa-eK4oz>`Xe}^>ZD9-Ql zxCx}@x{hmD`>d!(Lo^e-q?kQONU@x9NO%zfSaaM__L1op#!b-y&${MR>x^MIm8iUiVDd0C+fZAh?gV9vvlRjnQc5u}Dl3PU z2C=W>K{`YLt`Ua37Urz-Xu`_WqcrEXsoQrOz*#^mPkq5430AT%UQ2XS^; z;a5l^9UBpIqd?5B``nqRW|rGy&DFY%$8;e$!m_#$f6#~j2&a{F+KcTSN z4)$S|@adH?5?Q8qli>jtM4C5W{qbtM$e_kxRc*G*r)Y{L5j4tSo7oy*H&J;P#SPhs zlOZxwZXa0^+!c@0&kW0E-S#R21`M9oNKKS(q^v9xCB6hK4*E++^a*4OFq!6WtWAXy zNsilKgda9X%h+Ypp)_&p_n=7lCIq(z1QMi-DP6m;;lIDBHOH6ee9>UZaakRgO^f0K zaF(FzF8(adZr(`rw45`&cH{d1PFWzPS-;d4nPKuHDO{5^qUN0yRAS|MCMO&A(jzq4 z=v&0N_mC7oC18z`DolY_$`}d7S}W70rtY)h($8D!ie;{oZR*r428r|J|iRuGP!*V z-?aQdwYx2p!D>4-#zt2#W&o|+jr>Y0=W4PVk6t22o@;Jsn4&+FUB1O@oxnZ%VO_>9 zomhY(5f-3-fVTI^n~ALkpy14e*}fISGQwT=Ts}NgKHFw9dbKnPk(HpJFqHQi3Uy-+ zoZHhLHy?fef#vBM3a8O@10U|)b#GR6W#C}%_MylqC9-r@lThNDmgCFX2606ED|W>S z6IWB9B7j_DPFYDc{3&C0{rjuYXK5i$u0>g2zusbzd>B;A%?PmAx-o5A^7*GjJIQL# zK4sX2yhgBb7NiBIk$7o6F|-wFeh7j`j{*HpHJrpyvtvlK1yx2-U#U-v>W+HY9PuY`7 zIjqba&uM?OV@fU2pL0pEY@eXxhZOYMHPhlM>T7(vPm{HtB^_=uLImr8WlOUZpWNJ( z4$X2iWU9|BcMoGIwp}ZHEcmB|aMLw?d6Fz(taaV?lzmE~w@N0R1Yv^I>O>#It+t+kfLGP1EX;u4Jze^K)g}|W8a(A-k$bB6_ss?z1 zEHyc9>}w}8^#^JVkoKl}E!7B5T9-xU*}!vhzT-MBRA8Y2IZ}&ff?zAnr=2w>U_Csp z=7pX0rWmwkK^m!ZVf00U&WO$p(09gczfOHG4E$~?#0oo_0_rVlX+x>MEt&_Q#m11u z+%^rfYorz#M1_8h4@sxw!9sPljVF=|5j0$`vmJe9p44 zivb32=~gI*f9-Vsy|FM)6(Vc%`T>P15DBvVZ}$Pn1V&F}K_705@AwyMKL2O;@o(2P zj;!|U;mEIzeBDHfbaP`GF`V!L^$j@QIx$=*V3Z`|@!CsjTf4JLm>bdMBpj%-s_H-g zBiDU|czB;Lg#JFiessdTHIm&nt0Rr@BhvsXDk~%MYps6lfF@%t_PMh$XD6Ln5rgDM zcf3eMz?6``Yr!kGdX$c~iLgK2Sm%z>awWn{JVr-9$U7Eca6!4f2sLbSRPD9TPEVc3 z(@n5G!i>B)#&+|QyB@o-xRwEC5B+C^{>A8sVvD>%v>(-=o@L~}!I&L0qS=y(EGzxz zhI?RZehS1#cxv_7abw}w)CF`;H*ICDT_rfi!2Q9AxJqs0@xP}Qsb_SDGwjWZ+JNoH z*50>K^T&dT_j$TXp{!T>D7OynWxzAH{ug)X?>qB5=xMVNKQG)p6?c5RKB}Lk#j#^& zD7~%bvqc`${k(b~ok7cot2kraPWknCwWAI@O;xltXDpd+37>Bn<{~$B6FqnPo=VZR zF6a6BetyPd(HDUb<{DsZ^W0#vSrkT0S5rG zKM{i=>h9w0HIi$Ome}eu{-?O)g{Vhb4Hu*~EK7P&WqI~_rY+1TN?^ah=>}M;*<^>ZVTY**o9VgH;Jk^H^`r4i;VS9MAR|c`%@ZoFTY6zzuXg5id0}?D5GT{ zOe^+#ttfWXW&4C}2?LaISU4})foyRm6QOmmZ^7S_e*c_ORMR5>ZEx{V%rfs zU($LIoxLn;s9aETt97GJy|E7>vg1g(Hyr+gJ6(5jVXvUu

)QK5KH>-q>tV#dbse z$m<(QQd#(7yO2JZBgnqtR&y~j@eA@WVl!6C71)o-Em5l{a53ZE(i!m51;1diesg8h zWiblhwO>Cis`bM;4L9tfEkH8p$BoXepr=1@r~7?<{&OrusmB^}?_GdxewY`Pt1Foo zyk@TLKv;r@c$bmi&tYmLCzTfsRHh7zLom7RFGYpsnMtjS>!9vXam6jjJk_AzQMA)! zh0n;V*{U7CNlDh0Z%M1Cc#wG|aaN@I!OI0ANa_dcM#M?Zr9X)i@U~4v#mMltoFS5- zyb@NCJ^oaR{<`x1S`x|?OD%dAtpJ%03M$qo?71hK1cF{HNMtq|yB)Q-tZipQ_0yFJsxpCweZ}!Dl?m_#NQDoBt6y! zJ4aZuvmU5RWg}*D9KEsYmnMmlTOZ)dc7An{z|zX3+DUq)(9n(Hc%93Z)!rhgxKKD+tkt)Prz z@E0%&#<6g-5UhxX@}w(?5lUC6oatg|ogw;|f7}JK)js{r)sNK_)@;O8R$opmdW%FP z65kZU`ThGDWu&1Z!ky)4tp)AGP~$kZ!~yx(fw}HhR-S7Gq>y!OVeT{ZO6V_fdG$%J zwSD%|p>%hfg#*h-JQ_4!om;;EpDA(Bky8TOok>plK+R0ylAhKf^Rht0rF0Ig7uKkP zsm;bTI)4NojrI>~-h z5OgMU8xq~k#p8xmO*&122Xb|3_yOcOm{lAS^L|~>qAk#0eRP=(xPneny*sd>sY8O* zwmO8ocFuz)H(P)1*KmPB9Gvj?7r=w9mMpeWrS7u9G2q5G6{&`jq~(v%CYWc|(<8Y5 z%2@Q#>Ig};Ap6vh+N_F7*|W&~!j}4(=d&{4sKY2w zNQ`KSdnJHD;&v7kLcrfrf~U6WsCM=C#p1e(AEyz!_rpItFK36!8Ha47TD-UT0#Mj6nh0++(y`8B2PI zv=$qGr(_TctYyru-BNISdb$)+`qka2BH-aB7?FfM!g&$95T&8t3nieypYNTDtHZb& z7i3Q=#3Gp5b7Xl+2z2dTAY9+i^DPO?b3H0AEfTWshu9xMg5Nf4aK?QydE+X8mvRuJ z+`Sy)C6yi>h+aFqxQ|duRsKaCA=@jzo9D=@{){Dxhvq?)LS(syHtIMfB}UuDg5C?3 zGCAw>Vo&$ZEQWxC(&I83u>F;Hyq(}?P59W$+7l98$iGa3>;7dn`WD-Ja@FTnj;DcU zxQYyK(~TVVzI5^@YGxw1?VSxMhNR-XUwJ6TI%k{)4KG&~^>MJ<$nW>v9e~wanG)y= zNe1J9k16pb&uLI+08}v&vkOygz5E~N5s?}%9-=1^USOm(6Fu~BhS3}CJj)9>3#^dcbyjeMTJL@Z%^Wg8?!x9{Kb>3r7$%E-O z9HakG1wJ-&<=;;6UFa7;YhSQGQJA!J#cQHzRc=1wwX!S0wjr)uJpgrTF&8O^?340_ z3riw+x5iuJ=R%QVGL7ioeCn*12t6?`#vpECEo8Y-a4po>cV5ko5>_agg&}uyOC4ay z^-mRY_uQ*6PlJU{2fFm>=$!B#sY1!E#B!(#mZPWi_N6=as`}9!-X%Ncsa`zXaG$yi zPZ^6P#6g;|v$0=MqRbI;blAh6j)0H0;F-TVy>4a;{`_bMqzZWlX6Wd_zPyH->76sI zf8f!GHg&SKt6*?LIt}$Ln$;u+NUVgE>6t@zVBeuHdeG z3V%T4v4b6{B2o9Q?4!B%kDv$Y?&$6)2=n!y9K1654ii}!s@rYP?YfPxQQ5ZFTU=dE z2^X#;AKq#l^H~Qw>g94?wT#YH8CWUvI6K%cPeR(d}$jZtlftBuQ zP(XF2oEw{WW5PIy|K|Sm!6&0-$*E0{SZd$s5tke0`eFiRWyKAYNh*~$&ewxF58|Q- zO;UDw!Fi>mmC3*8ioTs6)byLVB)9KJ9>$XA$i5C=G^ml1#C`UpHWbhK zg48y?O<9#*c_$PO$t8U#X}r9*MknRp`KpzAUUBrZU-h%dG_s;SUwI_6x+X&Ot2yqQ z($q7Sk9AGRp?Z-m%p(JKKEkkFE8{Ye-~J|5e1jQELX1oSM-E|g8Y*2Fex?>4T0U&0 zME68TIwnIzimq!uykzN#H}E@GB%%ot9`O^9=rJ%u!-)no87PQ3$(u6fHq}Z zDay6b%V|BkaK`LLK8e0OsqMz0_8rWBbtINNXgY1mxh)x;U*5HLWR0yiFCaZm_+fr$ zc5RmGU;=DCjh6lmn9@{SM(d~{bY^O>BhBq*G;PDaOR=vj+pV`lSNt4q4ZlsdH>U~+ zB^!S)&1TJfnK5(JeC_pyvhRNatT;z%-tBeq+sigubQ)UBQ0+#IhS`{PIs7krH}Bl# zPt*9uEZ{R*@UOTcgMMwD?il`X_-M=)HFdK6T^UVuSDYGr13x z&}{=eFo=5@TU&KF?eTfX@D+0s344z$U|zUF;$Mi1P7BU0+Sk6D!|`&$qVp~a@~S%X zD66~JSD(W?HcA9-!hI|%;{=qQkIpDtzqo*e@|&hP3Sh*r5Qem(hcC2S8qJIH(4(M&Of`f*VW$OVbmKGm;z z%!PXtpI_G6h5@r|fXP+FHb~QF!aTP~8?YFBC^G?qkI>{;x-cN4p+&Olhe$c{eaH!^tBglf|X z)8RR*PpSbb8 zfkCq&S`A(h_6`5;w8t|mQhelT> z>0&{eHyM-HtVCd^U<#nPMxV2-=k)f5_4ZJnf8Snz668`Gf{^jJ5T`_^f$1LxRcKF{ z2gaJnKSPo$_pBbprzp3j|M~S}0RLqdO?1TrLDe!D-jC=LvsNq=LnCJvpJKASH?VLdfa7H+mJrcq-6LFnl7B;9(>rSTn+H1>;Pk!iE`!n1pKMTDZ zCoPS%COL~EhNjzssR{#1R10ACJvuX)6;sn4dauJd*;n&Bxy=oowqwj!4~?<*g6u4H@3w4BV({Ithj6Fi_iTVJ?)MV57>H}n~D03TsB zs-1Fb28=hu=jX0KIgHjK1YvmE0$J~3N2ybYM2I{s`sdwStWnuJKP`jn@#+RaH2silzqmXHe@P+ZNxcpcRRs2L%V3BSxtLf8inR1GuKY6p zSbCwGNnPKEl;k_JQ($Yf1+-xsUFzKU5QTQd4@(qxgor437t0NMq?qigQlZ{i+Hpie z20qy@so2sdUGB^k5OXfX1t_5DD$96RQ&7y6-tA5Sz6z~++nwiPJ6P+%MfCv{oSbvn z#ejDQtTf(WknruSltOh1@sTDZs2&V8Z@uGi@iS(3M z;1Mjm*|H|@wYH)~Rmzj6f0LhtX;j)%+g{L}QFY7CE+f{ImPb9!*=rxLIxA9MBc^!4 z6Q{ei{ncw|I`!IZz`$71X+9}MnTfqErx|5zA?$Do&LwkXhy^p5k{TW>vNV`WhcLLC z3y#LgE4m4^hXK{sehqc_6O=qT9JJzTFLQ%fr%0a_Kk8^ukxH{CHlh&IxrJX3PLG^h z{)R~{D2pqOt-zf~s-pvCrUK0obedraA0MWznz}7yya;pW94-v%8|8_L6$0O-Ik&8j z`L9gL@VPBJ^`xY^^V_f^LJW=lk*c#lX{uVE0Ypk7)tAGt>qnIHZ<~XODNfFT!44_vds_VJK29!Zc!Q*ZqXf z4F7KFAC1|N+pn2Wv_`!8?Kl2`bZx-T!q8_0q{nTFL-1>yIo4cFQdErK20Y0PRG)d$ zB)3&Q*vq}7&V;_Dir!#as@WohTaHlbBDB%Q5NO{nHF z0lJzoyy^KHFLEf&Nv$}GUROx-lM4^7$eWldqnU8@e9IHPk(p-%z4I%XJJsA~rSsem zYZFOT%TFk3#ft0sXrZy`n?MND&V1)B1hR?=RYykz1Uh>v>BCq*1 ziVIPR%#*&(velWqplV%5o#l!h$Bc)KA##_B{F-Z%C^Ru&Vz36{AK-)$xj)a&vieY>rIa?> z!R0?KMS4W5@!?RX6Vs$}H(1E=A+BZO)Q5u4mxM@Q}V?W)txEEo;x`63GCOZyOa?$`e3KhlzEXa#a?AIxmV<({Ju^Y0T`GkU*x+A-yLZP zv+WtbT~GnmoE^Ka_M z@x_)`yX1|F6MOW?(ht{Rs{Nx`&tN)#y&D#8Tt%g>0<{^HA^~4WXwjYLyyL#d%Y)D! zzOWn-MOkl?6`-$F=D^DQYqDXY4EkZzBHhDgDaYN>L^dA|_3t#W9=^T?r1TXSj>{6)V?qLDoJnxyyp#4ZN6j~>@Y?(M=+NpBJC*Tuv>w#PP?A1TP#nNzW z0^pzD9SYX=-<;8$_)k^Di2h!g91e|#e(*nY#-%sv1hQN25hhkRvmvO>EtU&25dbKQ z&?}<5U&>k7eIXUWCO`mD} zz1#aT>dDNLjEDPCM9DUh7JDi*(X1uWgWpJA<_Gk;;*w2Ibp z#KSv}W%&{Ky%lPn)6t?0zej>qDpyg8$BtpI{KW5Tl?AIu=h=ZLr~$8)-ctOEzxt>a ztGFi8vz^v*M%i4?8?g2cWC#u1zVayMc`TYdVZz0F*x?p#-q6CQU*>aUvTCXO*ilzC z?pgY(5u#(RTA-mSkp1}p-8GrDP}E#odYTgT96VjfBTCHrPsADApVNDa4`jIzhfPb+ zL(6(%3ha7Y1@=obxPhUUIDFP>0QgW5u6)D(OsydA$L8$PWb65CNsk`;4BkNydF*l=2oZ=f;|7A^|7P}^H5e<3?-1Yuv-IBF|cRcG&tTXK|9#?@i<9oD-#LI@YD zzm&+>kRy>S03WE_)hBT7A+?QSkDbxdn#~Zmwb2RBoAD_j!JsKvvt?s< zTX@|GC8Ykx;!vW0r9I=dm!A}RQfZ6b`T`(r9vFeb^;{LTaJCftHo5B>W58!nDU~Ea z4{g;p&wrDwU@ci)j?^uvwV5^5*eDNbCIebG>xQsu5H7oDHE(z-@m>@7Vi+T{iS#UF z`!`|D^O_krX)Usx!=NQwq{@6{89O^PctCjYq8zr(P`ndF$!IGGQ!p0E~PJ@WM;$=9ER|*@=y-fjF+CH z(cu$misDS8m#w;u18PTke#-{DE_kdiNSiM71K4(Q{Akl&p-eo>y&pLOLz`+#1(?s5 zS|G~Rx(^RRih5-C)DmG#fc=J-ctpYI1vUM(C-%mq5z{K^n<-6+-$~{`(<%FK0Wqi~ ziJ_Fha^eG*@#mEI(1l`4>PaW;MW}jIvID-TUIOmeP`Kqmq>nsaLc?DYuRS(DaEX3i;(%Lg+Vi zQ(18H;?r*18zzqLhir~%yDWuy^wRq(bzz1RF~JTgLxHHh|+b!C4+Xop9EeFsz|-ZX5j^5t%#I^k~Cp6)Ajwm;<(yffkxlko~m zmay!c3OnP6W?shv8SUFjE}FAi#-9*fY=3M*26!}GC)RdrFj+&VM{iRS)k9$#jv_|s?1z$jE<9VRGvJEmNrGa#f>84L7PR@Fj23qJ zx2QBA<#$S%trdJ<1zOn7h9zEP9_4AP%b>>He9hL7cAlVCL(n~P_tGDN0nFfTMATF( zaoHo^IZ5eTEIM0;lmEP%QP2fZ(PBcHuJ*v849*Z%r?K8mi zVQ-BNQ#u799eYM%trHHa#$(jcsgYB;i+2inkxG6V-Fzf zl@Io_OB@*7KJw?-7h3Wesoon~5I!qM4{7t62qumdR^T&#mKFXh72-@_Of2Hft;(0X z-;SoPQ@s1FiPGJcpQmzc3{2R_=?SqlM`XO=TkRS|2x7-jDJpMlvroF_a!dS;+yXW2 zGNi4TZ;MNCt;Nl^O#mk%Fs>X0U?i;zz37EJ-4y% zR5~D`p_0`~J=h$XR6&NKHu4F;d(mvO0)9?*zN2-d7h%+C6nV%hVE@dsit=!9oC*H`mPX&ODL zwsqApe(MLR(0Cu_0xYmlwtFxu_wPu*Wvs||72#fM8MJfi_*u57_>-F`#7RI&?vfML z{P@dS-cEjx#>m6A^&JM5B zdLr5n58gI8uB-+h-F;3+156is50CL zsQCy#y@ww2?X{fi3B_66ZD#LnJGwp=_@I1X82v0)X>;4nFc^}yt8T@Xy|{& z(#4oDUe8wQFtraJ)nxq3QSdPJxv=Mt7C5k<%J8077zQkX)A%@{<(QT3abAQ(I11Wu zqs>LrnbFJm{c=17O%eX_M4D5Z$|m+aB-mr{#g|&sq$;(fv&TtsBlC)#cPC?B4F|oy*DLx}}GanndT3^goW%8Ofs>@a%xV@Pi?Y3$1KBuMElQZwwJ#z|P-uEc-@8#D$s_-S1`ZJa#} zJm|w=6ng*|kE%iTWZKHQk79yK#tkHR@bFS)fYfyjXaQ96@B=WuonfCz{GYk2!Yn}4 zMF!N0fr1q{Y}EMw(e`lB)Kc9PM>3htegph8U&p}ORA2SibGGTl>xx+OmrPMHM^MyI z#9XLMCz0E9Hx`wg6w6~(`R#fP#$d4#UgUXsdHsB?@Z~Q2xL(<=rpPF`bogR#&yuwi zsIzrEwcFxG71#=Lp}BBHe-ZesaLpM2L8D2&vux_R<>9j!(6u#D18l+m+a-rg_ z%{q{$?$ukVeUA4wrw3HZ!-49;s28*s)h@ee`R2==>y3?z{6F+mZK&DS;=+V{qo6fJ zKuBgn?Z~e^R1JctTAme0y7MTh0p4{<-R*I_;Ud&QfjTQwdammaC#bBZEo&2)-5A~W zIJosqzn28g{L_Tzr=;BYyiHq#Se+Q|IK$&%w#uq2{UgNpXlIO_?3ogy=h~I%vwTzR zmOI!&>efu}j!0a3+{@X8jW?)o)V7{m4|#aYwf5(J!$9Jh-JJkNg`-x@X~;eNJEvzf zu9h^xecxv{R-s?2PGjWBpcber19zqn`*y>&)#ZLY3Jp-YjR>CI@LQsVc6cD5q4o44Hl=E?yXF$ zG6aD8an+H=5waZi#U)|DXhASWXqG`CU=y9e32cVUBmSzdnQthS#r}vV z!`|abc}<8Os=~zYG(B-o!N_^@U--5PQcH5RX|b_}NuH^RJdeLCJQo617l28UbQceT=5 z6fTcpO(ejqX}|%Fc12*txq4tx7$B+XvHH*-#TFTi7@ts4$zla%JXs?l`T)|RyF};_ zWZ@~6IRL6B?aJl;PnV#ybS0GqPLgM8Is_5;OaO4M3B|Hv1GYycWxCE#*T)nR>ip(C zBZ6|#x*V%)Lq5lvLdhrj!X5Q+K_0Ur z^U7GE0+rC)WpJg^qLN;D>DF(evY-ILBty)F*C9IEJ3iIleVZ*9{ef0ZFkSDj+J*Ej zkTAi(VR4*}i;8msv{J0ExB~%Inua(+y|7Yrq7dj1xDFSD_fxbM&w4BDLh)h^`Fj)+1Iqtbg}ZH+}s*ml=bNE{5l)> zeF?6a4=s{4?U7N*K4L^(YlfUX{_jZ&)M#YSv4q5GodA^ars6UFmYdv_F;Mypj=9uQ z+ZfhW+g>x5!~j^ z;Aqm$fXKE_DNM3LICT&dA$PWnZ4p`M*8QwFf6F^@JLi+(oM=#FIU1?pV;13VnT*dR zjoL`P%ISmaQ5K0U%uvG!Hc?8P_4DXd^gOZZenEP1ruj$f@tN!p7d@C0p&0XJ@i#|@{AM#_)OwrMc=&e66c3W+w)chA0cnUV-q&+i9?IlD|Uc&12nLbaqW47?eZM{Kqyzdzh$M*d2HPOyw1AEdPA zIqD`0OVZe_vayxRwqdsA6UO!y)@7ZO-9}(dz8^L*ERX4C7kjC|{lfI#;7FexruXfB zfvOk}AQKf1em3s1gpir_NXG#>%QHCa-6i6;JWz7xYBK4EsB1RB1eHm_Ai?pqe{(Y< z@KeZ`yhZ{jXkMwgKQh54TVl&Y0jkIplNFi5)?(}4w3gEoc(j#78LymTZ_C%#NP7P% zeRs3Yn8tCt9hJ8SvZ2j*1i_?PQ4+M)Hc)1_#`DdPi!dH0VmTenW9(gNyGqpMSd|cUbC`{{%`LEkX&T{yZ}W zS|GW>m*;koqj!m5&p^MF&9X2vj?&E=K zY7=|mN^NlVpdA>D6ifB&vt}wJ;`8O)t9GcYl_~&|jz3be4Ozo)^j9;+F3~m$+tX?A@(>H7%0F^3kmT z`Sxq6JT)VDs3(P7^km84+#oN(Gl@Nc|49!&O zns$sH!7VqOsi_1kD_Q#FEwLErCDRU*e%_Fr5$gkSfjNWF{ifUkWdmuSz3$=TZ_@3;RdZYK+o2~jx)r3{GQ7=O>0+o{#4RwU&sVcOVt+kY8oGsKX<@z#!czD79@aC?7ih)v}N?iuWT*2_~m)= zYHh!3y<9|Jl0j6gQ^T9}2)*}KV%~7bPS%|6Krc|D9n3OM9P@PDjE5Mrq8@Jm?T}qW z6ZVh_i@oiFVcvgwQQUUd$|_vCzP3R*l~h1oMK9V=z7B0$n`b z2le&!xhS@SCt2W9?vx%x@ zgA>K)OP-H9Qe`bvU+;zn->Z|2BY!&=qfd$)=5DVT2bY%d?ng`pf}uba+$b&Ytr` zPP_i*U}NFmh5fGA{al`fLl9RbAX{VOM69bWy1M!^{Kqe8@o_^)jU~uQzf~^;82kp6 zMtk`PA!td>CI4R?B^V4Z{*LmhrOI~;2|R+Rtho%#db_1&1N4o|fKTtUi~m|Q`6`;9 zT69=y4)w``-v*aRFBJ6J6u%~0O=v~Q>Sg7XE&fhZFfT_Z^(qnv4g95GOLaS0XQ&X= zrw*b0bi)sF{t|jOf}RHKVzE9Jx`^Nn6!{sU=2F}zbT*A$Jy|`1R*Botqx}yobk|KG zLG+G!G3R0Ojq>$|=>%;whUYLi+%<@?;3KFQwe40+4Da4(&K(Ox0x;agR1nyk3tS8} zbArq+%RsJKB!rR+1x`5Es9>P^s;fWY&I1ciuVMtVO$#`Y@80XGZpsE9&2mEI?*Q4 zTvyY*f!MX7IQCc!j=1t%N|kQ29($5A>I$%%9Zkib_?>#8es}3db^1JepTtIs7pk_8 z&SKdrb(FN@_u!_n>%jA}Ta{uVNnki8qikSU<1+W*2WD9I2abCH&ZX2G(*wIzk)&?B z_2;w^{x|Gli{tXJ-;iSxAN6aw;C);~(c9OFAUd>I_Fp`~?9?S%OZKaR)qj;6xBhM! zFAIqIBn*auc;@)SJ_)V7oXIlcA9mc>AD#<%KgU5vRcptL}z#Gy?oRb_<_=d`*#Ata?TttHU zIZ{1_EV;IH7&v1Fl9JzGy3n)<-Y5mK+$2>(eZmm7$L~Y39)B|Y2wsF6G;vtiS_zBF z2S@|%+8II5ldY)g=?To$BIS+cyY(w(P(=pDRd)H9xFQgg?WrnflVo1mdE3?;Oo|C3g zk$GsZ^c_*wA{aw;=sCgv?L^c!UX3>F`J9r_vlInH!poM001?cF0WQ4J7@wCIX1$9SI-h=5mSzx z$uST=(!AI3m7|a#pPVR2kuD`MTv=D++d4~=KBS|WW$fIrtNr?jC`hCsb{+-in-)y( zR?9$IEZz%JC2geLx>SybT)$+p3}V64L~S!(t%L9>T7FdVnnz8wr$JKqI$GQu3b)a{ zH)J4N@TEe%ITvBmQgsg~sLTun)}h`W4uIxC8c4O={i}@JSbtu3g=No&2^VXe_GP-_ zV2QAe)e(pKpuy4pE_|C-kV3HgDxY7P=*qe(mN59KY4X~OHYSm8fW1`-#DIe5d^J!Wt-#)4bB~mU-Aad!S;Z`QC~sQGt-8ew7Ue7l8dd} zl%*UnbU#fYIzBwp&%F$gu>#W)0a#OIEmwe~1uFy!g`vJ=5<esR>)dh#xZ=bu7m(?^sf+4a+=mu;DuhM!Fp1ScV- z4CoS0i1n8m{zwxv4jd(}j5IuK9 zJK>$OOtkETNQBMmFq(?-JqvQ(Qcj?4S9Z)QvgbQ9CJYceSc^Ha#(Sp0v?TX&P#Vye zCUeBn4;yy(1tg`RGH0;##H`;uW3o3D8q?w~oaUM(;p0fE28GmZ7P9ZEOk4QvBpnjQ zX;Tz6J~V>)gyX}q^b`lnPOolMxI>}EXSR;p#HW=GYd!4FyMHqC!a}sQk_A4BgJKBg z^HaQ$VnYV3iJT=BVuAaUkajN7QO$LrX3I1K9p^zsod^r$5O_2_tGH6P5iX)^Xx_KV zaVm62t{7TBbHR>cyx4Mas(2jeZ(gi(D7!||DL^SoGyz{xoyLA0Q^SGq^oh;(&$Iy< zwbW7IvK&6hMxotHNtnH4zGTS&7(u+7Gs>kUEB?nq<&|o)fs5ZbJQk7glxxGwDweF8 zr>!yRYp)>oIq&LXGO^}T8jVpP87&N!MZ_ z<%O3k1>8X6$Vtz!5T4x0W8R25?cas5quO%^=<5foI)mr9rBDjfAzWm^M-x6%=sHUM z*$^6snUfHWCB??bwp2E4cEulw;F_5_sj78Bzxr2UeRV}0hOoo;M81@xVi8+Wg>f-f z?$Ma324>)OMmK{|(|+_ze_`o*i1%8<%j2r4`ziM4lj9Ay;-v0o&5IVivcM)ji9#1n zJ^A@m8uwlEey3&hebEt8l9I(#2p?mITy)Z63>46H zIXCtn8^?XXI>4nk9xIf=*3M-v5>tnFt3m`iqn>!;a#3Q@iy9oOY+Taz*Gbu#Ve_*A zZoigvc&J+T+G3~rddLxPR!?46SX5c>9F^unA?1+dM5Q?*f}swmX!0_J$sOtU_V*g^ z)3vtR)}S-8vmv~%g0Ob^8du&C)106g zs~6O``Uh9Y9!Ar$T+~)?Q|%BwzTLuyZ*rdwsZkXrDBFp{q|+ww2KY?>2OWs_7FZ$j z3j?w2-n=CwP%unE9!549Si*`p>m2%FKi~u+i0)U{Vn5WPNa7Iuj|0r@=mw>*I0iaX zJu^L=qXDQrRoicHvVV%GdC|GDtuEtyU_B}QEIB^Cy^WATvM0BjL|it#tw=n{kqB+; z@N>$g-4E>P*x$(tWJ)@&A8QKVBLoX^xZ-u*4VCD_vV}R!W!rdfe5&FN0J4=o;E3St z3U~AUfrP2f4qfHdGnP5 zn$xsfo1HKN@?o9RWgVqBg6q1AikUt1W&-cZQi9n+83?cU9U-SWn++QV*T-kM$4yhq z)3B+$zmaXoU~2tHD3ro)WQX*$nj@uvUx_`Rq*T6vkSB9%FE9pcXu+V-DAIR|#gZJ1lQVfc9E5;Dur4 zCx#Adv*H8aa8FYvj*tvzM2^)kkU@~E9?~#W@5ffDbi6l%kwfr^Vh)H?)TB4y44asv za*2IT+H@eLtN7d^13P2(m}H7cGR#7)atdJSKb?n^=jAD-XtCrnVD4_Hca?RIaQAek zlz`)Z`-th^D%7BvdE5sC8vpYys~(W+H$XlSZ?k9MRns_B$TZQ;5_k8prp2D{cIP6Q zh}o#1%1GNnk$Kr$!7d5GyUWB*+@z(sV8(sVW~$98IL&wvqLI+77uFJOZfy*sZaX}C z;4@5+gbO$5?`m4jW2}dt#;iQ<`3QA2?ylVV&V-EI=EJ<7mFc7ysFd<^oq0?4d6pPE z%^Kq#)-~Pi11C%w96p<*fG8IpVGVI2e3NK1H0J5A+`B>BkFryrS5bexE^u#!ACHF9 z2$ClZS9L2#e6LDDb@eC7cQvfS6eUg_vh1W6r=e?K;`Q`Mo_Px)T4B(J4Zj=4!Rfnk z(x5EW2kr}3o6zjAYQfBD+lbgQUZ)Cx%ay-BPYyL1+xkqNHYHOkjtcNKWg&UNeA0Xw zpOBY@vpByHmsQl)t)^GX^vd}4S@y+E;5)sDujR1 z^})1@mkl+6l18n>`EOXnW+zn!Hs=$48Ch=^l#I-`4kuTq;O+_^5f9T}0bU0kaScIT z_6&Wx1Z1Pt=c9Dc{4;;1s7_7@4@FivWS_yIX_z*E*%D-T$Tu}eq?b7nt6uOjN*y2X zOlX+aAIw|&N~1%_)JRBJ@8yECZqY+A&qEk+)1K6?){Eu=M^#J!TsWYGB!|rVd`@0d z_>p7Ml%-C=w2frmh9Q>6WJ`9$Af|vRvRqYIP(#cuPv|%XM{ck~b%VdZaFEii;qPAG ziq#8AOxvH43YrwK_RymYEg2l&bP9)ndm;ny+RN5SX|4O%R2Sec$I{pInB6D_M9P!U z!0@1Db{(qYqs?iTkRaOdF%fe7RLi{5=~{`NSCFuRa**1I+>BJ8Ef8l`h7J*e-88OC z?h_WZ1ENMkU-X3R>#JyyAE}DOHbO}n;9Ee%49Gd-FGc>5bYaVpdCOpHd@#{fy7^q; z3-X2BkHxjJAT>HCdiRVw#U&-an2uqy`-awOg9H(o#rrl0k6tAou=NwmHWZ}R$7l=O zN=yerXa(Q`?Kq=e^--+i;)0@NM2&l4V+5gHvb!-FcYdUQ@CuV}iG@;iMm4kKDo*=* zCPl46^d1@uHKxnW+x8vKfrc6MNgMH}P8PP9Dz#%Md!!m?n4<~W#la#&F@>r%t3`@; z-sPq_Uew}`O?Ecr5Ku`Cf?ReA8!10Yvr(H*lp}JgS{ZMmb1f$ljkh7kaU7k+0V^^q+qYV11k$5=k%MFHh%lugLr)XEru-0->V7ox`D zt4ufju*67_{pBqgrm>^c)FegZ&7GLz5sbe9-d6@z6js}SY>l*?9=5|aDg?b5&{jWk zjXr$0+nHgWu?`Vhy~o?O+zzfIZrPS2Y-@3vzy9w6+J@g(z#S_&kAj^!n)0T2v?hP! z$8Cb{ z{~1|q$J-OkcOzw2?mh{QMFv=j4iu4CFZ~*)A~xREQ0n9@%#JJrxnwwq`wN|Yu2^RE z%*8zQVV`8NU?7$_shwsW51Pkc;@R|V`%b9}qqPoyo2n=VHImnOhkrY6YbbPuazEKc zvVh9SW1yaHKjb`Q>F*IzhdR7weWd(8`3VPmzV91Hb0q?+I8~y8w}#Ho`2J zzSxsATKy9rXwA!ltR?xdJ!MIqrgNDNk(;5IOYp7vhrVUYTYdPwUxN26CWVuEGuqmr zJ~-*g{O(Fg4>z?mAfebIgLaBmJCKhxkl}D=I1<;1|02TlXCB&R(j-ODJI=P!$A^2O zmuCgBOlCfwQB~)P%!82u#sVq}%rcRFkY#G`tN*yXQfT?RRG)kN9d?qj>R*nQIE%VR9j>H`a?SfGly939_w0DHFtGx+ zj#A==vii;83n70~*ZGn<0;+!&{??7qI!)CW>>u8VNq;3rNTzu6L5vKp>MNI=WmU9h z@naXyn+1_bvnlk6?sy;Y6gyaF?<=r163$8OJrn7{2Wu^Y=5UFSUw1UD4{7KWiXN{& zS?B=o?b2Bo_IG7S>|<9=MvOOjqDArq6&C(5G1itjKneN+Mwziu)A>J0o1V??GCU4d z(Dme3SeKR|K9qBHW~q#XtL6l+$z)EiD@4VKV%s`O0h*2s3xsv}-CvYIlk?u5?(I9@ z#|Y9|h4cAv3L<_vr`{pr;aF~0^b?JM8P78T5WRB7CdOPyOOmt6lqq^R=MaavJ5;nD z1i&AgFQ91Dh^X>+lt^WoM-0HiMdvVej6HbgiU)D!F-I@2nMz~H(YDqC3CilPA;4(y zaVi|yX*tt^S5i}+L|MBUD>Jptt8mAg<4SdhhTmrt<&ApQ);0y9x;2$?h@Yrx!3aG(Ol%oeB1Is|t?MHT_dCi}ingEKV$sKP8LZ zCQG{qO66*bJa1~t)Z_dd`(lquK=GOvk=o3iy5>v_&OBpPS+9<R#5atY z4Dorx33~c73vQa84Q+^n9k=^SzjvM6!ngb#eZg>>1kWe7VdP4Q*TfAt556W{B>ogO z{^DYrW>ZA4s=^W1`_{4fc0uM_s>b0Ba1+6lT>B}YB!PO6Y%;_Md!l&&wjpb%;g5xy zO=|ouc7EnTs^0>_NP6sH+oQfWISqqoeX-T~Kgs(%g@_R&G0N3LIty?$3+dbG^*-wN zn!CgqC76mqkdoq=a*O8}_d&vEG2RI!-_)*I`PMga6e-K=*-lMkC-kShtWx7KPh6Lr z=^l`TB<+3uDn~RH_KhKJsyt+l#52hM)(rb5ubd3Vv}%hpOzRu|)!bA0c!I z%K|$z_wYLn8@rv}4cNUQ`;ZyAK5PCf_8d)a2Qa$9PZhG{&inY!+`6OEyywGe^^wB9 zy`8Z?BeGOeyafRIO_;ro#tBANA~QH}RG62hazNm0kdhdVxm-*%|)>>rP|sz;3O z4!F&#zM@z_Z&BFp@TNsHu9-HzYWYEO-Vw<%|G-j1Jq~ZHV>y!%IiZgS^+kAhX&E0S<>y>|w0`Gj@4**4Eb2<@dLMAdNQJXSf1(`~Z=~6rjVY{u5w3{MCNJ?5O!k)dRmwlhywbVDtIy z%x|zGJ?8R!BuX$kV4rYCoQ=64#qjFi@8+%I<&%r=!_iQ7t}1jp^O-3u@fQhje#9TF z5&}BAxwyCp{cv+|yxsJCJIwsBqk*^Rt$f(r_5U<)p$<%==4`~utf88y{5$YHO&iks zDR|(!sM%Uksdo))t_|zP@KUSFT<=-Lj&gij=UYY(^WcaIrg?Iu@cFWa$*OL-s+a>)G`%#bgB{uVg{4T}K73Xe$ z^JUgIAHsXkt@f=%y~KJUGKZ(FVwBZghV_Z~c|K6Hjc(%C$qQ4>cW81q&sR_V-tO>W zz_0^vJK1^iHNVZ?SQx~Hb+C1M<9RjS-AeQ0D%E-c^gK$%ow-xuv?ID=G!$)U)vxV9R!2p%Z$R z`7f1f4pk)%HXq!k{z5d~DP^v!nr(~Bx{E~>aIYUpWEb#C%>(p4UvIU44VB$yp#PYw zG{bhZyPr18(XR6zQ!Ae+yQk;U{-r@kaG(%o>{`oI4D}sEY=!qEo6!LJ*YzkIX${{% zotKFCRswbdC0csn719~}#nYNNsAr&1kP9dv!4p$a81x@~cB6ucutQ4Uhd7XBj22#8 zo`UtWxc~wezN9HzKvg7!#~`XK1GnIW?`&JWr80WXY@MzL4P<&J*QTo-my`GQWEf2r zZ9?dhml2*2Ee#i%2A4n#JCE>xrbBy#fgD2=?S!V7`q^0UvIwI@1SBz!UnMVleRW8JiOx#m-MA>^G+piGi39*pa$KYiW;Rz@EgR~S5ue(kL@FYQ=H%`p?AUByHrHwX# zfH~&)XqkH&kdPw*WiCk;~ec!tT0l`Kp45z^NrCy6BOij@P)`kuCas3 zug9HEkd^o*;-Xm&JQSw4349J|}|THszNWOo`L+>Nlk>4gUe0yVA0eD>J*l0U%jrw-iwREU?0$h<9Lv zt@MSXF=zEABaV^jGXg?Um0cp)^UTJ2P3#q#;1{#P){uXjVc_~bgkK@ApwwT&XZZhO{s3) zmua5h${f#nePK zQh6rdd31Jlp;6R@nzR1?R^Ge#(jb4qr25g>iu>du8jV3K6^8&PPURPCeUOsC6gryU zu`a_Tiv+3fh6Zh<;uY)ZTK33qSG}e#xPmo%iDN!eHj4KzE-;6dxYI>B1a{uy5H`A5 z*FKUtb_1-dNp1ZA=dqS7?1CIfTN8Ko3+?;wedp_$-<8!E)}I+sI0DJa%Vi z*)U7BzhskpmBIJzDI&ZG#*34=ev(v0Z4+ogpC?k^D~wIS9pFR1B+`U0NZNhba0L>#D?~>d%eOqwCxFOHb;Or%ug>=S;0c|{V$Eu) zs)QnS1n}6VCn6bvB`$|P%9~m^jh$N|QTNZGSru!6^o*TX6?51Bn=oex?;FyQjjpwN zd5Lc5!=V_{HUr%SWs(r&bYpe8%?_1br$M-VSmmIB#^D~M4hW5R7i~;%H5|YOTr#6% zbGRampvK~%ap*uwy}4Rfdcwe#PZ3yVnm~$*ph@e$a#Qb=?)(zGP@*jM{N4f5FjFa( z>JG$DbR>{Z?8ys-8(KWbw1^>)e>)JX2`iLieF7d9*e;~$5%EcuFs#QR?abd!7h7_B zeR{u&z=;`}=S|rfGU^CTT{aVjUEqnqC?40j1vP1^%bzZ&LxMq^hm=g1^0UtBFACSj ztyS*P0|5&MqhKDbQoo%2x8h=F=|ASA*N+Q9Etyh|m!hI{C9Ohq0d*sFP{<96S{O)P z1hFIJ-Zh*Fdha2$lxYysnJ^AL2yR5K`!V@yo0L3lCW?HqlT@16AN3v(#>%16%_zdk zRoME5iOO%eEpsO%jbhg4F~_#5%X>SQb!Sbz~&w*OGy41 zSxvp}1Lac~C{#8=nyHkX@{nAvmp*=y!U1t;+Vg9f0^Q|ujVB6dzegj5RK>GY+%#V8 zV}}$yAe-wDN5UiwIh`(P$LaP%ksV)@V%L3SC$D&UPcGW=+4vU64RG&bCRARFR%n%v zmOWO{Bf8F_gJeUtcxQ2|?i}ymkZww~viuQmxJTsrb#>Y4Mi*#gHa=73Ak>YV^l#;q zFWWM^j6R2}+$T5B$cnxj@rfSo43AD$k3&l0KeoEju!sA9S%QG8#gO%(;3ZYO;96tY zQe1gCRCex3`M@x?bW_817vsL@CIL5n1_F~|&RZZWZ$Fmk?1(e7?h#|$+n@W~(_CjH z*Xax*_en*0VaLnH&9m%|Z06Y5Z1N3*;iw5(|FF0yOtXeLwty=lmPW5}!oWo=gy!?+ zDws`RPdU=;Wx(qvfUy5?Y4jC6O5Q4(kM_@v+B!VXo+uU3pIr2+6G1Ol;bm_cgQ!qU zb+tikE@MS{>WU9BsUIaQM?^y*PXf$cl$qPKI!JsSUO)!Sh^my4?M;smpWH5v0>`j~ zpsq5R_-TDyvZ2>rxm!{&4~LX3 z8LbFBhon`+G?|EIk=EW`V`{Nq=^A2130&B_ zpx?__hI|~rg*}=UJK{VK6I`&*{5;HH=9LMs*xfzp;}utIP0-8D+OP7JevF_yt$Bbf%`3e?ZRO}7rVuqJxi zY3yXkNfx&Tf^M+GyoLZa{K*>=0yTY7*!SaBcfCMiegZHwL=91HCd3NmAo!mI0HQcw&Vm%CfbY_o95jhUWrzbD-KuOl)AIL>Z-mEN2>e9F~oB;+y z-eBeb7r@9}o~KZwI5Vu+MoM~ykj>0(VsO@rLWPkMYJZbn<(>;1WKrSL8DVj(>fO^i z^nLfAGkV$6rKSpl_DNcROn<+CsOvo}NC5z=ZF4AC{hwwp+Rp^XDd|H5CcDrO1rP{V zve8D1H)+AHX?2!kpLXgKHrY?l=Wl>LckCb%arp6jm(#~+XVx;F(V=ESHCfW8m>)9_ z0-up+C&L~WKI9u5iA%EuZ_80CL=>5tPsO~zpSyX2t-^6m<(LvxmFR=a0~5=%e)I_c z32=cBC)u!`Jh>7#@mw7GB^pz;xul3ukNA<$ZeritZbX;8;?9)}A4oF1jXedFNSl>y z*`6-M9`+}%X4&wP>VRmj$GO2df{j~9U|FV42Nkd6tDfv&EZZo0FAckE93Fl>OMz&Q z&om3Ew-A~fK8o<#=nitzcn3!pqVzv6%E9vim~`vQ?NzThiD&3`{NpSAHGqPb4d7ls zd>@3+V!`UfB~J{uQi%sSpxruw zlOz1Mx1HS@bib}gMqZl`4}Lw{v1AJa!NH}3`O386KoB+;kfSXrSgAa35q>01m*?lV zgyp&cj|6GVQzXmqueq1Wy!Vj@@U8Kd=3A8u&UM}(rjI!()^vo zYgS#UX3{lw@9VY$H?{jtO&58L+Ktsrdr^6O$Ex*zY@PF8rqSBPvu!uolWp6!ttY!C z*JMs^s>yt^ZQHhOzMc2{dj5v{bML*cYpwNNf7}er&kh;760rqu5sA+AQW-OywyBo( zuwNVL{ig@T7w(Y)>^0=oO=dcstMtvj^X_ykCmd&0$0x1ywfVSh!tNK#$$2{uqpifP zK%7yiFiSU9Hhulam1W}D@7@lAtFFKLh5J7I!j`k13NPH#Po-?rNJfqb& z96Qr{?hoA`cH0*dam6Ncd3U~nr7EK1N+6NXr__yZoc>Dmhs)<;K)xkU_=ZvQpo3(< zJ9nx>{PV6K+#o3(wd>hUq5y}V^dzJH#jWv}qj3RqwF8ALK_9+1pyQ`Cuo*kka2cmu zA7IT&ibuEf{r@N^Z@5o1uW(2%*OS2CJ#6!A0-qtw)Ii3A%jLIiD@IBk%Y6Y0sKvQo zONe;p?Y47^+?($lXR>0k0$&9Bg7?MudI?`mdA|hga$-SA-%qCttuv+{ihm>u#o}<5 z1DLd}vXyJDZIo3+E^qAVjV8_5Y2P8E1Al?Nc#iuoBO|F&D6U|TTY?fNyc*$Xb{;}_ zst54L463t0Bo&x*+>DIBwp#nQna0M(i6Lx>|I?TbMm`K1ty?CYG?_Gvn2K-Gg-43p zFS2Pek0G3A)2+je$6Wlw8Gkkh0;3>wD-#JvpAS#x+--+n3Ug^nGGZH{*A2yL)Qtvr z&^4IZIX`><q?xBSoy9vV^IaS59}?W*N+j=CU?|#UT&bF7C27OhM$}~7LG)0712r3i~jKf;et{1 zBtw)FUYstf$8I0R#*fj|k6+=iNi;kT$IuU6eUb52`?SsOm*s1wj0Q#mAxb0ZX<{!X zvv9uI`vYbAhuPIduwwgAJw6TJrnCtaGcsIof}G<4kvlHG9#%3y1x9hM2&Pb6++EQh z`98PkBz27eXUwgmG)arMx2Jj;#bA;O%P8o>q&)=@7(#-FQz~^KhoAs~UG~^B5F#P< zL$lf>2+OR`Vh@MzoA4FZC!#?VmmbufAJOshTbkJ)$wacv&kCB|7fNcuie6>>L?5AOnn)2V^0BAH*DQ<&D;!Ue;O#&xS^I0#fTYDRZ;Q zwZ*yxBlp;oS2owkug3#4Rj<^DKkA!6Dw|OcwDxr!k*lkE9jui|8gdl#h@l|yyt6Al zQ5Zn*?h6{P*Q$*;tMo=lbvHW@S`rsG|BBO#kTMp97Kml2W{|FB@Me<0k&@VmmL8b{ zz+Z&WD6ZH|w&(!*?+dXvfBJ4#Xdhn;G>c>*V~2S0DP3o0v~K0x?SRtlT1XtvXdIbiU>d1Gz}VbhB@c=wI4SB z)V)l0)E#H^L`g;f;M;4Lsi96Zo#TobJ$u2?pk0UY_hzrqNeoPkq@XDi;-)BKGeZ{o zodpc_VnJ&{ne<2o*o{$!iD+;j9Xl3ih4=~kruUXRN7g6v#j{T#e?z26)22d8$-B1t zZRv2d3}n?;2CYM&uM3~Tj=JP1k<_xVAp6uw{~=PL$ijKET(yL|*WaR# zReS_%?H|SJ)-X89Qs$cPuCTsK1UJ0Tc#UIFk@ElK!Oa$iMd(St8=;R7z>{sTzco4g z2pe+Z_m7m!nttnL`biib82VUaSD!8G#jG%+iSacu^UI3i_bROHZ;}+ya~$-?Nu@Rq zj1%<{Y*k5(3)h7tSRLX!1HZkwq#}#l669|BIORQp|)lX_B_a?l8lk=k+2ubA+^0gCm9z906g z?|>z+C6K3Ip-iPil;QW(raq|}hl%l^uPe(O4Y^W|eF&)k;@lhI6(;2v)@aa^NABWf zCWhx^7wzgRYHSir_j*w5dCHhJoC>8B{R)QqmRRU<`AW|kUsP-j6C`-#>KQ}OKNE?Pm@$Qr!}F4!Akz&8 zW!89m;fIna!g(q-^0aI7rXZN#ksSL;%ThUh5KU`%Ni zw}#lYn&rFt?Bg!Z*C|vWbXT1V5xHS5sT8TMsj0lYgNgq^fDORFv50SP#dd~43Pj1& z2W+t^djUV_Paf3vkFsFGs3U5F@7GPYU@6puxuf>Tq73-a)1xGU$pwHSU&QNdmU*-J zkOZn4znR%OyY29};Oruiw%`duE=W;*cXYNF`u5AN$>H&7mUypju+z&Z`;>!pO&R{4 zoaFLD=F3qOWo)@2f36?1WrNO<3CAnh%S=dOcCfMyy3Wpo3;OJEh&m-gx4R7DBJb&r zXZ^XOGN>vc*Ja;xHhQ?va`O;l?3j4R`3@Hp4c|$;;&}CF3$;@~uhcDyU~QCi#$`zT zJ)6rlN^U|Zb}aQak8MvxVnU3L`?U*+1?HZzMD8O3YgKR-$(X32>Nj_%fa6C-z%h~$ z77R6hn&8?%oNDOVV$$?!Q)K%16U%X(DMz&ZV26EBZm)c^H;Z9%dxRokHK`92Q>T6C zC>YSe%zeT-l>L#)MfpbSAW_Ofs5Pe=Ybi$<<^t&CK*blF@_HnEoH`}K0W9fV^e4GZ z(pq_)6|Mg8p&v0*S(;_>%>HsISGsr~e|j-;_DsmS5m;_RoAJYVBc2zg3s21LkIS2) zBbMGyv<0610KKy4Z7A)CL{paNN>m5Yo)=%ZfiQ^`0EEWJf{vz46&}6v<;?vY z>Jf0UdHsPwT#qKIj_f&0rXYQ-S@ayyqJ2=Ljue^z$e+VH_orxTV=c*9RSs#o<7~2R zL6JTv>=wLcO`vMu&?+pt+iWWdx~~L?!2+S%VNMimhX~yK_MQX(7Fm|e)kyF6v*~>lBltDPHYX_XR#c$1}@uF zMy?m4jsN}Ff@hIct-FZ`-0crOlabnp;w8Zqp0M#(u|9modR(>%d!%;UrtS`8P8iV7nA>G=!gG zzQOSWGoDy@mUe4FaC|(db#wRzUwueKMn&zD|CWuf6HOoM>-Y~h8c)2hxx~TdWDk(t zaEBTDJnhz?{`@LM!VeM8l8MW2L_+Yf8VmY@i-#liT3dql)-}T?VTuZpiB*5nTkWol zlc`N;L#z%P-R;wVRGBa@h|RBomk+B(wtzd??KY_GFS46FH6mdXu??sJOazx4>2x&6 zP-;MeRr(MdZhHy_x)=y7vUvF9r}N4T$`q(y7j}6bs>UD{q;+O~-5<+QhI6g^bU}5l zRdULenDP^UXIl_#P%t6)=Tfa12dC0$eOU?dwcMVLX7}b?vgSkCbDb#bb%Sn5SK|t2 zntoogz1mt*?cLfLUpXXSJL&q(W_E1&Hp`1S65SA z)_)sCxAIb_>$Pn4i~D=LdFl$#>kjZUL|B9QDuep6(zMq{1TwnoFChFs!K=)l`t=o_ zeiOs|c-On)EZp%Pj<2+}v2@e~)5G2n(GPWzswQy}!<6;WwB%t2Y->n0sz@JGmdhI7 z=}w=XP7CX53>a5q!|$%n`*?j!L0^UE*$96izq~E_j7Ox`8$77M&?jh5__89xdA>Wv z{CNG0TaojdyFHL`tWj5eq6kNaCAb87Y;&)N8h8on*gvW9fs7Sm$jtu*Zx8EyVIqoO zwdPZ^TWkN8rxoiLYU1X^x|m1zn_OO{3`BQG9y_cAOE8?hgsL4kb4O>(O`sPsHy`6W zBqqhgG2w!d-GMgx{l-V>td4V{u=8_tA1W^SB=fAdJ5zr>ReT;#b-0@DqpP9k^9+xL z|LD7LuwoXw?3J(oMcvv|k}73Fv_5dj!3KQ!1&5jEFtbW)+rH(7YEt9q2>#kC#4UHO z{`}Dv`+iF6YPEbnKCaFFVHyn;$%qEMLDdNUVhZx)X0Uvk{4Q&urTPol2S`~BN4nIJ zM&+_N9&|2QEn@tk5S44HZc<_S`yW6X<3`htiI(&jh?psVQE3uD!O(g{UMp5LNb|cJ zUd~AsoU_}CR(|wnR0ZVGHZQ9`@*(xA+hw8RwePhtb>x1#tVzDMM=@>Tvfaw(@7gU% zt-oax_Or;$+uHJL*z(I|1jjIYE4bS#;yYNn!z8y}Pcm?j3wrU7PfjOG`H1K(a{tv`h)n!b1*;>!^A6ZFJb z14TM6S?F-*DYz4%k!8S*CFWgh*zjTg2(V3YSie<9j6T)1IZ9rAd7$Sh@=BNmYt&xV zvT>_08h)87TQ*u)OFP6qVYq#aaeFxltBsUpi2srI`|;>7uLek?5;#f;g<$)S;cKn_ z!yZpd>fV$;?NMF|cVzP(J9$g{q^>?9(-mdivb++eqv+C0wi*faq6bNLBiqUcZr=x= zALZQL*5*aAg+k**I)!0x1d&KxYfPHnH|pD2m!4U&s%4fxzM`8PW6=P|+0e(4thQtX z4XE@7rPt*2H+!|yi195%;Wm|b24p1(Kb{0apkAP8xdXe9fNTqZ+YwT90+PpwiJS>E zHyhn<;Vt@UL%CSfpv8>47qs@+`$7}Qi-2hk8sI5VG(&-0JeP>3Ld znP8dwvArmDynh6Ruv9;_o>Glrv3_Yus1lwe>vg6Iv}lMBs(YUtjkD(_+kh^Z_&;f2SHmFUH-5x1MnJ)zUQ zL2d$D@ft1et2{4hq+gWqby<9fy7crkzBmO5n8Sm3bO?{eQVUQ8KM7@Ur|U)6*wwqb z^;T44vAAL3HYkEiYC>sF#sP$rV&dabc3ZCgks}&{Mz=RDcp?Lr;E*$&uCg$Z#!$js zBwRx#-;3;{Y4uUr?L%a*j>+FyA&aJ$P%1S~?1hD}Y->Bgamgjr6;#B5C>_vH@#*^S&~ER4~#EaG-~?U@o|CCI741CDXKiteyU5V#v;|Ty2 zh63Il_YfYnYWwq_jHXlpC3kIIbz*IAOLX$SSV;xNABDA%bl!eP%yhWCZSG=6+!`l@AsK#$!FuXPK{4qGVo7T)YHa!n`CNKYUO%4ovAsQ9wub||$nTsNWgC*gKG-DL z7c5v9cT?8dK1-@X7b(I*!fg=%kfXo9SQRistPJ8j0C8mJ^Bn|-yu66q?A6w4G4jE9Y%<$#DoPJU6tvkRlO=LtvE5e(M4 z&{#Hveqp1g6#B}ek@0Rb$!j1irhllJ@S3m%+i0&;l>VH@)0sdM(}#@ZhM8^rRkcMy zE|Achh%3Ekr#14(o$8#L+Dt-Z@lz2ve~xi2*ba+4;zzSw+K~q61$rf}!sB0|2zNgE z?$kN9{L8TBv_7whl7wc_od*=cJ}$4TPuMWy3)?0S7pM|Ta9+!Vxdl>rETdG$&jpHO zSFxg^Y>;9cE;~pef<_;^?qzne;TDXBzssaC`zthnCc*4P=%>#EQ3H__tu}XBbE-2^ zPP&Q1Ojg;o`qD7bElbT~xP}mfL}`iVT1&U7s?4laTq=VPdKyZ%uU+^587p+`X#So% zO*O6vtZX&m5yCdNDoOMZ?$uLUb+9x0$xmSs!mEgUo6dQ7BiL2jWEr+^9W9p=`rpemPD+I=>m{!e-@x4V#D9^$VkWcQdw+Pr2T`7*0=SV zwNiE&??>y;V+mutH34;K9n|tJcPUH5pKE^hEe*zleCn9j?bN$isCP=1F6S;9TwkAm zTimd_&)=wsO0}hp6TA@X*#MPyd58G4(%AAT3jT*h9`y&bjuPt?#tMJ)x}9e-#w*+) z7un%BGpYJ_ezG9WKN>GT4&VrDrQIx{v<4lfi)WfDu7wsPYb%VOhI zqFF_hC$A+M&0t21a=ma>&1E7KohuUzv&VjAD3$fdUtb z%!12Zhj3Nf4N~?@m+tXhUnpR{kRdAWuhRo;=W||S9%e6Z6T!soz6b^8T-&!;-GTYr zRjx2)O5C-<2AIu%tbfq30Tl38vUK41@I_Zj1V_Wc1VY^)5YH{0N}RV2)W>BY;|2*Um!?fs}JAgkX#eb=I6X zoA;H!11BGZaGIGCWcmTRUq!ZYtx)C8!U57ju9#k)BD+k-0<`LL6UN6B?7)y3vVgP~ zH5(gTAV#Ldh8P(auGU0t{}&^JXTgZ#nHT&r8sM+$nP%KWR%Oda`Q3L_>WX+8V({U> zeJw!OKF2E!IEU0s&8k~NCSDguw)%SRp5RxKOP21<$1SUU-F{7mjU!Xr*-~aQA@^Bf z7o)f0eI*QR4KZHbJ~-*|y^I_u(%tN|UfW`(SY+;tE~+7_Cx<{aNQ#-Je4_X!MBDgt z>iZM}#DaP8;}VR9%Ug;k0#a-E)l;s%pt`gx>k#o;SH}-KaE`dbvSXDIjz3ap!c@%1 z-Mlkp(khplk}TY1E}E$_TrY^Fv+iXR*=8p)+MgUE&p7F?6=9nyn@5sK!UGU!OoqV( zBo{-6eWs}4?wEz7A=6?T@k4(SPq|Qa&%=`0PyV<&{0gcu-D=34w_TF_8Od(B^e){7 zeBvh8nPHnS21OGVluFe;j{2~3@sP%$Y%NYt3rqRv4#woEbC5c8sNxyX>d@C`Izs)n zMvEIZvKQ~H)JfNg6mgMQ!wV|9L1tBTGI-N)nEa848J^mh>L- z`b4%@Zp_d8FUsNh8|ARcggKJ7^XEH?b^AY14t_c-?&}gr3t8z@x{sc_!?o?RZ)k;r zKfH#UYDP{SVySB<1{&X2UV?fCR6YbpK_(-HUAtdW9qP) z+_hWOkMRK9-(|Vs*7Tz!)1RCdwtTpptqxMf^9OOn_SCU?<(Zd6mH89L!?7Cv>3vkM zd2L997i@9H1!Io`z>}iR<*>wY!Q?kT*(rBM0L?}h(vZL1{4ZzDP*4kjznJ3Uf)|I4 z9SJT1vySgF+6fb`XczO7ZTh0{^Az_MhEMjae%%bMJg(~K^Ns`GeV4m1)%Bo^XODo; zwb}F4hpnAA`>i-tU?Br?4+2Tj-X^~LDq_bY0nzpz0G+uceb5*e!1Q^x8m7R z(P2TndCwC;UA1wNlG|j-_1^Y=DSG;p@O{;zyE_)k)i1kZ%`%6s)!DF{C@dxY@8tTi z^o|cZ84`Y{oAWK*tGotaU-C%yZW`z%6E%AJ_ek<<#b@~m3VF`241WFN@RjQU{pCQ{ z3v+$oOBDGbY|U#v{tFCQZ)R)4oZyQWriA+gN-JuR!MOLvN1E=FwsRcOao~}2gKMCo zpy|p&7t1pkRTgU@Oa0zGO{FTi4_!y3)TV8;gPna1qyUfvQ+_o>5?wHs%K9&YNb^z_ zryqJk>PE@)3;c|0b6_Q_bwvL?#zWokSwFlfYs&p2gwjoGeFu^6rvqrnsq-5GOA{OUa@)MJUMNqow;RT5=Vc;*#!iES^zm?< zq&H@|;h{s%2{)JpeJ~(D68vd!ck=g}%3e9|f@-HdkB=HV%e5;G`x32Eg@jn*UTl6} z?*FLCWMg*9jB|L4DO@lY7P6&HF7RJ=CheYSP9GGl1-S$r-amj;82ZfM@9hzGCm!wy z;8CghO;r<-xA|zeQ`txiUX_L}qMgmsQ>0&0SI&6{yvnz4!|hbIBFrr1BdTB3xH+&ZOePacno!ai?H1D`6&$RX|M}Cu^sK**4^$BQ=TZ8quC7 zvF}UkOlB~Uy=1-9_2ZPH`u#G_M?FS>Y8z$+Hi#fJ(fX6Oc4dRZ)C+8Z6%`RCU!9|I z4Q`k9C&gqKJ!8gY-T?{<*R??Z;T$CI5UI}7n_o#3f4UxF?=$|bsh%47rUtxqqGI=^ zxExCa%yxDa|OmoFi7Bau!N00Q$Naxk+AMzN_Hct3; zRA3F4Gtj(I?7{`5kjNj;X$zUyku;OjO}0Fy zG!;z|Ui}X~(K7TigM{I84pRmbtHAn!GV5S$7mB(_H`#=VPv5-LzH5^;G2$3%#TZ)y z4gwSPSGP0~_LDOYZALrMH6;$Lg+24Bqvp>XGnX=r;jVIB^%`vp^APcUZd(IMapob* z)cI!1S`}Gp*3cGEfA7ms@AKO1Ab%V0lANb|E`0^(Nbw)_{LjN+AwMSmew$ocf zsHjB`15H>g!;Q*t15HMSiA(XN?uGtl6xl_2^uc0G&}ic#UBxL4eEn-+zSxRYtcA81N@2Rz!#Amw z<{PiY!B*K%chT1QmQI&utv0s=X=b=+<(2*sf|_%dr#73wFo*=imv{uGLe2cNl|H(QZNM93Ta*_@?JGRqh`!s(EYwpT_1*@`+9QNv5a zXSU#zyV2Mn>Q}Z>teiEXf0dsI*FVO~PsPv3b87yi77B9%F+i zr1nA+t?+iVMwDpI#YueTk(*%O2y+~MHL$4ZTi^L{@-yW}DIydqjbp{X9Z}X}^?U;F zy961!OtzwQV$u_e7+S7QpuB(#$X2|synoiQ;8>3qLzid#A$clh#cjsj3FZ^Gby>X* zwE5jg?N)NOGv~xi3sc8|7r9P5J?J zWqUi6!z0+q1T`Y!%D|LDnp@d~DQ)-J-N&NPZ3U{``F85Jo&=SaM?!+_e29nxh@0&I zw{tRL^QH~S>G2K9%qf)L_brNgf;E)GJeI^aRL=J-L#?gn#|e>hX9-@}oTMpqp3X^$ z-;JmIckQ83bY&g@>Yhaar;)EWfUF;0D*qE3@*sajY0hIx9K=upGQ(R@c(xdwlPXFe zZX_kfEbY|OFbq@aXl8&=EpaS%^wQ<}8g?WEetyUX>7FfRkZoH&5zgq0{*ZkNtnjoY zp=0&|(uX{iP}Qb4{7NesO_@Tll1fF0@|{-UZtJ(_g=piz!+;5u6rDFuCq()Kr6Zoo z$Y5&eM$*YF$1jV2v8V2TqL5%zYPgRw8qZOmZ&;&r?B6L zn}UwD$r>z0l@V$4m+BC?c{by|VlE;EYm^Q$hZVlb@rJ338}N<8e-Xc@h#?_Xy=0P@ zWnYx~${KyHgqteGp^c+vT>0U{UG=Xi>=xJk8{kB z%L5(b%m&mhZ8GxHjY6G(pM%OK*q@kZ9ufV1*Jfb(C=r(qZ@<>)%WlojH_!MH6jIHs zP8|7taH~G-o-{!hKxTv7kc;7Q3J%+}m4GH1Z`5K{izVE2zTZ{9UohxFc1!I)bi$T? zUaAPM7h>8k6O2dEuNaeu5l+I-%mNn5!7HHWZV^tPKb_;C@8Qi_Wk*y>>G>JWqxWPw z=Syn}>+(sYTy&+ESH4(+UKZsCVZipa;7tUa6<3(E6}v~jM*{YC)qrHs})?0!r+(=Q;PU&jOzL+6IW7QR2# zzWLfj~CDbB)mB-rY?5`>{qIa(!)x@!K*6viQ0>!_(G6pA~n)_JrHw)$A^n%#q=OBTgvahY{y0q$Zh{qTU2})l|C+D1#>W zxyCvFfzN^-{sP%;tM_jIFVrC?4HUD688hKmq!t0@>;D)e+`gF(^Acze{%x9JAdxRo zfj^jPAhNtbnztz)A=xq>wW8?d$q|{n?mJd_9+npHjz%9#iq-YqPUlN)zV7z-_XECO zhjd?+?Pnd*R=x5K59i0rPs=XOE^B4BR2-253PNjb)GOQEI~6waG7i-|?jG`17bcdL z>P?p8>g5d`%LSk8?^06d*IF-%Tk;vz5c&Ou%WLev4=c3>&NgOwbdBP4jk(DyYhgik z`H!tU*?I&Yy$S1mLc67hY5Ow$HXA@j;JdUnfd`dXrhA;F_CdgpdMEiw&HgJ{VD{spFO2N9(1tcG z!F`{FXFn|aaxiU+u|^^ePHcXS&95=r9pJctoC$|DRwm0stGE4AT&Sw`V?iC>^+IJ1 zEIehI*ns+GNwcEnmNT@s26q@J-8ef8?@YAa4N0^_I9<}|a=qM$*%EJ0!Fh`JF3vaz zwb$7q8UTv>`pkV*{c~yJ=l2%0U9dPd3oQY7ekZh!s^!#u<{)GM3_Oj?s`4wNU>GK} z^z`unNk*{iCi{y3n7i65x@f{|$v_|UEe{KV#yB<^DAe*Ssr5Q{ZT@?deJ*!+T?#Hx zghUtS7-?818y&g#WXedI=I?;-%R`bxu~+78zeTxb0PUI?V3J&YcF`!?J&kB`DYipW zm((i=-Hgt`WZJE{*9GYZs(+byYV$1YbJ_aZ&5=#!j~kQ)yay=g=WF$^E0+O0H$_q0 zm%0Uw;qHGg=<^NBAm;1mI+O2=4UAFmp%#Hvf;J_V(ZS+OE>MKy2V}g;pJsC01@R52 zM7_+rVwXln*gu4XMP#*rs(sG@%_^952M@#^ozehSg=^{W5)&0 zD(ntn8B}=JOuQA-p2wCG_b`NUp6Zns2@bj~M3e63N!ZfbRu6p>zypOqCHN-B0}8>o zI%PLZwSQ`ic~FKzk7}Uc_vF=cWH7TCK@Apyh}HVjj@k79qtqvJLYXEeaytv zu6emhI?g`#%v!Zd8!`5nIUs2*k2N*aK^|8BWgn7aVd`8&!EkGc|Lm5|Qt+R2darQ( zDqt2NiWu0WvECkzDjrP)_08HXLSc7JG|8sm5*F!xnzmxMPk+ndsUe|Ix8H~sVz*k| z#@)tskNe_tD|4=&`toyjc|KXBU3)ck8N}KgAn6e6x3Wj1+Us)4eWI11r#8FgplLFM zfnte&2TfuS$h@>(zxg*Hq^H9E7x{5(ABpLc=$N)jm2-7oSIsjnoc5!Gy1&YE{QNkQ+;LBI=$v+ z3Rnaob}F4Rvz(>!$Xcm@Ci05jmFVXjns7q{4|n4w7G;jVojNYZIsFU2hkzYy95meQ z4r7|Ffco5AKA;W5xFEqN5BZVEfOUgTW<$@D71QK^h)Ht`a9Uuig(1ej<|EzUBxvZN z9j#lQRESb@o26($mgU>62=~>d*Q1t`NFfA~mVlJnNCsP}$7lQwP{p3WFN36(I@_GuDR-3xuyW71sMDPIljKhFTx|BBBA{ z-E5AhgHsmfWAzC9DY+zUOv7ipRBpg;Mmy8(7b4v7Nt_rR3;%KqQOLwH6}0Y-ek zTU0canuc@4D;;r_FMyhdb$h9gnZQeS;s<4qB#+_5%>fi3L1m7e=Et+cOx?gDpUN4X z!il$znX#g+#Ly4w*;p6$MQFxUaY6Wg3|7AWfjuILs>Hr6FU{v^i_p}=&>MfG4_eDc%_>QqOnBP7`YTvNi^5e*q-Dy9Dlb@?EaMyCj232 zEdR`OSlKPebf$rQzbYUqvq8q84{dERo0W1RkI!^e2F>)g^Etc0k~tXG)a;TFdaY`` zM7`=wSC8fS`CEwVp#H(#9~`@>%*+!bZ!wXavAORzaV1JWg<6PO7Nq)6gMHL zp2+wCe)c0U!m0wNrZMBJ(tA4gP_Zqy9)l(s8Qz+Xvo|lNm%N?~>+I6k z4rfwGe4=jFX7lBG_mna`r8EOsxFAAdorkadAuWntbX4NVJ&%aZwgT&<|FHB(FPj;i zMPIeOB19}h1r>)<4Tl~r&x5RKsBZNb(1++M*W4|faIc3lWy@;`C~xTpUFYTc{-GR~Gr4nQvqpGb`9q()2EZC|Bq>Vr!cGt8*i%WBLZT z(1z5pwv3juk*?z|d_C>+W%R(3Mt4%zOSG&GHba@5BsXn;cvm4xdl6?`>>GnXgUn?u zv+mQB{z6EeEVSoe+^QARokV(0%q*rTs}eCY2pQaN4kSI>1aQxR+i31_+c#}t^!re` zi`$0WD+thB8M9D}Vgq_sWy8&6`8%Csr=3m@YWv<@zSJN`Y`A?XSO3^%70@w=#%c;! z^a7mPWgny-9)VocvB;{bTE;Qjt6;$=q`e!8Pm$$L=}fumS*GYMwSg8(_MRu5yw z`#!#CzbNo*=ZzZb-eSnTEy~{lPUxs`)%afFbu+jp6i=yg-2t2u7#m3&)-$ls*!M~R z|5(WWs@?-{?#(F82Xrhc9sso@U{EJ`ouYIy}%fH{)IcDlp_(6iirnVDv7Tt4zP zkd3Xy#Y#fMVds=jLDKy~@2aW~1ZG`uV)KxJt;{eg(v*>J(XR0^LsI5~-KDzyNJ^)* z@SPaETxwk%5j%9}mCRw^At76fDst45hqX$X>%>DK=lS6-F$xjPkgWiK*o!9bq4h}@ z)#NxAo2}%0mXCNqwh1q;{0_^bN`It5cC~&@Z@)6qf;cY9A&6S@du|kX?*o0Rj9#nj zTrxdmA1;&-_w^+#z{?wnPGIR@q2^}hoX<1kBbk}GKc-GBvd6XpE3pvNX#~UMJf|@5 zg_@l`48ih`S%vT~IvT`O3but|Hl5PD=`Gn5>7w#N(CXwvjHGeL{6zVO1NH#&f3=I&Z|!243G-h_cfq%l>ev5Bi~L8sn3qDU{L?^6D2KEAFRf4tU#+nV0;X;mA=tQ-YS3mr=L6!1A0vIDJFA zp|U^Vu{&Onx?6j9gjTH4&xeF(X=9n4tMc+QmZA%>4-D^)&tUMQ71U!6Hct8v1vVF! z8dxR#!zv%^PYXyF=>jTk~)XVr^xchv^-7^Kav5O(LVjG9a7 zbym;1;F#s~OHp^M4C3VHh|!ll+q$lHYzpXllmU$4Y3#n3$XM?gN^*h${PQZPkznFR zcJDl#OUEbEaJw%YmCof>yia11TKK6Ycb=bT zA$KW*JMQu|$Yz@IE7`Mn50S`SWG-B1g}Wc~Zh<3nYgpJWAt~b;=YSUb_Hy+;%y{w- zgoEf4xu0$SN=b&;sHFrR|6pK!Aw%e>*B3(uffSl9Ym^cy?>B2ez(noR?5J87@gMyEu3LDr6? zZAkUvJP)ga$U(CC+^yG^?5X_w|SJ5Y{VDT7sYJpvBk)%u!| zM8$~qP;lcAQm#>^nqahxFRz<~NaI8zR;l9&qKXq_pw9$45wUEbct)8t_f#o`=blFK zRT{$sycG@&g2)C*CI5`>Fl(E7&S|S;)|zxr%eqK!Y1jJZ!DX6N;0&r^&Lse>6kv_| zdEa?nu+FStb7s#Y7YXrPxwR)DX-7Np!^pQm>J_aF(Ff!Z0MVr+G>sRiB5LhWwr%Mw zSv8kG&~89ndnxB1fj8ikdc!zx zlYjCVai00yl_Oos0<3qvUX|*DnnR4mIx_H1$rr0@+lU2xzQ>R(H!ih>%)^L$rb1zA zw+h9Ofb*JR(dznRn*mi%n6O$)WDqZz_1xOv*;9A&h+n6@E#`BePyeh!?Vca#wAh+t z!kkv({!T!Oox*+&}-z7{@C@mnEyU|{6rh0h!cJzR7IA`N+h z5O_6J5d5SlV*X96S|9rNEQSY$jn6`Z7P)6cO!bZ;$TMgy%-ltq?H8&Vej+5ax=JL# zbK(&`!DqNask|Q^yfYo*C>gY0pnfpvr2gfgxEO)u`N@ZJrw8bJpYqSBS5wUSc;g3Be3v#GQspmX`p%Nc2`tfU*?8Up# zb|mf+*~`G9C3slvBv!HiunV{?pQK)N|FINY6T5jXl=P|8$E}CO3d_$MYdDpnHUtuH zJjV=fo=RSNL^bJlMa2{2YsA6KS4U>$+keJ7uGA&emI+^+z|HHx=A#MUi~|Vf2*{S_ z$q&I?;v4DWFhV}|+iC^|CuFEN5HFP_p6#H4@|1=qG*h&zxz+@Cg%ad8a?@v%kq$bP z9<6?G8!9$Uq|*M~#eFljm?d9DYBRooJq#qg0;MU6#Y0T|F@nj9ke0_b8v~VR{0bwe zgkeK)dx**&;60?xOqxBT*3|)y#5UJBRF!8b2s(BYlNA%!5k0Sm_oc1H-&9pP;TY~* z4mz9h2tAJR0&VSHM2n<+e;e#MXlbVPAMWeF{L^lX?J~mi)a{l5+~%j7vp&O@uW^H;imzKllB5_5y4`=jk1GicNDSy_UFt-JUGI%Nz4TF zd@?6M&x16kO~qmG1C0mRZSA8l=|^TadyUCE_Jf!+f9G*B`RTPJhRKN4R!r~}Va6zn z=Dk4Y`Kk&2Y!KBJ!hPEqrE*Y^csR_fCMuhFhTR3sT82b&xiDnwP3sY`da%}E;T?R8 z7LUvPqHyfLH6uP1IC>8}B%&0YSmPF=uCo;Cx^!yEL=v_nGCqMYNVtDWnd*U2OMHAG zP0aU2(ud`K7iy0cek`)`K+Pt<;4KM^i@b;^o#CL0rd<@2i6;Sze+ep+b-xcqcsDxt z>3sQdn8tmRwW0zl39fd56kS;xvdslonK%gc)Z-GlzCnPmn8d?KIA6-)8D&toIu>z9 zF`y;W#ccpI#565@(73(5`CXk`^jPJ<3FuFEBMso64y}N1%=zcX0U3^avj)t%Y<_*}uQtrN*g~ z`t>>2bGl)w+L8~BR+3pSZ1 zQ5Cb7;(6;W>t2~>7S_Jb-j??ccjC&;vWPb`4o2Ll?>ggEhVpJ!LSASQhbl&>DxETA zPibqw?wiCjk<5L=RORAz7!Y?OC__B|>b!d&6cn5EYy4Vf_Qd-q!QZ=LIav_$CH>PJ z4?WhFqh8g>o1S3gnVSbJX-pZ?N%kirBw~v!q;xE7MfHNyZ4k7)fltt`oz?CtnDF(9 zwMY~7g-z}O7WF2t19p8HS0x%cV#MWX#6it7L$1&)6=B-3XG%?`c+M0TSs-2_qdmti zTU-nEBFqjc7gN-NrlgycM$U|aR-Q>fKLUR=2e(S-zZ-;C>0|q2wETmp3(qGnXjFK9 za-copU?YB8XXTAjG_!=Y%ePPsP-J$_sOp z!f5kv!Q*TXex0o2eC@(Hi|{4PUblLt1sujpER-^Eb&^q`WNc}tLlKF)EfUS4O^Ni~v zdu2C>*I(ONgMz+6^8oxBp3`h}~^mug! zqL@>kRcsD@#R_?3Qkic*jI*4XQ+4z{v-gTLm6SW6REh;C4F4g19)7!wB4NqE zPP-t0L^C~KS-SNv(Y%U`RbEBVGHrJP;;(qLf1Qb4u1{zYI^GktX7EbkYD?I1l6@f_ zy?Qnl+(w6Q58RA-vnuX+-p=@DYL-Lr>6f2AXvceQaq+fS>URT6MTD4TNX-*z$$oQU&en}j!xCWb|IaZ(``ZGAcJ6g|C|M*f zi`5XH44Iy2E`1^J>tjcD98!p_HIjMR$`n;R*8_+_5l@}qjWVa{Qx{%HVEQC?&h)%odC7Vbn{)#iZ1Y@z!{6j(XbiWenVT=ZJdv4tV( zqsVxXA-d8z$TxJzBD|Kv*Ort1$R5mWB!bel{_D5!kl^t&+30UK z=CkIhY~W#dYSJzZZ{&fJSZ8-KGi`U4zvd?! zXEpA&Hm&6C=c_2_8eT-3(Z}?t%FeOTV^Dv$WO#Ca#AKoVS!_FZhVSiqev)=pe2-B# zJ+;wTFjRkK?Kz~|M)Tz3E&XV_89ioiwB5?J7;Y~bK1?JYfHjmTnR@T?Q7AyZm+11_ z0q-Tg22?&2P0-(p8N;#9*TjpVZM zpg<~Ykn*)GmWPKTm-(7=#ac$R6ZI2D=hU&}#78324jp&i6?GHJI1#v6;t9X*zqOkg z6%FUrRcYMs%AOplCBj0@tU-d`(8Td?JL4~^wGJzrWpf6D(k_biRLkjhA)Im9-o2So zjSoQ4O$mh8SagnlvK9DGWld}69mMGV7y2j=W~S?8^EH~dRdSbk=rm3JKkh=t9)~U6 zk`0#PF9rQ6tzvU)h^9_Ak$on^sk_-;BtAkB-sQ)d4s@Khq34csAknn5D}i}= z8};XX$>ywe^ytXy0>dfJWZV>12X?)WCQ&F@%do>W z7*=KrZ#L74Q%TzxgYeeMDR<`xS7;n!mMkpX%QPIHyeI%@A=_$QN{H3ppTJ`{E#X+E zC|GhQd&HkDSucAk4X3O#JGMah!orx?BLYj}Y5S{cN2auLmEvwvd$AgK&p;D}>b=4jUvk}BE0X7*q*(q1* zr5X$L0KStPIL4T<&iEF^>qLz>E+0kV^iPBdGSmoNOJNm=IB(q;(#7nqAb7hda~m$2 z4wNGX8B9s74p>KhDSh#{jNCiIei4Gv_aK#Kf{Ynza4se1M)V^TvE#Jqk4)8wxC2*5 zh6zF@P8aRfv|8x9edV&m&lXyxSvdISnELNTX=CD` zOvvkfrzeS`)ucmFhC}<4M7lK5Z#b)HKT@EfZ_!g-@_MaGT&(yQ2IsTS2So|lzJ<$v zQ)7mY{3`zJ`hiTdZLb)!6v&r8a_@p31CFeWpJ~)8V>S1ORd_n6#vbj%)~!h1xiFC& zQHpf$*jzovjYoeQ<}D32uU}_X*fgxkwFRz%tp_0kdo8$~{2m+zke;{Px|%{__ZGxt z>&uXht=j9iXA zP>^+7FnmVqHF``_!#kQSkdOi4Gz0=&1n|kG6cULPBNuYFt%jk4ORlT^%z~9?NXv3> z9vP$YNoVaGYBr+^R$UA&w5!_jSXI~0&_1C~P`Fgbs~o58l10c=x97g(&kcjVdK59f z{Mf>4`s6s1T&ujjk%MtRcgX`$gi~o};jVx+#(6AyLOsgH8?G%Iy}y@HE9`g1XoX2i zDdn#&_1xqISs0pnhB#_Yjp0o z5EqGCuffJt!MGGY4{a;EQga$Jd^aLC(5Pc_EFma3kc?F9So8k(Zd4`ETq$!>nbFC$nD*Ry+!Xb819&_JV@HY>Dw%Py*uNCJ*JJZo z0lmruQVSt6<-5USLSWgTuK{8n)OSMq1|VDR4+&<64Lc$di!5deYt zsEoM$e;YEN3(~cd$`0_pxHoIs!Q@bT3DhnqRHE%pP~1;>S5uI8m8Pg@=U4!}p}_sa zX39QZY*fc-Hod#o_vlS&ACh^BhhDJK$;DNeTYGPXqiTG8-Bg2O0391PKkvXM@!ACyJ9=R_iz9LUP;_yj8{k~BZ3D9B33~r$= zWU`xVLqq|2GVXGy@m90N;~at5$fEW2?k{&uBWf(#Zms`Ke}6lL z)g5w@w!LR+YTnOgaZjh6Tanju^hL7pqhe{rtP=MzM&L|cHI%>6km=NX^-3Zq3Q8_` z!I+)7E~Eu)BJXwSc`ar~s6}+I1EXl!e9BJTxwiNxGaupSB1!{?K|Km&aX-V0ws?OD zob7vEm(8#oUBeuYsOA!azY{pK63v8z?A=|E@0l4^Ualu-Q?6rYT+69IK)lcfxl@g2 zvk|YEd+Qx^#oOckpa@a77E`>}>lh%Xwah4!?xA;J8HI5edc~ZF+=uG*yWMQv8zMq) zT42H_C0DzgHm~eNK|wco^)9Vccp*}QlpgJQA|xYlz#r>@nmq495z+=ZFvo);BmHv69~pJVBuP5SKYawS<$yPBf#F+q!} z7hG}JDh-rPoK|yd4nJhtO_E3@vI{<5Q>{=VT05mYv!~Eaj8AccWg3(JP#NelWvURZ z5$!eP)}T*?@;}RW=Wz99lAlM4_Mm6@lr!C-c4DeWe8|i_C}R&<$aw$aXlG93lb~nk zT9nJdvA^T)Hs}wT_8M^=FTXOOF=?dNv6{rk@V)}M`FeAngTv8Wrqk7`e`ANyvAj-< zW`XOY88rFD1E>6uMo&q5tEEIId?a@XDDb~G%WZUEI~iI^-Er7`nQKx-={WqYMVDEb z;}}tpMHLn1e=*md2y@sO&8`Q=1S7=m6_}uP8U4Ydif9Cte6-q7?9ktf_MoZ9+KThf zI=L8zBQOBNo)Yrxhs{jW=JV9ZzGn2Ta6J}T1Jkoe=-mM46)WGHZ(7X@augrp=*^Dj z{;@#qX_AD0yfsy)%)u<1<$Kjasmj{R|D(lf{y2(g-ah%lTAMrIhN%7LxL8=rh_tq6cT1 zQ(78;x-c3%bz0!M_1Bsltxxq%&6Jp^*w6s8jY&-PoQ(85?Il$-#|=>t}Mq zwckty*+bO-LB(o%08#I_QKvUw6aTG={(_+bpn{=7007`e{k#5ikZt}!#r`GJa(KLE ze(I3vhx{S+jl%=_=;7nP6Vk?@pQg-Bo5mLY#;u)LJd|y=_H<}I#PMj*$hf&IS8KIa zJl_r*-p~5|Jahk;U0r?}T95d`xXk}wO|~Ok?n9;icEiDm;?ofMXLaSc;;gcR=Af~G zvg6@X{Y7Zj&pNcg@*^LcO=y|*yYfxe2VV{D`+M5KJdClYwQ_X3Cely)kl|Nvo5Py* zdQHFCMaL&PPH3C;(~v#cq3~Nh%+}(zB**PpiqTYS*e%WlW1?ed_n2JnaL|q($m)5y zg=RyCL3^}!VSq=YtunLC^vkW&<}wl{hHS`h)n4#0RSQjJm36sfEp?Qddz0>X^(Xno z(c@M0+e;aF@I-4(%FqPpGecYE_b>W-CH z7j<)PxcTt$;T8kU#dtp02WDErn>14;81A&Y=(*$04YPVPx?980OWR^E*2E1t(?UB+ zXzlh=?m!Jz{6(~FzHh#ucf<|;J|u;6)*GQ!&u7FH4dj<1zqpIb)dU}Xr^`sWV(_eM zeaC09g->56shy!}(_t?je^LhZg7o(VqwZHx=tMw1xb0nM$Vcr)R6X2nPkP%*1tHe; z8aJ?p^`-+U0F&Dux~&iQhD>!d`#?0GZo(8l%^|kfyob~arDGg0yWohz*z;N^aFt84GXEPE{tzz}J9v?? zXR(=q=7=!(3^UyX9{}s4P_P{M#;2@)&8F$^Bh6a~Q4Ltt5a%<(XR1zsJNsyB zR8J$%@;7_uk?)g^ab&}~a%zUNr2Y|ah)%W{!~N363JUi`sdc-NafATVAExJJCn_ zv9$~_Vp<2t4la(0yMzvc3a1hL&vFeq9l_4}4Wn|g(ZmGISpbgHit*n^p|T${$pe-BOn}_8+n5u=6!!*k(=6@KWjOJ zo+!ATO$AX+X}y<>(UC^OYQ8Q# z+&~>vNyJ`uL2z2$#y*=j`kqE6gXpf&jY!*z>WLZ?hUJt0xUeA{;xUgjZ;3iJYA_%A zJD>eP|5)A@3=+ZjCvmOn_75pccm2U%h{!vq$mj}xYrz>n)hCOB_@3?v+YZdaY4Grs zw-KWPth6Ou4zCJd1+TH&iQ`Sj#huO~nj+{rO-G3N>UI*rUw+&Wqd6l9f>R^mGG9j0 z6boK4(rXzGWj2Q}T%v;(Z<+nsee)baXS9t*S8aIA$a5e5_7FZeT?7WSLf{UJP6Fg7o_~LAZU(=;hxh#7X$Rp|TizYidG70E@ zv4Xcg0x)4`bY@b-P(#xkW|c0r>WC~v$Hes#GW(j6q>Mu+t?Q2GN?kUa_zP|dbn&eO z=qR^Ch^=856b90B_@y1hjK%^Pt9d2*+}|1`q0+euH-{K z*@Y1T28y;($ee16`*Gt8Wy{a&s(i@;2?^sA8LjaUjjAU1TaKXAzC_>CAh+Kx99hAH zt8`|MKbG7pNHY5up7?t~jiqWFbcjtDh`4t1DMYX!YYA(6x`vXXA3m$I*s(ES===TXGQ^XzuAeiIH?9NCtBSHG(Kh9(AGe>feg$n&5 zHbB81d$#0oi^=ay7&mf?)Lz3mL&J5HW}eWAxsD(x3b!xrg~>inV$Ony^xH*@uIVIT zJ=JA|VMAcc+I7#^T{-l696auG8O7iZ-v*EWbOk(P6>=80Wi9nUDYmFV=g@s8AQ2Y3 zqFa6R{ECbLtDY|l!mOa5M6YA`EeKMv)<<#-3W_4rUVmh;Tw$vA$ToXlH45^!H@FMr zKl};2XS|-WdX+VGG~!*}D^KRuQ}5&wz@XZ6h7e40DxEUt%#fot&6PBPl|Qy_T+iCs zO4;6Jj;i~R3F;`v*U%67VBLB>!NPJPZ`#h0s69h|f%PIh=i{iSj2b{eEYHe#3YUdF zty!D{-djW8I67WPXMAGfA?|y_8$ue#Zm$&R!dQDN8IZpJ*4^xph^HKkr`46g;d9$g z+46(brNytKyodA+M3xD|?-)cUDlH^?_$pad!ir0@2$Cq5o9g^j31sS&DEw8V9`*+*M@VZh*nM{Jf2MKU>yLhV%M8T&js!iU=lum2))>xwi0+81K|U z;>}LiK@T(j7_pTWo4oKnyBZMqfs~2+U4+h*s4s8Us%J<2lN$%G4Wms!4&S?DSLSXw z2holI2N69m@+F{>JEEOoXieHV{Oytu7*7*4~x9V+3f0j2V_a31sZCxI`*;>)rv zqalmskA~ywL+sLykM%rR{RT*Gv~?-*Vum>qLyD{ox%x5Z3y+1i(-!^-=u6=Y>$_n> zEdkQsE+8&K*MvCb5$Fp|)(H9>-8ON@kqzq{{zcrS>Op=nmlsl@!iIi{i8Ne>y4T!o zU)Ng4n3OoDfmoRmzoBOYy4}gD2JiY^Q4fj5V6b(u^DF%~kNnp&-@wVJ8PjuY-BUn^ z7T?}yHfY; zk6*+5g%YLn&{%;R>3An7VWX1liB{M~S*_t39cb+|xQ(akVRUl7Tsr0|;UjhXuag8c zd&?hF2^P--=}YbKV@haRNxI8~Bw%Ty=)vuK62`pQO^uiwAt;#g_JbakMoNlWa|onr zu8WtiR_xAh>d_Xb8$)i-WYAK8m$kBxp%^9oZs=cpu@{>Mm4{!jE$j20a zqiVuS+EOB~Mvc47Qb8VHO%fg)GJiV(<@3dB&zmM7UDC|5esqQKHfO2EmeV7WD4!06 z3R|tW1(y&cdBv)~z9-4!BI?@`hLD>%$v5oe2jlZuo}PQJ4H^Y7xuh<3+%};mM17;^ zf-_>Z7gx$lA?=;}9-ALCC_FOy^snvQo7uj^;Be_(XI~NpI7>$oIpe2SlwIjCNLMe=B>H0$gGjH`KElZ_yN%Yew{&?L~;}-l^)6A zQdd}dc8&D_3mgg#>Bk}jSRTC#Y~}DurGv-u@U!ry+yuI6PG{l2GXis_pu6oNk>ibz zdhLrC(&^Z*x+;Uc_lQponsaWKJ(3o;L@nOdE57%<{GFd0~DanP)Bd zvb@KK^@z$2D2LD*ErqgZL;t`Z7f*?%(fwEAlwApkI>m}Q^g7cDii0}$KUb!gp2%YV z%3hz>__lySgt~orPNVFe{zn#~ic*;-+=Wx1h1|!y3oGSXdPi4rAn~Hx%POU^ITg8n zx!KuYuX?}irF<>ROKOX>bAB?bcoi~>%QmRKHF$GrGP^NRmQ3)|px>1nPke*w)q^Oi z$bK8M#NWp2tp7eko(Hac&&WKEYJTv`2rW)p@N6n?h#ePdA05TBKiPxqG0Sb2r>;6N zHV~f|q&{qJZ^^$JwW{BBww#Qxn?Dd8z&0waKCQS}E@dCETDUc55xqZ_=XQLrLKtG( zwtBTatDF2QNg>x-p5_YQ9Bh@hP2x;XI*kQRSD*FyI+x|1((pxL7IOCi-drGj)Kie3 zbJaqvmUV5p^R1P4!QVbAzoq1Z7t3>T#9Gj&1Nshgk*PaVe^_CUzMB7abL`;NPxdNT z5Bs@0D_gZRod(4kYX3R4cZp)gy+z@_W=i?e?T*IMocP%-Vj_igSIu&Tr z^U(0R&DaAy=UL6r;T|o4Oe;KKqST)H+1JIQj@k~nV#S(TqQ%wh#=D?;x~>C&9izFG zf?Ed+GFhFKE=az(e-OqNUB;`ziyiP)@#{sjuCrnr^f4_NyLYTJ^PYYwVZNO1_rP&D z0w=mm5evXSUF4|w1l1G40c;+7IwJ$_@oqGfCvz9{bNnL#(E)$$xY2Z!`2pGcdCT3x zV9;vU?^<#NvB|h{=z81rv!OY83`lRXucI!gwTHrDwaOR%uteFx(I zBjl+Gx8$&4Z?r_&4__E?#c#rtz>^}_W8sbR9b*toV)_>8EH;)#$nkkcHX-jdf9Ir! z&63dE1B}7$h;|q0)}(#MAppLEu8#=x-4G83+|48aTjP8`JxK{@I!}qtcJ|yYsgH%p z1Tu%h50Kg4C1ZcSfQM`bjX|Jk^Olx|Jd*8bJA~R~cRZy>K?rW#L1r&j^}T13UfB}L zV7%N0GQdD5a^x{}L=WBYM(G(4jyzXfkJ>B64B&lWt9xcAXr{Kkd<`^95)A@WT+HH# zXp(?e;nUY+Po2vpcrbkh7dE&cNy>|RO8X1@(W|onehzBjp=eQ;9oOcd6Z?3zxDYo4 z#`R6TCJ>TK?31MA3H$^m8BQnDwF>!>_aJs5R-ruTgSwv91mi;dXTy%8slN_Qm0)pK;QanCN7DX#k}Qq=Gw9OSadh?NJ&@0MsvC_*XSK)858 z3)L=zz9}~u2*`%Q(r>iUXE;DB7+9R;gIph}^CS0JE^%3wbsT?Y=xV4vQ1@(!f+mG85c+~JB}Kq%pVXtZKp>q!5Aa0 zgEcVFYiA>AWqIgxu7F<`Zyo!hF&B2Y;EsU0KH~^Otp?*{_=@50SFnaZ;TvlvtR1dV zPN;xmdryW3)Wyo;oV`KWxb*FWwW{*02KwV$O0}A5Vqv?hI?cm-1^Q3X$R$$e*lp+V+kzvA<9y7pb^c~q5 z8VWk$l}5$iJ#cu&98%rgryzL0w%?4_PRLn%r|BCLk^OJ@LLxjO#eSrfW*{dzR|!$_ zh(}%{2bTBN-GvcoJ)KFT;jbYx2}nMgu#WfTjsEB$h9ijQENQSt1C3t|RQhHYV)mD3 zbTJE}h;C)$e=b+0vI@tGzG~+RF6!uJak$c?njrfGoDq9DN{5st9*JPsKY~}KqF0-y zhlaimm?<5vFFKr_8%-8UAryVpnxRq$cs=#ZJ!UKEVmm+qH{;Hwux)E8sau@eYvw6* zW@2tEtvP7JYemJ4Oh+0^io;dJc4fIEg&+GnIV6YFlu*3a5z8{3wxhhS@i=@Oa@i3K zK0vUY@}|Pd{9}`6$m$|M6iI%!!s?AVS0BNZFFxHH_oCd5^%!FwybcUe%=LUL7*u}s ze=xjOhX~i%UQ9@sRjv-eG4rIqbsYntW<0LmBVB_Hf7B(W1)lgN_%VLeT6oS$xr4oO zg~jXllQ-Fvx_8?ZJHP?ZaGvRh_#93-mnux+5nbF~cnDX`gaRK&hO0Pc`}|#X}1jSkZf_;eg;sZGxa; zj|asy5d!rS0>c@JyZ#8gi+?aVSws17SH@3SCd#5{Cup>`U+hMH<-}11df3@>`2w7Lv_#h zXtvqfOFm!f#qk$^l3kR4r}zS1%x4KpWxMnoBcs55CDXNp5jX5(B1X=5lSC9=9-~X1 z`i?`I+MiSNfKOdgePr-}7o*`EdVq`4hqeV`9QkSC^~G>?Y#5H5gV8o@$fbfVzIou? z@K%CxuyF>AJW6=Ra0!PO2o&ef>ywQ1rq7?LPGl-dtuANe^f367q>O%QvPY8@x1J-a zQ91tC3QhW5M^=5vBRM1)`9)6k=H9kE871pPVZLJEG0}ObUn~eY-hL!7R=$KgIV**O za0N>i&7HSXk9sLWR5Ek(Hwqr-eOaQ5gCd!sQkcgtf7heTgDfKPD0u+6M!vEwW zsYSlL$5`V8JL2i?-B--`_N9_;*@8ooa^8^hdIEHjEB_$(wPppxYqEHI%%?rkYws|^ zk+`<6@v!S|cix`U_Ae=s29}#HE#363aAI;Rp~z%lkufjpmaSD0OI$fU&Kq%>iwTs$ zq@0FQ7L1e1#(QkkE1Bl&GkZCLlBELVMBSIA&gy)#)l5kjVIYwjz!f5Pr=uCFS-QcP zu5n#yF!AmC+; zzD95g)8QB0xvk30kCVgd!E{Iy3X-}f=*;3CUUtAWC)`O*djgX;`d&;f=q{K}^Kub# z6I`gBT&Qumc{s4C%M?gJG{8Bwe7Qxdv+QNNt|t^hLO(q#rr(}Uo%pn}PYc6(s`CZ` zLTI8F(M@f?#VULmjvc2fWNKCLhe`aipJjwM51zEQUS~0!y#vn<9IzHkpLe6oz#``X9T+%;2f91Q_=P&;j2>gx|)XhyGHi8bQrwQjM zMnh_Qcpu-8-EOuf7gp+Ug3YrFn2YdIh}3}{NL87RScmvNF1S3lDcjD<9(Bi7orH4NV$(hyXR$%sksHXg#D%Pv9Z z5TzbJhW>{BI#wee85*d^i6m_Gnga8_p5Tp7e!%7GLdqa{&HkFI0CLEw0>P~uE``(G zW@aY|i0aEdj^{H?zoJ-Gye%vM^m(QDG0meOt_Q}crlN_X(Qq1>^}(tX@6dX*65`v zwTlk5^0)X02nY4xf6zL7G|xR3C6S)JnR@+Uf~9B;pc%CxqV9pG@N)Fuso_=%c3tT>!NtEjFtY0L zb#&%F4KF!+u`fT=Vc2^9Y<6IA|9NLn-PH1ID`M%a$REy{Q8CeDw$-yXpN}bReHcWx z7Un}SFgf*${z2s7`seiRcJZ=)i}&q)^{Kbm^rhJ1`e?Swved${WP96y_t|H?{>LlJ zYN5Tq2G&%;KPT{fUi2Gb$k~l+zJzMsZZQ(ldv1OOPkG&JHDvn_$J}rJpk@8hb(?)_ zsU1>%B$v0h#0A(UQ0Yrtsqu(yfMFDgYBMils?G29enJUqe-x2St#qn9Pm)hVHs-v~)2j06B z4whE3fr|4E_a&?*WSAPm&U>y(0s zXA5!H@v5SG#-P$)QXv}0qu*W9HG)iVD&BXukWTzpia&(;?bw#bxkUYOx+Tz+CQJqW zDXj+uA9q&UvuQgIdLt-rw9>j2Qd=t9Re%yJ_h*eLl9L$ ze(s;ou;(Stl5{>yuY?4dzq+$x!xF5dA_z8ss6W13FS?D2D3$nKm0{RF_qexK_qp^0 zM`Gqe(if|i1v*hMl6~NSVzIijKD-BB#-Q)w-)Q=3We|{*kV6%OE5e|Pwyg8}1_ec+ zjq;0dHml0_!VZrgr1GR%N7-Owy{12T#be2`*_q1-Au`DNPDt5f=oC?Sxf0H{@X016 zQSE>kzUE>2=8~MNgkSdQRT|?fWHMl!AzBxDH|7`QCeSff6hx@O7!(=o??j@URb*Yr zUl66wnn>Tss))9bq%+j2QBlmm9BAl&`9Xf4k}@eLC|W=wN{km)`H|XIu5p{jJV2u+ zi)hUjQKA7rCT(D37;J#l8hLaT<(q%&4G`UdE0tuk3HH3Ed;}llIMPm*1?K6938sWh zDOunnJP0*t9JN-GC0s;>K_CQji!(^$}ITjEQ>T! z6*`0!Ev@tQV|rDJA$ZLQ26pn`p3ZoXKYr%U_A8# z8Q$l_G5&3};+^x`f|n&=L>Cbb*{7}&e(609{ z0tKgV?`<=z)Rf$lAqUn1Q55v5&}D67)M$X$z?Kcc3Ix@KAz2w%9wpiS! zeGaIo+3Y2G=y7pDlE%L_!K^oz)dN$9(4UkI3>I9ornv0ti^p`KL3SK{7I6^Z5Q1w% zm;C54{psLeT%gg1T#&pGrFkSo_Eu*n_y+{$GMnhSBJ2uYz>+I^#3@j z(yQsT;5tIkg48PlvSxZ2MbXt$TC_gspD6Zh@Io;DXN;8Rl*rB;2G$-C`KzBPX6FXr z!8lP#^KG|T2<94|Bv$o_06k}-AN4FTZ%~a}Mv)0=P(06~-_$At=aqVg(jQfrYY#qW z>=Y9RlxCsQ66wCU^qr8g02?~3x;ViCf;FU&N?eR!`j8Pp3b4l-8qUxiOcV!ZUBVm= zSL20NF`Od8rwD^3nMK!D(N^g9F)Da!fG+yQG(&KdF`f+Ip;GFR3m&&Ig5avknc1(9 zirW158s49umyX^fwHLgL1$6keq#()dHuy8i4as-;$@Vj&(#Y{u;-11LA=#q}Ue3;t z&9Wfv#iwsNY9QAF(t?Gw-q~|F33JFc@nf%(YZU}K9)}*zFC_q2be~hF3Y(A+)(h+S zPHu{llV&_Zl2v?Z065jh5OpxFn8EU&ZqpkkVTd812os281ofRhJfPpkhn1Gq1^i2e z8e#E18&45+jP#@^1F|VPbIYFkJo+$Dyd8C&z3#5MUOXGH9lE8CROg=Eoq=tJa~{?V z6i~_^9X<(3)Fh3a_-CNFGCRwvy`?ceo>Sf`qyztdFVtAS15%#FD&Tu5T6`O z5XDm4I|_&Kk^2aW&V;?0&KExjR!_ZazT7*O3CnrkN0hQe@ti=;C+EL*!am4o@(Hg) z+vzb8Vm%vi|JiFY?WyF;f#ZEA=6&X6;(FxnofkrOeOGR*&N$Y4i zS5Yr?*VdxE2w0nQA?+tMvbu(>_OiX8 z5Zr*Sx^Dd|A5bKC^z}Aylk#qE28MmngSe5fPmY~$?kGdcm(VQNUweZvv&~8I8Zg%d z^T(W6gQvCjVH{|V{)5Gc0coEZzh?xyOaiQnDfxVbefUJ6Ov$V`BhsAKO_0#ImW`Ms zJO?{T1rqqjW*0tpUTica3WC6a#2yQVnzSQ4F?dI1j3)jRXS*SDdSGUm>4NhN)hmO^ zNV=8f%~lnqP-q$ol7rb;f`z*5S~2WLa-{JrX+I!avPgnW#e;a~WT?>W15dfoG(GKd{-fz*?95gcj0!9C}kzyziD1OK|4Vj{-tt zD0S0YW8=(G`$6Y4B%LI(Nk_kc=vh3X3jd$dR||;x!it*v`q&AAgSzrRHa9Mso32Zo zNY@&TC!hm=g5yDe;Ii!@^ZE9=vh5r?m$&h7I`2c~_&9f&Ds+k!CE(!Aa1T<8J8m{q z9(vKEhJSw^Yvq3YhTEKmP5%SC$P`j|)zMNL?y()MQEzX+jcjlmj?v?2KCGir)oJ_( zDsRon{BNV}%)>$N$9~dKPnBJ(23`VS~;I7!u26qtjKVb+h)T^i8ea_0zWO z+e7GZYlmic$!5LlK1u>aaaHs1cHn#G^(OO5;FRB?R^pab=}?bWZJW-_`XlnFKcIEk z5(_Rwt>Lxy+D;wo>eg2cc^x_t!{qL;!|J8e&A0hC2=1&qWre*TdJ`w`ALHn6H>c_1 zx2vu?(g9+IFBNvn;pE<Zp62Zc^<~!{fJ@uzc5j`J09LR&WoXAgH!1h9(8ju*M%>P- zm@$;&RwAr@2&#Zxc<<3z`8VrY4aN9IN{9ANwv-^?<*_$3>004$nJ|*GF7?LrD8%)t zo4t)n=l-XgPB;J+Yj+`Tlp}4z80id*oN{TVM zOUQ3bvU$t|gXsDr3K~(O9{}|NpN;gfof6!KxfmEs(0d9C-1%E_T7rt#np+Wf+&ii_ zvaEnMk=f+_9cKQ4OU|)?eCt-~4!99RZW5SI<>_Ul*YA#)dScLKY3=5V?`#GlmZoZ( zUX99mlurBh#~OTpz#ocVtkMFypnSXIGY2s+;jt*@)_xhc{}4+`T)*YL|KSZ_Fr!h0 zR=*k~+yrTh%#XDRsz1xIE}1-e9mD&O5h17@U7WD z*uGM40hj!HhQ3T042X}Q=g}U_B2Wk*V*7Cufvg}|A4#%PkOuG;fE>DkF3KV6!u@d! zmIY?qJ^h(S=tB>n7e+;=XG8%ltaK7kLsrB_z#Y5Q1%=o^!c;@k{GF0I?k3$$f$?3V z`(qVBuPAQ1tP6kX#6vqMBv7s7!XN8E4>F?-l%x8SHdCzMup|(dPrtW%@T3~v*wa5u zG5{pQYXP%9TKSTTCf^43a+G#Ca0b_h424D^63LL+@|e8}xtpYlB}R))%i@G!#i?L5 zy08idq$&ut#G+b3NZdZ_mE1|`VZ22xwE_)`2pG>IiD03?h$kO>$_^}w{wjm$$Bw{f zrQt6`!#fmqHbyFUVJjN60Mo-cAl&O+{zK5lmSvs-$Wj2Hyr2qDSM!QI?;s=u&kU=x zi|};i?5n6iMJ1?r%1DGz4o2@8uX3s+fi{Q_xg65wQ2c4L!UQKYzm zb_Kg@!sx5TKJnVvXed^RlNBXwqa^uFlpvH3hyrDs9_Q4*pGr{*Vg5xt6a|a}Z~TOa zQdCA*Xa*1eFk_%@233d>WA_O{Gb-n(v14%Xe+VY|r|FmthmX_UeyFA5!%KUP2~7+r|h>@Fl`p&k~*>BFP6@-QpHWMI}Xa zB*iN{z>!HriuHL9?izSTk)I2c0Po2{F2GtC@@JY@ZBN_r)%j1WQ(%yRnMFAO-{5$KZ7ji3ifEc$#i!LyhP(Nz@&=rtohi7 zD>@JO%asTc6HcxzS}4$OygXvTMZP8^;X|)b%)+4(J}u|MdVO@*>#JC3^}%xGXT#at z#Ao16RoO!f*5=g-feaaW06`*fu0*Ii<+4(9##%SCk+5)p0#|doP&6#-dA*?PWbrtx=p;h^<~EHCxrK=B%| zgxSvZ5K;XNB%JVIE~%7kU@bpQT+GHmh>ls>oZ94~sF}`2P+8glr&eL*ttq|)NsI(i zPCB^DpXD$1WT+W@LhW2?ReVT2N+STezZ0?A0WV`O+1A`PN5PIzoldZy?D=Q~km1$R z32P%YA%}4L5DwVn2rkls@H}{pI#qjo17p1yZHE1O@^O&I_gx$i>! zU@VrdqY+XDu2@utjk}7TXl$%T5{($j>IE<;|1DH@gt7cdKq?I;ia1wEqSY0*Kra$w z0Uf*n1ts9ew^tF5UXtm$Jt-&ULjs-$BpRloP}T^2(y@Y?_bK9ozKcGd(2h#HDy(yK zep0a9nWR`_Xy%?S1MQ0=a}Is9+~sY(D=AgfEEz$YvS0d#&R8Bs9cDB#85_!;2?Vha z?QesEIlm#6Ta23UVV%BZ8tnbbV(J#n-@6pj7p~(!QoJZwqHZ92X_#?(&ek3K82Gb_ zp2B>85vDZr($|BJtR2u4_us*mePnr!S2AEKA}@yK*s~OIQmhTT-l1p$v;uOvphDl%aT{t`xgts^c-Io%WQs4cMjLtT!m<@{P~%T{>;EOH|9`?Gny5dphHs zB}y56uQ59{N-k4Z;rM}#VYI&_DORe)fEZ1Ma}=X7gY%OK`t^Z^Ro8Rw?ULLn=2H=TUCh+EBB3ao7w4uxuG0KW-oN*k%@?)1PY>T0Fq$W0eLV_jt5N-5hu>gbYR7@{8n}E!zY=}%r4nxw2 zc7Xz!O*O>&IW+-J#EY&3s^JskGvKHy&Ad(uP9XqP(lm%ZeZW48NF<+9z4bH>j}be< z09S-k7X2!W-6lX(-4jKFv)xB6%;1*gka`MhZ{yEW((gUeQpz1}3QHJeo5n2H*0q}k z$pMBA-$D}C*N}r!^ct$SL1b)7j{;1Yjttr@=9LKlBRuoSjIv-Ek(e@H;*IdYe$L_b z*G%5fx8218^OmMjm8Yavg%O<@$XgUQ_c;KG2y&+vOqwZP*n1=(p78vRbTg1gj^FbT zhJEcYTuHig9i$TqG9gwT7mff4lbx2hin+d!9S|a`2Er8kB~4z9x_9Q}o{IspqG){T zpk5LLS`kpZgQynrC|wUuVFZDGgiFgSD0@bflD1W=^@P1oYy3K@p)y_M=t%t-b}RHG zd2#}gDM=n-GA?ns|4dsY4bhb2dB_-KF-s|)Y%+QAOskxBC z>r57mn2gP=WSk-!B$QXq&~D^~Lvw9h174L1Yz{In8iIH%A9=NWpUNEBtjR5*Q|;nW1!&QMqP@O*!#0j7-cujp(!g(&kssUA8_&)`wY{ z(sl=+)Pk9}ZQZ|QR0pY5Y#%HY3a5^%`ym>iW-pXYWF7qGURgcf5sF~d)+Zdz+&pO# z;fJUn%$o*#|HA*~_JynFYBqQW%N6G%3?p=b#ALj0Q(-+NBSIHycI4k<$6H{!M#JA^ zQJgv)r}4ypq(^roNbMg9P_RXK#Kbvylu}(o*>Rl%iWG7b5j8Epyk+I#luKS$8{<4GN{hC-`oe z0E1UJ(UO6p6Q)i5C=XCwzY72a>>^R&oz(tS466BcNAEjGPu`!VUsrocmjn&rj2|B_ zM|QlB@SX4&$XmCmqAVOKAz62#@KU60B=v^pyAEs_6I<{9v2~8!fraT3jcwbuZQJhH zwr!(hCmq|iZQHi3n>+J$e#BYpJa5(B1=L-p~+EY>GKF&W)wpek<*R$qRg-!JYuFtvz2>s2Bp0rynN~8{WmNc&5*eN>duWqp?=Mq-KX%^%kyQuZoQCdQ?Jbn{d#HswuXpe%5Cc!R) z+t#Uk9m4HxBAeT>Rs3$y+C*AwOWKMg?jH3#g!dHnM(^76XyVkkwCyq3*B?MmboR`d z{~~VXzb3?Q2GrHJ@(frU)W`o3=eTJ7DoiBcDm|F>5|)~eRPV*sY-pRc!B2DB2DJ&I zl2wNgMN`@|lybe`7$69cBS?;NugxMR2azB^1iqYp1D%`S87Hxy2hk47&}{l)(@Sx6 z^z0wbci+=gQ9b@e_@VNu715jFy`-G&79ZBf5bR6pm7!uwz6Cp--zeu#Yc*~rE5Bm< zwB2w!-lWBEdLLu+EEOjCVDl^S(cgmK-Icpb>;6i z?ev-1z~QI)_mu|xQg^kt_%E(Ju#xGxXr%2*^ndv_~r_BmU~&;OKAQ z6moF)gAcx~uiQ_p!Mgg?EE@Gq^=TMR&iqn3p=M|5t76A@5FXqOD@T*nPT#QCudK}j zBz!&+`gIEV-R03mRsE4akqSVv00(}KAU?o%qYu~N6-hh>cjvDt0f&4UL~I9ES+jxZ z!Jqb>Ob|!Y9sqxry63<-^Db<#UtsmbIzE6rnq0C*tMxgEKFn~yw-y;mBI`J_$!>hy zNwCpvII!fg5ld|CS@BA% zrX^*{if7>kzU_-JbdU7QPV@5+q9Lvy>U_98uX{N96q}an3|g4!H@a3j2axtt{PD7Dw0~)} z#j`^0HyGRTYaU25=_SPxvbcmgetp*Hz}%AzxjCEO;KtPlL(g;EZUro5nxK0x!t5!+ zOGxIF2dY!2*4Cx>_i+Z|QT=&fs{x=0WGH64?nh#BFo8p#$h@@ARn%^tec%h(M2ABA zkZe~rLpCp=$k|TR*3C%_zN`}sh5($eP&dfZqS6T;TO16D{xmF?&L@b8 z;{SdEw82hfc3>shmUSEq!sl#S$_r;eDFWaP%wYhO644v;Yj+U$i9Yl~7G-MfPPt5@ ziC_}IG^F$mxQ}b(=Vh8ek-4je@$4)!Unv}JLagR-A=fBaNeoY#{}@ss4C`~l z|9SYwGKTXQ*-%+*9;-GJasuQk6cJPgn?GwPPm9xN7sRW-&xw8~dwkuPO2_&8?2I$G zxhROMUp_==dq@hCc>dJ%$e3s4JY^LivF}yj540%{TaN;G_lA9lMovOK(53a1`J5BD z6?m4*(2fLv$kVcOWa!`F%p@_9j30nKwE6*p873smQhDY@QM%sHjM9`akk`s>R^ynE zIA8LtxziTL$92!N60lx_*eyeZvCS;{BRbh2)hg%GZxWCKjHNCM2L7R*_m~I(RT?ya zq($t0L=u(&14ZsQwC&bX9q{65bubG|!kDMWY?@QtjgCs~E>c7!1p;N;j}Ot2;JQ$K z#9#=l!DR=9B(+$!sDQ$=#;8KL2iZl|)G7fFfFHf$h(c;H&8P)|C^J%ckr;2 zp7b$)wZ%0=ir1Ajjg7v)5#-RjlbXa?_Ck>K`m{i?SJPA)6NBG+MvBjI>ngSzm=;J4 zb}(R6&Wzbru2;LEf>hn&v=Bw|E|Y29nzrcQL~PI)ll+++)=SnH2vCmkc*-3ru&72M zP_fDq7x6{c8pM-N2-5@##L;Z2@#mJ2`L~guPRmxiNK|ncFDylZ2+rv}`JOH;rbV{? znr$Y-h4-LE*R{GL6Y=^%&$I9-7DbV_dIuhNBuh$3>|=0PFaN69vQeeStUcYNFEYy) z>$pg9tERNSYn(Ace7A&PwQNrx@dz>XxSEK6A9Qz+JRdX(cjq>m0CJ}hcZa6D2V~VO|KKG28i9!n ztIXgtp>Oi-+)e^$2;*6#BQ;_Q^Ai1Xmk&_Sss5aY#H9RIo`)AqH91ke@rJskllgoxmuHza4^*)z9dNZ8B&awzLRVtG^tYum} zWsVRf3(SX?se-1?9)5!?UI33Y!8luqyb}jzP{D`F>9;>`l-3PtoGN1iBZVsPA@3ph znq(&Z;aQ(RE$OEX4;eg@Q2607RH#tRH%YUU`HI zcXcE9Kz$yRSHX4N*dJRy{nlZ%$IuUVe;YctzYlc0WA}2Q*h`iEh(8J64aK1&?6@22 z%cfv{1C-^l{8;IsXgnb=t2j_U!xj&$GqeWoFui?I74M10-jSezu0GhseKm2E35~!~ z|GdU}H>WA^>zK4wieoPdpe>Tduidu4?k!NAig*w(711Fdx&Wp-Jd7vNl+3{8-9s5+ zCzN)rFyYBHMLp#-{%zb5p$=2o92c_^*mlB-Bvx=jelS9NR}eUiB;P8qolLsz=KXj> zf@)M*@oz=xhDlSN5m@W?)1DBl$PN4Nf-YC%Mu@q>T@^G*T^Sn`Z}_9cVpN_mDSi+P z7&S$4rY7XIie@qiHd6sNgg8)N@T@72CP5wVU`|r5I>Dmh^6g_RM{`;IWDYY^;iF)N?bZ4if&-cu!L`4=SEz( zGxIaa@Mg}DA1-@pEt)X1N%K%TPnOpCwgJN1_EKHZ)DcFWBRK%$)VzD;LemKnjesD+{&(A2_TeUwU7*ug z*A#iderPN3F2T)HS!k|0Ar&mHitzz6VWo$eY9bJS_VP~ zjz6=yglM6%e!vkg33uaH7XC57B0^x&+EV95w#{!wjCcBz7Dfl^+!|E$mx#LVHPOF_ zqoRH{zn&be+W0X-wsarfw2!*#T^JLOsPwOA4vQA^s}9`8*sdAZqYcSt zU^~pRTPAwTmn4|JKgRvRW;hr+3Q`X>1i6x1_^RYgz>3)M*7L^95?0bX$VHG0BTy)csq+)u_;}bjHio#f@+NL-ilS%N>dvg;6 z6n0y{sX=dYydUGfSb^+M@=Qa21a;V_)g-y<@41fSWf_1e;m|i9t@OWWdSE09445gN zz`3EYQ`}L#r{iVdwe5jpFP{#?qm>fipeUT%(`G6`^ZP5V9Szh(6Xod_ZlF@5Y87-n zbY3Pav@83YdOpZjCeU*k?q)(K7alv(k&>5bdZO`vP!W&8%w-vaUz4tZ-BGB6;kr#9T4eCn+j(14E9<^b-k$Vd;-vsy-79ip+@gCq zv!Xt)D1N2x!DCN8UnqH2H;XL+FA&Tkhb!(hUhTM&CUNnvxs<9qp$RZ9+%&COBE^4E zp)u4ML;p~kY50AA%WLZEUY=GwuLNj8JKwoIlr7KV(f&hf3QG&pnFlX~ z(=&SQ2x43Az!f!b7L4*9_IodY68$Z=-iaLU`!QzExT!Ai!^9hH)9su2(Z(CBwE2pk zrPM6DhF5X1$P9pptxWUf_2vEmx08}ZCI2@sN~!}yt>-|!n|mJmm1k`KkG$wV`>?Hu zS6Nw|?k}rzhJTF46FLX4z5zZAe2Msbuf>d=50RXAC z*ltnX-b4-U_x!wWV1BVbRv2Q#{r1Fs6stGz^DwG~?@lyA^4#Tsh2~b=o^TBYaf#p|b=-Ee!o zGf-G-FlpPif8I0E;cVw**Rto+wpLZIB)q8J%i&lCTz}x8qrW6NehaKOM>%S*n!fNu z)p%|lkz~R6d7LGR4k7klr$X{4*EhCs!qPX|dxx23ET#W%!3e9L>uR7iH#Y<8J8Y@d z=c?W5N`C@wCM6HY)@*rdE^W@cj1txnOXNXiwrx*&CYk8o%}#o%$9S^tq!QD;xGSfv z(y0EC@}9JRW5la%tP)FC?e*b?uqU9g>x#$oHGm(xAtJqv)+=kz{M5sZD8%`t$`{Rd={? z6nOO|?V%ak7Y7n+;f)Fce-!bNP27{H`}6qpE6NZwtn$N2Rpci+Z_lh3tX2F}R+(=` zke1E}3lr8qgs!w3_AG^S3Hiq_se&k6znNNV65xL0RC_Us;=$3X>07!kHW|R6aSs;h zr|w*O9ujwE)3Fmq%iuGK*m18ggoA^{tQP%Ihe=Y(c%B9PSDaG9Y`2vT@nIsI=dTu? z3d@HcP7|Y=@ukvM3H3`@ZQ1FJ<3pM}<;7i!dwZ)uF%A>c`8a+bT8Aw=gtz{hIb_EP zq<$TS)aQe}4g=JRjy+E^;K?%2i`+F@_w(J4-Hy764<_C#Bm6(e4@cx z!U)Xt0|>@ce?XdZxyd_IY#13f-%$fXi}=g?aRXwk2FjQETQeBE@P`D#OOSDyTa6Y~ z8bkL#_|nC4kYaBmB04OA z8U!qvR3zoAoB+f&j7Wx^gVq{$G#)};MhL@PAz zeJ3{e(fkooj#oF3X@9VazZKU17=o{Ihv|fpM&IA9v*)gMjd{TsxXUL7T_wSXAgIu# zH2ex(Mw^IZfPtpK7|#urP_6ipFrHBAMLVd$mIa%sBNPIdq3hPbSYg!C$3x1DTs1@; zB^AUV$Qrs+1RNOsWm=QGHJAj=Ca@2;rEd#04k0NK7ChA-L`ngxGsX?~VkAbf$upP< zy2k;RkJjEEY6NDm#v$1hkLVLCZ%MQakjv#4o%y8$12kgfo@6db;S@lf`YA4|UjJql zMG=1r!wE$Z6By=V4M=hTBm=z+ zZTE2A6Lc<-Y(b<(_V_mtZ#A?yTWg)b)<1TOGpwp|LmDx z2;E~WcrvqVS#ko4N^Q*z6grP@ZMa-}9Gf=`BFv~#0!y}tEA@kWBVF;$D z)3^`YGXoFi(-xt++a6d&v(tHC1Hsh%TW4M_9^<~RR`ujWE6pZFB9*x9P0TV`;%1YV z!`6FG{_;lC97Ttr;|ChK1Qz#R1q^GD&s+v1xFPa?CpjSFM0@Xrqw6aDC`1;?(4R)j z!J?kxGWFOeBk%QJj>bbl`v;4@SU+NBy$GTtaFaC<0#v&``{5l!`O7tP);_Co>nhls zE$O12l1v9XIXy@cuPR2+&)caWz9=Qqr-7I4Xwhs1DFfYj7uQ97iPNS4?ko33w18$8z7PXv2 zTvg%_VA%nwTd3+-C6VH%KF!=$zf6uqvawP4JSf#06<0dw0}oenn!cZQ;c)Qgq6IMq zYZMq)L0bsO9TM#izR^{{+af&RAeKX&Xlche^@aXLFH%1?OoscA14;SJ$zm7-utQ!| z#&g&dBhaUeU4-E0w(Ke~H@;RRtnydJEg>DpW;~-YpiHcyXP&yRSU~+_`!v+kgJ#_b zQM-1(^{V7TAxP-bLt(kw)~&}iPU|?w`{1$lycf9ecRr1pK}R*J8@{4z`$SGXxhops zQDh@_Q-6MTKUG&wig50QlHsaxeI>3*^Bf%2TY~%}%3Gb|^N8po5Ri?t?8^ORNxoT2L;2n~B`vsOXl)nAdtHSLiz&j#LWGBP)nwP`B^A1=sp z+v%S4d&G~R<*SqtVqx-Om0$$Q67&P)$>AXsazntIZAWCpBjVA>u>gNd2;-(@<+o`H zT~h94$)cKXszjn)X0^%3p{Iw^H|}eCI#muQO45#WB^S->lIZaJ=0cVWsFPaz%1}mD zF4UtW;dw;O{`|OSm(!W`P+a9&8eaAh<*P=9^gOUU%V=Vb_%=$sG&*OUkM-~95d|tw z42Uc&3DJF0w8RX!#E5<+ThmlnhFiycra|~F9r##VPGtEt&vpcUcSf z&6z!DD(0`7b6;CfPMaNR`^1-R*M&}3(@E(qj)CDh#aqfq^>P@IGAk!p3}~#>656zT zecZ5E3VDZi+n0vk3jn?PsF?8?!EE7joRf=#Ya+_G6;T01!ETCNC8DTT&Uk+%&$Vei z?Dg-6-SAQV951EXHHa!aTK`*Y&D8Cg*Y5^8;yy{v!aTPY`T;X>A;SigsqS938l~N7 z(PkVq!&0iT*Rf~*{RIidIF_nYZ7rUCT#rgW4cy|BeTgRZiyA`-)hU$7*iZTp6AP@6 zjDgaycWISe+hB>;iPWwAs#MI_Iqr~NsZ@>JGQ5(xFTNq6iMYh0-N&We>znxb>(W3ul|X>i;qP-qSAfeOd{ zRza=Zql@`@~uGMUai7(KKcvp!)>sPY>E-bVFNqYoa1B- zNk6pp@bq}>)p}6V7Ta_fL6fr&)#;w*`zFqhf78nmHLXtzXOLT3Ran$4r4VS zGK(ou{hYd1qx!5zLDGe~_R%f66yM%(hOS;41wEd;12bAtdU2Yxg7SJ3xJ*Z zCrOWiNXtnagc709xz7upEx`T&mgeW|D-2Pgq~P-dd_P=XUf%e99T0ymIOkQd-Oa{L zv8a92ZZy?F+3Z}ETi{pFe*1m^p{WUPPT63r8=>A9sk3fzz&_aBP9WliT%myb7X0=< z!SKf3sTN&+uu_K3H#Qgb_ENcE9JpI<(QV?qqQf-U2_WfmIxMtRFPa0c^rr}2GrkigfWVJ zda!FO{MM9N4a;v|^1S@s_HBv$)(jqrsA5&yv%`vW#g``N-LXDvFyE`oL^D~d%~;2) z>;z9;fp2(3VO_~q$kc8{*jX9JI4suSYeRpu0-hb*x(97o-5xjwLtray(XN?Kh zgZ&~GUO`Nr)ZO9X*_H?w^#cg&P;|EtUAGa%`x1>BDZ*G~y`dKrw5#`I5 z2j0wxCsR{FA{+@6AdKV4~`tR;hDDybrd%`1~NgIgl~V8<-}u9HDUTDWY9kk?1>nc>vMh z{v+e%(rUUi;8^=OHwdWs4*PK|CA>#zl)iP~pCVjQpDo1uxa+sz#1^*oN9YurV~&mMs{n zMe>i}V9stUM2Gd+&B5~8z6`J4=60G=GTEKDw+?sTD3DxM?{{u+F$mp9P7a0|fi?{2hN@#H{dZ`6JM<<#=-KlGPN#8H;q$ z4#VjC7M-p3riS&toLrLI?T%{tpN7L1KsAVPG){-ZK`QzKIijdDzCAb-%SHwwS(&jA zLN-C?Pu&@#`}k&YL6=;QQg<)-@-gnElw1hUC$w{RYpZV6iBqgTS zsRbj{YnqM%A&O%rbo7Ytf%ZI@;?Y>l%IK}bv+XjqB~m~ZcQFCz`Q%f6xxRxv3pk%$ zmz#TX$*%I4wvX&iN@SOSbZWpPcL6=rfgcMr6CFerMG%We;N|@1_gwfJo_mwjrx?WYKgHRbvwX%$p-x7K?+aQJA%P)Gr2*L`8&_n``CH* z3AJDk{+s7r$27$_qj$!1a!y=})Y4L*YRD{36&JxambYRf=tkpSX?`;aWql9;;7d=4 z8471~!EnT@aPGuDgX_uT+S|Z{8lw1xcB$w1ida%zT5Ywe#xZ3f*6E8B2r)mwI<M~TvNz=MEFx`O6a(<0S&4$pr{F>b*chAS z6POA6FpkX-xcryq1epVfJ#M zy9vS&R!mc=-(|mwRFo)h;Hgv`bBtJMQ#8v*X(&xMr@RYH8f`>WOxUsLcu(E|={OGB zyi`F}kHX2C<&pkg(lP1Z0Lozufw9N0Ro250NLe~0{GSU@5tLyR!h%-T&)&aHl?3&< z#@tz$_l}Da-%~7@uo5%vg9ENmOsbp)rBQTU4F8CUUzP_Eu`9~MTrlVf$)J@);@Lbk zYoq@99hdNa_32A5_MNL*@V>b87=32UX2UmnoBid}b73C4^9_tRZFNhJqY9l#8(9!|l~ZH7S*^56g71((Aj9jr zGxqG%pZk)zUZhTQd&+NQ$s{Rm8KTr}$)dEP7H#vEuOQIpJou+>&()v~sFm;Z%|T_; z)vxSsIw+x+=Vts?K#QhIV)KvsG4Q{40i8n74(`Mu(a#DaDqjySyh_j(mNg+)N#Uk_P+8HXkjHKh4Jff`!Ui)TAEq71o zpjHggys0$h@|C>(mR*cBY^GDk;uHU9W6SoS^ZW$w=92+=+S%5U{k7LPsimYC;MBy+ zsSV539{G}u?J$9I3mJA>mtoqJN(KMjJW9|;MX5LupY>`f9s?v`yk?yX8%_y$^tYm~am<_OF=pEnwlb$ycq}Bw&h>3IZA6BbfaEM$fbN8`=j7fc3Dl|iUy~ck6)N`=ZK6KWMVFd) zwm%ko7A^NfzJd_)nRtgZrL$7Sgyo4NaBVA%RF!%te}%P-GQ-CKDriQD9Hib=?Qrfi z+HgA6@95nFO7HHA!O^SA{ADB1J;t27XM7K|JEl_W!1Cju={1evlFh`Z(D=1MzgNVl zPx4A9)M~lHB2qO$cG@C-?VkAP`NfDwYE+&wzyQB1)5Au){HRz3ANs0$tPM@wWc@&7 zewiaAwn|&Bndjzqf)6;J+#ehI7p1jbmsv6B}`JA!yuYdO11ZqOekiq z8Jn4Cm(>-DY!VV%TO+ycyso%RIubfLb}O+ zTR-fs#bAaq0=o4b4HO4knjG~^e4~JQTOlu3aMW+DHOiypq8;Yb^mOpJmpv&r3 zsS+Y%i&eXlLqY45BO0F~cDIQ+QA!-h8OA2(F6k7JgUc&o`0XHfn)xr)th&Y>V(3(K zu2(!ja-xmb-9!2m$0xvGJ3({G`Yt(`ZZ$FY)sB)u@~}#` z@Ccl=>U`tB$v1$9IC8UVFHmPXngiI6BXlf>q65$%Beer(zaY@&mvH_=F+ZN~XpP#^4`1ZF8kFVOIRSVy(T zMzs@*zESVety_Xx>U!M;ZW8}r!}OYD@$CQS+T76lt$V)DA?M#7C57PJs(skpeH;*VTl61qG#9EY@kt{WyJX=JF>_{KyLWfDFPGkf&~iG1!NBh!3Y+I+ zCduy(a%^@e&G`J1Fj4Q)S3z$iiepUiFos_-=4f7maQ}92D&?FFN0qcqG4r*^#`&#R zr|r_kVb{V4qEk@j*>N1nLm?u%O_)JjBoG0OMVP)+-75?KvDqR=f{=H ztlB}hMSBt;YS_|VQQ>;Et~$<-&*OgE^LsktezB_>Z8ZtkQMYW6p=6fTbXb_f{UTYy zdfIadxyl&Bc^YmDmNetVRxLxV&_jHWEhgCC`KUuw*6n<|M(-agUCm2=rZ6~((pInpjj(s7!%fz>{S!KH((o60FzICOjOR6E|CsY-`lk9{m_zr8G z!)jQ0@zz*mn0JcZv|wt@W7*4RU(M`JSbP!wXM)<-zk@v~kb&D?wr~qad?ETBQ!F9G zy|)lF+QdJj^EO?C@UXHFAI3&NtN8UiFL9NBWPwTh61-K^VP4Efb$*E-UkHCZKoj-* z7;X)vn#=(+t*%%Hy^!K!uaTfsERtz|`O5|jOu5B?Sw&}TNb$)b%lgu@ByMVMN`McfC7&ULr`W&*q^`AF*?Q@LXO zL+7{D2IM&~1ERH^__4ZB#uJGGOi4+tyQaau3k8=jgU}h8IdlH!LQJC*5(y>=S6+IV zbjM>^>GND<3`qd)7KW-tltgm>VpGf=tX1cvk6|NX787tMH zT!}8AHbn+%)9akn=km_r_dQ}X-(>~4Bw$+n;k+Q9(cVRR$c($qnB^Ezhun3fV%E#( z4*BzRqad316bJghocrB6lG|uK{$?b&zig=%h~O9^UlNNHyR^@AwAancb7Hse5-U{# zo6oLlm>0#|R49;9cw?5_xLk^s!(L|+G(s)>${k7rS;cIq z<0fQCgJIOzEu;xIe1$v1OuCOdv{%t+DP}@D8tyWP2g+(faK?$@C7DnkTvv8R-FBku zIc|K4ah8%`tPIiLGs*3AYR=qbNfGONnc%qjc*Q1#!BbM^_(&3A2orky>NLcj&Be<$ zSLfA?gwq5zfrc;0*NiM{F~4aMCz^wZdj}DMjL2M^#a7%XX2c^;dab^D1A2+ys{s^+ za5o%v98wkk(Y-;ukStb)$>N4U{YYu7Mb$~JH^m}?=b@bfk-*D>oa=$A0&ko(=cn_v z-}?xl+oWMBv9rz?l;lgJtKcI)f$fqxEad~e9O z!-_FI^5WWLL=O_&F#wMNz}s78uJOYvH~~Zq($@xBbJG1s(+F^r}i_N1LuAfwq5mo7`N zm5GTdi=5a`Fdi9b3Rjwxj(w(u?vaKvWj)1e zqszRLQ2bOMwOQ&r#`djBU-p3j?VyG5d7PAEl*7bd(omGO?|Y@{pRRTlZ&2QU?}n+O z-2y;AAzL-dkOBOH_G@b|IE)^pOeg7lrM72KN$iNSwncOp-Eb0~6}RR1<;z8xQri4~ zoK;E_*yfM0=R70)O;yn&zV2i;vLdI2Cgc~}dIPeo6_d37O_PEqLRN3(i=;dKX%lBn znIvo4VTl0@RpG+ECfsWa- z-z_l=I`fFJjU2fZqjJmpc#S_L@^IUKJ09rYJ7I)AN`^kS?=mzBt(RrQpT@1#CmY_+ z7+0nX;61ppWZFZFc;cI=N0Xlt{jm)y&+3v}lvq2$Lo?TMTY3)5FbN2Q};^ zVNSj%Uak4-oPcXhCzFR0)JI#U{LWnOB75zq=3*r3HoLio`sSk8CePST2_;Qm6?zdi zgyQi`kLSz3D3nUqVcO1_KKiZX|D=}GIv*mWX)k!`3JZo#H6VSXy(d5n`W_^`ysr*6 zZ*lnz16_YDx`SeYs`Drh1V<;{Op@!tG=35NM5A(6Gu;MCbqc2d0Ac>*_tBF>DYYX; ziH)9=GKeQal0`u9;yS8&>E%+M;<hyFt?!3`eG2mLS;Abs4!dXt@)Q z=`bNEK>1m2zDll5%;{9?*&!H(XOhaP-(}-gi1CYTEe-hK>QA<$|6Vv;sH;8eGS6K} z3bz7!BL=KV7Zs%#=n(aTVwp4^S09+7R2-WJ%;h}HS)^ll zXT(N65tXiDsr;D_Z$UGSq)r^#2z|pZFshNKz)opFN(c)YO)|txHrNwAfFL#~PNF#b z%l;8{BcbeiCkdO~bl=VooyU8RXt#Cwz1JZBb9R1`n`x1Whm-t}iceFAe1k$C&z@vI z0t?!~%wqKr^6v0GA4^7254A5uBkxK}5GIimo1fhRL-P&sw{682 z9uocOVyhk_OMTBzq7^Qh2Fj}Zd|#67lz|$$!w<1eZ@O29{}x&qAs#4-Pvj=V&dd>H z*OW+ z?~GAE|F=mBu`{U~;{II_3`%;E?R4@zxmTs7Ux%j^K+9Cw*i?0QVFV$z&qiNkR)k@QHDzE zpqk%^_wBA>nNa+7>a|8;xkB&I5UA7-A1CMAj@^%^><{4ou}$B}eVQS*3MzPUHlJ-jkdGkc5dt+AP;p0^sSvlRQ-`YL0G zzut}Y8z>=>pR+nn{6<(trB#Om_PF6!`)GdhdkpsH5u$!BJ|-yV@(Wk*4kieB=rg-8LV3<4_il-JB!$n?;6| zX~Pv7*{MDC9&DqmUoj?{q+z?VMQ5A#Uc2Y3?frFns9jXF)S|kIM$s@thHcPO^4!N0 z(R9ZdRrlK?3k?vU@H6qHeBOZmpwCp3h}jdok#d47%H-rz@E$I_Vss}7G(3QK8J@)L zv-V&c;-pfd0q!W_*lcWoJbVL~;Ry{?7ZuF*2sKoXHs!4)+aUz`Xw@@_+)cwDQl{(; z!2z^*j4Ur+;q58nF3O_r-(i%^Er943q-z@HqoQWUG<)l>tU==wdFocSI zjm~a#g4pE|9Ier&D`^ENr?cWLI};@hAMrki+4=HvY7k<1ll1N@&UI?%F z9v=|CZ(ttQTPI)FxG<+jdt@w1Pb1H%71quZYTK_|3)azoyJ*vJQ87gg-8E}42t`V< z+`v;LHEO-P>pWeF*JbA`XZIOl#4g$ce^j#z9OKM-sUc-2W)1E~U#v2ZcnlB7*&#Q# zco?Ir+dhy07%rZ;z1+o`AaOc)0)Qh#MM3Nxsv}-vGKXLl;!kd)Sgln>>kp9MNZBEg z%l`_Y#Fv0bB=vo0ME1<+rX?UWXX{}gh(F$)US5(`3%=n_*p{0KYAmLhUwOv|m5?+v z2X>SMRv|l49OQpwxzsmh;uL1W0sBhP-Qm-JtGt@>b%#~kt-m}vo7j-MHnv-|@P*Wt z7lMSqW4hyzj2IxS??`mLzujF*R7H5gB~)%kJ@60g0wCV>Z>yNS31BQP%*XCiFOwuJ z*{RChZAK2s3F3_8PL#5md3iSp+Z+)Pi3avNIsMqsvk*#H8xiGsUCpL$7!tIA9@!S* z<%bM2*3rke+nS0ZT3^z+e9{ z!sk3HU}Tic+lUKYsIkF;YLNmU(yyspNhJDSsZ<%HV$+1Qsrm^x$WZn_b-AZwB(6)H zpbRm)@>J-FVBD-lIGwm(w^!J>r^P%8;~zuwvq#e7xeplNtfZvT{(CM7nsNjVBq=|N zi1m}7^u4lw`7))mWvseYT@!CawbR9jdjPf5H<1?p$@*k(=R|N7Bc}#Mnaa2ztPxQP zK`&jkRP%_50aWuCY08CL3yFS>9bUaTuQVNa#<^>66yJsWF~DNt|ADPn-ro}_OM8jJ ztn;fAO0;IGk8tUS`&e>;W6aE=T9#nDnst>;zN;89sf+}1DFT~7>%)cO4>L^HDg^gd zFyo9SEPva;Wd|j-3Q(BbE9snaQV<92q{EiLi4(g2m?{jsJ)apmw#9j7jb?ixo>OM# zxxmtaj~RH)mQqX5KGV)(`V#(eN9UeYVGMpeaHxyLmpPszlWBY?)w@>*S7*C)55X8mH6UYg ziC~XrN(7n~14Ec7JvEgyR+~s%&aCh_{!~Q>!}n|DNvppu2Xo6XX$U9Z?w9&mjIikQ zzM*QvD5so-&#R=2l5OBv`pR)ywp7hYJ!hANOX+so=SOs#PuVO6E?ubowpOK$mnkDD2vd20r^c>@bHs>C8rV_zaDZN9~#2vSlg1YRbZ4ZUy zOhB$z@yE~BdhSTr#*gbd8k8R2n;-gP8?3oBvVO1NR zrPslAV2PJ|miW}`>zvk&$BW(S^cQm)w>}Ujg0Is;QSU$!Y^u9NilfaDnBAr^Pp;OjfO$k$rLyQlv&{rBX*t@bvC4G>8G?DC%2+|Npxu!fplX`7aXVD@e3F81U( z!}`0%fHkk)!Lk?R7!%=;k%UFq+NsR<(@W-vRE(0=lnN{#9Y}W*r*AvRwdNAo>ayO@ zy>}@wJ##W&%8nW}Ky&|=1jQI~8(oiU062WSupQBR>;b%x%^+s;azU+Ts0}4Ml$74HIzd-=fqY?7A~uV897G>142wmEZ_eAJcH`y(ugHaXiP+Wk7K1f=sJ%se4xYC&DkO$wfC{9e}?w!v4^QrVPlu5j29!fF=#tE zX_K?u@4% zc35va&H8#q#lLMG1)xC~aYzHZ8jdX6XiAoAARyOP1Bzg4Ysm^1S0Q@F z>^T5beVASaRyZUHm%*%jtf2SP%Ds_vdN>c>lK7`Ca-(USqZx zK&QQ^SQ!f}uH?Q(acoDBXYCa`dNev$lvS6d?fE99Lw~BU$IxiZdeg#)>ikv_Wwy?I zr$zbq`;4G>bd`*GDv=ep>{Fs26zv%YjdUH$ZVnV4#(E%JE+4|WOr)>|4K{|3syc0V z*?J~~+t!n5%<}OfZ6qPKcVh+HUmWuukcow7?ymBguuL<5Hyd9QV;CoVk*`1FlBdVv zbR1@=5JL{gT%|3EWW!rGmGhJr%^Bn(fp0rQM%RsdQouc@a*jt&)=|VZnn*`oU0gJW zBJlGDm!|&;iUkqGiaIIJMi>EgUhl+R@Rtal$qNppVa=!=VgJAO-UY7eBK;o+#Y<{l zfEmHOoJ>zQGtu&{Wp>$JZ@YeLciXM(07s<|j7z55w?!mFB_t#yBr_u-B0@q$G$leL zLP9fQ35ke^$R%Zoh=~2YpJzUo!=+vK```Yr@BV(-?wn`lGoR-^&&)hC=MZwi^Wo_m z=j5&VaKL@-U!|>C89t-wLT5zD=W#236&W?y>$deDxWAF>c~8efcRu)K*Z13;W*#4! zAA2OG@PXSlU3t%~-}!hQnLDs;;mN&0Ge@+3l$80|$`|LYi+FRW&sT?*kNd^$?;aZz z^3I`>BgMNdw|_qI#L4X^7B>V2j19bQT~Pa<)_1tv_`#g=)C){{r-_f zcYm92opjg1SAR3L;;+9s@XNgmK8u|4wMW={w#?4@g1}8XzOCN9YQwg+6Yn0I_SQRl z?`!bBqHNQwO)Gc!-x+aRT=V$j7gpVCyZG^uZRfV%wZ7%G8J*J?7vz7s^w643Bcj_L zS=l^p=;QNGA4&Uiz+-=S>iEmI9Ju&j_JceAJkIs>)8AO1I`zb-z6Ix73RihdN!e0u zub6ghW$BtpSAEx69{R`dxWD)RCTh%rzhBKeor)o%dVu)&+rQ*tqhsyk4;#nc8}QDw zVYh!!^J3#&vt(mG_eE37KPaDWfAcq!?posd$+6^d*I!#cVX^C7GwOE*{_eF8PoCQI z_+MW7bdarJv2T=l^1yJtQ)E%t`8r*9q9^58q$V#jv5+~qX! z)kAN*-2c1ZK3|@2*6hkPd|OdFMZ#9%kAJP_xBeRzx$WJE*|p8 zzhwh3xJmTDxBfD=~)_mYA-+unLy&Ln%%ai5| zzia!S2i`mKhK(z#R$cz^BXz=W)iu{{u3L4-tq+S5m_ zd}@kiL-G4*SL}Oa$`c)v-WxJ|;QIsj&;4cG{D-f3U{J>W%MZVL_58@+{VjF*!Z%0F zA9Hf~qrbZ)%-dz>xm%L$zh1g$)uy+yhQHR&$;tEL#D4$upJ)5T!IPZTpPL>$P<=WL z##ik4p97%x*{nb64cPQY1`wgQ7(jo$r8jV={>VoM?$Ch&KMr{7kJdOVXz(Jv<%AAc zejK<;f26(bG`Kfl*ZZu|08-kbCRH&3XT4=WFJQ9XV(A4u`K5-ZpZ@4-9WYw1(p&D* z0jm!5ZZRI6)G=SF1C#W(4B&RXW|* zm$Q8xN$u>jkv{?k@rTXAAKJ5r-{9Kr8vgKigUumMaVCYYmfh#KWvTs;p5I9+Ix>gaV>O|>Jq^pk6gna_RBPL#r(dR8e;q(rH`F6 zV)-NBI&R#xl|Nn?P9IOtnnEAZyIH8HoI9zey%2r#WBhDO0)1%6p3aw4d;0s2x$%zG z{Go6@opd!nd-4fx9FoMf4|mbWPDMScGmabISVA8=LSyJ7W=Imvf&X941x$qPKj=dr zT~0$3A7|-D7BZ_RV6E{T6)P<$-lW3g zN)(%@sG?%#L=;g}d_lz>ND8>@r()0~6#i6{QE|g$6g#MBL_rIge>dJQ2>DPea%exg zkogfYw~&OXR5+7_4^VNEifc*29aJ<>F^44FO~pkjmXny7RCtnvk5f@W#a!ZKw}|`VkI074K4U72$Y_ifSrm5~6r2+Nk&)Av#XQSi<%=6*W}Mg@i8m{!hewm?Z3= z;=hQUgH(7DCy!Hcj*3~tNi-ErRNP6NBvWx2ak7qzaw?_~Cs9;1p&)Cz6ShDq%Bh%1 z*mhIVM#W0PmPUmOVe_UUpNdI@Z5tI|QL%(jrcyD0#Ju_+in+M^kp3UlJlhu!UA$uH z(mNL4ZCSkH_MhRO_3*O~^!5P^laY5XT5^ZwwndAVSXSO~_dP3@|J?AiK7MwGpGU(_ z7Yt}2x7g|5@Xzz&q)S;Hc(zaKzv!1M|2O>~EDgAOld)bg1+?D>d>Ik=40=`z~B&zcc8y7&|OZ*AGLG0;PvAiYK&dSIn@&Ezj2s} z!{#%@a<^ZQJ=o6{Y#HkC7U&rGK-MoX6fc6)>!@85BFMfb)ZZ2~#bSSKJy0lyiSsKh zp#h=6BOJO?55;;nfxP|wLe{!E^y3D*zr~?xwSVa9)f+8A_D4eP!68r=Nj${tAWaaR zm?O)2o6lHNE}KtKAh;t9vbKWk?uG`YT6}Cl_SK>O{u>#Qrb6F9|MhGA0)j1}!K_(L zV(-Aq9nc2Z#}aMr6;n%oL4g5&K9=DIk-3&&KmSP%e3DJ3=*%d&EVk9F{T}PB3k!4s zhv=p@R*4_%;o~0|Y-h7+48riHu&ez7Z2lHV5fT*Y6XF*b(6>>8{U$iDrZI1-8`Q6M zc&=G!qs7-2M6RiO&s585|4|P8m^BM9_hdfUKX3zB4-T>0{6nBX-P}!H-A$G$e zj{XSk^+cp`s+$Uq;izJR&*1fTA3x|pm*@&&QnQ(1d(DO60)0`zm^5Aq4qS(!qQ;(P za>s$~;DHa@*4W{$8*D)#YXjE=U)3FnJ;ZOFJ=pMAKVLi0USq0bt!>R3e^VD5RhZD6 zX8lM9Wb5q#zJBm%gGEDh<>uX7$J-w+w#G(IVGane)>CX{dg${srMCyHf%Yg^!BuSD6Z;^}&W(YtsTpR~JqZI^dy_}N6wCF! zU)Py6Uh)kL^$xMvLL8m$TEn|7HvhnYHNmiwC2%z>&U76@b*)L=$M#SlgyI*trx_+G z&O=BefIbrH=kqXyRgRwst-Y;a#4{GtqIm_vpDrv1WIgb+y4W8uGL=g?cq*8LX8K@E z1q#H+Kny0o5PL8@x+g1Uctb3+2axxSFxA6CJBAliJwbK{yE;TC65cW78eY5J&eN9` z(Hyv%syxN&;6KJo`nWUwb_`@N;=b-Dmi2))pS8+Yx{V9NNN_d7KE_*`O*Ns^L`D-G zVos3lQ9Fe*IKh>sRdgp<2d`l#P{E3attObwe}io!2E0MdkVOk@-PQ;+0t{)4p4xP# zW)MfuGOa~3u{qtb2dIY*K18#IzUJuZ=idGfD-vJZrcLl|-EVEHL+nAyZPkK>+>6@! z7$FpbS+~7HuclQCRyu(cC3RMMMnGEr@HzxRNAIKQnS>0r`A@NQ^JFz=FlPaQzL=k| zkf2#p&A!?&Q8N^GQbWazqa2rDGM$)ujzDNcT3r#|fgIg6%PRZXu%yuV(VSAvE{3Lb zEAk>jdlduA&j-`05fre9B8#v>Nw_{%1KmRz;+M{^%`uii&OR)o$BK}(SQMI`=pZzv zN|=)dQ;$Um`5q}8lYWm!eLEs?Z!U>Al8rJ0NNBK~hBwbAy%to;s`@lR{Tw5N8EP~) zuqX%VHonzzV=b9q*Vo!W$EARpYV@IJ^HEcAHypaby03R3li?3Q-P<{h*GxvC_CDq$ zC8l9Ms6_K01Qydav}C4;qpiP`Rvg_=4~04+tju`eAgCLRc8e{#LO9$hbF!0a0}xx6 zYiSzRbb~+<#F0;DM1P&>d2HmhKL#T$!nL`XbUTDlC<9qHB5)eBh%xC{0&HuHwI9|y z{DvZW4ZW<(z^~TT?t6K@T)fB6{uyQw}rs1ieugp8~V) zBj9bM{@eoB#%4NnJg~C$)b3Wdt?o{v@er)pLx?F_(%FI;!{x+`CgJKpLZWtf_Fx~| zdYdoA^0kN9eL~pxCFapv47bryTShZ!eWEk9KhEG$8|!Cfy|!F`h<Aj|LgtCD_L`x4ND#GjS&_cUD9ze=c?PA6E;9>f7H{_ zD!Rk4C$i)5+dyr1rO2hl=BY|$Jj!(Qs;JUU(MlXPt*#%3AbX23fMG;Cm|yLGEohii zlKxKjNrm?!K|Bm}(g-Udc7Hp~9@_XDvI*0^Qj8nq)1_sVV~}1=GAm{3I84KtMh@Zt zp_$Avw`P%qK<7^zrwIFFVME;<8^9Ano#w4Ajz3J%X$ZvrWEcYCSx9i5>TMS|f;gzO$Xm9IU zLAHmpC}Vi2J7`Ex)dhoHZSHgQJq-;<4IU@DhwE_!u?hy|xkh{BIEiRJWeExmP@2%D zO4g2r@I3Ukn6Ie4GHs7do%sa`N31&SsY1nnwR5NV&oA?;K=W5Z~l6tXrbFm%mY z3y-YsVcE!0U<;`)q=;*iX{8?Hq3(vBhE$aMb6|J%FyfY}AS}_4*GH2+bX45rC6Agj z{jZ`|Y+w?Ep?YnosYy-SMNE|r;wmp^d5s9gTT0fgqdRMnQ75fOLCe}JnK89^ouWJc z>L42&LUa1-3<+s*X6{<_`ORmCNrRzLkG0nC_Wm4 zTji+6LsiNUMm*J{7f~ZOz@$ZNSbSIqy^O$~qkEPyHYZR}?-}eUUTH&%okjO>JuYyJ zH0MZ~-RGi>Ln>C3%Gyfi9;*hmv0Tg1({bv9AD9k_Tq0e8+i(eIJ3mPm&x<2_9tL@~ zj~tBKMEq~?)xCgzdW(MEIC!<*(i?b5fApjd-1XzY?fN4Fz%?lTM=aIr-}$Q5fja!m z68#(8U;l!b0eq(S=?&!Tk8ai9GJu!#M+RWmTMXbf?Jcs}NJZbF{1J5;Ivb$%HNQ<} zJnFAMGFqJV76a)0$at%FAER@xKBP|L>rTDCi`2)rw~RiQgjl?du_%+et#_hsLXqaQ@h2Wiof0K@ODZt_E=4pDq4)mHABIc(Bwv2GM63OqWzzMed;3rFMS(s< zJK!+ZJaf%^`Hs+!wF6xa8&@kldmuAz)SGwfomT)^pY6SnbqhXp``l#>6-87`z&gX_ zzo|G!#lmY)JWqw&wHRaxm)g_i7IX(;$KcbA>|wevJ(egwLd6*>t|yAIR5VhtoXEb7 z0%YH$=Ym@8RGgyXI(i;OMFSNRK+$D9k#+iUvcn0F9~H$^SP0J(RD43kErjO zzW9rw**4xWp7sHIUL4hHlzO(0`gdHc1e*VUu0oQWqk7Bcd2voJ(Jr;YvweMYe5Qy!x_Fxh?%Jwdk7__Dbv@->9j614o2x-okRZ7 zA+@TdKyaZ%V@%6QDq*#Rj*&ISg$exxz1O7yGi%7`9j|p5TGjCBnG?>E26$ZiuFb{7GsB8ITGAjhkoI{(@!H#;MD#YL2Ec7iFi3pV_^?jr1VX5=!) zvg2(nrlIUk{J`mPVUw>ZYgo)d$s?qD)ZfM(YSU0MbQiHROizk(Fj*^K?O71pu{ibXK8gs27mm zrewW3<)DiOdIT{Pph7>J{Bpj}pxyBD=d&wCN@eKPt#Y~mfV(aFAsVhCk0heHp6HZ| z-_(8B;lNWn+c#a5kIFQ|I2@IA%kJyfEsZlEI#<=5!ia|awE#y4pzwD&onZ1M*{L|2 zplbvAfu?yKe{~=qi|O2IM=~lUZww_)y&Lf#;c|?xza|Al5P{)ekeLSpZp3?LjwcZm=r!gH*KZ{VOM|}LHg#thA z;FxckYet}%Qa^p>xJbu>{F2^b9C+xBT|e98Q+B=UdUcI~PX82I+-cG#KwaJDy>y4U zhp%A3PYpd72sO=G)`j9`_FDEkeVQZdaD(2N*vFLx^I%(_vy^`uw$+z9e8j5akB(F6 zx`!|5j&2xYy+0dPZ`WhCKG2A|-a*}%s`-?;mnd9apbF2G{ z>M?Sw)k(&E+h1f~A?s#4$>LZSkvK+nEOC-`=`JGj3Yq$VlWaQaBHG5vhMs@= z+z`<@K}Np}_XuCq$B0P!-&0#5`U6oDFzb8pE zY?JNh_+7S1*Iy;^#x}X|cS*dlO>&+YO|;l1x%)?x9@r+)*`tXr+a&zNXrje73IAxc zC>g2S#8u`ujuvN!lTG5j9W9E7%VU!;K%evwEyLxRUpUFKI1f=fT-L2~k{K^?Ykn++ zmn;uaHC&b)0^E`BA!0_z*lNJr%RNNI2-&Pb)0HC$du1DsOfDGG+m zgmS=*_2@HDrnd6%A0y8Bvx+B7*S-Er5=c1Ib+1J$uefTlRR2IMzl|sS-*0U z+2L1+GOOJ0<0Qjhy+Wi-mS-P#lIdk=og`c1NWhjWM9Cx>_B!Cg-DAbR$ujH&;QFIu zMYUB{odcZz*;tWkm5JX1ZrnCbWB_>#1dM-UoG6|I%z$e?7$@Saa_ifGTfQ47;;k~W z3UK7U@$_!o-vDP8jTg0(Wmyyik2ybHButwpc_g_=$uK|&BvwlvDeW7J$h6Ax4**x}xl(LJtUj+P zt@_B7B6AffEo#q15$(fDOYE8`(jH)?6;@v*G9P56wH}=$O1)WW`7cZsjrX(CQh%5% zYVKD`+xoFp#N4lx7N2{yC|#wLR=4{aQSYskR@8BgsI@7jrBq)lYF8#es@+{wlRGP|<>Q-0yBjO5>Fb+C zzME27+LoE3(oHEXeD_R|?5>oS@*3LRmD0M7&lE@9mD2Lg&J^2$oII_tai%D8Q%c+3 zIa8FnE2UM$%p#aQZU4)&hy(VtsJvMs-d!nee;L}{l+vQ=W{DJcrL?TJS>lYlQd-20 z*`nNCDedeFvqc^FAy3Q6o-MN6mC_oEXNydCrL=@|vxzQyTK&b@B6XChw5Tm}L=(>h zZO_gTrEaXWf>-B=GFMhw%+Wa_#*LM>@BKL zTFY~DMI&Guo>l2{MXsAtT5SGYk?E$CR`3D$VWp)t%oS;_N@-`?=Za2OrL^)L^F*|p zQd-U4c_PP6DJ?E*o``T$N=rB~kND-`+3?Xk(c-FEjM|-=s@_V;hA-4zNmLqN{cUsEOq)cv}FV47-r?s`r7o{#T zc^rb~xAR4{i%gq~#b@VyQQ;!Pe(xk(!xo6ME;2V1E6yhuPzR3M$=er*Mi-fq3GGDf zTp-F^WM&y@YS#i$Cb(cH>lVH?1@qtsfr3o0moi@=IZ zfX9vzSY8G&{WyVIc#^U`e}Py_;@7@S07+k5KoAL^`3^xOdqN>WBzo8hf>`oK23hd( zq6K2Lk~`-lbzrgg78A%*|JHX2WTBfGWSJ{UsFg)N_8x&OaY`wHEO7Lx1tLVr8(y|R zJf!43&mhZN@jkV(y!qt>vb<@h31oTW&Jd`{>na=C7l`5kBydI-?9-p8=E(4cA{na3 z)SSL`AsLyb=CB1t}S0pSH2`*}CZcbV#QZ&QHy|_^1 zx=5ayOJ8Q}JT+fQB!kG#zN8IqJ*n_lkxM^9GT1bd1@{`x=@^VQB!mF+Y3b= z9Ezsq<`WA=RzH1ec9rM9UMQljBC*2{heap4N9!sXYLYJ*DJAN-?~fe zyIzTv)p(aEnxe#NkNbrvoT9`!`t~oVKU*vQufGuOUP`R!7k)`FFQm4f{-vm%uEeU_ z@+(m^O^KD7{VQ?SONrI_`L9IDbR||%!V=LkO^KEJ?h?`Fs*AN)ww+rd;`@_WQJ*dm zh5cpNWf-pYOGH+GUcQGnE)fO&c`z1!wM3Nkmt|LD>Gw5vpp{mB%My{&U$)G1k`3Q* zYsMWIq~9^JxD{Azb}kX={bljJSO|qJC1SJ`+q-3{*xO$w`eUuObt#dc5#6;H_E;M3e8eH&s{QOr`YzrQxpcQKChd(hq$zm)i_Nd}ymxRkgn&H&u;Jii-z7;wUi zOG&J(0>IVD+`q9DaCGWYajd^AuK-;78gtN8OYLtkehwPRS^VzyRwwR{%QzEq0cW0CDsrGv z+LSkxvy5!`$g`{l-gsnv#C@_qL7&Dd#>t!Y#2UuKQ+>^+EN}S_=+jxxJe=Q(2pr$Y z{abbdF8+$~oP7>(S2Mp`_7dQN7VZ<70l4)WmaF*?;H}>+CA*%<2b|u??J*^QkNutX z)_w+XXZSMGdCWP$MO&AV>8ic}d|}%%k}LHh;FPDAiCn}xew2rjn6M59(Xqp{UYN`$wr^Tza_Y}}ucnuli_$?d zwHfU>uPqmqgJeV)EFS$P!yV6%7v(J{4%zGK3K)NR4%+KW7*Ec-fQ!$l_KyLVR4pgr z@|pqX*D;>$n_;PhChi};3vlN}#>wlT_V1R9CMbhEq;B&HQUQC#{%tD=Pw~g}?lUWh z=Y(c@H(>>-F*ghvF59<4L=INl+KSW_gon4a`I#$3;UF#5kh_B9YA;6r>fV_vd&8v54%mx!G&&)n=J^Y+~>&d=t^m-3gpMe0qmViBZ#`W{g_UAEonBop)R z5tTP`lB4BofU{-wuc7kP--_b-GWWMmGWPS|iVQDVc^{M?|2vWHC97Mr#~(!D&C30%-uR=azDY(TpgpJlkD_>%%zhql*Pi=CtCx&_ z32@7)`$WWa8IcOO?eF)AI4`vf-+thJadx(BK7jU`FYgzb)0OK+Cp{n%=BSKJ%7+h# z#MvtF=0CkkG|ZPV#ps_>vPx{7t`=zNovTFWG@0@N+EZVDP#n8iCR74$sC!T(->8=M z+hc7a9C#?}8C7B<-9!(P;17Y!{Vt<@RR*pFL_9b<-sW^PI2kBI_n> zIGvK3^-+vG6#7_TmIT(qUH#;Pw(facs7XDhHgiG4~bGQ*;xxXd;h~C0S@;$;L@`X)54~?3GlI}{KbX&GW-JIj6#2LY_3dh1AM;A zUsQUj>{VgrI*~D3@{n(7Tqg==OP)A#UJMY))6_Po>U4m}!wTaW4ClD;K+!r~TU;Fs z6v;Qqq&;X)|1wZi&sUkR$dvUo@@i7hURSeTluncR>3}nKJ|fOelQ{cD!gP$ z4&cg*kBAG}BCYu4Akj8kO*+SFg2eWlRMxCGHdr*yl9lhFe`ZOrX!1fL8gOD~u!xzV zwqj{-gwP6!L(u+DLqzQywJHkV6DrcYWNZWaro-dpC0Nl%Eq(^WW4djAP=W{!-gMSFAPpG5)e^*P|oxBo2K=gXY00H=2SS;Wqf z6&C^L9SU=%LB%s=Ml&i2H_GTAH~^27dFR8OwPbwVlUtlcq&gvKdLA(H&#jYT{9M4u zY3JaKWZwsXkpoY}A<5Zhz?tJ@I!;LEGm3z!BqA{x|gmDskgv^==I5 zqjjj%Um?R^LuFqVD!ItwzlBQWb5A-`ZaR(5N^%dPk~LnIeTquc2dLmAq~>p^Y;8pa z#}*x1F&HkyZFLqkW8_gR*)Z=Tar!M9)WJed%v0b^oh&q@0oG(mI{{deC8-v$CQJGE zfI**R$=n_3Oh*wUOH>wM^x-p~{8OkP`OdOz{}L5s+*y|7E!&($v!{|J_IXtBgk?!T zq@J)WiKkHkHkPIAJSs@{vn+*?+nq(7G-WB?kA^T#!xU!&L~fc+t}0IeMh=?n(pCW& zd2O;wW+Pza!O1Rh9f0vF$r2Z}!xF$=I%*jQ)jPuwLCjlZyoIVM0 zqyjKyvVVzjiy4l(oDIPE`I~Bw*ad@Uhd&Lw441LFsKh;k3bM`hl+8}qhYGUe`@<XvwPMD(`++f zWRdqG(VVkur?V))yOe2eKa5KGWEo4ll;TfNiLuJ&B2+4$jKLEmq(4C=H4T+xR@p(> z=7w@q@{w+)^<7FEDrcQ#IA$DTvg2FSvpJ`m@zFDo%$rkfS%+ic#oo+h)t+6>qT>NI z+&TasTg6PqR>gt&2bjr%+-K1T?-G+o_ryc@s}z$d-=h+4Q%t79(jpcKU}7@waDuZq z?-G*{QG1|bcg1A#K2(a`6_e+)QK><~m6$9kMg{M% zCX>&pcbLh-i>M@^Z>cZp>tQI^Y^tW^(&+z@E;w5$~p%`*~FIToseCnW%ssGnxMmDo9f@lSeC1 zNzs`+D)TB)UrkHheSZOj)EJUvkyQ+4Axm~aflriC|nh8=2RD8xDF>tJ&L1G}~a|Vfl z=mrLvfi?v*12tc8J2Oz!$S^aI@g>8|KztL!ih(5A`~#|IIAM{$c`sbkg#$sslYmo^ zH$xz3+6Fkyg#$rcG~gl^$)u}dQ9%Zk13~6)RFLH4K+w4dmGc8sAZUCMl_VD}5G2X- zd8o%uBCU7T01UsLNrr8I0p5U|7D(1)18%vNNtV|DuAiz%WrSFlbe$rZ+JcHfvJ*%B-=P7!dC+X_1c(iJDRxh}Vad+4neTGP zQeg{VBu!ezW0XDx7=0qHh1+chj7@&t48*Qzz(7;5085u=RG;n0X68Jr;LMfC&m^e+ z_1O4iB>~1JzX98w$`@3hr1e;dAG(~=h%Fz?3fPwTHRGN#` zJ4v+jNG?SMqQ|$O()d0qNG_b|L?!J5^@QiJ<_c7Rnn!YEm3l%84Ow~)mE?Z1Ihj`- zqh!LTXi({frbd9+{!_l8>H=VFt7)n$ZwHKYLGw&(@4{XN14uc*_2MG|=M9oc$T^hn z1dJ_t3HFOIy8#1F*b@-)1;E%o=f;4?(iBvHD0V+8Wf`b|aQHz~_T`{LyL+0DwjV=< zw();KrTK(cJ?BBjEzKSTvP_NJYXE9x97aW1mvMV4!h-1jLOI=slI5_hFw z35mT7mXpz?fUzef|7zLt3fkxK_PpUZ;PW@}{`^cUwcp4a^rGZcnBr!cXF;I)3*g52 zyn3mPe-+%%l5tbuM`eJK{p5K$EDid_`Wd=P%K}_6O?Dtb8TJ|A*x9oFR_wj^yoULD znk>LZz2!Z?Ei*Z=*N3M=S6;FaTlLIrz%jFB!#xP1=K=4Z&w8#)eI0tfNglOfX@3rI z`z-d~#Ah?W!wlAcGcu7P545mqO5OYhR%TOW5q9j`GXZ1Y$7_Sn0M}w~vKg9;e>M{X zXSR&kic0M%RAO$FwL4I`(1}X?Y-PT!2M<8U^JN^}P)WRqO3ECWorFq$S{7@74I5XD z3i8^Cui#1P&NqR2nk-32B|aaO3~ZWc^IrHjREn{C&p{Y%gOK3%5Jxm)L9RKmPuP6<_-P=WZoxNF^?!!q;u$@mzR;u*5G3fl&c*J&8_BNuD?WKJ4%yUQJH`wRI;Yr>ZRN#y} z8Z&kr#o7ZI58B=59YqBuq@jsh6sSZGR0|n=y`^*1FNUu5d(^%v*k`SMolRJrgv;c9 z9z*sSvCUB8~|{qVvt|5g5e0rr58+MCox0NWM7Ri#Dfdx{i)vbj5&o;}kA=|pKzwl{&&S?(FEHcl7mN$O0U>>N7~sDu z#P(RvXTua*_%)5tpn%o>wl$NTmX0Ob+5_S@IK&nb8mvDcK|m~ct>1cITsWOPY^gu> z*W~jFT({0Ir00Xn{Z|Fqee8bgyFDQi!xbj|H+6L)^hhJJF3{KR*bF5T5!@NnTIJsK zL7r2viL>Iu-dN${dGlbGfu4@m5iYWA^8k5%FV<>@hEg}{;ISTq^}j;wFCun$Z6$8}-%E$GkF}(L38Lj_9ka z75dm0HT1YnVxo5#s~YB1U_Se_ixu5;%uY^2G{&*xGZ`=XRAIjGiiR0?E=yr_o> zr$Et(UjcO(R_h0W8e42Dj{CczHY1FNI(e9^ccpIDM06$8D?FU<>5e=A2zMjBS>omV~d?-cCE7& zU9B0_OUP2azVPhf_|57bbg|ySe*Om#&KxG|3PHHi;$+1EoJrVW3ebqAKh;lW#r30Z z*1Dt1h^YFOs|S5&wg-h@fI#-Pe5?Zp%Dm4(pbG;X$3}_(g%EQx(eTF=J?AX5O1-Gt zd?4@nV|X48e@{XmmRPi42@?Z4;Zo+F2c5dLSUaC7?nl#(b+A14{AOo&jABetlruGh zHRRJ*5TKu$)=xHM4wjLz1Gt~H2=5_Y>%TeVF&N8h{s(*Z(b_|L3%H6(0apn)Z~yQZ za(@Gytd!?8>35r`n0(RzS^0H;?q)4}1AYI;Qx%aSj%O0e%0+vlWp)ny$!Ke4>%3bfP(odQMlwLBgZ5bfcB}Sql!K^CeGLuAq1J z4bW?FMoJR&;1yQNC7Qt*)^?b$h+0j=aCyE6>PA2EHlC<@PgwY~xH9hmaF_Ixbq&Cc z^Ajtj08QL|%veOA8U(_&UdEku0{HD1o&y}FEFut*tq3GE=>(9zGYNoseSN99D&cN! zb+&e(pN_kaNsDloSChpHEj{vgu@DL#6=A)<;l$G z0sD5Fw+MUUKv{N~mHJuhK0;>?=jHex#uI<`7(ujutvdo5;r(P@D`+Ibrj!cnMx)yV zX40U3*33_t1}3l-M1wy2pi%#pqH*>cjRs}AOh474AT6pj)o)!Tx6ZRtFKZX_Wbjh@ z-GhJm#KVPt zQH3bd@1Uq!9~4X9Rurq-HHwtyGAZ_&&WL2?_Wm;bTt6#ywbmq}KPFHnLcgX$B+XhM zB(vU8BuhFpl9c8$&DMKHBbrh7%b2xR>S?Y07Tst5i!<7MFqUuvgxd$m+)fa#8}4MK z9FIv@vn6e6G+W01NoH<#=5E#tpP(ZZT|yT`Ca_+v~rE(!F1*cM`Q^7s{Y%=kWU$PMd)Echw@VEh>$ zi`P`I=~HJ|uJI1_!|zqEp6%;n_w|}R-RCABuhrhZcAuF(KHf9DX5W1Cj9K0{+opNh dXWD&ddS8qA63>c&Bb=Omy=bM^{{gn`(y0Id literal 0 HcmV?d00001 From 585f6fe2a29a1f69b18806b7ffd3a6f9743604e5 Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Thu, 4 Jan 2024 13:02:02 +0100 Subject: [PATCH 248/350] Upgrade required JVM version to 17.0.8 Previous versions are affected by https://bugs.openjdk.org/browse/JDK-8294677 --- README.md | 2 +- .../src/main/java/io/trino/server/TrinoSystemRequirements.java | 2 +- docs/src/main/sphinx/installation/deployment.md | 2 +- pom.xml | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/README.md b/README.md index 6c2ee3efeda0d..3de72f5f15583 100644 --- a/README.md +++ b/README.md @@ -34,7 +34,7 @@ information about reporting vulnerabilities. ## Build requirements * Mac OS X or Linux -* Java 17.0.5+, 64-bit +* Java 17.0.8+, 64-bit * Docker * Turn SELinux or other systems disabling write access to the local checkout off, to allow containers to mount parts of the Trino source tree diff --git a/core/trino-main/src/main/java/io/trino/server/TrinoSystemRequirements.java b/core/trino-main/src/main/java/io/trino/server/TrinoSystemRequirements.java index 7abb563cc108b..d5ce9b4e969b7 100644 --- a/core/trino-main/src/main/java/io/trino/server/TrinoSystemRequirements.java +++ b/core/trino-main/src/main/java/io/trino/server/TrinoSystemRequirements.java @@ -94,7 +94,7 @@ else if ("Mac OS X".equals(osName)) { private static void verifyJavaVersion() { - Version required = Version.parse("17.0.5"); + Version required = Version.parse("17.0.8"); if (Runtime.version().compareTo(required) < 0) { failRequirement("Trino requires Java %s at minimum (found %s)", required, Runtime.version()); diff --git a/docs/src/main/sphinx/installation/deployment.md b/docs/src/main/sphinx/installation/deployment.md index 22d7b635b36b0..1d5ef0704b9f2 100644 --- a/docs/src/main/sphinx/installation/deployment.md +++ b/docs/src/main/sphinx/installation/deployment.md @@ -35,7 +35,7 @@ ### Java runtime environment -Trino requires a 64-bit version of Java 17, with a minimum required version of 17.0.5. +Trino requires a 64-bit version of Java 17, with a minimum required version of 17.0.8. Earlier major versions such as Java 8 or Java 11 do not work. Newer major versions such as Java 18 or 19, are not supported -- they may work, but are not tested. diff --git a/pom.xml b/pom.xml index cbf17b622194d..c839ebe6d1ac6 100644 --- a/pom.xml +++ b/pom.xml @@ -142,7 +142,7 @@ true true true - 17.0.5 + 17.0.8 -missing ${project.basedir} 8 From b769deb3514673ad5bde5dfff81d19717164ff65 Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Thu, 4 Jan 2024 10:04:25 +0100 Subject: [PATCH 249/350] Update AWS SDK v1 to 1.12.630 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index c839ebe6d1ac6..43790b502678e 100644 --- a/pom.xml +++ b/pom.xml @@ -176,7 +176,7 @@ 4.13.1 14.0.1 1.11.3 - 1.12.618 + 1.12.630 4.17.0 7.5.1 87 From e90d1c34b2a4125a9039f9d5652f11a2b6e67d7b Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Thu, 4 Jan 2024 10:04:57 +0100 Subject: [PATCH 250/350] Update AWS SDK v2 to 2.22.10 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 43790b502678e..f73fd3267aa25 100644 --- a/pom.xml +++ b/pom.xml @@ -295,7 +295,7 @@ software.amazon.awssdk bom - 2.21.45 + 2.22.10 pom import From 2c931e845ec1a2f1ce2cc29285578cb7eb33251d Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Thu, 4 Jan 2024 10:06:50 +0100 Subject: [PATCH 251/350] Update JDBI to 3.43.0 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index f73fd3267aa25..b2cf68b50d0c4 100644 --- a/pom.xml +++ b/pom.xml @@ -271,7 +271,7 @@ org.jdbi jdbi3-bom - 3.42.0 + 3.43.0 pom import From 19e4ec74163ca02b9a4124cf32a5f081b61a05d5 Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Thu, 4 Jan 2024 10:08:30 +0100 Subject: [PATCH 252/350] Update flywaydb to 10.4.1 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index b2cf68b50d0c4..7354d0be33a87 100644 --- a/pom.xml +++ b/pom.xml @@ -183,7 +183,7 @@ 1.21 1.0.8 2.23.0 - 10.3.0 + 10.4.1 1.43.3 1.4.3 5.14.0 From d9eb1483cc98e6ebd2876cbeb09aacba5378d0d0 Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Thu, 4 Jan 2024 10:08:53 +0100 Subject: [PATCH 253/350] Update Jetty to 11.0.19 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 7354d0be33a87..89f12f571b91e 100644 --- a/pom.xml +++ b/pom.xml @@ -263,7 +263,7 @@ org.eclipse.jetty jetty-bom - 11.0.18 + 11.0.19 pom import From 5abec90c2e4116f90c543b8b017ca711e54a549c Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Thu, 4 Jan 2024 10:09:39 +0100 Subject: [PATCH 254/350] Update swagger to 2.2.20 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 89f12f571b91e..7967210a14cc7 100644 --- a/pom.xml +++ b/pom.xml @@ -194,7 +194,7 @@ ${dep.airlift.version} 1.13.1 3.25.1 - 2.2.19 + 2.2.20 2.1.2 2.0.62.Final 201 From a2843df55637ccef814563230af91f6543e49750 Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Thu, 4 Jan 2024 10:11:54 +0100 Subject: [PATCH 255/350] Update datasketches-java to 5.0.1 --- plugin/trino-iceberg/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/plugin/trino-iceberg/pom.xml b/plugin/trino-iceberg/pom.xml index 3facee721c37a..1f5337cd9d5f0 100644 --- a/plugin/trino-iceberg/pom.xml +++ b/plugin/trino-iceberg/pom.xml @@ -180,7 +180,7 @@ org.apache.datasketches datasketches-java - 4.2.0 + 5.0.1 From 0298829f13d2c9c0dd96537761b38b6e0fc705eb Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Thu, 4 Jan 2024 10:17:07 +0100 Subject: [PATCH 256/350] Update keycloak to 23.0.3 --- plugin/trino-iceberg/pom.xml | 2 +- .../io/trino/plugin/iceberg/containers/KeycloakContainer.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/plugin/trino-iceberg/pom.xml b/plugin/trino-iceberg/pom.xml index 1f5337cd9d5f0..a4bff116e72b3 100644 --- a/plugin/trino-iceberg/pom.xml +++ b/plugin/trino-iceberg/pom.xml @@ -578,7 +578,7 @@ org.keycloak keycloak-core - 22.0.5 + 23.0.3 test diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/containers/KeycloakContainer.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/containers/KeycloakContainer.java index a43074b990629..4ba4fa4ea869c 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/containers/KeycloakContainer.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/containers/KeycloakContainer.java @@ -30,7 +30,7 @@ public class KeycloakContainer extends BaseTestContainer { - public static final String DEFAULT_IMAGE = "quay.io/keycloak/keycloak:21.1.2"; + public static final String DEFAULT_IMAGE = "quay.io/keycloak/keycloak:23.0.3"; public static final String DEFAULT_HOST_NAME = "keycloak"; public static final String DEFAULT_USER_NAME = "admin"; From 423731d4bdefd8483ea9545bdd0a3abf47cd0e36 Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Thu, 4 Jan 2024 10:18:19 +0100 Subject: [PATCH 257/350] Update databricks-jdbc to 2.6.36 --- testing/trino-product-tests-launcher/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/testing/trino-product-tests-launcher/pom.xml b/testing/trino-product-tests-launcher/pom.xml index 986f9a209202d..64d87208fcf31 100644 --- a/testing/trino-product-tests-launcher/pom.xml +++ b/testing/trino-product-tests-launcher/pom.xml @@ -149,7 +149,7 @@ com.databricks databricks-jdbc - 2.6.32 + 2.6.36 runtime From afece61c7ba4ef84e9208a93b97415288148055a Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Thu, 4 Jan 2024 10:38:13 +0100 Subject: [PATCH 258/350] Update byte-buddy to 1.14.11 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 7967210a14cc7..d56672dbf6f99 100644 --- a/pom.xml +++ b/pom.xml @@ -1617,7 +1617,7 @@ net.bytebuddy byte-buddy - 1.14.10 + 1.14.11 From c64cd0fdeaa493eb0dd0b9b7f8c7064d35dd4682 Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Thu, 4 Jan 2024 10:40:35 +0100 Subject: [PATCH 259/350] Update javassist to 3.30.2-GA --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index d56672dbf6f99..aed9e211bddc2 100644 --- a/pom.xml +++ b/pom.xml @@ -1958,7 +1958,7 @@ org.javassist javassist - 3.29.2-GA + 3.30.2-GA From f194d726048733218daefa954fbe0dbdbdba6732 Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Thu, 4 Jan 2024 10:44:17 +0100 Subject: [PATCH 260/350] Update MariaDB driver to 3.3.2 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index aed9e211bddc2..dd47137aab17b 100644 --- a/pom.xml +++ b/pom.xml @@ -1988,7 +1988,7 @@ org.mariadb.jdbc mariadb-java-client - 3.3.1 + 3.3.2 From e0625149fd8466d8a9dc589b4714a705277a78db Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Thu, 4 Jan 2024 11:17:43 +0100 Subject: [PATCH 261/350] Update s3mock-testcontainers to 3.3.0 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index dd47137aab17b..4d98eb000e8e0 100644 --- a/pom.xml +++ b/pom.xml @@ -303,7 +303,7 @@ com.adobe.testing s3mock-testcontainers - 3.2.0 + 3.3.0 From 1674a1895c0370dbf25010329751640608fdeebb Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Thu, 4 Jan 2024 11:20:58 +0100 Subject: [PATCH 262/350] Updater oauth2-oidc-sdk to 11.9 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 4d98eb000e8e0..239261c493944 100644 --- a/pom.xml +++ b/pom.xml @@ -592,7 +592,7 @@ com.nimbusds oauth2-oidc-sdk - 11.8 + 11.9 jdk11 From d824960d3a797a1b8c8aa94b53f90fbb3bd4df93 Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Thu, 4 Jan 2024 11:22:16 +0100 Subject: [PATCH 263/350] Update checker-equal to 3.42.0 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 239261c493944..176e613cb0f35 100644 --- a/pom.xml +++ b/pom.xml @@ -1915,7 +1915,7 @@ org.checkerframework checker-qual - 3.41.0 + 3.42.0 From 3332ff6b3dfc93158598ef0e28518940beb33cb9 Mon Sep 17 00:00:00 2001 From: Yuya Ebihara Date: Thu, 4 Jan 2024 21:20:13 +0900 Subject: [PATCH 264/350] Remove redundant ? from SHOW COLUMNS in SqlBase.g4 --- .../src/main/antlr4/io/trino/grammar/sql/SqlBase.g4 | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/trino-grammar/src/main/antlr4/io/trino/grammar/sql/SqlBase.g4 b/core/trino-grammar/src/main/antlr4/io/trino/grammar/sql/SqlBase.g4 index c13530d62a80f..cea6360647f34 100644 --- a/core/trino-grammar/src/main/antlr4/io/trino/grammar/sql/SqlBase.g4 +++ b/core/trino-grammar/src/main/antlr4/io/trino/grammar/sql/SqlBase.g4 @@ -160,7 +160,7 @@ statement (LIKE pattern=string (ESCAPE escape=string)?)? #showSchemas | SHOW CATALOGS (LIKE pattern=string (ESCAPE escape=string)?)? #showCatalogs - | SHOW COLUMNS (FROM | IN) qualifiedName? + | SHOW COLUMNS (FROM | IN) qualifiedName (LIKE pattern=string (ESCAPE escape=string)?)? #showColumns | SHOW STATS FOR qualifiedName #showStats | SHOW STATS FOR '(' rootQuery ')' #showStatsForQuery From d94856da926e5e4937810b7c83c55b218c4f0a09 Mon Sep 17 00:00:00 2001 From: Yuya Ebihara Date: Fri, 5 Jan 2024 13:25:33 +0900 Subject: [PATCH 265/350] Specify table name in testPartitionProjectionIgnore --- .../src/test/java/io/trino/plugin/hive/TestHive3OnDataLake.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHive3OnDataLake.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHive3OnDataLake.java index 24d40204c58f1..91c5a06cffbd1 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHive3OnDataLake.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHive3OnDataLake.java @@ -1548,7 +1548,7 @@ public void testPartitionProjectionIgnore() hiveMinioDataLake.getHiveHadoop().runOnHive( "ALTER TABLE " + hiveTestTableName + " SET TBLPROPERTIES ( 'trino.partition_projection.ignore'='TRUE' )"); // Flush cache to get new definition - computeActual("CALL system.flush_metadata_cache()"); + computeActual("CALL system.flush_metadata_cache(schema_name => '" + HIVE_TEST_SCHEMA + "', table_name => '" + tableName + "')"); // Verify query execution works computeActual(createInsertStatement( From 75854f4d99aaf33f50fdad9797098a8fb1f0701a Mon Sep 17 00:00:00 2001 From: athultr1997 Date: Mon, 1 Jan 2024 14:50:08 +0530 Subject: [PATCH 266/350] Fixes failure when building trino-server alone Added a dummy test in trino-server --- .../test/java/io/trino/server/TestDummy.java | 22 +++++++++++++++++++ 1 file changed, 22 insertions(+) create mode 100644 core/trino-server/src/test/java/io/trino/server/TestDummy.java diff --git a/core/trino-server/src/test/java/io/trino/server/TestDummy.java b/core/trino-server/src/test/java/io/trino/server/TestDummy.java new file mode 100644 index 0000000000000..b560df431cb69 --- /dev/null +++ b/core/trino-server/src/test/java/io/trino/server/TestDummy.java @@ -0,0 +1,22 @@ +/* + * 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.server; + +import org.junit.jupiter.api.Test; + +public class TestDummy +{ + @Test + public void buildRequiresTestToExist() {} +} From 6f9f8cf3a89b33ad4d77433a5a537c7050c086ef Mon Sep 17 00:00:00 2001 From: Manfred Moser Date: Thu, 28 Dec 2023 16:42:41 -0800 Subject: [PATCH 267/350] Increase stale operations count - Currently uses default 30 and thats too low - Also update to latest version of the action - Also configure to only process PRs --- .github/workflows/stale.yml | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/.github/workflows/stale.yml b/.github/workflows/stale.yml index 82cb8ccfa07b9..746d71a35a52f 100644 --- a/.github/workflows/stale.yml +++ b/.github/workflows/stale.yml @@ -12,7 +12,7 @@ jobs: runs-on: ubuntu-latest if: github.repository == 'trinodb/trino' steps: - - uses: actions/stale@v8.0.0 + - uses: actions/stale@v9.0.0 with: stale-pr-message: 'This pull request has gone a while without any activity. Tagging the Trino developer relations team: @bitsondatadev @colebow @mosabua' days-before-pr-stale: 21 @@ -21,3 +21,7 @@ jobs: stale-pr-label: 'stale' start-date: '2023-01-01T00:00:00Z' exempt-draft-pr: true + operations-per-run: 200 + # Avoid processing issues completely, see https://github.com/actions/stale/issues/1112 + days-before-issue-stale: -1 + day-before-issue-closed: -1 From 2e1de3b151e57d01658f7001a700db5fe5979175 Mon Sep 17 00:00:00 2001 From: Jack Klamer Date: Wed, 27 Dec 2023 11:33:33 -0600 Subject: [PATCH 268/350] Allow union data to conform to smaller union HIVE/AVRO: It is possible for data that is written using a 3 element union to be read with a 2 element union provided that either all data types can be coerced (already possible) or the offending data type(s) isn't present. This change delays all type errors to read time to allow more type leniency. --- .../hive/formats/avro/AvroPageDataReader.java | 20 ++++++++++- ...tAvroPageDataReaderWithoutTypeManager.java | 35 +++++++++++++++++++ 2 files changed, 54 insertions(+), 1 deletion(-) diff --git a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/avro/AvroPageDataReader.java b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/avro/AvroPageDataReader.java index 2d3f1bfb4acc1..365f421ed7cc8 100644 --- a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/avro/AvroPageDataReader.java +++ b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/avro/AvroPageDataReader.java @@ -208,11 +208,29 @@ private static BlockBuildingDecoder createBlockBuildingDecoderForAction(Resolver yield new ReaderUnionCoercedIntoRowBlockBuildingDecoder((Resolver.ReaderUnion) action, typeManager); } } - case ERROR -> throw new AvroTypeException("Resolution action returned with error " + action); + case ERROR -> new TypeErrorThrower((Resolver.ErrorAction) action); case SKIP -> throw new IllegalStateException("Skips filtered by row step"); }; } + private static class TypeErrorThrower + extends BlockBuildingDecoder + { + private final Resolver.ErrorAction action; + + public TypeErrorThrower(Resolver.ErrorAction action) + { + this.action = requireNonNull(action, "action is null"); + } + + @Override + protected void decodeIntoBlock(Decoder decoder, BlockBuilder builder) + throws IOException + { + throw new IOException(new AvroTypeException("Resolution action returned with error " + action)); + } + } + // Different plugins may have different Avro Schema to Type mappings // that are currently transforming GenericDatumReader returned objects into their target type during the record reading process // This block building decoder allows plugin writers to port that code directly and use within this reader diff --git a/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/avro/TestAvroPageDataReaderWithoutTypeManager.java b/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/avro/TestAvroPageDataReaderWithoutTypeManager.java index bcc88c9068b9c..2234c0a3bed4a 100644 --- a/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/avro/TestAvroPageDataReaderWithoutTypeManager.java +++ b/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/avro/TestAvroPageDataReaderWithoutTypeManager.java @@ -15,6 +15,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; import io.airlift.slice.Slice; import io.trino.filesystem.TrinoInputFile; import io.trino.spi.Page; @@ -335,4 +336,38 @@ public void testCoercionOfUnionToStruct() assertThat(totalRecords).isEqualTo(3); } } + + @Test + public void testRead3UnionWith2UnionDataWith2Union() + throws IOException, AvroTypeException + { + Schema twoUnion = Schema.createUnion(Schema.create(Schema.Type.NULL), Schema.create(Schema.Type.INT)); + Schema threeUnion = Schema.createUnion(Schema.create(Schema.Type.NULL), Schema.create(Schema.Type.INT), Schema.create(Schema.Type.STRING)); + + Schema twoUnionRecord = SchemaBuilder.builder() + .record("aRecord") + .fields() + .name("aField") + .type(twoUnion) + .noDefault() + .endRecord(); + + Schema threeUnionRecord = SchemaBuilder.builder() + .record("aRecord") + .fields() + .name("aField") + .type(threeUnion) + .noDefault() + .endRecord(); + + // write a file with the 3 union schema, using 2 union data + TrinoInputFile inputFile = createWrittenFileWithData(threeUnionRecord, ImmutableList.copyOf(Iterables.transform(new RandomData(twoUnionRecord, 1000), object -> (GenericRecord) object))); + + //read the file with the 2 union schema and ensure that no error thrown + try (AvroFileReader avroFileReader = new AvroFileReader(inputFile, twoUnionRecord, NoOpAvroTypeManager.INSTANCE)) { + while (avroFileReader.hasNext()) { + assertThat(avroFileReader.next()).isNotNull(); + } + } + } } From b0e06b39de0af62967270559ec6b80406deb1154 Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Fri, 5 Jan 2024 13:40:36 +0100 Subject: [PATCH 269/350] Update oshi-core to 6.4.10 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 176e613cb0f35..571e26add78e3 100644 --- a/pom.xml +++ b/pom.xml @@ -500,7 +500,7 @@ com.github.oshi oshi-core - 6.4.8 + 6.4.10 From 0e168cef8f60d0572a6bcccb25254d04f3d4b1d6 Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Fri, 5 Jan 2024 13:41:29 +0100 Subject: [PATCH 270/350] Update grpc-bom to 1.60.1 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 571e26add78e3..35d6b750bc149 100644 --- a/pom.xml +++ b/pom.xml @@ -247,7 +247,7 @@ io.grpc grpc-bom - 1.60.0 + 1.60.1 pom import From 1c91e5731bdc30b889cd5ed3512f367679317607 Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Mon, 8 Jan 2024 12:33:10 +0100 Subject: [PATCH 271/350] Update joda-time to 2.12.6 --- pom.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/pom.xml b/pom.xml index 35d6b750bc149..bcd35d8606085 100644 --- a/pom.xml +++ b/pom.xml @@ -187,6 +187,7 @@ 1.43.3 1.4.3 5.14.0 + 2.12.6 0.12.3 3.6.1 1.9.20 From 9b646d8a5f0874ab7845264c2cd30ad7bfc04792 Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Mon, 8 Jan 2024 12:39:25 +0100 Subject: [PATCH 272/350] Update arrow to 14.0.2 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index bcd35d8606085..6205fd08a0e5f 100644 --- a/pom.xml +++ b/pom.xml @@ -174,7 +174,7 @@ 1.10.2 239 4.13.1 - 14.0.1 + 14.0.2 1.11.3 1.12.630 4.17.0 From 8fb9730e7e32c5b6103f8035bc928cde024f8ff7 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Mon, 8 Jan 2024 13:28:23 +0100 Subject: [PATCH 273/350] Detect leaked containers when running with JUnit With TestNG, `ManageTestResources` was attempting to prevent resource leaks, including container leaks, in tests. It relied on certain common test patterns to operate (like storing resource on instance fields). This commit attempts to provide similar functionality for JUnit. For now it's limited to containers. As an added bonus, it works regardless of how the test class is written. --- plugin/trino-accumulo/pom.xml | 6 ++ .../accumulo/TestingAccumuloServer.java | 2 + testing/trino-testing-containers/pom.xml | 6 ++ .../junit/ReportLeakedContainers.java | 87 +++++++++++++++++++ ...it.platform.launcher.TestExecutionListener | 1 + 5 files changed, 102 insertions(+) create mode 100644 testing/trino-testing-containers/src/main/java/io/trino/testing/containers/junit/ReportLeakedContainers.java create mode 100644 testing/trino-testing-containers/src/main/resources/META-INF/services/org.junit.platform.launcher.TestExecutionListener diff --git a/plugin/trino-accumulo/pom.xml b/plugin/trino-accumulo/pom.xml index 1f1c1697b4f74..8cf27ac319e85 100644 --- a/plugin/trino-accumulo/pom.xml +++ b/plugin/trino-accumulo/pom.xml @@ -267,6 +267,12 @@ + + io.trino + trino-testing-containers + test + + io.trino trino-testing-services diff --git a/plugin/trino-accumulo/src/test/java/io/trino/plugin/accumulo/TestingAccumuloServer.java b/plugin/trino-accumulo/src/test/java/io/trino/plugin/accumulo/TestingAccumuloServer.java index f8a6049019c48..9854ac0966c0f 100644 --- a/plugin/trino-accumulo/src/test/java/io/trino/plugin/accumulo/TestingAccumuloServer.java +++ b/plugin/trino-accumulo/src/test/java/io/trino/plugin/accumulo/TestingAccumuloServer.java @@ -14,6 +14,7 @@ package io.trino.plugin.accumulo; import io.trino.testing.TestingProperties; +import io.trino.testing.containers.junit.ReportLeakedContainers; import org.apache.accumulo.core.client.AccumuloException; import org.apache.accumulo.core.client.AccumuloSecurityException; import org.apache.accumulo.core.client.Connector; @@ -63,6 +64,7 @@ private TestingAccumuloServer() // TODO Change this class to not be a singleton // https://github.com/trinodb/trino/issues/5842 accumuloContainer.start(); + ReportLeakedContainers.ignoreContainerId(accumuloContainer.getContainerId()); } public String getInstanceName() diff --git a/testing/trino-testing-containers/pom.xml b/testing/trino-testing-containers/pom.xml index 1541f026e5853..76607d3769197 100644 --- a/testing/trino-testing-containers/pom.xml +++ b/testing/trino-testing-containers/pom.xml @@ -61,6 +61,12 @@ trino-testing-services + + org.junit.platform + junit-platform-launcher + true + + org.rnorth.duct-tape duct-tape diff --git a/testing/trino-testing-containers/src/main/java/io/trino/testing/containers/junit/ReportLeakedContainers.java b/testing/trino-testing-containers/src/main/java/io/trino/testing/containers/junit/ReportLeakedContainers.java new file mode 100644 index 0000000000000..e41a54bea4ecc --- /dev/null +++ b/testing/trino-testing-containers/src/main/java/io/trino/testing/containers/junit/ReportLeakedContainers.java @@ -0,0 +1,87 @@ +/* + * 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.testing.containers.junit; + +import com.github.dockerjava.api.DockerClient; +import com.github.dockerjava.api.model.Container; +import io.airlift.log.Logger; +import org.junit.platform.launcher.TestExecutionListener; +import org.junit.platform.launcher.TestPlan; +import org.testcontainers.DockerClientFactory; + +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static com.google.common.collect.ImmutableList.toImmutableList; +import static java.lang.Boolean.getBoolean; +import static java.util.Objects.requireNonNull; +import static java.util.stream.Collectors.joining; + +public final class ReportLeakedContainers +{ + private ReportLeakedContainers() {} + + private static final Logger log = Logger.get(ReportLeakedContainers.class); + private static final boolean DISABLED = getBoolean("ReportLeakedContainers.disabled"); + + private static final Set ignoredIds = Collections.synchronizedSet(new HashSet<>()); + + public static void ignoreContainerId(String containerId) + { + ignoredIds.add(requireNonNull(containerId, "containerId is null")); + } + + // Separate class so that ReportLeakedContainers.ignoreContainerId can be called without pulling junit platform onto classpath + public static class Listener + implements TestExecutionListener + { + @Override + public void testPlanExecutionFinished(TestPlan testPlan) + { + if (DISABLED) { + log.info("ReportLeakedContainers disabled"); + return; + } + log.info("Checking for leaked containers"); + + @SuppressWarnings("resource") // Throws when close is attempted, as this is a global instance. + DockerClient dockerClient = DockerClientFactory.lazyClient(); + + List containers = dockerClient.listContainersCmd() + .withLabelFilter(Map.of(DockerClientFactory.TESTCONTAINERS_SESSION_ID_LABEL, DockerClientFactory.SESSION_ID)) + .exec() + .stream() + .filter(container -> !ignoredIds.contains(container.getId())) + .collect(toImmutableList()); + + if (!containers.isEmpty()) { + log.error("Leaked containers: %s", containers.stream() + .map(container -> toStringHelper("container") + .add("id", container.getId()) + .add("image", container.getImage()) + .add("imageId", container.getImageId()) + .toString()) + .collect(joining(", ", "[", "]"))); + + // JUnit does not fail on a listener exception. + System.err.println("JVM will be terminated"); + System.exit(1); + } + } + } +} diff --git a/testing/trino-testing-containers/src/main/resources/META-INF/services/org.junit.platform.launcher.TestExecutionListener b/testing/trino-testing-containers/src/main/resources/META-INF/services/org.junit.platform.launcher.TestExecutionListener new file mode 100644 index 0000000000000..c80b71364750e --- /dev/null +++ b/testing/trino-testing-containers/src/main/resources/META-INF/services/org.junit.platform.launcher.TestExecutionListener @@ -0,0 +1 @@ +io.trino.testing.containers.junit.ReportLeakedContainers$Listener From 2be084caf148281ba5b07de8f169667df6e015cc Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Mon, 27 Nov 2023 19:26:03 -0800 Subject: [PATCH 274/350] Return empty table list in Iceberg REST Catalog when invalid namespace is provided --- .../catalog/rest/TrinoRestCatalog.java | 22 +++++----- .../iceberg/catalog/BaseTrinoCatalogTest.java | 41 +++++++++++++++++++ 2 files changed, 53 insertions(+), 10 deletions(-) diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/rest/TrinoRestCatalog.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/rest/TrinoRestCatalog.java index 29f6ef8e24431..71cd72315e401 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/rest/TrinoRestCatalog.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/rest/TrinoRestCatalog.java @@ -183,16 +183,7 @@ public void renameNamespace(ConnectorSession session, String source, String targ public List listTables(ConnectorSession session, Optional namespace) { SessionContext sessionContext = convert(session); - List namespaces; - - if (namespace.isPresent() && namespaceExists(session, namespace.get())) { - namespaces = ImmutableList.of(Namespace.of(namespace.get())); - } - else { - namespaces = listNamespaces(session).stream() - .map(Namespace::of) - .collect(toImmutableList()); - } + List namespaces = listNamespaces(session, namespace); ImmutableList.Builder tables = ImmutableList.builder(); for (Namespace restNamespace : namespaces) { @@ -555,4 +546,15 @@ private static TableIdentifier toIdentifier(SchemaTableName schemaTableName) { return TableIdentifier.of(schemaTableName.getSchemaName(), schemaTableName.getTableName()); } + + private List listNamespaces(ConnectorSession session, Optional namespace) + { + if (namespace.isEmpty()) { + return listNamespaces(session).stream() + .map(Namespace::of) + .collect(toImmutableList()); + } + + return ImmutableList.of(Namespace.of(namespace.get())); + } } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/BaseTrinoCatalogTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/BaseTrinoCatalogTest.java index cd896dcd9895f..69830786be64e 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/BaseTrinoCatalogTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/BaseTrinoCatalogTest.java @@ -385,6 +385,47 @@ public void testView() } } + @Test + public void testListTables() + throws Exception + { + TrinoCatalog catalog = createTrinoCatalog(false); + TrinoPrincipal principal = new TrinoPrincipal(PrincipalType.USER, SESSION.getUser()); + String ns1 = "ns1"; + String ns2 = "ns2"; + + catalog.createNamespace(SESSION, ns1, defaultNamespaceProperties(ns1), principal); + catalog.createNamespace(SESSION, ns2, defaultNamespaceProperties(ns2), principal); + SchemaTableName table1 = new SchemaTableName(ns1, "t1"); + SchemaTableName table2 = new SchemaTableName(ns2, "t2"); + catalog.newCreateTableTransaction( + SESSION, + table1, + new Schema(Types.NestedField.of(1, true, "col1", Types.LongType.get())), + PartitionSpec.unpartitioned(), + SortOrder.unsorted(), + arbitraryTableLocation(catalog, SESSION, table1), + ImmutableMap.of()) + .commitTransaction(); + + catalog.newCreateTableTransaction( + SESSION, + table2, + new Schema(Types.NestedField.of(1, true, "col1", Types.LongType.get())), + PartitionSpec.unpartitioned(), + SortOrder.unsorted(), + arbitraryTableLocation(catalog, SESSION, table2), + ImmutableMap.of()) + .commitTransaction(); + + // No namespace provided, all tables across all namespaces should be returned + assertThat(catalog.listTables(SESSION, Optional.empty())).containsAll(ImmutableList.of(table1, table2)); + // Namespace is provided and exists + assertThat(catalog.listTables(SESSION, Optional.of(ns1))).isEqualTo(ImmutableList.of(table1)); + // Namespace is provided and does not exist + assertThat(catalog.listTables(SESSION, Optional.of("non_existing"))).isEmpty(); + } + private String arbitraryTableLocation(TrinoCatalog catalog, ConnectorSession session, SchemaTableName schemaTableName) throws Exception { From 5c2b05910cdad7862fd50929320c9d7b7ed60c24 Mon Sep 17 00:00:00 2001 From: Manfred Moser Date: Mon, 8 Jan 2024 10:40:38 -0800 Subject: [PATCH 275/350] Fix typo in stalebot config --- .github/workflows/stale.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/stale.yml b/.github/workflows/stale.yml index 746d71a35a52f..a137d9ee2ba21 100644 --- a/.github/workflows/stale.yml +++ b/.github/workflows/stale.yml @@ -24,4 +24,4 @@ jobs: operations-per-run: 200 # Avoid processing issues completely, see https://github.com/actions/stale/issues/1112 days-before-issue-stale: -1 - day-before-issue-closed: -1 + days-before-issue-close: -1 From 09ef5b5aa240a3768cc1cb2576e9d06a23cbd7a1 Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Mon, 4 Dec 2023 10:51:21 +0100 Subject: [PATCH 276/350] Require JDK 21 to run Trino --- .github/workflows/ci.yml | 1 - README.md | 6 +-- .../DecoratingListeningExecutorService.java | 37 +------------------ .../trino/server/TrinoSystemRequirements.java | 2 +- .../java/io/trino/server/TrinoServer.java | 4 +- core/trino-server-rpm/src/main/rpm/preinstall | 8 +--- .../java/io/trino/server/rpm/ServerIT.java | 2 - docs/src/main/sphinx/functions/conversion.md | 2 +- docs/src/main/sphinx/functions/regexp.md | 6 +-- .../main/sphinx/installation/deployment.md | 10 ++--- docs/src/main/sphinx/security/tls.md | 6 +-- pom.xml | 4 +- 12 files changed, 22 insertions(+), 66 deletions(-) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index e7faf76eddf1d..29644bba1f268 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -57,7 +57,6 @@ jobs: fail-fast: false matrix: java-version: - - 17 # Keep testing on JDK 17 to ensure basic backward compatibility - 21 timeout-minutes: 45 steps: diff --git a/README.md b/README.md index 3de72f5f15583..e93d0c57c9e7a 100644 --- a/README.md +++ b/README.md @@ -34,7 +34,7 @@ information about reporting vulnerabilities. ## Build requirements * Mac OS X or Linux -* Java 17.0.8+, 64-bit +* Java 21.0.1+, 64-bit * Docker * Turn SELinux or other systems disabling write access to the local checkout off, to allow containers to mount parts of the Trino source tree @@ -70,8 +70,8 @@ After opening the project in IntelliJ, double check that the Java SDK is properly configured for the project: * Open the File menu and select Project Structure -* In the SDKs section, ensure that JDK 17 is selected (create one if none exist) -* In the Project section, ensure the Project language level is set to 17 +* In the SDKs section, ensure that JDK 21 is selected (create one if none exist) +* In the Project section, ensure the Project language level is set to 21 ### Running a testing server diff --git a/core/trino-main/src/main/java/io/trino/dispatcher/DecoratingListeningExecutorService.java b/core/trino-main/src/main/java/io/trino/dispatcher/DecoratingListeningExecutorService.java index e9ef1b4fe9f2c..b53b4d9d43104 100644 --- a/core/trino-main/src/main/java/io/trino/dispatcher/DecoratingListeningExecutorService.java +++ b/core/trino-main/src/main/java/io/trino/dispatcher/DecoratingListeningExecutorService.java @@ -16,45 +16,23 @@ import com.google.common.util.concurrent.ForwardingListeningExecutorService; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; -import jakarta.annotation.Nullable; -import java.lang.invoke.MethodHandle; -import java.lang.reflect.Method; import java.time.Duration; import java.util.Collection; import java.util.List; import java.util.concurrent.Callable; import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import static com.google.common.base.Throwables.throwIfUnchecked; import static com.google.common.collect.ImmutableList.toImmutableList; -import static io.trino.util.Reflection.methodHandle; import static java.util.Objects.requireNonNull; public class DecoratingListeningExecutorService extends ForwardingListeningExecutorService implements ListeningExecutorService { - // TODO remove after requiring Java 19+ for runtime. - private static final @Nullable MethodHandle CLOSE_METHOD; - - static { - Method closeMethod; - try { - closeMethod = ExecutorService.class.getMethod("close"); - } - catch (NoSuchMethodException e) { - closeMethod = null; - } - CLOSE_METHOD = closeMethod != null - ? methodHandle(closeMethod) - : null; - } - private final ListeningExecutorService delegate; private final TaskDecorator decorator; @@ -194,21 +172,10 @@ public boolean awaitTermination(Duration duration) return super.awaitTermination(duration); } - // TODO This is temporary, until Guava's ForwardingExecutorService has the method in their interface. See https://github.com/google/guava/issues/6296 - //@Override + @Override public void close() { - if (CLOSE_METHOD == null) { - throw new UnsupportedOperationException("ExecutorService.close has close() method since Java 19. " + - "The DecoratingListeningExecutorService supports the method only when run with Java 19 runtime."); - } - try { - CLOSE_METHOD.invoke(delegate()); - } - catch (Throwable e) { - throwIfUnchecked(e); - throw new RuntimeException(e); - } + delegate.close(); } public interface TaskDecorator diff --git a/core/trino-main/src/main/java/io/trino/server/TrinoSystemRequirements.java b/core/trino-main/src/main/java/io/trino/server/TrinoSystemRequirements.java index d5ce9b4e969b7..d3ec3cba17926 100644 --- a/core/trino-main/src/main/java/io/trino/server/TrinoSystemRequirements.java +++ b/core/trino-main/src/main/java/io/trino/server/TrinoSystemRequirements.java @@ -94,7 +94,7 @@ else if ("Mac OS X".equals(osName)) { private static void verifyJavaVersion() { - Version required = Version.parse("17.0.8"); + Version required = Version.parse("21.0.1"); if (Runtime.version().compareTo(required) < 0) { failRequirement("Trino requires Java %s at minimum (found %s)", required, Runtime.version()); diff --git a/core/trino-server-main/src/main/java/io/trino/server/TrinoServer.java b/core/trino-server-main/src/main/java/io/trino/server/TrinoServer.java index 8cdd002f28666..b53af87c41ed5 100644 --- a/core/trino-server-main/src/main/java/io/trino/server/TrinoServer.java +++ b/core/trino-server-main/src/main/java/io/trino/server/TrinoServer.java @@ -29,8 +29,8 @@ public static void main(String[] args) String javaVersion = nullToEmpty(StandardSystemProperty.JAVA_VERSION.value()); String majorVersion = javaVersion.split("\\D", 2)[0]; Integer major = Ints.tryParse(majorVersion); - if (major == null || major < 17) { - System.err.println(format("ERROR: Trino requires Java 17+ (found %s)", javaVersion)); + if (major == null || major < 21) { + System.err.println(format("ERROR: Trino requires Java 21+ (found %s)", javaVersion)); System.exit(100); } diff --git a/core/trino-server-rpm/src/main/rpm/preinstall b/core/trino-server-rpm/src/main/rpm/preinstall index 0bf1e4a8fae4a..dbbf425b2c6cb 100644 --- a/core/trino-server-rpm/src/main/rpm/preinstall +++ b/core/trino-server-rpm/src/main/rpm/preinstall @@ -22,7 +22,7 @@ check_if_correct_java_version() { # candidate for JAVA_HOME). JAVA_VERSION=$(java_version "$1") JAVA_MAJOR=$(echo "$JAVA_VERSION" | cut -d'.' -f1) - if [ "$JAVA_MAJOR" -ge "17" ]; then + if [ "$JAVA_MAJOR" -ge "21" ]; then echo "$1" >/tmp/trino-rpm-install-java-home return 0 else @@ -34,10 +34,6 @@ check_if_correct_java_version() { if ! check_if_correct_java_version "$JAVA_HOME"; then java_found=false for candidate in \ - /usr/lib/jvm/java-17-* \ - /usr/lib/jvm/zulu-17 \ - /usr/lib/jvm/temurin-17 \ - /usr/lib/jvm/temurin-17-* \ /usr/lib/jvm/java-21-* \ /usr/lib/jvm/zulu-21 \ /usr/lib/jvm/temurin-21 \ @@ -61,7 +57,7 @@ if [ "$java_found" = false ]; then +======================================================================+ | Error: Required Java version could not be found | +----------------------------------------------------------------------+ -| JDK 17 was not detected. | +| JDK 21 was not detected. | | Recommended JDK distribution is Eclipse Temurin. | | Installation guide: https://adoptium.net/installation/linux/ | | | diff --git a/core/trino-server-rpm/src/test/java/io/trino/server/rpm/ServerIT.java b/core/trino-server-rpm/src/test/java/io/trino/server/rpm/ServerIT.java index 37b6dbb0b1f24..2b52768005263 100644 --- a/core/trino-server-rpm/src/test/java/io/trino/server/rpm/ServerIT.java +++ b/core/trino-server-rpm/src/test/java/io/trino/server/rpm/ServerIT.java @@ -56,7 +56,6 @@ public ServerIT() @Test public void testInstall() { - testInstall("17"); testInstall("21"); } @@ -107,7 +106,6 @@ private void testInstall(String javaVersion) public void testUninstall() throws Exception { - testUninstall("17"); testUninstall("21"); } diff --git a/docs/src/main/sphinx/functions/conversion.md b/docs/src/main/sphinx/functions/conversion.md index 7de546ab7ed8b..8fb09e16c005b 100644 --- a/docs/src/main/sphinx/functions/conversion.md +++ b/docs/src/main/sphinx/functions/conversion.md @@ -22,7 +22,7 @@ Like {func}`cast`, but returns null if the cast fails. ## Formatting :::{function} format(format, args...) -> varchar -Returns a formatted string using the specified [format string](https://docs.oracle.com/en/java/javase/17/docs/api/java.base/java/util/Formatter.html#syntax) +Returns a formatted string using the specified [format string](https://docs.oracle.com/en/java/javase/21/docs/api/java.base/java/util/Formatter.html#syntax) and arguments: ``` diff --git a/docs/src/main/sphinx/functions/regexp.md b/docs/src/main/sphinx/functions/regexp.md index cbc853e0ecc7c..e78939d96eb43 100644 --- a/docs/src/main/sphinx/functions/regexp.md +++ b/docs/src/main/sphinx/functions/regexp.md @@ -184,6 +184,6 @@ SELECT regexp_split('1a 2b 14m', '\s*[a-z]+\s*'); -- [1, 2, 14, ] ``` ::: -[capturing group number]: https://docs.oracle.com/en/java/javase/17/docs/api/java.base/java/util/regex/Pattern.html#gnumber -[capturing groups]: https://docs.oracle.com/en/java/javase/17/docs/api/java.base/java/util/regex/Pattern.html#cg -[java pattern]: https://docs.oracle.com/en/java/javase/17/docs/api/java.base/java/util/regex/Pattern.html +[capturing group number]: https://docs.oracle.com/en/java/javase/21/docs/api/java.base/java/util/regex/Pattern.html#gnumber +[capturing groups]: https://docs.oracle.com/en/java/javase/21/docs/api/java.base/java/util/regex/Pattern.html#cg +[java pattern]: https://docs.oracle.com/en/java/javase/21/docs/api/java.base/java/util/regex/Pattern.html diff --git a/docs/src/main/sphinx/installation/deployment.md b/docs/src/main/sphinx/installation/deployment.md index 1d5ef0704b9f2..b3139babc7d5d 100644 --- a/docs/src/main/sphinx/installation/deployment.md +++ b/docs/src/main/sphinx/installation/deployment.md @@ -35,19 +35,15 @@ ### Java runtime environment -Trino requires a 64-bit version of Java 17, with a minimum required version of 17.0.8. -Earlier major versions such as Java 8 or Java 11 do not work. -Newer major versions such as Java 18 or 19, are not supported -- they may work, but are not tested. +Trino requires a 64-bit version of Java 21, with a minimum required version of 21.0.1. +Earlier major versions such as Java 8, Java 11 or Java 17 do not work. +Newer major versions such as Java 22 are not supported -- they may work, but are not tested. We recommend using the Eclipse Temurin OpenJDK distribution from [Adoptium](https://adoptium.net/) as the JDK for Trino, as Trino is tested against that distribution. Eclipse Temurin is also the JDK used by the [Trino Docker image](https://hub.docker.com/r/trinodb/trino). -If you are using Java 17 or 18, the JVM must be configured to use UTF-8 as the default charset by -adding `-Dfile.encoding=UTF-8` to `etc/jvm.config`. Starting with Java 19, the Java default -charset is UTF-8, so this configuration is not needed. - (requirements-python)= ### Python diff --git a/docs/src/main/sphinx/security/tls.md b/docs/src/main/sphinx/security/tls.md index 775a20f4e61b5..f46caa1fcdc3a 100644 --- a/docs/src/main/sphinx/security/tls.md +++ b/docs/src/main/sphinx/security/tls.md @@ -26,8 +26,8 @@ using TLS 1.2 and TLS 1.3 certificates. The server rejects TLS 1.1, TLS 1.0, and all SSL format certificates. The Trino server does not specify a set of supported ciphers, instead deferring -to the defaults set by the JVM version in use. The documentation for Java 17 -lists its [supported cipher suites](https://docs.oracle.com/en/java/javase/17/security/oracle-providers.html#GUID-7093246A-31A3-4304-AC5F-5FB6400405E2__SUNJSSE_CIPHER_SUITES). +to the defaults set by the JVM version in use. The documentation for Java 21 +lists its [supported cipher suites](https://docs.oracle.com/en/java/javase/21/security/oracle-providers.html#GUID-7093246A-31A3-4304-AC5F-5FB6400405E2__SUNJSSE_CIPHER_SUITES). Run the following two-line code on the same JVM from the same vendor as configured on the coordinator to determine that JVM's default cipher list. @@ -56,7 +56,7 @@ considered in conjunction with your organization's security managers. Using a different suite may require downloading and installing a different SunJCE implementation package. Some locales may have export restrictions on cipher suites. See the discussion in Java documentation that begins with [Customizing -the Encryption Algorithm Providers](https://docs.oracle.com/en/java/javase/17/security/java-secure-socket-extension-jsse-reference-guide.html#GUID-316FB978-7588-442E-B829-B4973DB3B584). +the Encryption Algorithm Providers](https://docs.oracle.com/en/java/javase/21/security/java-secure-socket-extension-jsse-reference-guide.html#GUID-316FB978-7588-442E-B829-B4973DB3B584). :::{note} If you manage the coordinator's direct TLS implementatation, monitor the CPU diff --git a/pom.xml b/pom.xml index 6205fd08a0e5f..43d1eec7f681a 100644 --- a/pom.xml +++ b/pom.xml @@ -137,12 +137,12 @@ - 17 + 21 true true true - 17.0.8 + 21.0.1 -missing ${project.basedir} 8 From 6021f331b4e50296bfc07aaafe937e05040418b5 Mon Sep 17 00:00:00 2001 From: Raunaq Morarka Date: Mon, 8 Jan 2024 16:16:34 +0530 Subject: [PATCH 277/350] Use isDynamicFilterFunction in ConnectorExpressionTranslator --- .../src/main/java/io/trino/sql/DynamicFilters.java | 6 +++++- .../io/trino/sql/planner/ConnectorExpressionTranslator.java | 4 ++-- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/core/trino-main/src/main/java/io/trino/sql/DynamicFilters.java b/core/trino-main/src/main/java/io/trino/sql/DynamicFilters.java index db7f5c8f94fcb..82bf825822eb4 100644 --- a/core/trino-main/src/main/java/io/trino/sql/DynamicFilters.java +++ b/core/trino-main/src/main/java/io/trino/sql/DynamicFilters.java @@ -193,7 +193,11 @@ public static Optional getDescriptor(Expression expression) private static boolean isDynamicFilterFunction(FunctionCall functionCall) { - CatalogSchemaFunctionName functionName = ResolvedFunction.extractFunctionName(functionCall.getName()); + return isDynamicFilterFunction(ResolvedFunction.extractFunctionName(functionCall.getName())); + } + + public static boolean isDynamicFilterFunction(CatalogSchemaFunctionName functionName) + { return functionName.equals(builtinFunctionName(Function.NAME)) || functionName.equals(builtinFunctionName(NullableFunction.NAME)); } diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/ConnectorExpressionTranslator.java b/core/trino-main/src/main/java/io/trino/sql/planner/ConnectorExpressionTranslator.java index e9d28509754cf..e376e754c8efe 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/ConnectorExpressionTranslator.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/ConnectorExpressionTranslator.java @@ -36,7 +36,6 @@ import io.trino.spi.type.RowType; import io.trino.spi.type.Type; import io.trino.spi.type.VarcharType; -import io.trino.sql.DynamicFilters; import io.trino.sql.PlannerContext; import io.trino.sql.tree.ArithmeticBinaryExpression; import io.trino.sql.tree.ArithmeticUnaryExpression; @@ -104,6 +103,7 @@ import static io.trino.spi.expression.StandardFunctions.SUBTRACT_FUNCTION_NAME; import static io.trino.spi.type.BooleanType.BOOLEAN; import static io.trino.spi.type.VarcharType.createVarcharType; +import static io.trino.sql.DynamicFilters.isDynamicFilterFunction; import static io.trino.sql.ExpressionUtils.combineConjuncts; import static io.trino.sql.ExpressionUtils.extractConjuncts; import static io.trino.sql.ExpressionUtils.isEffectivelyLiteral; @@ -682,7 +682,7 @@ protected Optional visitFunctionCall(FunctionCall node, Voi } CatalogSchemaFunctionName functionName = ResolvedFunction.extractFunctionName(node.getName()); - checkArgument(!builtinFunctionName(DynamicFilters.Function.NAME).equals(functionName), "Dynamic filter has no meaning for a connector, it should not be translated into ConnectorExpression"); + checkArgument(!isDynamicFilterFunction(functionName), "Dynamic filter has no meaning for a connector, it should not be translated into ConnectorExpression"); // literals should be handled by isEffectivelyLiteral case above checkArgument(!builtinFunctionName(LITERAL_FUNCTION_NAME).equals(functionName), "Unexpected literal function"); From bd8821d1c6026d535376817b4fae68fde5706049 Mon Sep 17 00:00:00 2001 From: Will Morrison Date: Mon, 8 Jan 2024 13:12:20 -0500 Subject: [PATCH 278/350] Document encryption of FTE spool --- .../sphinx/admin/fault-tolerant-execution.md | 18 +++++++++++++++++- 1 file changed, 17 insertions(+), 1 deletion(-) diff --git a/docs/src/main/sphinx/admin/fault-tolerant-execution.md b/docs/src/main/sphinx/admin/fault-tolerant-execution.md index 864b8e4425450..a1624221440fc 100644 --- a/docs/src/main/sphinx/admin/fault-tolerant-execution.md +++ b/docs/src/main/sphinx/admin/fault-tolerant-execution.md @@ -80,6 +80,10 @@ execution on a Trino cluster: - Enable compression of spooling data. Setting to `true` is recommended when using an [exchange manager](fte-exchange-manager). - ``false`` +* - `fault-tolerant-execution.exchange-encryption-enabled` + - Enable encryption of spooling data, see [Encryption](fte-encryption) for details. + Setting this property to false is not recommended if Trino processes sensitive data. + - ``true`` ::: (fte-retry-policy)= @@ -145,6 +149,14 @@ with a `TASK` retry policy for large batch queries, separate from another cluster that handles short queries. ::: +(fte-encryption)= +## Encryption + +Trino encrypts data before spooling it to storage. This prevents access to query data +by anyone besides the Trino cluster that wrote it, including administrators of the +storage system. A new encryption key is randomly generated for every query, and keys +are discarded once a query is completed. + ## Advanced configuration You can further configure fault-tolerant execution with the following @@ -449,7 +461,11 @@ the property may be configured for: - AWS S3, GCS * - `exchange.s3.endpoint` - S3 storage endpoint server if using an S3-compatible storage system that - is not AWS. If using AWS S3, this can be ignored. If using GCS, set it + is not AWS. If using AWS S3, this can be ignored unless HTTPS is required + by an AWS bucket policy. If TLS is required, then this property can be + set to an https endpoint such as ``https://s3.us-east-1.amazonaws.com``. + Note that TLS is redundant due to {ref}`automatic encryption `. + If using GCS, set it to `https://storage.googleapis.com`. - - Any S3-compatible storage From 2fd2feeca027a253faaebf8ab347b5fe09e9056d Mon Sep 17 00:00:00 2001 From: Raunaq Morarka Date: Tue, 9 Jan 2024 10:58:33 +0530 Subject: [PATCH 279/350] Cache result of evaluating constraint per partition in iceberg --- .../plugin/iceberg/IcebergSplitSource.java | 52 ++++++++++++++----- .../iceberg/BaseIcebergConnectorTest.java | 12 +++++ 2 files changed, 51 insertions(+), 13 deletions(-) diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplitSource.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplitSource.java index 1adab5feb5e76..5212772b99e90 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplitSource.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplitSource.java @@ -15,11 +15,15 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Stopwatch; +import com.google.common.cache.CacheBuilder; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Maps; import com.google.common.io.Closer; import io.airlift.units.DataSize; import io.airlift.units.Duration; +import io.trino.cache.NonEvictableCache; import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.filesystem.TrinoInputFile; @@ -61,6 +65,7 @@ import java.util.OptionalLong; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.function.Predicate; import java.util.function.Supplier; import static com.google.common.base.Preconditions.checkArgument; @@ -72,6 +77,8 @@ import static com.google.common.collect.Sets.intersection; import static com.google.common.math.LongMath.saturatedAdd; import static io.airlift.slice.Slices.utf8Slice; +import static io.trino.cache.CacheUtils.uncheckedCacheGet; +import static io.trino.cache.SafeCaches.buildNonEvictableCache; import static io.trino.plugin.iceberg.ExpressionConverter.isConvertableToIcebergExpression; import static io.trino.plugin.iceberg.ExpressionConverter.toIcebergExpression; import static io.trino.plugin.iceberg.IcebergColumnHandle.fileModifiedTimeColumnHandle; @@ -109,7 +116,7 @@ public class IcebergSplitSource private final DynamicFilter dynamicFilter; private final long dynamicFilteringWaitTimeoutMillis; private final Stopwatch dynamicFilterWaitStopwatch; - private final Constraint constraint; + private final PartitionConstraintMatcher partitionConstraintMatcher; private final TypeManager typeManager; private final Closer closer = Closer.create(); private final double minimumAssignedSplitWeight; @@ -151,7 +158,7 @@ public IcebergSplitSource( this.dynamicFilter = requireNonNull(dynamicFilter, "dynamicFilter is null"); this.dynamicFilteringWaitTimeoutMillis = dynamicFilteringWaitTimeout.toMillis(); this.dynamicFilterWaitStopwatch = Stopwatch.createStarted(); - this.constraint = requireNonNull(constraint, "constraint is null"); + this.partitionConstraintMatcher = new PartitionConstraintMatcher(constraint); this.typeManager = requireNonNull(typeManager, "typeManager is null"); this.recordScannedFiles = recordScannedFiles; this.minimumAssignedSplitWeight = minimumAssignedSplitWeight; @@ -309,7 +316,7 @@ private boolean pruneFileScanTask(FileScanTask fileScanTask, boolean fileHasNoDe } } - return !partitionMatchesConstraint(identityPartitionColumns, partitionValues, constraint); + return !partitionConstraintMatcher.matches(identityPartitionColumns, partitionValues); } private boolean noDataColumnsProjected(FileScanTask fileScanTask) @@ -436,19 +443,38 @@ else if (upperBound != null) { return Domain.create(ValueSet.ofRanges(statisticsRange), mayContainNulls); } - static boolean partitionMatchesConstraint( - Set identityPartitionColumns, - Supplier> partitionValues, - Constraint constraint) + private static class PartitionConstraintMatcher { - // We use Constraint just to pass functional predicate here from DistributedExecutionPlanner - verify(constraint.getSummary().isAll()); + private final NonEvictableCache, Boolean> partitionConstraintResults; + private final Optional>> predicate; + private final Optional> predicateColumns; + + private PartitionConstraintMatcher(Constraint constraint) + { + // We use Constraint just to pass functional predicate here from DistributedExecutionPlanner + verify(constraint.getSummary().isAll()); + this.predicate = constraint.predicate(); + this.predicateColumns = constraint.getPredicateColumns(); + this.partitionConstraintResults = buildNonEvictableCache(CacheBuilder.newBuilder().maximumSize(1000)); + } - if (constraint.predicate().isEmpty() || - intersection(constraint.getPredicateColumns().orElseThrow(), identityPartitionColumns).isEmpty()) { - return true; + boolean matches( + Set identityPartitionColumns, + Supplier> partitionValuesSupplier) + { + if (predicate.isEmpty()) { + return true; + } + Set predicatePartitionColumns = intersection(predicateColumns.orElseThrow(), identityPartitionColumns); + if (predicatePartitionColumns.isEmpty()) { + return true; + } + Map partitionValues = partitionValuesSupplier.get(); + return uncheckedCacheGet( + partitionConstraintResults, + ImmutableMap.copyOf(Maps.filterKeys(partitionValues, predicatePartitionColumns::contains)), + () -> predicate.orElseThrow().test(partitionValues)); } - return constraint.predicate().get().test(partitionValues.get()); } @VisibleForTesting diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java index ba732ec9dacf0..4570584a180fe 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java @@ -4236,6 +4236,18 @@ public void testSplitPruningForFilterOnPartitionColumn() verifySplitCount("SELECT * FROM " + tableName + " WHERE regionkey % 5 = 3", 1); assertUpdate("DROP TABLE " + tableName); + + // Partition by multiple columns + assertUpdate(noRedistributeWrites, "CREATE TABLE " + tableName + " WITH (partitioning = ARRAY['regionkey', 'nationkey']) AS SELECT * FROM nation", 25); + // Create 2 files per partition + assertUpdate(noRedistributeWrites, "INSERT INTO " + tableName + " SELECT * FROM nation", 25); + // sanity check that table contains exactly 50 files + assertThat(computeScalar("SELECT count(*) FROM \"" + tableName + "$files\"")).isEqualTo(50L); + + verifySplitCount("SELECT * FROM " + tableName + " WHERE regionkey % 5 = 3", 10); + verifySplitCount("SELECT * FROM " + tableName + " WHERE (regionkey * 2) - nationkey = 0", 6); + + assertUpdate("DROP TABLE " + tableName); } @Test From 31e8e36250a2f8f438d325428f6d88b6832ebf0f Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 3 Jan 2024 09:44:04 +0100 Subject: [PATCH 280/350] Reduce test boilerplate --- .../plugin/iceberg/TestIcebergTableName.java | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergTableName.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergTableName.java index 5758307909a38..a2cc507b94406 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergTableName.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergTableName.java @@ -31,15 +31,15 @@ public void testParse() assertParseNameAndType("abc$snapshots", "abc", TableType.SNAPSHOTS); assertNoValidTableType("abc$data"); - assertInvalid("abc@123", "Invalid Iceberg table name: abc@123"); - assertInvalid("abc@xyz", "Invalid Iceberg table name: abc@xyz"); + assertInvalid("abc@123"); + assertInvalid("abc@xyz"); assertNoValidTableType("abc$what"); - assertInvalid("abc@123$data@456", "Invalid Iceberg table name: abc@123$data@456"); - assertInvalid("abc@123$snapshots", "Invalid Iceberg table name: abc@123$snapshots"); - assertInvalid("abc$snapshots@456", "Invalid Iceberg table name: abc$snapshots@456"); - assertInvalid("xyz$data@456", "Invalid Iceberg table name: xyz$data@456"); - assertInvalid("abc$partitions@456", "Invalid Iceberg table name: abc$partitions@456"); - assertInvalid("abc$manifests@456", "Invalid Iceberg table name: abc$manifests@456"); + assertInvalid("abc@123$data@456"); + assertInvalid("abc@123$snapshots"); + assertInvalid("abc$snapshots@456"); + assertInvalid("xyz$data@456"); + assertInvalid("abc$partitions@456"); + assertInvalid("abc$manifests@456"); } @Test @@ -78,11 +78,11 @@ public void testTableNameWithType() assertThat(IcebergTableName.tableNameWithType("abc", TableType.HISTORY)).isEqualTo("abc$history"); } - private static void assertInvalid(String inputName, String message) + private static void assertInvalid(String inputName) { assertTrinoExceptionThrownBy(() -> IcebergTableName.tableTypeFrom(inputName)) .hasErrorCode(NOT_SUPPORTED) - .hasMessage(message); + .hasMessage("Invalid Iceberg table name: " + inputName); } private static void assertNoValidTableType(String inputName) From 79f5732c0f6bd50c5e59b34a5ad2da6473a38b43 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 3 Jan 2024 09:55:39 +0100 Subject: [PATCH 281/350] Remove redundant variable --- .../io/trino/plugin/iceberg/IcebergMetadata.java | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java index e6e4ca12a869d..f4513e943dc84 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java @@ -572,16 +572,15 @@ private Optional getRawSystemTable(ConnectorSession session, Schema if (tableType.isEmpty()) { return Optional.empty(); } - SchemaTableName systemTableName = new SchemaTableName(tableName.getSchemaName(), IcebergTableName.tableNameWithType(name, tableType.get())); return switch (tableType.get()) { case DATA -> throw new VerifyException("Unexpected DATA table type"); // Handled above. - case HISTORY -> Optional.of(new HistoryTable(systemTableName, table)); - case SNAPSHOTS -> Optional.of(new SnapshotsTable(systemTableName, typeManager, table)); - case PARTITIONS -> Optional.of(new PartitionTable(systemTableName, typeManager, table, getCurrentSnapshotId(table))); - case MANIFESTS -> Optional.of(new ManifestsTable(systemTableName, table, getCurrentSnapshotId(table))); - case FILES -> Optional.of(new FilesTable(systemTableName, typeManager, table, getCurrentSnapshotId(table))); - case PROPERTIES -> Optional.of(new PropertiesTable(systemTableName, table)); - case REFS -> Optional.of(new RefsTable(systemTableName, table)); + case HISTORY -> Optional.of(new HistoryTable(tableName, table)); + case SNAPSHOTS -> Optional.of(new SnapshotsTable(tableName, typeManager, table)); + case PARTITIONS -> Optional.of(new PartitionTable(tableName, typeManager, table, getCurrentSnapshotId(table))); + case MANIFESTS -> Optional.of(new ManifestsTable(tableName, table, getCurrentSnapshotId(table))); + case FILES -> Optional.of(new FilesTable(tableName, typeManager, table, getCurrentSnapshotId(table))); + case PROPERTIES -> Optional.of(new PropertiesTable(tableName, table)); + case REFS -> Optional.of(new RefsTable(tableName, table)); case MATERIALIZED_VIEW_STORAGE -> throw new VerifyException("Unexpected MATERIALIZED_VIEW_STORAGE table type"); }; } From 15f438f13cf174dac2a03153cf84d6c0e5c8545c Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 3 Jan 2024 09:56:18 +0100 Subject: [PATCH 282/350] Unify table type switch handling --- .../src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java index f4513e943dc84..d674896ede2ef 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java @@ -573,7 +573,7 @@ private Optional getRawSystemTable(ConnectorSession session, Schema return Optional.empty(); } return switch (tableType.get()) { - case DATA -> throw new VerifyException("Unexpected DATA table type"); // Handled above. + case DATA, MATERIALIZED_VIEW_STORAGE -> throw new VerifyException("Unexpected table type: " + tableType.get()); // Handled above. case HISTORY -> Optional.of(new HistoryTable(tableName, table)); case SNAPSHOTS -> Optional.of(new SnapshotsTable(tableName, typeManager, table)); case PARTITIONS -> Optional.of(new PartitionTable(tableName, typeManager, table, getCurrentSnapshotId(table))); @@ -581,7 +581,6 @@ private Optional getRawSystemTable(ConnectorSession session, Schema case FILES -> Optional.of(new FilesTable(tableName, typeManager, table, getCurrentSnapshotId(table))); case PROPERTIES -> Optional.of(new PropertiesTable(tableName, table)); case REFS -> Optional.of(new RefsTable(tableName, table)); - case MATERIALIZED_VIEW_STORAGE -> throw new VerifyException("Unexpected MATERIALIZED_VIEW_STORAGE table type"); }; } From f04afb560116a603716fa820f5d5a1fc2847287f Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 3 Jan 2024 13:55:10 +0100 Subject: [PATCH 283/350] Remove obsolete table property checks These were made obsolete by commit 62acd1b1cd69c54fd50bc0a64afb68f50ec0f167. --- .../trino/plugin/deltalake/DeltaLakeMetadata.java | 13 ------------- .../deltalake/procedure/RegisterTableProcedure.java | 8 -------- 2 files changed, 21 deletions(-) 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 ee79488b7fe0d..5fc2f5defa035 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 @@ -965,13 +965,6 @@ public void createTable(ConnectorSession session, ConnectorTableMetadata tableMe Table table = buildTable(session, schemaTableName, location, external); - // Ensure the table has queryId set. This is relied on for exception handling - String queryId = session.getQueryId(); - verify( - getQueryId(table).orElseThrow(() -> new IllegalArgumentException("Query id is not present")).equals(queryId), - "Table '%s' does not have correct query id set", - table); - PrincipalPrivileges principalPrivileges = buildInitialPrivilegeSet(table.getOwner().orElseThrow()); // As a precaution, clear the caches statisticsAccess.invalidateCache(schemaTableName, Optional.of(location)); @@ -1232,12 +1225,6 @@ public Optional finishCreateTable( SchemaTableName schemaTableName = schemaTableName(schemaName, tableName); Table table = buildTable(session, schemaTableName, location, handle.isExternal()); - // Ensure the table has queryId set. This is relied on for exception handling - String queryId = session.getQueryId(); - verify( - getQueryId(table).orElseThrow(() -> new IllegalArgumentException("Query id is not present")).equals(queryId), - "Table '%s' does not have correct query id set", - table); ColumnMappingMode columnMappingMode = handle.getColumnMappingMode(); String schemaString = handle.getSchemaString(); diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/procedure/RegisterTableProcedure.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/procedure/RegisterTableProcedure.java index de3c38e216cb2..76e3a3e892b57 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/procedure/RegisterTableProcedure.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/procedure/RegisterTableProcedure.java @@ -41,12 +41,10 @@ import java.util.Optional; import static com.google.common.base.Strings.isNullOrEmpty; -import static com.google.common.base.Verify.verify; import static io.trino.plugin.base.util.Procedures.checkProcedureArgument; import static io.trino.plugin.deltalake.DeltaLakeErrorCode.DELTA_LAKE_FILESYSTEM_ERROR; import static io.trino.plugin.deltalake.DeltaLakeErrorCode.DELTA_LAKE_INVALID_TABLE; import static io.trino.plugin.deltalake.DeltaLakeMetadata.buildTable; -import static io.trino.plugin.deltalake.DeltaLakeMetadata.getQueryId; import static io.trino.plugin.deltalake.transactionlog.TransactionLogUtil.getTransactionLogDir; import static io.trino.plugin.hive.metastore.MetastoreUtil.buildInitialPrivilegeSet; import static io.trino.spi.StandardErrorCode.GENERIC_USER_ERROR; @@ -177,12 +175,6 @@ private void doRegisterTable( throw new TrinoException(DELTA_LAKE_INVALID_TABLE, "Failed to access table location: " + tableLocation, e); } - // Ensure the table has queryId set. This is relied on for exception handling - String queryId = session.getQueryId(); - verify( - getQueryId(table).orElseThrow(() -> new IllegalArgumentException("Query id is not present")).equals(queryId), - "Table '%s' does not have correct query id set", - table); metastore.createTable(session, table, principalPrivileges); } } From 439e4be22f5aa6a0654b36ee632c2cbcb2dcea2c Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 3 Jan 2024 14:02:17 +0100 Subject: [PATCH 284/350] Don't fail Iceberg on `$partitions` table name Like in Hive connector, return table not found instead of exception when querying `$partitions` (not prefixed with a table name). --- .../trino/plugin/iceberg/IcebergMetadata.java | 7 +++- .../plugin/iceberg/IcebergTableName.java | 5 +++ .../iceberg/BaseIcebergConnectorTest.java | 41 +++++++++++++------ .../TestIcebergReadVersionedTable.java | 4 +- .../plugin/iceberg/TestIcebergStatistics.java | 4 +- 5 files changed, 43 insertions(+), 18 deletions(-) diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java index d674896ede2ef..2e28724451f59 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java @@ -227,6 +227,7 @@ import static io.trino.plugin.iceberg.IcebergSessionProperties.isQueryPartitionFilterRequired; import static io.trino.plugin.iceberg.IcebergSessionProperties.isStatisticsEnabled; import static io.trino.plugin.iceberg.IcebergTableName.isDataTable; +import static io.trino.plugin.iceberg.IcebergTableName.isIcebergTableName; import static io.trino.plugin.iceberg.IcebergTableName.isMaterializedViewStorage; import static io.trino.plugin.iceberg.IcebergTableName.tableNameFrom; import static io.trino.plugin.iceberg.IcebergTableProperties.FILE_FORMAT_PROPERTY; @@ -390,6 +391,10 @@ public ConnectorTableHandle getTableHandle( throw new TrinoException(NOT_SUPPORTED, "Read table with start version is not supported"); } + if (!isIcebergTableName(tableName.getTableName())) { + return null; + } + if (isMaterializedViewStorage(tableName.getTableName())) { verify(endVersion.isEmpty(), "Materialized views do not support versioned queries"); @@ -550,7 +555,7 @@ public Optional getSystemTable(ConnectorSession session, SchemaTabl private Optional getRawSystemTable(ConnectorSession session, SchemaTableName tableName) { - if (isDataTable(tableName.getTableName()) || isMaterializedViewStorage(tableName.getTableName())) { + if (!isIcebergTableName(tableName.getTableName()) || isDataTable(tableName.getTableName()) || isMaterializedViewStorage(tableName.getTableName())) { return Optional.empty(); } diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergTableName.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergTableName.java index 8e5582cc2fb9c..9e838efe90743 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergTableName.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergTableName.java @@ -33,6 +33,11 @@ private IcebergTableName() {} "(?[^$@]+)" + "(?:\\$(?[^@]+))?"); + public static boolean isIcebergTableName(String tableName) + { + return TABLE_PATTERN.matcher(tableName).matches(); + } + public static String tableNameWithType(String tableName, TableType tableType) { requireNonNull(tableName, "tableName is null"); diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java index 4570584a180fe..87be8cdaae39c 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java @@ -79,7 +79,6 @@ import java.time.format.DateTimeFormatter; import java.util.HashSet; import java.util.List; -import java.util.Locale; import java.util.Map; import java.util.NoSuchElementException; import java.util.Optional; @@ -142,6 +141,7 @@ import static java.time.ZoneOffset.UTC; import static java.time.format.DateTimeFormatter.ISO_OFFSET_DATE_TIME; import static java.util.Collections.nCopies; +import static java.util.Locale.ENGLISH; import static java.util.Objects.requireNonNull; import static java.util.UUID.randomUUID; import static java.util.concurrent.TimeUnit.MILLISECONDS; @@ -3221,7 +3221,7 @@ protected void testBucketTransformForType( String greaterValueInSameBucket, String valueInOtherBucket) { - String tableName = format("test_bucket_transform%s", type.toLowerCase(Locale.ENGLISH)); + String tableName = format("test_bucket_transform%s", type.toLowerCase(ENGLISH)); assertUpdate(format("CREATE TABLE %s (d %s) WITH (partitioning = ARRAY['bucket(d, 2)'])", tableName, type)); assertUpdate(format("INSERT INTO %s VALUES (NULL), (%s), (%s), (%s)", tableName, value, greaterValueInSameBucket, valueInOtherBucket), 4); @@ -5039,7 +5039,7 @@ public void testOptimizeTimePartitionedTable() private void testOptimizeTimePartitionedTable(String dataType, String partitioningFormat, int expectedFilesAfterOptimize) { String tableName = "test_optimize_time_partitioned_" + - (dataType + "_" + partitioningFormat).toLowerCase(Locale.ENGLISH).replaceAll("[^a-z0-9_]", ""); + (dataType + "_" + partitioningFormat).toLowerCase(ENGLISH).replaceAll("[^a-z0-9_]", ""); assertUpdate(format("CREATE TABLE %s(p %s, val varchar) WITH (partitioning = ARRAY['%s'])", tableName, dataType, format(partitioningFormat, "p"))); // Do several inserts so ensure more than one input file @@ -5201,7 +5201,7 @@ public void testOptimizeSnapshot() long snapshotId = getCurrentSnapshotId(tableName); assertUpdate("INSERT INTO " + tableName + " VALUES 22", 1); assertThatThrownBy(() -> query("ALTER TABLE \"%s@%d\" EXECUTE OPTIMIZE".formatted(tableName, snapshotId))) - .hasMessage(format("Invalid Iceberg table name: %s@%d", tableName, snapshotId)); + .hasMessage(format("line 1:7: Table 'iceberg.tpch.\"%s@%s\"' does not exist", tableName, snapshotId)); assertThat(query("SELECT * FROM " + tableName)) .matches("VALUES 11, 22"); @@ -5613,7 +5613,7 @@ public void testExpireSnapshotsOnSnapshot() long snapshotId = getCurrentSnapshotId(tableName); assertUpdate("INSERT INTO " + tableName + " VALUES 22", 1); assertThatThrownBy(() -> query("ALTER TABLE \"%s@%d\" EXECUTE EXPIRE_SNAPSHOTS".formatted(tableName, snapshotId))) - .hasMessage(format("Invalid Iceberg table name: %s@%d", tableName, snapshotId)); + .hasMessage(format("line 1:7: Table 'iceberg.tpch.\"%s@%s\"' does not exist", tableName, snapshotId)); assertThat(query("SELECT * FROM " + tableName)) .matches("VALUES 11, 22"); @@ -5805,7 +5805,7 @@ public void testRemoveOrphanFilesOnSnapshot() long snapshotId = getCurrentSnapshotId(tableName); assertUpdate("INSERT INTO " + tableName + " VALUES 22", 1); assertThatThrownBy(() -> query("ALTER TABLE \"%s@%d\" EXECUTE REMOVE_ORPHAN_FILES".formatted(tableName, snapshotId))) - .hasMessage(format("Invalid Iceberg table name: %s@%d", tableName, snapshotId)); + .hasMessage(format("line 1:7: Table 'iceberg.tpch.\"%s@%s\"' does not exist", tableName, snapshotId)); assertThat(query("SELECT * FROM " + tableName)) .matches("VALUES 11, 22"); @@ -5948,19 +5948,19 @@ public void testModifyingOldSnapshotIsNotPossible() assertUpdate(format("INSERT INTO %s VALUES 4,5,6", tableName), 3); assertQuery(format("SELECT * FROM %s FOR VERSION AS OF %d", tableName, oldSnapshotId), "VALUES 1,2,3"); assertThatThrownBy(() -> query(format("INSERT INTO \"%s@%d\" VALUES 7,8,9", tableName, oldSnapshotId))) - .hasMessage(format("Invalid Iceberg table name: %s@%d", tableName, oldSnapshotId)); + .hasMessage(format("Table 'iceberg.tpch.\"%s@%s\"' does not exist", tableName, oldSnapshotId)); assertThatThrownBy(() -> query(format("DELETE FROM \"%s@%d\" WHERE col = 5", tableName, oldSnapshotId))) - .hasMessage(format("Invalid Iceberg table name: %s@%d", tableName, oldSnapshotId)); + .hasMessage(format("line 1:1: Table 'iceberg.tpch.\"%s@%s\"' does not exist", tableName, oldSnapshotId)); assertThatThrownBy(() -> query(format("UPDATE \"%s@%d\" SET col = 50 WHERE col = 5", tableName, oldSnapshotId))) - .hasMessage(format("Invalid Iceberg table name: %s@%d", tableName, oldSnapshotId)); + .hasMessage(format("line 1:1: Table 'iceberg.tpch.\"%s@%s\"' does not exist", tableName, oldSnapshotId)); assertThatThrownBy(() -> query(format("INSERT INTO \"%s@%d\" VALUES 7,8,9", tableName, getCurrentSnapshotId(tableName)))) - .hasMessage(format("Invalid Iceberg table name: %s@%d", tableName, getCurrentSnapshotId(tableName))); + .hasMessage(format("Table 'iceberg.tpch.\"%s@%s\"' does not exist", tableName, getCurrentSnapshotId(tableName))); assertThatThrownBy(() -> query(format("DELETE FROM \"%s@%d\" WHERE col = 9", tableName, getCurrentSnapshotId(tableName)))) - .hasMessage(format("Invalid Iceberg table name: %s@%d", tableName, getCurrentSnapshotId(tableName))); + .hasMessage(format("line 1:1: Table 'iceberg.tpch.\"%s@%s\"' does not exist", tableName, getCurrentSnapshotId(tableName))); assertThatThrownBy(() -> assertUpdate(format("UPDATE \"%s@%d\" set col = 50 WHERE col = 5", tableName, getCurrentSnapshotId(tableName)))) - .hasMessage(format("Invalid Iceberg table name: %s@%d", tableName, getCurrentSnapshotId(tableName))); + .hasMessage(format("line 1:1: Table 'iceberg.tpch.\"%s@%s\"' does not exist", tableName, getCurrentSnapshotId(tableName))); assertThatThrownBy(() -> query(format("ALTER TABLE \"%s@%d\" EXECUTE OPTIMIZE", tableName, oldSnapshotId))) - .hasMessage(format("Invalid Iceberg table name: %s@%d", tableName, oldSnapshotId)); + .hasMessage(format("line 1:7: Table 'iceberg.tpch.\"%s@%s\"' does not exist", tableName, oldSnapshotId)); assertQuery(format("SELECT * FROM %s", tableName), "VALUES 1,2,3,4,5,6"); assertUpdate("DROP TABLE " + tableName); @@ -7603,6 +7603,21 @@ private void testTimePrecisionOnCreateTableAsSelectWithNoData(String inputType, } } + @Test + public void testSystemTables() + { + String catalog = getSession().getCatalog().orElseThrow(); + String schema = getSession().getSchema().orElseThrow(); + for (TableType tableType : TableType.values()) { + if (tableType != TableType.DATA) { + // Like a system table. Make sure this is "table not found". + assertQueryFails( + "TABLE \"$%s\"".formatted(tableType.name().toLowerCase(ENGLISH)), + "\\Qline 1:1: Table '%s.%s.\"$%s\"' does not exist".formatted(catalog, schema, tableType.name().toLowerCase(ENGLISH))); + } + } + } + @Override protected Optional filterSetColumnTypesDataProvider(SetColumnTypeSetup setup) { diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergReadVersionedTable.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergReadVersionedTable.java index 92374ea8742aa..eb33197680b1f 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergReadVersionedTable.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergReadVersionedTable.java @@ -90,10 +90,10 @@ public void testSelectTableWithEndLongTimestampWithTimezone() public void testEndVersionInTableNameAndForClauseShouldFail() { assertQueryFails("SELECT * FROM \"test_iceberg_read_versioned_table@" + v1SnapshotId + "\" FOR VERSION AS OF " + v1SnapshotId, - "Invalid Iceberg table name: test_iceberg_read_versioned_table@%d".formatted(v1SnapshotId)); + "line 1:15: Table 'iceberg.tpch.\"test_iceberg_read_versioned_table@%d\"' does not exist".formatted(v1SnapshotId)); assertQueryFails("SELECT * FROM \"test_iceberg_read_versioned_table@" + v1SnapshotId + "\" FOR TIMESTAMP AS OF " + timestampLiteral(v1EpochMillis, 9), - "Invalid Iceberg table name: test_iceberg_read_versioned_table@%d".formatted(v1SnapshotId)); + "line 1:15: Table 'iceberg.tpch.\"test_iceberg_read_versioned_table@%d\"' does not exist".formatted(v1SnapshotId)); } @Test diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergStatistics.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergStatistics.java index dc86ff1beb924..8b2a0632de086 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergStatistics.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergStatistics.java @@ -597,7 +597,7 @@ public void testAnalyzeSnapshot() long snapshotId = getCurrentSnapshotId(tableName); assertUpdate("INSERT INTO " + tableName + " VALUES 22", 1); assertThatThrownBy(() -> query("ANALYZE \"%s@%d\"".formatted(tableName, snapshotId))) - .hasMessage(format("Invalid Iceberg table name: %s@%d", tableName, snapshotId)); + .hasMessage(format("line 1:1: Table 'iceberg.tpch.\"%s@%s\"' does not exist", tableName, snapshotId)); assertThat(query("SELECT * FROM " + tableName)) .matches("VALUES 11, 22"); @@ -695,7 +695,7 @@ public void testDropStatsSnapshot() long snapshotId = getCurrentSnapshotId(tableName); assertUpdate("INSERT INTO " + tableName + " VALUES 22", 1); assertThatThrownBy(() -> query("ALTER TABLE \"%s@%d\" EXECUTE DROP_EXTENDED_STATS".formatted(tableName, snapshotId))) - .hasMessage(format("Invalid Iceberg table name: %s@%d", tableName, snapshotId)); + .hasMessage(format("line 1:7: Table 'iceberg.tpch.\"%s@%s\"' does not exist", tableName, snapshotId)); assertThat(query("SELECT * FROM " + tableName)) .matches("VALUES 11, 22"); From 4f9f2c60840eec73a7574b25e3d465cefdd09a55 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 3 Jan 2024 14:16:00 +0100 Subject: [PATCH 285/350] Simplify IcebergTableName exception handling Some names were rejected with `TrinoException` and some with graceful fallback (`return Optional.empty`). Since name validity is now checked by `IcebergTableName.isIcebergTableName` we do not need to be lax in all other methods. --- .../trino/plugin/iceberg/IcebergMetadata.java | 9 +-- .../plugin/iceberg/IcebergTableName.java | 70 +++++++++---------- .../iceberg/BaseIcebergConnectorTest.java | 5 ++ .../plugin/iceberg/TestIcebergTableName.java | 59 ++++++++++------ 4 files changed, 76 insertions(+), 67 deletions(-) diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java index 2e28724451f59..d50818352c422 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java @@ -573,12 +573,9 @@ private Optional getRawSystemTable(ConnectorSession session, Schema return Optional.empty(); } - Optional tableType = IcebergTableName.tableTypeFrom(tableName.getTableName()); - if (tableType.isEmpty()) { - return Optional.empty(); - } - return switch (tableType.get()) { - case DATA, MATERIALIZED_VIEW_STORAGE -> throw new VerifyException("Unexpected table type: " + tableType.get()); // Handled above. + TableType tableType = IcebergTableName.tableTypeFrom(tableName.getTableName()); + return switch (tableType) { + case DATA, MATERIALIZED_VIEW_STORAGE -> throw new VerifyException("Unexpected table type: " + tableType); // Handled above. case HISTORY -> Optional.of(new HistoryTable(tableName, table)); case SNAPSHOTS -> Optional.of(new SnapshotsTable(tableName, typeManager, table)); case PARTITIONS -> Optional.of(new PartitionTable(tableName, typeManager, table, getCurrentSnapshotId(table))); diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergTableName.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergTableName.java index 9e838efe90743..63790f48bb050 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergTableName.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergTableName.java @@ -13,15 +13,15 @@ */ package io.trino.plugin.iceberg; -import io.trino.spi.TrinoException; - -import java.util.Optional; import java.util.regex.Matcher; import java.util.regex.Pattern; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Verify.verify; import static io.trino.plugin.iceberg.TableType.DATA; import static io.trino.plugin.iceberg.TableType.MATERIALIZED_VIEW_STORAGE; -import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; import static java.util.Locale.ENGLISH; import static java.util.Objects.requireNonNull; @@ -29,9 +29,17 @@ public final class IcebergTableName { private IcebergTableName() {} - private static final Pattern TABLE_PATTERN = Pattern.compile("" + - "(?
[^$@]+)" + - "(?:\\$(?[^@]+))?"); + private static final Pattern TABLE_PATTERN; + + static { + String referencableTableTypes = Stream.of(TableType.values()) + .filter(tableType -> tableType != DATA) + .map(tableType -> tableType.name().toLowerCase(ENGLISH)) + .collect(Collectors.joining("|")); + TABLE_PATTERN = Pattern.compile("" + + "(?
[^$@]+)" + + "(?:\\$(?(?i:" + referencableTableTypes + ")))?"); + } public static boolean isIcebergTableName(String tableName) { @@ -44,52 +52,38 @@ public static String tableNameWithType(String tableName, TableType tableType) return tableName + "$" + tableType.name().toLowerCase(ENGLISH); } - public static String tableNameFrom(String name) + public static String tableNameFrom(String validIcebergTableName) { - Matcher match = TABLE_PATTERN.matcher(name); - if (!match.matches()) { - throw new TrinoException(NOT_SUPPORTED, "Invalid Iceberg table name: " + name); - } - + Matcher match = TABLE_PATTERN.matcher(validIcebergTableName); + checkArgument(match.matches(), "Invalid Iceberg table name: %s", validIcebergTableName); return match.group("table"); } - public static Optional tableTypeFrom(String name) + public static TableType tableTypeFrom(String validIcebergTableName) { - Matcher match = TABLE_PATTERN.matcher(name); - if (!match.matches()) { - throw new TrinoException(NOT_SUPPORTED, "Invalid Iceberg table name: " + name); - } + Matcher match = TABLE_PATTERN.matcher(validIcebergTableName); + checkArgument(match.matches(), "Invalid Iceberg table name: %s", validIcebergTableName); + String typeString = match.group("type"); if (typeString == null) { - return Optional.of(DATA); - } - try { - TableType parsedType = TableType.valueOf(typeString.toUpperCase(ENGLISH)); - if (parsedType == DATA) { - // $data cannot be encoded in table name - return Optional.empty(); - } - return Optional.of(parsedType); - } - catch (IllegalArgumentException e) { - return Optional.empty(); + return DATA; } + TableType parsedType = TableType.valueOf(typeString.toUpperCase(ENGLISH)); + // $data cannot be encoded in table name + verify(parsedType != DATA, "parsedType is unexpectedly DATA"); + return parsedType; } - public static boolean isDataTable(String name) + public static boolean isDataTable(String validIcebergTableName) { - Matcher match = TABLE_PATTERN.matcher(name); - if (!match.matches()) { - throw new TrinoException(NOT_SUPPORTED, "Invalid Iceberg table name: " + name); - } + Matcher match = TABLE_PATTERN.matcher(validIcebergTableName); + checkArgument(match.matches(), "Invalid Iceberg table name: %s", validIcebergTableName); String typeString = match.group("type"); return typeString == null; } - public static boolean isMaterializedViewStorage(String name) + public static boolean isMaterializedViewStorage(String validIcebergTableName) { - Optional tableType = tableTypeFrom(name); - return tableType.isPresent() && tableType.get() == MATERIALIZED_VIEW_STORAGE; + return tableTypeFrom(validIcebergTableName) == MATERIALIZED_VIEW_STORAGE; } } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java index 87be8cdaae39c..a6f825ee77701 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java @@ -7616,6 +7616,11 @@ public void testSystemTables() "\\Qline 1:1: Table '%s.%s.\"$%s\"' does not exist".formatted(catalog, schema, tableType.name().toLowerCase(ENGLISH))); } } + + // given the base table exists + assertQuerySucceeds("TABLE nation"); + // verify that $ results in table not found + assertQueryFails("TABLE \"nation$foo\"", "\\Qline 1:1: Table '%s.%s.\"nation$foo\"' does not exist".formatted(catalog, schema)); } @Override diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergTableName.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergTableName.java index a2cc507b94406..2be5163468466 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergTableName.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergTableName.java @@ -15,11 +15,8 @@ import org.junit.jupiter.api.Test; -import java.util.Optional; - -import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; -import static io.trino.testing.assertions.TrinoExceptionAssert.assertTrinoExceptionThrownBy; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; public class TestIcebergTableName { @@ -30,10 +27,10 @@ public void testParse() assertParseNameAndType("abc$history", "abc", TableType.HISTORY); assertParseNameAndType("abc$snapshots", "abc", TableType.SNAPSHOTS); - assertNoValidTableType("abc$data"); + assertInvalid("abc$data"); assertInvalid("abc@123"); assertInvalid("abc@xyz"); - assertNoValidTableType("abc$what"); + assertInvalid("abc$what"); assertInvalid("abc@123$data@456"); assertInvalid("abc@123$snapshots"); assertInvalid("abc$snapshots@456"); @@ -47,28 +44,47 @@ public void testIsDataTable() { assertThat(IcebergTableName.isDataTable("abc")).isTrue(); - assertThat(IcebergTableName.isDataTable("abc$data")).isFalse(); // it's invalid + assertThatThrownBy(() -> IcebergTableName.isDataTable("abc$data")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid Iceberg table name: abc$data"); + assertThat(IcebergTableName.isDataTable("abc$history")).isFalse(); - assertThat(IcebergTableName.isDataTable("abc$invalid")).isFalse(); + + assertThatThrownBy(() -> IcebergTableName.isDataTable("abc$invalid")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid Iceberg table name: abc$invalid"); } @Test public void testTableNameFrom() { assertThat(IcebergTableName.tableNameFrom("abc")).isEqualTo("abc"); - assertThat(IcebergTableName.tableNameFrom("abc$data")).isEqualTo("abc"); + + assertThatThrownBy(() -> IcebergTableName.tableNameFrom("abc$data")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid Iceberg table name: abc$data"); + assertThat(IcebergTableName.tableNameFrom("abc$history")).isEqualTo("abc"); - assertThat(IcebergTableName.tableNameFrom("abc$invalid")).isEqualTo("abc"); + + assertThatThrownBy(() -> IcebergTableName.tableNameFrom("abc$invalid")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid Iceberg table name: abc$invalid"); } @Test public void testTableTypeFrom() { - assertThat(IcebergTableName.tableTypeFrom("abc")).isEqualTo(Optional.of(TableType.DATA)); - assertThat(IcebergTableName.tableTypeFrom("abc$data")).isEqualTo(Optional.empty()); // it's invalid - assertThat(IcebergTableName.tableTypeFrom("abc$history")).isEqualTo(Optional.of(TableType.HISTORY)); + assertThat(IcebergTableName.tableTypeFrom("abc")).isEqualTo(TableType.DATA); - assertThat(IcebergTableName.tableTypeFrom("abc$invalid")).isEqualTo(Optional.empty()); + assertThatThrownBy(() -> IcebergTableName.tableTypeFrom("abc$data")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid Iceberg table name: abc$data"); + + assertThat(IcebergTableName.tableTypeFrom("abc$history")).isEqualTo(TableType.HISTORY); + + assertThatThrownBy(() -> IcebergTableName.tableTypeFrom("abc$invalid")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid Iceberg table name: abc$invalid"); } @Test @@ -80,20 +96,17 @@ public void testTableNameWithType() private static void assertInvalid(String inputName) { - assertTrinoExceptionThrownBy(() -> IcebergTableName.tableTypeFrom(inputName)) - .hasErrorCode(NOT_SUPPORTED) - .hasMessage("Invalid Iceberg table name: " + inputName); - } + assertThat(IcebergTableName.isIcebergTableName(inputName)).isFalse(); - private static void assertNoValidTableType(String inputName) - { - assertThat(IcebergTableName.tableTypeFrom(inputName)) - .isEmpty(); + assertThatThrownBy(() -> IcebergTableName.tableTypeFrom(inputName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid Iceberg table name: " + inputName); } private static void assertParseNameAndType(String inputName, String tableName, TableType tableType) { + assertThat(IcebergTableName.isIcebergTableName(inputName)).isTrue(); assertThat(IcebergTableName.tableNameFrom(inputName)).isEqualTo(tableName); - assertThat(IcebergTableName.tableTypeFrom(inputName)).isEqualTo(Optional.of(tableType)); + assertThat(IcebergTableName.tableTypeFrom(inputName)).isEqualTo(tableType); } } From abe48f95923c144d19fc6fcb438d140cbc65a274 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Thu, 4 Jan 2024 11:44:15 +0100 Subject: [PATCH 286/350] Improve argument validation message --- .../main/java/io/trino/filesystem/local/LocalFileSystem.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalFileSystem.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalFileSystem.java index 7c3c7dba27ed6..dc5069260d8d1 100644 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalFileSystem.java +++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalFileSystem.java @@ -48,7 +48,7 @@ public class LocalFileSystem public LocalFileSystem(Path rootPath) { this.rootPath = rootPath; - checkArgument(Files.isDirectory(rootPath), "root is not a directory"); + checkArgument(Files.isDirectory(rootPath), "root is not a directory: %s", rootPath); } @Override From 41687efb28090291d948d772fda1ba16a5474b2b Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Thu, 4 Jan 2024 12:16:16 +0100 Subject: [PATCH 287/350] Define tables required by AbstractTestAggregations --- .../hive/TestHiveDistributedAggregations.java | 4 +--- ...TestHiveFaultTolerantExecutionAggregations.java | 3 +-- ...itioningFaultTolerantExecutionAggregations.java | 4 +--- .../io/trino/testing/AbstractTestAggregations.java | 14 ++++++++++++++ 4 files changed, 17 insertions(+), 8 deletions(-) diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveDistributedAggregations.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveDistributedAggregations.java index 13a5fe418f09e..7c4c17dc85d60 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveDistributedAggregations.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveDistributedAggregations.java @@ -16,8 +16,6 @@ import io.trino.testing.AbstractTestAggregations; import io.trino.testing.QueryRunner; -import static io.trino.tpch.TpchTable.getTables; - public class TestHiveDistributedAggregations extends AbstractTestAggregations { @@ -26,7 +24,7 @@ protected QueryRunner createQueryRunner() throws Exception { return HiveQueryRunner.builder() - .setInitialTables(getTables()) + .setInitialTables(REQUIRED_TPCH_TABLES) .build(); } } diff --git a/testing/trino-faulttolerant-tests/src/test/java/io/trino/faulttolerant/hive/TestHiveFaultTolerantExecutionAggregations.java b/testing/trino-faulttolerant-tests/src/test/java/io/trino/faulttolerant/hive/TestHiveFaultTolerantExecutionAggregations.java index 19dcd88fbaacd..da94ddc79b672 100644 --- a/testing/trino-faulttolerant-tests/src/test/java/io/trino/faulttolerant/hive/TestHiveFaultTolerantExecutionAggregations.java +++ b/testing/trino-faulttolerant-tests/src/test/java/io/trino/faulttolerant/hive/TestHiveFaultTolerantExecutionAggregations.java @@ -26,7 +26,6 @@ import static io.trino.plugin.exchange.filesystem.containers.MinioStorage.getExchangeManagerProperties; import static io.trino.testing.TestingNames.randomNameSuffix; -import static io.trino.tpch.TpchTable.getTables; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; @TestInstance(PER_CLASS) @@ -49,7 +48,7 @@ protected QueryRunner createQueryRunner(Map extraProperties) runner.installPlugin(new FileSystemExchangePlugin()); runner.loadExchangeManager("filesystem", getExchangeManagerProperties(minioStorage)); }) - .setInitialTables(getTables()) + .setInitialTables(REQUIRED_TPCH_TABLES) .build(); } diff --git a/testing/trino-faulttolerant-tests/src/test/java/io/trino/faulttolerant/hive/TestHiveRuntimeAdaptivePartitioningFaultTolerantExecutionAggregations.java b/testing/trino-faulttolerant-tests/src/test/java/io/trino/faulttolerant/hive/TestHiveRuntimeAdaptivePartitioningFaultTolerantExecutionAggregations.java index e09a4ddff0c3d..55765c94c309c 100644 --- a/testing/trino-faulttolerant-tests/src/test/java/io/trino/faulttolerant/hive/TestHiveRuntimeAdaptivePartitioningFaultTolerantExecutionAggregations.java +++ b/testing/trino-faulttolerant-tests/src/test/java/io/trino/faulttolerant/hive/TestHiveRuntimeAdaptivePartitioningFaultTolerantExecutionAggregations.java @@ -22,8 +22,6 @@ import java.util.Map; -import static io.trino.tpch.TpchTable.getTables; - public class TestHiveRuntimeAdaptivePartitioningFaultTolerantExecutionAggregations extends AbstractTestFaultTolerantExecutionAggregations { @@ -42,7 +40,7 @@ protected QueryRunner createQueryRunner(Map extraProperties) runner.loadExchangeManager("filesystem", ImmutableMap.of("exchange.base-directories", System.getProperty("java.io.tmpdir") + "/trino-local-file-system-exchange-manager")); }) - .setInitialTables(getTables()) + .setInitialTables(REQUIRED_TPCH_TABLES) .build(); } } diff --git a/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestAggregations.java b/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestAggregations.java index c1ca47508ad10..6f5c7667d97b7 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestAggregations.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/AbstractTestAggregations.java @@ -14,21 +14,35 @@ package io.trino.testing; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; import io.trino.Session; import io.trino.spi.type.TimeZoneKey; +import io.trino.tpch.TpchTable; import org.intellij.lang.annotations.Language; import org.junit.jupiter.api.Test; import java.util.List; +import java.util.Set; import static io.trino.SystemSessionProperties.MARK_DISTINCT_STRATEGY; import static io.trino.testing.MaterializedResult.resultBuilder; import static io.trino.testing.QueryAssertions.assertEqualsIgnoreOrder; +import static io.trino.tpch.TpchTable.LINE_ITEM; +import static io.trino.tpch.TpchTable.NATION; +import static io.trino.tpch.TpchTable.ORDERS; +import static io.trino.tpch.TpchTable.REGION; import static org.assertj.core.api.Assertions.assertThat; public abstract class AbstractTestAggregations extends AbstractTestQueryFramework { + protected static final Set> REQUIRED_TPCH_TABLES = ImmutableSet.>builder() + .add(LINE_ITEM) + .add(NATION) + .add(ORDERS) + .add(REGION) + .build(); + @Test public void testCountBoolean() { From 5113d9bb31ae013cf0fb8368f1dee0220f26280c Mon Sep 17 00:00:00 2001 From: Slawomir Pajak Date: Tue, 19 Dec 2023 12:17:35 +0100 Subject: [PATCH 288/350] Test hive metastore operations involving retries --- .../TestingThriftHiveMetastoreBuilder.java | 10 +++ .../metastore/AbstractTestHiveMetastore.java | 5 ++ .../metastore/TestBridgingHiveMetastore.java | 82 ++++++++++++++++++- .../MetastoreClientAdapterProvider.java | 19 +++++ ...stingTokenAwareMetastoreClientFactory.java | 12 ++- 5 files changed, 125 insertions(+), 3 deletions(-) create mode 100644 plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/MetastoreClientAdapterProvider.java diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestingThriftHiveMetastoreBuilder.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestingThriftHiveMetastoreBuilder.java index dccc93a9768ce..e89932a6ac465 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestingThriftHiveMetastoreBuilder.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestingThriftHiveMetastoreBuilder.java @@ -18,6 +18,7 @@ import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.filesystem.hdfs.HdfsFileSystemFactory; import io.trino.plugin.hive.metastore.HiveMetastoreConfig; +import io.trino.plugin.hive.metastore.thrift.MetastoreClientAdapterProvider; import io.trino.plugin.hive.metastore.thrift.TestingTokenAwareMetastoreClientFactory; import io.trino.plugin.hive.metastore.thrift.ThriftHiveMetastoreFactory; import io.trino.plugin.hive.metastore.thrift.ThriftMetastore; @@ -32,6 +33,7 @@ import static io.trino.plugin.base.security.UserNameProvider.SIMPLE_USER_NAME_PROVIDER; import static io.trino.plugin.hive.HiveTestUtils.HDFS_ENVIRONMENT; import static io.trino.plugin.hive.HiveTestUtils.HDFS_FILE_SYSTEM_STATS; +import static io.trino.plugin.hive.metastore.thrift.TestingTokenAwareMetastoreClientFactory.TIMEOUT; import static java.util.Objects.requireNonNull; import static java.util.concurrent.Executors.newFixedThreadPool; @@ -58,6 +60,14 @@ public TestingThriftHiveMetastoreBuilder metastoreClient(HostAndPort address, Du return this; } + public TestingThriftHiveMetastoreBuilder metastoreClient(HostAndPort address, MetastoreClientAdapterProvider metastoreClientAdapterProvider) + { + requireNonNull(address, "address is null"); + checkState(tokenAwareMetastoreClientFactory == null, "Metastore client already set"); + tokenAwareMetastoreClientFactory = new TestingTokenAwareMetastoreClientFactory(HiveTestUtils.SOCKS_PROXY, address, TIMEOUT, metastoreClientAdapterProvider); + return this; + } + public TestingThriftHiveMetastoreBuilder metastoreClient(HostAndPort address) { requireNonNull(address, "address is null"); diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/AbstractTestHiveMetastore.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/AbstractTestHiveMetastore.java index 82460db7532c9..7b3987bae1bc1 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/AbstractTestHiveMetastore.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/AbstractTestHiveMetastore.java @@ -45,6 +45,11 @@ public void setMetastore(HiveMetastore metastore) this.metastore = metastore; } + protected HiveMetastore getMetastore() + { + return metastore; + } + @Test void testCreateDatabase() { diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/TestBridgingHiveMetastore.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/TestBridgingHiveMetastore.java index f8242a403cfbd..b95010c7338bd 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/TestBridgingHiveMetastore.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/TestBridgingHiveMetastore.java @@ -13,14 +13,31 @@ */ package io.trino.plugin.hive.metastore; +import io.trino.plugin.hive.SchemaAlreadyExistsException; +import io.trino.plugin.hive.TableAlreadyExistsException; import io.trino.plugin.hive.containers.HiveHadoop; import io.trino.plugin.hive.metastore.thrift.BridgingHiveMetastore; +import io.trino.plugin.hive.metastore.thrift.MetastoreClientAdapterProvider; +import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreClient; import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreConfig; import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Test; import org.junit.jupiter.api.TestInstance; import org.junit.jupiter.api.parallel.Execution; +import java.lang.reflect.InvocationTargetException; +import java.util.Map; +import java.util.Optional; + +import static com.google.common.reflect.Reflection.newProxy; +import static io.trino.plugin.hive.HiveMetadata.TRINO_QUERY_ID_NAME; +import static io.trino.plugin.hive.HiveStorageFormat.PARQUET; +import static io.trino.plugin.hive.TableType.EXTERNAL_TABLE; import static io.trino.plugin.hive.TestingThriftHiveMetastoreBuilder.testingThriftHiveMetastoreBuilder; +import static io.trino.plugin.hive.metastore.PrincipalPrivileges.NO_PRIVILEGES; +import static io.trino.plugin.hive.metastore.StorageFormat.fromHiveStorageFormat; +import static io.trino.testing.TestingNames.randomNameSuffix; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; @@ -36,8 +53,22 @@ final class TestBridgingHiveMetastore hiveHadoop = HiveHadoop.builder().build(); hiveHadoop.start(); + MetastoreClientAdapterProvider metastoreClientAdapterProvider = delegate -> newProxy(ThriftMetastoreClient.class, (proxy, method, methodArgs) -> { + Object result; + try { + result = method.invoke(delegate, methodArgs); + } + catch (InvocationTargetException e) { + throw e.getCause(); + } + if (method.getName().equals("createDatabase") || method.getName().equals("createTable")) { + throw new RuntimeException("Test-simulated Hive Metastore timeout exception"); + } + return result; + }); + setMetastore(new BridgingHiveMetastore(testingThriftHiveMetastoreBuilder() - .metastoreClient(hiveHadoop.getHiveMetastoreEndpoint()) + .metastoreClient(hiveHadoop.getHiveMetastoreEndpoint(), metastoreClientAdapterProvider) .thriftMetastoreConfig(new ThriftMetastoreConfig().setDeleteFilesOnDrop(true)) .build())); } @@ -47,4 +78,53 @@ void afterAll() { hiveHadoop.stop(); } + + @Test + public void testCreateDatabaseWithRetries() + { + // This test is similar to AbstractTestHiveMetastore#testCreateDatabase but with simulating timeout in ThriftMetastoreClient + String databaseName = "test_database_" + randomNameSuffix(); + Database.Builder database = Database.builder() + .setDatabaseName(databaseName) + .setParameters(Map.of(TRINO_QUERY_ID_NAME, "query_id")) + .setOwnerName(Optional.empty()) + .setOwnerType(Optional.empty()); + getMetastore().createDatabase(database.build()); + + database.setParameters(Map.of(TRINO_QUERY_ID_NAME, "another_query_id")); + assertThatThrownBy(() -> getMetastore().createDatabase(database.build())) + .isInstanceOf(SchemaAlreadyExistsException.class); + + getMetastore().dropDatabase(databaseName, false); + } + + @Test + public void testCreateTableWithRetries() + { + // This test is similar to AbstractTestHiveMetastore#testCreateTable but with simulating timeout in ThriftMetastoreClient + String databaseName = "test_database_" + randomNameSuffix(); + Database.Builder database = Database.builder() + .setDatabaseName(databaseName) + .setOwnerName(Optional.empty()) + .setOwnerType(Optional.empty()); + getMetastore().createDatabase(database.build()); + + String tableName = "test_table" + randomNameSuffix(); + Table.Builder table = Table.builder() + .setDatabaseName(databaseName) + .setTableName(tableName) + .setParameters(Map.of(TRINO_QUERY_ID_NAME, "query_id")) + .setTableType(EXTERNAL_TABLE.name()) + .setOwner(Optional.empty()); + table.getStorageBuilder() + .setStorageFormat(fromHiveStorageFormat(PARQUET)); + getMetastore().createTable(table.build(), NO_PRIVILEGES); + + table.setParameters(Map.of(TRINO_QUERY_ID_NAME, "another_query_id")); + assertThatThrownBy(() -> getMetastore().createTable(table.build(), NO_PRIVILEGES)) + .isInstanceOf(TableAlreadyExistsException.class); + + getMetastore().dropTable(databaseName, tableName, false); + getMetastore().dropDatabase(databaseName, false); + } } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/MetastoreClientAdapterProvider.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/MetastoreClientAdapterProvider.java new file mode 100644 index 0000000000000..479448cc9a259 --- /dev/null +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/MetastoreClientAdapterProvider.java @@ -0,0 +1,19 @@ +/* + * 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.hive.metastore.thrift; + +public interface MetastoreClientAdapterProvider +{ + ThriftMetastoreClient createThriftMetastoreClientAdapter(ThriftMetastoreClient delegate); +} diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestingTokenAwareMetastoreClientFactory.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestingTokenAwareMetastoreClientFactory.java index 0fa8a586d8dcb..1f2508261d160 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestingTokenAwareMetastoreClientFactory.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestingTokenAwareMetastoreClientFactory.java @@ -31,21 +31,29 @@ public class TestingTokenAwareMetastoreClientFactory private final DefaultThriftMetastoreClientFactory factory; private final HostAndPort address; + private final MetastoreClientAdapterProvider metastoreClientAdapterProvider; + public TestingTokenAwareMetastoreClientFactory(Optional socksProxy, HostAndPort address) { - this(socksProxy, address, TIMEOUT); + this(socksProxy, address, TIMEOUT, delegate -> delegate); } public TestingTokenAwareMetastoreClientFactory(Optional socksProxy, HostAndPort address, Duration timeout) + { + this(socksProxy, address, timeout, delegate -> delegate); + } + + public TestingTokenAwareMetastoreClientFactory(Optional socksProxy, HostAndPort address, Duration timeout, MetastoreClientAdapterProvider metastoreClientAdapterProvider) { this.factory = new DefaultThriftMetastoreClientFactory(Optional.empty(), socksProxy, timeout, timeout, AUTHENTICATION, "localhost"); this.address = requireNonNull(address, "address is null"); + this.metastoreClientAdapterProvider = requireNonNull(metastoreClientAdapterProvider, "metastoreClientAdapterProvider is null"); } @Override public ThriftMetastoreClient createMetastoreClient(Optional delegationToken) throws TException { - return factory.create(address, delegationToken); + return metastoreClientAdapterProvider.createThriftMetastoreClientAdapter(factory.create(address, delegationToken)); } } From a6eb444a2d518e3b889c38dbeef0305165e85b65 Mon Sep 17 00:00:00 2001 From: Slawomir Pajak Date: Tue, 19 Dec 2023 12:43:28 +0100 Subject: [PATCH 289/350] Ignore TableNotFoundException on retries when dropping the table --- .../metastore/thrift/ThriftHiveMetastore.java | 17 ++++++++++- .../metastore/AbstractTestHiveMetastore.java | 18 +++++++++++ .../metastore/TestBridgingHiveMetastore.java | 30 ++++++++++++++++++- 3 files changed, 63 insertions(+), 2 deletions(-) diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHiveMetastore.java index 7ae93dc4e1ab3..8f08137cf14d9 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/thrift/ThriftHiveMetastore.java @@ -103,6 +103,7 @@ import java.util.Set; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; import java.util.stream.Stream; @@ -1062,13 +1063,27 @@ public void createTable(Table table) @Override public void dropTable(String databaseName, String tableName, boolean deleteData) { + AtomicInteger attemptCount = new AtomicInteger(); try { retry() .stopOn(NoSuchObjectException.class) .stopOnIllegalExceptions() .run("dropTable", stats.getDropTable().wrap(() -> { try (ThriftMetastoreClient client = createMetastoreClient()) { - Table table = client.getTable(databaseName, tableName); + attemptCount.incrementAndGet(); + Table table; + try { + table = client.getTable(databaseName, tableName); + } + catch (NoSuchObjectException e) { + if (attemptCount.get() == 1) { + // Throw exception only on first attempt. + throw e; + } + // If table is not found on consecutive attempts it was probably dropped on first attempt and timeout occurred. + // Exception in such case can be safely ignored and dropping table is finished. + return null; + } client.dropTable(databaseName, tableName, deleteData); String tableLocation = table.getSd().getLocation(); if (deleteFilesOnDrop && deleteData && isManagedTable(table) && !isNullOrEmpty(tableLocation)) { diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/AbstractTestHiveMetastore.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/AbstractTestHiveMetastore.java index 7b3987bae1bc1..f54d8da54c1f9 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/AbstractTestHiveMetastore.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/AbstractTestHiveMetastore.java @@ -15,6 +15,7 @@ import io.trino.plugin.hive.SchemaAlreadyExistsException; import io.trino.plugin.hive.TableAlreadyExistsException; +import io.trino.spi.connector.TableNotFoundException; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.TestInstance; import org.junit.jupiter.api.parallel.Execution; @@ -30,6 +31,7 @@ import static io.trino.plugin.hive.TableType.EXTERNAL_TABLE; import static io.trino.plugin.hive.metastore.PrincipalPrivileges.NO_PRIVILEGES; import static io.trino.plugin.hive.metastore.StorageFormat.fromHiveStorageFormat; +import static io.trino.testing.TestingNames.randomNameSuffix; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; @@ -102,4 +104,20 @@ void testCreateTable() metastore.dropTable(databaseName, tableName, false); metastore.dropDatabase(databaseName, false); } + + @Test + public void testDropNotExistingTable() + { + String databaseName = "test_database_" + randomNameSuffix(); + Database.Builder database = Database.builder() + .setDatabaseName(databaseName) + .setOwnerName(Optional.empty()) + .setOwnerType(Optional.empty()); + getMetastore().createDatabase(database.build()); + + assertThatThrownBy(() -> getMetastore().dropTable(databaseName, "not_existing", false)) + .isInstanceOf(TableNotFoundException.class); + + getMetastore().dropDatabase(databaseName, false); + } } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/TestBridgingHiveMetastore.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/TestBridgingHiveMetastore.java index b95010c7338bd..59fa33ca076ee 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/TestBridgingHiveMetastore.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/TestBridgingHiveMetastore.java @@ -37,6 +37,7 @@ import static io.trino.plugin.hive.metastore.PrincipalPrivileges.NO_PRIVILEGES; import static io.trino.plugin.hive.metastore.StorageFormat.fromHiveStorageFormat; import static io.trino.testing.TestingNames.randomNameSuffix; +import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; @@ -61,7 +62,7 @@ final class TestBridgingHiveMetastore catch (InvocationTargetException e) { throw e.getCause(); } - if (method.getName().equals("createDatabase") || method.getName().equals("createTable")) { + if (method.getName().equals("createDatabase") || method.getName().equals("createTable") || method.getName().equals("dropTable")) { throw new RuntimeException("Test-simulated Hive Metastore timeout exception"); } return result; @@ -127,4 +128,31 @@ public void testCreateTableWithRetries() getMetastore().dropTable(databaseName, tableName, false); getMetastore().dropDatabase(databaseName, false); } + + @Test + public void testDropTableWithRetries() + { + String databaseName = "test_database_" + randomNameSuffix(); + Database.Builder database = Database.builder() + .setDatabaseName(databaseName) + .setOwnerName(Optional.empty()) + .setOwnerType(Optional.empty()); + getMetastore().createDatabase(database.build()); + + String tableName = "test_table" + randomNameSuffix(); + Table.Builder table = Table.builder() + .setDatabaseName(databaseName) + .setTableName(tableName) + .setTableType(EXTERNAL_TABLE.name()) + .setOwner(Optional.empty()); + table.getStorageBuilder() + .setStorageFormat(fromHiveStorageFormat(PARQUET)); + getMetastore().createTable(table.build(), NO_PRIVILEGES); + + assertThat(getMetastore().getTable(databaseName, tableName)).isPresent(); + getMetastore().dropTable(databaseName, tableName, false); + assertThat(getMetastore().getTable(databaseName, tableName)).isEmpty(); + + getMetastore().dropDatabase(databaseName, false); + } } From 4817ac87dbd9c120f7a4d290a116291e0fea8077 Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Tue, 2 Jan 2024 10:59:31 +0100 Subject: [PATCH 290/350] Add trino-opensearch plugin This is a preparatory step: opensearch plugin is an exact copy of elasticsearch plugin --- core/trino-server/src/main/provisio/trino.xml | 6 + plugin/trino-opensearch/pom.xml | 436 ++++ .../elasticsearch/AwsSecurityConfig.java | 99 + .../plugin/elasticsearch/BuiltinColumns.java | 91 + .../elasticsearch/CountQueryPageSource.java | 93 + .../elasticsearch/DecoderDescriptor.java | 62 + .../ElasticsearchColumnHandle.java | 98 + .../elasticsearch/ElasticsearchConfig.java | 360 +++ .../elasticsearch/ElasticsearchConnector.java | 104 + .../ElasticsearchConnectorFactory.java | 71 + .../ElasticsearchConnectorModule.java | 68 + .../elasticsearch/ElasticsearchErrorCode.java | 44 + .../elasticsearch/ElasticsearchMetadata.java | 716 ++++++ .../ElasticsearchPageSourceProvider.java | 79 + .../elasticsearch/ElasticsearchPlugin.java | 44 + .../ElasticsearchQueryBuilder.java | 172 ++ .../elasticsearch/ElasticsearchSplit.java | 99 + .../ElasticsearchSplitManager.java | 64 + .../ElasticsearchTableHandle.java | 170 ++ .../ElasticsearchTransactionHandle.java | 22 + .../elasticsearch/NodesSystemTable.java | 105 + .../PassthroughQueryPageSource.java | 91 + .../plugin/elasticsearch/PasswordConfig.java | 51 + .../elasticsearch/ScanQueryPageSource.java | 324 +++ .../client/AwsRequestSigner.java | 114 + .../client/BackpressureRestClient.java | 158 ++ .../BackpressureRestHighLevelClient.java | 142 ++ .../elasticsearch/client/CountResponse.java | 34 + .../client/ElasticsearchClient.java | 795 +++++++ .../client/ElasticsearchNode.java | 46 + .../elasticsearch/client/IndexMetadata.java | 145 ++ .../elasticsearch/client/NodesResponse.java | 90 + .../client/SearchShardsResponse.java | 87 + .../plugin/elasticsearch/client/Shard.java | 53 + .../elasticsearch/decoders/ArrayDecoder.java | 75 + .../elasticsearch/decoders/BigintDecoder.java | 90 + .../decoders/BooleanDecoder.java | 89 + .../elasticsearch/decoders/Decoder.java | 24 + .../elasticsearch/decoders/DoubleDecoder.java | 95 + .../decoders/IdColumnDecoder.java | 43 + .../decoders/IntegerDecoder.java | 100 + .../decoders/IpAddressDecoder.java | 124 ++ .../decoders/RawJsonDecoder.java | 89 + .../elasticsearch/decoders/RealDecoder.java | 95 + .../elasticsearch/decoders/RowDecoder.java | 132 ++ .../decoders/ScoreColumnDecoder.java | 42 + .../decoders/SmallintDecoder.java | 99 + .../decoders/SourceColumnDecoder.java | 43 + .../decoders/TimestampDecoder.java | 118 + .../decoders/TinyintDecoder.java | 99 + .../decoders/VarbinaryDecoder.java | 80 + .../decoders/VarcharDecoder.java | 79 + .../plugin/elasticsearch/ptf/RawQuery.java | 148 ++ .../BaseElasticsearchConnectorTest.java | 1962 +++++++++++++++++ .../ElasticsearchExternalQueryRunner.java | 49 + .../elasticsearch/ElasticsearchLoader.java | 146 ++ .../ElasticsearchNginxProxy.java | 103 + .../ElasticsearchQueryRunner.java | 162 ++ .../elasticsearch/ElasticsearchServer.java | 85 + .../elasticsearch/TestAwsSecurityConfig.java | 58 + .../TestElasticsearch6ConnectorTest.java | 63 + .../TestElasticsearch7ConnectorTest.java | 38 + .../TestElasticsearchBackpressure.java | 86 + .../TestElasticsearchConfig.java | 119 + .../TestElasticsearchMetadata.java | 47 + ...tElasticsearchOpenSearchConnectorTest.java | 38 + .../TestElasticsearchQueryBuilder.java | 138 ++ .../TestPasswordAuthentication.java | 125 ++ .../elasticsearch/TestPasswordConfig.java | 49 + .../client/TestExtractAddress.java | 39 + .../src/test/resources/elasticsearch.yml | 4 + .../src/test/resources/roles.yml | 6 + .../trino-opensearch/src/test/resources/users | 1 + .../src/test/resources/users_roles | 1 + pom.xml | 1 + 75 files changed, 10117 insertions(+) create mode 100644 plugin/trino-opensearch/pom.xml create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/AwsSecurityConfig.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/BuiltinColumns.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/CountQueryPageSource.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/DecoderDescriptor.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchColumnHandle.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchConfig.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchConnector.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchConnectorFactory.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchConnectorModule.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchErrorCode.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchMetadata.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchPageSourceProvider.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchPlugin.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchQueryBuilder.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchSplit.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchSplitManager.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchTableHandle.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchTransactionHandle.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/NodesSystemTable.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/PassthroughQueryPageSource.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/PasswordConfig.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ScanQueryPageSource.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/AwsRequestSigner.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/BackpressureRestClient.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/BackpressureRestHighLevelClient.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/CountResponse.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/ElasticsearchClient.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/ElasticsearchNode.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/IndexMetadata.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/NodesResponse.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/SearchShardsResponse.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/Shard.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/ArrayDecoder.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/BigintDecoder.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/BooleanDecoder.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/Decoder.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/DoubleDecoder.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/IdColumnDecoder.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/IntegerDecoder.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/IpAddressDecoder.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/RawJsonDecoder.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/RealDecoder.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/RowDecoder.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/ScoreColumnDecoder.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/SmallintDecoder.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/SourceColumnDecoder.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/TimestampDecoder.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/TinyintDecoder.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/VarbinaryDecoder.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/VarcharDecoder.java create mode 100644 plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ptf/RawQuery.java create mode 100644 plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/BaseElasticsearchConnectorTest.java create mode 100644 plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchExternalQueryRunner.java create mode 100644 plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchLoader.java create mode 100644 plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchNginxProxy.java create mode 100644 plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchQueryRunner.java create mode 100644 plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchServer.java create mode 100644 plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestAwsSecurityConfig.java create mode 100644 plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearch6ConnectorTest.java create mode 100644 plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearch7ConnectorTest.java create mode 100644 plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchBackpressure.java create mode 100644 plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchConfig.java create mode 100644 plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchMetadata.java create mode 100644 plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchOpenSearchConnectorTest.java create mode 100644 plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchQueryBuilder.java create mode 100644 plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestPasswordAuthentication.java create mode 100644 plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestPasswordConfig.java create mode 100644 plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/client/TestExtractAddress.java create mode 100644 plugin/trino-opensearch/src/test/resources/elasticsearch.yml create mode 100644 plugin/trino-opensearch/src/test/resources/roles.yml create mode 100644 plugin/trino-opensearch/src/test/resources/users create mode 100644 plugin/trino-opensearch/src/test/resources/users_roles diff --git a/core/trino-server/src/main/provisio/trino.xml b/core/trino-server/src/main/provisio/trino.xml index a11e4ade027e5..2987eb5d00076 100644 --- a/core/trino-server/src/main/provisio/trino.xml +++ b/core/trino-server/src/main/provisio/trino.xml @@ -218,6 +218,12 @@ + + + + + + diff --git a/plugin/trino-opensearch/pom.xml b/plugin/trino-opensearch/pom.xml new file mode 100644 index 0000000000000..2a94e70b982d2 --- /dev/null +++ b/plugin/trino-opensearch/pom.xml @@ -0,0 +1,436 @@ + + + 4.0.0 + + + io.trino + trino-root + 436-SNAPSHOT + ../../pom.xml + + + trino-opensearch + trino-plugin + Trino - OpenSearch Connector + + + ${project.parent.basedir} + 6.8.23 + + + + + com.amazonaws + aws-java-sdk-core + + + + org.apache.httpcomponents + httpclient + + + + + + com.amazonaws + aws-java-sdk-sts + + + + com.fasterxml.jackson.core + jackson-core + + + + com.fasterxml.jackson.core + jackson-databind + + + + com.google.guava + guava + + + + com.google.inject + guice + + + + dev.failsafe + failsafe + + + + io.airlift + bootstrap + + + + io.airlift + concurrent + + + + io.airlift + configuration + + + + io.airlift + json + + + + io.airlift + log + + + + io.airlift + stats + + + + io.airlift + units + + + + io.trino + trino-plugin-toolkit + + + + jakarta.annotation + jakarta.annotation-api + + + + jakarta.validation + jakarta.validation-api + + + + org.apache.httpcomponents + httpasyncclient + 4.1.2 + + + + commons-logging + commons-logging + + + + + + org.apache.httpcomponents + httpclient + 4.5.14 + + + + commons-logging + commons-logging + + + + + + org.apache.httpcomponents + httpcore + 4.4.16 + + + + org.apache.httpcomponents + httpcore-nio + 4.4.16 + + + + org.elasticsearch + elasticsearch + ${dep.elasticsearch.version} + + + org.apache.logging.log4j + log4j-api + + + + org.apache.lucene + lucene-analyzers-common + + + + org.elasticsearch + jna + + + + + + org.elasticsearch + elasticsearch-core + ${dep.elasticsearch.version} + + + + org.elasticsearch + elasticsearch-x-content + ${dep.elasticsearch.version} + + + org.yaml + snakeyaml + + + + + + org.elasticsearch.client + elasticsearch-rest-client + ${dep.elasticsearch.version} + + + + commons-logging + commons-logging + + + + + + org.elasticsearch.client + elasticsearch-rest-high-level-client + ${dep.elasticsearch.version} + + + org.apache.logging.log4j + log4j-api + + + + + + org.weakref + jmxutils + + + + com.fasterxml.jackson.core + jackson-annotations + provided + + + + io.airlift + slice + provided + + + + io.opentelemetry + opentelemetry-api + provided + + + + io.opentelemetry + opentelemetry-context + provided + + + + io.trino + trino-spi + provided + + + + org.openjdk.jol + jol-core + provided + + + + io.airlift + log-manager + runtime + + + + io.airlift + node + runtime + + + + io.airlift + http-server + test + + + + io.airlift + junit-extensions + test + + + + io.airlift + testing + test + + + + io.trino + trino-client + test + + + + io.trino + trino-jmx + test + + + + io.trino + trino-main + test + + + commons-codec + commons-codec + + + + + + io.trino + trino-main + test-jar + test + + + commons-codec + commons-codec + + + + + + io.trino + trino-testing + test + + + + io.trino + trino-testing-services + test + + + + org.openjdk.jmh + jmh-core + + + + + + io.trino + trino-tpch + test + + + + io.trino.tpch + tpch + test + + + + org.assertj + assertj-core + test + + + + org.eclipse.jetty.toolchain + jetty-jakarta-servlet-api + test + + + + org.jetbrains + annotations + test + + + + org.junit.jupiter + junit-jupiter-api + test + + + + org.junit.jupiter + junit-jupiter-engine + test + + + + org.testcontainers + elasticsearch + test + + + + org.testcontainers + nginx + test + + + + org.testcontainers + testcontainers + test + + + + + + + + org.apache.maven.plugins + maven-enforcer-plugin + + + + + + org.apache.lucene:lucene-core + + + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + + + org.elasticsearch:elasticsearch-x-content + + + + + + + diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/AwsSecurityConfig.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/AwsSecurityConfig.java new file mode 100644 index 0000000000000..8baf2b7b54853 --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/AwsSecurityConfig.java @@ -0,0 +1,99 @@ +/* + * 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.elasticsearch; + +import io.airlift.configuration.Config; +import io.airlift.configuration.ConfigDescription; +import io.airlift.configuration.ConfigSecuritySensitive; +import io.airlift.configuration.DefunctConfig; +import jakarta.validation.constraints.NotNull; + +import java.util.Optional; + +@DefunctConfig("elasticsearch.aws.use-instance-credentials") +public class AwsSecurityConfig +{ + private String accessKey; + private String secretKey; + private String region; + private String iamRole; + private String externalId; + + @NotNull + public Optional getAccessKey() + { + return Optional.ofNullable(accessKey); + } + + @Config("elasticsearch.aws.access-key") + public AwsSecurityConfig setAccessKey(String key) + { + this.accessKey = key; + return this; + } + + @NotNull + public Optional getSecretKey() + { + return Optional.ofNullable(secretKey); + } + + @Config("elasticsearch.aws.secret-key") + @ConfigSecuritySensitive + public AwsSecurityConfig setSecretKey(String key) + { + this.secretKey = key; + return this; + } + + public String getRegion() + { + return region; + } + + @Config("elasticsearch.aws.region") + public AwsSecurityConfig setRegion(String region) + { + this.region = region; + return this; + } + + @NotNull + public Optional getIamRole() + { + return Optional.ofNullable(iamRole); + } + + @Config("elasticsearch.aws.iam-role") + @ConfigDescription("Optional AWS IAM role to assume for authenticating. If set, this role will be used to get credentials to sign requests to ES.") + public AwsSecurityConfig setIamRole(String iamRole) + { + this.iamRole = iamRole; + return this; + } + + @NotNull + public Optional getExternalId() + { + return Optional.ofNullable(externalId); + } + + @Config("elasticsearch.aws.external-id") + @ConfigDescription("Optional external id to pass to AWS STS while assuming a role") + public AwsSecurityConfig setExternalId(String externalId) + { + this.externalId = externalId; + return this; + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/BuiltinColumns.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/BuiltinColumns.java new file mode 100644 index 0000000000000..0af3e0fe9b950 --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/BuiltinColumns.java @@ -0,0 +1,91 @@ +/* + * 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.elasticsearch; + +import io.trino.plugin.elasticsearch.decoders.IdColumnDecoder; +import io.trino.plugin.elasticsearch.decoders.ScoreColumnDecoder; +import io.trino.plugin.elasticsearch.decoders.SourceColumnDecoder; +import io.trino.spi.connector.ColumnHandle; +import io.trino.spi.connector.ColumnMetadata; +import io.trino.spi.type.Type; + +import java.util.Map; +import java.util.Optional; + +import static com.google.common.collect.ImmutableMap.toImmutableMap; +import static io.trino.spi.type.RealType.REAL; +import static io.trino.spi.type.VarcharType.VARCHAR; +import static java.util.Arrays.stream; +import static java.util.function.Function.identity; + +enum BuiltinColumns +{ + ID("_id", VARCHAR, new IdColumnDecoder.Descriptor(), true), + SOURCE("_source", VARCHAR, new SourceColumnDecoder.Descriptor(), false), + SCORE("_score", REAL, new ScoreColumnDecoder.Descriptor(), false); + + private static final Map COLUMNS_BY_NAME = stream(values()) + .collect(toImmutableMap(BuiltinColumns::getName, identity())); + + private final String name; + private final Type type; + private final DecoderDescriptor decoderDescriptor; + private final boolean supportsPredicates; + + BuiltinColumns(String name, Type type, DecoderDescriptor decoderDescriptor, boolean supportsPredicates) + { + this.name = name; + this.type = type; + this.decoderDescriptor = decoderDescriptor; + this.supportsPredicates = supportsPredicates; + } + + public static Optional of(String name) + { + return Optional.ofNullable(COLUMNS_BY_NAME.get(name)); + } + + public static boolean isBuiltinColumn(String name) + { + return COLUMNS_BY_NAME.containsKey(name); + } + + public String getName() + { + return name; + } + + public Type getType() + { + return type; + } + + public ColumnMetadata getMetadata() + { + return ColumnMetadata.builder() + .setName(name) + .setType(type) + .setHidden(true) + .build(); + } + + public ColumnHandle getColumnHandle() + { + return new ElasticsearchColumnHandle( + name, + type, + decoderDescriptor, + supportsPredicates); + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/CountQueryPageSource.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/CountQueryPageSource.java new file mode 100644 index 0000000000000..4b920b218e06b --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/CountQueryPageSource.java @@ -0,0 +1,93 @@ +/* + * 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.elasticsearch; + +import io.trino.plugin.elasticsearch.client.ElasticsearchClient; +import io.trino.spi.Page; +import io.trino.spi.connector.ConnectorPageSource; + +import static io.trino.plugin.elasticsearch.ElasticsearchQueryBuilder.buildSearchQuery; +import static java.lang.Math.toIntExact; +import static java.util.Objects.requireNonNull; + +class CountQueryPageSource + implements ConnectorPageSource +{ + // This implementation of the page source is used whenever a query doesn't reference any columns + // from the ES table. We need to limit the number of rows per page in case there are projections + // in the query that can cause page sizes to explode. For example: SELECT rand() FROM some_table + // TODO (https://github.com/trinodb/trino/issues/16824) allow connector to return pages of arbitrary row count and handle this gracefully in engine + private static final int BATCH_SIZE = 10000; + + private final long readTimeNanos; + private long remaining; + + public CountQueryPageSource(ElasticsearchClient client, ElasticsearchTableHandle table, ElasticsearchSplit split) + { + requireNonNull(client, "client is null"); + requireNonNull(table, "table is null"); + requireNonNull(split, "split is null"); + + long start = System.nanoTime(); + long count = client.count( + split.getIndex(), + split.getShard(), + buildSearchQuery(table.getConstraint().transformKeys(ElasticsearchColumnHandle.class::cast), table.getQuery(), table.getRegexes())); + readTimeNanos = System.nanoTime() - start; + + if (table.getLimit().isPresent()) { + count = Math.min(table.getLimit().getAsLong(), count); + } + + remaining = count; + } + + @Override + public boolean isFinished() + { + return remaining == 0; + } + + @Override + public Page getNextPage() + { + int batch = toIntExact(Math.min(BATCH_SIZE, remaining)); + remaining -= batch; + + return new Page(batch); + } + + @Override + public long getReadTimeNanos() + { + return readTimeNanos; + } + + @Override + public long getCompletedBytes() + { + return 0; + } + + @Override + public long getMemoryUsage() + { + return 0; + } + + @Override + public void close() + { + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/DecoderDescriptor.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/DecoderDescriptor.java new file mode 100644 index 0000000000000..adadf4036a0c1 --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/DecoderDescriptor.java @@ -0,0 +1,62 @@ +/* + * 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.elasticsearch; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import io.trino.plugin.elasticsearch.decoders.ArrayDecoder; +import io.trino.plugin.elasticsearch.decoders.BigintDecoder; +import io.trino.plugin.elasticsearch.decoders.BooleanDecoder; +import io.trino.plugin.elasticsearch.decoders.Decoder; +import io.trino.plugin.elasticsearch.decoders.DoubleDecoder; +import io.trino.plugin.elasticsearch.decoders.IdColumnDecoder; +import io.trino.plugin.elasticsearch.decoders.IntegerDecoder; +import io.trino.plugin.elasticsearch.decoders.IpAddressDecoder; +import io.trino.plugin.elasticsearch.decoders.RawJsonDecoder; +import io.trino.plugin.elasticsearch.decoders.RealDecoder; +import io.trino.plugin.elasticsearch.decoders.RowDecoder; +import io.trino.plugin.elasticsearch.decoders.ScoreColumnDecoder; +import io.trino.plugin.elasticsearch.decoders.SmallintDecoder; +import io.trino.plugin.elasticsearch.decoders.SourceColumnDecoder; +import io.trino.plugin.elasticsearch.decoders.TimestampDecoder; +import io.trino.plugin.elasticsearch.decoders.TinyintDecoder; +import io.trino.plugin.elasticsearch.decoders.VarbinaryDecoder; +import io.trino.plugin.elasticsearch.decoders.VarcharDecoder; + +@JsonTypeInfo( + use = JsonTypeInfo.Id.NAME, + property = "@type") +@JsonSubTypes({ + @JsonSubTypes.Type(value = BooleanDecoder.Descriptor.class, name = "boolean"), + @JsonSubTypes.Type(value = SmallintDecoder.Descriptor.class, name = "smallint"), + @JsonSubTypes.Type(value = TinyintDecoder.Descriptor.class, name = "tinyint"), + @JsonSubTypes.Type(value = IntegerDecoder.Descriptor.class, name = "integer"), + @JsonSubTypes.Type(value = BigintDecoder.Descriptor.class, name = "bigint"), + @JsonSubTypes.Type(value = TimestampDecoder.Descriptor.class, name = "timestamp"), + @JsonSubTypes.Type(value = RealDecoder.Descriptor.class, name = "real"), + @JsonSubTypes.Type(value = DoubleDecoder.Descriptor.class, name = "double"), + @JsonSubTypes.Type(value = VarcharDecoder.Descriptor.class, name = "varchar"), + @JsonSubTypes.Type(value = VarbinaryDecoder.Descriptor.class, name = "varbinary"), + @JsonSubTypes.Type(value = IpAddressDecoder.Descriptor.class, name = "ipAddress"), + @JsonSubTypes.Type(value = RowDecoder.Descriptor.class, name = "row"), + @JsonSubTypes.Type(value = ArrayDecoder.Descriptor.class, name = "array"), + @JsonSubTypes.Type(value = RawJsonDecoder.Descriptor.class, name = "rawJson"), + @JsonSubTypes.Type(value = IdColumnDecoder.Descriptor.class, name = "idColumn"), + @JsonSubTypes.Type(value = ScoreColumnDecoder.Descriptor.class, name = "scoreColumn"), + @JsonSubTypes.Type(value = SourceColumnDecoder.Descriptor.class, name = "sourceColumn"), +}) +public interface DecoderDescriptor +{ + Decoder createDecoder(); +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchColumnHandle.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchColumnHandle.java new file mode 100644 index 0000000000000..8b22b3cb1bdf3 --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchColumnHandle.java @@ -0,0 +1,98 @@ +/* + * 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.elasticsearch; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.trino.spi.connector.ColumnHandle; +import io.trino.spi.type.Type; + +import java.util.Objects; + +import static java.util.Objects.requireNonNull; + +public final class ElasticsearchColumnHandle + implements ColumnHandle +{ + private final String name; + private final Type type; + private final DecoderDescriptor decoderDescriptor; + private final boolean supportsPredicates; + + @JsonCreator + public ElasticsearchColumnHandle( + @JsonProperty("name") String name, + @JsonProperty("type") Type type, + @JsonProperty("decoderDescriptor") DecoderDescriptor decoderDescriptor, + @JsonProperty("supportsPredicates") boolean supportsPredicates) + { + this.name = requireNonNull(name, "name is null"); + this.type = requireNonNull(type, "type is null"); + this.decoderDescriptor = requireNonNull(decoderDescriptor, "decoderDescriptor is null"); + this.supportsPredicates = supportsPredicates; + } + + @JsonProperty + public String getName() + { + return name; + } + + @JsonProperty + public Type getType() + { + return type; + } + + @JsonProperty + public DecoderDescriptor getDecoderDescriptor() + { + return decoderDescriptor; + } + + @JsonProperty + public boolean isSupportsPredicates() + { + return supportsPredicates; + } + + @Override + public int hashCode() + { + return Objects.hash(name, type, decoderDescriptor, supportsPredicates); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if ((obj == null) || (getClass() != obj.getClass())) { + return false; + } + + ElasticsearchColumnHandle other = (ElasticsearchColumnHandle) obj; + return this.supportsPredicates == other.supportsPredicates && + Objects.equals(this.getName(), other.getName()) && + Objects.equals(this.getType(), other.getType()) && + Objects.equals(this.getDecoderDescriptor(), other.getDecoderDescriptor()); + } + + @Override + public String toString() + { + return getName() + "::" + getType(); + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchConfig.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchConfig.java new file mode 100644 index 0000000000000..339dd233e6af5 --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchConfig.java @@ -0,0 +1,360 @@ +/* + * 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.elasticsearch; + +import io.airlift.configuration.Config; +import io.airlift.configuration.ConfigDescription; +import io.airlift.configuration.ConfigSecuritySensitive; +import io.airlift.configuration.DefunctConfig; +import io.airlift.configuration.validation.FileExists; +import io.airlift.units.Duration; +import io.airlift.units.MinDuration; +import jakarta.validation.constraints.Min; +import jakarta.validation.constraints.NotNull; + +import java.io.File; +import java.util.List; +import java.util.Optional; + +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static java.util.concurrent.TimeUnit.MINUTES; +import static java.util.concurrent.TimeUnit.SECONDS; + +@DefunctConfig({ + "elasticsearch.max-hits", + "elasticsearch.cluster-name", + "searchguard.ssl.certificate-format", + "searchguard.ssl.pemcert-filepath", + "searchguard.ssl.pemkey-filepath", + "searchguard.ssl.pemkey-password", + "searchguard.ssl.pemtrustedcas-filepath", + "searchguard.ssl.keystore-filepath", + "searchguard.ssl.keystore-password", + "searchguard.ssl.truststore-filepath", + "searchguard.ssl.truststore-password", + "elasticsearch.table-description-directory", + "elasticsearch.max-request-retries", + "elasticsearch.max-request-retry-time"}) +public class ElasticsearchConfig +{ + public enum Security + { + AWS, + PASSWORD, + } + + private List hosts; + private int port = 9200; + private String defaultSchema = "default"; + private int scrollSize = 1_000; + private Duration scrollTimeout = new Duration(1, MINUTES); + private Duration requestTimeout = new Duration(10, SECONDS); + private Duration connectTimeout = new Duration(1, SECONDS); + private Duration backoffInitDelay = new Duration(500, MILLISECONDS); + private Duration backoffMaxDelay = new Duration(20, SECONDS); + private Duration maxRetryTime = new Duration(30, SECONDS); + private Duration nodeRefreshInterval = new Duration(1, MINUTES); + private int maxHttpConnections = 25; + private int httpThreadCount = Runtime.getRuntime().availableProcessors(); + + private boolean tlsEnabled; + private File keystorePath; + private File trustStorePath; + private String keystorePassword; + private String truststorePassword; + private boolean ignorePublishAddress; + private boolean verifyHostnames = true; + + private Security security; + + @NotNull + public List getHosts() + { + return hosts; + } + + @Config("elasticsearch.host") + public ElasticsearchConfig setHosts(List hosts) + { + this.hosts = hosts; + return this; + } + + public int getPort() + { + return port; + } + + @Config("elasticsearch.port") + public ElasticsearchConfig setPort(int port) + { + this.port = port; + return this; + } + + @NotNull + public String getDefaultSchema() + { + return defaultSchema; + } + + @Config("elasticsearch.default-schema-name") + @ConfigDescription("Default schema name to use") + public ElasticsearchConfig setDefaultSchema(String defaultSchema) + { + this.defaultSchema = defaultSchema; + return this; + } + + @NotNull + @Min(1) + public int getScrollSize() + { + return scrollSize; + } + + @Config("elasticsearch.scroll-size") + @ConfigDescription("Scroll batch size") + public ElasticsearchConfig setScrollSize(int scrollSize) + { + this.scrollSize = scrollSize; + return this; + } + + @NotNull + public Duration getScrollTimeout() + { + return scrollTimeout; + } + + @Config("elasticsearch.scroll-timeout") + @ConfigDescription("Scroll timeout") + public ElasticsearchConfig setScrollTimeout(Duration scrollTimeout) + { + this.scrollTimeout = scrollTimeout; + return this; + } + + @NotNull + public Duration getRequestTimeout() + { + return requestTimeout; + } + + @Config("elasticsearch.request-timeout") + @ConfigDescription("Elasticsearch request timeout") + public ElasticsearchConfig setRequestTimeout(Duration requestTimeout) + { + this.requestTimeout = requestTimeout; + return this; + } + + @NotNull + public Duration getConnectTimeout() + { + return connectTimeout; + } + + @Config("elasticsearch.connect-timeout") + @ConfigDescription("Elasticsearch connect timeout") + public ElasticsearchConfig setConnectTimeout(Duration timeout) + { + this.connectTimeout = timeout; + return this; + } + + @NotNull + public Duration getBackoffInitDelay() + { + return backoffInitDelay; + } + + @Config("elasticsearch.backoff-init-delay") + @ConfigDescription("Initial delay to wait between backpressure retries") + public ElasticsearchConfig setBackoffInitDelay(Duration backoffInitDelay) + { + this.backoffInitDelay = backoffInitDelay; + return this; + } + + @NotNull + public Duration getBackoffMaxDelay() + { + return backoffMaxDelay; + } + + @Config("elasticsearch.backoff-max-delay") + @ConfigDescription("Maximum delay to wait between backpressure retries") + public ElasticsearchConfig setBackoffMaxDelay(Duration backoffMaxDelay) + { + this.backoffMaxDelay = backoffMaxDelay; + return this; + } + + @NotNull + public Duration getMaxRetryTime() + { + return maxRetryTime; + } + + @Config("elasticsearch.max-retry-time") + @ConfigDescription("Maximum timeout in case of multiple retries") + public ElasticsearchConfig setMaxRetryTime(Duration maxRetryTime) + { + this.maxRetryTime = maxRetryTime; + return this; + } + + @NotNull + @MinDuration("1ms") + public Duration getNodeRefreshInterval() + { + return nodeRefreshInterval; + } + + @Config("elasticsearch.node-refresh-interval") + @ConfigDescription("How often to refresh the list of available Elasticsearch nodes") + public ElasticsearchConfig setNodeRefreshInterval(Duration nodeRefreshInterval) + { + this.nodeRefreshInterval = nodeRefreshInterval; + return this; + } + + @Config("elasticsearch.max-http-connections") + @ConfigDescription("Maximum number of persistent HTTP connections to Elasticsearch") + public ElasticsearchConfig setMaxHttpConnections(int size) + { + this.maxHttpConnections = size; + return this; + } + + @NotNull + public int getMaxHttpConnections() + { + return maxHttpConnections; + } + + @Config("elasticsearch.http-thread-count") + @ConfigDescription("Number of threads handling HTTP connections to Elasticsearch") + public ElasticsearchConfig setHttpThreadCount(int count) + { + this.httpThreadCount = count; + return this; + } + + @NotNull + public int getHttpThreadCount() + { + return httpThreadCount; + } + + public boolean isTlsEnabled() + { + return tlsEnabled; + } + + @Config("elasticsearch.tls.enabled") + public ElasticsearchConfig setTlsEnabled(boolean tlsEnabled) + { + this.tlsEnabled = tlsEnabled; + return this; + } + + public Optional<@FileExists File> getKeystorePath() + { + return Optional.ofNullable(keystorePath); + } + + @Config("elasticsearch.tls.keystore-path") + public ElasticsearchConfig setKeystorePath(File path) + { + this.keystorePath = path; + return this; + } + + public Optional getKeystorePassword() + { + return Optional.ofNullable(keystorePassword); + } + + @Config("elasticsearch.tls.keystore-password") + @ConfigSecuritySensitive + public ElasticsearchConfig setKeystorePassword(String password) + { + this.keystorePassword = password; + return this; + } + + public Optional<@FileExists File> getTrustStorePath() + { + return Optional.ofNullable(trustStorePath); + } + + @Config("elasticsearch.tls.truststore-path") + public ElasticsearchConfig setTrustStorePath(File path) + { + this.trustStorePath = path; + return this; + } + + public Optional getTruststorePassword() + { + return Optional.ofNullable(truststorePassword); + } + + @Config("elasticsearch.tls.truststore-password") + @ConfigSecuritySensitive + public ElasticsearchConfig setTruststorePassword(String password) + { + this.truststorePassword = password; + return this; + } + + public boolean isVerifyHostnames() + { + return verifyHostnames; + } + + @Config("elasticsearch.tls.verify-hostnames") + public ElasticsearchConfig setVerifyHostnames(boolean verify) + { + this.verifyHostnames = verify; + return this; + } + + public boolean isIgnorePublishAddress() + { + return ignorePublishAddress; + } + + @Config("elasticsearch.ignore-publish-address") + public ElasticsearchConfig setIgnorePublishAddress(boolean ignorePublishAddress) + { + this.ignorePublishAddress = ignorePublishAddress; + return this; + } + + @NotNull + public Optional getSecurity() + { + return Optional.ofNullable(security); + } + + @Config("elasticsearch.security") + public ElasticsearchConfig setSecurity(Security security) + { + this.security = security; + return this; + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchConnector.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchConnector.java new file mode 100644 index 0000000000000..6014954e6acf7 --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchConnector.java @@ -0,0 +1,104 @@ +/* + * 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.elasticsearch; + +import com.google.common.collect.ImmutableSet; +import com.google.inject.Inject; +import io.airlift.bootstrap.LifeCycleManager; +import io.trino.spi.connector.Connector; +import io.trino.spi.connector.ConnectorMetadata; +import io.trino.spi.connector.ConnectorPageSourceProvider; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.ConnectorSplitManager; +import io.trino.spi.connector.ConnectorTransactionHandle; +import io.trino.spi.connector.SystemTable; +import io.trino.spi.function.table.ConnectorTableFunction; +import io.trino.spi.transaction.IsolationLevel; + +import java.util.Set; + +import static io.trino.spi.transaction.IsolationLevel.READ_COMMITTED; +import static io.trino.spi.transaction.IsolationLevel.checkConnectorSupports; +import static java.util.Objects.requireNonNull; + +public class ElasticsearchConnector + implements Connector +{ + private final LifeCycleManager lifeCycleManager; + private final ElasticsearchMetadata metadata; + private final ElasticsearchSplitManager splitManager; + private final ElasticsearchPageSourceProvider pageSourceProvider; + private final NodesSystemTable nodesSystemTable; + private final Set connectorTableFunctions; + + @Inject + public ElasticsearchConnector( + LifeCycleManager lifeCycleManager, + ElasticsearchMetadata metadata, + ElasticsearchSplitManager splitManager, + ElasticsearchPageSourceProvider pageSourceProvider, + NodesSystemTable nodesSystemTable, + Set connectorTableFunctions) + { + this.lifeCycleManager = requireNonNull(lifeCycleManager, "lifeCycleManager is null"); + this.metadata = requireNonNull(metadata, "metadata is null"); + this.splitManager = requireNonNull(splitManager, "splitManager is null"); + this.pageSourceProvider = requireNonNull(pageSourceProvider, "pageSourceProvider is null"); + this.nodesSystemTable = requireNonNull(nodesSystemTable, "nodesSystemTable is null"); + this.connectorTableFunctions = ImmutableSet.copyOf(requireNonNull(connectorTableFunctions, "connectorTableFunctions is null")); + } + + @Override + public ConnectorTransactionHandle beginTransaction(IsolationLevel isolationLevel, boolean readOnly, boolean autoCommit) + { + checkConnectorSupports(READ_COMMITTED, isolationLevel); + return ElasticsearchTransactionHandle.INSTANCE; + } + + @Override + public ConnectorMetadata getMetadata(ConnectorSession session, ConnectorTransactionHandle transactionHandle) + { + return metadata; + } + + @Override + public ConnectorSplitManager getSplitManager() + { + return splitManager; + } + + @Override + public ConnectorPageSourceProvider getPageSourceProvider() + { + return pageSourceProvider; + } + + @Override + public Set getSystemTables() + { + return ImmutableSet.of(nodesSystemTable); + } + + @Override + public Set getTableFunctions() + { + return connectorTableFunctions; + } + + @Override + public final void shutdown() + { + lifeCycleManager.stop(); + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchConnectorFactory.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchConnectorFactory.java new file mode 100644 index 0000000000000..fcf3383362701 --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchConnectorFactory.java @@ -0,0 +1,71 @@ +/* + * 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.elasticsearch; + +import com.google.inject.Injector; +import io.airlift.bootstrap.Bootstrap; +import io.airlift.json.JsonModule; +import io.trino.plugin.base.CatalogName; +import io.trino.plugin.base.TypeDeserializerModule; +import io.trino.plugin.base.jmx.ConnectorObjectNameGeneratorModule; +import io.trino.plugin.base.jmx.MBeanServerModule; +import io.trino.spi.NodeManager; +import io.trino.spi.connector.Connector; +import io.trino.spi.connector.ConnectorContext; +import io.trino.spi.connector.ConnectorFactory; +import org.weakref.jmx.guice.MBeanModule; + +import java.util.Map; + +import static io.trino.plugin.base.Versions.checkStrictSpiVersionMatch; +import static java.util.Objects.requireNonNull; + +public class ElasticsearchConnectorFactory + implements ConnectorFactory +{ + ElasticsearchConnectorFactory() {} + + @Override + public String getName() + { + return "elasticsearch"; + } + + @Override + public Connector create(String catalogName, Map config, ConnectorContext context) + { + requireNonNull(catalogName, "catalogName is null"); + requireNonNull(config, "config is null"); + checkStrictSpiVersionMatch(context, this); + + Bootstrap app = new Bootstrap( + new MBeanModule(), + new MBeanServerModule(), + new ConnectorObjectNameGeneratorModule("io.trino.plugin.elasticsearch", "trino.plugin.elasticsearch"), + new JsonModule(), + new TypeDeserializerModule(context.getTypeManager()), + new ElasticsearchConnectorModule(), + binder -> { + binder.bind(NodeManager.class).toInstance(context.getNodeManager()); + binder.bind(CatalogName.class).toInstance(new CatalogName(catalogName)); + }); + + Injector injector = app + .doNotInitializeLogging() + .setRequiredConfigurationProperties(config) + .initialize(); + + return injector.getInstance(ElasticsearchConnector.class); + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchConnectorModule.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchConnectorModule.java new file mode 100644 index 0000000000000..2cda1656af15b --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchConnectorModule.java @@ -0,0 +1,68 @@ +/* + * 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.elasticsearch; + +import com.google.inject.Binder; +import com.google.inject.Scopes; +import io.airlift.configuration.AbstractConfigurationAwareModule; +import io.trino.plugin.elasticsearch.client.ElasticsearchClient; +import io.trino.plugin.elasticsearch.ptf.RawQuery; +import io.trino.spi.function.table.ConnectorTableFunction; + +import static com.google.inject.multibindings.Multibinder.newSetBinder; +import static com.google.inject.multibindings.OptionalBinder.newOptionalBinder; +import static io.airlift.configuration.ConditionalModule.conditionalModule; +import static io.airlift.configuration.ConfigBinder.configBinder; +import static io.trino.plugin.elasticsearch.ElasticsearchConfig.Security.AWS; +import static io.trino.plugin.elasticsearch.ElasticsearchConfig.Security.PASSWORD; +import static java.util.function.Predicate.isEqual; +import static org.weakref.jmx.guice.ExportBinder.newExporter; + +public class ElasticsearchConnectorModule + extends AbstractConfigurationAwareModule +{ + @Override + protected void setup(Binder binder) + { + binder.bind(ElasticsearchConnector.class).in(Scopes.SINGLETON); + binder.bind(ElasticsearchMetadata.class).in(Scopes.SINGLETON); + binder.bind(ElasticsearchSplitManager.class).in(Scopes.SINGLETON); + binder.bind(ElasticsearchPageSourceProvider.class).in(Scopes.SINGLETON); + binder.bind(ElasticsearchClient.class).in(Scopes.SINGLETON); + binder.bind(NodesSystemTable.class).in(Scopes.SINGLETON); + + newExporter(binder).export(ElasticsearchClient.class).withGeneratedName(); + + configBinder(binder).bindConfig(ElasticsearchConfig.class); + + newOptionalBinder(binder, AwsSecurityConfig.class); + newOptionalBinder(binder, PasswordConfig.class); + + newSetBinder(binder, ConnectorTableFunction.class).addBinding().toProvider(RawQuery.class).in(Scopes.SINGLETON); + + install(conditionalModule( + ElasticsearchConfig.class, + config -> config.getSecurity() + .filter(isEqual(AWS)) + .isPresent(), + conditionalBinder -> configBinder(conditionalBinder).bindConfig(AwsSecurityConfig.class))); + + install(conditionalModule( + ElasticsearchConfig.class, + config -> config.getSecurity() + .filter(isEqual(PASSWORD)) + .isPresent(), + conditionalBinder -> configBinder(conditionalBinder).bindConfig(PasswordConfig.class))); + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchErrorCode.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchErrorCode.java new file mode 100644 index 0000000000000..59af305ae31dd --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchErrorCode.java @@ -0,0 +1,44 @@ +/* + * 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.elasticsearch; + +import io.trino.spi.ErrorCode; +import io.trino.spi.ErrorCodeSupplier; +import io.trino.spi.ErrorType; + +import static io.trino.spi.ErrorType.EXTERNAL; +import static io.trino.spi.ErrorType.USER_ERROR; + +public enum ElasticsearchErrorCode + implements ErrorCodeSupplier +{ + ELASTICSEARCH_CONNECTION_ERROR(0, EXTERNAL), + ELASTICSEARCH_INVALID_RESPONSE(1, EXTERNAL), + ELASTICSEARCH_SSL_INITIALIZATION_FAILURE(2, EXTERNAL), + ELASTICSEARCH_QUERY_FAILURE(3, USER_ERROR), + ELASTICSEARCH_INVALID_METADATA(4, USER_ERROR); + + private final ErrorCode errorCode; + + ElasticsearchErrorCode(int code, ErrorType type) + { + errorCode = new ErrorCode(code + 0x0503_0000, name(), type); + } + + @Override + public ErrorCode toErrorCode() + { + return errorCode; + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchMetadata.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchMetadata.java new file mode 100644 index 0000000000000..ef06f4372aaef --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchMetadata.java @@ -0,0 +1,716 @@ +/* + * 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.elasticsearch; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.inject.Inject; +import io.airlift.slice.Slice; +import io.trino.plugin.base.expression.ConnectorExpressions; +import io.trino.plugin.elasticsearch.client.ElasticsearchClient; +import io.trino.plugin.elasticsearch.client.IndexMetadata; +import io.trino.plugin.elasticsearch.client.IndexMetadata.DateTimeType; +import io.trino.plugin.elasticsearch.client.IndexMetadata.ObjectType; +import io.trino.plugin.elasticsearch.client.IndexMetadata.PrimitiveType; +import io.trino.plugin.elasticsearch.client.IndexMetadata.ScaledFloatType; +import io.trino.plugin.elasticsearch.decoders.ArrayDecoder; +import io.trino.plugin.elasticsearch.decoders.BigintDecoder; +import io.trino.plugin.elasticsearch.decoders.BooleanDecoder; +import io.trino.plugin.elasticsearch.decoders.DoubleDecoder; +import io.trino.plugin.elasticsearch.decoders.IntegerDecoder; +import io.trino.plugin.elasticsearch.decoders.IpAddressDecoder; +import io.trino.plugin.elasticsearch.decoders.RawJsonDecoder; +import io.trino.plugin.elasticsearch.decoders.RealDecoder; +import io.trino.plugin.elasticsearch.decoders.RowDecoder; +import io.trino.plugin.elasticsearch.decoders.SmallintDecoder; +import io.trino.plugin.elasticsearch.decoders.TimestampDecoder; +import io.trino.plugin.elasticsearch.decoders.TinyintDecoder; +import io.trino.plugin.elasticsearch.decoders.VarbinaryDecoder; +import io.trino.plugin.elasticsearch.decoders.VarcharDecoder; +import io.trino.plugin.elasticsearch.ptf.RawQuery.RawQueryFunctionHandle; +import io.trino.spi.TrinoException; +import io.trino.spi.connector.ColumnHandle; +import io.trino.spi.connector.ColumnMetadata; +import io.trino.spi.connector.ConnectorMetadata; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.ConnectorTableHandle; +import io.trino.spi.connector.ConnectorTableMetadata; +import io.trino.spi.connector.ConnectorTableProperties; +import io.trino.spi.connector.Constraint; +import io.trino.spi.connector.ConstraintApplicationResult; +import io.trino.spi.connector.LimitApplicationResult; +import io.trino.spi.connector.SchemaTableName; +import io.trino.spi.connector.SchemaTablePrefix; +import io.trino.spi.connector.TableColumnsMetadata; +import io.trino.spi.connector.TableFunctionApplicationResult; +import io.trino.spi.expression.Call; +import io.trino.spi.expression.ConnectorExpression; +import io.trino.spi.expression.Constant; +import io.trino.spi.expression.Variable; +import io.trino.spi.function.table.ConnectorTableFunctionHandle; +import io.trino.spi.predicate.Domain; +import io.trino.spi.predicate.TupleDomain; +import io.trino.spi.type.ArrayType; +import io.trino.spi.type.RowType; +import io.trino.spi.type.StandardTypes; +import io.trino.spi.type.Type; +import io.trino.spi.type.TypeManager; +import io.trino.spi.type.TypeSignature; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.OptionalLong; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Verify.verifyNotNull; +import static com.google.common.collect.ImmutableList.toImmutableList; +import static com.google.common.collect.ImmutableSet.toImmutableSet; +import static com.google.common.collect.Iterators.singletonIterator; +import static io.airlift.slice.SliceUtf8.getCodePointAt; +import static io.airlift.slice.SliceUtf8.lengthOfCodePoint; +import static io.trino.plugin.elasticsearch.ElasticsearchTableHandle.Type.QUERY; +import static io.trino.plugin.elasticsearch.ElasticsearchTableHandle.Type.SCAN; +import static io.trino.spi.StandardErrorCode.INVALID_FUNCTION_ARGUMENT; +import static io.trino.spi.expression.StandardFunctions.LIKE_FUNCTION_NAME; +import static io.trino.spi.type.BigintType.BIGINT; +import static io.trino.spi.type.BooleanType.BOOLEAN; +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.TimestampType.TIMESTAMP_MILLIS; +import static io.trino.spi.type.TinyintType.TINYINT; +import static io.trino.spi.type.VarbinaryType.VARBINARY; +import static io.trino.spi.type.VarcharType.VARCHAR; +import static java.lang.String.format; +import static java.util.Collections.emptyIterator; +import static java.util.Locale.ENGLISH; +import static java.util.Objects.requireNonNull; + +public class ElasticsearchMetadata + implements ConnectorMetadata +{ + private static final String PASSTHROUGH_QUERY_RESULT_COLUMN_NAME = "result"; + private static final ColumnMetadata PASSTHROUGH_QUERY_RESULT_COLUMN_METADATA = ColumnMetadata.builder() + .setName(PASSTHROUGH_QUERY_RESULT_COLUMN_NAME) + .setType(VARCHAR) + .setNullable(true) + .setHidden(false) + .build(); + + private static final Map PASSTHROUGH_QUERY_COLUMNS = ImmutableMap.of( + PASSTHROUGH_QUERY_RESULT_COLUMN_NAME, + new ElasticsearchColumnHandle( + PASSTHROUGH_QUERY_RESULT_COLUMN_NAME, + VARCHAR, + new VarcharDecoder.Descriptor(PASSTHROUGH_QUERY_RESULT_COLUMN_NAME), + false)); + + // See https://www.elastic.co/guide/en/elasticsearch/reference/current/regexp-syntax.html + private static final Set REGEXP_RESERVED_CHARACTERS = IntStream.of('.', '?', '+', '*', '|', '{', '}', '[', ']', '(', ')', '"', '#', '@', '&', '<', '>', '~') + .boxed() + .collect(toImmutableSet()); + + private final Type ipAddressType; + private final ElasticsearchClient client; + private final String schemaName; + + @Inject + public ElasticsearchMetadata(TypeManager typeManager, ElasticsearchClient client, ElasticsearchConfig config) + { + this.ipAddressType = typeManager.getType(new TypeSignature(StandardTypes.IPADDRESS)); + this.client = requireNonNull(client, "client is null"); + this.schemaName = config.getDefaultSchema(); + } + + @Override + public List listSchemaNames(ConnectorSession session) + { + return ImmutableList.of(schemaName); + } + + @Override + public ElasticsearchTableHandle getTableHandle(ConnectorSession session, SchemaTableName tableName) + { + requireNonNull(tableName, "tableName is null"); + + if (tableName.getSchemaName().equals(schemaName)) { + String[] parts = tableName.getTableName().split(":", 2); + String table = parts[0]; + Optional query = Optional.empty(); + if (parts.length == 2) { + query = Optional.of(parts[1]); + } + + if (client.indexExists(table) && !client.getIndexMetadata(table).getSchema().getFields().isEmpty()) { + return new ElasticsearchTableHandle(SCAN, schemaName, table, query); + } + } + + return null; + } + + @Override + public ConnectorTableMetadata getTableMetadata(ConnectorSession session, ConnectorTableHandle table) + { + ElasticsearchTableHandle handle = (ElasticsearchTableHandle) table; + + if (isPassthroughQuery(handle)) { + return new ConnectorTableMetadata( + new SchemaTableName(handle.getSchema(), handle.getIndex()), + ImmutableList.of(PASSTHROUGH_QUERY_RESULT_COLUMN_METADATA)); + } + return getTableMetadata(handle.getSchema(), handle.getIndex()); + } + + private ConnectorTableMetadata getTableMetadata(String schemaName, String tableName) + { + InternalTableMetadata internalTableMetadata = makeInternalTableMetadata(schemaName, tableName); + return new ConnectorTableMetadata(new SchemaTableName(schemaName, tableName), internalTableMetadata.getColumnMetadata()); + } + + private InternalTableMetadata makeInternalTableMetadata(ConnectorTableHandle table) + { + ElasticsearchTableHandle handle = (ElasticsearchTableHandle) table; + return makeInternalTableMetadata(handle.getSchema(), handle.getIndex()); + } + + private InternalTableMetadata makeInternalTableMetadata(String schema, String tableName) + { + IndexMetadata metadata = client.getIndexMetadata(tableName); + List fields = getColumnFields(metadata); + return new InternalTableMetadata(new SchemaTableName(schema, tableName), makeColumnMetadata(fields), makeColumnHandles(fields)); + } + + private List getColumnFields(IndexMetadata metadata) + { + Map counts = metadata.getSchema() + .getFields().stream() + .collect(Collectors.groupingBy(f -> f.getName().toLowerCase(ENGLISH), Collectors.counting())); + + return metadata.getSchema().getFields().stream() + .filter(field -> toTrino(field) != null && counts.get(field.getName().toLowerCase(ENGLISH)) <= 1) + .collect(toImmutableList()); + } + + private List makeColumnMetadata(List fields) + { + ImmutableList.Builder result = ImmutableList.builder(); + + for (BuiltinColumns builtinColumn : BuiltinColumns.values()) { + result.add(builtinColumn.getMetadata()); + } + + for (IndexMetadata.Field field : fields) { + result.add(ColumnMetadata.builder() + .setName(field.getName()) + .setType(toTrino(field).getType()) + .build()); + } + return result.build(); + } + + private Map makeColumnHandles(List fields) + { + ImmutableMap.Builder result = ImmutableMap.builder(); + + for (BuiltinColumns builtinColumn : BuiltinColumns.values()) { + result.put(builtinColumn.getName(), builtinColumn.getColumnHandle()); + } + + for (IndexMetadata.Field field : fields) { + TypeAndDecoder converted = toTrino(field); + result.put(field.getName(), new ElasticsearchColumnHandle( + field.getName(), + converted.getType(), + converted.getDecoderDescriptor(), + supportsPredicates(field.getType()))); + } + + return result.buildOrThrow(); + } + + private static boolean supportsPredicates(IndexMetadata.Type type) + { + if (type instanceof DateTimeType) { + return true; + } + + if (type instanceof PrimitiveType) { + switch (((PrimitiveType) type).getName().toLowerCase(ENGLISH)) { + case "boolean": + case "byte": + case "short": + case "integer": + case "long": + case "double": + case "float": + case "keyword": + return true; + } + } + + return false; + } + + private TypeAndDecoder toTrino(IndexMetadata.Field field) + { + return toTrino("", field); + } + + private TypeAndDecoder toTrino(String prefix, IndexMetadata.Field field) + { + String path = appendPath(prefix, field.getName()); + + checkArgument(!field.asRawJson() || !field.isArray(), format("A column, (%s) cannot be declared as a Trino array and also be rendered as json.", path)); + + if (field.asRawJson()) { + return new TypeAndDecoder(VARCHAR, new RawJsonDecoder.Descriptor(path)); + } + + if (field.isArray()) { + TypeAndDecoder element = toTrino(path, elementField(field)); + return new TypeAndDecoder(new ArrayType(element.getType()), new ArrayDecoder.Descriptor(element.getDecoderDescriptor())); + } + + IndexMetadata.Type type = field.getType(); + if (type instanceof PrimitiveType primitiveType) { + switch (primitiveType.getName()) { + case "float": + return new TypeAndDecoder(REAL, new RealDecoder.Descriptor(path)); + case "double": + return new TypeAndDecoder(DOUBLE, new DoubleDecoder.Descriptor(path)); + case "byte": + return new TypeAndDecoder(TINYINT, new TinyintDecoder.Descriptor(path)); + case "short": + return new TypeAndDecoder(SMALLINT, new SmallintDecoder.Descriptor(path)); + case "integer": + return new TypeAndDecoder(INTEGER, new IntegerDecoder.Descriptor(path)); + case "long": + return new TypeAndDecoder(BIGINT, new BigintDecoder.Descriptor(path)); + case "text": + case "keyword": + return new TypeAndDecoder(VARCHAR, new VarcharDecoder.Descriptor(path)); + case "ip": + return new TypeAndDecoder(ipAddressType, new IpAddressDecoder.Descriptor(path, ipAddressType)); + case "boolean": + return new TypeAndDecoder(BOOLEAN, new BooleanDecoder.Descriptor(path)); + case "binary": + return new TypeAndDecoder(VARBINARY, new VarbinaryDecoder.Descriptor(path)); + } + } + else if (type instanceof ScaledFloatType) { + return new TypeAndDecoder(DOUBLE, new DoubleDecoder.Descriptor(path)); + } + else if (type instanceof DateTimeType dateTimeType) { + if (dateTimeType.getFormats().isEmpty()) { + return new TypeAndDecoder(TIMESTAMP_MILLIS, new TimestampDecoder.Descriptor(path)); + } + // otherwise, skip -- we don't support custom formats, yet + } + else if (type instanceof ObjectType objectType) { + ImmutableList.Builder rowFieldsBuilder = ImmutableList.builder(); + ImmutableList.Builder decoderFields = ImmutableList.builder(); + for (IndexMetadata.Field rowField : objectType.getFields()) { + String name = rowField.getName(); + TypeAndDecoder child = toTrino(path, rowField); + + if (child != null) { + decoderFields.add(new RowDecoder.NameAndDescriptor(name, child.getDecoderDescriptor())); + rowFieldsBuilder.add(RowType.field(name, child.getType())); + } + } + + List rowFields = rowFieldsBuilder.build(); + if (!rowFields.isEmpty()) { + return new TypeAndDecoder(RowType.from(rowFields), new RowDecoder.Descriptor(path, decoderFields.build())); + } + + // otherwise, skip -- row types must have at least 1 field + } + + return null; + } + + private static String appendPath(String base, String element) + { + if (base.isEmpty()) { + return element; + } + + return base + "." + element; + } + + public static IndexMetadata.Field elementField(IndexMetadata.Field field) + { + checkArgument(field.isArray(), "Cannot get element field from a non-array field"); + return new IndexMetadata.Field(field.asRawJson(), false, field.getName(), field.getType()); + } + + @Override + public List listTables(ConnectorSession session, Optional schemaName) + { + if (schemaName.isPresent() && !schemaName.get().equals(this.schemaName)) { + return ImmutableList.of(); + } + + ImmutableList.Builder result = ImmutableList.builder(); + Set indexes = ImmutableSet.copyOf(client.getIndexes()); + + indexes.stream() + .map(index -> new SchemaTableName(this.schemaName, index)) + .forEach(result::add); + + client.getAliases().entrySet().stream() + .filter(entry -> indexes.contains(entry.getKey())) + .flatMap(entry -> entry.getValue().stream() + .map(alias -> new SchemaTableName(this.schemaName, alias))) + .distinct() + .forEach(result::add); + + return result.build(); + } + + @Override + public Map getColumnHandles(ConnectorSession session, ConnectorTableHandle tableHandle) + { + ElasticsearchTableHandle table = (ElasticsearchTableHandle) tableHandle; + + if (isPassthroughQuery(table)) { + return PASSTHROUGH_QUERY_COLUMNS; + } + + InternalTableMetadata tableMetadata = makeInternalTableMetadata(tableHandle); + return tableMetadata.getColumnHandles(); + } + + @Override + public ColumnMetadata getColumnMetadata(ConnectorSession session, ConnectorTableHandle tableHandle, ColumnHandle columnHandle) + { + ElasticsearchTableHandle table = (ElasticsearchTableHandle) tableHandle; + ElasticsearchColumnHandle column = (ElasticsearchColumnHandle) columnHandle; + + if (isPassthroughQuery(table)) { + if (column.getName().equals(PASSTHROUGH_QUERY_RESULT_COLUMN_METADATA.getName())) { + return PASSTHROUGH_QUERY_RESULT_COLUMN_METADATA; + } + + throw new IllegalArgumentException(format("Unexpected column for table '%s$query': %s", table.getIndex(), column.getName())); + } + + return BuiltinColumns.of(column.getName()) + .map(BuiltinColumns::getMetadata) + .orElse(ColumnMetadata.builder() + .setName(column.getName()) + .setType(column.getType()) + .build()); + } + + @Override + public Map> listTableColumns(ConnectorSession session, SchemaTablePrefix prefix) + { + throw new UnsupportedOperationException("The deprecated listTableColumns is not supported because streamTableColumns is implemented instead"); + } + + @Override + public Iterator streamTableColumns(ConnectorSession session, SchemaTablePrefix prefix) + { + if (prefix.getSchema().isPresent() && !prefix.getSchema().get().equals(schemaName)) { + return emptyIterator(); + } + + if (prefix.getSchema().isPresent() && prefix.getTable().isPresent()) { + ConnectorTableMetadata metadata = getTableMetadata(prefix.getSchema().get(), prefix.getTable().get()); + return singletonIterator(TableColumnsMetadata.forTable(metadata.getTable(), metadata.getColumns())); + } + + return listTables(session, prefix.getSchema()).stream() + .map(name -> getTableMetadata(name.getSchemaName(), name.getTableName())) + .map(tableMetadata -> TableColumnsMetadata.forTable(tableMetadata.getTable(), tableMetadata.getColumns())) + .iterator(); + } + + @Override + public ConnectorTableProperties getTableProperties(ConnectorSession session, ConnectorTableHandle table) + { + ElasticsearchTableHandle handle = (ElasticsearchTableHandle) table; + + return new ConnectorTableProperties( + handle.getConstraint(), + Optional.empty(), + Optional.empty(), + ImmutableList.of()); + } + + @Override + public Optional> applyLimit(ConnectorSession session, ConnectorTableHandle table, long limit) + { + ElasticsearchTableHandle handle = (ElasticsearchTableHandle) table; + + if (isPassthroughQuery(handle)) { + // limit pushdown currently not supported passthrough query + return Optional.empty(); + } + + if (handle.getLimit().isPresent() && handle.getLimit().getAsLong() <= limit) { + return Optional.empty(); + } + + handle = new ElasticsearchTableHandle( + handle.getType(), + handle.getSchema(), + handle.getIndex(), + handle.getConstraint(), + handle.getRegexes(), + handle.getQuery(), + OptionalLong.of(limit)); + + return Optional.of(new LimitApplicationResult<>(handle, false, false)); + } + + @Override + public Optional> applyFilter(ConnectorSession session, ConnectorTableHandle table, Constraint constraint) + { + ElasticsearchTableHandle handle = (ElasticsearchTableHandle) table; + + if (isPassthroughQuery(handle)) { + // filter pushdown currently not supported for passthrough query + return Optional.empty(); + } + + Map supported = new HashMap<>(); + Map unsupported = new HashMap<>(); + Map domains = constraint.getSummary().getDomains().orElseThrow(() -> new IllegalArgumentException("constraint summary is NONE")); + for (Map.Entry entry : domains.entrySet()) { + ElasticsearchColumnHandle column = (ElasticsearchColumnHandle) entry.getKey(); + + if (column.isSupportsPredicates()) { + supported.put(column, entry.getValue()); + } + else { + unsupported.put(column, entry.getValue()); + } + } + + TupleDomain oldDomain = handle.getConstraint(); + TupleDomain newDomain = oldDomain.intersect(TupleDomain.withColumnDomains(supported)); + + ConnectorExpression oldExpression = constraint.getExpression(); + Map newRegexes = new HashMap<>(handle.getRegexes()); + List expressions = ConnectorExpressions.extractConjuncts(constraint.getExpression()); + List notHandledExpressions = new ArrayList<>(); + for (ConnectorExpression expression : expressions) { + if (expression instanceof Call call) { + if (isSupportedLikeCall(call)) { + List arguments = call.getArguments(); + String variableName = ((Variable) arguments.get(0)).getName(); + ElasticsearchColumnHandle column = (ElasticsearchColumnHandle) constraint.getAssignments().get(variableName); + verifyNotNull(column, "No assignment for %s", variableName); + String columnName = column.getName(); + Object pattern = ((Constant) arguments.get(1)).getValue(); + Optional escape = Optional.empty(); + if (arguments.size() == 3) { + escape = Optional.of((Slice) ((Constant) arguments.get(2)).getValue()); + } + + if (!newRegexes.containsKey(columnName) && pattern instanceof Slice) { + IndexMetadata metadata = client.getIndexMetadata(handle.getIndex()); + if (metadata.getSchema() + .getFields().stream() + .anyMatch(field -> columnName.equals(field.getName()) && field.getType() instanceof PrimitiveType && "keyword".equals(((PrimitiveType) field.getType()).getName()))) { + newRegexes.put(columnName, likeToRegexp((Slice) pattern, escape)); + continue; + } + } + } + } + notHandledExpressions.add(expression); + } + + ConnectorExpression newExpression = ConnectorExpressions.and(notHandledExpressions); + if (oldDomain.equals(newDomain) && oldExpression.equals(newExpression)) { + return Optional.empty(); + } + + handle = new ElasticsearchTableHandle( + handle.getType(), + handle.getSchema(), + handle.getIndex(), + newDomain, + newRegexes, + handle.getQuery(), + handle.getLimit()); + + return Optional.of(new ConstraintApplicationResult<>(handle, TupleDomain.withColumnDomains(unsupported), newExpression, false)); + } + + protected static boolean isSupportedLikeCall(Call call) + { + if (!LIKE_FUNCTION_NAME.equals(call.getFunctionName())) { + return false; + } + + List arguments = call.getArguments(); + if (arguments.size() < 2 || arguments.size() > 3) { + return false; + } + + if (!(arguments.get(0) instanceof Variable) || !(arguments.get(1) instanceof Constant)) { + return false; + } + + if (arguments.size() == 3) { + return arguments.get(2) instanceof Constant; + } + + return true; + } + + protected static String likeToRegexp(Slice pattern, Optional escape) + { + Optional escapeChar = escape.map(ElasticsearchMetadata::getEscapeChar); + StringBuilder regex = new StringBuilder(); + boolean escaped = false; + int position = 0; + while (position < pattern.length()) { + int currentChar = getCodePointAt(pattern, position); + position += lengthOfCodePoint(currentChar); + checkEscape(!escaped || currentChar == '%' || currentChar == '_' || currentChar == escapeChar.get()); + if (!escaped && escapeChar.isPresent() && currentChar == escapeChar.get()) { + escaped = true; + } + else { + switch (currentChar) { + case '%': + regex.append(escaped ? "%" : ".*"); + escaped = false; + break; + case '_': + regex.append(escaped ? "_" : "."); + escaped = false; + break; + case '\\': + regex.append("\\\\"); + break; + default: + // escape special regex characters + if (REGEXP_RESERVED_CHARACTERS.contains(currentChar)) { + regex.append('\\'); + } + + regex.appendCodePoint(currentChar); + escaped = false; + } + } + } + + checkEscape(!escaped); + return regex.toString(); + } + + private static void checkEscape(boolean condition) + { + if (!condition) { + throw new TrinoException(INVALID_FUNCTION_ARGUMENT, "Escape character must be followed by '%', '_' or the escape character itself"); + } + } + + private static char getEscapeChar(Slice escape) + { + String escapeString = escape.toStringUtf8(); + if (escapeString.length() == 1) { + return escapeString.charAt(0); + } + throw new TrinoException(INVALID_FUNCTION_ARGUMENT, "Escape string must be a single character"); + } + + private static boolean isPassthroughQuery(ElasticsearchTableHandle table) + { + return table.getType().equals(QUERY); + } + + @Override + public Optional> applyTableFunction(ConnectorSession session, ConnectorTableFunctionHandle handle) + { + if (!(handle instanceof RawQueryFunctionHandle)) { + return Optional.empty(); + } + + ConnectorTableHandle tableHandle = ((RawQueryFunctionHandle) handle).getTableHandle(); + List columnHandles = ImmutableList.copyOf(getColumnHandles(session, tableHandle).values()); + return Optional.of(new TableFunctionApplicationResult<>(tableHandle, columnHandles)); + } + + private static class InternalTableMetadata + { + private final SchemaTableName tableName; + private final List columnMetadata; + private final Map columnHandles; + + public InternalTableMetadata( + SchemaTableName tableName, + List columnMetadata, + Map columnHandles) + { + this.tableName = tableName; + this.columnMetadata = columnMetadata; + this.columnHandles = columnHandles; + } + + public SchemaTableName getTableName() + { + return tableName; + } + + public List getColumnMetadata() + { + return columnMetadata; + } + + public Map getColumnHandles() + { + return columnHandles; + } + } + + private static class TypeAndDecoder + { + private final Type type; + private final DecoderDescriptor decoderDescriptor; + + public TypeAndDecoder(Type type, DecoderDescriptor decoderDescriptor) + { + this.type = type; + this.decoderDescriptor = decoderDescriptor; + } + + public Type getType() + { + return type; + } + + public DecoderDescriptor getDecoderDescriptor() + { + return decoderDescriptor; + } + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchPageSourceProvider.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchPageSourceProvider.java new file mode 100644 index 0000000000000..4744121748624 --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchPageSourceProvider.java @@ -0,0 +1,79 @@ +/* + * 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.elasticsearch; + +import com.google.inject.Inject; +import io.trino.plugin.elasticsearch.client.ElasticsearchClient; +import io.trino.spi.connector.ColumnHandle; +import io.trino.spi.connector.ConnectorPageSource; +import io.trino.spi.connector.ConnectorPageSourceProvider; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.ConnectorSplit; +import io.trino.spi.connector.ConnectorTableHandle; +import io.trino.spi.connector.ConnectorTransactionHandle; +import io.trino.spi.connector.DynamicFilter; +import io.trino.spi.type.TypeManager; + +import java.util.List; + +import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.trino.plugin.elasticsearch.ElasticsearchTableHandle.Type.QUERY; +import static java.util.Objects.requireNonNull; + +public class ElasticsearchPageSourceProvider + implements ConnectorPageSourceProvider +{ + private final ElasticsearchClient client; + private final TypeManager typeManager; + + @Inject + public ElasticsearchPageSourceProvider(ElasticsearchClient client, TypeManager typeManager) + { + this.client = requireNonNull(client, "client is null"); + this.typeManager = requireNonNull(typeManager, "typeManager is null"); + } + + @Override + public ConnectorPageSource createPageSource( + ConnectorTransactionHandle transaction, + ConnectorSession session, + ConnectorSplit split, + ConnectorTableHandle table, + List columns, + DynamicFilter dynamicFilter) + { + requireNonNull(split, "split is null"); + requireNonNull(table, "table is null"); + + ElasticsearchTableHandle elasticsearchTable = (ElasticsearchTableHandle) table; + ElasticsearchSplit elasticsearchSplit = (ElasticsearchSplit) split; + + if (elasticsearchTable.getType().equals(QUERY)) { + return new PassthroughQueryPageSource(client, elasticsearchTable); + } + + if (columns.isEmpty()) { + return new CountQueryPageSource(client, elasticsearchTable, elasticsearchSplit); + } + + return new ScanQueryPageSource( + client, + typeManager, + elasticsearchTable, + elasticsearchSplit, + columns.stream() + .map(ElasticsearchColumnHandle.class::cast) + .collect(toImmutableList())); + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchPlugin.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchPlugin.java new file mode 100644 index 0000000000000..b65805c0368ef --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchPlugin.java @@ -0,0 +1,44 @@ +/* + * 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.elasticsearch; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableList; +import io.trino.spi.Plugin; +import io.trino.spi.connector.ConnectorFactory; + +import static java.util.Objects.requireNonNull; + +public class ElasticsearchPlugin + implements Plugin +{ + private final ConnectorFactory connectorFactory; + + public ElasticsearchPlugin() + { + connectorFactory = new ElasticsearchConnectorFactory(); + } + + @VisibleForTesting + ElasticsearchPlugin(ElasticsearchConnectorFactory factory) + { + connectorFactory = requireNonNull(factory, "factory is null"); + } + + @Override + public synchronized Iterable getConnectorFactories() + { + return ImmutableList.of(connectorFactory); + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchQueryBuilder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchQueryBuilder.java new file mode 100644 index 0000000000000..9cbad441af708 --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchQueryBuilder.java @@ -0,0 +1,172 @@ +/* + * 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.elasticsearch; + +import com.google.common.collect.ImmutableList; +import io.airlift.slice.Slice; +import io.trino.spi.predicate.Domain; +import io.trino.spi.predicate.Range; +import io.trino.spi.predicate.TupleDomain; +import io.trino.spi.type.Type; +import org.elasticsearch.index.query.BoolQueryBuilder; +import org.elasticsearch.index.query.ExistsQueryBuilder; +import org.elasticsearch.index.query.MatchAllQueryBuilder; +import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.index.query.QueryStringQueryBuilder; +import org.elasticsearch.index.query.RangeQueryBuilder; +import org.elasticsearch.index.query.RegexpQueryBuilder; +import org.elasticsearch.index.query.TermQueryBuilder; + +import java.time.Instant; +import java.time.ZoneOffset; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkState; +import static com.google.common.collect.Iterables.getOnlyElement; +import static io.trino.spi.type.BigintType.BIGINT; +import static io.trino.spi.type.BooleanType.BOOLEAN; +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.TimestampType.TIMESTAMP_MILLIS; +import static io.trino.spi.type.Timestamps.MICROSECONDS_PER_MILLISECOND; +import static io.trino.spi.type.TinyintType.TINYINT; +import static io.trino.spi.type.VarcharType.VARCHAR; +import static java.lang.Math.floorDiv; +import static java.lang.Math.toIntExact; +import static java.time.format.DateTimeFormatter.ISO_DATE_TIME; + +public final class ElasticsearchQueryBuilder +{ + private ElasticsearchQueryBuilder() {} + + public static QueryBuilder buildSearchQuery(TupleDomain constraint, Optional query, Map regexes) + { + BoolQueryBuilder queryBuilder = new BoolQueryBuilder(); + if (constraint.getDomains().isPresent()) { + for (Map.Entry entry : constraint.getDomains().get().entrySet()) { + ElasticsearchColumnHandle column = entry.getKey(); + Domain domain = entry.getValue(); + + checkArgument(!domain.isNone(), "Unexpected NONE domain for %s", column.getName()); + if (!domain.isAll()) { + addPredicateToQueryBuilder(queryBuilder, column.getName(), domain, column.getType()); + } + } + } + + regexes.forEach((name, value) -> queryBuilder.filter(new BoolQueryBuilder().must(new RegexpQueryBuilder(name, value)))); + + query.map(QueryStringQueryBuilder::new) + .ifPresent(queryBuilder::must); + + if (queryBuilder.hasClauses()) { + return queryBuilder; + } + return new MatchAllQueryBuilder(); + } + + private static void addPredicateToQueryBuilder(BoolQueryBuilder queryBuilder, String columnName, Domain domain, Type type) + { + checkArgument(domain.getType().isOrderable(), "Domain type must be orderable"); + + if (domain.getValues().isNone()) { + queryBuilder.mustNot(new ExistsQueryBuilder(columnName)); + return; + } + + if (domain.getValues().isAll()) { + queryBuilder.filter(new ExistsQueryBuilder(columnName)); + return; + } + + List shouldClauses = getShouldClauses(columnName, domain, type); + if (shouldClauses.size() == 1) { + queryBuilder.filter(getOnlyElement(shouldClauses)); + return; + } + if (shouldClauses.size() > 1) { + BoolQueryBuilder boolQueryBuilder = new BoolQueryBuilder(); + shouldClauses.forEach(boolQueryBuilder::should); + queryBuilder.filter(boolQueryBuilder); + return; + } + } + + private static List getShouldClauses(String columnName, Domain domain, Type type) + { + ImmutableList.Builder shouldClauses = ImmutableList.builder(); + for (Range range : domain.getValues().getRanges().getOrderedRanges()) { + checkState(!range.isAll(), "Invalid range for column: %s", columnName); + if (range.isSingleValue()) { + shouldClauses.add(new TermQueryBuilder(columnName, getValue(type, range.getSingleValue()))); + } + else { + RangeQueryBuilder rangeQueryBuilder = new RangeQueryBuilder(columnName); + if (!range.isLowUnbounded()) { + Object lowBound = getValue(type, range.getLowBoundedValue()); + if (range.isLowInclusive()) { + rangeQueryBuilder.gte(lowBound); + } + else { + rangeQueryBuilder.gt(lowBound); + } + } + if (!range.isHighUnbounded()) { + Object highBound = getValue(type, range.getHighBoundedValue()); + if (range.isHighInclusive()) { + rangeQueryBuilder.lte(highBound); + } + else { + rangeQueryBuilder.lt(highBound); + } + } + shouldClauses.add(rangeQueryBuilder); + } + } + if (domain.isNullAllowed()) { + shouldClauses.add(new BoolQueryBuilder().mustNot(new ExistsQueryBuilder(columnName))); + } + return shouldClauses.build(); + } + + private static Object getValue(Type type, Object value) + { + if (type.equals(BOOLEAN) || + type.equals(TINYINT) || + type.equals(SMALLINT) || + type.equals(INTEGER) || + type.equals(BIGINT) || + type.equals(DOUBLE)) { + return value; + } + if (type.equals(REAL)) { + return Float.intBitsToFloat(toIntExact(((Long) value))); + } + if (type.equals(VARCHAR)) { + return ((Slice) value).toStringUtf8(); + } + if (type.equals(TIMESTAMP_MILLIS)) { + return Instant.ofEpochMilli(floorDiv((Long) value, MICROSECONDS_PER_MILLISECOND)) + .atZone(ZoneOffset.UTC) + .toLocalDateTime() + .format(ISO_DATE_TIME); + } + throw new IllegalArgumentException("Unhandled type: " + type); + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchSplit.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchSplit.java new file mode 100644 index 0000000000000..a1ed6592e834a --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchSplit.java @@ -0,0 +1,99 @@ +/* + * 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.elasticsearch; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableList; +import io.airlift.slice.SizeOf; +import io.trino.spi.HostAddress; +import io.trino.spi.connector.ConnectorSplit; + +import java.util.List; +import java.util.Optional; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static io.airlift.slice.SizeOf.estimatedSizeOf; +import static io.airlift.slice.SizeOf.instanceSize; +import static io.airlift.slice.SizeOf.sizeOf; +import static java.util.Objects.requireNonNull; + +public class ElasticsearchSplit + implements ConnectorSplit +{ + private static final int INSTANCE_SIZE = instanceSize(ElasticsearchSplit.class); + + private final String index; + private final int shard; + private final Optional address; + + @JsonCreator + public ElasticsearchSplit( + @JsonProperty("index") String index, + @JsonProperty("shard") int shard, + @JsonProperty("address") Optional address) + { + this.index = requireNonNull(index, "index is null"); + this.shard = shard; + this.address = requireNonNull(address, "address is null"); + } + + @JsonProperty + public String getIndex() + { + return index; + } + + @JsonProperty + public int getShard() + { + return shard; + } + + @JsonProperty + public Optional getAddress() + { + return address; + } + + @Override + public List getAddresses() + { + return address.map(host -> ImmutableList.of(HostAddress.fromString(host))) + .orElseGet(ImmutableList::of); + } + + @Override + public Object getInfo() + { + return this; + } + + @Override + public long getRetainedSizeInBytes() + { + return INSTANCE_SIZE + + estimatedSizeOf(index) + + sizeOf(address, SizeOf::estimatedSizeOf); + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("index", index) + .add("shard", shard) + .toString(); + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchSplitManager.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchSplitManager.java new file mode 100644 index 0000000000000..b8e70d9350f15 --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchSplitManager.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.elasticsearch; + +import com.google.inject.Inject; +import io.trino.plugin.elasticsearch.client.ElasticsearchClient; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.ConnectorSplitManager; +import io.trino.spi.connector.ConnectorSplitSource; +import io.trino.spi.connector.ConnectorTableHandle; +import io.trino.spi.connector.ConnectorTransactionHandle; +import io.trino.spi.connector.Constraint; +import io.trino.spi.connector.DynamicFilter; +import io.trino.spi.connector.FixedSplitSource; + +import java.util.List; +import java.util.Optional; + +import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.trino.plugin.elasticsearch.ElasticsearchTableHandle.Type.QUERY; +import static java.util.Objects.requireNonNull; + +public class ElasticsearchSplitManager + implements ConnectorSplitManager +{ + private final ElasticsearchClient client; + + @Inject + public ElasticsearchSplitManager(ElasticsearchClient client) + { + this.client = requireNonNull(client, "client is null"); + } + + @Override + public ConnectorSplitSource getSplits( + ConnectorTransactionHandle transaction, + ConnectorSession session, + ConnectorTableHandle table, + DynamicFilter dynamicFilter, + Constraint constraint) + { + ElasticsearchTableHandle tableHandle = (ElasticsearchTableHandle) table; + + if (tableHandle.getType().equals(QUERY)) { + return new FixedSplitSource(new ElasticsearchSplit(tableHandle.getIndex(), 0, Optional.empty())); + } + List splits = client.getSearchShards(tableHandle.getIndex()).stream() + .map(shard -> new ElasticsearchSplit(shard.getIndex(), shard.getId(), shard.getAddress())) + .collect(toImmutableList()); + + return new FixedSplitSource(splits); + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchTableHandle.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchTableHandle.java new file mode 100644 index 0000000000000..a381ef99b34dd --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchTableHandle.java @@ -0,0 +1,170 @@ +/* + * 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.elasticsearch; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableMap; +import io.trino.spi.connector.ColumnHandle; +import io.trino.spi.connector.ConnectorTableHandle; +import io.trino.spi.predicate.TupleDomain; + +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.OptionalLong; +import java.util.stream.Collectors; + +import static java.util.Objects.requireNonNull; + +public final class ElasticsearchTableHandle + implements ConnectorTableHandle +{ + public enum Type + { + SCAN, QUERY + } + + private final Type type; + private final String schema; + private final String index; + private final TupleDomain constraint; + private final Map regexes; + private final Optional query; + private final OptionalLong limit; + + public ElasticsearchTableHandle(Type type, String schema, String index, Optional query) + { + this.type = requireNonNull(type, "type is null"); + this.schema = requireNonNull(schema, "schema is null"); + this.index = requireNonNull(index, "index is null"); + this.query = requireNonNull(query, "query is null"); + + constraint = TupleDomain.all(); + regexes = ImmutableMap.of(); + limit = OptionalLong.empty(); + } + + @JsonCreator + public ElasticsearchTableHandle( + @JsonProperty("type") Type type, + @JsonProperty("schema") String schema, + @JsonProperty("index") String index, + @JsonProperty("constraint") TupleDomain constraint, + @JsonProperty("regexes") Map regexes, + @JsonProperty("query") Optional query, + @JsonProperty("limit") OptionalLong limit) + { + this.type = requireNonNull(type, "type is null"); + this.schema = requireNonNull(schema, "schema is null"); + this.index = requireNonNull(index, "index is null"); + this.constraint = requireNonNull(constraint, "constraint is null"); + this.regexes = ImmutableMap.copyOf(requireNonNull(regexes, "regexes is null")); + this.query = requireNonNull(query, "query is null"); + this.limit = requireNonNull(limit, "limit is null"); + } + + @JsonProperty + public Type getType() + { + return type; + } + + @JsonProperty + public String getSchema() + { + return schema; + } + + @JsonProperty + public String getIndex() + { + return index; + } + + @JsonProperty + public TupleDomain getConstraint() + { + return constraint; + } + + @JsonProperty + public Map getRegexes() + { + return regexes; + } + + @JsonProperty + public OptionalLong getLimit() + { + return limit; + } + + @JsonProperty + public Optional getQuery() + { + return query; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ElasticsearchTableHandle that = (ElasticsearchTableHandle) o; + return type == that.type && + schema.equals(that.schema) && + index.equals(that.index) && + constraint.equals(that.constraint) && + regexes.equals(that.regexes) && + query.equals(that.query) && + limit.equals(that.limit); + } + + @Override + public int hashCode() + { + return Objects.hash(type, schema, index, constraint, regexes, query, limit); + } + + @Override + public String toString() + { + StringBuilder builder = new StringBuilder(); + builder.append(type + ":" + index); + + StringBuilder attributes = new StringBuilder(); + if (!regexes.isEmpty()) { + attributes.append("regexes=["); + attributes.append(regexes.entrySet().stream() + .map(regex -> regex.getKey() + ":" + regex.getValue()) + .collect(Collectors.joining(", "))); + attributes.append("]"); + } + limit.ifPresent(value -> attributes.append("limit=" + value)); + query.ifPresent(value -> attributes.append("query" + value)); + + if (attributes.length() > 0) { + builder.append("("); + builder.append(attributes); + builder.append(")"); + } + + return builder.toString(); + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchTransactionHandle.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchTransactionHandle.java new file mode 100644 index 0000000000000..cf42b03b5d458 --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchTransactionHandle.java @@ -0,0 +1,22 @@ +/* + * 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.elasticsearch; + +import io.trino.spi.connector.ConnectorTransactionHandle; + +public enum ElasticsearchTransactionHandle + implements ConnectorTransactionHandle +{ + INSTANCE +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/NodesSystemTable.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/NodesSystemTable.java new file mode 100644 index 0000000000000..84385dbedbf30 --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/NodesSystemTable.java @@ -0,0 +1,105 @@ +/* + * 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.elasticsearch; + +import com.google.common.collect.ImmutableList; +import com.google.inject.Inject; +import io.trino.plugin.elasticsearch.client.ElasticsearchClient; +import io.trino.plugin.elasticsearch.client.ElasticsearchNode; +import io.trino.spi.Node; +import io.trino.spi.NodeManager; +import io.trino.spi.Page; +import io.trino.spi.block.BlockBuilder; +import io.trino.spi.connector.ColumnMetadata; +import io.trino.spi.connector.ConnectorPageSource; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.ConnectorTableMetadata; +import io.trino.spi.connector.ConnectorTransactionHandle; +import io.trino.spi.connector.FixedPageSource; +import io.trino.spi.connector.SchemaTableName; +import io.trino.spi.connector.SystemTable; +import io.trino.spi.predicate.TupleDomain; + +import java.util.Set; + +import static io.trino.spi.type.VarcharType.VARCHAR; +import static io.trino.spi.type.VarcharType.createUnboundedVarcharType; +import static java.util.Objects.requireNonNull; + +public class NodesSystemTable + implements SystemTable +{ + private static final ConnectorTableMetadata METADATA = new ConnectorTableMetadata( + new SchemaTableName("system", "nodes"), + ImmutableList.builder() + .add(new ColumnMetadata("trino_node_id", createUnboundedVarcharType())) + .add(new ColumnMetadata("trino_node_address", createUnboundedVarcharType())) + .add(new ColumnMetadata("elasticsearch_node_id", createUnboundedVarcharType())) + .add(new ColumnMetadata("elasticsearch_node_address", createUnboundedVarcharType())) + .build()); + + private final ElasticsearchClient client; + private final Node currentNode; + + @Inject + public NodesSystemTable(NodeManager nodeManager, ElasticsearchClient client) + { + requireNonNull(nodeManager, "nodeManager is null"); + + this.client = requireNonNull(client, "client is null"); + currentNode = nodeManager.getCurrentNode(); + } + + @Override + public Distribution getDistribution() + { + return Distribution.ALL_NODES; + } + + @Override + public ConnectorTableMetadata getTableMetadata() + { + return METADATA; + } + + @Override + public ConnectorPageSource pageSource(ConnectorTransactionHandle transaction, ConnectorSession session, TupleDomain constraint) + { + Set nodes = client.getNodes(); + + BlockBuilder nodeId = VARCHAR.createBlockBuilder(null, nodes.size()); + BlockBuilder trinoAddress = VARCHAR.createBlockBuilder(null, nodes.size()); + BlockBuilder elasticsearchNodeId = VARCHAR.createBlockBuilder(null, nodes.size()); + BlockBuilder elasticsearchAddress = VARCHAR.createBlockBuilder(null, nodes.size()); + + for (ElasticsearchNode node : nodes) { + VARCHAR.writeString(nodeId, currentNode.getNodeIdentifier()); + VARCHAR.writeString(trinoAddress, currentNode.getHostAndPort().toString()); + VARCHAR.writeString(elasticsearchNodeId, node.getId()); + + if (node.getAddress().isPresent()) { + VARCHAR.writeString(elasticsearchAddress, node.getAddress().get()); + } + else { + elasticsearchAddress.appendNull(); + } + } + + return new FixedPageSource(ImmutableList.of(new Page( + nodeId.build(), + trinoAddress.build(), + elasticsearchNodeId.build(), + elasticsearchAddress.build()))); + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/PassthroughQueryPageSource.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/PassthroughQueryPageSource.java new file mode 100644 index 0000000000000..47e49d9c40039 --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/PassthroughQueryPageSource.java @@ -0,0 +1,91 @@ +/* + * 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.elasticsearch; + +import com.google.common.collect.ImmutableList; +import io.airlift.slice.Slices; +import io.trino.plugin.elasticsearch.client.ElasticsearchClient; +import io.trino.spi.Page; +import io.trino.spi.PageBuilder; +import io.trino.spi.block.BlockBuilder; +import io.trino.spi.connector.ConnectorPageSource; + +import java.io.IOException; + +import static io.trino.spi.type.VarcharType.VARCHAR; +import static java.util.Objects.requireNonNull; + +public class PassthroughQueryPageSource + implements ConnectorPageSource +{ + private final long readTimeNanos; + private final String result; + private boolean done; + + public PassthroughQueryPageSource(ElasticsearchClient client, ElasticsearchTableHandle table) + { + requireNonNull(client, "client is null"); + requireNonNull(table, "table is null"); + + long start = System.nanoTime(); + result = client.executeQuery(table.getIndex(), table.getQuery().get()); + readTimeNanos = System.nanoTime() - start; + } + + @Override + public long getCompletedBytes() + { + return result.length(); + } + + @Override + public long getReadTimeNanos() + { + return readTimeNanos; + } + + @Override + public boolean isFinished() + { + return done; + } + + @Override + public Page getNextPage() + { + if (done) { + return null; + } + + done = true; + + PageBuilder page = new PageBuilder(1, ImmutableList.of(VARCHAR)); + page.declarePosition(); + BlockBuilder column = page.getBlockBuilder(0); + VARCHAR.writeSlice(column, Slices.utf8Slice(result)); + return page.build(); + } + + @Override + public long getMemoryUsage() + { + return 0; + } + + @Override + public void close() + throws IOException + { + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/PasswordConfig.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/PasswordConfig.java new file mode 100644 index 0000000000000..2271e1da6b5fe --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/PasswordConfig.java @@ -0,0 +1,51 @@ +/* + * 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.elasticsearch; + +import io.airlift.configuration.Config; +import io.airlift.configuration.ConfigSecuritySensitive; +import jakarta.validation.constraints.NotNull; + +public class PasswordConfig +{ + private String user; + private String password; + + @NotNull + public String getUser() + { + return user; + } + + @Config("elasticsearch.auth.user") + public PasswordConfig setUser(String user) + { + this.user = user; + return this; + } + + @NotNull + public String getPassword() + { + return password; + } + + @Config("elasticsearch.auth.password") + @ConfigSecuritySensitive + public PasswordConfig setPassword(String password) + { + this.password = password; + return this; + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ScanQueryPageSource.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ScanQueryPageSource.java new file mode 100644 index 0000000000000..8bb5e60e9209d --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ScanQueryPageSource.java @@ -0,0 +1,324 @@ +/* + * 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.elasticsearch; + +import com.google.common.collect.AbstractIterator; +import com.google.common.collect.ImmutableList; +import io.airlift.log.Logger; +import io.trino.plugin.elasticsearch.client.ElasticsearchClient; +import io.trino.plugin.elasticsearch.decoders.Decoder; +import io.trino.spi.Page; +import io.trino.spi.block.Block; +import io.trino.spi.block.BlockBuilder; +import io.trino.spi.block.PageBuilderStatus; +import io.trino.spi.connector.ConnectorPageSource; +import io.trino.spi.type.RowType; +import io.trino.spi.type.Type; +import io.trino.spi.type.TypeManager; +import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.search.SearchHit; +import org.elasticsearch.search.SearchHits; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.OptionalLong; +import java.util.function.Supplier; + +import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.trino.plugin.elasticsearch.BuiltinColumns.SOURCE; +import static io.trino.plugin.elasticsearch.BuiltinColumns.isBuiltinColumn; +import static io.trino.plugin.elasticsearch.ElasticsearchQueryBuilder.buildSearchQuery; +import static io.trino.spi.type.TimestampType.TIMESTAMP_MILLIS; +import static java.util.Objects.requireNonNull; +import static java.util.function.Predicate.isEqual; +import static java.util.stream.Collectors.toList; + +public class ScanQueryPageSource + implements ConnectorPageSource +{ + private static final Logger LOG = Logger.get(ScanQueryPageSource.class); + + private final List decoders; + + private final SearchHitIterator iterator; + private final BlockBuilder[] columnBuilders; + private final List columns; + private long totalBytes; + private long readTimeNanos; + + public ScanQueryPageSource( + ElasticsearchClient client, + TypeManager typeManager, + ElasticsearchTableHandle table, + ElasticsearchSplit split, + List columns) + { + requireNonNull(client, "client is null"); + requireNonNull(typeManager, "typeManager is null"); + requireNonNull(columns, "columns is null"); + + this.columns = ImmutableList.copyOf(columns); + + decoders = createDecoders(columns); + + // When the _source field is requested, we need to bypass column pruning when fetching the document + boolean needAllFields = columns.stream() + .map(ElasticsearchColumnHandle::getName) + .anyMatch(isEqual(SOURCE.getName())); + + // Columns to fetch as doc_fields instead of pulling them out of the JSON source + // This is convenient for types such as DATE, TIMESTAMP, etc, which have multiple possible + // representations in JSON, but a single normalized representation as doc_field. + List documentFields = flattenFields(columns).entrySet().stream() + .filter(entry -> entry.getValue().equals(TIMESTAMP_MILLIS)) + .map(Map.Entry::getKey) + .collect(toImmutableList()); + + columnBuilders = columns.stream() + .map(ElasticsearchColumnHandle::getType) + .map(type -> type.createBlockBuilder(null, 1)) + .toArray(BlockBuilder[]::new); + + List requiredFields = columns.stream() + .map(ElasticsearchColumnHandle::getName) + .filter(name -> !isBuiltinColumn(name)) + .collect(toList()); + + // sorting by _doc (index order) get special treatment in Elasticsearch and is more efficient + Optional sort = Optional.of("_doc"); + + if (table.getQuery().isPresent()) { + // However, if we're using a custom Elasticsearch query, use default sorting. + // Documents will be scored and returned based on relevance + sort = Optional.empty(); + } + + long start = System.nanoTime(); + SearchResponse searchResponse = client.beginSearch( + split.getIndex(), + split.getShard(), + buildSearchQuery(table.getConstraint().transformKeys(ElasticsearchColumnHandle.class::cast), table.getQuery(), table.getRegexes()), + needAllFields ? Optional.empty() : Optional.of(requiredFields), + documentFields, + sort, + table.getLimit()); + readTimeNanos += System.nanoTime() - start; + this.iterator = new SearchHitIterator(client, () -> searchResponse, table.getLimit()); + } + + @Override + public long getCompletedBytes() + { + return totalBytes; + } + + @Override + public long getReadTimeNanos() + { + return readTimeNanos + iterator.getReadTimeNanos(); + } + + @Override + public boolean isFinished() + { + return !iterator.hasNext(); + } + + @Override + public long getMemoryUsage() + { + return 0; + } + + @Override + public void close() + { + iterator.close(); + } + + @Override + public Page getNextPage() + { + long size = 0; + while (size < PageBuilderStatus.DEFAULT_MAX_PAGE_SIZE_IN_BYTES && iterator.hasNext()) { + SearchHit hit = iterator.next(); + Map document = hit.getSourceAsMap(); + + for (int i = 0; i < decoders.size(); i++) { + String field = columns.get(i).getName(); + decoders.get(i).decode(hit, () -> getField(document, field), columnBuilders[i]); + } + + if (hit.getSourceRef() != null) { + totalBytes += hit.getSourceRef().length(); + } + + size = Arrays.stream(columnBuilders) + .mapToLong(BlockBuilder::getSizeInBytes) + .sum(); + } + + Block[] blocks = new Block[columnBuilders.length]; + for (int i = 0; i < columnBuilders.length; i++) { + blocks[i] = columnBuilders[i].build(); + columnBuilders[i] = columnBuilders[i].newBlockBuilderLike(null); + } + + return new Page(blocks); + } + + public static Object getField(Map document, String field) + { + Object value = document.get(field); + if (value == null) { + Map result = new HashMap<>(); + String prefix = field + "."; + for (Map.Entry entry : document.entrySet()) { + String key = entry.getKey(); + if (key.startsWith(prefix)) { + result.put(key.substring(prefix.length()), entry.getValue()); + } + } + + if (!result.isEmpty()) { + return result; + } + } + + return value; + } + + private Map flattenFields(List columns) + { + Map result = new HashMap<>(); + + for (ElasticsearchColumnHandle column : columns) { + flattenFields(result, column.getName(), column.getType()); + } + + return result; + } + + private void flattenFields(Map result, String fieldName, Type type) + { + if (type instanceof RowType) { + for (RowType.Field field : ((RowType) type).getFields()) { + flattenFields(result, appendPath(fieldName, field.getName().get()), field.getType()); + } + } + else { + result.put(fieldName, type); + } + } + + private List createDecoders(List columns) + { + return columns.stream() + .map(ElasticsearchColumnHandle::getDecoderDescriptor) + .map(DecoderDescriptor::createDecoder) + .collect(toImmutableList()); + } + + private static String appendPath(String base, String element) + { + if (base.isEmpty()) { + return element; + } + + return base + "." + element; + } + + private static class SearchHitIterator + extends AbstractIterator + { + private final ElasticsearchClient client; + private final Supplier first; + private final OptionalLong limit; + + private SearchHits searchHits; + private String scrollId; + private int currentPosition; + + private long readTimeNanos; + private long totalRecordCount; + + public SearchHitIterator(ElasticsearchClient client, Supplier first, OptionalLong limit) + { + this.client = client; + this.first = first; + this.limit = limit; + this.totalRecordCount = 0; + } + + public long getReadTimeNanos() + { + return readTimeNanos; + } + + @Override + protected SearchHit computeNext() + { + if (limit.isPresent() && totalRecordCount == limit.getAsLong()) { + // No more record is necessary. + return endOfData(); + } + + if (scrollId == null) { + long start = System.nanoTime(); + SearchResponse response = first.get(); + readTimeNanos += System.nanoTime() - start; + reset(response); + } + else if (currentPosition == searchHits.getHits().length) { + long start = System.nanoTime(); + SearchResponse response = client.nextPage(scrollId); + readTimeNanos += System.nanoTime() - start; + reset(response); + } + + if (currentPosition == searchHits.getHits().length) { + return endOfData(); + } + + SearchHit hit = searchHits.getAt(currentPosition); + currentPosition++; + totalRecordCount++; + + return hit; + } + + private void reset(SearchResponse response) + { + scrollId = response.getScrollId(); + searchHits = response.getHits(); + currentPosition = 0; + } + + public void close() + { + if (scrollId != null) { + try { + client.clearScroll(scrollId); + } + catch (Exception e) { + // ignore + LOG.debug(e, "Error clearing scroll"); + } + } + } + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/AwsRequestSigner.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/AwsRequestSigner.java new file mode 100644 index 0000000000000..7619b263625eb --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/AwsRequestSigner.java @@ -0,0 +1,114 @@ +/* + * 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.elasticsearch.client; + +import com.amazonaws.DefaultRequest; +import com.amazonaws.auth.AWS4Signer; +import com.amazonaws.auth.AWSCredentialsProvider; +import com.amazonaws.http.HttpMethodName; +import org.apache.http.Header; +import org.apache.http.HttpEntityEnclosingRequest; +import org.apache.http.HttpHost; +import org.apache.http.HttpRequest; +import org.apache.http.HttpRequestInterceptor; +import org.apache.http.NameValuePair; +import org.apache.http.client.utils.URIBuilder; +import org.apache.http.entity.BasicHttpEntity; +import org.apache.http.message.BasicHeader; +import org.apache.http.protocol.HttpContext; + +import java.io.IOException; +import java.io.InputStream; +import java.net.URI; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; + +import static com.google.common.base.Preconditions.checkState; +import static com.google.common.collect.ImmutableMap.toImmutableMap; +import static java.lang.String.CASE_INSENSITIVE_ORDER; +import static org.apache.http.protocol.HttpCoreContext.HTTP_TARGET_HOST; + +class AwsRequestSigner + implements HttpRequestInterceptor +{ + private static final String SERVICE_NAME = "es"; + private final AWSCredentialsProvider credentialsProvider; + private final AWS4Signer signer; + + public AwsRequestSigner(String region, AWSCredentialsProvider credentialsProvider) + { + this.credentialsProvider = credentialsProvider; + this.signer = new AWS4Signer(); + + signer.setServiceName(SERVICE_NAME); + signer.setRegionName(region); + } + + @Override + public void process(HttpRequest request, HttpContext context) + throws IOException + { + String method = request.getRequestLine().getMethod(); + + URI uri = URI.create(request.getRequestLine().getUri()); + URIBuilder uriBuilder = new URIBuilder(uri); + + Map> parameters = new TreeMap<>(CASE_INSENSITIVE_ORDER); + for (NameValuePair parameter : uriBuilder.getQueryParams()) { + parameters.computeIfAbsent(parameter.getName(), key -> new ArrayList<>()) + .add(parameter.getValue()); + } + + Map headers = Arrays.stream(request.getAllHeaders()) + .collect(toImmutableMap(Header::getName, Header::getValue)); + + InputStream content = null; + if (request instanceof HttpEntityEnclosingRequest enclosingRequest) { + if (enclosingRequest.getEntity() != null) { + content = enclosingRequest.getEntity().getContent(); + } + } + + DefaultRequest awsRequest = new DefaultRequest<>(SERVICE_NAME); + + HttpHost host = (HttpHost) context.getAttribute(HTTP_TARGET_HOST); + if (host != null) { + awsRequest.setEndpoint(URI.create(host.toURI())); + } + awsRequest.setHttpMethod(HttpMethodName.fromValue(method)); + awsRequest.setResourcePath(uri.getRawPath()); + awsRequest.setContent(content); + awsRequest.setParameters(parameters); + awsRequest.setHeaders(headers); + + signer.sign(awsRequest, credentialsProvider.getCredentials()); + + Header[] newHeaders = awsRequest.getHeaders().entrySet().stream() + .map(entry -> new BasicHeader(entry.getKey(), entry.getValue())) + .toArray(Header[]::new); + + request.setHeaders(newHeaders); + + InputStream newContent = awsRequest.getContent(); + checkState(newContent == null || request instanceof HttpEntityEnclosingRequest); + if (newContent != null) { + BasicHttpEntity entity = new BasicHttpEntity(); + entity.setContent(newContent); + ((HttpEntityEnclosingRequest) request).setEntity(entity); + } + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/BackpressureRestClient.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/BackpressureRestClient.java new file mode 100644 index 0000000000000..eebc163c2f4dd --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/BackpressureRestClient.java @@ -0,0 +1,158 @@ +/* + * 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.elasticsearch.client; + +import com.google.common.base.Stopwatch; +import dev.failsafe.Failsafe; +import dev.failsafe.FailsafeException; +import dev.failsafe.RetryPolicy; +import dev.failsafe.event.ExecutionAttemptedEvent; +import dev.failsafe.event.ExecutionCompletedEvent; +import dev.failsafe.function.CheckedSupplier; +import io.airlift.log.Logger; +import io.airlift.stats.TimeStat; +import io.trino.plugin.elasticsearch.ElasticsearchConfig; +import org.apache.http.Header; +import org.apache.http.HttpEntity; +import org.apache.http.HttpHost; +import org.elasticsearch.client.Node; +import org.elasticsearch.client.Request; +import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.client.Response; +import org.elasticsearch.client.ResponseException; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.rest.RestStatus; + +import java.io.IOException; +import java.util.Map; + +import static com.google.common.base.Throwables.throwIfInstanceOf; +import static com.google.common.base.Throwables.throwIfUnchecked; +import static com.google.common.collect.ImmutableList.toImmutableList; +import static java.time.temporal.ChronoUnit.MILLIS; +import static java.util.Arrays.stream; +import static java.util.Objects.requireNonNull; +import static java.util.concurrent.TimeUnit.MILLISECONDS; + +public class BackpressureRestClient +{ + private static final Logger log = Logger.get(BackpressureRestClient.class); + + private final RestClient delegate; + private final RetryPolicy retryPolicy; + private final TimeStat backpressureStats; + private final ThreadLocal stopwatch = ThreadLocal.withInitial(Stopwatch::createUnstarted); + + public BackpressureRestClient(RestClient delegate, ElasticsearchConfig config, TimeStat backpressureStats) + { + this.delegate = requireNonNull(delegate, "restClient is null"); + this.backpressureStats = requireNonNull(backpressureStats, "backpressureStats is null"); + retryPolicy = RetryPolicy.builder() + .withMaxAttempts(-1) + .withMaxDuration(java.time.Duration.ofMillis(config.getMaxRetryTime().toMillis())) + .withBackoff(config.getBackoffInitDelay().toMillis(), config.getBackoffMaxDelay().toMillis(), MILLIS) + .withJitter(0.125) + .handleIf(BackpressureRestClient::isBackpressure) + .onFailedAttempt(this::onFailedAttempt) + .onSuccess(this::onComplete) + .onFailure(this::onComplete) + .build(); + } + + public void setHosts(HttpHost... hosts) + { + delegate.setNodes(stream(hosts) + .map(Node::new) + .collect(toImmutableList())); + } + + public Response performRequest(String method, String endpoint, Header... headers) + throws IOException + { + return executeWithRetries(() -> delegate.performRequest(toRequest(method, endpoint, headers))); + } + + public Response performRequest(String method, String endpoint, Map params, HttpEntity entity, Header... headers) + throws IOException + { + return executeWithRetries(() -> delegate.performRequest(toRequest(method, endpoint, params, entity, headers))); + } + + private static Request toRequest(String method, String endpoint, Map params, HttpEntity entity, Header... headers) + { + Request request = toRequest(method, endpoint, headers); + requireNonNull(params, "parameters cannot be null"); + for (Map.Entry entry : params.entrySet()) { + request.addParameter(entry.getKey(), entry.getValue()); + } + request.setEntity(entity); + return request; + } + + private static Request toRequest(String method, String endpoint, Header... headers) + { + requireNonNull(headers, "headers cannot be null"); + Request request = new Request(method, endpoint); + RequestOptions.Builder options = request.getOptions().toBuilder(); + for (Header header : headers) { + options.addHeader(header.getName(), header.getValue()); + } + request.setOptions(options); + return request; + } + + public void close() + throws IOException + { + delegate.close(); + } + + private static boolean isBackpressure(Throwable throwable) + { + return (throwable instanceof ResponseException) && + (((ResponseException) throwable).getResponse().getStatusLine().getStatusCode() == RestStatus.TOO_MANY_REQUESTS.getStatus()); + } + + private void onComplete(ExecutionCompletedEvent executionCompletedEvent) + { + if (stopwatch.get().isRunning()) { + long delayMillis = stopwatch.get().elapsed(MILLISECONDS); + log.debug("Adding %s milliseconds to backpressure stats", delayMillis); + stopwatch.get().reset(); + backpressureStats.add(delayMillis, MILLISECONDS); + } + } + + private Response executeWithRetries(CheckedSupplier supplier) + throws IOException + { + try { + return Failsafe.with(retryPolicy).get(supplier); + } + catch (FailsafeException e) { + Throwable throwable = e.getCause(); + throwIfInstanceOf(throwable, IOException.class); + throwIfUnchecked(throwable); + throw new RuntimeException("Unexpected cause from FailsafeException", throwable); + } + } + + private void onFailedAttempt(ExecutionAttemptedEvent executionAttemptedEvent) + { + log.debug("REST attempt failed: %s", executionAttemptedEvent.getLastException()); + if (!stopwatch.get().isRunning()) { + stopwatch.get().start(); + } + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/BackpressureRestHighLevelClient.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/BackpressureRestHighLevelClient.java new file mode 100644 index 0000000000000..391283144cbec --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/BackpressureRestHighLevelClient.java @@ -0,0 +1,142 @@ +/* + * 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.elasticsearch.client; + +import com.google.common.base.Stopwatch; +import dev.failsafe.Failsafe; +import dev.failsafe.FailsafeException; +import dev.failsafe.RetryPolicy; +import dev.failsafe.event.ExecutionAttemptedEvent; +import dev.failsafe.event.ExecutionCompletedEvent; +import dev.failsafe.function.CheckedSupplier; +import io.airlift.log.Logger; +import io.airlift.stats.TimeStat; +import io.trino.plugin.elasticsearch.ElasticsearchConfig; +import org.elasticsearch.ElasticsearchStatusException; +import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.action.search.ClearScrollRequest; +import org.elasticsearch.action.search.ClearScrollResponse; +import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.SearchScrollRequest; +import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.client.RestClientBuilder; +import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.rest.RestStatus; + +import java.io.Closeable; +import java.io.IOException; + +import static com.google.common.base.Throwables.throwIfInstanceOf; +import static com.google.common.base.Throwables.throwIfUnchecked; +import static java.time.temporal.ChronoUnit.MILLIS; +import static java.util.Objects.requireNonNull; +import static java.util.concurrent.TimeUnit.MILLISECONDS; + +public class BackpressureRestHighLevelClient + implements Closeable +{ + private static final Logger log = Logger.get(BackpressureRestHighLevelClient.class); + + private final RestHighLevelClient delegate; + private final BackpressureRestClient backpressureRestClient; + private final RetryPolicy retryPolicy; + private final TimeStat backpressureStats; + private final ThreadLocal stopwatch = ThreadLocal.withInitial(Stopwatch::createUnstarted); + + public BackpressureRestHighLevelClient(RestClientBuilder restClientBuilder, ElasticsearchConfig config, TimeStat backpressureStats) + { + this.backpressureStats = requireNonNull(backpressureStats, "backpressureStats is null"); + delegate = new RestHighLevelClient(requireNonNull(restClientBuilder, "restClientBuilder is null")); + backpressureRestClient = new BackpressureRestClient(delegate.getLowLevelClient(), config, backpressureStats); + retryPolicy = RetryPolicy.builder() + .withMaxAttempts(-1) + .withMaxDuration(java.time.Duration.ofMillis(config.getMaxRetryTime().toMillis())) + .withBackoff(config.getBackoffInitDelay().toMillis(), config.getBackoffMaxDelay().toMillis(), MILLIS) + .withJitter(0.125) + .handleIf(BackpressureRestHighLevelClient::isBackpressure) + .onFailedAttempt(this::onFailedAttempt) + .onSuccess(this::onComplete) + .onFailure(this::onComplete) + .build(); + } + + public BackpressureRestClient getLowLevelClient() + { + return backpressureRestClient; + } + + @Override + public void close() + throws IOException + { + delegate.close(); + } + + public SearchResponse search(SearchRequest searchRequest) + throws IOException + { + return executeWithRetries(() -> delegate.search(searchRequest, RequestOptions.DEFAULT)); + } + + public SearchResponse searchScroll(SearchScrollRequest searchScrollRequest) + throws IOException + { + return executeWithRetries(() -> delegate.scroll(searchScrollRequest, RequestOptions.DEFAULT)); + } + + public ClearScrollResponse clearScroll(ClearScrollRequest clearScrollRequest) + throws IOException + { + return executeWithRetries(() -> delegate.clearScroll(clearScrollRequest, RequestOptions.DEFAULT)); + } + + private static boolean isBackpressure(Throwable throwable) + { + return (throwable instanceof ElasticsearchStatusException) && + (((ElasticsearchStatusException) throwable).status() == RestStatus.TOO_MANY_REQUESTS); + } + + private void onComplete(ExecutionCompletedEvent executionCompletedEvent) + { + if (stopwatch.get().isRunning()) { + long delayMillis = stopwatch.get().elapsed(MILLISECONDS); + log.debug("Adding %s milliseconds to backpressure stats", delayMillis); + stopwatch.get().reset(); + backpressureStats.add(delayMillis, MILLISECONDS); + } + } + + private T executeWithRetries(CheckedSupplier supplier) + throws IOException + { + try { + return Failsafe.with(retryPolicy).get(supplier); + } + catch (FailsafeException e) { + Throwable throwable = e.getCause(); + throwIfInstanceOf(throwable, IOException.class); + throwIfUnchecked(throwable); + throw new RuntimeException("Unexpected cause from FailsafeException", throwable); + } + } + + private void onFailedAttempt(ExecutionAttemptedEvent executionAttemptedEvent) + { + log.debug("REST attempt failed: %s", executionAttemptedEvent.getLastException()); + if (!stopwatch.get().isRunning()) { + stopwatch.get().start(); + } + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/CountResponse.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/CountResponse.java new file mode 100644 index 0000000000000..15e5de2b0bbc2 --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/CountResponse.java @@ -0,0 +1,34 @@ +/* + * 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.elasticsearch.client; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +public class CountResponse +{ + private final long count; + + @JsonCreator + public CountResponse(@JsonProperty("count") long count) + { + this.count = count; + } + + @JsonProperty + public long getCount() + { + return count; + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/ElasticsearchClient.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/ElasticsearchClient.java new file mode 100644 index 0000000000000..3fcea05054f1a --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/ElasticsearchClient.java @@ -0,0 +1,795 @@ +/* + * 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.elasticsearch.client; + +import com.amazonaws.auth.AWSCredentialsProvider; +import com.amazonaws.auth.AWSStaticCredentialsProvider; +import com.amazonaws.auth.BasicAWSCredentials; +import com.amazonaws.auth.DefaultAWSCredentialsProviderChain; +import com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider; +import com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClientBuilder; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.NullNode; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Sets; +import com.google.inject.Inject; +import io.airlift.json.JsonCodec; +import io.airlift.json.ObjectMapperProvider; +import io.airlift.log.Logger; +import io.airlift.stats.TimeStat; +import io.airlift.units.Duration; +import io.trino.plugin.elasticsearch.AwsSecurityConfig; +import io.trino.plugin.elasticsearch.ElasticsearchConfig; +import io.trino.plugin.elasticsearch.PasswordConfig; +import io.trino.spi.TrinoException; +import jakarta.annotation.PostConstruct; +import jakarta.annotation.PreDestroy; +import org.apache.http.HttpEntity; +import org.apache.http.HttpHost; +import org.apache.http.auth.AuthScope; +import org.apache.http.auth.UsernamePasswordCredentials; +import org.apache.http.client.CredentialsProvider; +import org.apache.http.client.config.RequestConfig; +import org.apache.http.conn.ssl.NoopHostnameVerifier; +import org.apache.http.entity.ByteArrayEntity; +import org.apache.http.entity.StringEntity; +import org.apache.http.impl.client.BasicCredentialsProvider; +import org.apache.http.impl.nio.client.HttpAsyncClientBuilder; +import org.apache.http.impl.nio.reactor.IOReactorConfig; +import org.apache.http.message.BasicHeader; +import org.apache.http.util.EntityUtils; +import org.elasticsearch.ElasticsearchStatusException; +import org.elasticsearch.action.search.ClearScrollRequest; +import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.SearchScrollRequest; +import org.elasticsearch.client.Response; +import org.elasticsearch.client.ResponseException; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestClientBuilder; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.search.builder.SearchSourceBuilder; +import org.weakref.jmx.Managed; +import org.weakref.jmx.Nested; + +import javax.net.ssl.SSLContext; + +import java.io.File; +import java.io.IOException; +import java.security.GeneralSecurityException; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.OptionalLong; +import java.util.Set; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.collect.ImmutableMap.toImmutableMap; +import static io.airlift.concurrent.Threads.daemonThreadsNamed; +import static io.airlift.json.JsonCodec.jsonCodec; +import static io.trino.plugin.base.ssl.SslUtils.createSSLContext; +import static io.trino.plugin.elasticsearch.ElasticsearchErrorCode.ELASTICSEARCH_CONNECTION_ERROR; +import static io.trino.plugin.elasticsearch.ElasticsearchErrorCode.ELASTICSEARCH_INVALID_METADATA; +import static io.trino.plugin.elasticsearch.ElasticsearchErrorCode.ELASTICSEARCH_INVALID_RESPONSE; +import static io.trino.plugin.elasticsearch.ElasticsearchErrorCode.ELASTICSEARCH_QUERY_FAILURE; +import static io.trino.plugin.elasticsearch.ElasticsearchErrorCode.ELASTICSEARCH_SSL_INITIALIZATION_FAILURE; +import static java.lang.StrictMath.toIntExact; +import static java.lang.String.format; +import static java.nio.charset.StandardCharsets.UTF_8; +import static java.util.concurrent.Executors.newSingleThreadScheduledExecutor; +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static org.elasticsearch.action.search.SearchType.QUERY_THEN_FETCH; + +public class ElasticsearchClient +{ + private static final Logger LOG = Logger.get(ElasticsearchClient.class); + + private static final JsonCodec SEARCH_SHARDS_RESPONSE_CODEC = jsonCodec(SearchShardsResponse.class); + private static final JsonCodec NODES_RESPONSE_CODEC = jsonCodec(NodesResponse.class); + private static final JsonCodec COUNT_RESPONSE_CODEC = jsonCodec(CountResponse.class); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapperProvider().get(); + + private static final Pattern ADDRESS_PATTERN = Pattern.compile("((?[^/]+)/)?(?.+):(?\\d+)"); + private static final Set NODE_ROLES = ImmutableSet.of("data", "data_content", "data_hot", "data_warm", "data_cold", "data_frozen"); + + private final BackpressureRestHighLevelClient client; + private final int scrollSize; + private final Duration scrollTimeout; + + private final AtomicReference> nodes = new AtomicReference<>(ImmutableSet.of()); + private final ScheduledExecutorService executor = newSingleThreadScheduledExecutor(daemonThreadsNamed("NodeRefresher")); + private final AtomicBoolean started = new AtomicBoolean(); + private final Duration refreshInterval; + private final boolean tlsEnabled; + private final boolean ignorePublishAddress; + + private final TimeStat searchStats = new TimeStat(MILLISECONDS); + private final TimeStat nextPageStats = new TimeStat(MILLISECONDS); + private final TimeStat countStats = new TimeStat(MILLISECONDS); + private final TimeStat backpressureStats = new TimeStat(MILLISECONDS); + + @Inject + public ElasticsearchClient( + ElasticsearchConfig config, + Optional awsSecurityConfig, + Optional passwordConfig) + { + client = createClient(config, awsSecurityConfig, passwordConfig, backpressureStats); + + this.ignorePublishAddress = config.isIgnorePublishAddress(); + this.scrollSize = config.getScrollSize(); + this.scrollTimeout = config.getScrollTimeout(); + this.refreshInterval = config.getNodeRefreshInterval(); + this.tlsEnabled = config.isTlsEnabled(); + } + + @PostConstruct + public void initialize() + { + if (!started.getAndSet(true)) { + // do the first refresh eagerly + refreshNodes(); + + executor.scheduleWithFixedDelay(this::refreshNodes, refreshInterval.toMillis(), refreshInterval.toMillis(), MILLISECONDS); + } + } + + @PreDestroy + public void close() + throws IOException + { + executor.shutdownNow(); + client.close(); + } + + private void refreshNodes() + { + // discover other nodes in the cluster and add them to the client + try { + Set nodes = fetchNodes(); + + HttpHost[] hosts = nodes.stream() + .map(ElasticsearchNode::getAddress) + .filter(Optional::isPresent) + .map(Optional::get) + .map(address -> HttpHost.create(format("%s://%s", tlsEnabled ? "https" : "http", address))) + .toArray(HttpHost[]::new); + + if (hosts.length > 0 && !ignorePublishAddress) { + client.getLowLevelClient().setHosts(hosts); + } + + this.nodes.set(nodes); + } + catch (Throwable e) { + // Catch all exceptions here since throwing an exception from executor#scheduleWithFixedDelay method + // suppresses all future scheduled invocations + LOG.error(e, "Error refreshing nodes"); + } + } + + private static BackpressureRestHighLevelClient createClient( + ElasticsearchConfig config, + Optional awsSecurityConfig, + Optional passwordConfig, + TimeStat backpressureStats) + { + RestClientBuilder builder = RestClient.builder( + config.getHosts().stream() + .map(httpHost -> new HttpHost(httpHost, config.getPort(), config.isTlsEnabled() ? "https" : "http")) + .toArray(HttpHost[]::new)) + .setMaxRetryTimeoutMillis(toIntExact(config.getMaxRetryTime().toMillis())); + + builder.setHttpClientConfigCallback(ignored -> { + RequestConfig requestConfig = RequestConfig.custom() + .setConnectTimeout(toIntExact(config.getConnectTimeout().toMillis())) + .setSocketTimeout(toIntExact(config.getRequestTimeout().toMillis())) + .build(); + + IOReactorConfig reactorConfig = IOReactorConfig.custom() + .setIoThreadCount(config.getHttpThreadCount()) + .build(); + + // the client builder passed to the call-back is configured to use system properties, which makes it + // impossible to configure concurrency settings, so we need to build a new one from scratch + HttpAsyncClientBuilder clientBuilder = HttpAsyncClientBuilder.create() + .setDefaultRequestConfig(requestConfig) + .setDefaultIOReactorConfig(reactorConfig) + .setMaxConnPerRoute(config.getMaxHttpConnections()) + .setMaxConnTotal(config.getMaxHttpConnections()); + if (config.isTlsEnabled()) { + buildSslContext(config.getKeystorePath(), config.getKeystorePassword(), config.getTrustStorePath(), config.getTruststorePassword()) + .ifPresent(clientBuilder::setSSLContext); + + if (!config.isVerifyHostnames()) { + clientBuilder.setSSLHostnameVerifier(NoopHostnameVerifier.INSTANCE); + } + } + + passwordConfig.ifPresent(securityConfig -> { + CredentialsProvider credentials = new BasicCredentialsProvider(); + credentials.setCredentials(AuthScope.ANY, new UsernamePasswordCredentials(securityConfig.getUser(), securityConfig.getPassword())); + clientBuilder.setDefaultCredentialsProvider(credentials); + }); + + awsSecurityConfig.ifPresent(securityConfig -> clientBuilder.addInterceptorLast(new AwsRequestSigner( + securityConfig.getRegion(), + getAwsCredentialsProvider(securityConfig)))); + + return clientBuilder; + }); + + return new BackpressureRestHighLevelClient(builder, config, backpressureStats); + } + + private static AWSCredentialsProvider getAwsCredentialsProvider(AwsSecurityConfig config) + { + AWSCredentialsProvider credentialsProvider = DefaultAWSCredentialsProviderChain.getInstance(); + + if (config.getAccessKey().isPresent() && config.getSecretKey().isPresent()) { + credentialsProvider = new AWSStaticCredentialsProvider(new BasicAWSCredentials( + config.getAccessKey().get(), + config.getSecretKey().get())); + } + + if (config.getIamRole().isPresent()) { + STSAssumeRoleSessionCredentialsProvider.Builder credentialsProviderBuilder = new STSAssumeRoleSessionCredentialsProvider.Builder(config.getIamRole().get(), "trino-session") + .withStsClient(AWSSecurityTokenServiceClientBuilder.standard() + .withRegion(config.getRegion()) + .withCredentials(credentialsProvider) + .build()); + config.getExternalId().ifPresent(credentialsProviderBuilder::withExternalId); + credentialsProvider = credentialsProviderBuilder.build(); + } + + return credentialsProvider; + } + + private static Optional buildSslContext( + Optional keyStorePath, + Optional keyStorePassword, + Optional trustStorePath, + Optional trustStorePassword) + { + if (keyStorePath.isEmpty() && trustStorePath.isEmpty()) { + return Optional.empty(); + } + + try { + return Optional.of(createSSLContext(keyStorePath, keyStorePassword, trustStorePath, trustStorePassword)); + } + catch (GeneralSecurityException | IOException e) { + throw new TrinoException(ELASTICSEARCH_SSL_INITIALIZATION_FAILURE, e); + } + } + + private Set fetchNodes() + { + NodesResponse nodesResponse = doRequest("/_nodes/http", NODES_RESPONSE_CODEC::fromJson); + + ImmutableSet.Builder result = ImmutableSet.builder(); + for (Map.Entry entry : nodesResponse.getNodes().entrySet()) { + String nodeId = entry.getKey(); + NodesResponse.Node node = entry.getValue(); + + if (!Sets.intersection(node.getRoles(), NODE_ROLES).isEmpty()) { + Optional address = node.getAddress() + .flatMap(ElasticsearchClient::extractAddress); + + result.add(new ElasticsearchNode(nodeId, address)); + } + } + + return result.build(); + } + + public Set getNodes() + { + return nodes.get(); + } + + public List getSearchShards(String index) + { + Map nodeById = getNodes().stream() + .collect(toImmutableMap(ElasticsearchNode::getId, Function.identity())); + + SearchShardsResponse shardsResponse = doRequest(format("/%s/_search_shards", index), SEARCH_SHARDS_RESPONSE_CODEC::fromJson); + + ImmutableList.Builder shards = ImmutableList.builder(); + List nodes = ImmutableList.copyOf(nodeById.values()); + + for (List shardGroup : shardsResponse.getShardGroups()) { + Optional candidate = shardGroup.stream() + .filter(shard -> shard.getNode() != null && nodeById.containsKey(shard.getNode())) + .min(this::shardPreference); + + SearchShardsResponse.Shard chosen; + ElasticsearchNode node; + if (candidate.isEmpty()) { + // pick an arbitrary shard with and assign to an arbitrary node + chosen = shardGroup.stream() + .min(this::shardPreference) + .get(); + node = nodes.get(chosen.getShard() % nodes.size()); + } + else { + chosen = candidate.get(); + node = nodeById.get(chosen.getNode()); + } + + shards.add(new Shard(chosen.getIndex(), chosen.getShard(), node.getAddress())); + } + + return shards.build(); + } + + private int shardPreference(SearchShardsResponse.Shard left, SearchShardsResponse.Shard right) + { + // Favor non-primary shards + if (left.isPrimary() == right.isPrimary()) { + return 0; + } + + return left.isPrimary() ? 1 : -1; + } + + public boolean indexExists(String index) + { + String path = format("/%s/_mappings", index); + + try { + Response response = client.getLowLevelClient() + .performRequest("GET", path); + + return response.getStatusLine().getStatusCode() == 200; + } + catch (ResponseException e) { + if (e.getResponse().getStatusLine().getStatusCode() == 404) { + return false; + } + throw new TrinoException(ELASTICSEARCH_CONNECTION_ERROR, e); + } + catch (IOException e) { + throw new TrinoException(ELASTICSEARCH_CONNECTION_ERROR, e); + } + } + + public List getIndexes() + { + return doRequest("/_cat/indices?h=index,docs.count,docs.deleted&format=json&s=index:asc", body -> { + try { + ImmutableList.Builder result = ImmutableList.builder(); + JsonNode root = OBJECT_MAPPER.readTree(body); + for (int i = 0; i < root.size(); i++) { + String index = root.get(i).get("index").asText(); + // make sure the index has mappings we can use to derive the schema + int docsCount = root.get(i).get("docs.count").asInt(); + int deletedDocsCount = root.get(i).get("docs.deleted").asInt(); + if (docsCount == 0 && deletedDocsCount == 0) { + // without documents, the index won't have any dynamic mappings, but maybe there are some explicit ones + if (getIndexMetadata(index).getSchema().getFields().isEmpty()) { + continue; + } + } + result.add(index); + } + return result.build(); + } + catch (IOException e) { + throw new TrinoException(ELASTICSEARCH_INVALID_RESPONSE, e); + } + }); + } + + public Map> getAliases() + { + return doRequest("/_aliases", body -> { + try { + ImmutableMap.Builder> result = ImmutableMap.builder(); + JsonNode root = OBJECT_MAPPER.readTree(body); + + Iterator> elements = root.fields(); + while (elements.hasNext()) { + Map.Entry element = elements.next(); + JsonNode aliases = element.getValue().get("aliases"); + Iterator aliasNames = aliases.fieldNames(); + if (aliasNames.hasNext()) { + result.put(element.getKey(), ImmutableList.copyOf(aliasNames)); + } + } + return result.buildOrThrow(); + } + catch (IOException e) { + throw new TrinoException(ELASTICSEARCH_INVALID_RESPONSE, e); + } + }); + } + + public IndexMetadata getIndexMetadata(String index) + { + String path = format("/%s/_mappings", index); + + return doRequest(path, body -> { + try { + JsonNode mappings = OBJECT_MAPPER.readTree(body) + .elements().next() + .get("mappings"); + + if (!mappings.elements().hasNext()) { + return new IndexMetadata(new IndexMetadata.ObjectType(ImmutableList.of())); + } + if (!mappings.has("properties")) { + // Older versions of ElasticSearch supported multiple "type" mappings + // for a given index. Newer versions support only one and don't + // expose it in the document. Here we skip it if it's present. + mappings = mappings.elements().next(); + + if (!mappings.has("properties")) { + return new IndexMetadata(new IndexMetadata.ObjectType(ImmutableList.of())); + } + } + + JsonNode metaNode = nullSafeNode(mappings, "_meta"); + + JsonNode metaProperties = nullSafeNode(metaNode, "trino"); + + //stay backwards compatible with _meta.presto namespace for meta properties for some releases + if (metaProperties.isNull()) { + metaProperties = nullSafeNode(metaNode, "presto"); + } + + return new IndexMetadata(parseType(mappings.get("properties"), metaProperties)); + } + catch (IOException e) { + throw new TrinoException(ELASTICSEARCH_INVALID_RESPONSE, e); + } + }); + } + + private IndexMetadata.ObjectType parseType(JsonNode properties, JsonNode metaProperties) + { + Iterator> entries = properties.fields(); + + ImmutableList.Builder result = ImmutableList.builder(); + while (entries.hasNext()) { + Map.Entry field = entries.next(); + + String name = field.getKey(); + JsonNode value = field.getValue(); + + //default type is object + String type = "object"; + if (value.has("type")) { + type = value.get("type").asText(); + } + JsonNode metaNode = nullSafeNode(metaProperties, name); + boolean isArray = !metaNode.isNull() && metaNode.has("isArray") && metaNode.get("isArray").asBoolean(); + boolean asRawJson = !metaNode.isNull() && metaNode.has("asRawJson") && metaNode.get("asRawJson").asBoolean(); + + // While it is possible to handle isArray and asRawJson in the same column by creating a ARRAY(VARCHAR) type, we chose not to take + // this route, as it will likely lead to confusion in dealing with array syntax in Trino and potentially nested array and other + // syntax when parsing the raw json. + if (isArray && asRawJson) { + throw new TrinoException(ELASTICSEARCH_INVALID_METADATA, + format("A column, (%s) cannot be declared as a Trino array and also be rendered as json.", name)); + } + + switch (type) { + case "date": + List formats = ImmutableList.of(); + if (value.has("format")) { + formats = Arrays.asList(value.get("format").asText().split("\\|\\|")); + } + result.add(new IndexMetadata.Field(asRawJson, isArray, name, new IndexMetadata.DateTimeType(formats))); + break; + case "scaled_float": + result.add(new IndexMetadata.Field(asRawJson, isArray, name, new IndexMetadata.ScaledFloatType(value.get("scaling_factor").asDouble()))); + break; + case "nested": + case "object": + if (value.has("properties")) { + result.add(new IndexMetadata.Field(asRawJson, isArray, name, parseType(value.get("properties"), metaNode))); + } + else { + LOG.debug("Ignoring empty object field: %s", name); + } + break; + + default: + result.add(new IndexMetadata.Field(asRawJson, isArray, name, new IndexMetadata.PrimitiveType(type))); + } + } + + return new IndexMetadata.ObjectType(result.build()); + } + + private JsonNode nullSafeNode(JsonNode jsonNode, String name) + { + if (jsonNode == null || jsonNode.isNull() || jsonNode.get(name) == null) { + return NullNode.getInstance(); + } + return jsonNode.get(name); + } + + public String executeQuery(String index, String query) + { + String path = format("/%s/_search", index); + + Response response; + try { + response = client.getLowLevelClient() + .performRequest( + "GET", + path, + ImmutableMap.of(), + new ByteArrayEntity(query.getBytes(UTF_8)), + new BasicHeader("Content-Type", "application/json"), + new BasicHeader("Accept-Encoding", "application/json")); + } + catch (IOException e) { + throw new TrinoException(ELASTICSEARCH_CONNECTION_ERROR, e); + } + + String body; + try { + body = EntityUtils.toString(response.getEntity()); + } + catch (IOException e) { + throw new TrinoException(ELASTICSEARCH_INVALID_RESPONSE, e); + } + + return body; + } + + public SearchResponse beginSearch(String index, int shard, QueryBuilder query, Optional> fields, List documentFields, Optional sort, OptionalLong limit) + { + SearchSourceBuilder sourceBuilder = SearchSourceBuilder.searchSource() + .query(query); + + if (limit.isPresent() && limit.getAsLong() < scrollSize) { + // Safe to cast it to int because scrollSize is int. + sourceBuilder.size(toIntExact(limit.getAsLong())); + } + else { + sourceBuilder.size(scrollSize); + } + + sort.ifPresent(sourceBuilder::sort); + + fields.ifPresent(values -> { + if (values.isEmpty()) { + sourceBuilder.fetchSource(false); + } + else { + sourceBuilder.fetchSource(values.toArray(new String[0]), null); + } + }); + documentFields.forEach(sourceBuilder::docValueField); + + LOG.debug("Begin search: %s:%s, query: %s", index, shard, sourceBuilder); + + SearchRequest request = new SearchRequest(index) + .searchType(QUERY_THEN_FETCH) + .preference("_shards:" + shard) + .scroll(new TimeValue(scrollTimeout.toMillis())) + .source(sourceBuilder); + + long start = System.nanoTime(); + try { + return client.search(request); + } + catch (IOException e) { + throw new TrinoException(ELASTICSEARCH_CONNECTION_ERROR, e); + } + catch (ElasticsearchStatusException e) { + Throwable[] suppressed = e.getSuppressed(); + if (suppressed.length > 0) { + Throwable cause = suppressed[0]; + if (cause instanceof ResponseException) { + throw propagate((ResponseException) cause); + } + } + + throw new TrinoException(ELASTICSEARCH_CONNECTION_ERROR, e); + } + finally { + searchStats.add(Duration.nanosSince(start)); + } + } + + public SearchResponse nextPage(String scrollId) + { + LOG.debug("Next page: %s", scrollId); + + SearchScrollRequest request = new SearchScrollRequest(scrollId) + .scroll(new TimeValue(scrollTimeout.toMillis())); + + long start = System.nanoTime(); + try { + return client.searchScroll(request); + } + catch (IOException e) { + throw new TrinoException(ELASTICSEARCH_CONNECTION_ERROR, e); + } + finally { + nextPageStats.add(Duration.nanosSince(start)); + } + } + + public long count(String index, int shard, QueryBuilder query) + { + SearchSourceBuilder sourceBuilder = SearchSourceBuilder.searchSource() + .query(query); + + LOG.debug("Count: %s:%s, query: %s", index, shard, sourceBuilder); + + long start = System.nanoTime(); + try { + Response response; + try { + response = client.getLowLevelClient() + .performRequest( + "GET", + format("/%s/_count?preference=_shards:%s", index, shard), + ImmutableMap.of(), + new StringEntity(sourceBuilder.toString()), + new BasicHeader("Content-Type", "application/json")); + } + catch (ResponseException e) { + throw propagate(e); + } + catch (IOException e) { + throw new TrinoException(ELASTICSEARCH_CONNECTION_ERROR, e); + } + + try { + return COUNT_RESPONSE_CODEC.fromJson(EntityUtils.toByteArray(response.getEntity())) + .getCount(); + } + catch (IOException e) { + throw new TrinoException(ELASTICSEARCH_INVALID_RESPONSE, e); + } + } + finally { + countStats.add(Duration.nanosSince(start)); + } + } + + public void clearScroll(String scrollId) + { + ClearScrollRequest request = new ClearScrollRequest(); + request.addScrollId(scrollId); + try { + client.clearScroll(request); + } + catch (IOException e) { + throw new TrinoException(ELASTICSEARCH_CONNECTION_ERROR, e); + } + } + + @Managed + @Nested + public TimeStat getSearchStats() + { + return searchStats; + } + + @Managed + @Nested + public TimeStat getNextPageStats() + { + return nextPageStats; + } + + @Managed + @Nested + public TimeStat getCountStats() + { + return countStats; + } + + @Managed + @Nested + public TimeStat getBackpressureStats() + { + return backpressureStats; + } + + private T doRequest(String path, ResponseHandler handler) + { + checkArgument(path.startsWith("/"), "path must be an absolute path"); + + Response response; + try { + response = client.getLowLevelClient() + .performRequest("GET", path); + } + catch (IOException e) { + throw new TrinoException(ELASTICSEARCH_CONNECTION_ERROR, e); + } + + String body; + try { + body = EntityUtils.toString(response.getEntity()); + } + catch (IOException e) { + throw new TrinoException(ELASTICSEARCH_INVALID_RESPONSE, e); + } + + return handler.process(body); + } + + private static TrinoException propagate(ResponseException exception) + { + HttpEntity entity = exception.getResponse().getEntity(); + + if (entity != null && entity.getContentType() != null) { + try { + JsonNode reason = OBJECT_MAPPER.readTree(entity.getContent()).path("error") + .path("root_cause") + .path(0) + .path("reason"); + + if (!reason.isMissingNode()) { + throw new TrinoException(ELASTICSEARCH_QUERY_FAILURE, reason.asText(), exception); + } + } + catch (IOException e) { + TrinoException result = new TrinoException(ELASTICSEARCH_QUERY_FAILURE, exception); + result.addSuppressed(e); + throw result; + } + } + + throw new TrinoException(ELASTICSEARCH_QUERY_FAILURE, exception); + } + + @VisibleForTesting + static Optional extractAddress(String address) + { + Matcher matcher = ADDRESS_PATTERN.matcher(address); + + if (!matcher.matches()) { + return Optional.empty(); + } + + String cname = matcher.group("cname"); + String ip = matcher.group("ip"); + String port = matcher.group("port"); + + if (cname != null) { + return Optional.of(cname + ":" + port); + } + + return Optional.of(ip + ":" + port); + } + + private interface ResponseHandler + { + T process(String body); + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/ElasticsearchNode.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/ElasticsearchNode.java new file mode 100644 index 0000000000000..32c51ec56fc74 --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/ElasticsearchNode.java @@ -0,0 +1,46 @@ +/* + * 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.elasticsearch.client; + +import java.util.Optional; + +import static java.util.Objects.requireNonNull; + +public class ElasticsearchNode +{ + private final String id; + private final Optional address; + + public ElasticsearchNode(String id, Optional address) + { + this.id = requireNonNull(id, "id is null"); + this.address = requireNonNull(address, "address is null"); + } + + public String getId() + { + return id; + } + + public Optional getAddress() + { + return address; + } + + @Override + public String toString() + { + return id + "@" + address.orElse(""); + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/IndexMetadata.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/IndexMetadata.java new file mode 100644 index 0000000000000..2a73fafeaabf2 --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/IndexMetadata.java @@ -0,0 +1,145 @@ +/* + * 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.elasticsearch.client; + +import com.google.common.collect.ImmutableList; + +import java.util.List; + +import static com.google.common.base.Preconditions.checkArgument; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +public class IndexMetadata +{ + private final ObjectType schema; + + public IndexMetadata(ObjectType schema) + { + this.schema = requireNonNull(schema, "schema is null"); + } + + public ObjectType getSchema() + { + return schema; + } + + public static class Field + { + private final boolean asRawJson; + private final boolean isArray; + private final String name; + private final Type type; + + public Field(boolean asRawJson, boolean isArray, String name, Type type) + { + checkArgument(!asRawJson || !isArray, + format("A column, (%s) cannot be declared as a Trino array and also be rendered as json.", name)); + this.asRawJson = asRawJson; + this.isArray = isArray; + this.name = requireNonNull(name, "name is null"); + this.type = requireNonNull(type, "type is null"); + } + + public boolean asRawJson() + { + return asRawJson; + } + + public boolean isArray() + { + return isArray; + } + + public String getName() + { + return name; + } + + public Type getType() + { + return type; + } + } + + public interface Type {} + + public static class PrimitiveType + implements Type + { + private final String name; + + public PrimitiveType(String name) + { + this.name = requireNonNull(name, "name is null"); + } + + public String getName() + { + return name; + } + } + + public static class DateTimeType + implements Type + { + private final List formats; + + public DateTimeType(List formats) + { + requireNonNull(formats, "formats is null"); + + this.formats = ImmutableList.copyOf(formats); + } + + public List getFormats() + { + return formats; + } + } + + public static class ObjectType + implements Type + { + private final List fields; + + public ObjectType(List fields) + { + requireNonNull(fields, "fields is null"); + + this.fields = ImmutableList.copyOf(fields); + } + + public List getFields() + { + return fields; + } + } + + public static class ScaledFloatType + implements Type + { + private final double scale; + + public ScaledFloatType(double scale) + { + this.scale = scale; + } + + public double getScale() + { + return scale; + } + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/NodesResponse.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/NodesResponse.java new file mode 100644 index 0000000000000..c8f2067d4168e --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/NodesResponse.java @@ -0,0 +1,90 @@ +/* + * 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.elasticsearch.client; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; + +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +import static java.util.Objects.requireNonNull; + +public class NodesResponse +{ + private final Map nodes; + + @JsonCreator + public NodesResponse(@JsonProperty("nodes") Map nodes) + { + requireNonNull(nodes, "nodes is null"); + + this.nodes = ImmutableMap.copyOf(nodes); + } + + public Map getNodes() + { + return nodes; + } + + public static class Node + { + private final Set roles; + private final Optional http; + + @JsonCreator + public Node( + @JsonProperty("roles") Set roles, + @JsonProperty("http") Optional http) + { + this.roles = ImmutableSet.copyOf(roles); + this.http = requireNonNull(http, "http is null"); + } + + public Set getRoles() + { + return roles; + } + + public Optional getAddress() + { + return http.map(Http::getAddress); + } + } + + public static class Http + { + private final String address; + + @JsonCreator + public Http(@JsonProperty("publish_address") String address) + { + this.address = address; + } + + public String getAddress() + { + return address; + } + + @Override + public String toString() + { + return address; + } + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/SearchShardsResponse.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/SearchShardsResponse.java new file mode 100644 index 0000000000000..a68d208272dc7 --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/SearchShardsResponse.java @@ -0,0 +1,87 @@ +/* + * 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.elasticsearch.client; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +import static java.util.Objects.requireNonNull; + +public class SearchShardsResponse +{ + private final List> shardGroups; + + @JsonCreator + public SearchShardsResponse(@JsonProperty("shards") List> shardGroups) + { + requireNonNull(shardGroups, "shardGroups is null"); + + this.shardGroups = ImmutableList.copyOf(shardGroups); + } + + public List> getShardGroups() + { + return shardGroups; + } + + public static class Shard + { + private final String index; + private final boolean primary; + private final String node; + private final int shard; + + @JsonCreator + public Shard( + @JsonProperty("index") String index, + @JsonProperty("shard") int shard, + @JsonProperty("primary") boolean primary, + @JsonProperty("node") String node) + { + this.index = requireNonNull(index, "index is null"); + this.shard = shard; + this.primary = primary; + this.node = requireNonNull(node, "node is null"); + } + + public String getIndex() + { + return index; + } + + public boolean isPrimary() + { + return primary; + } + + public String getNode() + { + return node; + } + + public int getShard() + { + return shard; + } + + @Override + public String toString() + { + return index + ":" + shard + "@" + node + (primary ? "[primary]" : "[replica]"); + } + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/Shard.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/Shard.java new file mode 100644 index 0000000000000..2237a89a7bfac --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/Shard.java @@ -0,0 +1,53 @@ +/* + * 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.elasticsearch.client; + +import java.util.Optional; + +import static java.util.Objects.requireNonNull; + +public class Shard +{ + private final String index; + private final int id; + private final Optional address; + + public Shard(String index, int id, Optional address) + { + this.index = requireNonNull(index, "index is null"); + this.id = id; + this.address = requireNonNull(address, "address is null"); + } + + public String getIndex() + { + return index; + } + + public int getId() + { + return id; + } + + public Optional getAddress() + { + return address; + } + + @Override + public String toString() + { + return index + ":" + id + "@" + address.orElse(""); + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/ArrayDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/ArrayDecoder.java new file mode 100644 index 0000000000000..2c6bd89b500cc --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/ArrayDecoder.java @@ -0,0 +1,75 @@ +/* + * 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.elasticsearch.decoders; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.trino.plugin.elasticsearch.DecoderDescriptor; +import io.trino.spi.block.ArrayBlockBuilder; +import io.trino.spi.block.BlockBuilder; +import org.elasticsearch.search.SearchHit; + +import java.util.List; +import java.util.function.Supplier; + +public class ArrayDecoder + implements Decoder +{ + private final Decoder elementDecoder; + + public ArrayDecoder(Decoder elementDecoder) + { + this.elementDecoder = elementDecoder; + } + + @Override + public void decode(SearchHit hit, Supplier getter, BlockBuilder output) + { + Object data = getter.get(); + + if (data == null) { + output.appendNull(); + } + else if (data instanceof List list) { + ((ArrayBlockBuilder) output).buildEntry(elementBuilder -> list.forEach(element -> elementDecoder.decode(hit, () -> element, elementBuilder))); + } + else { + ((ArrayBlockBuilder) output).buildEntry(elementBuilder -> elementDecoder.decode(hit, () -> data, elementBuilder)); + } + } + + public static class Descriptor + implements DecoderDescriptor + { + private final DecoderDescriptor elementDescriptor; + + @JsonCreator + public Descriptor(DecoderDescriptor elementDescriptor) + { + this.elementDescriptor = elementDescriptor; + } + + @JsonProperty + public DecoderDescriptor getElementDescriptor() + { + return elementDescriptor; + } + + @Override + public Decoder createDecoder() + { + return new ArrayDecoder(elementDescriptor.createDecoder()); + } + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/BigintDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/BigintDecoder.java new file mode 100644 index 0000000000000..5543361744ac7 --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/BigintDecoder.java @@ -0,0 +1,90 @@ +/* + * 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.elasticsearch.decoders; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.trino.plugin.elasticsearch.DecoderDescriptor; +import io.trino.spi.TrinoException; +import io.trino.spi.block.BlockBuilder; +import org.elasticsearch.search.SearchHit; + +import java.util.function.Supplier; + +import static io.trino.spi.StandardErrorCode.TYPE_MISMATCH; +import static io.trino.spi.type.BigintType.BIGINT; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +public class BigintDecoder + implements Decoder +{ + private final String path; + + public BigintDecoder(String path) + { + this.path = requireNonNull(path, "path is null"); + } + + @Override + public void decode(SearchHit hit, Supplier getter, BlockBuilder output) + { + Object value = getter.get(); + if (value == null) { + output.appendNull(); + } + else if (value instanceof Number) { + BIGINT.writeLong(output, ((Number) value).longValue()); + } + else if (value instanceof String stringValue) { + if (stringValue.isEmpty()) { + output.appendNull(); + return; + } + try { + BIGINT.writeLong(output, Long.parseLong(stringValue)); + } + catch (NumberFormatException e) { + throw new TrinoException(TYPE_MISMATCH, format("Cannot parse value for field '%s' as BIGINT: %s", path, value)); + } + } + else { + throw new TrinoException(TYPE_MISMATCH, format("Expected a numeric value for field '%s' of type BIGINT: %s [%s]", path, value, value.getClass().getSimpleName())); + } + } + + public static class Descriptor + implements DecoderDescriptor + { + private final String path; + + @JsonCreator + public Descriptor(String path) + { + this.path = path; + } + + @JsonProperty + public String getPath() + { + return path; + } + + @Override + public Decoder createDecoder() + { + return new BigintDecoder(path); + } + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/BooleanDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/BooleanDecoder.java new file mode 100644 index 0000000000000..80895325e19d8 --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/BooleanDecoder.java @@ -0,0 +1,89 @@ +/* + * 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.elasticsearch.decoders; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.trino.plugin.elasticsearch.DecoderDescriptor; +import io.trino.spi.TrinoException; +import io.trino.spi.block.BlockBuilder; +import org.elasticsearch.search.SearchHit; + +import java.util.function.Supplier; + +import static io.trino.spi.StandardErrorCode.TYPE_MISMATCH; +import static io.trino.spi.type.BooleanType.BOOLEAN; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +public class BooleanDecoder + implements Decoder +{ + private final String path; + + public BooleanDecoder(String path) + { + this.path = requireNonNull(path, "path is null"); + } + + @Override + public void decode(SearchHit hit, Supplier getter, BlockBuilder output) + { + Object value = getter.get(); + if (value == null) { + output.appendNull(); + } + else if (value instanceof Boolean) { + BOOLEAN.writeBoolean(output, (Boolean) value); + } + else if (value instanceof String) { + if (value.equals("true")) { + BOOLEAN.writeBoolean(output, true); + } + else if (value.equals("false") || value.equals("")) { + BOOLEAN.writeBoolean(output, false); + } + else { + throw new TrinoException(TYPE_MISMATCH, format("Cannot parse value for field '%s' as BOOLEAN: %s", path, value)); + } + } + else { + throw new TrinoException(TYPE_MISMATCH, format("Expected a boolean value for field %s of type BOOLEAN: %s [%s]", path, value, value.getClass().getSimpleName())); + } + } + + public static class Descriptor + implements DecoderDescriptor + { + private final String path; + + @JsonCreator + public Descriptor(String path) + { + this.path = path; + } + + @JsonProperty + public String getPath() + { + return path; + } + + @Override + public Decoder createDecoder() + { + return new BooleanDecoder(path); + } + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/Decoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/Decoder.java new file mode 100644 index 0000000000000..8588a5b1300c1 --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/Decoder.java @@ -0,0 +1,24 @@ +/* + * 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.elasticsearch.decoders; + +import io.trino.spi.block.BlockBuilder; +import org.elasticsearch.search.SearchHit; + +import java.util.function.Supplier; + +public interface Decoder +{ + void decode(SearchHit hit, Supplier getter, BlockBuilder output); +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/DoubleDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/DoubleDecoder.java new file mode 100644 index 0000000000000..c0c27083c8cce --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/DoubleDecoder.java @@ -0,0 +1,95 @@ +/* + * 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.elasticsearch.decoders; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.trino.plugin.elasticsearch.DecoderDescriptor; +import io.trino.spi.TrinoException; +import io.trino.spi.block.BlockBuilder; +import org.elasticsearch.search.SearchHit; + +import java.util.function.Supplier; + +import static io.trino.spi.StandardErrorCode.TYPE_MISMATCH; +import static io.trino.spi.type.DoubleType.DOUBLE; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +public class DoubleDecoder + implements Decoder +{ + private final String path; + + public DoubleDecoder(String path) + { + this.path = requireNonNull(path, "path is null"); + } + + @Override + public void decode(SearchHit hit, Supplier getter, BlockBuilder output) + { + Object value = getter.get(); + if (value == null) { + output.appendNull(); + return; + } + + double decoded; + if (value instanceof Number number) { + decoded = number.doubleValue(); + } + else if (value instanceof String stringValue) { + if (stringValue.isEmpty()) { + output.appendNull(); + return; + } + try { + decoded = Double.parseDouble(stringValue); + } + catch (NumberFormatException e) { + throw new TrinoException(TYPE_MISMATCH, format("Cannot parse value for field '%s' as DOUBLE: %s", path, value)); + } + } + else { + throw new TrinoException(TYPE_MISMATCH, format("Expected a numeric value for field %s of type DOUBLE: %s [%s]", path, value, value.getClass().getSimpleName())); + } + + DOUBLE.writeDouble(output, decoded); + } + + public static class Descriptor + implements DecoderDescriptor + { + private final String path; + + @JsonCreator + public Descriptor(String path) + { + this.path = path; + } + + @JsonProperty + public String getPath() + { + return path; + } + + @Override + public Decoder createDecoder() + { + return new DoubleDecoder(path); + } + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/IdColumnDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/IdColumnDecoder.java new file mode 100644 index 0000000000000..f99c42da63f68 --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/IdColumnDecoder.java @@ -0,0 +1,43 @@ +/* + * 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.elasticsearch.decoders; + +import io.airlift.slice.Slices; +import io.trino.plugin.elasticsearch.DecoderDescriptor; +import io.trino.spi.block.BlockBuilder; +import org.elasticsearch.search.SearchHit; + +import java.util.function.Supplier; + +import static io.trino.spi.type.VarcharType.VARCHAR; + +public class IdColumnDecoder + implements Decoder +{ + @Override + public void decode(SearchHit hit, Supplier getter, BlockBuilder output) + { + VARCHAR.writeSlice(output, Slices.utf8Slice(hit.getId())); + } + + public static class Descriptor + implements DecoderDescriptor + { + @Override + public Decoder createDecoder() + { + return new IdColumnDecoder(); + } + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/IntegerDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/IntegerDecoder.java new file mode 100644 index 0000000000000..3581075310290 --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/IntegerDecoder.java @@ -0,0 +1,100 @@ +/* + * 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.elasticsearch.decoders; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.trino.plugin.elasticsearch.DecoderDescriptor; +import io.trino.spi.TrinoException; +import io.trino.spi.block.BlockBuilder; +import org.elasticsearch.search.SearchHit; + +import java.util.function.Supplier; + +import static io.trino.spi.StandardErrorCode.TYPE_MISMATCH; +import static io.trino.spi.type.IntegerType.INTEGER; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +public class IntegerDecoder + implements Decoder +{ + private final String path; + + public IntegerDecoder(String path) + { + this.path = requireNonNull(path, "path is null"); + } + + @Override + public void decode(SearchHit hit, Supplier getter, BlockBuilder output) + { + Object value = getter.get(); + + if (value == null) { + output.appendNull(); + return; + } + + long decoded; + if (value instanceof Number number) { + decoded = number.longValue(); + } + else if (value instanceof String stringValue) { + if (stringValue.isEmpty()) { + output.appendNull(); + return; + } + try { + decoded = Long.parseLong(stringValue); + } + catch (NumberFormatException e) { + throw new TrinoException(TYPE_MISMATCH, format("Cannot parse value for field '%s' as INTEGER: %s", path, value)); + } + } + else { + throw new TrinoException(TYPE_MISMATCH, format("Expected a numeric value for field '%s' of type INTEGER: %s [%s]", path, value, value.getClass().getSimpleName())); + } + + if (decoded < Integer.MIN_VALUE || decoded > Integer.MAX_VALUE) { + throw new TrinoException(TYPE_MISMATCH, format("Value out of range for field '%s' of type INTEGER: %s", path, decoded)); + } + + INTEGER.writeLong(output, decoded); + } + + public static class Descriptor + implements DecoderDescriptor + { + private final String path; + + @JsonCreator + public Descriptor(String path) + { + this.path = path; + } + + @JsonProperty + public String getPath() + { + return path; + } + + @Override + public Decoder createDecoder() + { + return new IntegerDecoder(path); + } + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/IpAddressDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/IpAddressDecoder.java new file mode 100644 index 0000000000000..bf2612b61f160 --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/IpAddressDecoder.java @@ -0,0 +1,124 @@ +/* + * 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.elasticsearch.decoders; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.net.InetAddresses; +import io.airlift.slice.Slice; +import io.airlift.slice.Slices; +import io.trino.plugin.elasticsearch.DecoderDescriptor; +import io.trino.spi.TrinoException; +import io.trino.spi.block.BlockBuilder; +import io.trino.spi.type.Type; +import org.elasticsearch.search.SearchHit; + +import java.util.function.Supplier; + +import static io.airlift.slice.Slices.wrappedBuffer; +import static io.trino.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR; +import static io.trino.spi.StandardErrorCode.INVALID_CAST_ARGUMENT; +import static io.trino.spi.StandardErrorCode.TYPE_MISMATCH; +import static java.lang.String.format; +import static java.lang.System.arraycopy; +import static java.util.Objects.requireNonNull; + +public class IpAddressDecoder + implements Decoder +{ + private final String path; + private final Type ipAddressType; + + public IpAddressDecoder(String path, Type type) + { + this.path = requireNonNull(path, "path is null"); + this.ipAddressType = requireNonNull(type, "type is null"); + } + + @Override + public void decode(SearchHit hit, Supplier getter, BlockBuilder output) + { + Object value = getter.get(); + if (value == null) { + output.appendNull(); + } + else if (value instanceof String address) { + Slice slice = castToIpAddress(Slices.utf8Slice(address)); + ipAddressType.writeSlice(output, slice); + } + else { + throw new TrinoException(TYPE_MISMATCH, format("Expected a string value for field '%s' of type IP: %s [%s]", path, value, value.getClass().getSimpleName())); + } + } + + // This is a copy of IpAddressOperators.castFromVarcharToIpAddress method + private Slice castToIpAddress(Slice slice) + { + byte[] address; + try { + address = InetAddresses.forString(slice.toStringUtf8()).getAddress(); + } + catch (IllegalArgumentException e) { + throw new TrinoException(INVALID_CAST_ARGUMENT, "Cannot cast value to IPADDRESS: " + slice.toStringUtf8()); + } + + byte[] bytes; + if (address.length == 4) { + bytes = new byte[16]; + bytes[10] = (byte) 0xff; + bytes[11] = (byte) 0xff; + arraycopy(address, 0, bytes, 12, 4); + } + else if (address.length == 16) { + bytes = address; + } + else { + throw new TrinoException(GENERIC_INTERNAL_ERROR, "Invalid InetAddress length: " + address.length); + } + + return wrappedBuffer(bytes); + } + + public static class Descriptor + implements DecoderDescriptor + { + private final String path; + private final Type ipAddressType; + + @JsonCreator + public Descriptor(String path, Type ipAddressType) + { + this.path = path; + this.ipAddressType = ipAddressType; + } + + @JsonProperty + public String getPath() + { + return path; + } + + @JsonProperty + public Type getIpAddressType() + { + return ipAddressType; + } + + @Override + public Decoder createDecoder() + { + return new IpAddressDecoder(path, ipAddressType); + } + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/RawJsonDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/RawJsonDecoder.java new file mode 100644 index 0000000000000..141113b393889 --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/RawJsonDecoder.java @@ -0,0 +1,89 @@ +/* + * 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.elasticsearch.decoders; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import io.airlift.json.ObjectMapperProvider; +import io.airlift.slice.Slices; +import io.trino.plugin.elasticsearch.DecoderDescriptor; +import io.trino.spi.TrinoException; +import io.trino.spi.block.BlockBuilder; +import org.elasticsearch.search.SearchHit; + +import java.util.function.Supplier; + +import static io.trino.spi.StandardErrorCode.TYPE_MISMATCH; +import static io.trino.spi.type.VarcharType.VARCHAR; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +public class RawJsonDecoder + implements Decoder +{ + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapperProvider().get(); + private final String path; + + public RawJsonDecoder(String path) + { + this.path = requireNonNull(path, "path is null"); + } + + @Override + public void decode(SearchHit hit, Supplier getter, BlockBuilder output) + { + Object value = getter.get(); + if (value == null) { + output.appendNull(); + } + else { + try { + String rawJsonValue = OBJECT_MAPPER.writeValueAsString(value); + VARCHAR.writeSlice(output, Slices.utf8Slice(rawJsonValue)); + } + catch (JsonProcessingException e) { + throw new TrinoException( + TYPE_MISMATCH, + format("Expected valid json for field '%s' marked to be rendered as JSON: %s [%s]", path, value, value.getClass().getSimpleName()), + e); + } + } + } + + public static class Descriptor + implements DecoderDescriptor + { + private final String path; + + @JsonCreator + public Descriptor(String path) + { + this.path = path; + } + + @JsonProperty + public String getPath() + { + return path; + } + + @Override + public Decoder createDecoder() + { + return new RawJsonDecoder(path); + } + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/RealDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/RealDecoder.java new file mode 100644 index 0000000000000..c9a09131a1842 --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/RealDecoder.java @@ -0,0 +1,95 @@ +/* + * 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.elasticsearch.decoders; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.trino.plugin.elasticsearch.DecoderDescriptor; +import io.trino.spi.TrinoException; +import io.trino.spi.block.BlockBuilder; +import org.elasticsearch.search.SearchHit; + +import java.util.function.Supplier; + +import static io.trino.spi.StandardErrorCode.TYPE_MISMATCH; +import static io.trino.spi.type.RealType.REAL; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +public class RealDecoder + implements Decoder +{ + private final String path; + + public RealDecoder(String path) + { + this.path = requireNonNull(path, "path is null"); + } + + @Override + public void decode(SearchHit hit, Supplier getter, BlockBuilder output) + { + Object value = getter.get(); + if (value == null) { + output.appendNull(); + return; + } + + float decoded; + if (value instanceof Number number) { + decoded = number.floatValue(); + } + else if (value instanceof String stringValue) { + if (stringValue.isEmpty()) { + output.appendNull(); + return; + } + try { + decoded = Float.parseFloat(stringValue); + } + catch (NumberFormatException e) { + throw new TrinoException(TYPE_MISMATCH, format("Cannot parse value for field '%s' as REAL: %s", path, value)); + } + } + else { + throw new TrinoException(TYPE_MISMATCH, format("Expected a numeric value for field %s of type REAL: %s [%s]", path, value, value.getClass().getSimpleName())); + } + + REAL.writeLong(output, Float.floatToRawIntBits(decoded)); + } + + public static class Descriptor + implements DecoderDescriptor + { + private final String path; + + @JsonCreator + public Descriptor(String path) + { + this.path = path; + } + + @JsonProperty + public String getPath() + { + return path; + } + + @Override + public Decoder createDecoder() + { + return new RealDecoder(path); + } + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/RowDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/RowDecoder.java new file mode 100644 index 0000000000000..abcaf7b3e75b2 --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/RowDecoder.java @@ -0,0 +1,132 @@ +/* + * 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.elasticsearch.decoders; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.trino.plugin.elasticsearch.DecoderDescriptor; +import io.trino.spi.TrinoException; +import io.trino.spi.block.BlockBuilder; +import io.trino.spi.block.RowBlockBuilder; +import org.elasticsearch.search.SearchHit; + +import java.util.List; +import java.util.Map; +import java.util.function.Supplier; + +import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.trino.plugin.elasticsearch.ScanQueryPageSource.getField; +import static io.trino.spi.StandardErrorCode.TYPE_MISMATCH; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +public class RowDecoder + implements Decoder +{ + private final String path; + private final List fieldNames; + private final List decoders; + + public RowDecoder(String path, List fieldNames, List decoders) + { + this.path = requireNonNull(path, "path is null"); + this.fieldNames = fieldNames; + this.decoders = decoders; + } + + @Override + public void decode(SearchHit hit, Supplier getter, BlockBuilder output) + { + Object data = getter.get(); + + if (data == null) { + output.appendNull(); + } + else if (data instanceof Map) { + ((RowBlockBuilder) output).buildEntry(fieldBuilders -> { + for (int i = 0; i < decoders.size(); i++) { + String field = fieldNames.get(i); + decoders.get(i).decode(hit, () -> getField((Map) data, field), fieldBuilders.get(i)); + } + }); + } + else { + throw new TrinoException(TYPE_MISMATCH, format("Expected object for field '%s' of type ROW: %s [%s]", path, data, data.getClass().getSimpleName())); + } + } + + public static class Descriptor + implements DecoderDescriptor + { + private final String path; + private final List fields; + + @JsonCreator + public Descriptor(String path, List fields) + { + this.path = path; + this.fields = fields; + } + + @JsonProperty + public String getPath() + { + return path; + } + + @JsonProperty + public List getFields() + { + return fields; + } + + @Override + public Decoder createDecoder() + { + return new RowDecoder( + path, + fields.stream() + .map(NameAndDescriptor::getName) + .collect(toImmutableList()), + fields.stream() + .map(field -> field.getDescriptor().createDecoder()) + .collect(toImmutableList())); + } + } + + public static class NameAndDescriptor + { + private final String name; + private final DecoderDescriptor descriptor; + + @JsonCreator + public NameAndDescriptor(String name, DecoderDescriptor descriptor) + { + this.name = name; + this.descriptor = descriptor; + } + + @JsonProperty + public String getName() + { + return name; + } + + @JsonProperty + public DecoderDescriptor getDescriptor() + { + return descriptor; + } + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/ScoreColumnDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/ScoreColumnDecoder.java new file mode 100644 index 0000000000000..55722e31cab49 --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/ScoreColumnDecoder.java @@ -0,0 +1,42 @@ +/* + * 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.elasticsearch.decoders; + +import io.trino.plugin.elasticsearch.DecoderDescriptor; +import io.trino.spi.block.BlockBuilder; +import org.elasticsearch.search.SearchHit; + +import java.util.function.Supplier; + +import static io.trino.spi.type.RealType.REAL; + +public class ScoreColumnDecoder + implements Decoder +{ + @Override + public void decode(SearchHit hit, Supplier getter, BlockBuilder output) + { + REAL.writeLong(output, Float.floatToRawIntBits(hit.getScore())); + } + + public static class Descriptor + implements DecoderDescriptor + { + @Override + public Decoder createDecoder() + { + return new ScoreColumnDecoder(); + } + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/SmallintDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/SmallintDecoder.java new file mode 100644 index 0000000000000..1e2f87e1b2729 --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/SmallintDecoder.java @@ -0,0 +1,99 @@ +/* + * 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.elasticsearch.decoders; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.trino.plugin.elasticsearch.DecoderDescriptor; +import io.trino.spi.TrinoException; +import io.trino.spi.block.BlockBuilder; +import org.elasticsearch.search.SearchHit; + +import java.util.function.Supplier; + +import static io.trino.spi.StandardErrorCode.TYPE_MISMATCH; +import static io.trino.spi.type.SmallintType.SMALLINT; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +public class SmallintDecoder + implements Decoder +{ + private final String path; + + public SmallintDecoder(String path) + { + this.path = requireNonNull(path, "path is null"); + } + + @Override + public void decode(SearchHit hit, Supplier getter, BlockBuilder output) + { + Object value = getter.get(); + if (value == null) { + output.appendNull(); + return; + } + + long decoded; + if (value instanceof Number number) { + decoded = number.longValue(); + } + else if (value instanceof String stringValue) { + if (stringValue.isEmpty()) { + output.appendNull(); + return; + } + try { + decoded = Long.parseLong(stringValue); + } + catch (NumberFormatException e) { + throw new TrinoException(TYPE_MISMATCH, format("Cannot parse value for field '%s' as SMALLINT: %s", path, value)); + } + } + else { + throw new TrinoException(TYPE_MISMATCH, format("Expected a numeric value for field '%s' of type SMALLINT: %s [%s]", path, value, value.getClass().getSimpleName())); + } + + if (decoded < Short.MIN_VALUE || decoded > Short.MAX_VALUE) { + throw new TrinoException(TYPE_MISMATCH, format("Value out of range for field '%s' of type SMALLINT: %s", path, decoded)); + } + + SMALLINT.writeLong(output, decoded); + } + + public static class Descriptor + implements DecoderDescriptor + { + private final String path; + + @JsonCreator + public Descriptor(String path) + { + this.path = path; + } + + @JsonProperty + public String getPath() + { + return path; + } + + @Override + public Decoder createDecoder() + { + return new SmallintDecoder(path); + } + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/SourceColumnDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/SourceColumnDecoder.java new file mode 100644 index 0000000000000..b08b173dd79aa --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/SourceColumnDecoder.java @@ -0,0 +1,43 @@ +/* + * 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.elasticsearch.decoders; + +import io.airlift.slice.Slices; +import io.trino.plugin.elasticsearch.DecoderDescriptor; +import io.trino.spi.block.BlockBuilder; +import org.elasticsearch.search.SearchHit; + +import java.util.function.Supplier; + +import static io.trino.spi.type.VarcharType.VARCHAR; + +public class SourceColumnDecoder + implements Decoder +{ + @Override + public void decode(SearchHit hit, Supplier getter, BlockBuilder output) + { + VARCHAR.writeSlice(output, Slices.utf8Slice(hit.getSourceAsString())); + } + + public static class Descriptor + implements DecoderDescriptor + { + @Override + public Decoder createDecoder() + { + return new SourceColumnDecoder(); + } + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/TimestampDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/TimestampDecoder.java new file mode 100644 index 0000000000000..5d3480586f07d --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/TimestampDecoder.java @@ -0,0 +1,118 @@ +/* + * 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.elasticsearch.decoders; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.primitives.Longs; +import io.trino.plugin.elasticsearch.DecoderDescriptor; +import io.trino.spi.TrinoException; +import io.trino.spi.block.BlockBuilder; +import org.elasticsearch.common.document.DocumentField; +import org.elasticsearch.search.SearchHit; + +import java.time.Instant; +import java.time.LocalDateTime; +import java.util.function.Supplier; + +import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; +import static io.trino.spi.StandardErrorCode.TYPE_MISMATCH; +import static io.trino.spi.type.TimestampType.TIMESTAMP_MILLIS; +import static io.trino.spi.type.Timestamps.MICROSECONDS_PER_MILLISECOND; +import static java.lang.String.format; +import static java.time.ZoneOffset.UTC; +import static java.time.format.DateTimeFormatter.ISO_DATE_TIME; +import static java.util.Objects.requireNonNull; + +public class TimestampDecoder + implements Decoder +{ + private final String path; + + public TimestampDecoder(String path) + { + this.path = requireNonNull(path, "path is null"); + } + + @Override + public void decode(SearchHit hit, Supplier getter, BlockBuilder output) + { + DocumentField documentField = hit.getFields().get(path); + Object value; + + if (documentField != null) { + if (documentField.getValues().size() > 1) { + throw new TrinoException(TYPE_MISMATCH, format("Expected single value for column '%s', found: %s", path, documentField.getValues().size())); + } + value = documentField.getValue(); + } + else { + value = getter.get(); + } + + if (value == null) { + output.appendNull(); + } + else { + LocalDateTime timestamp; + if (value instanceof String valueString) { + Long epochMillis = Longs.tryParse(valueString); + if (epochMillis != null) { + timestamp = LocalDateTime.ofInstant(Instant.ofEpochMilli(epochMillis), UTC); + } + else { + timestamp = ISO_DATE_TIME.parse(valueString, LocalDateTime::from); + } + } + else if (value instanceof Number) { + timestamp = LocalDateTime.ofInstant(Instant.ofEpochMilli(((Number) value).longValue()), UTC); + } + else { + throw new TrinoException(NOT_SUPPORTED, format( + "Unsupported representation for field '%s' of type TIMESTAMP: %s [%s]", + path, + value, + value.getClass().getSimpleName())); + } + + long epochMicros = timestamp.atOffset(UTC).toInstant().toEpochMilli() * MICROSECONDS_PER_MILLISECOND; + + TIMESTAMP_MILLIS.writeLong(output, epochMicros); + } + } + + public static class Descriptor + implements DecoderDescriptor + { + private final String path; + + @JsonCreator + public Descriptor(String path) + { + this.path = path; + } + + @JsonProperty + public String getPath() + { + return path; + } + + @Override + public Decoder createDecoder() + { + return new TimestampDecoder(path); + } + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/TinyintDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/TinyintDecoder.java new file mode 100644 index 0000000000000..ac6c5f3802e21 --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/TinyintDecoder.java @@ -0,0 +1,99 @@ +/* + * 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.elasticsearch.decoders; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.trino.plugin.elasticsearch.DecoderDescriptor; +import io.trino.spi.TrinoException; +import io.trino.spi.block.BlockBuilder; +import org.elasticsearch.search.SearchHit; + +import java.util.function.Supplier; + +import static io.trino.spi.StandardErrorCode.TYPE_MISMATCH; +import static io.trino.spi.type.TinyintType.TINYINT; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +public class TinyintDecoder + implements Decoder +{ + private final String path; + + public TinyintDecoder(String path) + { + this.path = requireNonNull(path, "path is null"); + } + + @Override + public void decode(SearchHit hit, Supplier getter, BlockBuilder output) + { + Object value = getter.get(); + if (value == null) { + output.appendNull(); + return; + } + + long decoded; + if (value instanceof Number number) { + decoded = number.longValue(); + } + else if (value instanceof String stringValue) { + if (stringValue.isEmpty()) { + output.appendNull(); + return; + } + try { + decoded = Long.parseLong(stringValue); + } + catch (NumberFormatException e) { + throw new TrinoException(TYPE_MISMATCH, format("Cannot parse value for field '%s' as TINYINT: %s", path, value)); + } + } + else { + throw new TrinoException(TYPE_MISMATCH, format("Expected a numeric value for field '%s' of type TINYINT: %s [%s]", path, value, value.getClass().getSimpleName())); + } + + if (decoded < Byte.MIN_VALUE || decoded > Byte.MAX_VALUE) { + throw new TrinoException(TYPE_MISMATCH, format("Value out of range for field '%s' of type TINYINT: %s", path, decoded)); + } + + TINYINT.writeLong(output, decoded); + } + + public static class Descriptor + implements DecoderDescriptor + { + private final String path; + + @JsonCreator + public Descriptor(String path) + { + this.path = path; + } + + @JsonProperty + public String getPath() + { + return path; + } + + @Override + public Decoder createDecoder() + { + return new TinyintDecoder(path); + } + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/VarbinaryDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/VarbinaryDecoder.java new file mode 100644 index 0000000000000..a81ce6a78c361 --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/VarbinaryDecoder.java @@ -0,0 +1,80 @@ +/* + * 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.elasticsearch.decoders; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.airlift.slice.Slices; +import io.trino.plugin.elasticsearch.DecoderDescriptor; +import io.trino.spi.TrinoException; +import io.trino.spi.block.BlockBuilder; +import org.elasticsearch.search.SearchHit; + +import java.util.Base64; +import java.util.function.Supplier; + +import static io.trino.spi.StandardErrorCode.TYPE_MISMATCH; +import static io.trino.spi.type.VarbinaryType.VARBINARY; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +public class VarbinaryDecoder + implements Decoder +{ + private final String path; + + public VarbinaryDecoder(String path) + { + this.path = requireNonNull(path, "path is null"); + } + + @Override + public void decode(SearchHit hit, Supplier getter, BlockBuilder output) + { + Object value = getter.get(); + if (value == null) { + output.appendNull(); + } + else if (value instanceof String) { + VARBINARY.writeSlice(output, Slices.wrappedBuffer(Base64.getDecoder().decode(value.toString()))); + } + else { + throw new TrinoException(TYPE_MISMATCH, format("Expected a string value for field '%s' of type VARBINARY: %s [%s]", path, value, value.getClass().getSimpleName())); + } + } + + public static class Descriptor + implements DecoderDescriptor + { + private final String path; + + @JsonCreator + public Descriptor(String path) + { + this.path = path; + } + + @JsonProperty + public String getPath() + { + return path; + } + + @Override + public Decoder createDecoder() + { + return new VarbinaryDecoder(path); + } + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/VarcharDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/VarcharDecoder.java new file mode 100644 index 0000000000000..70dd2d1755385 --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/VarcharDecoder.java @@ -0,0 +1,79 @@ +/* + * 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.elasticsearch.decoders; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.airlift.slice.Slices; +import io.trino.plugin.elasticsearch.DecoderDescriptor; +import io.trino.spi.TrinoException; +import io.trino.spi.block.BlockBuilder; +import org.elasticsearch.search.SearchHit; + +import java.util.function.Supplier; + +import static io.trino.spi.StandardErrorCode.TYPE_MISMATCH; +import static io.trino.spi.type.VarcharType.VARCHAR; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +public class VarcharDecoder + implements Decoder +{ + private final String path; + + public VarcharDecoder(String path) + { + this.path = requireNonNull(path, "path is null"); + } + + @Override + public void decode(SearchHit hit, Supplier getter, BlockBuilder output) + { + Object value = getter.get(); + if (value == null) { + output.appendNull(); + } + else if (value instanceof String || value instanceof Number) { + VARCHAR.writeSlice(output, Slices.utf8Slice(value.toString())); + } + else { + throw new TrinoException(TYPE_MISMATCH, format("Expected a string or numeric value for field '%s' of type VARCHAR: %s [%s]", path, value, value.getClass().getSimpleName())); + } + } + + public static class Descriptor + implements DecoderDescriptor + { + private final String path; + + @JsonCreator + public Descriptor(String path) + { + this.path = path; + } + + @JsonProperty + public String getPath() + { + return path; + } + + @Override + public Decoder createDecoder() + { + return new VarcharDecoder(path); + } + } +} diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ptf/RawQuery.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ptf/RawQuery.java new file mode 100644 index 0000000000000..c891b704e60d7 --- /dev/null +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ptf/RawQuery.java @@ -0,0 +1,148 @@ +/* + * 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.elasticsearch.ptf; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.inject.Inject; +import com.google.inject.Provider; +import io.airlift.slice.Slice; +import io.trino.plugin.elasticsearch.ElasticsearchColumnHandle; +import io.trino.plugin.elasticsearch.ElasticsearchMetadata; +import io.trino.plugin.elasticsearch.ElasticsearchTableHandle; +import io.trino.spi.connector.ColumnHandle; +import io.trino.spi.connector.ColumnSchema; +import io.trino.spi.connector.ConnectorAccessControl; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.ConnectorTableHandle; +import io.trino.spi.connector.ConnectorTableSchema; +import io.trino.spi.connector.ConnectorTransactionHandle; +import io.trino.spi.function.table.AbstractConnectorTableFunction; +import io.trino.spi.function.table.Argument; +import io.trino.spi.function.table.ConnectorTableFunction; +import io.trino.spi.function.table.ConnectorTableFunctionHandle; +import io.trino.spi.function.table.Descriptor; +import io.trino.spi.function.table.ScalarArgument; +import io.trino.spi.function.table.ScalarArgumentSpecification; +import io.trino.spi.function.table.TableFunctionAnalysis; + +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.trino.plugin.elasticsearch.ElasticsearchTableHandle.Type.QUERY; +import static io.trino.spi.function.table.ReturnTypeSpecification.GenericTable.GENERIC_TABLE; +import static io.trino.spi.type.VarcharType.VARCHAR; +import static java.util.Objects.requireNonNull; +import static java.util.stream.Collectors.toList; + +public class RawQuery + implements Provider +{ + public static final String SCHEMA_NAME = "system"; + public static final String NAME = "raw_query"; + + private final ElasticsearchMetadata metadata; + + @Inject + public RawQuery(ElasticsearchMetadata metadata) + { + this.metadata = requireNonNull(metadata, "metadata is null"); + } + + @Override + public ConnectorTableFunction get() + { + return new RawQueryFunction(metadata); + } + + public static class RawQueryFunction + extends AbstractConnectorTableFunction + { + private final ElasticsearchMetadata metadata; + + public RawQueryFunction(ElasticsearchMetadata metadata) + { + super( + SCHEMA_NAME, + NAME, + List.of( + ScalarArgumentSpecification.builder() + .name("SCHEMA") + .type(VARCHAR) + .build(), + ScalarArgumentSpecification.builder() + .name("INDEX") + .type(VARCHAR) + .build(), + ScalarArgumentSpecification.builder() + .name("QUERY") + .type(VARCHAR) + .build()), + GENERIC_TABLE); + this.metadata = requireNonNull(metadata, "metadata is null"); + } + + @Override + public TableFunctionAnalysis analyze( + ConnectorSession session, + ConnectorTransactionHandle transaction, + Map arguments, + ConnectorAccessControl accessControl) + { + String schema = ((Slice) ((ScalarArgument) arguments.get("SCHEMA")).getValue()).toStringUtf8(); + String index = ((Slice) ((ScalarArgument) arguments.get("INDEX")).getValue()).toStringUtf8(); + String query = ((Slice) ((ScalarArgument) arguments.get("QUERY")).getValue()).toStringUtf8(); + + ElasticsearchTableHandle tableHandle = new ElasticsearchTableHandle(QUERY, schema, index, Optional.of(query)); + ConnectorTableSchema tableSchema = metadata.getTableSchema(session, tableHandle); + Map columnsByName = metadata.getColumnHandles(session, tableHandle); + List columns = tableSchema.getColumns().stream() + .map(ColumnSchema::getName) + .map(columnsByName::get) + .collect(toImmutableList()); + + Descriptor returnedType = new Descriptor(columns.stream() + .map(ElasticsearchColumnHandle.class::cast) + .map(column -> new Descriptor.Field(column.getName(), Optional.of(column.getType()))) + .collect(toList())); + + RawQueryFunctionHandle handle = new RawQueryFunctionHandle(tableHandle); + + return TableFunctionAnalysis.builder() + .returnedType(returnedType) + .handle(handle) + .build(); + } + } + + public static class RawQueryFunctionHandle + implements ConnectorTableFunctionHandle + { + private final ElasticsearchTableHandle tableHandle; + + @JsonCreator + public RawQueryFunctionHandle(@JsonProperty("tableHandle") ElasticsearchTableHandle tableHandle) + { + this.tableHandle = requireNonNull(tableHandle, "tableHandle is null"); + } + + @JsonProperty + public ConnectorTableHandle getTableHandle() + { + return tableHandle; + } + } +} diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/BaseElasticsearchConnectorTest.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/BaseElasticsearchConnectorTest.java new file mode 100644 index 0000000000000..60586c95bbf59 --- /dev/null +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/BaseElasticsearchConnectorTest.java @@ -0,0 +1,1962 @@ +/* + * 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.elasticsearch; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.net.HostAndPort; +import io.trino.Session; +import io.trino.spi.type.VarcharType; +import io.trino.sql.planner.plan.LimitNode; +import io.trino.testing.AbstractTestQueries; +import io.trino.testing.BaseConnectorTest; +import io.trino.testing.MaterializedResult; +import io.trino.testing.QueryRunner; +import io.trino.testing.TestingConnectorBehavior; +import io.trino.tpch.TpchTable; +import org.apache.http.HttpHost; +import org.apache.http.entity.ContentType; +import org.apache.http.nio.entity.NStringEntity; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestHighLevelClient; +import org.intellij.lang.annotations.Language; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; + +import java.io.IOException; +import java.time.LocalDateTime; +import java.util.List; +import java.util.Map; + +import static io.trino.plugin.elasticsearch.ElasticsearchQueryRunner.createElasticsearchQueryRunner; +import static io.trino.spi.type.DoubleType.DOUBLE; +import static io.trino.spi.type.VarcharType.VARCHAR; +import static io.trino.testing.MaterializedResult.resultBuilder; +import static io.trino.testing.TestingNames.randomNameSuffix; +import static java.lang.String.format; +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; + +@TestInstance(PER_CLASS) +public abstract class BaseElasticsearchConnectorTest + extends BaseConnectorTest +{ + private final String image; + private final String catalogName; + private ElasticsearchServer elasticsearch; + protected RestHighLevelClient client; + + BaseElasticsearchConnectorTest(String image, String catalogName) + { + this.image = image; + this.catalogName = catalogName; + } + + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + elasticsearch = new ElasticsearchServer(image, ImmutableMap.of()); + + HostAndPort address = elasticsearch.getAddress(); + client = new RestHighLevelClient(RestClient.builder(new HttpHost(address.getHost(), address.getPort()))); + + return createElasticsearchQueryRunner( + elasticsearch.getAddress(), + TpchTable.getTables(), + ImmutableMap.of(), + ImmutableMap.of(), + 3, + catalogName); + } + + @AfterAll + public final void destroy() + throws IOException + { + elasticsearch.stop(); + elasticsearch = null; + client.close(); + client = null; + } + + @Override + protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior) + { + return switch (connectorBehavior) { + case SUPPORTS_ADD_COLUMN, + SUPPORTS_COMMENT_ON_COLUMN, + SUPPORTS_COMMENT_ON_TABLE, + SUPPORTS_CREATE_MATERIALIZED_VIEW, + SUPPORTS_CREATE_SCHEMA, + SUPPORTS_CREATE_TABLE, + SUPPORTS_CREATE_VIEW, + SUPPORTS_DELETE, + SUPPORTS_INSERT, + SUPPORTS_LIMIT_PUSHDOWN, + SUPPORTS_MERGE, + SUPPORTS_RENAME_COLUMN, + SUPPORTS_RENAME_TABLE, + SUPPORTS_ROW_TYPE, + SUPPORTS_SET_COLUMN_TYPE, + SUPPORTS_TOPN_PUSHDOWN, + SUPPORTS_UPDATE -> false; + default -> super.hasBehavior(connectorBehavior); + }; + } + + /** + * This method overrides the default values used for the data provider + * of the test {@link AbstractTestQueries#testLargeIn(int)} by taking + * into account that by default Elasticsearch supports only up to `1024` + * clauses in query. + *

+ * Consult `index.query.bool.max_clause_count` elasticsearch.yml setting + * for more details. + * + * @return the amount of clauses to be used in large queries + */ + @Override + protected List largeInValuesCountData() + { + return ImmutableList.of(200, 500, 1000); + } + + @Test + public void testWithoutBackpressure() + { + assertQuerySucceeds("SELECT * FROM orders"); + // Check that JMX stats show no sign of backpressure + assertQueryReturnsEmptyResult(format("SELECT 1 FROM jmx.current.\"trino.plugin.elasticsearch.client:*name=%s*\" WHERE \"backpressurestats.alltime.count\" > 0", catalogName)); + assertQueryReturnsEmptyResult(format("SELECT 1 FROM jmx.current.\"trino.plugin.elasticsearch.client:*name=%s*\" WHERE \"backpressurestats.alltime.max\" > 0", catalogName)); + } + + @Test + @Override + public void testSelectAll() + { + // List columns explicitly, as there's no defined order in Elasticsearch + assertQuery("SELECT orderkey, custkey, orderstatus, totalprice, orderdate, orderpriority, clerk, shippriority, comment FROM orders"); + } + + @Override + protected MaterializedResult getDescribeOrdersResult() + { + // The column metadata for the Elasticsearch connector tables are provided + // based on the column name in alphabetical order. + return resultBuilder(getSession(), VARCHAR, VARCHAR, VARCHAR, VARCHAR) + .row("clerk", "varchar", "", "") + .row("comment", "varchar", "", "") + .row("custkey", "bigint", "", "") + .row("orderdate", "timestamp(3)", "", "") + .row("orderkey", "bigint", "", "") + .row("orderpriority", "varchar", "", "") + .row("orderstatus", "varchar", "", "") + .row("shippriority", "bigint", "", "") + .row("totalprice", "real", "", "") + .build(); + } + + @Test + @Override + public void testPredicateReflectedInExplain() + { + // The format of the string representation of what gets shown in the table scan is connector-specific + // and there's no requirement that the conform to a specific shape or contain certain keywords. + + assertExplain( + "EXPLAIN SELECT name FROM nation WHERE nationkey = 42", + "nationkey::bigint", "::\\s\\[\\[42\\]\\]"); + } + + @Test + @Override + public void testSortItemsReflectedInExplain() + { + // The format of the string representation of what gets shown in the table scan is connector-specific + // and there's no requirement that the conform to a specific shape or contain certain keywords. + assertExplain( + "EXPLAIN SELECT name FROM nation ORDER BY nationkey DESC NULLS LAST LIMIT 5", + "TopNPartial\\[count = 5, orderBy = \\[nationkey DESC"); + } + + @Test + @Override + public void testShowCreateTable() + { + assertThat(computeActual("SHOW CREATE TABLE orders").getOnlyValue()) + .isEqualTo(format("CREATE TABLE %s.tpch.orders (\n", catalogName) + + " clerk varchar,\n" + + " comment varchar,\n" + + " custkey bigint,\n" + + " orderdate timestamp(3),\n" + + " orderkey bigint,\n" + + " orderpriority varchar,\n" + + " orderstatus varchar,\n" + + " shippriority bigint,\n" + + " totalprice real\n" + + ")"); + } + + @Test + @Override + public void testShowColumns() + { + assertThat(query("SHOW COLUMNS FROM orders")).matches(getDescribeOrdersResult()); + } + + @Test + public void testNullPredicate() + throws IOException + { + String indexName = "null_predicate1"; + @Language("JSON") + String properties = "" + + "{" + + " \"properties\":{" + + " \"null_keyword\": { \"type\": \"keyword\" }," + + " \"custkey\": { \"type\": \"keyword\" }" + + " }" + + "}"; + createIndex(indexName, properties); + index(indexName, ImmutableMap.builder() + .put("null_keyword", 32) + .put("custkey", 1301) + .buildOrThrow()); + + assertQueryReturnsEmptyResult("SELECT * FROM null_predicate1 WHERE null_keyword IS NULL"); + assertQueryReturnsEmptyResult("SELECT * FROM null_predicate1 WHERE null_keyword = '10' OR null_keyword IS NULL"); + + assertQuery("SELECT custkey, null_keyword FROM null_predicate1 WHERE null_keyword = '32' OR null_keyword IS NULL", "VALUES (1301, 32)"); + assertQuery("SELECT custkey FROM null_predicate1 WHERE null_keyword = '32' OR null_keyword IS NULL", "VALUES (1301)"); + + // not null filter + // filtered column is selected + assertQuery("SELECT custkey, null_keyword FROM null_predicate1 WHERE null_keyword IS NOT NULL", "VALUES (1301, 32)"); + assertQuery("SELECT custkey, null_keyword FROM null_predicate1 WHERE null_keyword = '32' OR null_keyword IS NOT NULL", "VALUES (1301, 32)"); + + // filtered column is not selected + assertQuery("SELECT custkey FROM null_predicate1 WHERE null_keyword = '32' OR null_keyword IS NOT NULL", "VALUES (1301)"); + + indexName = "null_predicate2"; + properties = "" + + "{" + + " \"properties\":{" + + " \"null_keyword\": { \"type\": \"keyword\" }," + + " \"custkey\": { \"type\": \"keyword\" }" + + " }" + + "}"; + createIndex(indexName, properties); + index(indexName, ImmutableMap.of("custkey", 1301)); + + // not null filter + assertQueryReturnsEmptyResult("SELECT * FROM null_predicate2 WHERE null_keyword IS NOT NULL"); + assertQueryReturnsEmptyResult("SELECT * FROM null_predicate2 WHERE null_keyword = '10' OR null_keyword IS NOT NULL"); + + // filtered column is selected + assertQuery("SELECT custkey, null_keyword FROM null_predicate2 WHERE null_keyword IS NULL", "VALUES (1301, NULL)"); + assertQuery("SELECT custkey, null_keyword FROM null_predicate2 WHERE null_keyword = '32' OR null_keyword IS NULL", "VALUES (1301, NULL)"); + + // filtered column is not selected + assertQuery("SELECT custkey FROM null_predicate2 WHERE null_keyword = '32' OR null_keyword IS NULL", "VALUES (1301)"); + + index(indexName, ImmutableMap.builder() + .put("null_keyword", 32) + .put("custkey", 1302) + .buildOrThrow()); + + assertQuery("SELECT custkey, null_keyword FROM null_predicate2 WHERE null_keyword = '32' OR null_keyword IS NULL", "VALUES (1301, NULL), (1302, 32)"); + assertQuery("SELECT custkey FROM null_predicate2 WHERE null_keyword = '32' OR null_keyword IS NULL", "VALUES (1301), (1302)"); + } + + @Test + public void testNestedFields() + throws IOException + { + String indexName = "data"; + index(indexName, ImmutableMap.builder() + .put("name", "nestfield") + .put("fields.fielda", 32) + .put("fields.fieldb", "valueb") + .buildOrThrow()); + + assertQuery( + "SELECT name, fields.fielda, fields.fieldb FROM data", + "VALUES ('nestfield', 32, 'valueb')"); + } + + @Test + public void testNameConflict() + throws IOException + { + String indexName = "name_conflict"; + index(indexName, ImmutableMap.builder() + .put("field", "value") + .put("Conflict", "conflict1") + .put("conflict", "conflict2") + .buildOrThrow()); + + assertQuery( + "SELECT * FROM name_conflict", + "VALUES ('value')"); + } + + @Test + public void testArrayFields() + throws IOException + { + String indexName = "test_arrays"; + + @Language("JSON") + String mapping = "" + + "{" + + " \"_meta\": {" + + " \"trino\": {" + + " \"a\": {" + + " \"b\": {" + + " \"y\": {" + + " \"isArray\": true" + + " }" + + " }" + + " }," + + " \"c\": {" + + " \"f\": {" + + " \"g\": {" + + " \"isArray\": true" + + " }," + + " \"isArray\": true" + + " }" + + " }," + + " \"j\": {" + + " \"isArray\": true" + + " }," + + " \"k\": {" + + " \"isArray\": true" + + " }" + + " }" + + " }," + + " \"properties\":{" + + " \"a\": {" + + " \"type\": \"object\"," + + " \"properties\": {" + + " \"b\": {" + + " \"type\": \"object\"," + + " \"properties\": {" + + " \"x\": {" + + " \"type\": \"integer\"" + + " }," + + " \"y\": {" + + " \"type\": \"keyword\"" + + " }" + + " } " + + " }" + + " }" + + " }," + + " \"c\": {" + + " \"type\": \"object\"," + + " \"properties\": {" + + " \"d\": {" + + " \"type\": \"keyword\"" + + " }," + + " \"e\": {" + + " \"type\": \"keyword\"" + + " }," + + " \"f\": {" + + " \"type\": \"object\"," + + " \"properties\": {" + + " \"g\": {" + + " \"type\": \"integer\"" + + " }," + + " \"h\": {" + + " \"type\": \"integer\"" + + " }" + + " } " + + " }" + + " }" + + " }," + + " \"i\": {" + + " \"type\": \"long\"" + + " }," + + " \"j\": {" + + " \"type\": \"long\"" + + " }," + + " \"k\": {" + + " \"type\": \"long\"" + + " }" + + " }" + + "}"; + + createIndex(indexName, mapping); + + index(indexName, ImmutableMap.builder() + .put("a", ImmutableMap.builder() + .put("b", ImmutableMap.builder() + .put("x", 1) + .put("y", ImmutableList.builder() + .add("hello") + .add("world") + .build()) + .buildOrThrow()) + .buildOrThrow()) + .put("c", ImmutableMap.builder() + .put("d", "foo") + .put("e", "bar") + .put("f", ImmutableList.>builder() + .add(ImmutableMap.builder() + .put("g", ImmutableList.builder() + .add(10) + .add(20) + .build()) + .put("h", 100) + .buildOrThrow()) + .add(ImmutableMap.builder() + .put("g", ImmutableList.builder() + .add(30) + .add(40) + .build()) + .put("h", 200) + .buildOrThrow()) + .build()) + .buildOrThrow()) + .put("j", ImmutableList.builder() + .add(50L) + .add(60L) + .build()) + .buildOrThrow()); + + assertQuery( + "SELECT a.b.y[1], c.f[1].g[2], c.f[2].g[1], j[2], k[1] FROM test_arrays", + "VALUES ('hello', 20, 30, 60, NULL)"); + } + + @Test + public void testAsRawJson() + throws IOException + { + String indexName = "raw_json_" + randomNameSuffix(); + + @Language("JSON") + String mapping = "" + + "{" + + " \"_meta\": {" + + " \"trino\": {" + + " \"es_object\": {" + + " \"array_of_string_arrays\": {" + + " \"asRawJson\": true" + + " }," + + " \"arrayOfIntArrays\": {" + + " \"asRawJson\": true" + + " }" + + " }," + + " \"es_array_object\": {" + + " \"isArray\": true," + + " \"array_of_string_arrays\": {" + + " \"asRawJson\": true" + + " }," + + " \"arrayOfIntArrays\": {" + + " \"asRawJson\": true" + + " }" + + " }," + + " \"es_raw_object\": {" + + " \"asRawJson\": true," + + " \"array_of_string_arrays\": {" + + " \"isArray\": true" + + " }," + + " \"arrayOfIntArrays\": {" + + " \"isArray\": true" + + " }" + + " }," + + " \"array_of_string_arrays\": {" + + " \"asRawJson\": true" + + " }," + + " \"array_of_long_arrays\": {" + + " \"asRawJson\": true" + + " }" + + " }" + + " }," + + " \"properties\": {" + + " \"es_object\": {" + + " \"type\": \"object\"," + + " \"properties\": {" + + " \"array_of_string_arrays\": {" + + " \"type\": \"keyword\"" + + " }," + + " \"arrayOfIntArrays\": {" + + " \"type\": \"integer\"" + + " }" + + " }" + + " }," + + " \"es_array_object\": {" + + " \"type\": \"object\"," + + " \"properties\": {" + + " \"array_of_string_arrays\": {" + + " \"type\": \"keyword\"" + + " }," + + " \"arrayOfIntArrays\": {" + + " \"type\": \"integer\"" + + " }" + + " }" + + " }," + + " \"es_raw_object\": {" + + " \"type\": \"object\"," + + " \"properties\": {" + + " \"array_of_string_arrays\": {" + + " \"type\": \"keyword\"" + + " }," + + " \"arrayOfIntArrays\": {" + + " \"type\": \"integer\"" + + " }" + + " }" + + " }," + + " \"array_of_string_arrays\": {" + + " \"type\": \"text\"" + + " }," + + " \"array_of_long_arrays\": {" + + " \"type\": \"long\"" + + " }," + + " \"order_field\": {" + + " \"type\": \"integer\"" + + " }" + + " }" + + "}"; + + createIndex(indexName, mapping); + + index(indexName, ImmutableMap.builder() + .put("es_object", ImmutableMap.builder() + .put("array_of_string_arrays", ImmutableList.>builder() + .add(ImmutableList.builder() + .add("abc") + .add("def") + .build()) + .build()) + .put("arrayOfIntArrays", ImmutableList.builder() + .add(123) + .add(ImmutableList.builder() + .add(234) + .add(345) + .build()) + .build()) + .buildOrThrow()) + .put("es_array_object", ImmutableMap.builder() + .put("array_of_string_arrays", ImmutableList.>builder() + .add(ImmutableList.builder() + .add("abc") + .add("def") + .build()) + .build()) + .put("arrayOfIntArrays", ImmutableList.builder() + .add(123) + .add(ImmutableList.builder() + .add(234) + .add(345) + .build()) + .build()) + .buildOrThrow()) + .put("es_raw_object", ImmutableMap.builder() + .put("array_of_string_arrays", ImmutableList.>builder() + .add(ImmutableList.builder() + .add("abc") + .add("def") + .build()) + .build()) + .put("arrayOfIntArrays", ImmutableList.builder() + .add(123) + .add(ImmutableList.builder() + .add(234) + .add(345) + .build()) + .build()) + .buildOrThrow()) + .put("array_of_string_arrays", ImmutableList.>builder() + .add(ImmutableList.builder() + .add("abc") + .add("def") + .build()) + .build()) + .put("array_of_long_arrays", ImmutableList.builder() + .add(123L) + .add(ImmutableList.builder() + .add(234L) + .add(345L) + .build()) + .build()) + .put("order_field", 1) + .buildOrThrow()); + + index(indexName, ImmutableMap.builder() + .put("es_object", ImmutableMap.builder() + .put("array_of_string_arrays", "Join the Trino Slack: https://trino.io/slack.html") + .put("arrayOfIntArrays", 867) + .buildOrThrow()) + .put("es_array_object", ImmutableMap.builder() + .put("array_of_string_arrays", "If you like Presto, you'll love Trino: https://trino.io/slack.html") + .put("arrayOfIntArrays", 321) + .buildOrThrow()) + .put("es_raw_object", ImmutableMap.builder() + .put("array_of_string_arrays", "The founders and core contributors of Presto, and are now working on Trino: https://trino.io/blog/2020/12/27/announcing-trino.html") + .put("arrayOfIntArrays", 654) + .buildOrThrow()) + .put("array_of_string_arrays", "Check out the bi-weekly Trino Community Broadcast https://trino.io/broadcast/") + .put("array_of_long_arrays", 5309L) + .put("order_field", 2) + .buildOrThrow()); + + MaterializedResult rows = computeActual("" + + "SELECT " + + "json_extract(array_of_string_arrays, '$[0][0]'), " + + "json_extract(array_of_string_arrays, '$[0][1]'), " + + "array_of_string_arrays, " + + "json_extract(array_of_long_arrays, '$[0]'), " + + "try(json_extract(array_of_long_arrays, '$[1][0]')), " + + "try(json_extract(array_of_long_arrays, '$[1][1]')), " + + "array_of_long_arrays " + + "FROM " + indexName + " " + + "ORDER BY order_field"); + + MaterializedResult expected = resultBuilder(getSession(), rows.getTypes()) + .row("\"abc\"", "\"def\"", "[[\"abc\",\"def\"]]", "123", "234", "345", "[123,[234,345]]") + .row(null, null, "\"Check out the bi-weekly Trino Community Broadcast https://trino.io/broadcast/\"", null, null, null, "5309") + .build(); + + assertThat(rows.getMaterializedRows()).isEqualTo(expected.getMaterializedRows()); + + MaterializedResult nestedRows = computeActual("" + + "SELECT " + + "json_extract(es_object.array_of_string_arrays, '$[0][0]'), " + + "json_extract(es_object.array_of_string_arrays, '$[0][1]'), " + + "es_object.array_of_string_arrays, " + + "json_extract(es_object.arrayOfIntArrays, '$[0]'), " + + "try(json_extract(es_object.arrayOfIntArrays, '$[1][0]')), " + + "try(json_extract(es_object.arrayOfIntArrays, '$[1][1]')), " + + "es_object.arrayOfIntArrays " + + "FROM " + indexName + " " + + "ORDER BY order_field"); + + MaterializedResult nestedExpected = resultBuilder(getSession(), nestedRows.getTypes()) + .row("\"abc\"", "\"def\"", "[[\"abc\",\"def\"]]", "123", "234", "345", "[123,[234,345]]") + .row(null, null, "\"Join the Trino Slack: https://trino.io/slack.html\"", null, null, null, "867") + .build(); + + assertThat(nestedRows.getMaterializedRows()).isEqualTo(nestedExpected.getMaterializedRows()); + + MaterializedResult arrayRows = computeActual("" + + "SELECT " + + "json_extract(es_array_object[1].array_of_string_arrays, '$[0][0]'), " + + "json_extract(es_array_object[1].array_of_string_arrays, '$[0][1]'), " + + "es_array_object[1].array_of_string_arrays, " + + "json_extract(es_array_object[1].arrayOfIntArrays, '$[0]'), " + + "try(json_extract(es_array_object[1].arrayOfIntArrays, '$[1][0]')), " + + "try(json_extract(es_array_object[1].arrayOfIntArrays, '$[1][1]')), " + + "es_array_object[1].arrayOfIntArrays " + + "FROM " + indexName + " " + + "ORDER BY order_field"); + + MaterializedResult arrayExpected = resultBuilder(getSession(), arrayRows.getTypes()) + .row("\"abc\"", "\"def\"", "[[\"abc\",\"def\"]]", "123", "234", "345", "[123,[234,345]]") + .row(null, null, "\"If you like Presto, you'll love Trino: https://trino.io/slack.html\"", null, null, null, "321") + .build(); + + assertThat(arrayRows.getMaterializedRows()).isEqualTo(arrayExpected.getMaterializedRows()); + + MaterializedResult rawRows = computeActual("" + + "SELECT " + + "json_extract(es_raw_object, '$.array_of_string_arrays[0][0]'), " + + "json_extract(es_raw_object, '$.array_of_string_arrays[0][1]'), " + + "json_extract(es_raw_object, '$.array_of_string_arrays'), " + + "json_extract(es_raw_object, '$.arrayOfIntArrays[0]'), " + + "try(json_extract(es_raw_object, '$.arrayOfIntArrays[1][0]')), " + + "try(json_extract(es_raw_object, '$.arrayOfIntArrays[1][1]')), " + + "json_extract(es_raw_object, '$.arrayOfIntArrays') " + + "FROM " + indexName + " " + + "ORDER BY order_field"); + + MaterializedResult rawRowsExpected = resultBuilder(getSession(), rawRows.getTypes()) + .row("\"abc\"", "\"def\"", "[[\"abc\",\"def\"]]", "123", "234", "345", "[123,[234,345]]") + .row(null, null, "\"The founders and core contributors of Presto, and are now working on Trino: https://trino.io/blog/2020/12/27/announcing-trino.html\"", null, null, null, "654") + .build(); + + assertThat(rawRows.getMaterializedRows()).isEqualTo(rawRowsExpected.getMaterializedRows()); + } + + @Test + public void testAsRawJsonForAllPrimitiveTypes() + throws IOException + { + String indexName = "raw_json_primitive_" + randomNameSuffix(); + + @Language("JSON") + String mapping = "" + + "{" + + " \"_meta\": {" + + " \"trino\": {" + + " \"es_binary\": {" + + " \"asRawJson\": true" + + " }," + + " \"es_boolean\": {" + + " \"asRawJson\": true" + + " }," + + " \"es_long\": {" + + " \"asRawJson\": true" + + " }," + + " \"es_integer\": {" + + " \"asRawJson\": true" + + " }," + + " \"es_short\": {" + + " \"asRawJson\": true" + + " }," + + " \"es_byte\": {" + + " \"asRawJson\": true" + + " }," + + " \"es_double\": {" + + " \"asRawJson\": true" + + " }," + + " \"es_float\": {" + + " \"asRawJson\": true" + + " }" + + " }" + + " }," + + " \"properties\": {" + + " \"es_binary\": {" + + " \"type\": \"binary\"" + + " }," + + " \"es_boolean\": {" + + " \"type\": \"boolean\"" + + " }," + + " \"es_long\": {" + + " \"type\": \"long\"" + + " }," + + " \"es_integer\": {" + + " \"type\": \"integer\"" + + " }," + + " \"es_short\": {" + + " \"type\": \"short\"" + + " }," + + " \"es_byte\": {" + + " \"type\": \"byte\"" + + " }," + + " \"es_double\": {" + + " \"type\": \"double\"" + + " }," + + " \"es_float\": {" + + " \"type\": \"float\"" + + " }," + + " \"order_field\": {" + + " \"type\": \"integer\"" + + " }" + + " }" + + "}"; + + createIndex(indexName, mapping); + + index(indexName, ImmutableMap.builder() + .put("es_binary", "test".getBytes(UTF_8)) + .put("es_boolean", true) + .put("es_long", (long) 123) + .put("es_integer", 123) + .put("es_short", (short) 123) + .put("es_byte", (byte) 123) + .put("es_double", (double) 123) + .put("es_float", (float) 123) + .put("order_field", 1) + .buildOrThrow()); + + MaterializedResult rows = computeActual("" + + "SELECT " + + "es_binary, " + + "es_boolean, " + + "es_long, " + + "es_integer, " + + "es_short, " + + "es_byte, " + + "es_double, " + + "es_float " + + "FROM " + indexName + " " + + "ORDER BY order_field"); + + MaterializedResult expected = resultBuilder(getSession(), rows.getTypes()) + .row("\"dGVzdA==\"", "true", "123", "123", "123", "123", "123.0", "123.0") + .build(); + assertThat(rows.getTypes()) + .hasOnlyElementsOfType(VarcharType.class); + + assertThat(rows.getMaterializedRows()).isEqualTo(expected.getMaterializedRows()); + + deleteIndex(indexName); + } + + @Test + public void testAsRawJsonCases() + throws IOException + { + String indexName = "raw_json_cases_" + randomNameSuffix(); + + @Language("JSON") + String mapping = "" + + "{" + + " \"_meta\": {" + + " \"trino\": {" + + " \"es_binary\": {" + + " \"asRawJson\": true" + + " }," + + " \"es_boolean\": {" + + " \"asRawJson\": true" + + " }," + + " \"es_timestamp\": {" + + " \"asRawJson\": true" + + " }" + + " }" + + " }," + + " \"properties\": {" + + " \"es_binary\": {" + + " \"type\": \"binary\"" + + " }," + + " \"es_boolean\": {" + + " \"type\": \"boolean\"" + + " }," + + " \"es_timestamp\": {" + + " \"type\": \"date\"" + + " }" + + " }" + + "}"; + + createIndex(indexName, mapping); + + index(indexName, ImmutableMap.builder() + .put("es_binary", "test".getBytes(UTF_8)) + .put("es_boolean", true) + .put("es_timestamp", 123) + .buildOrThrow()); + + MaterializedResult rows = computeActual("" + + "SELECT " + + "es_binary, " + + "es_boolean, " + + "es_timestamp " + + "FROM " + indexName); + + MaterializedResult expected = resultBuilder(getSession(), rows.getTypes()) + .row("\"dGVzdA==\"", "true", "123") + .build(); + + assertThat(rows.getMaterializedRows()).isEqualTo(expected.getMaterializedRows()); + assertThat(rows.getTypes()) + .hasOnlyElementsOfType(VarcharType.class); + + deleteIndex(indexName); + } + + @Test + public void testAsRawJsonAndIsArraySameFieldException() + throws IOException + { + String indexName = "raw_json_array_exception" + randomNameSuffix(); + + @Language("JSON") + String mapping = "" + + "{" + + " \"_meta\": {" + + " \"trino\": {" + + " \"array_raw_field\": {" + + " \"asRawJson\": true," + + " \"isArray\": true" + + " }" + + " }" + + " }," + + " \"properties\": {" + + " \"array_raw_field\": {" + + " \"type\": \"text\"" + + " }" + + " }" + + "}"; + + createIndex(indexName, mapping); + + index(indexName, ImmutableMap.of("array_raw_field", "test")); + + assertThatThrownBy(() -> computeActual("SELECT array_raw_field FROM " + indexName)) + .hasMessage("A column, (array_raw_field) cannot be declared as a Trino array and also be rendered as json."); + + deleteIndex(indexName); + } + + @Test + public void testMixedArray() + throws IOException + { + String indexName = "test_mixed_arrays"; + + @Language("JSON") + String mapping = "" + + "{" + + " \"_meta\": {" + + " \"trino\": {" + + " \"a\": {" + + " \"isArray\": true" + + " }" + + " }" + + " }," + + " \"properties\": {" + + " \"a\": {" + + " \"type\": \"keyword\"" + + " }" + + " }" + + "}"; + + createIndex(indexName, mapping); + + index(indexName, ImmutableMap.of()); + + index(indexName, ImmutableMap.of("a", "hello")); + + index(indexName, ImmutableMap.of("a", ImmutableList.of("foo", "bar"))); + + assertQuery( + "SELECT a FROM test_mixed_arrays", + "VALUES NULL, ARRAY['hello'], ARRAY['foo', 'bar']"); + } + + @Test + public void testEmptyNumericFields() + throws IOException + { + String indexName = "emptynumeric"; + + @Language("JSON") + String mapping = "" + + "{" + + " \"properties\": { " + + " \"byte_column\": {\"type\": \"byte\"}," + + " \"short_column\": {\"type\": \"short\"}," + + " \"integer_column\": {\"type\": \"integer\"}," + + " \"long_column\": {\"type\": \"long\"}," + + " \"float_column\": {\"type\": \"float\"}," + + " \"scaled_float_column\": {\"type\": \"scaled_float\", \"scaling_factor\": 100}," + + " \"double_column\": {\"type\": \"double\"}" + + " }" + + "}"; + + createIndex(indexName, mapping); + index(indexName, ImmutableMap.builder() + .put("byte_column", "") + .put("short_column", "") + .put("integer_column", "") + .put("long_column", "") + .put("float_column", "") + .put("scaled_float_column", "") + .put("double_column", "") + .buildOrThrow()); + + assertQuery( + "SELECT byte_column, short_column, integer_column, long_column, float_column, scaled_float_column, double_column FROM emptynumeric", + "VALUES (NULL, NULL, NULL, NULL, NULL, NULL, NULL)"); + + deleteIndex(indexName); + } + + @Test + public void testEmptyObjectFields() + throws IOException + { + String indexName = "emptyobject"; + index(indexName, ImmutableMap.builder() + .put("name", "stringfield") + .put("emptyobject", ImmutableMap.of()) + .put("fields.fielda", 32) + .put("fields.fieldb", ImmutableMap.of()) + .buildOrThrow()); + + assertQuery( + "SELECT name, fields.fielda FROM emptyobject", + "VALUES ('stringfield', 32)"); + } + + @Test + public void testNestedVariants() + throws IOException + { + String indexName = "nested_variants"; + + index(indexName, + ImmutableMap.of("a", + ImmutableMap.of("b", + ImmutableMap.of("c", + "value1")))); + + index(indexName, + ImmutableMap.of("a.b", + ImmutableMap.of("c", + "value2"))); + + index(indexName, + ImmutableMap.of("a", + ImmutableMap.of("b.c", + "value3"))); + + index(indexName, + ImmutableMap.of("a.b.c", "value4")); + + assertQuery( + "SELECT a.b.c FROM nested_variants", + "VALUES 'value1', 'value2', 'value3', 'value4'"); + } + + @Test + public void testLike() + throws IOException + { + String indexName = "like_test"; + + @Language("JSON") + String mappings = "" + + "{" + + " \"properties\": { " + + " \"keyword_column\": { \"type\": \"keyword\" }," + + " \"text_column\": { \"type\": \"text\" }" + + " }" + + "}"; + + createIndex(indexName, mappings); + + index(indexName, ImmutableMap.builder() + .put("keyword_column", "so.me tex\\t") + .put("text_column", "so.me tex\\t") + .buildOrThrow()); + + // Add another document to make sure '.' is escaped and not treated as any character + index(indexName, ImmutableMap.builder() + .put("keyword_column", "soome tex\\t") + .put("text_column", "soome tex\\t") + .buildOrThrow()); + + // Add another document to make sure '%' can be escaped and not treated as any character + index(indexName, ImmutableMap.builder() + .put("keyword_column", "soome%text") + .put("text_column", "soome%text") + .buildOrThrow()); + + // Add another document to make sure utf8 character sequence length is right + index(indexName, ImmutableMap.builder() + .put("keyword_column", "中文") + .put("text_column", "中文") + .buildOrThrow()); + + // Add another document to make sure utf8 character sequence length is right + index(indexName, ImmutableMap.builder() + .put("keyword_column", "こんにちは") + .put("text_column", "こんにちは") + .buildOrThrow()); + + // Add another document to make sure utf8 character sequence length is right + index(indexName, ImmutableMap.builder() + .put("keyword_column", "안녕하세요") + .put("text_column", "안녕하세요") + .buildOrThrow()); + + // Add another document to make sure utf8 character sequence length is right + index(indexName, ImmutableMap.builder() + .put("keyword_column", "Привет") + .put("text_column", "Привет") + .buildOrThrow()); + + assertThat(query("" + + "SELECT " + + "keyword_column " + + "FROM " + indexName + " " + + "WHERE keyword_column LIKE 's_.m%ex\\t'")) + .matches("VALUES VARCHAR 'so.me tex\\t'") + .isFullyPushedDown(); + + assertThat(query("" + + "SELECT " + + "text_column " + + "FROM " + indexName + " " + + "WHERE text_column LIKE 's_.m%ex\\t'")) + .matches("VALUES VARCHAR 'so.me tex\\t'"); + + assertThat(query("" + + "SELECT " + + "text_column " + + "FROM " + indexName + " " + + "WHERE keyword_column LIKE 'soome$%%' ESCAPE '$'")) + .matches("VALUES VARCHAR 'soome%text'") + .isFullyPushedDown(); + + assertThat(query("" + + "SELECT " + + "text_column " + + "FROM " + indexName + " " + + "WHERE keyword_column LIKE '中%'")) + .matches("VALUES VARCHAR '中文'") + .isFullyPushedDown(); + + assertThat(query("" + + "SELECT " + + "text_column " + + "FROM " + indexName + " " + + "WHERE keyword_column LIKE 'こんに%'")) + .matches("VALUES VARCHAR 'こんにちは'") + .isFullyPushedDown(); + + assertThat(query("" + + "SELECT " + + "text_column " + + "FROM " + indexName + " " + + "WHERE keyword_column LIKE '안녕하%'")) + .matches("VALUES VARCHAR '안녕하세요'") + .isFullyPushedDown(); + + assertThat(query("" + + "SELECT " + + "text_column " + + "FROM " + indexName + " " + + "WHERE keyword_column LIKE 'При%'")) + .matches("VALUES VARCHAR 'Привет'") + .isFullyPushedDown(); + } + + @Test + public void testDataTypes() + throws IOException + { + String indexName = "types"; + + @Language("JSON") + String mappings = "" + + "{" + + " \"properties\": { " + + " \"boolean_column\": { \"type\": \"boolean\" }," + + " \"float_column\": { \"type\": \"float\" }," + + " \"double_column\": { \"type\": \"double\" }," + + " \"integer_column\": { \"type\": \"integer\" }," + + " \"long_column\": { \"type\": \"long\" }," + + " \"keyword_column\": { \"type\": \"keyword\" }," + + " \"text_column\": { \"type\": \"text\" }," + + " \"binary_column\": { \"type\": \"binary\" }," + + " \"timestamp_column\": { \"type\": \"date\" }," + + " \"ipv4_column\": { \"type\": \"ip\" }," + + " \"ipv6_column\": { \"type\": \"ip\" }," + + " \"scaled_float_column\": { \"type\": \"scaled_float\", \"scaling_factor\": 100 }" + + " }" + + "}"; + + createIndex(indexName, mappings); + + index(indexName, ImmutableMap.builder() + .put("boolean_column", true) + .put("float_column", 1.0f) + .put("double_column", 1.0d) + .put("integer_column", 1) + .put("long_column", 1L) + .put("keyword_column", "cool") + .put("text_column", "some text") + .put("binary_column", new byte[] {(byte) 0xCA, (byte) 0xFE}) + .put("timestamp_column", 0) + .put("ipv4_column", "1.2.3.4") + .put("ipv6_column", "2001:db8:0:0:1:0:0:1") + .put("scaled_float_column", 123456.78d) + .buildOrThrow()); + + MaterializedResult rows = computeActual("" + + "SELECT " + + "boolean_column, " + + "float_column, " + + "double_column, " + + "integer_column, " + + "long_column, " + + "keyword_column, " + + "text_column, " + + "binary_column, " + + "timestamp_column, " + + "ipv4_column, " + + "ipv6_column, " + + "scaled_float_column " + + "FROM types"); + + MaterializedResult expected = resultBuilder(getSession(), rows.getTypes()) + .row( + true, + 1.0f, + 1.0d, + 1, + 1L, + "cool", + "some text", + new byte[] {(byte) 0xCA, (byte) 0xFE}, + LocalDateTime.of(1970, 1, 1, 0, 0), + "1.2.3.4", + "2001:db8::1:0:0:1", + 123456.78d) + .build(); + + assertThat(rows.getMaterializedRows()).isEqualTo(expected.getMaterializedRows()); + } + + @Test + public void testTableWithUnsupportedTypes() + throws IOException + { + String indexName = "unsupported_types"; + + @Language("JSON") + String mappings = "" + + "{" + + " \"properties\": { " + + " \"long_column\": { \"type\": \"long\" }," + + " \"unsupported_type\": { \"type\": \"completion\"}" + + " }" + + "}"; + + createIndex(indexName, mappings); + + index(indexName, ImmutableMap.builder() + .put("long_column", 1L) + .put("unsupported_type", ImmutableList.of("foo", "bar")) + .buildOrThrow()); + + MaterializedResult rows = computeActual("SELECT * FROM unsupported_types"); + MaterializedResult expected = resultBuilder(getSession(), rows.getTypes()) + .row(1L) + .build(); + + assertThat(rows.getMaterializedRows()).isEqualTo(expected.getMaterializedRows()); + } + + @Test + public void testBoolean() + throws IOException + { + String indexName = "booleans"; + + @Language("JSON") + String mappings = "" + + "{" + + " \"properties\": { " + + " \"boolean_column\": { \"type\": \"boolean\" }" + + " }" + + "}"; + + createIndex(indexName, mappings); + + index(indexName, ImmutableMap.of("boolean_column", true)); + + index(indexName, ImmutableMap.of("boolean_column", "true")); + + index(indexName, ImmutableMap.of("boolean_column", false)); + + index(indexName, ImmutableMap.of("boolean_column", "false")); + + index(indexName, ImmutableMap.of("boolean_column", "")); + + MaterializedResult rows = computeActual("SELECT boolean_column FROM booleans"); + + MaterializedResult expected = resultBuilder(getSession(), rows.getTypes()) + .row(true) + .row(true) + .row(false) + .row(false) + .row(false) + .build(); + + assertThat(rows.getMaterializedRows()).containsExactlyInAnyOrderElementsOf(expected.getMaterializedRows()); + } + + @Test + public void testTimestamps() + throws IOException + { + String indexName = "timestamps"; + + @Language("JSON") + String mappings = "" + + "{" + + " \"properties\": { " + + " \"timestamp_column\": { \"type\": \"date\" }" + + " }" + + "}"; + + createIndex(indexName, mappings); + + index(indexName, ImmutableMap.of("timestamp_column", "2015-01-01")); + + index(indexName, ImmutableMap.of("timestamp_column", "2015-01-01T12:10:30Z")); + + index(indexName, ImmutableMap.of("timestamp_column", 1420070400001L)); + + index(indexName, ImmutableMap.of("timestamp_column", "1420070400001")); + + MaterializedResult rows = computeActual("SELECT timestamp_column FROM timestamps"); + + MaterializedResult expected = resultBuilder(getSession(), rows.getTypes()) + .row(LocalDateTime.parse("2015-01-01T00:00:00")) + .row(LocalDateTime.parse("2015-01-01T12:10:30")) + .row(LocalDateTime.parse("2015-01-01T00:00:00.001")) + .row(LocalDateTime.parse("2015-01-01T00:00:00.001")) + .build(); + + assertThat(rows.getMaterializedRows()).containsExactlyInAnyOrderElementsOf(expected.getMaterializedRows()); + } + + @Test + public void testNestedTimestamps() + throws IOException + { + String indexName = "nested_timestamps"; + + @Language("JSON") + String mappings = "" + + "{" + + " \"properties\":{" + + " \"field\": {" + + " \"properties\": {" + + " \"timestamp_column\": { \"type\": \"date\" }" + + " }" + + " }" + + " }" + + "}"; + + createIndex(indexName, mappings); + + index(indexName, ImmutableMap.of("field", ImmutableMap.of("timestamp_column", 0))); + index(indexName, ImmutableMap.of("field", ImmutableMap.of("timestamp_column", "1"))); + index(indexName, ImmutableMap.of("field", ImmutableMap.of("timestamp_column", "1970-01-01T01:01:00+0000"))); + + assertThat(query("SELECT field.timestamp_column FROM " + indexName)) + .matches("VALUES " + + "(TIMESTAMP '1970-01-01 00:00:00.000')," + + "(TIMESTAMP '1970-01-01 00:00:00.001')," + + "(TIMESTAMP '1970-01-01 01:01:00.000')"); + } + + @Test + public void testScaledFloat() + throws Exception + { + String indexName = "scaled_float_type"; + + @Language("JSON") + String mappings = "" + + "{" + + " \"properties\": { " + + " \"text_column\": { \"type\": \"text\" }," + + " \"scaled_float_column\": { \"type\": \"scaled_float\", \"scaling_factor\": 100 }" + + " }" + + "}"; + + createIndex(indexName, mappings); + + index(indexName, ImmutableMap.builder() + .put("text_column", "foo") + .put("scaled_float_column", 123.4567d) + .buildOrThrow()); + + index(indexName, ImmutableMap.builder() + .put("text_column", "bar") + .put("scaled_float_column", 123.46d) + .buildOrThrow()); + + index(indexName, ImmutableMap.builder() + .put("text_column", "random value") + .put("scaled_float_column", 9.8d) + .buildOrThrow()); + + // Trino query filters in the engine, so the rounding (dependent on scaling factor) does not impact results + assertThat(query(""" + SELECT text_column, scaled_float_column + FROM scaled_float_type + WHERE scaled_float_column = 123.46 + """)) + .matches(resultBuilder(getSession(), ImmutableList.of(VARCHAR, DOUBLE)) + .row("bar", 123.46d) + .build()); + } + + @Test + public void testCoercions() + throws IOException + { + String indexName = "coercions"; + + @Language("JSON") + String mappings = "" + + "{" + + " \"properties\": { " + + " \"float_column\": { \"type\": \"float\" }," + + " \"double_column\": { \"type\": \"double\" }," + + " \"integer_column\": { \"type\": \"integer\" }," + + " \"long_column\": { \"type\": \"long\" }" + + " }" + + "}"; + + createIndex(indexName, mappings); + + index(indexName, ImmutableMap.builder() + .put("float_column", "1.0") + .put("double_column", "1.0") + .put("integer_column", "1") + .put("long_column", "1") + .buildOrThrow()); + + MaterializedResult rows = computeActual("" + + "SELECT " + + "float_column, " + + "double_column, " + + "integer_column, " + + "long_column " + + "FROM coercions"); + + MaterializedResult expected = resultBuilder(getSession(), rows.getTypes()) + .row(1.0f, 1.0d, 1, 1L) + .build(); + + assertThat(rows.getMaterializedRows()).isEqualTo(expected.getMaterializedRows()); + } + + @Test + public void testFilters() + throws IOException + { + String indexName = "filter_pushdown"; + + @Language("JSON") + String mappings = "" + + "{" + + " \"properties\": { " + + " \"boolean_column\": { \"type\": \"boolean\" }," + + " \"float_column\": { \"type\": \"float\" }," + + " \"double_column\": { \"type\": \"double\" }," + + " \"integer_column\": { \"type\": \"integer\" }," + + " \"long_column\": { \"type\": \"long\" }," + + " \"keyword_column\": { \"type\": \"keyword\" }," + + " \"text_column\": { \"type\": \"text\" }," + + " \"binary_column\": { \"type\": \"binary\" }," + + " \"timestamp_column\": { \"type\": \"date\" }," + + " \"ipv4_column\": { \"type\": \"ip\" }," + + " \"ipv6_column\": { \"type\": \"ip\" }" + + " }" + + "}"; + + createIndex(indexName, mappings); + + index(indexName, ImmutableMap.builder() + .put("boolean_column", true) + .put("byte_column", 1) + .put("short_column", 2) + .put("integer_column", 3) + .put("long_column", 4L) + .put("float_column", 1.0f) + .put("double_column", 1.0d) + .put("keyword_column", "cool") + .put("text_column", "some text") + .put("binary_column", new byte[] {(byte) 0xCA, (byte) 0xFE}) + .put("timestamp_column", 1569888000000L) + .put("ipv4_column", "1.2.3.4") + .put("ipv6_column", "2001:db8:0:0:1:0:0:1") + .buildOrThrow()); + + // _score column + assertQuery("SELECT count(*) FROM \"filter_pushdown: cool\" WHERE _score > 0", "VALUES 1"); + + // boolean + assertQuery("SELECT count(*) FROM filter_pushdown WHERE boolean_column = true", "VALUES 1"); + assertQuery("SELECT count(*) FROM filter_pushdown WHERE boolean_column = false", "VALUES 0"); + + // tinyint + assertQuery("SELECT count(*) FROM filter_pushdown WHERE byte_column = 1", "VALUES 1"); + assertQuery("SELECT count(*) FROM filter_pushdown WHERE byte_column = 0", "VALUES 0"); + assertQuery("SELECT count(*) FROM filter_pushdown WHERE byte_column > 1", "VALUES 0"); + assertQuery("SELECT count(*) FROM filter_pushdown WHERE byte_column < 1", "VALUES 0"); + assertQuery("SELECT count(*) FROM filter_pushdown WHERE byte_column > 0", "VALUES 1"); + assertQuery("SELECT count(*) FROM filter_pushdown WHERE byte_column < 10", "VALUES 1"); + + // smallint + assertQuery("SELECT count(*) FROM filter_pushdown WHERE short_column = 2", "VALUES 1"); + assertQuery("SELECT count(*) FROM filter_pushdown WHERE short_column > 2", "VALUES 0"); + assertQuery("SELECT count(*) FROM filter_pushdown WHERE short_column < 2", "VALUES 0"); + assertQuery("SELECT count(*) FROM filter_pushdown WHERE short_column = 0", "VALUES 0"); + assertQuery("SELECT count(*) FROM filter_pushdown WHERE short_column > 0", "VALUES 1"); + assertQuery("SELECT count(*) FROM filter_pushdown WHERE short_column < 10", "VALUES 1"); + + // integer + assertQuery("SELECT count(*) FROM filter_pushdown WHERE integer_column = 3", "VALUES 1"); + assertQuery("SELECT count(*) FROM filter_pushdown WHERE integer_column > 3", "VALUES 0"); + assertQuery("SELECT count(*) FROM filter_pushdown WHERE integer_column < 3", "VALUES 0"); + assertQuery("SELECT count(*) FROM filter_pushdown WHERE integer_column = 0", "VALUES 0"); + assertQuery("SELECT count(*) FROM filter_pushdown WHERE integer_column > 0", "VALUES 1"); + assertQuery("SELECT count(*) FROM filter_pushdown WHERE integer_column < 10", "VALUES 1"); + + // bigint + assertQuery("SELECT count(*) FROM filter_pushdown WHERE long_column = 4", "VALUES 1"); + assertQuery("SELECT count(*) FROM filter_pushdown WHERE long_column > 4", "VALUES 0"); + assertQuery("SELECT count(*) FROM filter_pushdown WHERE long_column < 4", "VALUES 0"); + assertQuery("SELECT count(*) FROM filter_pushdown WHERE long_column = 0", "VALUES 0"); + assertQuery("SELECT count(*) FROM filter_pushdown WHERE long_column > 0", "VALUES 1"); + assertQuery("SELECT count(*) FROM filter_pushdown WHERE long_column < 10", "VALUES 1"); + + // real + assertQuery("SELECT count(*) FROM filter_pushdown WHERE float_column = 1.0", "VALUES 1"); + assertQuery("SELECT count(*) FROM filter_pushdown WHERE float_column > 1.0", "VALUES 0"); + assertQuery("SELECT count(*) FROM filter_pushdown WHERE float_column < 1.0", "VALUES 0"); + assertQuery("SELECT count(*) FROM filter_pushdown WHERE float_column = 0.0", "VALUES 0"); + assertQuery("SELECT count(*) FROM filter_pushdown WHERE float_column > 0.0", "VALUES 1"); + assertQuery("SELECT count(*) FROM filter_pushdown WHERE float_column < 10.0", "VALUES 1"); + + // double + assertQuery("SELECT count(*) FROM filter_pushdown WHERE double_column = 1.0", "VALUES 1"); + assertQuery("SELECT count(*) FROM filter_pushdown WHERE double_column > 1.0", "VALUES 0"); + assertQuery("SELECT count(*) FROM filter_pushdown WHERE double_column < 1.0", "VALUES 0"); + assertQuery("SELECT count(*) FROM filter_pushdown WHERE double_column = 0.0", "VALUES 0"); + assertQuery("SELECT count(*) FROM filter_pushdown WHERE double_column > 0.0", "VALUES 1"); + assertQuery("SELECT count(*) FROM filter_pushdown WHERE double_column < 10.0", "VALUES 1"); + + // varchar + assertQuery("SELECT count(*) FROM filter_pushdown WHERE keyword_column = 'cool'", "VALUES 1"); + assertQuery("SELECT count(*) FROM filter_pushdown WHERE keyword_column = 'bar'", "VALUES 0"); + assertQuery("SELECT count(*) FROM filter_pushdown WHERE text_column = 'some text'", "VALUES 1"); + assertQuery("SELECT count(*) FROM filter_pushdown WHERE text_column = 'some'", "VALUES 0"); + + // binary + assertQuery("SELECT count(*) FROM filter_pushdown WHERE binary_column = x'CAFE'", "VALUES 1"); + assertQuery("SELECT count(*) FROM filter_pushdown WHERE binary_column = x'ABCD'", "VALUES 0"); + + // timestamp + assertQuery("SELECT count(*) FROM filter_pushdown WHERE timestamp_column = TIMESTAMP '2019-10-01 00:00:00'", "VALUES 1"); + assertQuery("SELECT count(*) FROM filter_pushdown WHERE timestamp_column > TIMESTAMP '2019-10-01 00:00:00'", "VALUES 0"); + assertQuery("SELECT count(*) FROM filter_pushdown WHERE timestamp_column < TIMESTAMP '2019-10-01 00:00:00'", "VALUES 0"); + assertQuery("SELECT count(*) FROM filter_pushdown WHERE timestamp_column = TIMESTAMP '2019-10-02 00:00:00'", "VALUES 0"); + assertQuery("SELECT count(*) FROM filter_pushdown WHERE timestamp_column > TIMESTAMP '2001-01-01 00:00:00'", "VALUES 1"); + assertQuery("SELECT count(*) FROM filter_pushdown WHERE timestamp_column < TIMESTAMP '2030-01-01 00:00:00'", "VALUES 1"); + + // ipaddress + assertQuery("SELECT count(*) FROM filter_pushdown WHERE ipv4_column = IPADDRESS '1.2.3.4'", "VALUES 1"); + assertQuery("SELECT count(*) FROM filter_pushdown WHERE ipv6_column = IPADDRESS '2001:db8::1:0:0:1'", "VALUES 1"); + } + + @Test + public void testLimitPushdown() + throws IOException + { + assertThat(query("SELECT name FROM nation LIMIT 30")).isNotFullyPushedDown(LimitNode.class); // Use high limit for result determinism + } + + @Test + public void testDataTypesNested() + throws IOException + { + String indexName = "types_nested"; + + @Language("JSON") + String properties = "" + + "{" + + " \"properties\":{" + + " \"field\": {" + + " \"properties\": {" + + " \"boolean_column\": { \"type\": \"boolean\" }," + + " \"float_column\": { \"type\": \"float\" }," + + " \"double_column\": { \"type\": \"double\" }," + + " \"integer_column\": { \"type\": \"integer\" }," + + " \"long_column\": { \"type\": \"long\" }," + + " \"keyword_column\": { \"type\": \"keyword\" }," + + " \"text_column\": { \"type\": \"text\" }," + + " \"binary_column\": { \"type\": \"binary\" }," + + " \"timestamp_column\": { \"type\": \"date\" }," + + " \"ipv4_column\": { \"type\": \"ip\" }," + + " \"ipv6_column\": { \"type\": \"ip\" }" + + " }" + + " }" + + " }" + + "}"; + + createIndex(indexName, properties); + + index(indexName, ImmutableMap.of( + "field", + ImmutableMap.builder() + .put("boolean_column", true) + .put("float_column", 1.0f) + .put("double_column", 1.0d) + .put("integer_column", 1) + .put("long_column", 1L) + .put("keyword_column", "cool") + .put("text_column", "some text") + .put("binary_column", new byte[] {(byte) 0xCA, (byte) 0xFE}) + .put("timestamp_column", 0) + .put("ipv4_column", "1.2.3.4") + .put("ipv6_column", "2001:db8:0:0:1:0:0:1") + .buildOrThrow())); + + MaterializedResult rows = computeActual("" + + "SELECT " + + "field.boolean_column, " + + "field.float_column, " + + "field.double_column, " + + "field.integer_column, " + + "field.long_column, " + + "field.keyword_column, " + + "field.text_column, " + + "field.binary_column, " + + "field.timestamp_column, " + + "field.ipv4_column, " + + "field.ipv6_column " + + "FROM types_nested"); + + MaterializedResult expected = resultBuilder(getSession(), rows.getTypes()) + .row(true, 1.0f, 1.0d, 1, 1L, "cool", "some text", new byte[] {(byte) 0xCA, (byte) 0xFE}, + LocalDateTime.of(1970, 1, 1, 0, 0), "1.2.3.4", "2001:db8::1:0:0:1") + .build(); + + assertThat(rows.getMaterializedRows()).isEqualTo(expected.getMaterializedRows()); + } + + @Test + public void testNestedTypeDataTypesNested() + throws IOException + { + String indexName = "nested_type_nested"; + + @Language("JSON") + String mappings = "" + + "{" + + " \"properties\":{" + + " \"nested_field\": {" + + " \"type\":\"nested\"," + + " \"properties\": {" + + " \"boolean_column\": { \"type\": \"boolean\" }," + + " \"float_column\": { \"type\": \"float\" }," + + " \"double_column\": { \"type\": \"double\" }," + + " \"integer_column\": { \"type\": \"integer\" }," + + " \"long_column\": { \"type\": \"long\" }," + + " \"keyword_column\": { \"type\": \"keyword\" }," + + " \"text_column\": { \"type\": \"text\" }," + + " \"binary_column\": { \"type\": \"binary\" }," + + " \"timestamp_column\": { \"type\": \"date\" }," + + " \"ipv4_column\": { \"type\": \"ip\" }," + + " \"ipv6_column\": { \"type\": \"ip\" }" + + " }" + + " }" + + " }" + + "}"; + + createIndex(indexName, mappings); + + index(indexName, ImmutableMap.of( + "nested_field", + ImmutableMap.builder() + .put("boolean_column", true) + .put("float_column", 1.0f) + .put("double_column", 1.0d) + .put("integer_column", 1) + .put("long_column", 1L) + .put("keyword_column", "cool") + .put("text_column", "some text") + .put("binary_column", new byte[] {(byte) 0xCA, (byte) 0xFE}) + .put("timestamp_column", 0) + .put("ipv4_column", "1.2.3.4") + .put("ipv6_column", "2001:db8:0:0:1:0:0:1") + .buildOrThrow())); + + MaterializedResult rows = computeActual("" + + "SELECT " + + "nested_field.boolean_column, " + + "nested_field.float_column, " + + "nested_field.double_column, " + + "nested_field.integer_column, " + + "nested_field.long_column, " + + "nested_field.keyword_column, " + + "nested_field.text_column, " + + "nested_field.binary_column, " + + "nested_field.timestamp_column, " + + "nested_field.ipv4_column, " + + "nested_field.ipv6_column " + + "FROM nested_type_nested"); + + MaterializedResult expected = resultBuilder(getSession(), rows.getTypes()) + .row(true, 1.0f, 1.0d, 1, 1L, "cool", "some text", new byte[] {(byte) 0xCA, (byte) 0xFE}, + LocalDateTime.of(1970, 1, 1, 0, 0), "1.2.3.4", "2001:db8::1:0:0:1") + .build(); + + assertThat(rows.getMaterializedRows()).isEqualTo(expected.getMaterializedRows()); + } + + @Test + public void testQueryString() + { + assertQuery("SELECT count(*) FROM \"orders: +packages -slyly\"", "VALUES 1639"); + } + + @Test + public void testMixedCase() + throws IOException + { + String indexName = "mixed_case"; + index(indexName, ImmutableMap.builder() + .put("Name", "john") + .put("AGE", 32) + .buildOrThrow()); + + assertQuery( + "SELECT name, age FROM mixed_case", + "VALUES ('john', 32)"); + + assertQuery( + "SELECT name, age FROM mixed_case WHERE name = 'john'", + "VALUES ('john', 32)"); + } + + @Test + public void testNumericKeyword() + throws IOException + { + String indexName = "numeric_keyword"; + @Language("JSON") + String properties = "" + + "{" + + " \"properties\":{" + + " \"numeric_keyword\": { \"type\": \"keyword\" }" + + " }" + + "}"; + createIndex(indexName, properties); + index(indexName, ImmutableMap.of("numeric_keyword", 20)); + + assertQuery( + "SELECT numeric_keyword FROM numeric_keyword", + "VALUES 20"); + assertQuery( + "SELECT numeric_keyword FROM numeric_keyword where numeric_keyword = '20'", + "VALUES 20"); + } + + @Test + public void testQueryStringError() + { + assertQueryFails("SELECT orderkey FROM \"orders: ++foo AND\"", "\\QFailed to parse query [ ++foo and]\\E"); + assertQueryFails("SELECT count(*) FROM \"orders: ++foo AND\"", "\\QFailed to parse query [ ++foo and]\\E"); + } + + @Test + public void testAlias() + throws IOException + { + String aliasName = format("alias_%s", randomNameSuffix()); + addAlias("orders", aliasName); + + assertQuery( + "SELECT count(*) FROM " + aliasName, + "SELECT count(*) FROM orders"); + } + + @Test + public void testSelectInformationSchemaForMultiIndexAlias() + throws IOException + { + addAlias("nation", "multi_alias"); + addAlias("region", "multi_alias"); + + // No duplicate entries should be found in information_schema.tables or information_schema.columns. + testSelectInformationSchemaTables(); + testSelectInformationSchemaColumns(); + } + + @Test // TODO (https://github.com/trinodb/trino/issues/2428) + @Disabled + public void testMultiIndexAlias() + throws IOException + { + addAlias("nation", "multi_alias"); + addAlias("region", "multi_alias"); + + assertQuery( + "SELECT count(*) FROM multi_alias", + "SELECT (SELECT count(*) FROM region) + (SELECT count(*) FROM nation)"); + } + + @Test + public void testEmptyIndexWithMappings() + throws IOException + { + String indexName = "test_empty_index_with_mappings"; + + @Language("JSON") + String mappings = "" + + "{" + + " \"properties\": { " + + " \"dummy_column\": { \"type\": \"long\" }" + + " }" + + "}"; + + createIndex(indexName, mappings); + + assertQuery(format("SELECT column_name FROM information_schema.columns WHERE table_name = '%s'", indexName), "VALUES ('dummy_column')"); + assertThat(computeActual("SHOW TABLES").getOnlyColumnAsSet()).contains(indexName); + assertQueryReturnsEmptyResult("SELECT * FROM " + indexName); + } + + @Test + public void testEmptyIndexNoMappings() + throws IOException + { + String indexName = "test_empty_index"; + + createIndex(indexName); + assertTableDoesNotExist(indexName); + } + + @Test + public void testEmptyAliasNoMappings() + throws IOException + { + String indexName = "test_empty_index_for_alias"; + String aliasName = "test_empty_alias"; + + createIndex(indexName); + addAlias(indexName, aliasName); + assertTableDoesNotExist(aliasName); + } + + @Test + public void testMissingIndex() + { + assertTableDoesNotExist("nonexistent_table"); + } + + @Test + public void testQueryTableFunction() + { + // select single record + assertQuery("SELECT json_query(result, 'lax $[0][0].hits.hits._source') " + + format("FROM TABLE(%s.system.raw_query(", catalogName) + + "schema => 'tpch', " + + "index => 'nation', " + + "query => '{\"query\": {\"match\": {\"name\": \"ALGERIA\"}}}')) t(result)", + "VALUES '{\"nationkey\":0,\"name\":\"ALGERIA\",\"regionkey\":0,\"comment\":\" haggle. carefully final deposits detect slyly agai\"}'"); + + // parameters + Session session = Session.builder(getSession()) + .addPreparedStatement( + "my_query", + format("SELECT json_query(result, 'lax $[0][0].hits.hits._source') FROM TABLE(%s.system.raw_query(schema => ?, index => ?, query => ?))", catalogName)) + .build(); + assertQuery( + session, + "EXECUTE my_query USING 'tpch', 'nation', '{\"query\": {\"match\": {\"name\": \"ALGERIA\"}}}'", + "VALUES '{\"nationkey\":0,\"name\":\"ALGERIA\",\"regionkey\":0,\"comment\":\" haggle. carefully final deposits detect slyly agai\"}'"); + + // select multiple records by range. Use array wrapper to wrap multiple results + assertQuery("SELECT array_sort(CAST(json_parse(json_query(result, 'lax $[0][0].hits.hits._source.name' WITH ARRAY WRAPPER)) AS array(varchar))) " + + format("FROM TABLE(%s.system.raw_query(", catalogName) + + "schema => 'tpch', " + + "index => 'nation', " + + "query => '{\"query\": {\"range\": {\"nationkey\": {\"gte\": 0,\"lte\": 3}}}}')) t(result)", + "VALUES ARRAY['ALGERIA', 'ARGENTINA', 'BRAZIL', 'CANADA']"); + + // use aggregations + @Language("JSON") + String query = "{\n" + + " \"size\": 0,\n" + + " \"aggs\" : {\n" + + " \"max_orderkey\" : { \"max\" : { \"field\" : \"orderkey\" } },\n" + + " \"sum_orderkey\" : { \"sum\" : { \"field\" : \"orderkey\" } }\n" + + " }\n" + + "}"; + + assertQuery( + format("WITH data(r) AS (" + + " SELECT CAST(json_parse(result) AS ROW(aggregations ROW(max_orderkey ROW(value BIGINT), sum_orderkey ROW(value BIGINT)))) " + + " FROM TABLE(%s.system.raw_query(" + + " schema => 'tpch', " + + " index => 'orders', " + + " query => '%s'))) " + + "SELECT r.aggregations.max_orderkey.value, r.aggregations.sum_orderkey.value " + + "FROM data", catalogName, query), + "VALUES (60000, 449872500)"); + + // no matches + assertQuery("SELECT json_query(result, 'lax $[0][0].hits.hits') " + + format("FROM TABLE(%s.system.raw_query(", catalogName) + + "schema => 'tpch', " + + "index => 'nation', " + + "query => '{\"query\": {\"match\": {\"name\": \"UTOPIA\"}}}')) t(result)", + "VALUES '[]'"); + + // syntax error + assertThatThrownBy(() -> query("SELECT * " + + format("FROM TABLE(%s.system.raw_query(", catalogName) + + "schema => 'tpch', " + + "index => 'nation', " + + "query => 'wrong syntax')) t(result)")) + .hasMessageContaining("json_parse_exception"); + } + + protected void assertTableDoesNotExist(String name) + { + assertQueryReturnsEmptyResult(format("SELECT * FROM information_schema.columns WHERE table_name = '%s'", name)); + assertThat(computeActual("SHOW TABLES").getOnlyColumnAsSet().contains(name)).isFalse(); + assertQueryFails("SELECT * FROM " + name, ".*Table '" + catalogName + ".tpch." + name + "' does not exist"); + } + + protected abstract String indexEndpoint(String index, String docId); + + private void index(String index, Map document) + throws IOException + { + String json = new ObjectMapper().writeValueAsString(document); + String endpoint = format("%s?refresh", indexEndpoint(index, String.valueOf(System.nanoTime()))); + client.getLowLevelClient() + .performRequest("PUT", endpoint, ImmutableMap.of(), new NStringEntity(json, ContentType.APPLICATION_JSON)); + } + + private void addAlias(String index, String alias) + throws IOException + { + client.getLowLevelClient() + .performRequest("PUT", format("/%s/_alias/%s", index, alias)); + + refreshIndex(alias); + } + + protected abstract String indexMapping(@Language("JSON") String properties); + + private void createIndex(String indexName) + throws IOException + { + client.getLowLevelClient().performRequest("PUT", "/" + indexName); + } + + private void createIndex(String indexName, @Language("JSON") String properties) + throws IOException + { + String mappings = indexMapping(properties); + client.getLowLevelClient() + .performRequest("PUT", "/" + indexName, ImmutableMap.of(), new NStringEntity(mappings, ContentType.APPLICATION_JSON)); + } + + private void refreshIndex(String index) + throws IOException + { + client.getLowLevelClient() + .performRequest("GET", format("/%s/_refresh", index)); + } + + private void deleteIndex(String indexName) + throws IOException + { + client.getLowLevelClient() + .performRequest("DELETE", "/" + indexName); + } +} diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchExternalQueryRunner.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchExternalQueryRunner.java new file mode 100644 index 0000000000000..beef029ed9486 --- /dev/null +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchExternalQueryRunner.java @@ -0,0 +1,49 @@ +/* + * 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.elasticsearch; + +import com.google.common.collect.ImmutableMap; +import com.google.common.net.HostAndPort; +import io.airlift.log.Logger; +import io.trino.testing.DistributedQueryRunner; +import io.trino.tpch.TpchTable; + +import static io.trino.plugin.elasticsearch.ElasticsearchQueryRunner.createElasticsearchQueryRunner; +import static java.lang.Integer.parseInt; + +public class ElasticsearchExternalQueryRunner +{ + private static final String HOSTNAME = System.getProperty("elasticsearch.host", "localhost"); + private static final int PORT = parseInt(System.getProperty("elasticsearch.port", "9200")); + + private ElasticsearchExternalQueryRunner() {} + + public static void main(String[] args) + throws Exception + { + // Please set hostname and port via VM options. e.g. "-Delasticsearch.host=localhost -Delasticsearch.port=9200" + // To start Elasticsearch: + // docker run -p 9200:9200 -e "discovery.type=single-node" docker.elastic.co/elasticsearch/elasticsearch:7.6.2 + DistributedQueryRunner queryRunner = createElasticsearchQueryRunner( + HostAndPort.fromParts(HOSTNAME, PORT), + TpchTable.getTables(), + ImmutableMap.of("http-server.http.port", "8080"), + ImmutableMap.of(), + 3); + + Logger log = Logger.get(ElasticsearchExternalQueryRunner.class); + log.info("======== SERVER STARTED ========"); + log.info("\n====\n%s\n====", queryRunner.getCoordinator().getBaseUrl()); + } +} diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchLoader.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchLoader.java new file mode 100644 index 0000000000000..35fdeb39f4831 --- /dev/null +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchLoader.java @@ -0,0 +1,146 @@ +/* + * 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.elasticsearch; + +import io.trino.Session; +import io.trino.client.Column; +import io.trino.client.QueryData; +import io.trino.client.QueryStatusInfo; +import io.trino.server.testing.TestingTrinoServer; +import io.trino.spi.type.Type; +import io.trino.spi.type.VarcharType; +import io.trino.testing.AbstractTestingTrinoClient; +import io.trino.testing.ResultsSession; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.support.WriteRequest; +import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.common.xcontent.XContentBuilder; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; + +import static com.google.common.base.Preconditions.checkState; +import static io.trino.spi.type.BigintType.BIGINT; +import static io.trino.spi.type.BooleanType.BOOLEAN; +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 java.util.Objects.requireNonNull; +import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; + +public class ElasticsearchLoader + extends AbstractTestingTrinoClient +{ + private final String tableName; + private final RestHighLevelClient client; + + public ElasticsearchLoader( + RestHighLevelClient client, + String tableName, + TestingTrinoServer trinoServer, + Session defaultSession) + { + super(trinoServer, defaultSession); + + this.tableName = requireNonNull(tableName, "tableName is null"); + this.client = requireNonNull(client, "client is null"); + } + + @Override + public ResultsSession getResultSession(Session session) + { + requireNonNull(session, "session is null"); + return new ElasticsearchLoadingSession(); + } + + private class ElasticsearchLoadingSession + implements ResultsSession + { + private final AtomicReference> types = new AtomicReference<>(); + + private ElasticsearchLoadingSession() {} + + @Override + public void addResults(QueryStatusInfo statusInfo, QueryData data) + { + if (types.get() == null && statusInfo.getColumns() != null) { + types.set(getTypes(statusInfo.getColumns())); + } + + if (data.getData() == null) { + return; + } + checkState(types.get() != null, "Type information is missing"); + List columns = statusInfo.getColumns(); + + BulkRequest request = new BulkRequest(); + for (List fields : data.getData()) { + try { + XContentBuilder dataBuilder = jsonBuilder().startObject(); + for (int i = 0; i < fields.size(); i++) { + Type type = types.get().get(i); + Object value = convertValue(fields.get(i), type); + dataBuilder.field(columns.get(i).getName(), value); + } + dataBuilder.endObject(); + + request.add(new IndexRequest(tableName, "doc").source(dataBuilder)); + } + catch (IOException e) { + throw new UncheckedIOException("Error loading data into Elasticsearch index: " + tableName, e); + } + } + + request.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); + try { + client.bulk(request); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public Void build(Map setSessionProperties, Set resetSessionProperties) + { + return null; + } + + private Object convertValue(Object value, Type type) + { + if (value == null) { + return null; + } + + if (type == BOOLEAN || type == DATE || type instanceof VarcharType) { + return value; + } + if (type == BIGINT) { + return ((Number) value).longValue(); + } + if (type == INTEGER) { + return ((Number) value).intValue(); + } + if (type == DOUBLE) { + return ((Number) value).doubleValue(); + } + throw new IllegalArgumentException("Unhandled type: " + type); + } + } +} diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchNginxProxy.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchNginxProxy.java new file mode 100644 index 0000000000000..298edeabaac77 --- /dev/null +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchNginxProxy.java @@ -0,0 +1,103 @@ +/* + * 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.elasticsearch; + +import com.google.common.net.HostAndPort; +import org.testcontainers.containers.Network; +import org.testcontainers.containers.NginxContainer; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; + +import static com.google.common.io.MoreFiles.deleteRecursively; +import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.testcontainers.utility.MountableFile.forHostPath; + +public class ElasticsearchNginxProxy +{ + private static final int PROXY_PORT = 9201; + private static final String NGINX_CONFIG_TEMPLATE = + "limit_req_zone $binary_remote_addr zone=mylimit:1m rate=REQUEST_PER_SECONDr/s;\n" + + "upstream elasticsearch {\n" + + " server elasticsearch-server:9200;\n" + + " keepalive 15;\n" + + "}\n" + + "server {\n" + + " access_log /var/log/nginx/access.log main;" + + " listen " + PROXY_PORT + ";\n" + + " location / {\n" + + " proxy_pass http://elasticsearch;\n" + + " proxy_redirect http://elasticsearch /;\n" + + " proxy_buffering off;\n" + + " proxy_http_version 1.1;\n" + + " proxy_set_header Connection \"Keep-Alive\";\n" + + " proxy_set_header Proxy-Connection \"Keep-Alive\";\n" + + " client_max_body_size 0;\n" + + " }\n" + + " location /_search/scroll {\n" + + " limit_req zone=mylimit;\n" + + " limit_req_status 429;\n" + + " proxy_pass http://elasticsearch;\n" + + " proxy_redirect http://elasticsearch /;\n" + + " proxy_buffering off;\n" + + " proxy_http_version 1.1;\n" + + " proxy_set_header Connection \"Keep-Alive\";\n" + + " proxy_set_header Proxy-Connection \"Keep-Alive\";\n" + + " client_max_body_size 0;\n" + + " }\n" + + " location ~ /.*/_search$ {\n" + + " limit_req zone=mylimit;\n" + + " limit_req_status 429;\n" + + " proxy_pass http://elasticsearch;\n" + + " proxy_redirect http://elasticsearch /;\n" + + " proxy_buffering off;\n" + + " proxy_http_version 1.1;\n" + + " proxy_set_header Connection \"Keep-Alive\";\n" + + " proxy_set_header Proxy-Connection \"Keep-Alive\";\n" + + " client_max_body_size 0;\n" + + " }\n" + + "}\n"; + + private final Path configurationPath; + private final NginxContainer container; + + public ElasticsearchNginxProxy(Network network, int requestsPerSecond) + throws IOException + { + container = new NginxContainer<>("nginx:1.19.8"); + container.withNetwork(network); + container.withNetworkAliases("elasticsearch-proxy"); + // Create the Nginx configuration file on host and copy it into a predefined path the container + configurationPath = Files.createTempDirectory("elasticsearchProxy"); + Path path = configurationPath.resolve("elasticsearch.conf"); + Files.writeString(path, NGINX_CONFIG_TEMPLATE.replace("REQUEST_PER_SECOND", String.valueOf(requestsPerSecond)), UTF_8); + container.withCopyFileToContainer(forHostPath(path), "/etc/nginx/conf.d/elasticsearch.conf"); + container.addExposedPort(PROXY_PORT); + container.start(); + } + + public void stop() + throws IOException + { + container.close(); + deleteRecursively(configurationPath, ALLOW_INSECURE); + } + + public HostAndPort getAddress() + { + return HostAndPort.fromString(container.getHost() + ":" + container.getMappedPort(PROXY_PORT)); + } +} diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchQueryRunner.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchQueryRunner.java new file mode 100644 index 0000000000000..73be8fd317517 --- /dev/null +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchQueryRunner.java @@ -0,0 +1,162 @@ +/* + * 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.elasticsearch; + +import com.google.common.collect.ImmutableMap; +import com.google.common.net.HostAndPort; +import io.airlift.log.Level; +import io.airlift.log.Logger; +import io.airlift.log.Logging; +import io.trino.metadata.QualifiedObjectName; +import io.trino.plugin.jmx.JmxPlugin; +import io.trino.plugin.tpch.TpchPlugin; +import io.trino.testing.DistributedQueryRunner; +import io.trino.testing.QueryRunner; +import io.trino.testing.TestingTrinoClient; +import io.trino.tpch.TpchTable; +import org.apache.http.HttpHost; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestHighLevelClient; + +import java.util.Map; + +import static io.airlift.testing.Closeables.closeAllSuppress; +import static io.airlift.units.Duration.nanosSince; +import static io.trino.plugin.elasticsearch.ElasticsearchServer.ELASTICSEARCH_7_IMAGE; +import static io.trino.plugin.tpch.TpchMetadata.TINY_SCHEMA_NAME; +import static io.trino.testing.TestingSession.testSessionBuilder; +import static java.lang.String.format; +import static java.util.Locale.ENGLISH; +import static java.util.concurrent.TimeUnit.SECONDS; + +public final class ElasticsearchQueryRunner +{ + static { + Logging logging = Logging.initialize(); + logging.setLevel("org.elasticsearch.client.RestClient", Level.OFF); + } + + private ElasticsearchQueryRunner() {} + + private static final Logger LOG = Logger.get(ElasticsearchQueryRunner.class); + private static final String TPCH_SCHEMA = "tpch"; + + public static DistributedQueryRunner createElasticsearchQueryRunner( + HostAndPort address, + Iterable> tables, + Map extraProperties, + Map extraConnectorProperties, + int nodeCount) + throws Exception + { + return createElasticsearchQueryRunner(address, tables, extraProperties, extraConnectorProperties, nodeCount, "elasticsearch"); + } + + public static DistributedQueryRunner createElasticsearchQueryRunner( + HostAndPort address, + Iterable> tables, + Map extraProperties, + Map extraConnectorProperties, + int nodeCount, + String catalogName) + throws Exception + { + RestHighLevelClient client = null; + DistributedQueryRunner queryRunner = null; + try { + queryRunner = DistributedQueryRunner.builder(testSessionBuilder() + .setCatalog(catalogName) + .setSchema(TPCH_SCHEMA) + .build()) + .setExtraProperties(extraProperties) + .setNodeCount(nodeCount) + .build(); + + queryRunner.installPlugin(new JmxPlugin()); + queryRunner.createCatalog("jmx", "jmx"); + + queryRunner.installPlugin(new TpchPlugin()); + queryRunner.createCatalog("tpch", "tpch"); + + ElasticsearchConnectorFactory testFactory = new ElasticsearchConnectorFactory(); + + installElasticsearchPlugin(address, queryRunner, catalogName, testFactory, extraConnectorProperties); + + TestingTrinoClient trinoClient = queryRunner.getClient(); + + LOG.info("Loading data..."); + + client = new RestHighLevelClient(RestClient.builder(HttpHost.create(address.toString()))); + long startTime = System.nanoTime(); + for (TpchTable table : tables) { + loadTpchTopic(client, trinoClient, table); + } + LOG.info("Loading complete in %s", nanosSince(startTime).toString(SECONDS)); + + return queryRunner; + } + catch (Exception e) { + closeAllSuppress(e, queryRunner, client); + throw e; + } + } + + private static void installElasticsearchPlugin( + HostAndPort address, + QueryRunner queryRunner, + String catalogName, + ElasticsearchConnectorFactory factory, + Map extraConnectorProperties) + { + queryRunner.installPlugin(new ElasticsearchPlugin(factory)); + Map config = ImmutableMap.builder() + .put("elasticsearch.host", address.getHost()) + .put("elasticsearch.port", Integer.toString(address.getPort())) + // Node discovery relies on the publish_address exposed via the Elasticseach API + // This doesn't work well within a docker environment that maps ES's port to a random public port + .put("elasticsearch.ignore-publish-address", "true") + .put("elasticsearch.default-schema-name", TPCH_SCHEMA) + .put("elasticsearch.scroll-size", "1000") + .put("elasticsearch.scroll-timeout", "1m") + .put("elasticsearch.request-timeout", "2m") + .putAll(extraConnectorProperties) + .buildOrThrow(); + + queryRunner.createCatalog(catalogName, "elasticsearch", config); + } + + private static void loadTpchTopic(RestHighLevelClient client, TestingTrinoClient trinoClient, TpchTable table) + { + long start = System.nanoTime(); + LOG.info("Running import for %s", table.getTableName()); + ElasticsearchLoader loader = new ElasticsearchLoader(client, table.getTableName().toLowerCase(ENGLISH), trinoClient.getServer(), trinoClient.getDefaultSession()); + loader.execute(format("SELECT * from %s", new QualifiedObjectName(TPCH_SCHEMA, TINY_SCHEMA_NAME, table.getTableName().toLowerCase(ENGLISH)))); + LOG.info("Imported %s in %s", table.getTableName(), nanosSince(start).convertToMostSuccinctTimeUnit()); + } + + public static void main(String[] args) + throws Exception + { + DistributedQueryRunner queryRunner = createElasticsearchQueryRunner( + new ElasticsearchServer(ELASTICSEARCH_7_IMAGE, ImmutableMap.of()).getAddress(), + TpchTable.getTables(), + ImmutableMap.of("http-server.http.port", "8080"), + ImmutableMap.of(), + 3); + + Logger log = Logger.get(ElasticsearchQueryRunner.class); + log.info("======== SERVER STARTED ========"); + log.info("\n====\n%s\n====", queryRunner.getCoordinator().getBaseUrl()); + } +} diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchServer.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchServer.java new file mode 100644 index 0000000000000..8f3da20229d14 --- /dev/null +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchServer.java @@ -0,0 +1,85 @@ +/* + * 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.elasticsearch; + +import com.google.common.net.HostAndPort; +import io.trino.testing.ResourcePresence; +import org.testcontainers.containers.Network; +import org.testcontainers.elasticsearch.ElasticsearchContainer; +import org.testcontainers.utility.DockerImageName; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Map; + +import static com.google.common.io.MoreFiles.deleteRecursively; +import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; +import static java.nio.charset.StandardCharsets.UTF_8; +import static java.nio.file.Files.createTempDirectory; +import static org.testcontainers.utility.MountableFile.forHostPath; + +public class ElasticsearchServer +{ + public static final String ELASTICSEARCH_7_IMAGE = "elasticsearch:7.0.0"; + + private final Path configurationPath; + private final ElasticsearchContainer container; + + public ElasticsearchServer(String image, Map configurationFiles) + throws IOException + { + this(Network.SHARED, image, configurationFiles); + } + + public ElasticsearchServer(Network network, String image, Map configurationFiles) + throws IOException + { + DockerImageName dockerImageName = DockerImageName.parse(image).asCompatibleSubstituteFor("docker.elastic.co/elasticsearch/elasticsearch"); + container = new ElasticsearchContainer(dockerImageName); + container.withNetwork(network); + container.withNetworkAliases("elasticsearch-server"); + container.withEnv("DISABLE_SECURITY_PLUGIN", "true"); // Required for OpenSearch container + + configurationPath = createTempDirectory(null); + for (Map.Entry entry : configurationFiles.entrySet()) { + String name = entry.getKey(); + String contents = entry.getValue(); + + Path path = configurationPath.resolve(name); + Files.writeString(path, contents, UTF_8); + container.withCopyFileToContainer(forHostPath(path), "/usr/share/elasticsearch/config/" + name); + } + + container.start(); + } + + public void stop() + throws IOException + { + container.close(); + deleteRecursively(configurationPath, ALLOW_INSECURE); + } + + @ResourcePresence + public boolean isRunning() + { + return container.getContainerId() != null; + } + + public HostAndPort getAddress() + { + return HostAndPort.fromString(container.getHttpHostAddress()); + } +} diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestAwsSecurityConfig.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestAwsSecurityConfig.java new file mode 100644 index 0000000000000..8bfb32eac484d --- /dev/null +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestAwsSecurityConfig.java @@ -0,0 +1,58 @@ +/* + * 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.elasticsearch; + +import com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.Test; + +import java.util.Map; + +import static io.airlift.configuration.testing.ConfigAssertions.assertFullMapping; +import static io.airlift.configuration.testing.ConfigAssertions.assertRecordedDefaults; +import static io.airlift.configuration.testing.ConfigAssertions.recordDefaults; + +public class TestAwsSecurityConfig +{ + @Test + public void testDefaults() + { + assertRecordedDefaults(recordDefaults(AwsSecurityConfig.class) + .setAccessKey(null) + .setSecretKey(null) + .setRegion(null) + .setIamRole(null) + .setExternalId(null)); + } + + @Test + public void testExplicitPropertyMappings() + { + Map properties = ImmutableMap.builder() + .put("elasticsearch.aws.access-key", "access") + .put("elasticsearch.aws.secret-key", "secret") + .put("elasticsearch.aws.region", "region") + .put("elasticsearch.aws.iam-role", "iamRole") + .put("elasticsearch.aws.external-id", "externalId") + .buildOrThrow(); + + AwsSecurityConfig expected = new AwsSecurityConfig() + .setAccessKey("access") + .setSecretKey("secret") + .setRegion("region") + .setIamRole("iamRole") + .setExternalId("externalId"); + + assertFullMapping(properties, expected); + } +} diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearch6ConnectorTest.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearch6ConnectorTest.java new file mode 100644 index 0000000000000..06b2c74992008 --- /dev/null +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearch6ConnectorTest.java @@ -0,0 +1,63 @@ +/* + * 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.elasticsearch; + +import com.google.common.collect.ImmutableMap; +import org.apache.http.entity.ContentType; +import org.apache.http.nio.entity.NStringEntity; +import org.intellij.lang.annotations.Language; +import org.junit.jupiter.api.Test; + +import java.io.IOException; + +import static java.lang.String.format; + +public class TestElasticsearch6ConnectorTest + extends BaseElasticsearchConnectorTest +{ + public TestElasticsearch6ConnectorTest() + { + super("docker.elastic.co/elasticsearch/elasticsearch-oss:6.6.0", "elasticsearch6"); + } + + @Test + public void testIndexWithMappingsButNoProperties() + throws IOException + { + String indexName = "test_empty_index_with_mappings_no_properties"; + + @Language("JSON") + String mappings = "{\"mappings\": " + + " {\"foo\": { \"dynamic\" : \"strict\" } }" + + "}"; + client.getLowLevelClient() + .performRequest("PUT", "/" + indexName, ImmutableMap.of(), new NStringEntity(mappings, ContentType.APPLICATION_JSON)); + + assertTableDoesNotExist(indexName); + } + + @Override + protected String indexEndpoint(String index, String docId) + { + return format("/%s/doc/%s", index, docId); + } + + @Override + protected String indexMapping(String properties) + { + return "{\"mappings\": " + + " {\"doc\": " + properties + "}" + + "}"; + } +} diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearch7ConnectorTest.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearch7ConnectorTest.java new file mode 100644 index 0000000000000..b48d76c34cc0b --- /dev/null +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearch7ConnectorTest.java @@ -0,0 +1,38 @@ +/* + * 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.elasticsearch; + +import static io.trino.plugin.elasticsearch.ElasticsearchServer.ELASTICSEARCH_7_IMAGE; +import static java.lang.String.format; + +public class TestElasticsearch7ConnectorTest + extends BaseElasticsearchConnectorTest +{ + public TestElasticsearch7ConnectorTest() + { + super(ELASTICSEARCH_7_IMAGE, "elasticsearch7"); + } + + @Override + protected String indexEndpoint(String index, String docId) + { + return format("/%s/_doc/%s", index, docId); + } + + @Override + protected String indexMapping(String properties) + { + return "{\"mappings\": " + properties + "}"; + } +} diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchBackpressure.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchBackpressure.java new file mode 100644 index 0000000000000..dbaf931760664 --- /dev/null +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchBackpressure.java @@ -0,0 +1,86 @@ +/* + * 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.elasticsearch; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.trino.testing.AbstractTestQueryFramework; +import io.trino.testing.QueryRunner; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.testcontainers.containers.Network; + +import java.io.IOException; + +import static io.trino.plugin.elasticsearch.ElasticsearchQueryRunner.createElasticsearchQueryRunner; +import static io.trino.tpch.TpchTable.ORDERS; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; + +@TestInstance(PER_CLASS) +public class TestElasticsearchBackpressure + extends AbstractTestQueryFramework +{ + private static final String image = "elasticsearch:7.0.0"; + + private Network network; + private ElasticsearchServer elasticsearch; + private ElasticsearchNginxProxy elasticsearchNginxProxy; + + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + network = Network.newNetwork(); + elasticsearch = new ElasticsearchServer(network, image, ImmutableMap.of()); + elasticsearchNginxProxy = new ElasticsearchNginxProxy(network, 1); + + return createElasticsearchQueryRunner( + elasticsearchNginxProxy.getAddress(), + ImmutableList.of(ORDERS), + ImmutableMap.of(), + ImmutableMap.of(), + // This test can only run on a single node, otherwise each node exports its own stats beans and they override each other + // You can only bind one such bean per JVM, so this causes problems with statistics being 0 despite backpressure handling + 1, + // Use a unique catalog name to make sure JMX stats beans are unique and not affected by other tests + "elasticsearch-backpressure"); + } + + @AfterAll + public final void destroy() + throws IOException + { + elasticsearchNginxProxy.stop(); + elasticsearchNginxProxy = null; + elasticsearch.stop(); + elasticsearch = null; + network.close(); + network = null; + } + + @Test + public void testQueryWithBackpressure() + { + // Check that JMX stats show no sign of backpressure + assertQueryReturnsEmptyResult("SELECT 1 FROM jmx.current.\"trino.plugin.elasticsearch.client:*\" WHERE \"backpressurestats.alltime.count\" > 0"); + assertQueryReturnsEmptyResult("SELECT 1 FROM jmx.current.\"trino.plugin.elasticsearch.client:*\" WHERE \"backpressurestats.alltime.max\" > 0"); + + assertQuerySucceeds("SELECT * FROM orders"); + + // Check that JMX stats show requests have been retried due to backpressure + assertQuery("SELECT DISTINCT 1 FROM jmx.current.\"trino.plugin.elasticsearch.client:*\" WHERE \"backpressurestats.alltime.count\" > 0", "VALUES 1"); + assertQuery("SELECT DISTINCT 1 FROM jmx.current.\"trino.plugin.elasticsearch.client:*\" WHERE \"backpressurestats.alltime.max\" > 0", "VALUES 1"); + } +} diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchConfig.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchConfig.java new file mode 100644 index 0000000000000..4a74a4096701f --- /dev/null +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchConfig.java @@ -0,0 +1,119 @@ +/* + * 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.elasticsearch; + +import com.google.common.collect.ImmutableMap; +import io.airlift.units.Duration; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Arrays; +import java.util.Map; + +import static io.airlift.configuration.testing.ConfigAssertions.assertFullMapping; +import static io.airlift.configuration.testing.ConfigAssertions.assertRecordedDefaults; +import static io.airlift.configuration.testing.ConfigAssertions.recordDefaults; +import static io.trino.plugin.elasticsearch.ElasticsearchConfig.Security.AWS; +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static java.util.concurrent.TimeUnit.MINUTES; +import static java.util.concurrent.TimeUnit.SECONDS; + +public class TestElasticsearchConfig +{ + @Test + public void testDefaults() + { + assertRecordedDefaults(recordDefaults(ElasticsearchConfig.class) + .setHosts(null) + .setPort(9200) + .setDefaultSchema("default") + .setScrollSize(1000) + .setScrollTimeout(new Duration(1, MINUTES)) + .setRequestTimeout(new Duration(10, SECONDS)) + .setConnectTimeout(new Duration(1, SECONDS)) + .setBackoffInitDelay(new Duration(500, MILLISECONDS)) + .setBackoffMaxDelay(new Duration(20, SECONDS)) + .setMaxRetryTime(new Duration(30, SECONDS)) + .setNodeRefreshInterval(new Duration(1, MINUTES)) + .setMaxHttpConnections(25) + .setHttpThreadCount(Runtime.getRuntime().availableProcessors()) + .setTlsEnabled(false) + .setKeystorePath(null) + .setKeystorePassword(null) + .setTrustStorePath(null) + .setTruststorePassword(null) + .setVerifyHostnames(true) + .setIgnorePublishAddress(false) + .setSecurity(null)); + } + + @Test + public void testExplicitPropertyMappings() + throws IOException + { + Path keystoreFile = Files.createTempFile(null, null); + Path truststoreFile = Files.createTempFile(null, null); + + Map properties = ImmutableMap.builder() + .put("elasticsearch.host", "example.com") + .put("elasticsearch.port", "9999") + .put("elasticsearch.default-schema-name", "test") + .put("elasticsearch.scroll-size", "4000") + .put("elasticsearch.scroll-timeout", "20s") + .put("elasticsearch.request-timeout", "1s") + .put("elasticsearch.connect-timeout", "10s") + .put("elasticsearch.backoff-init-delay", "100ms") + .put("elasticsearch.backoff-max-delay", "15s") + .put("elasticsearch.max-retry-time", "10s") + .put("elasticsearch.node-refresh-interval", "10m") + .put("elasticsearch.max-http-connections", "100") + .put("elasticsearch.http-thread-count", "30") + .put("elasticsearch.tls.enabled", "true") + .put("elasticsearch.tls.keystore-path", keystoreFile.toString()) + .put("elasticsearch.tls.keystore-password", "keystore-password") + .put("elasticsearch.tls.truststore-path", truststoreFile.toString()) + .put("elasticsearch.tls.truststore-password", "truststore-password") + .put("elasticsearch.tls.verify-hostnames", "false") + .put("elasticsearch.ignore-publish-address", "true") + .put("elasticsearch.security", "AWS") + .buildOrThrow(); + + ElasticsearchConfig expected = new ElasticsearchConfig() + .setHosts(Arrays.asList("example.com")) + .setPort(9999) + .setDefaultSchema("test") + .setScrollSize(4000) + .setScrollTimeout(new Duration(20, SECONDS)) + .setRequestTimeout(new Duration(1, SECONDS)) + .setConnectTimeout(new Duration(10, SECONDS)) + .setBackoffInitDelay(new Duration(100, MILLISECONDS)) + .setBackoffMaxDelay(new Duration(15, SECONDS)) + .setMaxRetryTime(new Duration(10, SECONDS)) + .setNodeRefreshInterval(new Duration(10, MINUTES)) + .setMaxHttpConnections(100) + .setHttpThreadCount(30) + .setTlsEnabled(true) + .setKeystorePath(keystoreFile.toFile()) + .setKeystorePassword("keystore-password") + .setTrustStorePath(truststoreFile.toFile()) + .setTruststorePassword("truststore-password") + .setVerifyHostnames(false) + .setIgnorePublishAddress(true) + .setSecurity(AWS); + + assertFullMapping(properties, expected); + } +} diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchMetadata.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchMetadata.java new file mode 100644 index 0000000000000..f1f7c17cf9ac2 --- /dev/null +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchMetadata.java @@ -0,0 +1,47 @@ +/* + * 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.elasticsearch; + +import io.airlift.slice.Slices; +import org.junit.jupiter.api.Test; + +import java.util.Optional; + +import static org.assertj.core.api.Assertions.assertThat; + +public class TestElasticsearchMetadata +{ + @Test + public void testLikeToRegexp() + { + assertThat(likeToRegexp("a_b_c", Optional.empty())).isEqualTo("a.b.c"); + assertThat(likeToRegexp("a%b%c", Optional.empty())).isEqualTo("a.*b.*c"); + assertThat(likeToRegexp("a%b_c", Optional.empty())).isEqualTo("a.*b.c"); + assertThat(likeToRegexp("a[b", Optional.empty())).isEqualTo("a\\[b"); + assertThat(likeToRegexp("a_\\_b", Optional.of("\\"))).isEqualTo("a._b"); + assertThat(likeToRegexp("a$_b", Optional.of("$"))).isEqualTo("a_b"); + assertThat(likeToRegexp("s_.m%ex\\t", Optional.of("$"))).isEqualTo("s.\\.m.*ex\\\\t"); + assertThat(likeToRegexp("\000%", Optional.empty())).isEqualTo("\000.*"); + assertThat(likeToRegexp("\000%", Optional.of("\000"))).isEqualTo("%"); + assertThat(likeToRegexp("中文%", Optional.empty())).isEqualTo("中文.*"); + assertThat(likeToRegexp("こんにちは%", Optional.empty())).isEqualTo("こんにちは.*"); + assertThat(likeToRegexp("안녕하세요%", Optional.empty())).isEqualTo("안녕하세요.*"); + assertThat(likeToRegexp("Привет%", Optional.empty())).isEqualTo("Привет.*"); + } + + private static String likeToRegexp(String pattern, Optional escapeChar) + { + return ElasticsearchMetadata.likeToRegexp(Slices.utf8Slice(pattern), escapeChar.map(Slices::utf8Slice)); + } +} diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchOpenSearchConnectorTest.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchOpenSearchConnectorTest.java new file mode 100644 index 0000000000000..ee0dd999729df --- /dev/null +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchOpenSearchConnectorTest.java @@ -0,0 +1,38 @@ +/* + * 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.elasticsearch; + +import static java.lang.String.format; + +public class TestElasticsearchOpenSearchConnectorTest + extends BaseElasticsearchConnectorTest +{ + public TestElasticsearchOpenSearchConnectorTest() + { + // 1.0.0 and 1.0.1 causes NotSslRecordException during the initialization + super("opensearchproject/opensearch:1.1.0", "opensearch"); + } + + @Override + protected String indexEndpoint(String index, String docId) + { + return format("/%s/_doc/%s", index, docId); + } + + @Override + protected String indexMapping(String properties) + { + return "{\"mappings\": " + properties + "}"; + } +} diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchQueryBuilder.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchQueryBuilder.java new file mode 100644 index 0000000000000..f040188b5f128 --- /dev/null +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchQueryBuilder.java @@ -0,0 +1,138 @@ +/* + * 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.elasticsearch; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.trino.plugin.elasticsearch.decoders.DoubleDecoder; +import io.trino.plugin.elasticsearch.decoders.IntegerDecoder; +import io.trino.plugin.elasticsearch.decoders.VarcharDecoder; +import io.trino.spi.predicate.Domain; +import io.trino.spi.predicate.Range; +import io.trino.spi.predicate.TupleDomain; +import io.trino.spi.predicate.ValueSet; +import org.elasticsearch.index.query.BoolQueryBuilder; +import org.elasticsearch.index.query.ExistsQueryBuilder; +import org.elasticsearch.index.query.MatchAllQueryBuilder; +import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.index.query.RangeQueryBuilder; +import org.elasticsearch.index.query.TermQueryBuilder; +import org.junit.jupiter.api.Test; + +import java.util.Map; +import java.util.Optional; + +import static io.trino.plugin.elasticsearch.ElasticsearchQueryBuilder.buildSearchQuery; +import static io.trino.spi.type.DoubleType.DOUBLE; +import static io.trino.spi.type.IntegerType.INTEGER; +import static io.trino.spi.type.VarcharType.VARCHAR; +import static org.assertj.core.api.Assertions.assertThat; + +public class TestElasticsearchQueryBuilder +{ + private static final ElasticsearchColumnHandle NAME = new ElasticsearchColumnHandle("name", VARCHAR, new VarcharDecoder.Descriptor("name"), true); + private static final ElasticsearchColumnHandle AGE = new ElasticsearchColumnHandle("age", INTEGER, new IntegerDecoder.Descriptor("age"), true); + private static final ElasticsearchColumnHandle SCORE = new ElasticsearchColumnHandle("score", DOUBLE, new DoubleDecoder.Descriptor("score"), true); + private static final ElasticsearchColumnHandle LENGTH = new ElasticsearchColumnHandle("length", DOUBLE, new DoubleDecoder.Descriptor("length"), true); + + @Test + public void testMatchAll() + { + assertQueryBuilder( + ImmutableMap.of(), + new MatchAllQueryBuilder()); + } + + @Test + public void testOneConstraint() + { + // SingleValue + assertQueryBuilder( + ImmutableMap.of(AGE, Domain.singleValue(INTEGER, 1L)), + new BoolQueryBuilder().filter(new TermQueryBuilder(AGE.getName(), 1L))); + + // Range + assertQueryBuilder( + ImmutableMap.of(SCORE, Domain.create(ValueSet.ofRanges(Range.range(DOUBLE, 65.0, false, 80.0, true)), false)), + new BoolQueryBuilder().filter(new RangeQueryBuilder(SCORE.getName()).gt(65.0).lte(80.0))); + + // List + assertQueryBuilder( + ImmutableMap.of(NAME, Domain.multipleValues(VARCHAR, ImmutableList.of("alice", "bob"))), + new BoolQueryBuilder().filter( + new BoolQueryBuilder() + .should(new TermQueryBuilder(NAME.getName(), "alice")) + .should(new TermQueryBuilder(NAME.getName(), "bob")))); + // all + assertQueryBuilder( + ImmutableMap.of(AGE, Domain.all(INTEGER)), + new MatchAllQueryBuilder()); + + // notNull + assertQueryBuilder( + ImmutableMap.of(AGE, Domain.notNull(INTEGER)), + new BoolQueryBuilder().filter(new ExistsQueryBuilder(AGE.getName()))); + + // isNull + assertQueryBuilder( + ImmutableMap.of(AGE, Domain.onlyNull(INTEGER)), + new BoolQueryBuilder().mustNot(new ExistsQueryBuilder(AGE.getName()))); + + // isNullAllowed + assertQueryBuilder( + ImmutableMap.of(AGE, Domain.singleValue(INTEGER, 1L, true)), + new BoolQueryBuilder().filter( + new BoolQueryBuilder() + .should(new TermQueryBuilder(AGE.getName(), 1L)) + .should(new BoolQueryBuilder().mustNot(new ExistsQueryBuilder(AGE.getName()))))); + } + + @Test + public void testMultiConstraint() + { + assertQueryBuilder( + ImmutableMap.of( + AGE, Domain.singleValue(INTEGER, 1L), + SCORE, Domain.create(ValueSet.ofRanges(Range.range(DOUBLE, 65.0, false, 80.0, true)), false)), + new BoolQueryBuilder() + .filter(new TermQueryBuilder(AGE.getName(), 1L)) + .filter(new RangeQueryBuilder(SCORE.getName()).gt(65.0).lte(80.0))); + + assertQueryBuilder( + ImmutableMap.of( + LENGTH, Domain.create(ValueSet.ofRanges(Range.range(DOUBLE, 160.0, true, 180.0, true)), false), + SCORE, Domain.create(ValueSet.ofRanges( + Range.range(DOUBLE, 65.0, false, 80.0, true), + Range.equal(DOUBLE, 90.0)), false)), + new BoolQueryBuilder() + .filter(new RangeQueryBuilder(LENGTH.getName()).gte(160.0).lte(180.0)) + .filter(new BoolQueryBuilder() + .should(new RangeQueryBuilder(SCORE.getName()).gt(65.0).lte(80.0)) + .should(new TermQueryBuilder(SCORE.getName(), 90.0)))); + + assertQueryBuilder( + ImmutableMap.of( + AGE, Domain.singleValue(INTEGER, 10L), + SCORE, Domain.onlyNull(DOUBLE)), + new BoolQueryBuilder() + .filter(new TermQueryBuilder(AGE.getName(), 10L)) + .mustNot(new ExistsQueryBuilder(SCORE.getName()))); + } + + private static void assertQueryBuilder(Map domains, QueryBuilder expected) + { + QueryBuilder actual = buildSearchQuery(TupleDomain.withColumnDomains(domains), Optional.empty(), Map.of()); + assertThat(actual).isEqualTo(expected); + } +} diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestPasswordAuthentication.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestPasswordAuthentication.java new file mode 100644 index 0000000000000..14cb228e229a0 --- /dev/null +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestPasswordAuthentication.java @@ -0,0 +1,125 @@ +/* + * 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.elasticsearch; + +import com.amazonaws.util.Base64; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.io.Resources; +import com.google.common.net.HostAndPort; +import io.trino.sql.query.QueryAssertions; +import io.trino.testing.DistributedQueryRunner; +import org.apache.http.HttpHost; +import org.apache.http.entity.ContentType; +import org.apache.http.message.BasicHeader; +import org.apache.http.nio.entity.NStringEntity; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestHighLevelClient; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.parallel.Execution; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; + +import static com.google.common.io.Resources.getResource; +import static io.airlift.testing.Closeables.closeAll; +import static io.trino.plugin.elasticsearch.ElasticsearchQueryRunner.createElasticsearchQueryRunner; +import static java.lang.String.format; +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; + +@TestInstance(PER_CLASS) +@Execution(CONCURRENT) +public class TestPasswordAuthentication +{ + private static final String USER = "elastic_user"; + private static final String PASSWORD = "123456"; + + private ElasticsearchServer elasticsearch; + private RestHighLevelClient client; + private QueryAssertions assertions; + + @BeforeAll + public void setUp() + throws Exception + { + // We use 7.8.0 because security became a non-commercial feature in recent versions + elasticsearch = new ElasticsearchServer("elasticsearch:7.8.0", ImmutableMap.builder() + .put("elasticsearch.yml", loadResource("elasticsearch.yml")) + .put("users", loadResource("users")) + .put("users_roles", loadResource("users_roles")) + .put("roles.yml", loadResource("roles.yml")) + .buildOrThrow()); + + HostAndPort address = elasticsearch.getAddress(); + client = new RestHighLevelClient(RestClient.builder(new HttpHost(address.getHost(), address.getPort()))); + + DistributedQueryRunner runner = createElasticsearchQueryRunner( + elasticsearch.getAddress(), + ImmutableList.of(), + ImmutableMap.of(), + ImmutableMap.builder() + .put("elasticsearch.security", "PASSWORD") + .put("elasticsearch.auth.user", USER) + .put("elasticsearch.auth.password", PASSWORD) + .buildOrThrow(), + 3); + + assertions = new QueryAssertions(runner); + } + + @AfterAll + public final void destroy() + throws IOException + { + closeAll( + () -> assertions.close(), + () -> elasticsearch.stop(), + () -> client.close()); + + assertions = null; + elasticsearch = null; + client = null; + } + + @Test + public void test() + throws IOException + { + String json = new ObjectMapper().writeValueAsString(ImmutableMap.of("value", 42L)); + + client.getLowLevelClient() + .performRequest( + "POST", + "/test/_doc?refresh", + ImmutableMap.of(), + new NStringEntity(json, ContentType.APPLICATION_JSON), + new BasicHeader("Authorization", format("Basic %s", Base64.encodeAsString(format("%s:%s", USER, PASSWORD).getBytes(StandardCharsets.UTF_8))))); + + assertThat(assertions.query("SELECT * FROM test")) + .matches("VALUES BIGINT '42'"); + } + + private static String loadResource(String file) + throws IOException + { + return Resources.toString(getResource(file), UTF_8); + } +} diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestPasswordConfig.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestPasswordConfig.java new file mode 100644 index 0000000000000..d767ac265363d --- /dev/null +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestPasswordConfig.java @@ -0,0 +1,49 @@ +/* + * 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.elasticsearch; + +import com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.Test; + +import java.util.Map; + +import static io.airlift.configuration.testing.ConfigAssertions.assertFullMapping; +import static io.airlift.configuration.testing.ConfigAssertions.assertRecordedDefaults; +import static io.airlift.configuration.testing.ConfigAssertions.recordDefaults; + +public class TestPasswordConfig +{ + @Test + public void testDefaults() + { + assertRecordedDefaults(recordDefaults(PasswordConfig.class) + .setUser(null) + .setPassword(null)); + } + + @Test + public void testExplicitPropertyMappings() + { + Map properties = ImmutableMap.builder() + .put("elasticsearch.auth.user", "user") + .put("elasticsearch.auth.password", "password") + .buildOrThrow(); + + PasswordConfig expected = new PasswordConfig() + .setUser("user") + .setPassword("password"); + + assertFullMapping(properties, expected); + } +} diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/client/TestExtractAddress.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/client/TestExtractAddress.java new file mode 100644 index 0000000000000..34a982f53f164 --- /dev/null +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/client/TestExtractAddress.java @@ -0,0 +1,39 @@ +/* + * 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.elasticsearch.client; + +import org.junit.jupiter.api.Test; + +import java.util.Optional; + +import static io.trino.plugin.elasticsearch.client.ElasticsearchClient.extractAddress; +import static org.assertj.core.api.Assertions.assertThat; + +public class TestExtractAddress +{ + @Test + public void test() + { + assertThat(extractAddress("node/1.2.3.4:9200")).isEqualTo(Optional.of("node:9200")); + assertThat(extractAddress("1.2.3.4:9200")).isEqualTo(Optional.of("1.2.3.4:9200")); + assertThat(extractAddress("node/1.2.3.4:9200")).isEqualTo(Optional.of("node:9200")); + assertThat(extractAddress("node/[fe80::1]:9200")).isEqualTo(Optional.of("node:9200")); + assertThat(extractAddress("[fe80::1]:9200")).isEqualTo(Optional.of("[fe80::1]:9200")); + + assertThat(extractAddress("")).isEqualTo(Optional.empty()); + assertThat(extractAddress("node/1.2.3.4")).isEqualTo(Optional.empty()); + assertThat(extractAddress("node/1.2.3.4:xxxx")).isEqualTo(Optional.empty()); + assertThat(extractAddress("1.2.3.4:xxxx")).isEqualTo(Optional.empty()); + } +} diff --git a/plugin/trino-opensearch/src/test/resources/elasticsearch.yml b/plugin/trino-opensearch/src/test/resources/elasticsearch.yml new file mode 100644 index 0000000000000..eb6bb9c410c30 --- /dev/null +++ b/plugin/trino-opensearch/src/test/resources/elasticsearch.yml @@ -0,0 +1,4 @@ +cluster.name: "docker-cluster" +network.host: 0.0.0.0 + +xpack.security.enabled: true diff --git a/plugin/trino-opensearch/src/test/resources/roles.yml b/plugin/trino-opensearch/src/test/resources/roles.yml new file mode 100644 index 0000000000000..0b1a96ec2607d --- /dev/null +++ b/plugin/trino-opensearch/src/test/resources/roles.yml @@ -0,0 +1,6 @@ +admin: + cluster: + - all + indices: + - names: '*' + privileges: [ all ] diff --git a/plugin/trino-opensearch/src/test/resources/users b/plugin/trino-opensearch/src/test/resources/users new file mode 100644 index 0000000000000..0a5fe3f1cbd1b --- /dev/null +++ b/plugin/trino-opensearch/src/test/resources/users @@ -0,0 +1 @@ +elastic_user:$2a$10$tbO62EbOfqMezJDBDWlxbuvIleeYeNlw30F5OgWMXzi1R8aXqnVni diff --git a/plugin/trino-opensearch/src/test/resources/users_roles b/plugin/trino-opensearch/src/test/resources/users_roles new file mode 100644 index 0000000000000..33f5f17c8aeef --- /dev/null +++ b/plugin/trino-opensearch/src/test/resources/users_roles @@ -0,0 +1 @@ +admin:elastic_user diff --git a/pom.xml b/pom.xml index 43d1eec7f681a..e0e1574ff03d4 100644 --- a/pom.xml +++ b/pom.xml @@ -92,6 +92,7 @@ plugin/trino-mongodb plugin/trino-mysql plugin/trino-mysql-event-listener + plugin/trino-opensearch plugin/trino-oracle plugin/trino-password-authenticators plugin/trino-phoenix5 From b26935b9759ac0842baf4ae1c5e87deb00204167 Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Tue, 2 Jan 2024 11:14:50 +0100 Subject: [PATCH 291/350] Rename classes and drop Elasticsearch tests --- .../elasticsearch/AwsSecurityConfig.java | 12 +- .../plugin/elasticsearch/BuiltinColumns.java | 2 +- .../elasticsearch/CountQueryPageSource.java | 8 +- .../elasticsearch/NodesSystemTable.java | 12 +- ...andle.java => OpenSearchColumnHandle.java} | 6 +- ...earchConfig.java => OpenSearchConfig.java} | 102 ++++++++--------- ...onnector.java => OpenSearchConnector.java} | 18 +-- ...y.java => OpenSearchConnectorFactory.java} | 8 +- ...le.java => OpenSearchConnectorModule.java} | 26 ++--- ...rrorCode.java => OpenSearchErrorCode.java} | 4 +- ...hMetadata.java => OpenSearchMetadata.java} | 48 ++++---- ...java => OpenSearchPageSourceProvider.java} | 16 +-- ...earchPlugin.java => OpenSearchPlugin.java} | 8 +- ...ilder.java => OpenSearchQueryBuilder.java} | 10 +- ...csearchSplit.java => OpenSearchSplit.java} | 6 +- ...nager.java => OpenSearchSplitManager.java} | 18 +-- ...Handle.java => OpenSearchTableHandle.java} | 8 +- ....java => OpenSearchTransactionHandle.java} | 2 +- .../PassthroughQueryPageSource.java | 4 +- .../plugin/elasticsearch/PasswordConfig.java | 4 +- .../elasticsearch/ScanQueryPageSource.java | 34 +++--- .../client/BackpressureRestClient.java | 4 +- .../BackpressureRestHighLevelClient.java | 4 +- ...earchClient.java => OpenSearchClient.java} | 46 ++++---- ...ticsearchNode.java => OpenSearchNode.java} | 4 +- .../plugin/elasticsearch/ptf/RawQuery.java | 24 ++-- ....java => BaseOpenSearchConnectorTest.java} | 10 +- .../ElasticsearchExternalQueryRunner.java | 49 --------- .../ElasticsearchNginxProxy.java | 103 ------------------ ...earchLoader.java => OpenSearchLoader.java} | 4 +- ...Runner.java => OpenSearchQueryRunner.java} | 34 +++--- ...earchServer.java => OpenSearchServer.java} | 6 +- .../elasticsearch/TestAwsSecurityConfig.java | 10 +- .../TestElasticsearch6ConnectorTest.java | 63 ----------- .../TestElasticsearch7ConnectorTest.java | 38 ------- .../TestElasticsearchBackpressure.java | 86 --------------- ...hConfig.java => TestOpenSearchConfig.java} | 50 ++++----- ....java => TestOpenSearchConnectorTest.java} | 6 +- ...adata.java => TestOpenSearchMetadata.java} | 4 +- ...r.java => TestOpenSearchQueryBuilder.java} | 14 +-- .../TestPasswordAuthentication.java | 6 +- .../elasticsearch/TestPasswordConfig.java | 4 +- .../client/TestExtractAddress.java | 2 +- 43 files changed, 285 insertions(+), 642 deletions(-) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/{ElasticsearchColumnHandle.java => OpenSearchColumnHandle.java} (94%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/{ElasticsearchConfig.java => OpenSearchConfig.java} (68%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/{ElasticsearchConnector.java => OpenSearchConnector.java} (87%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/{ElasticsearchConnectorFactory.java => OpenSearchConnectorFactory.java} (92%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/{ElasticsearchConnectorModule.java => OpenSearchConnectorModule.java} (72%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/{ElasticsearchErrorCode.java => OpenSearchErrorCode.java} (93%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/{ElasticsearchMetadata.java => OpenSearchMetadata.java} (93%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/{ElasticsearchPageSourceProvider.java => OpenSearchPageSourceProvider.java} (81%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/{ElasticsearchPlugin.java => OpenSearchPlugin.java} (86%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/{ElasticsearchQueryBuilder.java => OpenSearchQueryBuilder.java} (94%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/{ElasticsearchSplit.java => OpenSearchSplit.java} (94%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/{ElasticsearchSplitManager.java => OpenSearchSplitManager.java} (71%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/{ElasticsearchTableHandle.java => OpenSearchTableHandle.java} (95%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/{ElasticsearchTransactionHandle.java => OpenSearchTransactionHandle.java} (94%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/{ElasticsearchClient.java => OpenSearchClient.java} (94%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/{ElasticsearchNode.java => OpenSearchNode.java} (92%) rename plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/{BaseElasticsearchConnectorTest.java => BaseOpenSearchConnectorTest.java} (99%) delete mode 100644 plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchExternalQueryRunner.java delete mode 100644 plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchNginxProxy.java rename plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/{ElasticsearchLoader.java => OpenSearchLoader.java} (98%) rename plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/{ElasticsearchQueryRunner.java => OpenSearchQueryRunner.java} (81%) rename plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/{ElasticsearchServer.java => OpenSearchServer.java} (92%) delete mode 100644 plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearch6ConnectorTest.java delete mode 100644 plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearch7ConnectorTest.java delete mode 100644 plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchBackpressure.java rename plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/{TestElasticsearchConfig.java => TestOpenSearchConfig.java} (70%) rename plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/{TestElasticsearchOpenSearchConnectorTest.java => TestOpenSearchConnectorTest.java} (87%) rename plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/{TestElasticsearchMetadata.java => TestOpenSearchMetadata.java} (93%) rename plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/{TestElasticsearchQueryBuilder.java => TestOpenSearchQueryBuilder.java} (86%) diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/AwsSecurityConfig.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/AwsSecurityConfig.java index 8baf2b7b54853..4fe4abb0b2afd 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/AwsSecurityConfig.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/AwsSecurityConfig.java @@ -16,12 +16,10 @@ import io.airlift.configuration.Config; import io.airlift.configuration.ConfigDescription; import io.airlift.configuration.ConfigSecuritySensitive; -import io.airlift.configuration.DefunctConfig; import jakarta.validation.constraints.NotNull; import java.util.Optional; -@DefunctConfig("elasticsearch.aws.use-instance-credentials") public class AwsSecurityConfig { private String accessKey; @@ -36,7 +34,7 @@ public Optional getAccessKey() return Optional.ofNullable(accessKey); } - @Config("elasticsearch.aws.access-key") + @Config("opensearch.aws.access-key") public AwsSecurityConfig setAccessKey(String key) { this.accessKey = key; @@ -49,7 +47,7 @@ public Optional getSecretKey() return Optional.ofNullable(secretKey); } - @Config("elasticsearch.aws.secret-key") + @Config("opensearch.aws.secret-key") @ConfigSecuritySensitive public AwsSecurityConfig setSecretKey(String key) { @@ -62,7 +60,7 @@ public String getRegion() return region; } - @Config("elasticsearch.aws.region") + @Config("opensearch.aws.region") public AwsSecurityConfig setRegion(String region) { this.region = region; @@ -75,7 +73,7 @@ public Optional getIamRole() return Optional.ofNullable(iamRole); } - @Config("elasticsearch.aws.iam-role") + @Config("opensearch.aws.iam-role") @ConfigDescription("Optional AWS IAM role to assume for authenticating. If set, this role will be used to get credentials to sign requests to ES.") public AwsSecurityConfig setIamRole(String iamRole) { @@ -89,7 +87,7 @@ public Optional getExternalId() return Optional.ofNullable(externalId); } - @Config("elasticsearch.aws.external-id") + @Config("opensearch.aws.external-id") @ConfigDescription("Optional external id to pass to AWS STS while assuming a role") public AwsSecurityConfig setExternalId(String externalId) { diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/BuiltinColumns.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/BuiltinColumns.java index 0af3e0fe9b950..621d00beeccaa 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/BuiltinColumns.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/BuiltinColumns.java @@ -82,7 +82,7 @@ public ColumnMetadata getMetadata() public ColumnHandle getColumnHandle() { - return new ElasticsearchColumnHandle( + return new OpenSearchColumnHandle( name, type, decoderDescriptor, diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/CountQueryPageSource.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/CountQueryPageSource.java index 4b920b218e06b..22f8e72dd40fc 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/CountQueryPageSource.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/CountQueryPageSource.java @@ -13,11 +13,11 @@ */ package io.trino.plugin.elasticsearch; -import io.trino.plugin.elasticsearch.client.ElasticsearchClient; +import io.trino.plugin.elasticsearch.client.OpenSearchClient; import io.trino.spi.Page; import io.trino.spi.connector.ConnectorPageSource; -import static io.trino.plugin.elasticsearch.ElasticsearchQueryBuilder.buildSearchQuery; +import static io.trino.plugin.elasticsearch.OpenSearchQueryBuilder.buildSearchQuery; import static java.lang.Math.toIntExact; import static java.util.Objects.requireNonNull; @@ -33,7 +33,7 @@ class CountQueryPageSource private final long readTimeNanos; private long remaining; - public CountQueryPageSource(ElasticsearchClient client, ElasticsearchTableHandle table, ElasticsearchSplit split) + public CountQueryPageSource(OpenSearchClient client, OpenSearchTableHandle table, OpenSearchSplit split) { requireNonNull(client, "client is null"); requireNonNull(table, "table is null"); @@ -43,7 +43,7 @@ public CountQueryPageSource(ElasticsearchClient client, ElasticsearchTableHandle long count = client.count( split.getIndex(), split.getShard(), - buildSearchQuery(table.getConstraint().transformKeys(ElasticsearchColumnHandle.class::cast), table.getQuery(), table.getRegexes())); + buildSearchQuery(table.getConstraint().transformKeys(OpenSearchColumnHandle.class::cast), table.getQuery(), table.getRegexes())); readTimeNanos = System.nanoTime() - start; if (table.getLimit().isPresent()) { diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/NodesSystemTable.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/NodesSystemTable.java index 84385dbedbf30..aaedda268f601 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/NodesSystemTable.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/NodesSystemTable.java @@ -15,8 +15,8 @@ import com.google.common.collect.ImmutableList; import com.google.inject.Inject; -import io.trino.plugin.elasticsearch.client.ElasticsearchClient; -import io.trino.plugin.elasticsearch.client.ElasticsearchNode; +import io.trino.plugin.elasticsearch.client.OpenSearchClient; +import io.trino.plugin.elasticsearch.client.OpenSearchNode; import io.trino.spi.Node; import io.trino.spi.NodeManager; import io.trino.spi.Page; @@ -49,11 +49,11 @@ public class NodesSystemTable .add(new ColumnMetadata("elasticsearch_node_address", createUnboundedVarcharType())) .build()); - private final ElasticsearchClient client; + private final OpenSearchClient client; private final Node currentNode; @Inject - public NodesSystemTable(NodeManager nodeManager, ElasticsearchClient client) + public NodesSystemTable(NodeManager nodeManager, OpenSearchClient client) { requireNonNull(nodeManager, "nodeManager is null"); @@ -76,14 +76,14 @@ public ConnectorTableMetadata getTableMetadata() @Override public ConnectorPageSource pageSource(ConnectorTransactionHandle transaction, ConnectorSession session, TupleDomain constraint) { - Set nodes = client.getNodes(); + Set nodes = client.getNodes(); BlockBuilder nodeId = VARCHAR.createBlockBuilder(null, nodes.size()); BlockBuilder trinoAddress = VARCHAR.createBlockBuilder(null, nodes.size()); BlockBuilder elasticsearchNodeId = VARCHAR.createBlockBuilder(null, nodes.size()); BlockBuilder elasticsearchAddress = VARCHAR.createBlockBuilder(null, nodes.size()); - for (ElasticsearchNode node : nodes) { + for (OpenSearchNode node : nodes) { VARCHAR.writeString(nodeId, currentNode.getNodeIdentifier()); VARCHAR.writeString(trinoAddress, currentNode.getHostAndPort().toString()); VARCHAR.writeString(elasticsearchNodeId, node.getId()); diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchColumnHandle.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchColumnHandle.java similarity index 94% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchColumnHandle.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchColumnHandle.java index 8b22b3cb1bdf3..7c19d17b3378f 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchColumnHandle.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchColumnHandle.java @@ -22,7 +22,7 @@ import static java.util.Objects.requireNonNull; -public final class ElasticsearchColumnHandle +public final class OpenSearchColumnHandle implements ColumnHandle { private final String name; @@ -31,7 +31,7 @@ public final class ElasticsearchColumnHandle private final boolean supportsPredicates; @JsonCreator - public ElasticsearchColumnHandle( + public OpenSearchColumnHandle( @JsonProperty("name") String name, @JsonProperty("type") Type type, @JsonProperty("decoderDescriptor") DecoderDescriptor decoderDescriptor, @@ -83,7 +83,7 @@ public boolean equals(Object obj) return false; } - ElasticsearchColumnHandle other = (ElasticsearchColumnHandle) obj; + OpenSearchColumnHandle other = (OpenSearchColumnHandle) obj; return this.supportsPredicates == other.supportsPredicates && Objects.equals(this.getName(), other.getName()) && Objects.equals(this.getType(), other.getType()) && diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchConfig.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchConfig.java similarity index 68% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchConfig.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchConfig.java index 339dd233e6af5..10b3c90b0ed07 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchConfig.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchConfig.java @@ -16,7 +16,6 @@ import io.airlift.configuration.Config; import io.airlift.configuration.ConfigDescription; import io.airlift.configuration.ConfigSecuritySensitive; -import io.airlift.configuration.DefunctConfig; import io.airlift.configuration.validation.FileExists; import io.airlift.units.Duration; import io.airlift.units.MinDuration; @@ -31,22 +30,7 @@ import static java.util.concurrent.TimeUnit.MINUTES; import static java.util.concurrent.TimeUnit.SECONDS; -@DefunctConfig({ - "elasticsearch.max-hits", - "elasticsearch.cluster-name", - "searchguard.ssl.certificate-format", - "searchguard.ssl.pemcert-filepath", - "searchguard.ssl.pemkey-filepath", - "searchguard.ssl.pemkey-password", - "searchguard.ssl.pemtrustedcas-filepath", - "searchguard.ssl.keystore-filepath", - "searchguard.ssl.keystore-password", - "searchguard.ssl.truststore-filepath", - "searchguard.ssl.truststore-password", - "elasticsearch.table-description-directory", - "elasticsearch.max-request-retries", - "elasticsearch.max-request-retry-time"}) -public class ElasticsearchConfig +public class OpenSearchConfig { public enum Security { @@ -84,8 +68,8 @@ public List getHosts() return hosts; } - @Config("elasticsearch.host") - public ElasticsearchConfig setHosts(List hosts) + @Config("opensearch.host") + public OpenSearchConfig setHosts(List hosts) { this.hosts = hosts; return this; @@ -96,8 +80,8 @@ public int getPort() return port; } - @Config("elasticsearch.port") - public ElasticsearchConfig setPort(int port) + @Config("opensearch.port") + public OpenSearchConfig setPort(int port) { this.port = port; return this; @@ -109,9 +93,9 @@ public String getDefaultSchema() return defaultSchema; } - @Config("elasticsearch.default-schema-name") + @Config("opensearch.default-schema-name") @ConfigDescription("Default schema name to use") - public ElasticsearchConfig setDefaultSchema(String defaultSchema) + public OpenSearchConfig setDefaultSchema(String defaultSchema) { this.defaultSchema = defaultSchema; return this; @@ -124,9 +108,9 @@ public int getScrollSize() return scrollSize; } - @Config("elasticsearch.scroll-size") + @Config("opensearch.scroll-size") @ConfigDescription("Scroll batch size") - public ElasticsearchConfig setScrollSize(int scrollSize) + public OpenSearchConfig setScrollSize(int scrollSize) { this.scrollSize = scrollSize; return this; @@ -138,9 +122,9 @@ public Duration getScrollTimeout() return scrollTimeout; } - @Config("elasticsearch.scroll-timeout") + @Config("opensearch.scroll-timeout") @ConfigDescription("Scroll timeout") - public ElasticsearchConfig setScrollTimeout(Duration scrollTimeout) + public OpenSearchConfig setScrollTimeout(Duration scrollTimeout) { this.scrollTimeout = scrollTimeout; return this; @@ -152,9 +136,9 @@ public Duration getRequestTimeout() return requestTimeout; } - @Config("elasticsearch.request-timeout") + @Config("opensearch.request-timeout") @ConfigDescription("Elasticsearch request timeout") - public ElasticsearchConfig setRequestTimeout(Duration requestTimeout) + public OpenSearchConfig setRequestTimeout(Duration requestTimeout) { this.requestTimeout = requestTimeout; return this; @@ -166,9 +150,9 @@ public Duration getConnectTimeout() return connectTimeout; } - @Config("elasticsearch.connect-timeout") + @Config("opensearch.connect-timeout") @ConfigDescription("Elasticsearch connect timeout") - public ElasticsearchConfig setConnectTimeout(Duration timeout) + public OpenSearchConfig setConnectTimeout(Duration timeout) { this.connectTimeout = timeout; return this; @@ -180,9 +164,9 @@ public Duration getBackoffInitDelay() return backoffInitDelay; } - @Config("elasticsearch.backoff-init-delay") + @Config("opensearch.backoff-init-delay") @ConfigDescription("Initial delay to wait between backpressure retries") - public ElasticsearchConfig setBackoffInitDelay(Duration backoffInitDelay) + public OpenSearchConfig setBackoffInitDelay(Duration backoffInitDelay) { this.backoffInitDelay = backoffInitDelay; return this; @@ -194,9 +178,9 @@ public Duration getBackoffMaxDelay() return backoffMaxDelay; } - @Config("elasticsearch.backoff-max-delay") + @Config("opensearch.backoff-max-delay") @ConfigDescription("Maximum delay to wait between backpressure retries") - public ElasticsearchConfig setBackoffMaxDelay(Duration backoffMaxDelay) + public OpenSearchConfig setBackoffMaxDelay(Duration backoffMaxDelay) { this.backoffMaxDelay = backoffMaxDelay; return this; @@ -208,9 +192,9 @@ public Duration getMaxRetryTime() return maxRetryTime; } - @Config("elasticsearch.max-retry-time") + @Config("opensearch.max-retry-time") @ConfigDescription("Maximum timeout in case of multiple retries") - public ElasticsearchConfig setMaxRetryTime(Duration maxRetryTime) + public OpenSearchConfig setMaxRetryTime(Duration maxRetryTime) { this.maxRetryTime = maxRetryTime; return this; @@ -223,17 +207,17 @@ public Duration getNodeRefreshInterval() return nodeRefreshInterval; } - @Config("elasticsearch.node-refresh-interval") + @Config("opensearch.node-refresh-interval") @ConfigDescription("How often to refresh the list of available Elasticsearch nodes") - public ElasticsearchConfig setNodeRefreshInterval(Duration nodeRefreshInterval) + public OpenSearchConfig setNodeRefreshInterval(Duration nodeRefreshInterval) { this.nodeRefreshInterval = nodeRefreshInterval; return this; } - @Config("elasticsearch.max-http-connections") + @Config("opensearch.max-http-connections") @ConfigDescription("Maximum number of persistent HTTP connections to Elasticsearch") - public ElasticsearchConfig setMaxHttpConnections(int size) + public OpenSearchConfig setMaxHttpConnections(int size) { this.maxHttpConnections = size; return this; @@ -245,9 +229,9 @@ public int getMaxHttpConnections() return maxHttpConnections; } - @Config("elasticsearch.http-thread-count") + @Config("opensearch.http-thread-count") @ConfigDescription("Number of threads handling HTTP connections to Elasticsearch") - public ElasticsearchConfig setHttpThreadCount(int count) + public OpenSearchConfig setHttpThreadCount(int count) { this.httpThreadCount = count; return this; @@ -264,8 +248,8 @@ public boolean isTlsEnabled() return tlsEnabled; } - @Config("elasticsearch.tls.enabled") - public ElasticsearchConfig setTlsEnabled(boolean tlsEnabled) + @Config("opensearch.tls.enabled") + public OpenSearchConfig setTlsEnabled(boolean tlsEnabled) { this.tlsEnabled = tlsEnabled; return this; @@ -276,8 +260,8 @@ public ElasticsearchConfig setTlsEnabled(boolean tlsEnabled) return Optional.ofNullable(keystorePath); } - @Config("elasticsearch.tls.keystore-path") - public ElasticsearchConfig setKeystorePath(File path) + @Config("opensearch.tls.keystore-path") + public OpenSearchConfig setKeystorePath(File path) { this.keystorePath = path; return this; @@ -288,9 +272,9 @@ public Optional getKeystorePassword() return Optional.ofNullable(keystorePassword); } - @Config("elasticsearch.tls.keystore-password") + @Config("opensearch.tls.keystore-password") @ConfigSecuritySensitive - public ElasticsearchConfig setKeystorePassword(String password) + public OpenSearchConfig setKeystorePassword(String password) { this.keystorePassword = password; return this; @@ -301,8 +285,8 @@ public ElasticsearchConfig setKeystorePassword(String password) return Optional.ofNullable(trustStorePath); } - @Config("elasticsearch.tls.truststore-path") - public ElasticsearchConfig setTrustStorePath(File path) + @Config("opensearch.tls.truststore-path") + public OpenSearchConfig setTrustStorePath(File path) { this.trustStorePath = path; return this; @@ -313,9 +297,9 @@ public Optional getTruststorePassword() return Optional.ofNullable(truststorePassword); } - @Config("elasticsearch.tls.truststore-password") + @Config("opensearch.tls.truststore-password") @ConfigSecuritySensitive - public ElasticsearchConfig setTruststorePassword(String password) + public OpenSearchConfig setTruststorePassword(String password) { this.truststorePassword = password; return this; @@ -326,8 +310,8 @@ public boolean isVerifyHostnames() return verifyHostnames; } - @Config("elasticsearch.tls.verify-hostnames") - public ElasticsearchConfig setVerifyHostnames(boolean verify) + @Config("opensearch.tls.verify-hostnames") + public OpenSearchConfig setVerifyHostnames(boolean verify) { this.verifyHostnames = verify; return this; @@ -338,8 +322,8 @@ public boolean isIgnorePublishAddress() return ignorePublishAddress; } - @Config("elasticsearch.ignore-publish-address") - public ElasticsearchConfig setIgnorePublishAddress(boolean ignorePublishAddress) + @Config("opensearch.ignore-publish-address") + public OpenSearchConfig setIgnorePublishAddress(boolean ignorePublishAddress) { this.ignorePublishAddress = ignorePublishAddress; return this; @@ -351,8 +335,8 @@ public Optional getSecurity() return Optional.ofNullable(security); } - @Config("elasticsearch.security") - public ElasticsearchConfig setSecurity(Security security) + @Config("opensearch.security") + public OpenSearchConfig setSecurity(Security security) { this.security = security; return this; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchConnector.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchConnector.java similarity index 87% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchConnector.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchConnector.java index 6014954e6acf7..513a5ca6db0ae 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchConnector.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchConnector.java @@ -32,22 +32,22 @@ import static io.trino.spi.transaction.IsolationLevel.checkConnectorSupports; import static java.util.Objects.requireNonNull; -public class ElasticsearchConnector +public class OpenSearchConnector implements Connector { private final LifeCycleManager lifeCycleManager; - private final ElasticsearchMetadata metadata; - private final ElasticsearchSplitManager splitManager; - private final ElasticsearchPageSourceProvider pageSourceProvider; + private final OpenSearchMetadata metadata; + private final OpenSearchSplitManager splitManager; + private final OpenSearchPageSourceProvider pageSourceProvider; private final NodesSystemTable nodesSystemTable; private final Set connectorTableFunctions; @Inject - public ElasticsearchConnector( + public OpenSearchConnector( LifeCycleManager lifeCycleManager, - ElasticsearchMetadata metadata, - ElasticsearchSplitManager splitManager, - ElasticsearchPageSourceProvider pageSourceProvider, + OpenSearchMetadata metadata, + OpenSearchSplitManager splitManager, + OpenSearchPageSourceProvider pageSourceProvider, NodesSystemTable nodesSystemTable, Set connectorTableFunctions) { @@ -63,7 +63,7 @@ public ElasticsearchConnector( public ConnectorTransactionHandle beginTransaction(IsolationLevel isolationLevel, boolean readOnly, boolean autoCommit) { checkConnectorSupports(READ_COMMITTED, isolationLevel); - return ElasticsearchTransactionHandle.INSTANCE; + return OpenSearchTransactionHandle.INSTANCE; } @Override diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchConnectorFactory.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchConnectorFactory.java similarity index 92% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchConnectorFactory.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchConnectorFactory.java index fcf3383362701..d535c9d753ea3 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchConnectorFactory.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchConnectorFactory.java @@ -31,10 +31,10 @@ import static io.trino.plugin.base.Versions.checkStrictSpiVersionMatch; import static java.util.Objects.requireNonNull; -public class ElasticsearchConnectorFactory +public class OpenSearchConnectorFactory implements ConnectorFactory { - ElasticsearchConnectorFactory() {} + OpenSearchConnectorFactory() {} @Override public String getName() @@ -55,7 +55,7 @@ public Connector create(String catalogName, Map config, Connecto new ConnectorObjectNameGeneratorModule("io.trino.plugin.elasticsearch", "trino.plugin.elasticsearch"), new JsonModule(), new TypeDeserializerModule(context.getTypeManager()), - new ElasticsearchConnectorModule(), + new OpenSearchConnectorModule(), binder -> { binder.bind(NodeManager.class).toInstance(context.getNodeManager()); binder.bind(CatalogName.class).toInstance(new CatalogName(catalogName)); @@ -66,6 +66,6 @@ public Connector create(String catalogName, Map config, Connecto .setRequiredConfigurationProperties(config) .initialize(); - return injector.getInstance(ElasticsearchConnector.class); + return injector.getInstance(OpenSearchConnector.class); } } diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchConnectorModule.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchConnectorModule.java similarity index 72% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchConnectorModule.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchConnectorModule.java index 2cda1656af15b..25f6bcc0c2593 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchConnectorModule.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchConnectorModule.java @@ -16,7 +16,7 @@ import com.google.inject.Binder; import com.google.inject.Scopes; import io.airlift.configuration.AbstractConfigurationAwareModule; -import io.trino.plugin.elasticsearch.client.ElasticsearchClient; +import io.trino.plugin.elasticsearch.client.OpenSearchClient; import io.trino.plugin.elasticsearch.ptf.RawQuery; import io.trino.spi.function.table.ConnectorTableFunction; @@ -24,27 +24,27 @@ import static com.google.inject.multibindings.OptionalBinder.newOptionalBinder; import static io.airlift.configuration.ConditionalModule.conditionalModule; import static io.airlift.configuration.ConfigBinder.configBinder; -import static io.trino.plugin.elasticsearch.ElasticsearchConfig.Security.AWS; -import static io.trino.plugin.elasticsearch.ElasticsearchConfig.Security.PASSWORD; +import static io.trino.plugin.elasticsearch.OpenSearchConfig.Security.AWS; +import static io.trino.plugin.elasticsearch.OpenSearchConfig.Security.PASSWORD; import static java.util.function.Predicate.isEqual; import static org.weakref.jmx.guice.ExportBinder.newExporter; -public class ElasticsearchConnectorModule +public class OpenSearchConnectorModule extends AbstractConfigurationAwareModule { @Override protected void setup(Binder binder) { - binder.bind(ElasticsearchConnector.class).in(Scopes.SINGLETON); - binder.bind(ElasticsearchMetadata.class).in(Scopes.SINGLETON); - binder.bind(ElasticsearchSplitManager.class).in(Scopes.SINGLETON); - binder.bind(ElasticsearchPageSourceProvider.class).in(Scopes.SINGLETON); - binder.bind(ElasticsearchClient.class).in(Scopes.SINGLETON); + binder.bind(OpenSearchConnector.class).in(Scopes.SINGLETON); + binder.bind(OpenSearchMetadata.class).in(Scopes.SINGLETON); + binder.bind(OpenSearchSplitManager.class).in(Scopes.SINGLETON); + binder.bind(OpenSearchPageSourceProvider.class).in(Scopes.SINGLETON); + binder.bind(OpenSearchClient.class).in(Scopes.SINGLETON); binder.bind(NodesSystemTable.class).in(Scopes.SINGLETON); - newExporter(binder).export(ElasticsearchClient.class).withGeneratedName(); + newExporter(binder).export(OpenSearchClient.class).withGeneratedName(); - configBinder(binder).bindConfig(ElasticsearchConfig.class); + configBinder(binder).bindConfig(OpenSearchConfig.class); newOptionalBinder(binder, AwsSecurityConfig.class); newOptionalBinder(binder, PasswordConfig.class); @@ -52,14 +52,14 @@ protected void setup(Binder binder) newSetBinder(binder, ConnectorTableFunction.class).addBinding().toProvider(RawQuery.class).in(Scopes.SINGLETON); install(conditionalModule( - ElasticsearchConfig.class, + OpenSearchConfig.class, config -> config.getSecurity() .filter(isEqual(AWS)) .isPresent(), conditionalBinder -> configBinder(conditionalBinder).bindConfig(AwsSecurityConfig.class))); install(conditionalModule( - ElasticsearchConfig.class, + OpenSearchConfig.class, config -> config.getSecurity() .filter(isEqual(PASSWORD)) .isPresent(), diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchErrorCode.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchErrorCode.java similarity index 93% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchErrorCode.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchErrorCode.java index 59af305ae31dd..abe224192be35 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchErrorCode.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchErrorCode.java @@ -20,7 +20,7 @@ import static io.trino.spi.ErrorType.EXTERNAL; import static io.trino.spi.ErrorType.USER_ERROR; -public enum ElasticsearchErrorCode +public enum OpenSearchErrorCode implements ErrorCodeSupplier { ELASTICSEARCH_CONNECTION_ERROR(0, EXTERNAL), @@ -31,7 +31,7 @@ public enum ElasticsearchErrorCode private final ErrorCode errorCode; - ElasticsearchErrorCode(int code, ErrorType type) + OpenSearchErrorCode(int code, ErrorType type) { errorCode = new ErrorCode(code + 0x0503_0000, name(), type); } diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchMetadata.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchMetadata.java similarity index 93% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchMetadata.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchMetadata.java index ef06f4372aaef..ff4b20c2de562 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchMetadata.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchMetadata.java @@ -19,12 +19,12 @@ import com.google.inject.Inject; import io.airlift.slice.Slice; import io.trino.plugin.base.expression.ConnectorExpressions; -import io.trino.plugin.elasticsearch.client.ElasticsearchClient; import io.trino.plugin.elasticsearch.client.IndexMetadata; import io.trino.plugin.elasticsearch.client.IndexMetadata.DateTimeType; import io.trino.plugin.elasticsearch.client.IndexMetadata.ObjectType; import io.trino.plugin.elasticsearch.client.IndexMetadata.PrimitiveType; import io.trino.plugin.elasticsearch.client.IndexMetadata.ScaledFloatType; +import io.trino.plugin.elasticsearch.client.OpenSearchClient; import io.trino.plugin.elasticsearch.decoders.ArrayDecoder; import io.trino.plugin.elasticsearch.decoders.BigintDecoder; import io.trino.plugin.elasticsearch.decoders.BooleanDecoder; @@ -87,8 +87,8 @@ import static com.google.common.collect.Iterators.singletonIterator; import static io.airlift.slice.SliceUtf8.getCodePointAt; import static io.airlift.slice.SliceUtf8.lengthOfCodePoint; -import static io.trino.plugin.elasticsearch.ElasticsearchTableHandle.Type.QUERY; -import static io.trino.plugin.elasticsearch.ElasticsearchTableHandle.Type.SCAN; +import static io.trino.plugin.elasticsearch.OpenSearchTableHandle.Type.QUERY; +import static io.trino.plugin.elasticsearch.OpenSearchTableHandle.Type.SCAN; import static io.trino.spi.StandardErrorCode.INVALID_FUNCTION_ARGUMENT; import static io.trino.spi.expression.StandardFunctions.LIKE_FUNCTION_NAME; import static io.trino.spi.type.BigintType.BIGINT; @@ -106,7 +106,7 @@ import static java.util.Locale.ENGLISH; import static java.util.Objects.requireNonNull; -public class ElasticsearchMetadata +public class OpenSearchMetadata implements ConnectorMetadata { private static final String PASSTHROUGH_QUERY_RESULT_COLUMN_NAME = "result"; @@ -119,7 +119,7 @@ public class ElasticsearchMetadata private static final Map PASSTHROUGH_QUERY_COLUMNS = ImmutableMap.of( PASSTHROUGH_QUERY_RESULT_COLUMN_NAME, - new ElasticsearchColumnHandle( + new OpenSearchColumnHandle( PASSTHROUGH_QUERY_RESULT_COLUMN_NAME, VARCHAR, new VarcharDecoder.Descriptor(PASSTHROUGH_QUERY_RESULT_COLUMN_NAME), @@ -131,11 +131,11 @@ public class ElasticsearchMetadata .collect(toImmutableSet()); private final Type ipAddressType; - private final ElasticsearchClient client; + private final OpenSearchClient client; private final String schemaName; @Inject - public ElasticsearchMetadata(TypeManager typeManager, ElasticsearchClient client, ElasticsearchConfig config) + public OpenSearchMetadata(TypeManager typeManager, OpenSearchClient client, OpenSearchConfig config) { this.ipAddressType = typeManager.getType(new TypeSignature(StandardTypes.IPADDRESS)); this.client = requireNonNull(client, "client is null"); @@ -149,7 +149,7 @@ public List listSchemaNames(ConnectorSession session) } @Override - public ElasticsearchTableHandle getTableHandle(ConnectorSession session, SchemaTableName tableName) + public OpenSearchTableHandle getTableHandle(ConnectorSession session, SchemaTableName tableName) { requireNonNull(tableName, "tableName is null"); @@ -162,7 +162,7 @@ public ElasticsearchTableHandle getTableHandle(ConnectorSession session, SchemaT } if (client.indexExists(table) && !client.getIndexMetadata(table).getSchema().getFields().isEmpty()) { - return new ElasticsearchTableHandle(SCAN, schemaName, table, query); + return new OpenSearchTableHandle(SCAN, schemaName, table, query); } } @@ -172,7 +172,7 @@ public ElasticsearchTableHandle getTableHandle(ConnectorSession session, SchemaT @Override public ConnectorTableMetadata getTableMetadata(ConnectorSession session, ConnectorTableHandle table) { - ElasticsearchTableHandle handle = (ElasticsearchTableHandle) table; + OpenSearchTableHandle handle = (OpenSearchTableHandle) table; if (isPassthroughQuery(handle)) { return new ConnectorTableMetadata( @@ -190,7 +190,7 @@ private ConnectorTableMetadata getTableMetadata(String schemaName, String tableN private InternalTableMetadata makeInternalTableMetadata(ConnectorTableHandle table) { - ElasticsearchTableHandle handle = (ElasticsearchTableHandle) table; + OpenSearchTableHandle handle = (OpenSearchTableHandle) table; return makeInternalTableMetadata(handle.getSchema(), handle.getIndex()); } @@ -239,7 +239,7 @@ private Map makeColumnHandles(List fi for (IndexMetadata.Field field : fields) { TypeAndDecoder converted = toTrino(field); - result.put(field.getName(), new ElasticsearchColumnHandle( + result.put(field.getName(), new OpenSearchColumnHandle( field.getName(), converted.getType(), converted.getDecoderDescriptor(), @@ -393,7 +393,7 @@ public List listTables(ConnectorSession session, Optional getColumnHandles(ConnectorSession session, ConnectorTableHandle tableHandle) { - ElasticsearchTableHandle table = (ElasticsearchTableHandle) tableHandle; + OpenSearchTableHandle table = (OpenSearchTableHandle) tableHandle; if (isPassthroughQuery(table)) { return PASSTHROUGH_QUERY_COLUMNS; @@ -406,8 +406,8 @@ public Map getColumnHandles(ConnectorSession session, Conn @Override public ColumnMetadata getColumnMetadata(ConnectorSession session, ConnectorTableHandle tableHandle, ColumnHandle columnHandle) { - ElasticsearchTableHandle table = (ElasticsearchTableHandle) tableHandle; - ElasticsearchColumnHandle column = (ElasticsearchColumnHandle) columnHandle; + OpenSearchTableHandle table = (OpenSearchTableHandle) tableHandle; + OpenSearchColumnHandle column = (OpenSearchColumnHandle) columnHandle; if (isPassthroughQuery(table)) { if (column.getName().equals(PASSTHROUGH_QUERY_RESULT_COLUMN_METADATA.getName())) { @@ -452,7 +452,7 @@ public Iterator streamTableColumns(ConnectorSession sessio @Override public ConnectorTableProperties getTableProperties(ConnectorSession session, ConnectorTableHandle table) { - ElasticsearchTableHandle handle = (ElasticsearchTableHandle) table; + OpenSearchTableHandle handle = (OpenSearchTableHandle) table; return new ConnectorTableProperties( handle.getConstraint(), @@ -464,7 +464,7 @@ public ConnectorTableProperties getTableProperties(ConnectorSession session, Con @Override public Optional> applyLimit(ConnectorSession session, ConnectorTableHandle table, long limit) { - ElasticsearchTableHandle handle = (ElasticsearchTableHandle) table; + OpenSearchTableHandle handle = (OpenSearchTableHandle) table; if (isPassthroughQuery(handle)) { // limit pushdown currently not supported passthrough query @@ -475,7 +475,7 @@ public Optional> applyLimit(Connect return Optional.empty(); } - handle = new ElasticsearchTableHandle( + handle = new OpenSearchTableHandle( handle.getType(), handle.getSchema(), handle.getIndex(), @@ -490,7 +490,7 @@ public Optional> applyLimit(Connect @Override public Optional> applyFilter(ConnectorSession session, ConnectorTableHandle table, Constraint constraint) { - ElasticsearchTableHandle handle = (ElasticsearchTableHandle) table; + OpenSearchTableHandle handle = (OpenSearchTableHandle) table; if (isPassthroughQuery(handle)) { // filter pushdown currently not supported for passthrough query @@ -501,7 +501,7 @@ public Optional> applyFilter(C Map unsupported = new HashMap<>(); Map domains = constraint.getSummary().getDomains().orElseThrow(() -> new IllegalArgumentException("constraint summary is NONE")); for (Map.Entry entry : domains.entrySet()) { - ElasticsearchColumnHandle column = (ElasticsearchColumnHandle) entry.getKey(); + OpenSearchColumnHandle column = (OpenSearchColumnHandle) entry.getKey(); if (column.isSupportsPredicates()) { supported.put(column, entry.getValue()); @@ -523,7 +523,7 @@ public Optional> applyFilter(C if (isSupportedLikeCall(call)) { List arguments = call.getArguments(); String variableName = ((Variable) arguments.get(0)).getName(); - ElasticsearchColumnHandle column = (ElasticsearchColumnHandle) constraint.getAssignments().get(variableName); + OpenSearchColumnHandle column = (OpenSearchColumnHandle) constraint.getAssignments().get(variableName); verifyNotNull(column, "No assignment for %s", variableName); String columnName = column.getName(); Object pattern = ((Constant) arguments.get(1)).getValue(); @@ -551,7 +551,7 @@ public Optional> applyFilter(C return Optional.empty(); } - handle = new ElasticsearchTableHandle( + handle = new OpenSearchTableHandle( handle.getType(), handle.getSchema(), handle.getIndex(), @@ -587,7 +587,7 @@ protected static boolean isSupportedLikeCall(Call call) protected static String likeToRegexp(Slice pattern, Optional escape) { - Optional escapeChar = escape.map(ElasticsearchMetadata::getEscapeChar); + Optional escapeChar = escape.map(OpenSearchMetadata::getEscapeChar); StringBuilder regex = new StringBuilder(); boolean escaped = false; int position = 0; @@ -643,7 +643,7 @@ private static char getEscapeChar(Slice escape) throw new TrinoException(INVALID_FUNCTION_ARGUMENT, "Escape string must be a single character"); } - private static boolean isPassthroughQuery(ElasticsearchTableHandle table) + private static boolean isPassthroughQuery(OpenSearchTableHandle table) { return table.getType().equals(QUERY); } diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchPageSourceProvider.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchPageSourceProvider.java similarity index 81% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchPageSourceProvider.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchPageSourceProvider.java index 4744121748624..dbce38ee79000 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchPageSourceProvider.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchPageSourceProvider.java @@ -14,7 +14,7 @@ package io.trino.plugin.elasticsearch; import com.google.inject.Inject; -import io.trino.plugin.elasticsearch.client.ElasticsearchClient; +import io.trino.plugin.elasticsearch.client.OpenSearchClient; import io.trino.spi.connector.ColumnHandle; import io.trino.spi.connector.ConnectorPageSource; import io.trino.spi.connector.ConnectorPageSourceProvider; @@ -28,17 +28,17 @@ import java.util.List; import static com.google.common.collect.ImmutableList.toImmutableList; -import static io.trino.plugin.elasticsearch.ElasticsearchTableHandle.Type.QUERY; +import static io.trino.plugin.elasticsearch.OpenSearchTableHandle.Type.QUERY; import static java.util.Objects.requireNonNull; -public class ElasticsearchPageSourceProvider +public class OpenSearchPageSourceProvider implements ConnectorPageSourceProvider { - private final ElasticsearchClient client; + private final OpenSearchClient client; private final TypeManager typeManager; @Inject - public ElasticsearchPageSourceProvider(ElasticsearchClient client, TypeManager typeManager) + public OpenSearchPageSourceProvider(OpenSearchClient client, TypeManager typeManager) { this.client = requireNonNull(client, "client is null"); this.typeManager = requireNonNull(typeManager, "typeManager is null"); @@ -56,8 +56,8 @@ public ConnectorPageSource createPageSource( requireNonNull(split, "split is null"); requireNonNull(table, "table is null"); - ElasticsearchTableHandle elasticsearchTable = (ElasticsearchTableHandle) table; - ElasticsearchSplit elasticsearchSplit = (ElasticsearchSplit) split; + OpenSearchTableHandle elasticsearchTable = (OpenSearchTableHandle) table; + OpenSearchSplit elasticsearchSplit = (OpenSearchSplit) split; if (elasticsearchTable.getType().equals(QUERY)) { return new PassthroughQueryPageSource(client, elasticsearchTable); @@ -73,7 +73,7 @@ public ConnectorPageSource createPageSource( elasticsearchTable, elasticsearchSplit, columns.stream() - .map(ElasticsearchColumnHandle.class::cast) + .map(OpenSearchColumnHandle.class::cast) .collect(toImmutableList())); } } diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchPlugin.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchPlugin.java similarity index 86% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchPlugin.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchPlugin.java index b65805c0368ef..0d31841fc31ca 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchPlugin.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchPlugin.java @@ -20,18 +20,18 @@ import static java.util.Objects.requireNonNull; -public class ElasticsearchPlugin +public class OpenSearchPlugin implements Plugin { private final ConnectorFactory connectorFactory; - public ElasticsearchPlugin() + public OpenSearchPlugin() { - connectorFactory = new ElasticsearchConnectorFactory(); + connectorFactory = new OpenSearchConnectorFactory(); } @VisibleForTesting - ElasticsearchPlugin(ElasticsearchConnectorFactory factory) + OpenSearchPlugin(OpenSearchConnectorFactory factory) { connectorFactory = requireNonNull(factory, "factory is null"); } diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchQueryBuilder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchQueryBuilder.java similarity index 94% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchQueryBuilder.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchQueryBuilder.java index 9cbad441af708..a7442b5e38959 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchQueryBuilder.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchQueryBuilder.java @@ -51,16 +51,16 @@ import static java.lang.Math.toIntExact; import static java.time.format.DateTimeFormatter.ISO_DATE_TIME; -public final class ElasticsearchQueryBuilder +public final class OpenSearchQueryBuilder { - private ElasticsearchQueryBuilder() {} + private OpenSearchQueryBuilder() {} - public static QueryBuilder buildSearchQuery(TupleDomain constraint, Optional query, Map regexes) + public static QueryBuilder buildSearchQuery(TupleDomain constraint, Optional query, Map regexes) { BoolQueryBuilder queryBuilder = new BoolQueryBuilder(); if (constraint.getDomains().isPresent()) { - for (Map.Entry entry : constraint.getDomains().get().entrySet()) { - ElasticsearchColumnHandle column = entry.getKey(); + for (Map.Entry entry : constraint.getDomains().get().entrySet()) { + OpenSearchColumnHandle column = entry.getKey(); Domain domain = entry.getValue(); checkArgument(!domain.isNone(), "Unexpected NONE domain for %s", column.getName()); diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchSplit.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchSplit.java similarity index 94% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchSplit.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchSplit.java index a1ed6592e834a..7954758240eb7 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchSplit.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchSplit.java @@ -29,17 +29,17 @@ import static io.airlift.slice.SizeOf.sizeOf; import static java.util.Objects.requireNonNull; -public class ElasticsearchSplit +public class OpenSearchSplit implements ConnectorSplit { - private static final int INSTANCE_SIZE = instanceSize(ElasticsearchSplit.class); + private static final int INSTANCE_SIZE = instanceSize(OpenSearchSplit.class); private final String index; private final int shard; private final Optional address; @JsonCreator - public ElasticsearchSplit( + public OpenSearchSplit( @JsonProperty("index") String index, @JsonProperty("shard") int shard, @JsonProperty("address") Optional address) diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchSplitManager.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchSplitManager.java similarity index 71% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchSplitManager.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchSplitManager.java index b8e70d9350f15..13de8c1309e3e 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchSplitManager.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchSplitManager.java @@ -14,7 +14,7 @@ package io.trino.plugin.elasticsearch; import com.google.inject.Inject; -import io.trino.plugin.elasticsearch.client.ElasticsearchClient; +import io.trino.plugin.elasticsearch.client.OpenSearchClient; import io.trino.spi.connector.ConnectorSession; import io.trino.spi.connector.ConnectorSplitManager; import io.trino.spi.connector.ConnectorSplitSource; @@ -28,16 +28,16 @@ import java.util.Optional; import static com.google.common.collect.ImmutableList.toImmutableList; -import static io.trino.plugin.elasticsearch.ElasticsearchTableHandle.Type.QUERY; +import static io.trino.plugin.elasticsearch.OpenSearchTableHandle.Type.QUERY; import static java.util.Objects.requireNonNull; -public class ElasticsearchSplitManager +public class OpenSearchSplitManager implements ConnectorSplitManager { - private final ElasticsearchClient client; + private final OpenSearchClient client; @Inject - public ElasticsearchSplitManager(ElasticsearchClient client) + public OpenSearchSplitManager(OpenSearchClient client) { this.client = requireNonNull(client, "client is null"); } @@ -50,13 +50,13 @@ public ConnectorSplitSource getSplits( DynamicFilter dynamicFilter, Constraint constraint) { - ElasticsearchTableHandle tableHandle = (ElasticsearchTableHandle) table; + OpenSearchTableHandle tableHandle = (OpenSearchTableHandle) table; if (tableHandle.getType().equals(QUERY)) { - return new FixedSplitSource(new ElasticsearchSplit(tableHandle.getIndex(), 0, Optional.empty())); + return new FixedSplitSource(new OpenSearchSplit(tableHandle.getIndex(), 0, Optional.empty())); } - List splits = client.getSearchShards(tableHandle.getIndex()).stream() - .map(shard -> new ElasticsearchSplit(shard.getIndex(), shard.getId(), shard.getAddress())) + List splits = client.getSearchShards(tableHandle.getIndex()).stream() + .map(shard -> new OpenSearchSplit(shard.getIndex(), shard.getId(), shard.getAddress())) .collect(toImmutableList()); return new FixedSplitSource(splits); diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchTableHandle.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchTableHandle.java similarity index 95% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchTableHandle.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchTableHandle.java index a381ef99b34dd..7610937df0c2d 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchTableHandle.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchTableHandle.java @@ -28,7 +28,7 @@ import static java.util.Objects.requireNonNull; -public final class ElasticsearchTableHandle +public final class OpenSearchTableHandle implements ConnectorTableHandle { public enum Type @@ -44,7 +44,7 @@ public enum Type private final Optional query; private final OptionalLong limit; - public ElasticsearchTableHandle(Type type, String schema, String index, Optional query) + public OpenSearchTableHandle(Type type, String schema, String index, Optional query) { this.type = requireNonNull(type, "type is null"); this.schema = requireNonNull(schema, "schema is null"); @@ -57,7 +57,7 @@ public ElasticsearchTableHandle(Type type, String schema, String index, Optional } @JsonCreator - public ElasticsearchTableHandle( + public OpenSearchTableHandle( @JsonProperty("type") Type type, @JsonProperty("schema") String schema, @JsonProperty("index") String index, @@ -126,7 +126,7 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - ElasticsearchTableHandle that = (ElasticsearchTableHandle) o; + OpenSearchTableHandle that = (OpenSearchTableHandle) o; return type == that.type && schema.equals(that.schema) && index.equals(that.index) && diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchTransactionHandle.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchTransactionHandle.java similarity index 94% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchTransactionHandle.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchTransactionHandle.java index cf42b03b5d458..9bd0ac8453ca9 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchTransactionHandle.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchTransactionHandle.java @@ -15,7 +15,7 @@ import io.trino.spi.connector.ConnectorTransactionHandle; -public enum ElasticsearchTransactionHandle +public enum OpenSearchTransactionHandle implements ConnectorTransactionHandle { INSTANCE diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/PassthroughQueryPageSource.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/PassthroughQueryPageSource.java index 47e49d9c40039..ea5628d60bb28 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/PassthroughQueryPageSource.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/PassthroughQueryPageSource.java @@ -15,7 +15,7 @@ import com.google.common.collect.ImmutableList; import io.airlift.slice.Slices; -import io.trino.plugin.elasticsearch.client.ElasticsearchClient; +import io.trino.plugin.elasticsearch.client.OpenSearchClient; import io.trino.spi.Page; import io.trino.spi.PageBuilder; import io.trino.spi.block.BlockBuilder; @@ -33,7 +33,7 @@ public class PassthroughQueryPageSource private final String result; private boolean done; - public PassthroughQueryPageSource(ElasticsearchClient client, ElasticsearchTableHandle table) + public PassthroughQueryPageSource(OpenSearchClient client, OpenSearchTableHandle table) { requireNonNull(client, "client is null"); requireNonNull(table, "table is null"); diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/PasswordConfig.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/PasswordConfig.java index 2271e1da6b5fe..9e36701e47b29 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/PasswordConfig.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/PasswordConfig.java @@ -28,7 +28,7 @@ public String getUser() return user; } - @Config("elasticsearch.auth.user") + @Config("opensearch.auth.user") public PasswordConfig setUser(String user) { this.user = user; @@ -41,7 +41,7 @@ public String getPassword() return password; } - @Config("elasticsearch.auth.password") + @Config("opensearch.auth.password") @ConfigSecuritySensitive public PasswordConfig setPassword(String password) { diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ScanQueryPageSource.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ScanQueryPageSource.java index 8bb5e60e9209d..f9f310e251415 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ScanQueryPageSource.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ScanQueryPageSource.java @@ -16,7 +16,7 @@ import com.google.common.collect.AbstractIterator; import com.google.common.collect.ImmutableList; import io.airlift.log.Logger; -import io.trino.plugin.elasticsearch.client.ElasticsearchClient; +import io.trino.plugin.elasticsearch.client.OpenSearchClient; import io.trino.plugin.elasticsearch.decoders.Decoder; import io.trino.spi.Page; import io.trino.spi.block.Block; @@ -41,7 +41,7 @@ import static com.google.common.collect.ImmutableList.toImmutableList; import static io.trino.plugin.elasticsearch.BuiltinColumns.SOURCE; import static io.trino.plugin.elasticsearch.BuiltinColumns.isBuiltinColumn; -import static io.trino.plugin.elasticsearch.ElasticsearchQueryBuilder.buildSearchQuery; +import static io.trino.plugin.elasticsearch.OpenSearchQueryBuilder.buildSearchQuery; import static io.trino.spi.type.TimestampType.TIMESTAMP_MILLIS; import static java.util.Objects.requireNonNull; import static java.util.function.Predicate.isEqual; @@ -56,16 +56,16 @@ public class ScanQueryPageSource private final SearchHitIterator iterator; private final BlockBuilder[] columnBuilders; - private final List columns; + private final List columns; private long totalBytes; private long readTimeNanos; public ScanQueryPageSource( - ElasticsearchClient client, + OpenSearchClient client, TypeManager typeManager, - ElasticsearchTableHandle table, - ElasticsearchSplit split, - List columns) + OpenSearchTableHandle table, + OpenSearchSplit split, + List columns) { requireNonNull(client, "client is null"); requireNonNull(typeManager, "typeManager is null"); @@ -77,7 +77,7 @@ public ScanQueryPageSource( // When the _source field is requested, we need to bypass column pruning when fetching the document boolean needAllFields = columns.stream() - .map(ElasticsearchColumnHandle::getName) + .map(OpenSearchColumnHandle::getName) .anyMatch(isEqual(SOURCE.getName())); // Columns to fetch as doc_fields instead of pulling them out of the JSON source @@ -89,12 +89,12 @@ public ScanQueryPageSource( .collect(toImmutableList()); columnBuilders = columns.stream() - .map(ElasticsearchColumnHandle::getType) + .map(OpenSearchColumnHandle::getType) .map(type -> type.createBlockBuilder(null, 1)) .toArray(BlockBuilder[]::new); List requiredFields = columns.stream() - .map(ElasticsearchColumnHandle::getName) + .map(OpenSearchColumnHandle::getName) .filter(name -> !isBuiltinColumn(name)) .collect(toList()); @@ -111,7 +111,7 @@ public ScanQueryPageSource( SearchResponse searchResponse = client.beginSearch( split.getIndex(), split.getShard(), - buildSearchQuery(table.getConstraint().transformKeys(ElasticsearchColumnHandle.class::cast), table.getQuery(), table.getRegexes()), + buildSearchQuery(table.getConstraint().transformKeys(OpenSearchColumnHandle.class::cast), table.getQuery(), table.getRegexes()), needAllFields ? Optional.empty() : Optional.of(requiredFields), documentFields, sort, @@ -202,11 +202,11 @@ public static Object getField(Map document, String field) return value; } - private Map flattenFields(List columns) + private Map flattenFields(List columns) { Map result = new HashMap<>(); - for (ElasticsearchColumnHandle column : columns) { + for (OpenSearchColumnHandle column : columns) { flattenFields(result, column.getName(), column.getType()); } @@ -225,10 +225,10 @@ private void flattenFields(Map result, String fieldName, Type type } } - private List createDecoders(List columns) + private List createDecoders(List columns) { return columns.stream() - .map(ElasticsearchColumnHandle::getDecoderDescriptor) + .map(OpenSearchColumnHandle::getDecoderDescriptor) .map(DecoderDescriptor::createDecoder) .collect(toImmutableList()); } @@ -245,7 +245,7 @@ private static String appendPath(String base, String element) private static class SearchHitIterator extends AbstractIterator { - private final ElasticsearchClient client; + private final OpenSearchClient client; private final Supplier first; private final OptionalLong limit; @@ -256,7 +256,7 @@ private static class SearchHitIterator private long readTimeNanos; private long totalRecordCount; - public SearchHitIterator(ElasticsearchClient client, Supplier first, OptionalLong limit) + public SearchHitIterator(OpenSearchClient client, Supplier first, OptionalLong limit) { this.client = client; this.first = first; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/BackpressureRestClient.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/BackpressureRestClient.java index eebc163c2f4dd..f22515777a38c 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/BackpressureRestClient.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/BackpressureRestClient.java @@ -22,7 +22,7 @@ import dev.failsafe.function.CheckedSupplier; import io.airlift.log.Logger; import io.airlift.stats.TimeStat; -import io.trino.plugin.elasticsearch.ElasticsearchConfig; +import io.trino.plugin.elasticsearch.OpenSearchConfig; import org.apache.http.Header; import org.apache.http.HttpEntity; import org.apache.http.HttpHost; @@ -54,7 +54,7 @@ public class BackpressureRestClient private final TimeStat backpressureStats; private final ThreadLocal stopwatch = ThreadLocal.withInitial(Stopwatch::createUnstarted); - public BackpressureRestClient(RestClient delegate, ElasticsearchConfig config, TimeStat backpressureStats) + public BackpressureRestClient(RestClient delegate, OpenSearchConfig config, TimeStat backpressureStats) { this.delegate = requireNonNull(delegate, "restClient is null"); this.backpressureStats = requireNonNull(backpressureStats, "backpressureStats is null"); diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/BackpressureRestHighLevelClient.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/BackpressureRestHighLevelClient.java index 391283144cbec..8a7b01ef478b6 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/BackpressureRestHighLevelClient.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/BackpressureRestHighLevelClient.java @@ -22,7 +22,7 @@ import dev.failsafe.function.CheckedSupplier; import io.airlift.log.Logger; import io.airlift.stats.TimeStat; -import io.trino.plugin.elasticsearch.ElasticsearchConfig; +import io.trino.plugin.elasticsearch.OpenSearchConfig; import org.elasticsearch.ElasticsearchStatusException; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.search.ClearScrollRequest; @@ -55,7 +55,7 @@ public class BackpressureRestHighLevelClient private final TimeStat backpressureStats; private final ThreadLocal stopwatch = ThreadLocal.withInitial(Stopwatch::createUnstarted); - public BackpressureRestHighLevelClient(RestClientBuilder restClientBuilder, ElasticsearchConfig config, TimeStat backpressureStats) + public BackpressureRestHighLevelClient(RestClientBuilder restClientBuilder, OpenSearchConfig config, TimeStat backpressureStats) { this.backpressureStats = requireNonNull(backpressureStats, "backpressureStats is null"); delegate = new RestHighLevelClient(requireNonNull(restClientBuilder, "restClientBuilder is null")); diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/ElasticsearchClient.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/OpenSearchClient.java similarity index 94% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/ElasticsearchClient.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/OpenSearchClient.java index 3fcea05054f1a..bed6a4b573991 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/ElasticsearchClient.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/OpenSearchClient.java @@ -34,7 +34,7 @@ import io.airlift.stats.TimeStat; import io.airlift.units.Duration; import io.trino.plugin.elasticsearch.AwsSecurityConfig; -import io.trino.plugin.elasticsearch.ElasticsearchConfig; +import io.trino.plugin.elasticsearch.OpenSearchConfig; import io.trino.plugin.elasticsearch.PasswordConfig; import io.trino.spi.TrinoException; import jakarta.annotation.PostConstruct; @@ -92,11 +92,11 @@ import static io.airlift.concurrent.Threads.daemonThreadsNamed; import static io.airlift.json.JsonCodec.jsonCodec; import static io.trino.plugin.base.ssl.SslUtils.createSSLContext; -import static io.trino.plugin.elasticsearch.ElasticsearchErrorCode.ELASTICSEARCH_CONNECTION_ERROR; -import static io.trino.plugin.elasticsearch.ElasticsearchErrorCode.ELASTICSEARCH_INVALID_METADATA; -import static io.trino.plugin.elasticsearch.ElasticsearchErrorCode.ELASTICSEARCH_INVALID_RESPONSE; -import static io.trino.plugin.elasticsearch.ElasticsearchErrorCode.ELASTICSEARCH_QUERY_FAILURE; -import static io.trino.plugin.elasticsearch.ElasticsearchErrorCode.ELASTICSEARCH_SSL_INITIALIZATION_FAILURE; +import static io.trino.plugin.elasticsearch.OpenSearchErrorCode.ELASTICSEARCH_CONNECTION_ERROR; +import static io.trino.plugin.elasticsearch.OpenSearchErrorCode.ELASTICSEARCH_INVALID_METADATA; +import static io.trino.plugin.elasticsearch.OpenSearchErrorCode.ELASTICSEARCH_INVALID_RESPONSE; +import static io.trino.plugin.elasticsearch.OpenSearchErrorCode.ELASTICSEARCH_QUERY_FAILURE; +import static io.trino.plugin.elasticsearch.OpenSearchErrorCode.ELASTICSEARCH_SSL_INITIALIZATION_FAILURE; import static java.lang.StrictMath.toIntExact; import static java.lang.String.format; import static java.nio.charset.StandardCharsets.UTF_8; @@ -104,9 +104,9 @@ import static java.util.concurrent.TimeUnit.MILLISECONDS; import static org.elasticsearch.action.search.SearchType.QUERY_THEN_FETCH; -public class ElasticsearchClient +public class OpenSearchClient { - private static final Logger LOG = Logger.get(ElasticsearchClient.class); + private static final Logger LOG = Logger.get(OpenSearchClient.class); private static final JsonCodec SEARCH_SHARDS_RESPONSE_CODEC = jsonCodec(SearchShardsResponse.class); private static final JsonCodec NODES_RESPONSE_CODEC = jsonCodec(NodesResponse.class); @@ -120,7 +120,7 @@ public class ElasticsearchClient private final int scrollSize; private final Duration scrollTimeout; - private final AtomicReference> nodes = new AtomicReference<>(ImmutableSet.of()); + private final AtomicReference> nodes = new AtomicReference<>(ImmutableSet.of()); private final ScheduledExecutorService executor = newSingleThreadScheduledExecutor(daemonThreadsNamed("NodeRefresher")); private final AtomicBoolean started = new AtomicBoolean(); private final Duration refreshInterval; @@ -133,8 +133,8 @@ public class ElasticsearchClient private final TimeStat backpressureStats = new TimeStat(MILLISECONDS); @Inject - public ElasticsearchClient( - ElasticsearchConfig config, + public OpenSearchClient( + OpenSearchConfig config, Optional awsSecurityConfig, Optional passwordConfig) { @@ -170,10 +170,10 @@ private void refreshNodes() { // discover other nodes in the cluster and add them to the client try { - Set nodes = fetchNodes(); + Set nodes = fetchNodes(); HttpHost[] hosts = nodes.stream() - .map(ElasticsearchNode::getAddress) + .map(OpenSearchNode::getAddress) .filter(Optional::isPresent) .map(Optional::get) .map(address -> HttpHost.create(format("%s://%s", tlsEnabled ? "https" : "http", address))) @@ -193,7 +193,7 @@ private void refreshNodes() } private static BackpressureRestHighLevelClient createClient( - ElasticsearchConfig config, + OpenSearchConfig config, Optional awsSecurityConfig, Optional passwordConfig, TimeStat backpressureStats) @@ -287,40 +287,40 @@ private static Optional buildSslContext( } } - private Set fetchNodes() + private Set fetchNodes() { NodesResponse nodesResponse = doRequest("/_nodes/http", NODES_RESPONSE_CODEC::fromJson); - ImmutableSet.Builder result = ImmutableSet.builder(); + ImmutableSet.Builder result = ImmutableSet.builder(); for (Map.Entry entry : nodesResponse.getNodes().entrySet()) { String nodeId = entry.getKey(); NodesResponse.Node node = entry.getValue(); if (!Sets.intersection(node.getRoles(), NODE_ROLES).isEmpty()) { Optional address = node.getAddress() - .flatMap(ElasticsearchClient::extractAddress); + .flatMap(OpenSearchClient::extractAddress); - result.add(new ElasticsearchNode(nodeId, address)); + result.add(new OpenSearchNode(nodeId, address)); } } return result.build(); } - public Set getNodes() + public Set getNodes() { return nodes.get(); } public List getSearchShards(String index) { - Map nodeById = getNodes().stream() - .collect(toImmutableMap(ElasticsearchNode::getId, Function.identity())); + Map nodeById = getNodes().stream() + .collect(toImmutableMap(OpenSearchNode::getId, Function.identity())); SearchShardsResponse shardsResponse = doRequest(format("/%s/_search_shards", index), SEARCH_SHARDS_RESPONSE_CODEC::fromJson); ImmutableList.Builder shards = ImmutableList.builder(); - List nodes = ImmutableList.copyOf(nodeById.values()); + List nodes = ImmutableList.copyOf(nodeById.values()); for (List shardGroup : shardsResponse.getShardGroups()) { Optional candidate = shardGroup.stream() @@ -328,7 +328,7 @@ public List getSearchShards(String index) .min(this::shardPreference); SearchShardsResponse.Shard chosen; - ElasticsearchNode node; + OpenSearchNode node; if (candidate.isEmpty()) { // pick an arbitrary shard with and assign to an arbitrary node chosen = shardGroup.stream() diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/ElasticsearchNode.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/OpenSearchNode.java similarity index 92% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/ElasticsearchNode.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/OpenSearchNode.java index 32c51ec56fc74..ad6486feb8bbd 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/ElasticsearchNode.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/OpenSearchNode.java @@ -17,12 +17,12 @@ import static java.util.Objects.requireNonNull; -public class ElasticsearchNode +public class OpenSearchNode { private final String id; private final Optional address; - public ElasticsearchNode(String id, Optional address) + public OpenSearchNode(String id, Optional address) { this.id = requireNonNull(id, "id is null"); this.address = requireNonNull(address, "address is null"); diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ptf/RawQuery.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ptf/RawQuery.java index c891b704e60d7..841ab3bcf1af8 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ptf/RawQuery.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ptf/RawQuery.java @@ -18,9 +18,9 @@ import com.google.inject.Inject; import com.google.inject.Provider; import io.airlift.slice.Slice; -import io.trino.plugin.elasticsearch.ElasticsearchColumnHandle; -import io.trino.plugin.elasticsearch.ElasticsearchMetadata; -import io.trino.plugin.elasticsearch.ElasticsearchTableHandle; +import io.trino.plugin.elasticsearch.OpenSearchColumnHandle; +import io.trino.plugin.elasticsearch.OpenSearchMetadata; +import io.trino.plugin.elasticsearch.OpenSearchTableHandle; import io.trino.spi.connector.ColumnHandle; import io.trino.spi.connector.ColumnSchema; import io.trino.spi.connector.ConnectorAccessControl; @@ -42,7 +42,7 @@ import java.util.Optional; import static com.google.common.collect.ImmutableList.toImmutableList; -import static io.trino.plugin.elasticsearch.ElasticsearchTableHandle.Type.QUERY; +import static io.trino.plugin.elasticsearch.OpenSearchTableHandle.Type.QUERY; import static io.trino.spi.function.table.ReturnTypeSpecification.GenericTable.GENERIC_TABLE; import static io.trino.spi.type.VarcharType.VARCHAR; import static java.util.Objects.requireNonNull; @@ -54,10 +54,10 @@ public class RawQuery public static final String SCHEMA_NAME = "system"; public static final String NAME = "raw_query"; - private final ElasticsearchMetadata metadata; + private final OpenSearchMetadata metadata; @Inject - public RawQuery(ElasticsearchMetadata metadata) + public RawQuery(OpenSearchMetadata metadata) { this.metadata = requireNonNull(metadata, "metadata is null"); } @@ -71,9 +71,9 @@ public ConnectorTableFunction get() public static class RawQueryFunction extends AbstractConnectorTableFunction { - private final ElasticsearchMetadata metadata; + private final OpenSearchMetadata metadata; - public RawQueryFunction(ElasticsearchMetadata metadata) + public RawQueryFunction(OpenSearchMetadata metadata) { super( SCHEMA_NAME, @@ -106,7 +106,7 @@ public TableFunctionAnalysis analyze( String index = ((Slice) ((ScalarArgument) arguments.get("INDEX")).getValue()).toStringUtf8(); String query = ((Slice) ((ScalarArgument) arguments.get("QUERY")).getValue()).toStringUtf8(); - ElasticsearchTableHandle tableHandle = new ElasticsearchTableHandle(QUERY, schema, index, Optional.of(query)); + OpenSearchTableHandle tableHandle = new OpenSearchTableHandle(QUERY, schema, index, Optional.of(query)); ConnectorTableSchema tableSchema = metadata.getTableSchema(session, tableHandle); Map columnsByName = metadata.getColumnHandles(session, tableHandle); List columns = tableSchema.getColumns().stream() @@ -115,7 +115,7 @@ public TableFunctionAnalysis analyze( .collect(toImmutableList()); Descriptor returnedType = new Descriptor(columns.stream() - .map(ElasticsearchColumnHandle.class::cast) + .map(OpenSearchColumnHandle.class::cast) .map(column -> new Descriptor.Field(column.getName(), Optional.of(column.getType()))) .collect(toList())); @@ -131,10 +131,10 @@ public TableFunctionAnalysis analyze( public static class RawQueryFunctionHandle implements ConnectorTableFunctionHandle { - private final ElasticsearchTableHandle tableHandle; + private final OpenSearchTableHandle tableHandle; @JsonCreator - public RawQueryFunctionHandle(@JsonProperty("tableHandle") ElasticsearchTableHandle tableHandle) + public RawQueryFunctionHandle(@JsonProperty("tableHandle") OpenSearchTableHandle tableHandle) { this.tableHandle = requireNonNull(tableHandle, "tableHandle is null"); } diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/BaseElasticsearchConnectorTest.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/BaseOpenSearchConnectorTest.java similarity index 99% rename from plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/BaseElasticsearchConnectorTest.java rename to plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/BaseOpenSearchConnectorTest.java index 60586c95bbf59..a71174571412e 100644 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/BaseElasticsearchConnectorTest.java +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/BaseOpenSearchConnectorTest.java @@ -42,7 +42,7 @@ import java.util.List; import java.util.Map; -import static io.trino.plugin.elasticsearch.ElasticsearchQueryRunner.createElasticsearchQueryRunner; +import static io.trino.plugin.elasticsearch.OpenSearchQueryRunner.createElasticsearchQueryRunner; import static io.trino.spi.type.DoubleType.DOUBLE; import static io.trino.spi.type.VarcharType.VARCHAR; import static io.trino.testing.MaterializedResult.resultBuilder; @@ -54,15 +54,15 @@ import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; @TestInstance(PER_CLASS) -public abstract class BaseElasticsearchConnectorTest +public abstract class BaseOpenSearchConnectorTest extends BaseConnectorTest { private final String image; private final String catalogName; - private ElasticsearchServer elasticsearch; + private OpenSearchServer elasticsearch; protected RestHighLevelClient client; - BaseElasticsearchConnectorTest(String image, String catalogName) + BaseOpenSearchConnectorTest(String image, String catalogName) { this.image = image; this.catalogName = catalogName; @@ -72,7 +72,7 @@ public abstract class BaseElasticsearchConnectorTest protected QueryRunner createQueryRunner() throws Exception { - elasticsearch = new ElasticsearchServer(image, ImmutableMap.of()); + elasticsearch = new OpenSearchServer(image, ImmutableMap.of()); HostAndPort address = elasticsearch.getAddress(); client = new RestHighLevelClient(RestClient.builder(new HttpHost(address.getHost(), address.getPort()))); diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchExternalQueryRunner.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchExternalQueryRunner.java deleted file mode 100644 index beef029ed9486..0000000000000 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchExternalQueryRunner.java +++ /dev/null @@ -1,49 +0,0 @@ -/* - * 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.elasticsearch; - -import com.google.common.collect.ImmutableMap; -import com.google.common.net.HostAndPort; -import io.airlift.log.Logger; -import io.trino.testing.DistributedQueryRunner; -import io.trino.tpch.TpchTable; - -import static io.trino.plugin.elasticsearch.ElasticsearchQueryRunner.createElasticsearchQueryRunner; -import static java.lang.Integer.parseInt; - -public class ElasticsearchExternalQueryRunner -{ - private static final String HOSTNAME = System.getProperty("elasticsearch.host", "localhost"); - private static final int PORT = parseInt(System.getProperty("elasticsearch.port", "9200")); - - private ElasticsearchExternalQueryRunner() {} - - public static void main(String[] args) - throws Exception - { - // Please set hostname and port via VM options. e.g. "-Delasticsearch.host=localhost -Delasticsearch.port=9200" - // To start Elasticsearch: - // docker run -p 9200:9200 -e "discovery.type=single-node" docker.elastic.co/elasticsearch/elasticsearch:7.6.2 - DistributedQueryRunner queryRunner = createElasticsearchQueryRunner( - HostAndPort.fromParts(HOSTNAME, PORT), - TpchTable.getTables(), - ImmutableMap.of("http-server.http.port", "8080"), - ImmutableMap.of(), - 3); - - Logger log = Logger.get(ElasticsearchExternalQueryRunner.class); - log.info("======== SERVER STARTED ========"); - log.info("\n====\n%s\n====", queryRunner.getCoordinator().getBaseUrl()); - } -} diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchNginxProxy.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchNginxProxy.java deleted file mode 100644 index 298edeabaac77..0000000000000 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchNginxProxy.java +++ /dev/null @@ -1,103 +0,0 @@ -/* - * 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.elasticsearch; - -import com.google.common.net.HostAndPort; -import org.testcontainers.containers.Network; -import org.testcontainers.containers.NginxContainer; - -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; - -import static com.google.common.io.MoreFiles.deleteRecursively; -import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; -import static java.nio.charset.StandardCharsets.UTF_8; -import static org.testcontainers.utility.MountableFile.forHostPath; - -public class ElasticsearchNginxProxy -{ - private static final int PROXY_PORT = 9201; - private static final String NGINX_CONFIG_TEMPLATE = - "limit_req_zone $binary_remote_addr zone=mylimit:1m rate=REQUEST_PER_SECONDr/s;\n" + - "upstream elasticsearch {\n" + - " server elasticsearch-server:9200;\n" + - " keepalive 15;\n" + - "}\n" + - "server {\n" + - " access_log /var/log/nginx/access.log main;" + - " listen " + PROXY_PORT + ";\n" + - " location / {\n" + - " proxy_pass http://elasticsearch;\n" + - " proxy_redirect http://elasticsearch /;\n" + - " proxy_buffering off;\n" + - " proxy_http_version 1.1;\n" + - " proxy_set_header Connection \"Keep-Alive\";\n" + - " proxy_set_header Proxy-Connection \"Keep-Alive\";\n" + - " client_max_body_size 0;\n" + - " }\n" + - " location /_search/scroll {\n" + - " limit_req zone=mylimit;\n" + - " limit_req_status 429;\n" + - " proxy_pass http://elasticsearch;\n" + - " proxy_redirect http://elasticsearch /;\n" + - " proxy_buffering off;\n" + - " proxy_http_version 1.1;\n" + - " proxy_set_header Connection \"Keep-Alive\";\n" + - " proxy_set_header Proxy-Connection \"Keep-Alive\";\n" + - " client_max_body_size 0;\n" + - " }\n" + - " location ~ /.*/_search$ {\n" + - " limit_req zone=mylimit;\n" + - " limit_req_status 429;\n" + - " proxy_pass http://elasticsearch;\n" + - " proxy_redirect http://elasticsearch /;\n" + - " proxy_buffering off;\n" + - " proxy_http_version 1.1;\n" + - " proxy_set_header Connection \"Keep-Alive\";\n" + - " proxy_set_header Proxy-Connection \"Keep-Alive\";\n" + - " client_max_body_size 0;\n" + - " }\n" + - "}\n"; - - private final Path configurationPath; - private final NginxContainer container; - - public ElasticsearchNginxProxy(Network network, int requestsPerSecond) - throws IOException - { - container = new NginxContainer<>("nginx:1.19.8"); - container.withNetwork(network); - container.withNetworkAliases("elasticsearch-proxy"); - // Create the Nginx configuration file on host and copy it into a predefined path the container - configurationPath = Files.createTempDirectory("elasticsearchProxy"); - Path path = configurationPath.resolve("elasticsearch.conf"); - Files.writeString(path, NGINX_CONFIG_TEMPLATE.replace("REQUEST_PER_SECOND", String.valueOf(requestsPerSecond)), UTF_8); - container.withCopyFileToContainer(forHostPath(path), "/etc/nginx/conf.d/elasticsearch.conf"); - container.addExposedPort(PROXY_PORT); - container.start(); - } - - public void stop() - throws IOException - { - container.close(); - deleteRecursively(configurationPath, ALLOW_INSECURE); - } - - public HostAndPort getAddress() - { - return HostAndPort.fromString(container.getHost() + ":" + container.getMappedPort(PROXY_PORT)); - } -} diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchLoader.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/OpenSearchLoader.java similarity index 98% rename from plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchLoader.java rename to plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/OpenSearchLoader.java index 35fdeb39f4831..2e11226d3b983 100644 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchLoader.java +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/OpenSearchLoader.java @@ -44,13 +44,13 @@ import static java.util.Objects.requireNonNull; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; -public class ElasticsearchLoader +public class OpenSearchLoader extends AbstractTestingTrinoClient { private final String tableName; private final RestHighLevelClient client; - public ElasticsearchLoader( + public OpenSearchLoader( RestHighLevelClient client, String tableName, TestingTrinoServer trinoServer, diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchQueryRunner.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/OpenSearchQueryRunner.java similarity index 81% rename from plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchQueryRunner.java rename to plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/OpenSearchQueryRunner.java index 73be8fd317517..b78a416ccec3d 100644 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchQueryRunner.java +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/OpenSearchQueryRunner.java @@ -33,23 +33,23 @@ import static io.airlift.testing.Closeables.closeAllSuppress; import static io.airlift.units.Duration.nanosSince; -import static io.trino.plugin.elasticsearch.ElasticsearchServer.ELASTICSEARCH_7_IMAGE; +import static io.trino.plugin.elasticsearch.OpenSearchServer.ELASTICSEARCH_7_IMAGE; import static io.trino.plugin.tpch.TpchMetadata.TINY_SCHEMA_NAME; import static io.trino.testing.TestingSession.testSessionBuilder; import static java.lang.String.format; import static java.util.Locale.ENGLISH; import static java.util.concurrent.TimeUnit.SECONDS; -public final class ElasticsearchQueryRunner +public final class OpenSearchQueryRunner { static { Logging logging = Logging.initialize(); logging.setLevel("org.elasticsearch.client.RestClient", Level.OFF); } - private ElasticsearchQueryRunner() {} + private OpenSearchQueryRunner() {} - private static final Logger LOG = Logger.get(ElasticsearchQueryRunner.class); + private static final Logger LOG = Logger.get(OpenSearchQueryRunner.class); private static final String TPCH_SCHEMA = "tpch"; public static DistributedQueryRunner createElasticsearchQueryRunner( @@ -89,7 +89,7 @@ public static DistributedQueryRunner createElasticsearchQueryRunner( queryRunner.installPlugin(new TpchPlugin()); queryRunner.createCatalog("tpch", "tpch"); - ElasticsearchConnectorFactory testFactory = new ElasticsearchConnectorFactory(); + OpenSearchConnectorFactory testFactory = new OpenSearchConnectorFactory(); installElasticsearchPlugin(address, queryRunner, catalogName, testFactory, extraConnectorProperties); @@ -116,20 +116,20 @@ private static void installElasticsearchPlugin( HostAndPort address, QueryRunner queryRunner, String catalogName, - ElasticsearchConnectorFactory factory, + OpenSearchConnectorFactory factory, Map extraConnectorProperties) { - queryRunner.installPlugin(new ElasticsearchPlugin(factory)); + queryRunner.installPlugin(new OpenSearchPlugin(factory)); Map config = ImmutableMap.builder() - .put("elasticsearch.host", address.getHost()) - .put("elasticsearch.port", Integer.toString(address.getPort())) + .put("opensearch.host", address.getHost()) + .put("opensearch.port", Integer.toString(address.getPort())) // Node discovery relies on the publish_address exposed via the Elasticseach API // This doesn't work well within a docker environment that maps ES's port to a random public port - .put("elasticsearch.ignore-publish-address", "true") - .put("elasticsearch.default-schema-name", TPCH_SCHEMA) - .put("elasticsearch.scroll-size", "1000") - .put("elasticsearch.scroll-timeout", "1m") - .put("elasticsearch.request-timeout", "2m") + .put("opensearch.ignore-publish-address", "true") + .put("opensearch.default-schema-name", TPCH_SCHEMA) + .put("opensearch.scroll-size", "1000") + .put("opensearch.scroll-timeout", "1m") + .put("opensearch.request-timeout", "2m") .putAll(extraConnectorProperties) .buildOrThrow(); @@ -140,7 +140,7 @@ private static void loadTpchTopic(RestHighLevelClient client, TestingTrinoClient { long start = System.nanoTime(); LOG.info("Running import for %s", table.getTableName()); - ElasticsearchLoader loader = new ElasticsearchLoader(client, table.getTableName().toLowerCase(ENGLISH), trinoClient.getServer(), trinoClient.getDefaultSession()); + OpenSearchLoader loader = new OpenSearchLoader(client, table.getTableName().toLowerCase(ENGLISH), trinoClient.getServer(), trinoClient.getDefaultSession()); loader.execute(format("SELECT * from %s", new QualifiedObjectName(TPCH_SCHEMA, TINY_SCHEMA_NAME, table.getTableName().toLowerCase(ENGLISH)))); LOG.info("Imported %s in %s", table.getTableName(), nanosSince(start).convertToMostSuccinctTimeUnit()); } @@ -149,13 +149,13 @@ public static void main(String[] args) throws Exception { DistributedQueryRunner queryRunner = createElasticsearchQueryRunner( - new ElasticsearchServer(ELASTICSEARCH_7_IMAGE, ImmutableMap.of()).getAddress(), + new OpenSearchServer(ELASTICSEARCH_7_IMAGE, ImmutableMap.of()).getAddress(), TpchTable.getTables(), ImmutableMap.of("http-server.http.port", "8080"), ImmutableMap.of(), 3); - Logger log = Logger.get(ElasticsearchQueryRunner.class); + Logger log = Logger.get(OpenSearchQueryRunner.class); log.info("======== SERVER STARTED ========"); log.info("\n====\n%s\n====", queryRunner.getCoordinator().getBaseUrl()); } diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchServer.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/OpenSearchServer.java similarity index 92% rename from plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchServer.java rename to plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/OpenSearchServer.java index 8f3da20229d14..ecccdd95fd831 100644 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchServer.java +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/OpenSearchServer.java @@ -30,20 +30,20 @@ import static java.nio.file.Files.createTempDirectory; import static org.testcontainers.utility.MountableFile.forHostPath; -public class ElasticsearchServer +public class OpenSearchServer { public static final String ELASTICSEARCH_7_IMAGE = "elasticsearch:7.0.0"; private final Path configurationPath; private final ElasticsearchContainer container; - public ElasticsearchServer(String image, Map configurationFiles) + public OpenSearchServer(String image, Map configurationFiles) throws IOException { this(Network.SHARED, image, configurationFiles); } - public ElasticsearchServer(Network network, String image, Map configurationFiles) + public OpenSearchServer(Network network, String image, Map configurationFiles) throws IOException { DockerImageName dockerImageName = DockerImageName.parse(image).asCompatibleSubstituteFor("docker.elastic.co/elasticsearch/elasticsearch"); diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestAwsSecurityConfig.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestAwsSecurityConfig.java index 8bfb32eac484d..f08c5b2476246 100644 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestAwsSecurityConfig.java +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestAwsSecurityConfig.java @@ -39,11 +39,11 @@ public void testDefaults() public void testExplicitPropertyMappings() { Map properties = ImmutableMap.builder() - .put("elasticsearch.aws.access-key", "access") - .put("elasticsearch.aws.secret-key", "secret") - .put("elasticsearch.aws.region", "region") - .put("elasticsearch.aws.iam-role", "iamRole") - .put("elasticsearch.aws.external-id", "externalId") + .put("opensearch.aws.access-key", "access") + .put("opensearch.aws.secret-key", "secret") + .put("opensearch.aws.region", "region") + .put("opensearch.aws.iam-role", "iamRole") + .put("opensearch.aws.external-id", "externalId") .buildOrThrow(); AwsSecurityConfig expected = new AwsSecurityConfig() diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearch6ConnectorTest.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearch6ConnectorTest.java deleted file mode 100644 index 06b2c74992008..0000000000000 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearch6ConnectorTest.java +++ /dev/null @@ -1,63 +0,0 @@ -/* - * 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.elasticsearch; - -import com.google.common.collect.ImmutableMap; -import org.apache.http.entity.ContentType; -import org.apache.http.nio.entity.NStringEntity; -import org.intellij.lang.annotations.Language; -import org.junit.jupiter.api.Test; - -import java.io.IOException; - -import static java.lang.String.format; - -public class TestElasticsearch6ConnectorTest - extends BaseElasticsearchConnectorTest -{ - public TestElasticsearch6ConnectorTest() - { - super("docker.elastic.co/elasticsearch/elasticsearch-oss:6.6.0", "elasticsearch6"); - } - - @Test - public void testIndexWithMappingsButNoProperties() - throws IOException - { - String indexName = "test_empty_index_with_mappings_no_properties"; - - @Language("JSON") - String mappings = "{\"mappings\": " + - " {\"foo\": { \"dynamic\" : \"strict\" } }" + - "}"; - client.getLowLevelClient() - .performRequest("PUT", "/" + indexName, ImmutableMap.of(), new NStringEntity(mappings, ContentType.APPLICATION_JSON)); - - assertTableDoesNotExist(indexName); - } - - @Override - protected String indexEndpoint(String index, String docId) - { - return format("/%s/doc/%s", index, docId); - } - - @Override - protected String indexMapping(String properties) - { - return "{\"mappings\": " + - " {\"doc\": " + properties + "}" + - "}"; - } -} diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearch7ConnectorTest.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearch7ConnectorTest.java deleted file mode 100644 index b48d76c34cc0b..0000000000000 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearch7ConnectorTest.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * 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.elasticsearch; - -import static io.trino.plugin.elasticsearch.ElasticsearchServer.ELASTICSEARCH_7_IMAGE; -import static java.lang.String.format; - -public class TestElasticsearch7ConnectorTest - extends BaseElasticsearchConnectorTest -{ - public TestElasticsearch7ConnectorTest() - { - super(ELASTICSEARCH_7_IMAGE, "elasticsearch7"); - } - - @Override - protected String indexEndpoint(String index, String docId) - { - return format("/%s/_doc/%s", index, docId); - } - - @Override - protected String indexMapping(String properties) - { - return "{\"mappings\": " + properties + "}"; - } -} diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchBackpressure.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchBackpressure.java deleted file mode 100644 index dbaf931760664..0000000000000 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchBackpressure.java +++ /dev/null @@ -1,86 +0,0 @@ -/* - * 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.elasticsearch; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import io.trino.testing.AbstractTestQueryFramework; -import io.trino.testing.QueryRunner; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; -import org.testcontainers.containers.Network; - -import java.io.IOException; - -import static io.trino.plugin.elasticsearch.ElasticsearchQueryRunner.createElasticsearchQueryRunner; -import static io.trino.tpch.TpchTable.ORDERS; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; - -@TestInstance(PER_CLASS) -public class TestElasticsearchBackpressure - extends AbstractTestQueryFramework -{ - private static final String image = "elasticsearch:7.0.0"; - - private Network network; - private ElasticsearchServer elasticsearch; - private ElasticsearchNginxProxy elasticsearchNginxProxy; - - @Override - protected QueryRunner createQueryRunner() - throws Exception - { - network = Network.newNetwork(); - elasticsearch = new ElasticsearchServer(network, image, ImmutableMap.of()); - elasticsearchNginxProxy = new ElasticsearchNginxProxy(network, 1); - - return createElasticsearchQueryRunner( - elasticsearchNginxProxy.getAddress(), - ImmutableList.of(ORDERS), - ImmutableMap.of(), - ImmutableMap.of(), - // This test can only run on a single node, otherwise each node exports its own stats beans and they override each other - // You can only bind one such bean per JVM, so this causes problems with statistics being 0 despite backpressure handling - 1, - // Use a unique catalog name to make sure JMX stats beans are unique and not affected by other tests - "elasticsearch-backpressure"); - } - - @AfterAll - public final void destroy() - throws IOException - { - elasticsearchNginxProxy.stop(); - elasticsearchNginxProxy = null; - elasticsearch.stop(); - elasticsearch = null; - network.close(); - network = null; - } - - @Test - public void testQueryWithBackpressure() - { - // Check that JMX stats show no sign of backpressure - assertQueryReturnsEmptyResult("SELECT 1 FROM jmx.current.\"trino.plugin.elasticsearch.client:*\" WHERE \"backpressurestats.alltime.count\" > 0"); - assertQueryReturnsEmptyResult("SELECT 1 FROM jmx.current.\"trino.plugin.elasticsearch.client:*\" WHERE \"backpressurestats.alltime.max\" > 0"); - - assertQuerySucceeds("SELECT * FROM orders"); - - // Check that JMX stats show requests have been retried due to backpressure - assertQuery("SELECT DISTINCT 1 FROM jmx.current.\"trino.plugin.elasticsearch.client:*\" WHERE \"backpressurestats.alltime.count\" > 0", "VALUES 1"); - assertQuery("SELECT DISTINCT 1 FROM jmx.current.\"trino.plugin.elasticsearch.client:*\" WHERE \"backpressurestats.alltime.max\" > 0", "VALUES 1"); - } -} diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchConfig.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearchConfig.java similarity index 70% rename from plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchConfig.java rename to plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearchConfig.java index 4a74a4096701f..cf0e98a7908cc 100644 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchConfig.java +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearchConfig.java @@ -26,17 +26,17 @@ import static io.airlift.configuration.testing.ConfigAssertions.assertFullMapping; import static io.airlift.configuration.testing.ConfigAssertions.assertRecordedDefaults; import static io.airlift.configuration.testing.ConfigAssertions.recordDefaults; -import static io.trino.plugin.elasticsearch.ElasticsearchConfig.Security.AWS; +import static io.trino.plugin.elasticsearch.OpenSearchConfig.Security.AWS; import static java.util.concurrent.TimeUnit.MILLISECONDS; import static java.util.concurrent.TimeUnit.MINUTES; import static java.util.concurrent.TimeUnit.SECONDS; -public class TestElasticsearchConfig +public class TestOpenSearchConfig { @Test public void testDefaults() { - assertRecordedDefaults(recordDefaults(ElasticsearchConfig.class) + assertRecordedDefaults(recordDefaults(OpenSearchConfig.class) .setHosts(null) .setPort(9200) .setDefaultSchema("default") @@ -68,30 +68,30 @@ public void testExplicitPropertyMappings() Path truststoreFile = Files.createTempFile(null, null); Map properties = ImmutableMap.builder() - .put("elasticsearch.host", "example.com") - .put("elasticsearch.port", "9999") - .put("elasticsearch.default-schema-name", "test") - .put("elasticsearch.scroll-size", "4000") - .put("elasticsearch.scroll-timeout", "20s") - .put("elasticsearch.request-timeout", "1s") - .put("elasticsearch.connect-timeout", "10s") - .put("elasticsearch.backoff-init-delay", "100ms") - .put("elasticsearch.backoff-max-delay", "15s") - .put("elasticsearch.max-retry-time", "10s") - .put("elasticsearch.node-refresh-interval", "10m") - .put("elasticsearch.max-http-connections", "100") - .put("elasticsearch.http-thread-count", "30") - .put("elasticsearch.tls.enabled", "true") - .put("elasticsearch.tls.keystore-path", keystoreFile.toString()) - .put("elasticsearch.tls.keystore-password", "keystore-password") - .put("elasticsearch.tls.truststore-path", truststoreFile.toString()) - .put("elasticsearch.tls.truststore-password", "truststore-password") - .put("elasticsearch.tls.verify-hostnames", "false") - .put("elasticsearch.ignore-publish-address", "true") - .put("elasticsearch.security", "AWS") + .put("opensearch.host", "example.com") + .put("opensearch.port", "9999") + .put("opensearch.default-schema-name", "test") + .put("opensearch.scroll-size", "4000") + .put("opensearch.scroll-timeout", "20s") + .put("opensearch.request-timeout", "1s") + .put("opensearch.connect-timeout", "10s") + .put("opensearch.backoff-init-delay", "100ms") + .put("opensearch.backoff-max-delay", "15s") + .put("opensearch.max-retry-time", "10s") + .put("opensearch.node-refresh-interval", "10m") + .put("opensearch.max-http-connections", "100") + .put("opensearch.http-thread-count", "30") + .put("opensearch.tls.enabled", "true") + .put("opensearch.tls.keystore-path", keystoreFile.toString()) + .put("opensearch.tls.keystore-password", "keystore-password") + .put("opensearch.tls.truststore-path", truststoreFile.toString()) + .put("opensearch.tls.truststore-password", "truststore-password") + .put("opensearch.tls.verify-hostnames", "false") + .put("opensearch.ignore-publish-address", "true") + .put("opensearch.security", "AWS") .buildOrThrow(); - ElasticsearchConfig expected = new ElasticsearchConfig() + OpenSearchConfig expected = new OpenSearchConfig() .setHosts(Arrays.asList("example.com")) .setPort(9999) .setDefaultSchema("test") diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchOpenSearchConnectorTest.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearchConnectorTest.java similarity index 87% rename from plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchOpenSearchConnectorTest.java rename to plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearchConnectorTest.java index ee0dd999729df..69ec6f37171f2 100644 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchOpenSearchConnectorTest.java +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearchConnectorTest.java @@ -15,10 +15,10 @@ import static java.lang.String.format; -public class TestElasticsearchOpenSearchConnectorTest - extends BaseElasticsearchConnectorTest +public class TestOpenSearchConnectorTest + extends BaseOpenSearchConnectorTest { - public TestElasticsearchOpenSearchConnectorTest() + public TestOpenSearchConnectorTest() { // 1.0.0 and 1.0.1 causes NotSslRecordException during the initialization super("opensearchproject/opensearch:1.1.0", "opensearch"); diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchMetadata.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearchMetadata.java similarity index 93% rename from plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchMetadata.java rename to plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearchMetadata.java index f1f7c17cf9ac2..1d3cdac91833a 100644 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchMetadata.java +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearchMetadata.java @@ -20,7 +20,7 @@ import static org.assertj.core.api.Assertions.assertThat; -public class TestElasticsearchMetadata +public class TestOpenSearchMetadata { @Test public void testLikeToRegexp() @@ -42,6 +42,6 @@ public void testLikeToRegexp() private static String likeToRegexp(String pattern, Optional escapeChar) { - return ElasticsearchMetadata.likeToRegexp(Slices.utf8Slice(pattern), escapeChar.map(Slices::utf8Slice)); + return OpenSearchMetadata.likeToRegexp(Slices.utf8Slice(pattern), escapeChar.map(Slices::utf8Slice)); } } diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchQueryBuilder.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearchQueryBuilder.java similarity index 86% rename from plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchQueryBuilder.java rename to plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearchQueryBuilder.java index f040188b5f128..c0ea7e3da88f9 100644 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchQueryBuilder.java +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearchQueryBuilder.java @@ -33,18 +33,18 @@ import java.util.Map; import java.util.Optional; -import static io.trino.plugin.elasticsearch.ElasticsearchQueryBuilder.buildSearchQuery; +import static io.trino.plugin.elasticsearch.OpenSearchQueryBuilder.buildSearchQuery; import static io.trino.spi.type.DoubleType.DOUBLE; import static io.trino.spi.type.IntegerType.INTEGER; import static io.trino.spi.type.VarcharType.VARCHAR; import static org.assertj.core.api.Assertions.assertThat; -public class TestElasticsearchQueryBuilder +public class TestOpenSearchQueryBuilder { - private static final ElasticsearchColumnHandle NAME = new ElasticsearchColumnHandle("name", VARCHAR, new VarcharDecoder.Descriptor("name"), true); - private static final ElasticsearchColumnHandle AGE = new ElasticsearchColumnHandle("age", INTEGER, new IntegerDecoder.Descriptor("age"), true); - private static final ElasticsearchColumnHandle SCORE = new ElasticsearchColumnHandle("score", DOUBLE, new DoubleDecoder.Descriptor("score"), true); - private static final ElasticsearchColumnHandle LENGTH = new ElasticsearchColumnHandle("length", DOUBLE, new DoubleDecoder.Descriptor("length"), true); + private static final OpenSearchColumnHandle NAME = new OpenSearchColumnHandle("name", VARCHAR, new VarcharDecoder.Descriptor("name"), true); + private static final OpenSearchColumnHandle AGE = new OpenSearchColumnHandle("age", INTEGER, new IntegerDecoder.Descriptor("age"), true); + private static final OpenSearchColumnHandle SCORE = new OpenSearchColumnHandle("score", DOUBLE, new DoubleDecoder.Descriptor("score"), true); + private static final OpenSearchColumnHandle LENGTH = new OpenSearchColumnHandle("length", DOUBLE, new DoubleDecoder.Descriptor("length"), true); @Test public void testMatchAll() @@ -130,7 +130,7 @@ public void testMultiConstraint() .mustNot(new ExistsQueryBuilder(SCORE.getName()))); } - private static void assertQueryBuilder(Map domains, QueryBuilder expected) + private static void assertQueryBuilder(Map domains, QueryBuilder expected) { QueryBuilder actual = buildSearchQuery(TupleDomain.withColumnDomains(domains), Optional.empty(), Map.of()); assertThat(actual).isEqualTo(expected); diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestPasswordAuthentication.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestPasswordAuthentication.java index 14cb228e229a0..37ac749f24b47 100644 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestPasswordAuthentication.java +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestPasswordAuthentication.java @@ -38,7 +38,7 @@ import static com.google.common.io.Resources.getResource; import static io.airlift.testing.Closeables.closeAll; -import static io.trino.plugin.elasticsearch.ElasticsearchQueryRunner.createElasticsearchQueryRunner; +import static io.trino.plugin.elasticsearch.OpenSearchQueryRunner.createElasticsearchQueryRunner; import static java.lang.String.format; import static java.nio.charset.StandardCharsets.UTF_8; import static org.assertj.core.api.Assertions.assertThat; @@ -52,7 +52,7 @@ public class TestPasswordAuthentication private static final String USER = "elastic_user"; private static final String PASSWORD = "123456"; - private ElasticsearchServer elasticsearch; + private OpenSearchServer elasticsearch; private RestHighLevelClient client; private QueryAssertions assertions; @@ -61,7 +61,7 @@ public void setUp() throws Exception { // We use 7.8.0 because security became a non-commercial feature in recent versions - elasticsearch = new ElasticsearchServer("elasticsearch:7.8.0", ImmutableMap.builder() + elasticsearch = new OpenSearchServer("elasticsearch:7.8.0", ImmutableMap.builder() .put("elasticsearch.yml", loadResource("elasticsearch.yml")) .put("users", loadResource("users")) .put("users_roles", loadResource("users_roles")) diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestPasswordConfig.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestPasswordConfig.java index d767ac265363d..e905dfa9b8440 100644 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestPasswordConfig.java +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestPasswordConfig.java @@ -36,8 +36,8 @@ public void testDefaults() public void testExplicitPropertyMappings() { Map properties = ImmutableMap.builder() - .put("elasticsearch.auth.user", "user") - .put("elasticsearch.auth.password", "password") + .put("opensearch.auth.user", "user") + .put("opensearch.auth.password", "password") .buildOrThrow(); PasswordConfig expected = new PasswordConfig() diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/client/TestExtractAddress.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/client/TestExtractAddress.java index 34a982f53f164..abf69d1eb1606 100644 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/client/TestExtractAddress.java +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/client/TestExtractAddress.java @@ -17,7 +17,7 @@ import java.util.Optional; -import static io.trino.plugin.elasticsearch.client.ElasticsearchClient.extractAddress; +import static io.trino.plugin.elasticsearch.client.OpenSearchClient.extractAddress; import static org.assertj.core.api.Assertions.assertThat; public class TestExtractAddress From 7fcf3befad5ba05113ea5afc3034fdc153f60ed2 Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Tue, 2 Jan 2024 11:17:21 +0100 Subject: [PATCH 292/350] Rename config properties to opensearch Also cleanup defunct configs --- .../trino/plugin/elasticsearch/OpenSearchConfig.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchConfig.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchConfig.java index 10b3c90b0ed07..afd6d9e8f22e4 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchConfig.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchConfig.java @@ -137,7 +137,7 @@ public Duration getRequestTimeout() } @Config("opensearch.request-timeout") - @ConfigDescription("Elasticsearch request timeout") + @ConfigDescription("OpenSearch request timeout") public OpenSearchConfig setRequestTimeout(Duration requestTimeout) { this.requestTimeout = requestTimeout; @@ -151,7 +151,7 @@ public Duration getConnectTimeout() } @Config("opensearch.connect-timeout") - @ConfigDescription("Elasticsearch connect timeout") + @ConfigDescription("OpenSearch connect timeout") public OpenSearchConfig setConnectTimeout(Duration timeout) { this.connectTimeout = timeout; @@ -208,7 +208,7 @@ public Duration getNodeRefreshInterval() } @Config("opensearch.node-refresh-interval") - @ConfigDescription("How often to refresh the list of available Elasticsearch nodes") + @ConfigDescription("How often to refresh the list of available nodes in the OpenSearch cluster") public OpenSearchConfig setNodeRefreshInterval(Duration nodeRefreshInterval) { this.nodeRefreshInterval = nodeRefreshInterval; @@ -216,7 +216,7 @@ public OpenSearchConfig setNodeRefreshInterval(Duration nodeRefreshInterval) } @Config("opensearch.max-http-connections") - @ConfigDescription("Maximum number of persistent HTTP connections to Elasticsearch") + @ConfigDescription("Maximum number of persistent HTTP connections to OpenSearch cluster") public OpenSearchConfig setMaxHttpConnections(int size) { this.maxHttpConnections = size; @@ -230,7 +230,7 @@ public int getMaxHttpConnections() } @Config("opensearch.http-thread-count") - @ConfigDescription("Number of threads handling HTTP connections to Elasticsearch") + @ConfigDescription("Number of threads handling HTTP connections to OpenSearch cluster") public OpenSearchConfig setHttpThreadCount(int count) { this.httpThreadCount = count; From d5f51416fcc9488a420f179642e5504d28788f1c Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Tue, 2 Jan 2024 11:18:37 +0100 Subject: [PATCH 293/350] Rename connector factory to opensearch --- .../plugin/elasticsearch/OpenSearchConnectorFactory.java | 2 +- .../trino/plugin/elasticsearch/OpenSearchQueryRunner.java | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchConnectorFactory.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchConnectorFactory.java index d535c9d753ea3..95fbd97c62345 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchConnectorFactory.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchConnectorFactory.java @@ -39,7 +39,7 @@ public class OpenSearchConnectorFactory @Override public String getName() { - return "elasticsearch"; + return "opensearch"; } @Override diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/OpenSearchQueryRunner.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/OpenSearchQueryRunner.java index b78a416ccec3d..a7355242c4eff 100644 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/OpenSearchQueryRunner.java +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/OpenSearchQueryRunner.java @@ -91,7 +91,7 @@ public static DistributedQueryRunner createElasticsearchQueryRunner( OpenSearchConnectorFactory testFactory = new OpenSearchConnectorFactory(); - installElasticsearchPlugin(address, queryRunner, catalogName, testFactory, extraConnectorProperties); + installOpenSearchPlugin(address, queryRunner, catalogName, testFactory, extraConnectorProperties); TestingTrinoClient trinoClient = queryRunner.getClient(); @@ -112,7 +112,7 @@ public static DistributedQueryRunner createElasticsearchQueryRunner( } } - private static void installElasticsearchPlugin( + private static void installOpenSearchPlugin( HostAndPort address, QueryRunner queryRunner, String catalogName, @@ -133,7 +133,7 @@ private static void installElasticsearchPlugin( .putAll(extraConnectorProperties) .buildOrThrow(); - queryRunner.createCatalog(catalogName, "elasticsearch", config); + queryRunner.createCatalog(catalogName, "opensearch", config); } private static void loadTpchTopic(RestHighLevelClient client, TestingTrinoClient trinoClient, TpchTable table) From 5d32a68c8a72ef0c7699849d7c778f2cb424f07a Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Tue, 2 Jan 2024 11:29:14 +0100 Subject: [PATCH 294/350] Rename exported JMX beans to match package names --- .../plugin/elasticsearch/OpenSearchConnectorFactory.java | 2 +- .../plugin/elasticsearch/BaseOpenSearchConnectorTest.java | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchConnectorFactory.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchConnectorFactory.java index 95fbd97c62345..5c11c93cf9abb 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchConnectorFactory.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchConnectorFactory.java @@ -52,7 +52,7 @@ public Connector create(String catalogName, Map config, Connecto Bootstrap app = new Bootstrap( new MBeanModule(), new MBeanServerModule(), - new ConnectorObjectNameGeneratorModule("io.trino.plugin.elasticsearch", "trino.plugin.elasticsearch"), + new ConnectorObjectNameGeneratorModule("io.trino.plugin.opensearch", "trino.plugin.opensearch"), new JsonModule(), new TypeDeserializerModule(context.getTypeManager()), new OpenSearchConnectorModule(), diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/BaseOpenSearchConnectorTest.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/BaseOpenSearchConnectorTest.java index a71174571412e..b8723d88e40a7 100644 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/BaseOpenSearchConnectorTest.java +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/BaseOpenSearchConnectorTest.java @@ -143,8 +143,8 @@ public void testWithoutBackpressure() { assertQuerySucceeds("SELECT * FROM orders"); // Check that JMX stats show no sign of backpressure - assertQueryReturnsEmptyResult(format("SELECT 1 FROM jmx.current.\"trino.plugin.elasticsearch.client:*name=%s*\" WHERE \"backpressurestats.alltime.count\" > 0", catalogName)); - assertQueryReturnsEmptyResult(format("SELECT 1 FROM jmx.current.\"trino.plugin.elasticsearch.client:*name=%s*\" WHERE \"backpressurestats.alltime.max\" > 0", catalogName)); + assertQueryReturnsEmptyResult(format("SELECT 1 FROM jmx.current.\"trino.plugin.opensearch.client:*name=%s*\" WHERE \"backpressurestats.alltime.count\" > 0", catalogName)); + assertQueryReturnsEmptyResult(format("SELECT 1 FROM jmx.current.\"trino.plugin.opensearch.client:*name=%s*\" WHERE \"backpressurestats.alltime.max\" > 0", catalogName)); } @Test From 36740e6182251cb63dd92bee98f800d314a691df Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Tue, 2 Jan 2024 13:00:45 +0100 Subject: [PATCH 295/350] Update lucene analyzer to 9.7.0 --- core/trino-main/pom.xml | 4 ++-- .../java/io/trino/operator/scalar/WordStemFunction.java | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/core/trino-main/pom.xml b/core/trino-main/pom.xml index 34b8648ccb90d..2722379dfe03f 100644 --- a/core/trino-main/pom.xml +++ b/core/trino-main/pom.xml @@ -335,8 +335,8 @@ org.apache.lucene - lucene-analyzers-common - 8.11.2 + lucene-analysis-common + 9.7.0 diff --git a/core/trino-main/src/main/java/io/trino/operator/scalar/WordStemFunction.java b/core/trino-main/src/main/java/io/trino/operator/scalar/WordStemFunction.java index e258d38f60efe..7ca6ebf76dd77 100644 --- a/core/trino-main/src/main/java/io/trino/operator/scalar/WordStemFunction.java +++ b/core/trino-main/src/main/java/io/trino/operator/scalar/WordStemFunction.java @@ -20,7 +20,7 @@ import io.trino.spi.function.LiteralParameters; import io.trino.spi.function.ScalarFunction; import io.trino.spi.function.SqlType; -import org.tartarus.snowball.SnowballProgram; +import org.tartarus.snowball.SnowballStemmer; import org.tartarus.snowball.ext.ArmenianStemmer; import org.tartarus.snowball.ext.BasqueStemmer; import org.tartarus.snowball.ext.CatalanStemmer; @@ -52,7 +52,7 @@ public final class WordStemFunction { private WordStemFunction() {} - private static final Map> STEMMERS = ImmutableMap.>builder() + private static final Map> STEMMERS = ImmutableMap.>builder() .put(utf8Slice("ca"), CatalanStemmer::new) .put(utf8Slice("da"), DanishStemmer::new) .put(utf8Slice("de"), German2Stemmer::new) @@ -90,14 +90,14 @@ public static Slice wordStem(@SqlType("varchar(x)") Slice slice) @SqlType("varchar(x)") public static Slice wordStem(@SqlType("varchar(x)") Slice slice, @SqlType("varchar(2)") Slice language) { - Supplier stemmer = STEMMERS.get(language); + Supplier stemmer = STEMMERS.get(language); if (stemmer == null) { throw new TrinoException(INVALID_FUNCTION_ARGUMENT, "Unknown stemmer language: " + language.toStringUtf8()); } return wordStem(slice, stemmer.get()); } - private static Slice wordStem(Slice slice, SnowballProgram stemmer) + private static Slice wordStem(Slice slice, SnowballStemmer stemmer) { stemmer.setCurrent(slice.toStringUtf8()); return stemmer.stem() ? utf8Slice(stemmer.getCurrent()) : slice; From 1744ceab9da12da9570c7e8ee3798318d63a998b Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Tue, 2 Jan 2024 13:11:13 +0100 Subject: [PATCH 296/350] Migrate OpenSearch plugin to OpenSearch java client Add tests for both OpenSearch 1.x and 2.x compatibility --- plugin/trino-opensearch/pom.xml | 96 +++++++----------- .../elasticsearch/NodesSystemTable.java | 18 ++-- .../elasticsearch/OpenSearchErrorCode.java | 10 +- .../OpenSearchPageSourceProvider.java | 14 +-- .../elasticsearch/OpenSearchQueryBuilder.java | 16 +-- .../elasticsearch/ScanQueryPageSource.java | 6 +- .../client/AwsRequestSigner.java | 2 +- .../client/BackpressureRestClient.java | 14 +-- .../BackpressureRestHighLevelClient.java | 26 ++--- .../client/OpenSearchClient.java | 81 ++++++++------- .../elasticsearch/decoders/ArrayDecoder.java | 2 +- .../elasticsearch/decoders/BigintDecoder.java | 2 +- .../decoders/BooleanDecoder.java | 2 +- .../elasticsearch/decoders/Decoder.java | 2 +- .../elasticsearch/decoders/DoubleDecoder.java | 2 +- .../decoders/IdColumnDecoder.java | 2 +- .../decoders/IntegerDecoder.java | 2 +- .../decoders/IpAddressDecoder.java | 2 +- .../decoders/RawJsonDecoder.java | 2 +- .../elasticsearch/decoders/RealDecoder.java | 2 +- .../elasticsearch/decoders/RowDecoder.java | 2 +- .../decoders/ScoreColumnDecoder.java | 2 +- .../decoders/SmallintDecoder.java | 2 +- .../decoders/SourceColumnDecoder.java | 2 +- .../decoders/TimestampDecoder.java | 4 +- .../decoders/TinyintDecoder.java | 2 +- .../decoders/VarbinaryDecoder.java | 2 +- .../decoders/VarcharDecoder.java | 2 +- .../BaseOpenSearchConnectorTest.java | 64 +++++++----- .../elasticsearch/OpenSearchLoader.java | 23 +++-- .../elasticsearch/OpenSearchQueryRunner.java | 18 ++-- .../elasticsearch/OpenSearchServer.java | 15 +-- .../TestOpenSearch2ConnectorTest.java | 36 +++++++ .../TestOpenSearchConnectorTest.java | 14 --- .../TestOpenSearchQueryBuilder.java | 12 +-- .../TestPasswordAuthentication.java | 89 +++++++++------- .../src/test/resources/elasticsearch.yml | 4 - .../src/test/resources/esnode-key.pem | 28 +++++ .../src/test/resources/esnode.pem | 21 ++++ .../src/test/resources/opensearch.yml | 32 ++++++ .../src/test/resources/roles.yml | 6 -- .../src/test/resources/root-ca.pem | 20 ++++ .../src/test/resources/serverkey.pem | 28 +++++ .../src/test/resources/truststore.jks | Bin 0 -> 1809 bytes .../trino-opensearch/src/test/resources/users | 1 - .../src/test/resources/users_roles | 1 - 46 files changed, 439 insertions(+), 294 deletions(-) create mode 100644 plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearch2ConnectorTest.java delete mode 100644 plugin/trino-opensearch/src/test/resources/elasticsearch.yml create mode 100644 plugin/trino-opensearch/src/test/resources/esnode-key.pem create mode 100644 plugin/trino-opensearch/src/test/resources/esnode.pem create mode 100644 plugin/trino-opensearch/src/test/resources/opensearch.yml delete mode 100644 plugin/trino-opensearch/src/test/resources/roles.yml create mode 100644 plugin/trino-opensearch/src/test/resources/root-ca.pem create mode 100644 plugin/trino-opensearch/src/test/resources/serverkey.pem create mode 100644 plugin/trino-opensearch/src/test/resources/truststore.jks delete mode 100644 plugin/trino-opensearch/src/test/resources/users delete mode 100644 plugin/trino-opensearch/src/test/resources/users_roles diff --git a/plugin/trino-opensearch/pom.xml b/plugin/trino-opensearch/pom.xml index 2a94e70b982d2..76a0af78a65e6 100644 --- a/plugin/trino-opensearch/pom.xml +++ b/plugin/trino-opensearch/pom.xml @@ -15,20 +15,13 @@ ${project.parent.basedir} - 6.8.23 + 2.11.1 com.amazonaws aws-java-sdk-core - - - - org.apache.httpcomponents - httpclient - - @@ -114,9 +107,8 @@ org.apache.httpcomponents httpasyncclient - 4.1.2 + 4.1.5 - commons-logging commons-logging @@ -129,7 +121,6 @@ httpclient 4.5.14 - commons-logging commons-logging @@ -150,51 +141,46 @@ - org.elasticsearch - elasticsearch - ${dep.elasticsearch.version} + org.opensearch + opensearch + ${dep.opensearch.version} - org.apache.logging.log4j - log4j-api - - - - org.apache.lucene - lucene-analyzers-common - - - - org.elasticsearch - jna + org.hdrhistogram + HdrHistogram - org.elasticsearch - elasticsearch-core - ${dep.elasticsearch.version} + org.opensearch + opensearch-common + ${dep.opensearch.version} + + + org.hdrhistogram + HdrHistogram + + - org.elasticsearch - elasticsearch-x-content - ${dep.elasticsearch.version} + org.opensearch + opensearch-core + ${dep.opensearch.version} - org.yaml - snakeyaml + org.hdrhistogram + HdrHistogram - org.elasticsearch.client - elasticsearch-rest-client - ${dep.elasticsearch.version} + org.opensearch.client + opensearch-rest-client + ${dep.opensearch.version} - commons-logging commons-logging @@ -203,13 +189,13 @@ - org.elasticsearch.client - elasticsearch-rest-high-level-client - ${dep.elasticsearch.version} + org.opensearch.client + opensearch-rest-high-level-client + ${dep.opensearch.version} - org.apache.logging.log4j - log4j-api + org.hdrhistogram + HdrHistogram @@ -267,6 +253,13 @@ runtime + + org.opensearch + opensearch-x-content + ${dep.opensearch.version} + runtime + + io.airlift http-server @@ -389,12 +382,6 @@ test - - org.testcontainers - nginx - test - - org.testcontainers testcontainers @@ -419,17 +406,6 @@ - - org.apache.maven.plugins - maven-dependency-plugin - - - - - org.elasticsearch:elasticsearch-x-content - - - diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/NodesSystemTable.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/NodesSystemTable.java index aaedda268f601..092787eb52a0d 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/NodesSystemTable.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/NodesSystemTable.java @@ -45,8 +45,8 @@ public class NodesSystemTable ImmutableList.builder() .add(new ColumnMetadata("trino_node_id", createUnboundedVarcharType())) .add(new ColumnMetadata("trino_node_address", createUnboundedVarcharType())) - .add(new ColumnMetadata("elasticsearch_node_id", createUnboundedVarcharType())) - .add(new ColumnMetadata("elasticsearch_node_address", createUnboundedVarcharType())) + .add(new ColumnMetadata("opensearch_node_id", createUnboundedVarcharType())) + .add(new ColumnMetadata("opensearch_node_address", createUnboundedVarcharType())) .build()); private final OpenSearchClient client; @@ -80,26 +80,26 @@ public ConnectorPageSource pageSource(ConnectorTransactionHandle transaction, Co BlockBuilder nodeId = VARCHAR.createBlockBuilder(null, nodes.size()); BlockBuilder trinoAddress = VARCHAR.createBlockBuilder(null, nodes.size()); - BlockBuilder elasticsearchNodeId = VARCHAR.createBlockBuilder(null, nodes.size()); - BlockBuilder elasticsearchAddress = VARCHAR.createBlockBuilder(null, nodes.size()); + BlockBuilder opensearchNodeId = VARCHAR.createBlockBuilder(null, nodes.size()); + BlockBuilder opensearchAddress = VARCHAR.createBlockBuilder(null, nodes.size()); for (OpenSearchNode node : nodes) { VARCHAR.writeString(nodeId, currentNode.getNodeIdentifier()); VARCHAR.writeString(trinoAddress, currentNode.getHostAndPort().toString()); - VARCHAR.writeString(elasticsearchNodeId, node.getId()); + VARCHAR.writeString(opensearchNodeId, node.getId()); if (node.getAddress().isPresent()) { - VARCHAR.writeString(elasticsearchAddress, node.getAddress().get()); + VARCHAR.writeString(opensearchAddress, node.getAddress().get()); } else { - elasticsearchAddress.appendNull(); + opensearchAddress.appendNull(); } } return new FixedPageSource(ImmutableList.of(new Page( nodeId.build(), trinoAddress.build(), - elasticsearchNodeId.build(), - elasticsearchAddress.build()))); + opensearchNodeId.build(), + opensearchAddress.build()))); } } diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchErrorCode.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchErrorCode.java index abe224192be35..51c1b432bc9d0 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchErrorCode.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchErrorCode.java @@ -23,11 +23,11 @@ public enum OpenSearchErrorCode implements ErrorCodeSupplier { - ELASTICSEARCH_CONNECTION_ERROR(0, EXTERNAL), - ELASTICSEARCH_INVALID_RESPONSE(1, EXTERNAL), - ELASTICSEARCH_SSL_INITIALIZATION_FAILURE(2, EXTERNAL), - ELASTICSEARCH_QUERY_FAILURE(3, USER_ERROR), - ELASTICSEARCH_INVALID_METADATA(4, USER_ERROR); + OPENSEARCH_CONNECTION_ERROR(0, EXTERNAL), + OPENSEARCH_INVALID_RESPONSE(1, EXTERNAL), + OPENSEARCH_SSL_INITIALIZATION_FAILURE(2, EXTERNAL), + OPENSEARCH_QUERY_FAILURE(3, USER_ERROR), + OPENSEARCH_INVALID_METADATA(4, USER_ERROR); private final ErrorCode errorCode; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchPageSourceProvider.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchPageSourceProvider.java index dbce38ee79000..368f7f78f5428 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchPageSourceProvider.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchPageSourceProvider.java @@ -56,22 +56,22 @@ public ConnectorPageSource createPageSource( requireNonNull(split, "split is null"); requireNonNull(table, "table is null"); - OpenSearchTableHandle elasticsearchTable = (OpenSearchTableHandle) table; - OpenSearchSplit elasticsearchSplit = (OpenSearchSplit) split; + OpenSearchTableHandle opensearchTable = (OpenSearchTableHandle) table; + OpenSearchSplit opensearchSplit = (OpenSearchSplit) split; - if (elasticsearchTable.getType().equals(QUERY)) { - return new PassthroughQueryPageSource(client, elasticsearchTable); + if (opensearchTable.getType().equals(QUERY)) { + return new PassthroughQueryPageSource(client, opensearchTable); } if (columns.isEmpty()) { - return new CountQueryPageSource(client, elasticsearchTable, elasticsearchSplit); + return new CountQueryPageSource(client, opensearchTable, opensearchSplit); } return new ScanQueryPageSource( client, typeManager, - elasticsearchTable, - elasticsearchSplit, + opensearchTable, + opensearchSplit, columns.stream() .map(OpenSearchColumnHandle.class::cast) .collect(toImmutableList())); diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchQueryBuilder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchQueryBuilder.java index a7442b5e38959..19549d85a45ea 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchQueryBuilder.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchQueryBuilder.java @@ -19,14 +19,14 @@ import io.trino.spi.predicate.Range; import io.trino.spi.predicate.TupleDomain; import io.trino.spi.type.Type; -import org.elasticsearch.index.query.BoolQueryBuilder; -import org.elasticsearch.index.query.ExistsQueryBuilder; -import org.elasticsearch.index.query.MatchAllQueryBuilder; -import org.elasticsearch.index.query.QueryBuilder; -import org.elasticsearch.index.query.QueryStringQueryBuilder; -import org.elasticsearch.index.query.RangeQueryBuilder; -import org.elasticsearch.index.query.RegexpQueryBuilder; -import org.elasticsearch.index.query.TermQueryBuilder; +import org.opensearch.index.query.BoolQueryBuilder; +import org.opensearch.index.query.ExistsQueryBuilder; +import org.opensearch.index.query.MatchAllQueryBuilder; +import org.opensearch.index.query.QueryBuilder; +import org.opensearch.index.query.QueryStringQueryBuilder; +import org.opensearch.index.query.RangeQueryBuilder; +import org.opensearch.index.query.RegexpQueryBuilder; +import org.opensearch.index.query.TermQueryBuilder; import java.time.Instant; import java.time.ZoneOffset; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ScanQueryPageSource.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ScanQueryPageSource.java index f9f310e251415..b1725cf945230 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ScanQueryPageSource.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ScanQueryPageSource.java @@ -26,9 +26,9 @@ import io.trino.spi.type.RowType; import io.trino.spi.type.Type; import io.trino.spi.type.TypeManager; -import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.search.SearchHit; -import org.elasticsearch.search.SearchHits; +import org.opensearch.action.search.SearchResponse; +import org.opensearch.search.SearchHit; +import org.opensearch.search.SearchHits; import java.util.Arrays; import java.util.HashMap; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/AwsRequestSigner.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/AwsRequestSigner.java index 7619b263625eb..dfefedc3c041c 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/AwsRequestSigner.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/AwsRequestSigner.java @@ -45,7 +45,7 @@ class AwsRequestSigner implements HttpRequestInterceptor { - private static final String SERVICE_NAME = "es"; + private static final String SERVICE_NAME = "aoss"; private final AWSCredentialsProvider credentialsProvider; private final AWS4Signer signer; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/BackpressureRestClient.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/BackpressureRestClient.java index f22515777a38c..0ecd2358544e7 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/BackpressureRestClient.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/BackpressureRestClient.java @@ -26,13 +26,13 @@ import org.apache.http.Header; import org.apache.http.HttpEntity; import org.apache.http.HttpHost; -import org.elasticsearch.client.Node; -import org.elasticsearch.client.Request; -import org.elasticsearch.client.RequestOptions; -import org.elasticsearch.client.Response; -import org.elasticsearch.client.ResponseException; -import org.elasticsearch.client.RestClient; -import org.elasticsearch.rest.RestStatus; +import org.opensearch.client.Node; +import org.opensearch.client.Request; +import org.opensearch.client.RequestOptions; +import org.opensearch.client.Response; +import org.opensearch.client.ResponseException; +import org.opensearch.client.RestClient; +import org.opensearch.core.rest.RestStatus; import java.io.IOException; import java.util.Map; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/BackpressureRestHighLevelClient.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/BackpressureRestHighLevelClient.java index 8a7b01ef478b6..4871950028e93 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/BackpressureRestHighLevelClient.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/BackpressureRestHighLevelClient.java @@ -23,17 +23,17 @@ import io.airlift.log.Logger; import io.airlift.stats.TimeStat; import io.trino.plugin.elasticsearch.OpenSearchConfig; -import org.elasticsearch.ElasticsearchStatusException; -import org.elasticsearch.action.ActionResponse; -import org.elasticsearch.action.search.ClearScrollRequest; -import org.elasticsearch.action.search.ClearScrollResponse; -import org.elasticsearch.action.search.SearchRequest; -import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.action.search.SearchScrollRequest; -import org.elasticsearch.client.RequestOptions; -import org.elasticsearch.client.RestClientBuilder; -import org.elasticsearch.client.RestHighLevelClient; -import org.elasticsearch.rest.RestStatus; +import org.opensearch.OpenSearchStatusException; +import org.opensearch.action.search.ClearScrollRequest; +import org.opensearch.action.search.ClearScrollResponse; +import org.opensearch.action.search.SearchRequest; +import org.opensearch.action.search.SearchResponse; +import org.opensearch.action.search.SearchScrollRequest; +import org.opensearch.client.RequestOptions; +import org.opensearch.client.RestClientBuilder; +import org.opensearch.client.RestHighLevelClient; +import org.opensearch.core.action.ActionResponse; +import org.opensearch.core.rest.RestStatus; import java.io.Closeable; import java.io.IOException; @@ -104,8 +104,8 @@ public ClearScrollResponse clearScroll(ClearScrollRequest clearScrollRequest) private static boolean isBackpressure(Throwable throwable) { - return (throwable instanceof ElasticsearchStatusException) && - (((ElasticsearchStatusException) throwable).status() == RestStatus.TOO_MANY_REQUESTS); + return (throwable instanceof OpenSearchStatusException) && + (((OpenSearchStatusException) throwable).status() == RestStatus.TOO_MANY_REQUESTS); } private void onComplete(ExecutionCompletedEvent executionCompletedEvent) diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/OpenSearchClient.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/OpenSearchClient.java index bed6a4b573991..e6911266fa941 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/OpenSearchClient.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/OpenSearchClient.java @@ -53,18 +53,18 @@ import org.apache.http.impl.nio.reactor.IOReactorConfig; import org.apache.http.message.BasicHeader; import org.apache.http.util.EntityUtils; -import org.elasticsearch.ElasticsearchStatusException; -import org.elasticsearch.action.search.ClearScrollRequest; -import org.elasticsearch.action.search.SearchRequest; -import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.action.search.SearchScrollRequest; -import org.elasticsearch.client.Response; -import org.elasticsearch.client.ResponseException; -import org.elasticsearch.client.RestClient; -import org.elasticsearch.client.RestClientBuilder; -import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.index.query.QueryBuilder; -import org.elasticsearch.search.builder.SearchSourceBuilder; +import org.opensearch.OpenSearchStatusException; +import org.opensearch.action.search.ClearScrollRequest; +import org.opensearch.action.search.SearchRequest; +import org.opensearch.action.search.SearchResponse; +import org.opensearch.action.search.SearchScrollRequest; +import org.opensearch.client.Response; +import org.opensearch.client.ResponseException; +import org.opensearch.client.RestClient; +import org.opensearch.client.RestClientBuilder; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.index.query.QueryBuilder; +import org.opensearch.search.builder.SearchSourceBuilder; import org.weakref.jmx.Managed; import org.weakref.jmx.Nested; @@ -92,17 +92,17 @@ import static io.airlift.concurrent.Threads.daemonThreadsNamed; import static io.airlift.json.JsonCodec.jsonCodec; import static io.trino.plugin.base.ssl.SslUtils.createSSLContext; -import static io.trino.plugin.elasticsearch.OpenSearchErrorCode.ELASTICSEARCH_CONNECTION_ERROR; -import static io.trino.plugin.elasticsearch.OpenSearchErrorCode.ELASTICSEARCH_INVALID_METADATA; -import static io.trino.plugin.elasticsearch.OpenSearchErrorCode.ELASTICSEARCH_INVALID_RESPONSE; -import static io.trino.plugin.elasticsearch.OpenSearchErrorCode.ELASTICSEARCH_QUERY_FAILURE; -import static io.trino.plugin.elasticsearch.OpenSearchErrorCode.ELASTICSEARCH_SSL_INITIALIZATION_FAILURE; +import static io.trino.plugin.elasticsearch.OpenSearchErrorCode.OPENSEARCH_CONNECTION_ERROR; +import static io.trino.plugin.elasticsearch.OpenSearchErrorCode.OPENSEARCH_INVALID_METADATA; +import static io.trino.plugin.elasticsearch.OpenSearchErrorCode.OPENSEARCH_INVALID_RESPONSE; +import static io.trino.plugin.elasticsearch.OpenSearchErrorCode.OPENSEARCH_QUERY_FAILURE; +import static io.trino.plugin.elasticsearch.OpenSearchErrorCode.OPENSEARCH_SSL_INITIALIZATION_FAILURE; import static java.lang.StrictMath.toIntExact; import static java.lang.String.format; import static java.nio.charset.StandardCharsets.UTF_8; import static java.util.concurrent.Executors.newSingleThreadScheduledExecutor; import static java.util.concurrent.TimeUnit.MILLISECONDS; -import static org.elasticsearch.action.search.SearchType.QUERY_THEN_FETCH; +import static org.opensearch.action.search.SearchType.QUERY_THEN_FETCH; public class OpenSearchClient { @@ -201,8 +201,7 @@ private static BackpressureRestHighLevelClient createClient( RestClientBuilder builder = RestClient.builder( config.getHosts().stream() .map(httpHost -> new HttpHost(httpHost, config.getPort(), config.isTlsEnabled() ? "https" : "http")) - .toArray(HttpHost[]::new)) - .setMaxRetryTimeoutMillis(toIntExact(config.getMaxRetryTime().toMillis())); + .toArray(HttpHost[]::new)); builder.setHttpClientConfigCallback(ignored -> { RequestConfig requestConfig = RequestConfig.custom() @@ -283,7 +282,7 @@ private static Optional buildSslContext( return Optional.of(createSSLContext(keyStorePath, keyStorePassword, trustStorePath, trustStorePassword)); } catch (GeneralSecurityException | IOException e) { - throw new TrinoException(ELASTICSEARCH_SSL_INITIALIZATION_FAILURE, e); + throw new TrinoException(OPENSEARCH_SSL_INITIALIZATION_FAILURE, e); } } @@ -371,10 +370,10 @@ public boolean indexExists(String index) if (e.getResponse().getStatusLine().getStatusCode() == 404) { return false; } - throw new TrinoException(ELASTICSEARCH_CONNECTION_ERROR, e); + throw new TrinoException(OPENSEARCH_CONNECTION_ERROR, e); } catch (IOException e) { - throw new TrinoException(ELASTICSEARCH_CONNECTION_ERROR, e); + throw new TrinoException(OPENSEARCH_CONNECTION_ERROR, e); } } @@ -400,7 +399,7 @@ public List getIndexes() return result.build(); } catch (IOException e) { - throw new TrinoException(ELASTICSEARCH_INVALID_RESPONSE, e); + throw new TrinoException(OPENSEARCH_INVALID_RESPONSE, e); } }); } @@ -424,7 +423,7 @@ public Map> getAliases() return result.buildOrThrow(); } catch (IOException e) { - throw new TrinoException(ELASTICSEARCH_INVALID_RESPONSE, e); + throw new TrinoException(OPENSEARCH_INVALID_RESPONSE, e); } }); } @@ -465,7 +464,7 @@ public IndexMetadata getIndexMetadata(String index) return new IndexMetadata(parseType(mappings.get("properties"), metaProperties)); } catch (IOException e) { - throw new TrinoException(ELASTICSEARCH_INVALID_RESPONSE, e); + throw new TrinoException(OPENSEARCH_INVALID_RESPONSE, e); } }); } @@ -494,7 +493,7 @@ private IndexMetadata.ObjectType parseType(JsonNode properties, JsonNode metaPro // this route, as it will likely lead to confusion in dealing with array syntax in Trino and potentially nested array and other // syntax when parsing the raw json. if (isArray && asRawJson) { - throw new TrinoException(ELASTICSEARCH_INVALID_METADATA, + throw new TrinoException(OPENSEARCH_INVALID_METADATA, format("A column, (%s) cannot be declared as a Trino array and also be rendered as json.", name)); } @@ -551,7 +550,7 @@ public String executeQuery(String index, String query) new BasicHeader("Accept-Encoding", "application/json")); } catch (IOException e) { - throw new TrinoException(ELASTICSEARCH_CONNECTION_ERROR, e); + throw new TrinoException(OPENSEARCH_CONNECTION_ERROR, e); } String body; @@ -559,7 +558,7 @@ public String executeQuery(String index, String query) body = EntityUtils.toString(response.getEntity()); } catch (IOException e) { - throw new TrinoException(ELASTICSEARCH_INVALID_RESPONSE, e); + throw new TrinoException(OPENSEARCH_INVALID_RESPONSE, e); } return body; @@ -603,9 +602,9 @@ public SearchResponse beginSearch(String index, int shard, QueryBuilder query, O return client.search(request); } catch (IOException e) { - throw new TrinoException(ELASTICSEARCH_CONNECTION_ERROR, e); + throw new TrinoException(OPENSEARCH_CONNECTION_ERROR, e); } - catch (ElasticsearchStatusException e) { + catch (OpenSearchStatusException e) { Throwable[] suppressed = e.getSuppressed(); if (suppressed.length > 0) { Throwable cause = suppressed[0]; @@ -614,7 +613,7 @@ public SearchResponse beginSearch(String index, int shard, QueryBuilder query, O } } - throw new TrinoException(ELASTICSEARCH_CONNECTION_ERROR, e); + throw new TrinoException(OPENSEARCH_CONNECTION_ERROR, e); } finally { searchStats.add(Duration.nanosSince(start)); @@ -633,7 +632,7 @@ public SearchResponse nextPage(String scrollId) return client.searchScroll(request); } catch (IOException e) { - throw new TrinoException(ELASTICSEARCH_CONNECTION_ERROR, e); + throw new TrinoException(OPENSEARCH_CONNECTION_ERROR, e); } finally { nextPageStats.add(Duration.nanosSince(start)); @@ -663,7 +662,7 @@ public long count(String index, int shard, QueryBuilder query) throw propagate(e); } catch (IOException e) { - throw new TrinoException(ELASTICSEARCH_CONNECTION_ERROR, e); + throw new TrinoException(OPENSEARCH_CONNECTION_ERROR, e); } try { @@ -671,7 +670,7 @@ public long count(String index, int shard, QueryBuilder query) .getCount(); } catch (IOException e) { - throw new TrinoException(ELASTICSEARCH_INVALID_RESPONSE, e); + throw new TrinoException(OPENSEARCH_INVALID_RESPONSE, e); } } finally { @@ -687,7 +686,7 @@ public void clearScroll(String scrollId) client.clearScroll(request); } catch (IOException e) { - throw new TrinoException(ELASTICSEARCH_CONNECTION_ERROR, e); + throw new TrinoException(OPENSEARCH_CONNECTION_ERROR, e); } } @@ -729,7 +728,7 @@ private T doRequest(String path, ResponseHandler handler) .performRequest("GET", path); } catch (IOException e) { - throw new TrinoException(ELASTICSEARCH_CONNECTION_ERROR, e); + throw new TrinoException(OPENSEARCH_CONNECTION_ERROR, e); } String body; @@ -737,7 +736,7 @@ private T doRequest(String path, ResponseHandler handler) body = EntityUtils.toString(response.getEntity()); } catch (IOException e) { - throw new TrinoException(ELASTICSEARCH_INVALID_RESPONSE, e); + throw new TrinoException(OPENSEARCH_INVALID_RESPONSE, e); } return handler.process(body); @@ -755,17 +754,17 @@ private static TrinoException propagate(ResponseException exception) .path("reason"); if (!reason.isMissingNode()) { - throw new TrinoException(ELASTICSEARCH_QUERY_FAILURE, reason.asText(), exception); + throw new TrinoException(OPENSEARCH_QUERY_FAILURE, reason.asText(), exception); } } catch (IOException e) { - TrinoException result = new TrinoException(ELASTICSEARCH_QUERY_FAILURE, exception); + TrinoException result = new TrinoException(OPENSEARCH_QUERY_FAILURE, exception); result.addSuppressed(e); throw result; } } - throw new TrinoException(ELASTICSEARCH_QUERY_FAILURE, exception); + throw new TrinoException(OPENSEARCH_QUERY_FAILURE, exception); } @VisibleForTesting diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/ArrayDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/ArrayDecoder.java index 2c6bd89b500cc..3f26319851ad0 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/ArrayDecoder.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/ArrayDecoder.java @@ -18,7 +18,7 @@ import io.trino.plugin.elasticsearch.DecoderDescriptor; import io.trino.spi.block.ArrayBlockBuilder; import io.trino.spi.block.BlockBuilder; -import org.elasticsearch.search.SearchHit; +import org.opensearch.search.SearchHit; import java.util.List; import java.util.function.Supplier; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/BigintDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/BigintDecoder.java index 5543361744ac7..8fcc25ef2eff0 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/BigintDecoder.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/BigintDecoder.java @@ -18,7 +18,7 @@ import io.trino.plugin.elasticsearch.DecoderDescriptor; import io.trino.spi.TrinoException; import io.trino.spi.block.BlockBuilder; -import org.elasticsearch.search.SearchHit; +import org.opensearch.search.SearchHit; import java.util.function.Supplier; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/BooleanDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/BooleanDecoder.java index 80895325e19d8..8be94c6d3cc8f 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/BooleanDecoder.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/BooleanDecoder.java @@ -18,7 +18,7 @@ import io.trino.plugin.elasticsearch.DecoderDescriptor; import io.trino.spi.TrinoException; import io.trino.spi.block.BlockBuilder; -import org.elasticsearch.search.SearchHit; +import org.opensearch.search.SearchHit; import java.util.function.Supplier; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/Decoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/Decoder.java index 8588a5b1300c1..26fef13fff0cf 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/Decoder.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/Decoder.java @@ -14,7 +14,7 @@ package io.trino.plugin.elasticsearch.decoders; import io.trino.spi.block.BlockBuilder; -import org.elasticsearch.search.SearchHit; +import org.opensearch.search.SearchHit; import java.util.function.Supplier; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/DoubleDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/DoubleDecoder.java index c0c27083c8cce..d978e2f7e19ef 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/DoubleDecoder.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/DoubleDecoder.java @@ -18,7 +18,7 @@ import io.trino.plugin.elasticsearch.DecoderDescriptor; import io.trino.spi.TrinoException; import io.trino.spi.block.BlockBuilder; -import org.elasticsearch.search.SearchHit; +import org.opensearch.search.SearchHit; import java.util.function.Supplier; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/IdColumnDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/IdColumnDecoder.java index f99c42da63f68..e3808d9333a21 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/IdColumnDecoder.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/IdColumnDecoder.java @@ -16,7 +16,7 @@ import io.airlift.slice.Slices; import io.trino.plugin.elasticsearch.DecoderDescriptor; import io.trino.spi.block.BlockBuilder; -import org.elasticsearch.search.SearchHit; +import org.opensearch.search.SearchHit; import java.util.function.Supplier; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/IntegerDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/IntegerDecoder.java index 3581075310290..2cb6660569f42 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/IntegerDecoder.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/IntegerDecoder.java @@ -18,7 +18,7 @@ import io.trino.plugin.elasticsearch.DecoderDescriptor; import io.trino.spi.TrinoException; import io.trino.spi.block.BlockBuilder; -import org.elasticsearch.search.SearchHit; +import org.opensearch.search.SearchHit; import java.util.function.Supplier; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/IpAddressDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/IpAddressDecoder.java index bf2612b61f160..83ec1c33d8c56 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/IpAddressDecoder.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/IpAddressDecoder.java @@ -22,7 +22,7 @@ import io.trino.spi.TrinoException; import io.trino.spi.block.BlockBuilder; import io.trino.spi.type.Type; -import org.elasticsearch.search.SearchHit; +import org.opensearch.search.SearchHit; import java.util.function.Supplier; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/RawJsonDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/RawJsonDecoder.java index 141113b393889..f0c56a7d7b4f1 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/RawJsonDecoder.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/RawJsonDecoder.java @@ -22,7 +22,7 @@ import io.trino.plugin.elasticsearch.DecoderDescriptor; import io.trino.spi.TrinoException; import io.trino.spi.block.BlockBuilder; -import org.elasticsearch.search.SearchHit; +import org.opensearch.search.SearchHit; import java.util.function.Supplier; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/RealDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/RealDecoder.java index c9a09131a1842..1c068074cd552 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/RealDecoder.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/RealDecoder.java @@ -18,7 +18,7 @@ import io.trino.plugin.elasticsearch.DecoderDescriptor; import io.trino.spi.TrinoException; import io.trino.spi.block.BlockBuilder; -import org.elasticsearch.search.SearchHit; +import org.opensearch.search.SearchHit; import java.util.function.Supplier; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/RowDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/RowDecoder.java index abcaf7b3e75b2..8b5a32db772af 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/RowDecoder.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/RowDecoder.java @@ -19,7 +19,7 @@ import io.trino.spi.TrinoException; import io.trino.spi.block.BlockBuilder; import io.trino.spi.block.RowBlockBuilder; -import org.elasticsearch.search.SearchHit; +import org.opensearch.search.SearchHit; import java.util.List; import java.util.Map; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/ScoreColumnDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/ScoreColumnDecoder.java index 55722e31cab49..405d28e5f15e9 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/ScoreColumnDecoder.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/ScoreColumnDecoder.java @@ -15,7 +15,7 @@ import io.trino.plugin.elasticsearch.DecoderDescriptor; import io.trino.spi.block.BlockBuilder; -import org.elasticsearch.search.SearchHit; +import org.opensearch.search.SearchHit; import java.util.function.Supplier; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/SmallintDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/SmallintDecoder.java index 1e2f87e1b2729..ef5c69a0a89dd 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/SmallintDecoder.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/SmallintDecoder.java @@ -18,7 +18,7 @@ import io.trino.plugin.elasticsearch.DecoderDescriptor; import io.trino.spi.TrinoException; import io.trino.spi.block.BlockBuilder; -import org.elasticsearch.search.SearchHit; +import org.opensearch.search.SearchHit; import java.util.function.Supplier; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/SourceColumnDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/SourceColumnDecoder.java index b08b173dd79aa..9a8e0d571ec2a 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/SourceColumnDecoder.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/SourceColumnDecoder.java @@ -16,7 +16,7 @@ import io.airlift.slice.Slices; import io.trino.plugin.elasticsearch.DecoderDescriptor; import io.trino.spi.block.BlockBuilder; -import org.elasticsearch.search.SearchHit; +import org.opensearch.search.SearchHit; import java.util.function.Supplier; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/TimestampDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/TimestampDecoder.java index 5d3480586f07d..524d818816eda 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/TimestampDecoder.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/TimestampDecoder.java @@ -19,8 +19,8 @@ import io.trino.plugin.elasticsearch.DecoderDescriptor; import io.trino.spi.TrinoException; import io.trino.spi.block.BlockBuilder; -import org.elasticsearch.common.document.DocumentField; -import org.elasticsearch.search.SearchHit; +import org.opensearch.common.document.DocumentField; +import org.opensearch.search.SearchHit; import java.time.Instant; import java.time.LocalDateTime; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/TinyintDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/TinyintDecoder.java index ac6c5f3802e21..bb3296cec0e84 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/TinyintDecoder.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/TinyintDecoder.java @@ -18,7 +18,7 @@ import io.trino.plugin.elasticsearch.DecoderDescriptor; import io.trino.spi.TrinoException; import io.trino.spi.block.BlockBuilder; -import org.elasticsearch.search.SearchHit; +import org.opensearch.search.SearchHit; import java.util.function.Supplier; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/VarbinaryDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/VarbinaryDecoder.java index a81ce6a78c361..2d4c5c99300c9 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/VarbinaryDecoder.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/VarbinaryDecoder.java @@ -19,7 +19,7 @@ import io.trino.plugin.elasticsearch.DecoderDescriptor; import io.trino.spi.TrinoException; import io.trino.spi.block.BlockBuilder; -import org.elasticsearch.search.SearchHit; +import org.opensearch.search.SearchHit; import java.util.Base64; import java.util.function.Supplier; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/VarcharDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/VarcharDecoder.java index 70dd2d1755385..6e4b2b6ae385f 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/VarcharDecoder.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/VarcharDecoder.java @@ -19,7 +19,7 @@ import io.trino.plugin.elasticsearch.DecoderDescriptor; import io.trino.spi.TrinoException; import io.trino.spi.block.BlockBuilder; -import org.elasticsearch.search.SearchHit; +import org.opensearch.search.SearchHit; import java.util.function.Supplier; diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/BaseOpenSearchConnectorTest.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/BaseOpenSearchConnectorTest.java index b8723d88e40a7..34bce5b7fcf1a 100644 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/BaseOpenSearchConnectorTest.java +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/BaseOpenSearchConnectorTest.java @@ -27,22 +27,21 @@ import io.trino.testing.TestingConnectorBehavior; import io.trino.tpch.TpchTable; import org.apache.http.HttpHost; -import org.apache.http.entity.ContentType; -import org.apache.http.nio.entity.NStringEntity; -import org.elasticsearch.client.RestClient; -import org.elasticsearch.client.RestHighLevelClient; import org.intellij.lang.annotations.Language; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.TestInstance; +import org.opensearch.client.Request; +import org.opensearch.client.RestClient; +import org.opensearch.client.RestHighLevelClient; import java.io.IOException; import java.time.LocalDateTime; import java.util.List; import java.util.Map; -import static io.trino.plugin.elasticsearch.OpenSearchQueryRunner.createElasticsearchQueryRunner; +import static io.trino.plugin.elasticsearch.OpenSearchQueryRunner.createOpenSearchQueryRunner; import static io.trino.spi.type.DoubleType.DOUBLE; import static io.trino.spi.type.VarcharType.VARCHAR; import static io.trino.testing.MaterializedResult.resultBuilder; @@ -59,7 +58,7 @@ public abstract class BaseOpenSearchConnectorTest { private final String image; private final String catalogName; - private OpenSearchServer elasticsearch; + private OpenSearchServer opensearch; protected RestHighLevelClient client; BaseOpenSearchConnectorTest(String image, String catalogName) @@ -72,13 +71,12 @@ public abstract class BaseOpenSearchConnectorTest protected QueryRunner createQueryRunner() throws Exception { - elasticsearch = new OpenSearchServer(image, ImmutableMap.of()); - - HostAndPort address = elasticsearch.getAddress(); + opensearch = new OpenSearchServer(image, false, ImmutableMap.of()); + HostAndPort address = opensearch.getAddress(); client = new RestHighLevelClient(RestClient.builder(new HttpHost(address.getHost(), address.getPort()))); - return createElasticsearchQueryRunner( - elasticsearch.getAddress(), + return createOpenSearchQueryRunner( + opensearch.getAddress(), TpchTable.getTables(), ImmutableMap.of(), ImmutableMap.of(), @@ -90,8 +88,8 @@ protected QueryRunner createQueryRunner() public final void destroy() throws IOException { - elasticsearch.stop(); - elasticsearch = null; + opensearch.stop(); + opensearch = null; client.close(); client = null; } @@ -124,10 +122,10 @@ protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior) /** * This method overrides the default values used for the data provider * of the test {@link AbstractTestQueries#testLargeIn(int)} by taking - * into account that by default Elasticsearch supports only up to `1024` + * into account that by default Opensearch 2.x supports only up to `1024` * clauses in query. *

- * Consult `index.query.bool.max_clause_count` elasticsearch.yml setting + * Consult `index.query.bool.max_clause_count` opensearch.yml setting * for more details. * * @return the amount of clauses to be used in large queries @@ -1910,53 +1908,63 @@ protected void assertTableDoesNotExist(String name) assertQueryFails("SELECT * FROM " + name, ".*Table '" + catalogName + ".tpch." + name + "' does not exist"); } - protected abstract String indexEndpoint(String index, String docId); + protected String indexEndpoint(String index, String docId) + { + return format("/%s/_doc/%s", index, docId); + } private void index(String index, Map document) throws IOException { String json = new ObjectMapper().writeValueAsString(document); String endpoint = format("%s?refresh", indexEndpoint(index, String.valueOf(System.nanoTime()))); - client.getLowLevelClient() - .performRequest("PUT", endpoint, ImmutableMap.of(), new NStringEntity(json, ContentType.APPLICATION_JSON)); + + Request request = new Request("PUT", endpoint); + request.setJsonEntity(json); + client.getLowLevelClient().performRequest(request); } private void addAlias(String index, String alias) throws IOException { - client.getLowLevelClient() - .performRequest("PUT", format("/%s/_alias/%s", index, alias)); + Request request = new Request("PUT", format("/%s/_alias/%s", index, alias)); + client.getLowLevelClient().performRequest(request); refreshIndex(alias); } - protected abstract String indexMapping(@Language("JSON") String properties); + protected String indexMapping(@Language("JSON") String properties) + { + return "{\"mappings\": " + properties + "}"; + } private void createIndex(String indexName) throws IOException { - client.getLowLevelClient().performRequest("PUT", "/" + indexName); + Request request = new Request("PUT", "/" + indexName); + client.getLowLevelClient().performRequest(request); } private void createIndex(String indexName, @Language("JSON") String properties) throws IOException { String mappings = indexMapping(properties); - client.getLowLevelClient() - .performRequest("PUT", "/" + indexName, ImmutableMap.of(), new NStringEntity(mappings, ContentType.APPLICATION_JSON)); + Request request = new Request("PUT", "/" + indexName); + request.setJsonEntity(mappings); + client.getLowLevelClient().performRequest(request); } private void refreshIndex(String index) throws IOException { - client.getLowLevelClient() - .performRequest("GET", format("/%s/_refresh", index)); + Request request = new Request("GET", format("/%s/_refresh", index)); + client.getLowLevelClient().performRequest(request); } private void deleteIndex(String indexName) throws IOException { - client.getLowLevelClient() - .performRequest("DELETE", "/" + indexName); + Request request = new Request("DELETE", "/" + indexName); + client.getLowLevelClient().performRequest(request); } } diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/OpenSearchLoader.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/OpenSearchLoader.java index 2e11226d3b983..5ecf706264ecc 100644 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/OpenSearchLoader.java +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/OpenSearchLoader.java @@ -22,11 +22,12 @@ import io.trino.spi.type.VarcharType; import io.trino.testing.AbstractTestingTrinoClient; import io.trino.testing.ResultsSession; -import org.elasticsearch.action.bulk.BulkRequest; -import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.support.WriteRequest; -import org.elasticsearch.client.RestHighLevelClient; -import org.elasticsearch.common.xcontent.XContentBuilder; +import org.opensearch.action.bulk.BulkRequest; +import org.opensearch.action.index.IndexRequest; +import org.opensearch.action.support.WriteRequest; +import org.opensearch.client.RequestOptions; +import org.opensearch.client.RestHighLevelClient; +import org.opensearch.core.xcontent.XContentBuilder; import java.io.IOException; import java.io.UncheckedIOException; @@ -42,7 +43,7 @@ import static io.trino.spi.type.DoubleType.DOUBLE; import static io.trino.spi.type.IntegerType.INTEGER; import static java.util.Objects.requireNonNull; -import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; +import static org.opensearch.common.xcontent.XContentFactory.jsonBuilder; public class OpenSearchLoader extends AbstractTestingTrinoClient @@ -66,15 +67,15 @@ public OpenSearchLoader( public ResultsSession getResultSession(Session session) { requireNonNull(session, "session is null"); - return new ElasticsearchLoadingSession(); + return new OpenSearchLoadingSession(); } - private class ElasticsearchLoadingSession + private class OpenSearchLoadingSession implements ResultsSession { private final AtomicReference> types = new AtomicReference<>(); - private ElasticsearchLoadingSession() {} + private OpenSearchLoadingSession() {} @Override public void addResults(QueryStatusInfo statusInfo, QueryData data) @@ -100,7 +101,7 @@ public void addResults(QueryStatusInfo statusInfo, QueryData data) } dataBuilder.endObject(); - request.add(new IndexRequest(tableName, "doc").source(dataBuilder)); + request.add(new IndexRequest(tableName).source(dataBuilder)); } catch (IOException e) { throw new UncheckedIOException("Error loading data into Elasticsearch index: " + tableName, e); @@ -109,7 +110,7 @@ public void addResults(QueryStatusInfo statusInfo, QueryData data) request.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); try { - client.bulk(request); + client.bulk(request, RequestOptions.DEFAULT); } catch (IOException e) { throw new RuntimeException(e); diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/OpenSearchQueryRunner.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/OpenSearchQueryRunner.java index a7355242c4eff..0f0c962009fd8 100644 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/OpenSearchQueryRunner.java +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/OpenSearchQueryRunner.java @@ -26,14 +26,14 @@ import io.trino.testing.TestingTrinoClient; import io.trino.tpch.TpchTable; import org.apache.http.HttpHost; -import org.elasticsearch.client.RestClient; -import org.elasticsearch.client.RestHighLevelClient; +import org.opensearch.client.RestClient; +import org.opensearch.client.RestHighLevelClient; import java.util.Map; import static io.airlift.testing.Closeables.closeAllSuppress; import static io.airlift.units.Duration.nanosSince; -import static io.trino.plugin.elasticsearch.OpenSearchServer.ELASTICSEARCH_7_IMAGE; +import static io.trino.plugin.elasticsearch.OpenSearchServer.OPENSEARCH_IMAGE; import static io.trino.plugin.tpch.TpchMetadata.TINY_SCHEMA_NAME; import static io.trino.testing.TestingSession.testSessionBuilder; import static java.lang.String.format; @@ -44,7 +44,7 @@ public final class OpenSearchQueryRunner { static { Logging logging = Logging.initialize(); - logging.setLevel("org.elasticsearch.client.RestClient", Level.OFF); + logging.setLevel("org.opensearch.client.RestClient", Level.OFF); } private OpenSearchQueryRunner() {} @@ -52,7 +52,7 @@ private OpenSearchQueryRunner() {} private static final Logger LOG = Logger.get(OpenSearchQueryRunner.class); private static final String TPCH_SCHEMA = "tpch"; - public static DistributedQueryRunner createElasticsearchQueryRunner( + public static DistributedQueryRunner createOpenSearchQueryRunner( HostAndPort address, Iterable> tables, Map extraProperties, @@ -60,10 +60,10 @@ public static DistributedQueryRunner createElasticsearchQueryRunner( int nodeCount) throws Exception { - return createElasticsearchQueryRunner(address, tables, extraProperties, extraConnectorProperties, nodeCount, "elasticsearch"); + return createOpenSearchQueryRunner(address, tables, extraProperties, extraConnectorProperties, nodeCount, "elasticsearch"); } - public static DistributedQueryRunner createElasticsearchQueryRunner( + public static DistributedQueryRunner createOpenSearchQueryRunner( HostAndPort address, Iterable> tables, Map extraProperties, @@ -148,8 +148,8 @@ private static void loadTpchTopic(RestHighLevelClient client, TestingTrinoClient public static void main(String[] args) throws Exception { - DistributedQueryRunner queryRunner = createElasticsearchQueryRunner( - new OpenSearchServer(ELASTICSEARCH_7_IMAGE, ImmutableMap.of()).getAddress(), + DistributedQueryRunner queryRunner = createOpenSearchQueryRunner( + new OpenSearchServer(OPENSEARCH_IMAGE, false, ImmutableMap.of()).getAddress(), TpchTable.getTables(), ImmutableMap.of("http-server.http.port", "8080"), ImmutableMap.of(), diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/OpenSearchServer.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/OpenSearchServer.java index ecccdd95fd831..45fc8296980e1 100644 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/OpenSearchServer.java +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/OpenSearchServer.java @@ -32,25 +32,26 @@ public class OpenSearchServer { - public static final String ELASTICSEARCH_7_IMAGE = "elasticsearch:7.0.0"; + public static final String OPENSEARCH_IMAGE = "opensearchproject/opensearch:2.11.0"; private final Path configurationPath; private final ElasticsearchContainer container; - public OpenSearchServer(String image, Map configurationFiles) + public OpenSearchServer(String image, boolean secured, Map configurationFiles) throws IOException { - this(Network.SHARED, image, configurationFiles); + this(Network.SHARED, image, secured, configurationFiles); } - public OpenSearchServer(Network network, String image, Map configurationFiles) + public OpenSearchServer(Network network, String image, boolean secured, Map configurationFiles) throws IOException { DockerImageName dockerImageName = DockerImageName.parse(image).asCompatibleSubstituteFor("docker.elastic.co/elasticsearch/elasticsearch"); container = new ElasticsearchContainer(dockerImageName); container.withNetwork(network); - container.withNetworkAliases("elasticsearch-server"); - container.withEnv("DISABLE_SECURITY_PLUGIN", "true"); // Required for OpenSearch container + container.withReuse(false); + container.withNetworkAliases("opensearch-server"); + container.withEnv("DISABLE_SECURITY_PLUGIN", Boolean.toString(!secured)); configurationPath = createTempDirectory(null); for (Map.Entry entry : configurationFiles.entrySet()) { @@ -59,7 +60,7 @@ public OpenSearchServer(Network network, String image, Map confi Path path = configurationPath.resolve(name); Files.writeString(path, contents, UTF_8); - container.withCopyFileToContainer(forHostPath(path), "/usr/share/elasticsearch/config/" + name); + container.withCopyFileToContainer(forHostPath(path, 0777), "/usr/share/opensearch/config/" + name); } container.start(); diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearch2ConnectorTest.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearch2ConnectorTest.java new file mode 100644 index 0000000000000..be0f594a6f23c --- /dev/null +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearch2ConnectorTest.java @@ -0,0 +1,36 @@ +/* + * 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.elasticsearch; + +import com.google.common.collect.ImmutableList; +import org.junit.jupiter.api.parallel.Isolated; + +import java.util.List; + +@Isolated +public class TestOpenSearch2ConnectorTest + extends BaseOpenSearchConnectorTest +{ + public TestOpenSearch2ConnectorTest() + { + super("opensearchproject/opensearch:2.2.0", "opensearch"); + } + + @Override + protected List largeInValuesCountData() + { + // 1000 IN fails with "Query contains too many nested clauses; maxClauseCount is set to 1024" + return ImmutableList.of(200, 500); + } +} diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearchConnectorTest.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearchConnectorTest.java index 69ec6f37171f2..4452f2214cdf7 100644 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearchConnectorTest.java +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearchConnectorTest.java @@ -13,8 +13,6 @@ */ package io.trino.plugin.elasticsearch; -import static java.lang.String.format; - public class TestOpenSearchConnectorTest extends BaseOpenSearchConnectorTest { @@ -23,16 +21,4 @@ public TestOpenSearchConnectorTest() // 1.0.0 and 1.0.1 causes NotSslRecordException during the initialization super("opensearchproject/opensearch:1.1.0", "opensearch"); } - - @Override - protected String indexEndpoint(String index, String docId) - { - return format("/%s/_doc/%s", index, docId); - } - - @Override - protected String indexMapping(String properties) - { - return "{\"mappings\": " + properties + "}"; - } } diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearchQueryBuilder.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearchQueryBuilder.java index c0ea7e3da88f9..1be2c4fd9b707 100644 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearchQueryBuilder.java +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearchQueryBuilder.java @@ -22,13 +22,13 @@ import io.trino.spi.predicate.Range; import io.trino.spi.predicate.TupleDomain; import io.trino.spi.predicate.ValueSet; -import org.elasticsearch.index.query.BoolQueryBuilder; -import org.elasticsearch.index.query.ExistsQueryBuilder; -import org.elasticsearch.index.query.MatchAllQueryBuilder; -import org.elasticsearch.index.query.QueryBuilder; -import org.elasticsearch.index.query.RangeQueryBuilder; -import org.elasticsearch.index.query.TermQueryBuilder; import org.junit.jupiter.api.Test; +import org.opensearch.index.query.BoolQueryBuilder; +import org.opensearch.index.query.ExistsQueryBuilder; +import org.opensearch.index.query.MatchAllQueryBuilder; +import org.opensearch.index.query.QueryBuilder; +import org.opensearch.index.query.RangeQueryBuilder; +import org.opensearch.index.query.TermQueryBuilder; import java.util.Map; import java.util.Optional; diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestPasswordAuthentication.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestPasswordAuthentication.java index 37ac749f24b47..cd77dd08e84f1 100644 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestPasswordAuthentication.java +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestPasswordAuthentication.java @@ -22,37 +22,42 @@ import io.trino.sql.query.QueryAssertions; import io.trino.testing.DistributedQueryRunner; import org.apache.http.HttpHost; -import org.apache.http.entity.ContentType; -import org.apache.http.message.BasicHeader; -import org.apache.http.nio.entity.NStringEntity; -import org.elasticsearch.client.RestClient; -import org.elasticsearch.client.RestHighLevelClient; +import org.apache.http.impl.nio.client.HttpAsyncClientBuilder; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.TestInstance; -import org.junit.jupiter.api.parallel.Execution; +import org.junit.jupiter.api.parallel.Isolated; +import org.opensearch.client.Request; +import org.opensearch.client.RequestOptions; +import org.opensearch.client.RestClient; +import org.opensearch.client.RestHighLevelClient; +import java.io.File; import java.io.IOException; +import java.net.URISyntaxException; import java.nio.charset.StandardCharsets; +import java.security.GeneralSecurityException; +import java.util.Optional; import static com.google.common.io.Resources.getResource; import static io.airlift.testing.Closeables.closeAll; -import static io.trino.plugin.elasticsearch.OpenSearchQueryRunner.createElasticsearchQueryRunner; +import static io.trino.plugin.base.ssl.SslUtils.createSSLContext; +import static io.trino.plugin.elasticsearch.OpenSearchQueryRunner.createOpenSearchQueryRunner; +import static io.trino.plugin.elasticsearch.OpenSearchServer.OPENSEARCH_IMAGE; import static java.lang.String.format; import static java.nio.charset.StandardCharsets.UTF_8; import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; +@Isolated @TestInstance(PER_CLASS) -@Execution(CONCURRENT) public class TestPasswordAuthentication { - private static final String USER = "elastic_user"; - private static final String PASSWORD = "123456"; + private static final String USER = "admin"; + private static final String PASSWORD = "admin"; - private OpenSearchServer elasticsearch; + private OpenSearchServer opensearch; private RestHighLevelClient client; private QueryAssertions assertions; @@ -60,42 +65,60 @@ public class TestPasswordAuthentication public void setUp() throws Exception { - // We use 7.8.0 because security became a non-commercial feature in recent versions - elasticsearch = new OpenSearchServer("elasticsearch:7.8.0", ImmutableMap.builder() - .put("elasticsearch.yml", loadResource("elasticsearch.yml")) - .put("users", loadResource("users")) - .put("users_roles", loadResource("users_roles")) - .put("roles.yml", loadResource("roles.yml")) + opensearch = new OpenSearchServer(OPENSEARCH_IMAGE, true, ImmutableMap.builder() + .put("opensearch.yml", loadResource("opensearch.yml")) + .put("esnode.pem", loadResource("esnode.pem")) + .put("esnode-key.pem", loadResource("esnode-key.pem")) + .put("root-ca.pem", loadResource("root-ca.pem")) .buildOrThrow()); - HostAndPort address = elasticsearch.getAddress(); - client = new RestHighLevelClient(RestClient.builder(new HttpHost(address.getHost(), address.getPort()))); + HostAndPort address = opensearch.getAddress(); + client = new RestHighLevelClient(RestClient.builder(new HttpHost(address.getHost(), address.getPort(), "https")) + .setHttpClientConfigCallback(this::setupSslContext)); - DistributedQueryRunner runner = createElasticsearchQueryRunner( - elasticsearch.getAddress(), + DistributedQueryRunner runner = createOpenSearchQueryRunner( + opensearch.getAddress(), ImmutableList.of(), ImmutableMap.of(), ImmutableMap.builder() - .put("elasticsearch.security", "PASSWORD") - .put("elasticsearch.auth.user", USER) - .put("elasticsearch.auth.password", PASSWORD) + .put("opensearch.security", "PASSWORD") + .put("opensearch.auth.user", USER) + .put("opensearch.auth.password", PASSWORD) + .put("opensearch.tls.enabled", "true") + .put("opensearch.tls.verify-hostnames", "false") + .put("opensearch.tls.truststore-path", new File(getResource("truststore.jks").toURI()).getPath()) + .put("opensearch.tls.truststore-password", "123456") .buildOrThrow(), 3); assertions = new QueryAssertions(runner); } + private HttpAsyncClientBuilder setupSslContext(HttpAsyncClientBuilder clientBuilder) + { + try { + return clientBuilder.setSSLContext(createSSLContext( + Optional.empty(), + Optional.empty(), + Optional.of(new File(Resources.getResource("truststore.jks").toURI())), + Optional.of("123456"))); + } + catch (GeneralSecurityException | IOException | URISyntaxException e) { + throw new RuntimeException(e); + } + } + @AfterAll public final void destroy() throws IOException { closeAll( () -> assertions.close(), - () -> elasticsearch.stop(), + () -> opensearch.stop(), () -> client.close()); assertions = null; - elasticsearch = null; + opensearch = null; client = null; } @@ -105,13 +128,11 @@ public void test() { String json = new ObjectMapper().writeValueAsString(ImmutableMap.of("value", 42L)); - client.getLowLevelClient() - .performRequest( - "POST", - "/test/_doc?refresh", - ImmutableMap.of(), - new NStringEntity(json, ContentType.APPLICATION_JSON), - new BasicHeader("Authorization", format("Basic %s", Base64.encodeAsString(format("%s:%s", USER, PASSWORD).getBytes(StandardCharsets.UTF_8))))); + Request request = new Request("POST", "/test/_doc?refresh"); + request.setJsonEntity(json); + request.setOptions(RequestOptions.DEFAULT.toBuilder() + .addHeader("Authorization", format("Basic %s", Base64.encodeAsString(format("%s:%s", USER, PASSWORD).getBytes(StandardCharsets.UTF_8))))); + client.getLowLevelClient().performRequest(request); assertThat(assertions.query("SELECT * FROM test")) .matches("VALUES BIGINT '42'"); diff --git a/plugin/trino-opensearch/src/test/resources/elasticsearch.yml b/plugin/trino-opensearch/src/test/resources/elasticsearch.yml deleted file mode 100644 index eb6bb9c410c30..0000000000000 --- a/plugin/trino-opensearch/src/test/resources/elasticsearch.yml +++ /dev/null @@ -1,4 +0,0 @@ -cluster.name: "docker-cluster" -network.host: 0.0.0.0 - -xpack.security.enabled: true diff --git a/plugin/trino-opensearch/src/test/resources/esnode-key.pem b/plugin/trino-opensearch/src/test/resources/esnode-key.pem new file mode 100644 index 0000000000000..6bbe34db2e2b2 --- /dev/null +++ b/plugin/trino-opensearch/src/test/resources/esnode-key.pem @@ -0,0 +1,28 @@ +-----BEGIN PRIVATE KEY----- +MIIEvgIBADANBgkqhkiG9w0BAQEFAASCBKgwggSkAgEAAoIBAQC3lnDqYCsnVWLx +lokBBhM4kJsOc+Dwp7FQWvcEEDQYIPHBUDoE1b1uPO10iwDy41+vCd+L3tkt5vgz +hkU+W+AMYOLVJSDUTU1oaFT3gQdAPBLssMZJ+mrVVysxK/BOkB1r4GRCj3+DaL0/ +YiIm+VT4fmt1bQ46C9AqPtGBNjjAheOfBVvGsp8j9zs296OtUnlDLYhtEPF5bSpj +yTn0rlKuW9Vc1zYNFW+oshLMJxmiLcFCFzLKRKdNLGNXs0JGy7BfjOGJ7b8zgzBe +8phua7VvDoYw4Hda8z8uNSBWd0bOOE0DT4mbbiC+M0G3c76TNceeZMGlkw6g0Vt4 +G5jFYTIXAgMBAAECggEAJE/GvKzRzEFYxST7qy6q7ZS4fobP9nmMmjKryet0ilM1 +zgESwLMP+TZm1xbJVMFBQytGQnGM7kXRUMuddYSRj5E4PGVQFX3GskB0kdG8/+kB +6zvZtHcp21GW8u5YZQmfMB+dZQbh9zlnuRotXB0Tv+StV3f/CN8eSMTNBmwVlIpw +N//eAC3/F9VbY6L3EhtSWwcqONPITsfetvYMb3VMQNgdrm23xYHN3LIqJeDLxYO6 +AdasfwLqs1KEChy4y0+aST9bYvIC2Kgpjm6oT5cIOmMUUmr3194lDCGRtU3J+kQN +fqzqNTMnrummvd2QSzESRWlAXyleduj6gHR+W+wa9QKBgQC7LRgPHoYJzbys71i9 +owv3ZGG9KDBA3wnbu1oCklGL1V/x6fuw/OTBt7ajP/zRJKczcg7Po0Z//fL4y0PD +L5npej9+hAaeARHiiNi7Seo5Qb/DtWj+TcwkIWVr3SikzG/md29kfMIHiEpc2kHY +qYMw7aXHBLUkx1DnS3mVAQXZdQKBgQD7F4/77ktEsas7GxbFycQ7YNMs4UgxcjRa +ezGKmQEJhLJU799WWg0LYpHR1r1SzZ5rp6vqIo0JDwjssZu/Hfr4hMtsjlU3XL7Q +1Ne9C45BrmSJ/VKETUTel5FFU9K136fMmyxUhAxKJeBonHNkt8s6AeU1QX4qnynd +ZlmoM38f2wKBgCOFQjyUSVGQ9o5G+as7RukS5q1z8/xLl15YppM4WztTKdsJ0Vbc +Fzlf+MvwN7Uk5IoRyhfDX3yon7RxNpHS9b1mEqiau3q+rzszRxX7x/RHsvGb+qnP +776ni8y1cqIbndy5q60BM/0j8yC91qMcQCZH4x7qmMm6/XJ3U0JgqFCFAoGBALvn +/TvHMBLfd0pLUnaAMVdSdZbvnQxL8DwKV5/mMu2BnceMn7HKwFpV0NSdhJ9z+2uL +Vk9ove0akE45+FlFQW08S9QYjFNbF/3GWxFBlZs3vEu8BUW3YaPG/9NHELhFARlQ +wxMYou3WfKMs+fpUxUTK6FI7igSxii4DC3Oh3i/jAoGBAJxvc5WwNEyogvCJ7uSy +dVT2PUeGmMDuuEf3C1Flk/ShzetpuvmKuC8DQsbEopIE92tAWwf3gLVIfAmH6bQy +xyuMjjVdl6L37sHX4jUzdf8hw5OF45LgiKfTiBVl2PsD2UT9RJEisk7Z5IvvW45W +RxzOk4AwW2uSgJD8NGTDmwLU +-----END PRIVATE KEY----- diff --git a/plugin/trino-opensearch/src/test/resources/esnode.pem b/plugin/trino-opensearch/src/test/resources/esnode.pem new file mode 100644 index 0000000000000..6b033c7f8de27 --- /dev/null +++ b/plugin/trino-opensearch/src/test/resources/esnode.pem @@ -0,0 +1,21 @@ +-----BEGIN CERTIFICATE----- +MIIDYTCCAkmgAwIBAgIVALANPI8Y3YDiH0EsyazPC0a305zjMA0GCSqGSIb3DQEB +CwUAMDQxMjAwBgNVBAMTKUVsYXN0aWMgQ2VydGlmaWNhdGUgVG9vbCBBdXRvZ2Vu +ZXJhdGVkIENBMB4XDTIzMTIzMDE3MzIyM1oXDTI2MTIyOTE3MzIyM1owHzEdMBsG +A1UEAxMUZWxhc3RpY3NlYXJjaC1zZXJ2ZXIwggEiMA0GCSqGSIb3DQEBAQUAA4IB +DwAwggEKAoIBAQC3lnDqYCsnVWLxlokBBhM4kJsOc+Dwp7FQWvcEEDQYIPHBUDoE +1b1uPO10iwDy41+vCd+L3tkt5vgzhkU+W+AMYOLVJSDUTU1oaFT3gQdAPBLssMZJ ++mrVVysxK/BOkB1r4GRCj3+DaL0/YiIm+VT4fmt1bQ46C9AqPtGBNjjAheOfBVvG +sp8j9zs296OtUnlDLYhtEPF5bSpjyTn0rlKuW9Vc1zYNFW+oshLMJxmiLcFCFzLK +RKdNLGNXs0JGy7BfjOGJ7b8zgzBe8phua7VvDoYw4Hda8z8uNSBWd0bOOE0DT4mb +biC+M0G3c76TNceeZMGlkw6g0Vt4G5jFYTIXAgMBAAGjfzB9MB0GA1UdDgQWBBSA +l2wp8zAFBrBMH+Zf+tS0dznNnjAfBgNVHSMEGDAWgBTqhtqQcPqrVyIj2evQlUs3 +iON1BjAwBgNVHREEKTAngglsb2NhbGhvc3SHBH8AAAGCFGVsYXN0aWNzZWFyY2gt +c2VydmVyMAkGA1UdEwQCMAAwDQYJKoZIhvcNAQELBQADggEBACYQKkoSJrQsFWpR +zTD+5visP/W/wUeHdQxU5iEesLsd43r7jKkZLRJzZgDtPSfYw3LLL/dZHVCtzdUg +NtarP+EaMp5YlGyEvMRiaIy2pFajHN9+wElImQhi5iw5CY6R/YL6m4ap2mVmIeMi +nvWKYaibMzNx2wouahIrEvANb3N93lr8T5LDfLPPLXmHAa0ebk2GbWt90mVdk6UZ +QQqI0VK8YlzR9ybp8jJ5ZRgXIegtn91Tts164+61wQQ2M6fV1le41m/1NENAzkIW +Q7LJvvqe+Q3YO8yBvxYP7ru/kKgBWHPyof+6mocqirwvrVLb+MPL+sKolcuXP0sS +c52vk0Q= +-----END CERTIFICATE----- diff --git a/plugin/trino-opensearch/src/test/resources/opensearch.yml b/plugin/trino-opensearch/src/test/resources/opensearch.yml new file mode 100644 index 0000000000000..d158eb73c4b06 --- /dev/null +++ b/plugin/trino-opensearch/src/test/resources/opensearch.yml @@ -0,0 +1,32 @@ +cluster.name: docker-cluster + +# Bind to all interfaces because we don't know what IP address Docker will assign to us. +network.host: 0.0.0.0 + +# # minimum_master_nodes need to be explicitly set when bound on a public IP +# # set to 1 to allow single node clusters +# discovery.zen.minimum_master_nodes: 1 + +# Setting network.host to a non-loopback address enables the annoying bootstrap checks. "Single-node" mode disables them again. +# discovery.type: single-node + +######## Start OpenSearch Security Demo Configuration ######## +# WARNING: revise all the lines below before you go into production +plugins.security.ssl.transport.pemcert_filepath: esnode.pem +plugins.security.ssl.transport.pemkey_filepath: esnode-key.pem +plugins.security.ssl.transport.pemtrustedcas_filepath: root-ca.pem +plugins.security.ssl.transport.enforce_hostname_verification: false +plugins.security.ssl.http.enabled: true +plugins.security.ssl.http.pemcert_filepath: esnode.pem +plugins.security.ssl.http.pemkey_filepath: esnode-key.pem +plugins.security.ssl.http.pemtrustedcas_filepath: root-ca.pem +plugins.security.ssl.http.clientauth_mode: OPTIONAL +plugins.security.allow_unsafe_democertificates: true +plugins.security.allow_default_init_securityindex: true +plugins.security.audit.type: internal_opensearch +plugins.security.enable_snapshot_restore_privilege: true +plugins.security.check_snapshot_restore_write_privileges: true +plugins.security.restapi.roles_enabled: ["all_access", "security_rest_api_access"] +plugins.security.system_indices.enabled: true +plugins.security.system_indices.indices: [".plugins-ml-model", ".plugins-ml-task", ".opendistro-alerting-config", ".opendistro-alerting-alert*", ".opendistro-anomaly-results*", ".opendistro-anomaly-detector*", ".opendistro-anomaly-checkpoints", ".opendistro-anomaly-detection-state", ".opendistro-reports-*", ".opensearch-notifications-*", ".opensearch-notebooks", ".opensearch-observability", ".opendistro-asynchronous-search-response*", ".replication-metadata-store"] +node.max_local_storage_nodes: 3 \ No newline at end of file diff --git a/plugin/trino-opensearch/src/test/resources/roles.yml b/plugin/trino-opensearch/src/test/resources/roles.yml deleted file mode 100644 index 0b1a96ec2607d..0000000000000 --- a/plugin/trino-opensearch/src/test/resources/roles.yml +++ /dev/null @@ -1,6 +0,0 @@ -admin: - cluster: - - all - indices: - - names: '*' - privileges: [ all ] diff --git a/plugin/trino-opensearch/src/test/resources/root-ca.pem b/plugin/trino-opensearch/src/test/resources/root-ca.pem new file mode 100644 index 0000000000000..4cf446a6cc277 --- /dev/null +++ b/plugin/trino-opensearch/src/test/resources/root-ca.pem @@ -0,0 +1,20 @@ +-----BEGIN CERTIFICATE----- +MIIDSTCCAjGgAwIBAgIUByq1K6VbZFocXfpnEohIw8NgEW4wDQYJKoZIhvcNAQEL +BQAwNDEyMDAGA1UEAxMpRWxhc3RpYyBDZXJ0aWZpY2F0ZSBUb29sIEF1dG9nZW5l +cmF0ZWQgQ0EwHhcNMjMxMjMwMTczMjIxWhcNMjYxMjI5MTczMjIxWjA0MTIwMAYD +VQQDEylFbGFzdGljIENlcnRpZmljYXRlIFRvb2wgQXV0b2dlbmVyYXRlZCBDQTCC +ASIwDQYJKoZIhvcNAQEBBQADggEPADCCAQoCggEBAImUzmkzHoVgREpIVHcjvtkV +oT4QjNvciH0bZOcnIUBpXq5ywBoSS3CjckF9mirkAJXZHS+M13155+JKO085Ewy7 +U684jsEVRS8GmNiKrKNYKFiwdKeQKIYtwmwgHMfJCEwcFaVOmJ2PhbiHiQj+8lMr +mRMwS+Wy7deci9uJuzykSHQ7LW7MNcZBCBkWBtctI2p4h21yYZzwn4UzsDGD7i02 +GNJ/CHG4M5QjLY9P/tCHoss9kHDUn7k/rnezk8nHZgf2XAwVYdJBbVeYvUe7HgtK ++35FeSACslOtgV2kQJpULwvh8wiqgP+/oIhNoNPW/NpyoOT4luQmJfxZV5SKJ08C +AwEAAaNTMFEwHQYDVR0OBBYEFOqG2pBw+qtXIiPZ69CVSzeI43UGMB8GA1UdIwQY +MBaAFOqG2pBw+qtXIiPZ69CVSzeI43UGMA8GA1UdEwEB/wQFMAMBAf8wDQYJKoZI +hvcNAQELBQADggEBABKr0j2hk5qzWs8uWR4yqxjL4+MQTSaBtvcMHCF6w1rCCRLt +4sIz7Gy0ecnqjfXscjrrrEA4ruYP4CpAvRXyYgmJ2F1jLYrMcmYnYupCwRl88ygO +dvjVTdAjOib0NBUsE1DA8l0idFYHipCV2izpKjF/AB1HnhWm1A++pC3nZ++x7o6u +lqOuTwEMDBMnwpc+CQpQPSa9QCIl60LNpbhH+rWAL2xm5fdLV/Hs3hFQvABGorgR +78F/kBfk30c/sM2tQog7By/ic6KIEFQhFGvDz2fAqt2BAAyO22nWEQZQI+JTBAH4 +tEBodOialM5LlPFAOU0gmr6q7g8lFIkVYwxadUw= +-----END CERTIFICATE----- diff --git a/plugin/trino-opensearch/src/test/resources/serverkey.pem b/plugin/trino-opensearch/src/test/resources/serverkey.pem new file mode 100644 index 0000000000000..6bbe34db2e2b2 --- /dev/null +++ b/plugin/trino-opensearch/src/test/resources/serverkey.pem @@ -0,0 +1,28 @@ +-----BEGIN PRIVATE KEY----- +MIIEvgIBADANBgkqhkiG9w0BAQEFAASCBKgwggSkAgEAAoIBAQC3lnDqYCsnVWLx +lokBBhM4kJsOc+Dwp7FQWvcEEDQYIPHBUDoE1b1uPO10iwDy41+vCd+L3tkt5vgz +hkU+W+AMYOLVJSDUTU1oaFT3gQdAPBLssMZJ+mrVVysxK/BOkB1r4GRCj3+DaL0/ +YiIm+VT4fmt1bQ46C9AqPtGBNjjAheOfBVvGsp8j9zs296OtUnlDLYhtEPF5bSpj +yTn0rlKuW9Vc1zYNFW+oshLMJxmiLcFCFzLKRKdNLGNXs0JGy7BfjOGJ7b8zgzBe +8phua7VvDoYw4Hda8z8uNSBWd0bOOE0DT4mbbiC+M0G3c76TNceeZMGlkw6g0Vt4 +G5jFYTIXAgMBAAECggEAJE/GvKzRzEFYxST7qy6q7ZS4fobP9nmMmjKryet0ilM1 +zgESwLMP+TZm1xbJVMFBQytGQnGM7kXRUMuddYSRj5E4PGVQFX3GskB0kdG8/+kB +6zvZtHcp21GW8u5YZQmfMB+dZQbh9zlnuRotXB0Tv+StV3f/CN8eSMTNBmwVlIpw +N//eAC3/F9VbY6L3EhtSWwcqONPITsfetvYMb3VMQNgdrm23xYHN3LIqJeDLxYO6 +AdasfwLqs1KEChy4y0+aST9bYvIC2Kgpjm6oT5cIOmMUUmr3194lDCGRtU3J+kQN +fqzqNTMnrummvd2QSzESRWlAXyleduj6gHR+W+wa9QKBgQC7LRgPHoYJzbys71i9 +owv3ZGG9KDBA3wnbu1oCklGL1V/x6fuw/OTBt7ajP/zRJKczcg7Po0Z//fL4y0PD +L5npej9+hAaeARHiiNi7Seo5Qb/DtWj+TcwkIWVr3SikzG/md29kfMIHiEpc2kHY +qYMw7aXHBLUkx1DnS3mVAQXZdQKBgQD7F4/77ktEsas7GxbFycQ7YNMs4UgxcjRa +ezGKmQEJhLJU799WWg0LYpHR1r1SzZ5rp6vqIo0JDwjssZu/Hfr4hMtsjlU3XL7Q +1Ne9C45BrmSJ/VKETUTel5FFU9K136fMmyxUhAxKJeBonHNkt8s6AeU1QX4qnynd +ZlmoM38f2wKBgCOFQjyUSVGQ9o5G+as7RukS5q1z8/xLl15YppM4WztTKdsJ0Vbc +Fzlf+MvwN7Uk5IoRyhfDX3yon7RxNpHS9b1mEqiau3q+rzszRxX7x/RHsvGb+qnP +776ni8y1cqIbndy5q60BM/0j8yC91qMcQCZH4x7qmMm6/XJ3U0JgqFCFAoGBALvn +/TvHMBLfd0pLUnaAMVdSdZbvnQxL8DwKV5/mMu2BnceMn7HKwFpV0NSdhJ9z+2uL +Vk9ove0akE45+FlFQW08S9QYjFNbF/3GWxFBlZs3vEu8BUW3YaPG/9NHELhFARlQ +wxMYou3WfKMs+fpUxUTK6FI7igSxii4DC3Oh3i/jAoGBAJxvc5WwNEyogvCJ7uSy +dVT2PUeGmMDuuEf3C1Flk/ShzetpuvmKuC8DQsbEopIE92tAWwf3gLVIfAmH6bQy +xyuMjjVdl6L37sHX4jUzdf8hw5OF45LgiKfTiBVl2PsD2UT9RJEisk7Z5IvvW45W +RxzOk4AwW2uSgJD8NGTDmwLU +-----END PRIVATE KEY----- diff --git a/plugin/trino-opensearch/src/test/resources/truststore.jks b/plugin/trino-opensearch/src/test/resources/truststore.jks new file mode 100644 index 0000000000000000000000000000000000000000..7f756310f03d1034f91d7670af70ed9546ede382 GIT binary patch literal 1809 zcmezO_TO6u1_mZLW-rRmFVRg-WME+IITh65$G{q)XKG-{z`*Ql(8TO%(8Of8fSHMr ziAjWAYpeFs=#(g#*k9>F9Ug}dCkW;l@Un4gwRyCC=VfH%W@RuiF*GtTU}FwtVHVbO z%}FdS$xK#oPAw|QOv_A8EJ;-e$Zl`k)da)$b(Y0oAghs@MlI0UL+H?=)D99W?$>Ad-y3}vR-2T=b?VTL|J_T#f z6gKdFy6NrpIo-EAciSxSD6!VfJ7apxkwa39?YgdVRz-VmQR18r^IMHK7&gDtHIukh z&r!I;c#5)azyH4r?Tb#^PAIrCf2aMr^39V^9#3Qc7Q-W&c*!w0e8yh)-E!Pszw2Bp z6__>!uWgK7;xH>jpZnowj#Ul+_b=%1U2yr@pIb!>p8S~hL{0TiWcZXWb$=#iMh3>k z!3Kc_vcMRVTTZfxRlL69we>IB4HrbfL#GUNP#dT z<9`-b17;wF9PGeU0StCV2BFoLY!^~af1u^CSM*a7XXlOBWZkYaMQQ3uubd7_)_m6B zEBkTP_kyyO+7}a19pQijpJJ6t!q~eeOueP^Qp>QOLDqeq=&~#P`&b%hb7NGnnn1%7j7Ke~j25h9W((BJW{jHIjH3+X4P_0aAz4@?6_SOE zQxl7lGjxkni^@`qfcXc1zS=&m;8lXQdT7$eX`PI0!WI)|^A$h%uzX`c)OQvE6A6Wn z2Lr5FuI|mVd0W!W@ab{uR{Rq4o#A39{J_Qk?qhn=|&>Cn>4@4Ea%)U7E{h#eG4`?qZ{v#evqx^I4;h zZJMwA-P-K?;lSD=zqR**&0K)s1GCqFqcCnLYOq@AT6n0}hD#|(DD*e z+oB_y6?oR*-?JZU?7!|m=-yt+6Y@+^Zo_Wb$5p?3R!Zs$6{j)0wN<}yxahS0_ej}* zwP&v?m|a_K|4_Xh^ z;$x+GU%L`l%r-VIyv?PTC8RC%fj7Un_FmK<|4D~yHlNq6Y-e06m*?A-n_YV;HFok+ zNk^`Zi$QymVlIAHd-=(zGF3uc@rCaEyTRMeRy}^V^&pFx@$#$J!gpND|7zmwaL!50 zdDF>#zvlhqyJ`2HpN~`1G$sE2f^FZtpEr eJa_$Mm#jAyQvO1(X1!~YT5^5*f9X3eneze3!obY{ literal 0 HcmV?d00001 diff --git a/plugin/trino-opensearch/src/test/resources/users b/plugin/trino-opensearch/src/test/resources/users deleted file mode 100644 index 0a5fe3f1cbd1b..0000000000000 --- a/plugin/trino-opensearch/src/test/resources/users +++ /dev/null @@ -1 +0,0 @@ -elastic_user:$2a$10$tbO62EbOfqMezJDBDWlxbuvIleeYeNlw30F5OgWMXzi1R8aXqnVni diff --git a/plugin/trino-opensearch/src/test/resources/users_roles b/plugin/trino-opensearch/src/test/resources/users_roles deleted file mode 100644 index 33f5f17c8aeef..0000000000000 --- a/plugin/trino-opensearch/src/test/resources/users_roles +++ /dev/null @@ -1 +0,0 @@ -admin:elastic_user From 64dfb8fd52297f83b0e9c09a7d538b3599c6b993 Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Tue, 2 Jan 2024 15:28:47 +0100 Subject: [PATCH 297/350] Enable testing of trino-opensearch module --- .github/workflows/ci.yml | 2 ++ .../launcher/env/environment/EnvMultinodeAllConnectors.java | 1 + .../conf/environment/multinode-all/opensearch.properties | 4 ++++ 3 files changed, 7 insertions(+) create mode 100644 testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-all/opensearch.properties diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 29644bba1f268..1b2ddb4c9317f 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -407,6 +407,7 @@ jobs: !:trino-memory, !:trino-mongodb, !:trino-mysql, + !:trino-opensearch, !:trino-oracle, !:trino-orc, !:trino-parquet, @@ -533,6 +534,7 @@ jobs: - { modules: plugin/trino-mariadb } - { modules: plugin/trino-mongodb } - { modules: plugin/trino-mysql } + - { modules: plugin/trino-opensearch } - { modules: plugin/trino-oracle } - { modules: plugin/trino-phoenix5 } - { modules: plugin/trino-pinot } diff --git a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/EnvMultinodeAllConnectors.java b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/EnvMultinodeAllConnectors.java index 0ba9cf6edd05b..12c958581ee87 100644 --- a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/EnvMultinodeAllConnectors.java +++ b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/EnvMultinodeAllConnectors.java @@ -69,6 +69,7 @@ public void extendEnvironment(Environment.Builder builder) "singlestore", "mongodb", "mysql", + "opensearch", "oracle", "phoenix5", "pinot", diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-all/opensearch.properties b/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-all/opensearch.properties new file mode 100644 index 0000000000000..255cfc60ef9ef --- /dev/null +++ b/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-all/opensearch.properties @@ -0,0 +1,4 @@ +connector.name=opensearch +opensearch.host=host1.invalid +opensearch.port=9200 +opensearch.default-schema-name=default From 576151c082c97d9b350b50ff688a3b0e57ed235b Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Tue, 2 Jan 2024 17:31:32 +0100 Subject: [PATCH 298/350] Add OpenSearch connector docs --- docs/src/main/sphinx/connector.md | 1 + docs/src/main/sphinx/connector/opensearch.md | 441 ++++++++++++++++++ .../src/main/sphinx/static/img/opensearch.png | Bin 0 -> 8614 bytes 3 files changed, 442 insertions(+) create mode 100644 docs/src/main/sphinx/connector/opensearch.md create mode 100644 docs/src/main/sphinx/static/img/opensearch.png diff --git a/docs/src/main/sphinx/connector.md b/docs/src/main/sphinx/connector.md index 3b86e28f3d563..c741fdedb5479 100644 --- a/docs/src/main/sphinx/connector.md +++ b/docs/src/main/sphinx/connector.md @@ -29,6 +29,7 @@ MariaDB Memory MongoDB MySQL +OpenSearch Oracle Phoenix Pinot diff --git a/docs/src/main/sphinx/connector/opensearch.md b/docs/src/main/sphinx/connector/opensearch.md new file mode 100644 index 0000000000000..544ce9bb22756 --- /dev/null +++ b/docs/src/main/sphinx/connector/opensearch.md @@ -0,0 +1,441 @@ +# OpenSearch connector + +```{raw} html + +``` + +The OpenSearch connector allows access to [OpenSearch](https://opensearch.org/) data from Trino. +This document describes how to configure a catalog with the OpenSearch connector to run SQL queries against OpenSearch. + +:::{note} +OpenSearch (1.1.0 or later) is required. +::: + +## Configuration + +To configure the OpenSearch connector, create a catalog properties file +`etc/catalog/example.properties` with the following content, replacing the +properties as appropriate for your setup: + +```text +connector.name=opensearch +opensearch.host=localhost +opensearch.port=9200 +opensearch.default-schema-name=default +``` + +### Configuration properties + +:::{list-table} OpenSearch configuration properties +:widths: 35, 55, 10 +:header-rows: 1 + +* - Property name + - Description + - Default +* - `opensearch.host` + - The comma-separated list of host names for the OpenSearch cluster to connect + to. This property is required. + - +* - `opensearch.port` + - Port to use to connect to OpenSearch. + - `9200` +* - `opensearch.default-schema-name` + - The schema that contains all tables defined without a qualifying schema + name. + - `default` +* - `opensearch.scroll-size` + - Sets the maximum number of hits that can be returned with each OpenSearch + scroll request. + - `1000` +* - `opensearch.scroll-timeout` + - Duration for OpenSearch to keep the [search context](https://opensearch.org/docs/latest/api-reference/scroll/) + alive for scroll requests. + - `1m` +* - `opensearch.request-timeout` + - Timeout duration for all OpenSearch requests. + - `10s` +* - `opensearch.connect-timeout` + - Timeout duration for all OpenSearch connection attempts. + - `1s` +* - `opensearch.backoff-init-delay` + - The minimum duration between backpressure retry attempts for a single + request to OpenSearch. Setting it too low can overwhelm an already + struggling cluster. + - `500ms` +* - `opensearch.backoff-max-delay` + - The maximum duration between backpressure retry attempts for a single + request. + - `20s` +* - `opensearch.max-retry-time` + - The maximum duration across all retry attempts for a single request. + - `20s` +* - `opensearch.node-refresh-interval` + - Duration between requests to refresh the list of available OpenSearch nodes. + - `1m` +* - `opensearch.ignore-publish-address` + - Disables using the address published by OpenSearch to connect for + queries. + - +::: + +### TLS security + +The connector provides additional security options to connect to OpenSearch clusters with TLS enabled. + +If your cluster uses globally-trusted certificates, you only need to +enable TLS. If you require custom configuration for certificates, the connector +supports key stores and trust stores in PEM or Java Key Store (JKS) format. + +The available configuration values are listed in the following table: + +:::{list-table} TLS configuration properties +:widths: 40, 60 +:header-rows: 1 + +* - Property name + - Description +* - `opensearch.tls.enabled` + - Enable TLS security. Defaults to `false`. +* - `opensearch.tls.keystore-path` + - The path to the [PEM](/security/inspect-pem) or [JKS](/security/inspect-jks) + key store. +* - `opensearch.tls.truststore-path` + - The path to [PEM](/security/inspect-pem) or [JKS](/security/inspect-jks) + trust store. +* - `opensearch.tls.keystore-password` + - The password for the key store specified by + `opensearch.tls.keystore-path`. +* - `opensearch.tls.truststore-password` + - The password for the trust store specified by + `opensearch.tls.truststore-path`. +* - `opensearch.tls.verify-hostnames` + - Flag to determine if the hostnames in the certificates must be verified. Defaults + to `true`. +::: + +(opensearch-type-mapping)= +## Type mapping + +Because Trino and OpenSearch each support types that the other does not, this +connector [maps some types](type-mapping-overview) when reading data. + +### OpenSearch type to Trino type mapping + +The connector maps OpenSearch types to the corresponding Trino types +according to the following table: + +:::{list-table} OpenSearch type to Trino type mapping +:widths: 30, 30, 50 +:header-rows: 1 + +* - OpenSearch type + - Trino type + - Notes +* - `BOOLEAN` + - `BOOLEAN` + - +* - `DOUBLE` + - `DOUBLE` + - +* - `FLOAT` + - `REAL` + - +* - `BYTE` + - `TINYINT` + - +* - `SHORT` + - `SMALLINT` + - +* - `INTEGER` + - `INTEGER` + - +* - `LONG` + - `BIGINT` + - +* - `KEYWORD` + - `VARCHAR` + - +* - `TEXT` + - `VARCHAR` + - +* - `DATE` + - `TIMESTAMP` + - For more information, see [](opensearch-date-types). +* - `IPADDRESS` + - `IP` + - +::: + +No other types are supported. + +(opensearch-array-types)= +### Array types + +Fields in OpenSearch can contain [zero or more values](https://opensearch.org/docs/latest/field-types/supported-field-types/date/#custom-formats) +, but there is no dedicated array type. To indicate a field contains an array, it can be annotated in a Trino-specific structure in +the [\_meta](https://opensearch.org/docs/latest/field-types/index/#get-a-mapping) section of the index mapping. + +For example, you can have an OpenSearch index that contains documents with the following structure: + +```json +{ + "array_string_field": ["trino","the","lean","machine-ohs"], + "long_field": 314159265359, + "id_field": "564e6982-88ee-4498-aa98-df9e3f6b6109", + "timestamp_field": "1987-09-17T06:22:48.000Z", + "object_field": { + "array_int_field": [86,75,309], + "int_field": 2 + } +} +``` + +The array fields of this structure can be defined by using the following command to add the field +property definition to the `_meta.trino` property of the target index mapping. + +```shell +curl --request PUT \ + --url localhost:9200/doc/_mapping \ + --header 'content-type: application/json' \ + --data ' +{ + "_meta": { + "trino":{ + "array_string_field":{ + "isArray":true + }, + "object_field":{ + "array_int_field":{ + "isArray":true + } + }, + } + } +}' +``` + +:::{note} +It is not allowed to use `asRawJson` and `isArray` flags simultaneously for the same column. +::: + +(opensearch-date-types)= +### Date types + +OpenSearch supports a wide array of [date] formats including +[built-in date formats] and also [custom date formats]. +The OpenSearch connector supports only the default `date` type. All other +date formats including [built-in date formats] and [custom date formats] are +not supported. Dates with the [format] property are ignored. + +### Raw JSON transform + +There are many occurrences where documents in OpenSearch have more complex +structures that are not represented in the mapping. For example, a single +`keyword` field can have widely different content including a single +`keyword` value, an array, or a multidimensional `keyword` array with any +level of nesting. + +```shell +curl --request PUT \ + --url localhost:9200/doc/_mapping \ + --header 'content-type: application/json' \ + --data ' +{ + "properties": { + "array_string_field":{ + "type": "keyword" + } + } +}' +``` + +Notice for the `array_string_field` that all the following documents are legal +for OpenSearch. See the [OpenSearch array documentation](https://opensearch.org/docs/latest/field-types/supported-field-types/index/#arrays) +for more details. + +```json +[ + { + "array_string_field": "trino" + }, + { + "array_string_field": ["trino","is","the","best"] + }, + { + "array_string_field": ["trino",["is","the","best"]] + }, + { + "array_string_field": ["trino",["is",["the","best"]]] + } +] +``` + +Further, OpenSearch supports types, such as +[k-NN vector](https://opensearch.org/docs/latest/field-types/supported-field-types/knn-vector/), +that are not supported in Trino. New types are constantly emerging which can +cause parsing exceptions for users that use of these types in OpenSearch. To +manage all of these scenarios, you can transform fields to raw JSON by +annotating it in a Trino-specific structure in the [\_meta](https://opensearch.org/docs/latest/field-types/index/) +section of the OpenSearch index mapping. This indicates to Trino that the field, and all +nested fields beneath, need to be cast to a `VARCHAR` field that contains +the raw JSON content. These fields can be defined by using the following command +to add the field property definition to the `_meta.trino` property of the +target index mapping. + +```shell +curl --request PUT \ + --url localhost:9200/doc/_mapping \ + --header 'content-type: application/json' \ + --data ' +{ + "_meta": { + "trino":{ + "array_string_field":{ + "asRawJson":true + } + } + } +}' +``` + +The preceding configurations causes Trino to return the `array_string_field` +field as a `VARCHAR` containing raw JSON. You can parse these fields with the +[built-in JSON functions](/functions/json). + +:::{note} +It is not allowed to use `asRawJson` and `isArray` flags simultaneously for the same column. +::: + +## Special columns + +The following hidden columns are available: + +| Column | Description | +|----------|-----------------------------------------------------| +| \_id | The OpenSearch document ID | +| \_score | The document score returned by the OpenSearch query | +| \_source | The source of the original document | + +(opensearch-full-text-queries)= +## Full text queries + +Trino SQL queries can be combined with OpenSearch queries by providing the [full text query] +as part of the table name, separated by a colon. For example: + +```sql +SELECT * FROM "tweets: +trino SQL^2" +``` + +## Predicate push down + +The connector supports [predicate push down](predicate-pushdown) for the following data types: + +| OpenSearch | Trino | Supported | +|--------------|---------------|---------------| +| `binary` | `VARBINARY` | `NO` | +| `boolean` | `BOOLEAN` | `YES` | +| `double` | `DOUBLE` | `YES` | +| `float` | `REAL` | `YES` | +| `byte` | `TINYINT` | `YES` | +| `short` | `SMALLINT` | `YES` | +| `integer` | `INTEGER` | `YES` | +| `long` | `BIGINT` | `YES` | +| `keyword` | `VARCHAR` | `YES` | +| `text` | `VARCHAR` | `NO` | +| `date` | `TIMESTAMP` | `YES` | +| `ip` | `IPADDRESS` | `NO` | +| (all others) | (unsupported) | (unsupported) | + +## Security +The connection to OpenSearch can use AWS or password authentication. + +### AWS authorization + +To enable AWS authorization using IAM policies, the `opensearch.security` option needs to be set to `AWS`. +Additionally, the following options must be configured: + +| Property name | Description | +|------------------------------|-------------------------------------------------------------------------------------------------------------------------------------| +| `opensearch.aws.region` | AWS region of the OpenSearch endpoint. This option is required. | +| `opensearch.aws.access-key` | AWS access key to use to connect to the OpenSearch domain. If not set, the default AWS credentials provider chain is used. | +| `opensearch.aws.secret-key` | AWS secret key to use to connect to the OpenSearch domain. If not set, the default AWS credentials provider chain is used. | +| `opensearch.aws.iam-role` | Optional ARN of an IAM role to assume to connect to OpenSearch. Note that the configured IAM user must be able to assume this role. | +| `opensearch.aws.external-id` | Optional external ID to pass while assuming an AWS IAM role. | + +### Password authentication + +To enable password authentication, the `opensearch.security` option must be set to `PASSWORD`. +Additionally the following options must be configured: + +| Property name | Description | +|----------------------------|--------------------------------------------| +| `opensearch.auth.user` | User name to use to connect to OpenSearch. | +| `opensearch.auth.password` | Password to use to connect to OpenSearch. | + +(opensearch-sql-support)= + +## SQL support + +The connector provides [globally available](sql-globally-available) and +[read operation](sql-read-operations) statements to access data and +metadata in the OpenSearch catalog. + +## Table functions + +The connector provides specific {doc}`table functions ` to +access OpenSearch. + +(opensearch-raw-query-function)= +### `raw_query(varchar) -> table` + +The `raw_query` function allows you to query the underlying database directly. +This function requires [OpenSearch Query DSL](https://opensearch.org/docs/latest/query-dsl/index/) +syntax, because the full query is pushed down and processed in OpenSearch. +This can be useful for accessing native features which are not available in +Trino, or for improving query performance in situations where running a query +natively may be faster. + +```{include} query-passthrough-warning.fragment +``` + +The `raw_query` function requires three parameters: + +- `schema`: The schema in the catalog that the query is to be executed on. +- `index`: The index in OpenSearch to search. +- `query`: The query to execute, written in OpenSearch [Query DSL](https://opensearch.org/docs/latest/query-dsl). + +Once executed, the query returns a single row containing the resulting JSON +payload returned by OpenSearch. + +For example, query the `example` catalog and use the `raw_query` table +function to search for documents in the `orders` index where the country name +is `ALGERIA`: + +``` +SELECT + * +FROM + TABLE( + example.system.raw_query( + schema => 'sales', + index => 'orders', + query => '{ + "query": { + "match": { + "name": "ALGERIA" + } + } + }' + ) + ); +``` + +```{include} query-table-function-ordering.fragment +``` + +[built-in date formats]: https://opensearch.org/docs/latest/field-types/supported-field-types/date/#custom-formats +[custom date formats]: https://opensearch.org/docs/latest/field-types/supported-field-types/date/#custom-formats +[date]: https://opensearch.org/docs/latest/field-types/supported-field-types/date/ +[format]: https://opensearch.org/docs/latest/query-dsl/term/range/#format +[full text query]: https://opensearch.org/docs/latest/query-dsl/full-text/query-string/ diff --git a/docs/src/main/sphinx/static/img/opensearch.png b/docs/src/main/sphinx/static/img/opensearch.png new file mode 100644 index 0000000000000000000000000000000000000000..113d451b346735e1b007f4e129acb870583bcd87 GIT binary patch literal 8614 zcmY*-bzB_0674Q5?u%1gireC@#ofBNTPd!EQrz98xKmt9DemqL#oZkW6dt|zd*6L8 z`6Y8QIddkH|B^5jC22HdB4hvnfF>&=srG8k{tiU=*V$n+N#oUkEX5SX0DzhplqX}j z*EPARjG7_<;Q1B+@C^h29$sC(`v8C|8vt-<1OV`-0s#098O^GKuN#5pP+1E_MF7Jq zMg+hC!2sA-1bpoPAQ9lN+A9Lc0pI+K)qr&Wa9{v{5NiPFACAte{R_$ePXAii9N_;H zb71~uo8`d%55GnLJljuFUJa6ijJ7iXz$E{701JhZOJAk5tu>%7P(=lPQ+rz$V>5db za~2O-pQPt zn}v;qjZzqyoSawznhyt7~jx@9H8%N%=R?zwN*0bg{Ph zUnV=}f696-koE5lD?1At>%ZEsp@M(C{K`(&=C7Im>I<_A{)hSh`Tofx$oe<=|8toC zR{9_BYpKG>f~@~un=mq=!3F{VKt?MoDW>58+|ftz&@)ea9!ry-s#nyN`;jzc_YN>+So60spE0M19#vi$PXu33_ERTuqh#6&h3WYA8QO8P+fs-!Li5^e> zoA`-(Zlf?Boo%{H<=*k*mzRveG(F3uv_Xv*K@R8L5x$olua~n2pO-M2B(=k%fT_ti zDR;e-sq$Tp9$7=mWG$d)c5}#ZVkL)_c!T4-a zIq5EcNF*_(odOQyZM(w#WTh!tRn!n@CU2wm2+2Qy)T}6PG>Q2eW0L8+_$9#VGz)v` z9P=Bv0a)L(1EoXe3-30D>+Wqcd8b&ED_7%4@~sg%%tXhZRrnC7&_Zc$Zh#I_auy>Z zicF6J9<990CHWiP0i9C3Zq+s(^g$2 zr6=(`Ri~VsQ9Zkvgrp1Zc;9P(_Oef;q3!U*swk9dW^uXz-!|l2oHLq9w9sS{4TnT8 zwdLN1Hu?n9GDrs$QbCitGa{}-ztKR2AY*DOio{{s>AD`Ol|&hx&;jiv-44xX0fls& zW9wn}pv)G5S=X0GFOOz1tWaE|j1MAfe}&Ug!G;!dSg;g$Jp35?GGF+SmY|Sfow!<# z+-ymvkQ9^lhH&vIUmqN{05-0ps=6M=knsjx3Tz%kR-`s;0TEr_(RZE)WPrv&)EegU z4CwNs50Qd3hjy(UU+JF9hCl1siMHJ8rTthh2A6xb7sNY`tbBvSe9dEmgQYCzf1BAN zZNfEQDokjMrG=`UDZ*^6s#-Lj>Qe&+`T{3Unsc8R>6Vg+-3fvviRSc>S88Rxd2$bz z@A8xiQxr$4(6f@0++?KH8;HPZD_Mcrj`;gktn$#UWzPw+@8C+78&pvc+47J1w&QW6C#neariz$xVqZeqa8$ zXX-?aP%e-oZ(lQfg8gIlaS_p2gBPzgvq%=TUDzRD zL}wvK^#FX7a0yF!3heE^l!Tt`GoFWDcY?$ncS0cEVikwj^qKn%_~dq4&&`)|7TWoj zPIH;AqVuwWoO4m;AItePJ+WK z#Q=^GtlPz>JjMww55`f(X)2F3Fv$DcZ;f3YgLd#J)fpzuTP$rmy{HJZE|40$1&->? z?ixjg6YE$dMbBJ9VeG|*Y+B+)n6L3iNIW7GDwiW2pHvS!`F^^kg=rOJ?Ky=mEdop@ zAFMAoq?I^qapH6p>QiP{%WZa(d!ccT1d66fPj(VD-4Xr7=nL}o`ETWmO1w|us5eb! zQ9Ijhe8zaEqreewjwsb6-z1IU3au*IWOdRh^t^wUQi+G>YD7~y5+AD#iDlgB+I5}` zX4DI3F*$%UQ!Xpz)t>ctvlvSM@l#P?IDCQl+mtq3-Nl9GTIagm3@Wz3KsBhLC=JwW zTa((L`dvK5x5jo)r^3TfU-iZo9y7FXE(UVe2P3kAs2T&RkH_7QM>K2{taD6$*e7`NPoSvvc`~@N> zb0L{kO||7|>ojsojlw8gJ$#aR;x&vt4uKdbxzL`Anw9>hn-x%(8KnPq z?DHhOSpP7MX=?utwG=MFlnOPHz!!sFKJrBl{`oshA}V@Y{mSL+P3Ed1dY zej?(f$=7)$c%jw8HXs6&t4JZ=yH&K40!ISR+dsSH)7M?P|6ryBTCx-BpPivJm*4&h+% zC=L7&9dKhL3FvKv^{tQf9x})=q|9N<&5<-!^crb&K(O?o)kn4GiBty5NQ*R*dn=4%Yd=8 za$Yv}@qOa-QfYy&2>{|OuR|Ingcogot)2;5hoUO7j};vzgPS2^c>Sy4gh5a=_g*#{ z5D}yA%Dl^_GN4sSYxFc(>Uca>b=rW{ax@!|33D}L5URca0@)b& zO}5hF*nTFFmF+Y{?vnJ;R6G5_po9`nyU$Ta7IA*f8d z`j9@}cfI%hDWaxc@&ndK;+SPOd0JlXZ1Ht7!c_Vxd1WE|$;d1IC?g^hOMqBKJvI2( zeZ5_L&O0>tR>hG@>(6>BsuinYTx>(8vltKOKV z7@f`3bV1?lIyxCYjTMe3VcpGq>%Opj(0SpW@yj$V%><|22^c6Q^JOOI(enWiG*UXY z`!(-i5)*JElPzP+xXlk)A4Av+qq0pY>kfS@J%MveT$@dtH{Xe-wIAmMOhe8X!pxbN z`^Q+nZTQuIRAY;7cL0~)!IfVyopaUG=Y3KX!LiejXM|`WqkHV|AngP>YGZKsbN^d8 ztr^cRYJ9zVjvreSoLguBPGd!yF+Yr|+%NYPPx%gOPypV)$zhVXG0*KyZZwuPasDoYiepK$Mh$W$?g&1hqIlY#^=df;kOZG zgr_)E41K3NbImA5eWxlfc|6n{3+sq5Ph1OF(NT)h-H8JbiDFR?V$5>(_uaj0aD2Uh z+4!*SaQ1p9)nKZR1bFy8_)oa~OOz3$UsA$Z&&y`)nRMypTPxnWNZHYE=6-KKp}aK$uqAcf7{ z@?B#swEmTH?b2UJZ~fbNKNUV$z;TbMG@kjcmEmHtj=U?25R9_`l`3h{Urz{-3u)#j z3qtFxJzFHVttE2z-{nimV-YvLbFfGdx@TU4Im21*C^;|d`$ai=at5=WojAA1Y6Zs` zWi765Kx$KXGS=iYlsp$WW5|L3bb(X5UQ8Jon3|g2`;0Z&g)HjR;&-G$A=C46Mc(gL zA_Y_xq)WW^U8gKKbAv@J>(5C*StOrLn+652&w9L<>&j<+SJ)o6+o)@P(U+jHfsN{< zu=OX%VJ%iM`9#_Yx%WK+Z?CJcuP@`<><1ffT4=C@8EcD6H-&_2tzEn-63bkg)udR$ zmiZVK?!$KSBRp@9Sfy>KXQ0X_d{oVb-%*g5y0?>2GXw(E8|>LB_nmL8Km3&UcVpz; zP5MX!vLNHvcuAdRrAuus@kbX_;r{g#H|>l^md)i&a;g~F==&=x5c2*;MA05~<+Jd) z=xyZailt&89cWQKQDHTZ%eU#0h|kQhPyPKWVN)Z;I3+8(*l|}>ZL7eJ!he( zlq2BC+mV9ojT>ie$6on9$gqZD&D&{7rBBs9ysfcw@=nZem8m4(C&v78XhTQpjuI%q z4;y&L2 zLGT&=dK9vsHd;-_ExK2gQmQP%id95vj?7oI1%87BTe%W5P>h8TFwT3{ruJoSgHTtM zX+p@c4;Et6&rRnc*{3d*GEu5r>@Lsyqp+E`oKT zv_*htW{tL3-DQWS7+n%Q6 zT&a4duwXy@jFhTPWmcowbC^IvxgKBi$X){obwaSj)H-`V;e0#vK85Z+q5L6Bl5F?0cnF`-UQ!U80Nz@g(XY*dMskDz)hwSpHBwI zh4;LOZ)7h;9PnvtHo)`mj4)}webPn^K-Ig=P#D}8$Q@L7xHEGgZ1V(1lk)F(ZpYWd z#+kdnh-FOr^!d-20lI@1jOdXCg76WCg4lGJSqxUnND)aePQ{4gxiIq3Fs>qAr0^O< z!W+`jz$n5%2TmSYfp-Bpu0&EAZ;e_456?GjlD?C3c%judd_fzO(ZuqYOqZzxxs^pi-hzwCh_;oC6TYr zcq=v5vxZ#gyF#UQ)olu?>65HuE`@arSQwb0u5)`$tV1tjhY;A>f#{F0bq#tPtlzIb zrsPE({YGElLE_*CI>LXc@T?2$`;3~o3Q)=FK2onW(yw68r%&l*FJv)jOSL^w-7LJA z5#X8lHma1KNO4P^ne#j$8@R?i+ysmrZYzrtuHEL#x&kbQ5{5Sk=0f76*3iuPN1`LV zDdPzJbFJj9*D?}OYTPO{W>J|%VF6zaf`6i9GZ&PhzcHI0qD^A! zc0^#7PTrM%8_)>#%KfHF-&qHOw^91moxO|Xd=bvrxOq>g2q4eaP%?g@BfH-;iFOo) z-Jn+!I@JZE2t|y(!s3^cwiuV($5^tSDQ3cX zpIor@NMmHq+XI6Ay%jT?xYfbWV1cx@FD5oBgrxF;CNm+q|a= z1>OpaSpFq|U)ih?OOeQuXC6%8>ia9?29yfh==~P*v)P!{=pIYZ1{XbPly0$x>VU{3 zaFM<9P&Y1g(x^}U<2#Hl!-Y@*uzeJ>Dfa(C9fG-zS1wz&_wm;^QsIhpldnCRkMT}U zmj?9;%_`fd%4ETQ^-x73gYkw+;@L%(+QCS4{D=4Om`+F%xObf734;dF4bJ`?O?%cl zr3LPlW<{Memcu3NFW*QQZv#veOhA?%UyLS{DfRJ1p72&C0qUgR#kX(pr1RUjVXBxu zwCTd~YE}Nhb&$p9KAULEkMUu`k=sf{GG2bkBiUV&zmWf8RQ-zsazv)o3}^EW+IZ6| zUy@HB1ct6VA0yO%Emlz1x2tcRcZ6?m`d#T%aE;tg?UUa~V zLVnw$T?;bu4O~J~CMu+c25jGUwbUr#<=d=v}kxx9AO(En~L^$C($xVVw5sQH?hSIjv+` zsA6JNS-jUQm7&Tb!Vt)1K^Lt_6ar1{jEwyswM`zoN2<~ErCmy4E;4r3#*;gNby^B% z8b>Sq`|Zc2ugxcaG&$Q&5_9ZkA8bZFK z)e%dte>*krO#KW@Lbyx$L>9(?4HiGM=hf3=xY_P@5fE)Q8m+>sX3?NBj;r)xStroe z043G_8nPqn?j0{@F;uw{vV&%r3X)&5I-BURq#5NA^Vvuu`;`jZNI!au&46071Q|~D z3AoNYW%9W(T|`axzhS!`XQjnxwG4h!j%*x2@18}t!kCx7QteEd!t1DzUQ`ljx!Fw{ zL$b?e9s>A9Njjot*^GVskyHcvy;NkqHn7~_l*sc)E-FPr8}5decvy+@3?JHY;Sta7 zT;#3~B`}T20BqS3A-oc$s_ND4kJ{*ezB2C;MiAbYOz6X({$gK2Ws)1-?m%Q>z@vRO z-H+n;SoOYvFO*Wj=IZ$qT7Ivh@Hp2$GRyzjUU=>@b98S4G*ObO$AsESGIx6?no3B- zAhVTrU{Q2$DKFfB=c$Mq7x*EJWB_#F;boswIWctEY*2@3tt5fKCKN`s`-efJH zi$c1o9ua!lSqPoGmutf=bU?pY${$5zs?VK88xPR6$TNKjuI@E(0VxM{Cpe8HR|6}M zs?w>RdXBsEa5{NEuk#`G?b!%r{3yiH1WBNDSlPYsJM|;N_s=7sUPnzv-FMy~!=Le) z+Z(v>I-yTSo3G9kja8p>*~A4AJn+se(E>)OQwK54R&;Z1?mmvcT+TyQ*pC31p^pyC zxyTV#HLw`6u9qLITfa$T@43XIz8}SU&f>zsiS!9c`mMRmTqm;X(Gp&R3TTZBZ$-@* zOGrv-*OFNaheiC^wOD6qAnJ>vDYy^5d}Q?T z8976S(Z-s;Sfmt#s)tF)*f~(uei)lyNW7;F%%Z#e+WuVB_!4OKOgRE|q9@7|>+Y(g zDpI;y?sOj1H+!mT-dt4h2&5FO+N2dV;6e4mvILv*uYTOQOTV~*vOfY&y?G3ZP2olr zn6CDzV>70d`Ig>Lvvi9R;+mgk71?cSITW$$-*seu|cOyQ>63#p@f%}@5@j}Fyn`rbHR1M@?zS8lZo-MH6o2yWHa z>G|Fqog%!w4~S)S<)w@pw$8h;j8FAjd`}3Bf<3ff&OIf$zQk66&z9f$6ngtGPPs#c zn05V~EOx!GLMD=81+-+iT|RXUZXUG7*b`HQ zY<}9SY0%8swWrGKX}+7if|gszcVYEoct5`+f0fhnjZggIig<6 zGsCf2Gpz8KMt+McB>~plGX8UwgfJ0_Aozu#<9=h(DfoQ@7VPTx_G0BjN{bLlNT;f> z3a#xLOliHScxv%Vw#jX|ir*u0M%y~wHg10h*Vt1e7hK1{;GcP(&pf-X_jmmiq?nO| zH4`O>A?K;iNARK9>=SS+EPy@rr0?0GDpM1Lhkov@{+OGOW_*-uAMiL3a(O9l&On$0 zZo9&cQ9W1MGmfqzAaRzs!=*tPOy5__jg_*vU#SP`Rh@WmOlQ{~`V zO*=@GLLRBZsKMurZwYpIMgPjl5L$8;?eob*TTyAXty#&1k7jET@wUpBoQ-F2GCj(=WF39Itlx&i&eodKV zSh_7Q6tX4jVK`oP40SK=TDam8&7MMy)8d)I&_9GL_1U`nx=m16=~L#alV*v&ABM34 zRUO#Zv7!Sdwn&*%ZJC27OETVDk+guU7P3mRYYK6$Vv3>9c%c}Wj>&jgPjJNsar z#>+s8_yqmn;DLpAI&c{6y-G{XYHgx$CbRt40M5-pr-iGCB`x{UFce6P+Nd5zI-Rzt z0{9nZ*7n<=i>cgmMGOefZxPQkQ#H_?+LV)AZ?(zA`{scW@&+*HK$L^ zzuww?B+F&wIyxIL`z;Q(@D!Tx1xS)j)>n$Cb@qB2{U3~(l*GeeSW60;KU{zq2iVy z8gqGx@V!8F0iXg=UNmWaZC7X5XOQ}8JxNRsKFQQe1G*#@e^u^EZMDAX$Eq!Z3^{OA zMYa}<*PvD*KGSMFWb2o|?nhnF9ZXWT-(vi|1J8;Hzl@^)Y-m}cOUCrRClCxAVyWc{8E zF@_dYVAw6eZHU0@<=d0(_yxaUR{mVfO_s7o_q*~n37>0#=eft2){%@k^!8+?W9@}Z XFZ@xuDdzQ={MuySDM?m}8~Ohq>59R$ literal 0 HcmV?d00001 From 1a9bcd4bb768b6411cc905e1d2b1eca03bbcdadc Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Tue, 2 Jan 2024 19:12:48 +0100 Subject: [PATCH 299/350] Add testing coverage against latest OpenSearch server --- .../TestOpenSearchLatestConnectorTest.java | 36 +++++++++++++++++++ 1 file changed, 36 insertions(+) create mode 100644 plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearchLatestConnectorTest.java diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearchLatestConnectorTest.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearchLatestConnectorTest.java new file mode 100644 index 0000000000000..096ace3be1b57 --- /dev/null +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearchLatestConnectorTest.java @@ -0,0 +1,36 @@ +/* + * 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.elasticsearch; + +import com.google.common.collect.ImmutableList; +import org.junit.jupiter.api.parallel.Isolated; + +import java.util.List; + +@Isolated +public class TestOpenSearchLatestConnectorTest + extends BaseOpenSearchConnectorTest +{ + public TestOpenSearchLatestConnectorTest() + { + super("opensearchproject/opensearch:latest", "opensearch"); + } + + @Override + protected List largeInValuesCountData() + { + // 1000 IN fails with "Query contains too many nested clauses; maxClauseCount is set to 1024" + return ImmutableList.of(200, 500); + } +} From 0c11680dd8bbcbe7a725f8f77a607b9da7fcfed3 Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Tue, 9 Jan 2024 11:39:13 +0100 Subject: [PATCH 300/350] Move OpenSearch plugin to it's own package --- .../AwsSecurityConfig.java | 2 +- .../BuiltinColumns.java | 8 +-- .../CountQueryPageSource.java | 7 ++- .../DecoderDescriptor.java | 38 ++++++------- .../NodesSystemTable.java | 6 +-- .../OpenSearchColumnHandle.java | 2 +- .../OpenSearchConfig.java | 2 +- .../OpenSearchConnector.java | 2 +- .../OpenSearchConnectorFactory.java | 2 +- .../OpenSearchConnectorModule.java | 12 ++--- .../OpenSearchErrorCode.java | 2 +- .../OpenSearchMetadata.java | 50 +++++++++-------- .../OpenSearchPageSourceProvider.java | 6 +-- .../OpenSearchPlugin.java | 2 +- .../OpenSearchQueryBuilder.java | 2 +- .../OpenSearchSplit.java | 2 +- .../OpenSearchSplitManager.java | 7 ++- .../OpenSearchTableHandle.java | 2 +- .../OpenSearchTransactionHandle.java | 2 +- .../PassthroughQueryPageSource.java | 4 +- .../PasswordConfig.java | 2 +- .../ScanQueryPageSource.java | 13 +++-- .../client/AwsRequestSigner.java | 2 +- .../client/BackpressureRestClient.java | 4 +- .../BackpressureRestHighLevelClient.java | 4 +- .../client/CountResponse.java | 2 +- .../client/IndexMetadata.java | 2 +- .../client/NodesResponse.java | 2 +- .../client/OpenSearchClient.java | 54 +++++++++---------- .../client/OpenSearchNode.java | 2 +- .../client/SearchShardsResponse.java | 2 +- .../client/Shard.java | 2 +- .../decoders/ArrayDecoder.java | 4 +- .../decoders/BigintDecoder.java | 4 +- .../decoders/BooleanDecoder.java | 4 +- .../decoders/Decoder.java | 2 +- .../decoders/DoubleDecoder.java | 4 +- .../decoders/IdColumnDecoder.java | 4 +- .../decoders/IntegerDecoder.java | 4 +- .../decoders/IpAddressDecoder.java | 4 +- .../decoders/RawJsonDecoder.java | 4 +- .../decoders/RealDecoder.java | 4 +- .../decoders/RowDecoder.java | 8 +-- .../decoders/ScoreColumnDecoder.java | 4 +- .../decoders/SmallintDecoder.java | 4 +- .../decoders/SourceColumnDecoder.java | 4 +- .../decoders/TimestampDecoder.java | 4 +- .../decoders/TinyintDecoder.java | 4 +- .../decoders/VarbinaryDecoder.java | 4 +- .../decoders/VarcharDecoder.java | 4 +- .../ptf/RawQuery.java | 11 ++-- .../BaseOpenSearchConnectorTest.java | 4 +- .../OpenSearchLoader.java | 2 +- .../OpenSearchQueryRunner.java | 4 +- .../OpenSearchServer.java | 2 +- .../TestAwsSecurityConfig.java | 2 +- .../TestOpenSearch2ConnectorTest.java | 2 +- .../TestOpenSearchConfig.java | 4 +- .../TestOpenSearchConnectorTest.java | 2 +- .../TestOpenSearchLatestConnectorTest.java | 2 +- .../TestOpenSearchMetadata.java | 2 +- .../TestOpenSearchQueryBuilder.java | 11 ++-- .../TestPasswordAuthentication.java | 6 +-- .../TestPasswordConfig.java | 2 +- .../client/TestExtractAddress.java | 4 +- 65 files changed, 186 insertions(+), 199 deletions(-) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/AwsSecurityConfig.java (98%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/BuiltinColumns.java (91%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/CountQueryPageSource.java (88%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/DecoderDescriptor.java (66%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/NodesSystemTable.java (96%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/OpenSearchColumnHandle.java (98%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/OpenSearchConfig.java (99%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/OpenSearchConnector.java (98%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/OpenSearchConnectorFactory.java (98%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/OpenSearchConnectorModule.java (86%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/OpenSearchErrorCode.java (97%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/OpenSearchMetadata.java (94%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/OpenSearchPageSourceProvider.java (93%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/OpenSearchPlugin.java (97%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/OpenSearchQueryBuilder.java (99%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/OpenSearchSplit.java (98%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/OpenSearchSplitManager.java (90%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/OpenSearchTableHandle.java (99%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/OpenSearchTransactionHandle.java (94%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/PassthroughQueryPageSource.java (95%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/PasswordConfig.java (97%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/ScanQueryPageSource.java (95%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/client/AwsRequestSigner.java (98%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/client/BackpressureRestClient.java (98%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/client/BackpressureRestHighLevelClient.java (98%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/client/CountResponse.java (95%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/client/IndexMetadata.java (98%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/client/NodesResponse.java (98%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/client/OpenSearchClient.java (93%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/client/OpenSearchNode.java (96%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/client/SearchShardsResponse.java (98%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/client/Shard.java (96%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/decoders/ArrayDecoder.java (95%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/decoders/BigintDecoder.java (96%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/decoders/BooleanDecoder.java (96%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/decoders/Decoder.java (94%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/decoders/DoubleDecoder.java (96%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/decoders/IdColumnDecoder.java (92%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/decoders/IntegerDecoder.java (96%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/decoders/IpAddressDecoder.java (97%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/decoders/RawJsonDecoder.java (96%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/decoders/RealDecoder.java (96%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/decoders/RowDecoder.java (92%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/decoders/ScoreColumnDecoder.java (91%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/decoders/SmallintDecoder.java (96%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/decoders/SourceColumnDecoder.java (92%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/decoders/TimestampDecoder.java (97%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/decoders/TinyintDecoder.java (96%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/decoders/VarbinaryDecoder.java (95%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/decoders/VarcharDecoder.java (95%) rename plugin/trino-opensearch/src/main/java/io/trino/plugin/{elasticsearch => opensearch}/ptf/RawQuery.java (94%) rename plugin/trino-opensearch/src/test/java/io/trino/plugin/{elasticsearch => opensearch}/BaseOpenSearchConnectorTest.java (99%) rename plugin/trino-opensearch/src/test/java/io/trino/plugin/{elasticsearch => opensearch}/OpenSearchLoader.java (99%) rename plugin/trino-opensearch/src/test/java/io/trino/plugin/{elasticsearch => opensearch}/OpenSearchQueryRunner.java (98%) rename plugin/trino-opensearch/src/test/java/io/trino/plugin/{elasticsearch => opensearch}/OpenSearchServer.java (98%) rename plugin/trino-opensearch/src/test/java/io/trino/plugin/{elasticsearch => opensearch}/TestAwsSecurityConfig.java (98%) rename plugin/trino-opensearch/src/test/java/io/trino/plugin/{elasticsearch => opensearch}/TestOpenSearch2ConnectorTest.java (96%) rename plugin/trino-opensearch/src/test/java/io/trino/plugin/{elasticsearch => opensearch}/TestOpenSearchConfig.java (97%) rename plugin/trino-opensearch/src/test/java/io/trino/plugin/{elasticsearch => opensearch}/TestOpenSearchConnectorTest.java (95%) rename plugin/trino-opensearch/src/test/java/io/trino/plugin/{elasticsearch => opensearch}/TestOpenSearchLatestConnectorTest.java (96%) rename plugin/trino-opensearch/src/test/java/io/trino/plugin/{elasticsearch => opensearch}/TestOpenSearchMetadata.java (98%) rename plugin/trino-opensearch/src/test/java/io/trino/plugin/{elasticsearch => opensearch}/TestOpenSearchQueryBuilder.java (93%) rename plugin/trino-opensearch/src/test/java/io/trino/plugin/{elasticsearch => opensearch}/TestPasswordAuthentication.java (96%) rename plugin/trino-opensearch/src/test/java/io/trino/plugin/{elasticsearch => opensearch}/TestPasswordConfig.java (97%) rename plugin/trino-opensearch/src/test/java/io/trino/plugin/{elasticsearch => opensearch}/client/TestExtractAddress.java (92%) diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/AwsSecurityConfig.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/AwsSecurityConfig.java similarity index 98% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/AwsSecurityConfig.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/AwsSecurityConfig.java index 4fe4abb0b2afd..16f86187def2b 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/AwsSecurityConfig.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/AwsSecurityConfig.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch; +package io.trino.plugin.opensearch; import io.airlift.configuration.Config; import io.airlift.configuration.ConfigDescription; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/BuiltinColumns.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/BuiltinColumns.java similarity index 91% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/BuiltinColumns.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/BuiltinColumns.java index 621d00beeccaa..57a3be5176b38 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/BuiltinColumns.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/BuiltinColumns.java @@ -11,11 +11,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch; +package io.trino.plugin.opensearch; -import io.trino.plugin.elasticsearch.decoders.IdColumnDecoder; -import io.trino.plugin.elasticsearch.decoders.ScoreColumnDecoder; -import io.trino.plugin.elasticsearch.decoders.SourceColumnDecoder; +import io.trino.plugin.opensearch.decoders.IdColumnDecoder; +import io.trino.plugin.opensearch.decoders.ScoreColumnDecoder; +import io.trino.plugin.opensearch.decoders.SourceColumnDecoder; import io.trino.spi.connector.ColumnHandle; import io.trino.spi.connector.ColumnMetadata; import io.trino.spi.type.Type; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/CountQueryPageSource.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/CountQueryPageSource.java similarity index 88% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/CountQueryPageSource.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/CountQueryPageSource.java index 22f8e72dd40fc..9487574234fbc 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/CountQueryPageSource.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/CountQueryPageSource.java @@ -11,13 +11,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch; +package io.trino.plugin.opensearch; -import io.trino.plugin.elasticsearch.client.OpenSearchClient; +import io.trino.plugin.opensearch.client.OpenSearchClient; import io.trino.spi.Page; import io.trino.spi.connector.ConnectorPageSource; -import static io.trino.plugin.elasticsearch.OpenSearchQueryBuilder.buildSearchQuery; import static java.lang.Math.toIntExact; import static java.util.Objects.requireNonNull; @@ -43,7 +42,7 @@ public CountQueryPageSource(OpenSearchClient client, OpenSearchTableHandle table long count = client.count( split.getIndex(), split.getShard(), - buildSearchQuery(table.getConstraint().transformKeys(OpenSearchColumnHandle.class::cast), table.getQuery(), table.getRegexes())); + OpenSearchQueryBuilder.buildSearchQuery(table.getConstraint().transformKeys(OpenSearchColumnHandle.class::cast), table.getQuery(), table.getRegexes())); readTimeNanos = System.nanoTime() - start; if (table.getLimit().isPresent()) { diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/DecoderDescriptor.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/DecoderDescriptor.java similarity index 66% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/DecoderDescriptor.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/DecoderDescriptor.java index adadf4036a0c1..899dedae04248 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/DecoderDescriptor.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/DecoderDescriptor.java @@ -11,28 +11,28 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch; +package io.trino.plugin.opensearch; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; -import io.trino.plugin.elasticsearch.decoders.ArrayDecoder; -import io.trino.plugin.elasticsearch.decoders.BigintDecoder; -import io.trino.plugin.elasticsearch.decoders.BooleanDecoder; -import io.trino.plugin.elasticsearch.decoders.Decoder; -import io.trino.plugin.elasticsearch.decoders.DoubleDecoder; -import io.trino.plugin.elasticsearch.decoders.IdColumnDecoder; -import io.trino.plugin.elasticsearch.decoders.IntegerDecoder; -import io.trino.plugin.elasticsearch.decoders.IpAddressDecoder; -import io.trino.plugin.elasticsearch.decoders.RawJsonDecoder; -import io.trino.plugin.elasticsearch.decoders.RealDecoder; -import io.trino.plugin.elasticsearch.decoders.RowDecoder; -import io.trino.plugin.elasticsearch.decoders.ScoreColumnDecoder; -import io.trino.plugin.elasticsearch.decoders.SmallintDecoder; -import io.trino.plugin.elasticsearch.decoders.SourceColumnDecoder; -import io.trino.plugin.elasticsearch.decoders.TimestampDecoder; -import io.trino.plugin.elasticsearch.decoders.TinyintDecoder; -import io.trino.plugin.elasticsearch.decoders.VarbinaryDecoder; -import io.trino.plugin.elasticsearch.decoders.VarcharDecoder; +import io.trino.plugin.opensearch.decoders.ArrayDecoder; +import io.trino.plugin.opensearch.decoders.BigintDecoder; +import io.trino.plugin.opensearch.decoders.BooleanDecoder; +import io.trino.plugin.opensearch.decoders.Decoder; +import io.trino.plugin.opensearch.decoders.DoubleDecoder; +import io.trino.plugin.opensearch.decoders.IdColumnDecoder; +import io.trino.plugin.opensearch.decoders.IntegerDecoder; +import io.trino.plugin.opensearch.decoders.IpAddressDecoder; +import io.trino.plugin.opensearch.decoders.RawJsonDecoder; +import io.trino.plugin.opensearch.decoders.RealDecoder; +import io.trino.plugin.opensearch.decoders.RowDecoder; +import io.trino.plugin.opensearch.decoders.ScoreColumnDecoder; +import io.trino.plugin.opensearch.decoders.SmallintDecoder; +import io.trino.plugin.opensearch.decoders.SourceColumnDecoder; +import io.trino.plugin.opensearch.decoders.TimestampDecoder; +import io.trino.plugin.opensearch.decoders.TinyintDecoder; +import io.trino.plugin.opensearch.decoders.VarbinaryDecoder; +import io.trino.plugin.opensearch.decoders.VarcharDecoder; @JsonTypeInfo( use = JsonTypeInfo.Id.NAME, diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/NodesSystemTable.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/NodesSystemTable.java similarity index 96% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/NodesSystemTable.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/NodesSystemTable.java index 092787eb52a0d..c2369f2290090 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/NodesSystemTable.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/NodesSystemTable.java @@ -11,12 +11,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch; +package io.trino.plugin.opensearch; import com.google.common.collect.ImmutableList; import com.google.inject.Inject; -import io.trino.plugin.elasticsearch.client.OpenSearchClient; -import io.trino.plugin.elasticsearch.client.OpenSearchNode; +import io.trino.plugin.opensearch.client.OpenSearchClient; +import io.trino.plugin.opensearch.client.OpenSearchNode; import io.trino.spi.Node; import io.trino.spi.NodeManager; import io.trino.spi.Page; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchColumnHandle.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchColumnHandle.java similarity index 98% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchColumnHandle.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchColumnHandle.java index 7c19d17b3378f..c3a9ac340ba71 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchColumnHandle.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchColumnHandle.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch; +package io.trino.plugin.opensearch; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchConfig.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchConfig.java similarity index 99% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchConfig.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchConfig.java index afd6d9e8f22e4..febbe8dfc0c15 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchConfig.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchConfig.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch; +package io.trino.plugin.opensearch; import io.airlift.configuration.Config; import io.airlift.configuration.ConfigDescription; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchConnector.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchConnector.java similarity index 98% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchConnector.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchConnector.java index 513a5ca6db0ae..0387fed5d8e7a 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchConnector.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchConnector.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch; +package io.trino.plugin.opensearch; import com.google.common.collect.ImmutableSet; import com.google.inject.Inject; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchConnectorFactory.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchConnectorFactory.java similarity index 98% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchConnectorFactory.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchConnectorFactory.java index 5c11c93cf9abb..7479f1cd1f07f 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchConnectorFactory.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchConnectorFactory.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch; +package io.trino.plugin.opensearch; import com.google.inject.Injector; import io.airlift.bootstrap.Bootstrap; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchConnectorModule.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchConnectorModule.java similarity index 86% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchConnectorModule.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchConnectorModule.java index 25f6bcc0c2593..4e4a4a5cfcfce 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchConnectorModule.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchConnectorModule.java @@ -11,21 +11,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch; +package io.trino.plugin.opensearch; import com.google.inject.Binder; import com.google.inject.Scopes; import io.airlift.configuration.AbstractConfigurationAwareModule; -import io.trino.plugin.elasticsearch.client.OpenSearchClient; -import io.trino.plugin.elasticsearch.ptf.RawQuery; +import io.trino.plugin.opensearch.client.OpenSearchClient; +import io.trino.plugin.opensearch.ptf.RawQuery; import io.trino.spi.function.table.ConnectorTableFunction; import static com.google.inject.multibindings.Multibinder.newSetBinder; import static com.google.inject.multibindings.OptionalBinder.newOptionalBinder; import static io.airlift.configuration.ConditionalModule.conditionalModule; import static io.airlift.configuration.ConfigBinder.configBinder; -import static io.trino.plugin.elasticsearch.OpenSearchConfig.Security.AWS; -import static io.trino.plugin.elasticsearch.OpenSearchConfig.Security.PASSWORD; import static java.util.function.Predicate.isEqual; import static org.weakref.jmx.guice.ExportBinder.newExporter; @@ -54,14 +52,14 @@ protected void setup(Binder binder) install(conditionalModule( OpenSearchConfig.class, config -> config.getSecurity() - .filter(isEqual(AWS)) + .filter(isEqual(OpenSearchConfig.Security.AWS)) .isPresent(), conditionalBinder -> configBinder(conditionalBinder).bindConfig(AwsSecurityConfig.class))); install(conditionalModule( OpenSearchConfig.class, config -> config.getSecurity() - .filter(isEqual(PASSWORD)) + .filter(isEqual(OpenSearchConfig.Security.PASSWORD)) .isPresent(), conditionalBinder -> configBinder(conditionalBinder).bindConfig(PasswordConfig.class))); } diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchErrorCode.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchErrorCode.java similarity index 97% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchErrorCode.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchErrorCode.java index 51c1b432bc9d0..c72e83f4d5041 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchErrorCode.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchErrorCode.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch; +package io.trino.plugin.opensearch; import io.trino.spi.ErrorCode; import io.trino.spi.ErrorCodeSupplier; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchMetadata.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchMetadata.java similarity index 94% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchMetadata.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchMetadata.java index ff4b20c2de562..b806093429cbb 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchMetadata.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchMetadata.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch; +package io.trino.plugin.opensearch; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -19,27 +19,27 @@ import com.google.inject.Inject; import io.airlift.slice.Slice; import io.trino.plugin.base.expression.ConnectorExpressions; -import io.trino.plugin.elasticsearch.client.IndexMetadata; -import io.trino.plugin.elasticsearch.client.IndexMetadata.DateTimeType; -import io.trino.plugin.elasticsearch.client.IndexMetadata.ObjectType; -import io.trino.plugin.elasticsearch.client.IndexMetadata.PrimitiveType; -import io.trino.plugin.elasticsearch.client.IndexMetadata.ScaledFloatType; -import io.trino.plugin.elasticsearch.client.OpenSearchClient; -import io.trino.plugin.elasticsearch.decoders.ArrayDecoder; -import io.trino.plugin.elasticsearch.decoders.BigintDecoder; -import io.trino.plugin.elasticsearch.decoders.BooleanDecoder; -import io.trino.plugin.elasticsearch.decoders.DoubleDecoder; -import io.trino.plugin.elasticsearch.decoders.IntegerDecoder; -import io.trino.plugin.elasticsearch.decoders.IpAddressDecoder; -import io.trino.plugin.elasticsearch.decoders.RawJsonDecoder; -import io.trino.plugin.elasticsearch.decoders.RealDecoder; -import io.trino.plugin.elasticsearch.decoders.RowDecoder; -import io.trino.plugin.elasticsearch.decoders.SmallintDecoder; -import io.trino.plugin.elasticsearch.decoders.TimestampDecoder; -import io.trino.plugin.elasticsearch.decoders.TinyintDecoder; -import io.trino.plugin.elasticsearch.decoders.VarbinaryDecoder; -import io.trino.plugin.elasticsearch.decoders.VarcharDecoder; -import io.trino.plugin.elasticsearch.ptf.RawQuery.RawQueryFunctionHandle; +import io.trino.plugin.opensearch.client.IndexMetadata; +import io.trino.plugin.opensearch.client.IndexMetadata.DateTimeType; +import io.trino.plugin.opensearch.client.IndexMetadata.ObjectType; +import io.trino.plugin.opensearch.client.IndexMetadata.PrimitiveType; +import io.trino.plugin.opensearch.client.IndexMetadata.ScaledFloatType; +import io.trino.plugin.opensearch.client.OpenSearchClient; +import io.trino.plugin.opensearch.decoders.ArrayDecoder; +import io.trino.plugin.opensearch.decoders.BigintDecoder; +import io.trino.plugin.opensearch.decoders.BooleanDecoder; +import io.trino.plugin.opensearch.decoders.DoubleDecoder; +import io.trino.plugin.opensearch.decoders.IntegerDecoder; +import io.trino.plugin.opensearch.decoders.IpAddressDecoder; +import io.trino.plugin.opensearch.decoders.RawJsonDecoder; +import io.trino.plugin.opensearch.decoders.RealDecoder; +import io.trino.plugin.opensearch.decoders.RowDecoder; +import io.trino.plugin.opensearch.decoders.SmallintDecoder; +import io.trino.plugin.opensearch.decoders.TimestampDecoder; +import io.trino.plugin.opensearch.decoders.TinyintDecoder; +import io.trino.plugin.opensearch.decoders.VarbinaryDecoder; +import io.trino.plugin.opensearch.decoders.VarcharDecoder; +import io.trino.plugin.opensearch.ptf.RawQuery.RawQueryFunctionHandle; import io.trino.spi.TrinoException; import io.trino.spi.connector.ColumnHandle; import io.trino.spi.connector.ColumnMetadata; @@ -87,8 +87,6 @@ import static com.google.common.collect.Iterators.singletonIterator; import static io.airlift.slice.SliceUtf8.getCodePointAt; import static io.airlift.slice.SliceUtf8.lengthOfCodePoint; -import static io.trino.plugin.elasticsearch.OpenSearchTableHandle.Type.QUERY; -import static io.trino.plugin.elasticsearch.OpenSearchTableHandle.Type.SCAN; import static io.trino.spi.StandardErrorCode.INVALID_FUNCTION_ARGUMENT; import static io.trino.spi.expression.StandardFunctions.LIKE_FUNCTION_NAME; import static io.trino.spi.type.BigintType.BIGINT; @@ -162,7 +160,7 @@ public OpenSearchTableHandle getTableHandle(ConnectorSession session, SchemaTabl } if (client.indexExists(table) && !client.getIndexMetadata(table).getSchema().getFields().isEmpty()) { - return new OpenSearchTableHandle(SCAN, schemaName, table, query); + return new OpenSearchTableHandle(OpenSearchTableHandle.Type.SCAN, schemaName, table, query); } } @@ -645,7 +643,7 @@ private static char getEscapeChar(Slice escape) private static boolean isPassthroughQuery(OpenSearchTableHandle table) { - return table.getType().equals(QUERY); + return table.getType().equals(OpenSearchTableHandle.Type.QUERY); } @Override diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchPageSourceProvider.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchPageSourceProvider.java similarity index 93% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchPageSourceProvider.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchPageSourceProvider.java index 368f7f78f5428..d4a6d3ae4c8c0 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchPageSourceProvider.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchPageSourceProvider.java @@ -11,10 +11,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch; +package io.trino.plugin.opensearch; import com.google.inject.Inject; -import io.trino.plugin.elasticsearch.client.OpenSearchClient; +import io.trino.plugin.opensearch.client.OpenSearchClient; import io.trino.spi.connector.ColumnHandle; import io.trino.spi.connector.ConnectorPageSource; import io.trino.spi.connector.ConnectorPageSourceProvider; @@ -28,7 +28,7 @@ import java.util.List; import static com.google.common.collect.ImmutableList.toImmutableList; -import static io.trino.plugin.elasticsearch.OpenSearchTableHandle.Type.QUERY; +import static io.trino.plugin.opensearch.OpenSearchTableHandle.Type.QUERY; import static java.util.Objects.requireNonNull; public class OpenSearchPageSourceProvider diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchPlugin.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchPlugin.java similarity index 97% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchPlugin.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchPlugin.java index 0d31841fc31ca..bd44b5d688f4c 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchPlugin.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchPlugin.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch; +package io.trino.plugin.opensearch; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchQueryBuilder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchQueryBuilder.java similarity index 99% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchQueryBuilder.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchQueryBuilder.java index 19549d85a45ea..9873c46e90102 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchQueryBuilder.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchQueryBuilder.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch; +package io.trino.plugin.opensearch; import com.google.common.collect.ImmutableList; import io.airlift.slice.Slice; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchSplit.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchSplit.java similarity index 98% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchSplit.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchSplit.java index 7954758240eb7..f3b63af37a52f 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchSplit.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchSplit.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch; +package io.trino.plugin.opensearch; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchSplitManager.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchSplitManager.java similarity index 90% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchSplitManager.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchSplitManager.java index 13de8c1309e3e..110a9245afa62 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchSplitManager.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchSplitManager.java @@ -11,10 +11,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch; +package io.trino.plugin.opensearch; import com.google.inject.Inject; -import io.trino.plugin.elasticsearch.client.OpenSearchClient; +import io.trino.plugin.opensearch.client.OpenSearchClient; import io.trino.spi.connector.ConnectorSession; import io.trino.spi.connector.ConnectorSplitManager; import io.trino.spi.connector.ConnectorSplitSource; @@ -28,7 +28,6 @@ import java.util.Optional; import static com.google.common.collect.ImmutableList.toImmutableList; -import static io.trino.plugin.elasticsearch.OpenSearchTableHandle.Type.QUERY; import static java.util.Objects.requireNonNull; public class OpenSearchSplitManager @@ -52,7 +51,7 @@ public ConnectorSplitSource getSplits( { OpenSearchTableHandle tableHandle = (OpenSearchTableHandle) table; - if (tableHandle.getType().equals(QUERY)) { + if (tableHandle.getType().equals(OpenSearchTableHandle.Type.QUERY)) { return new FixedSplitSource(new OpenSearchSplit(tableHandle.getIndex(), 0, Optional.empty())); } List splits = client.getSearchShards(tableHandle.getIndex()).stream() diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchTableHandle.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchTableHandle.java similarity index 99% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchTableHandle.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchTableHandle.java index 7610937df0c2d..e95cf612e7124 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchTableHandle.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchTableHandle.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch; +package io.trino.plugin.opensearch; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchTransactionHandle.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchTransactionHandle.java similarity index 94% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchTransactionHandle.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchTransactionHandle.java index 9bd0ac8453ca9..ebef379d43895 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/OpenSearchTransactionHandle.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchTransactionHandle.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch; +package io.trino.plugin.opensearch; import io.trino.spi.connector.ConnectorTransactionHandle; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/PassthroughQueryPageSource.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/PassthroughQueryPageSource.java similarity index 95% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/PassthroughQueryPageSource.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/PassthroughQueryPageSource.java index ea5628d60bb28..f29269259d3d2 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/PassthroughQueryPageSource.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/PassthroughQueryPageSource.java @@ -11,11 +11,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch; +package io.trino.plugin.opensearch; import com.google.common.collect.ImmutableList; import io.airlift.slice.Slices; -import io.trino.plugin.elasticsearch.client.OpenSearchClient; +import io.trino.plugin.opensearch.client.OpenSearchClient; import io.trino.spi.Page; import io.trino.spi.PageBuilder; import io.trino.spi.block.BlockBuilder; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/PasswordConfig.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/PasswordConfig.java similarity index 97% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/PasswordConfig.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/PasswordConfig.java index 9e36701e47b29..392f7507ce063 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/PasswordConfig.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/PasswordConfig.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch; +package io.trino.plugin.opensearch; import io.airlift.configuration.Config; import io.airlift.configuration.ConfigSecuritySensitive; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ScanQueryPageSource.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/ScanQueryPageSource.java similarity index 95% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ScanQueryPageSource.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/ScanQueryPageSource.java index b1725cf945230..99edcbca8cd2a 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ScanQueryPageSource.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/ScanQueryPageSource.java @@ -11,13 +11,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch; +package io.trino.plugin.opensearch; import com.google.common.collect.AbstractIterator; import com.google.common.collect.ImmutableList; import io.airlift.log.Logger; -import io.trino.plugin.elasticsearch.client.OpenSearchClient; -import io.trino.plugin.elasticsearch.decoders.Decoder; +import io.trino.plugin.opensearch.client.OpenSearchClient; +import io.trino.plugin.opensearch.decoders.Decoder; import io.trino.spi.Page; import io.trino.spi.block.Block; import io.trino.spi.block.BlockBuilder; @@ -39,9 +39,8 @@ import java.util.function.Supplier; import static com.google.common.collect.ImmutableList.toImmutableList; -import static io.trino.plugin.elasticsearch.BuiltinColumns.SOURCE; -import static io.trino.plugin.elasticsearch.BuiltinColumns.isBuiltinColumn; -import static io.trino.plugin.elasticsearch.OpenSearchQueryBuilder.buildSearchQuery; +import static io.trino.plugin.opensearch.BuiltinColumns.SOURCE; +import static io.trino.plugin.opensearch.BuiltinColumns.isBuiltinColumn; import static io.trino.spi.type.TimestampType.TIMESTAMP_MILLIS; import static java.util.Objects.requireNonNull; import static java.util.function.Predicate.isEqual; @@ -111,7 +110,7 @@ public ScanQueryPageSource( SearchResponse searchResponse = client.beginSearch( split.getIndex(), split.getShard(), - buildSearchQuery(table.getConstraint().transformKeys(OpenSearchColumnHandle.class::cast), table.getQuery(), table.getRegexes()), + OpenSearchQueryBuilder.buildSearchQuery(table.getConstraint().transformKeys(OpenSearchColumnHandle.class::cast), table.getQuery(), table.getRegexes()), needAllFields ? Optional.empty() : Optional.of(requiredFields), documentFields, sort, diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/AwsRequestSigner.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/client/AwsRequestSigner.java similarity index 98% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/AwsRequestSigner.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/client/AwsRequestSigner.java index dfefedc3c041c..e93017aae848c 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/AwsRequestSigner.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/client/AwsRequestSigner.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch.client; +package io.trino.plugin.opensearch.client; import com.amazonaws.DefaultRequest; import com.amazonaws.auth.AWS4Signer; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/BackpressureRestClient.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/client/BackpressureRestClient.java similarity index 98% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/BackpressureRestClient.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/client/BackpressureRestClient.java index 0ecd2358544e7..7c6d0538cc17f 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/BackpressureRestClient.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/client/BackpressureRestClient.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch.client; +package io.trino.plugin.opensearch.client; import com.google.common.base.Stopwatch; import dev.failsafe.Failsafe; @@ -22,7 +22,7 @@ import dev.failsafe.function.CheckedSupplier; import io.airlift.log.Logger; import io.airlift.stats.TimeStat; -import io.trino.plugin.elasticsearch.OpenSearchConfig; +import io.trino.plugin.opensearch.OpenSearchConfig; import org.apache.http.Header; import org.apache.http.HttpEntity; import org.apache.http.HttpHost; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/BackpressureRestHighLevelClient.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/client/BackpressureRestHighLevelClient.java similarity index 98% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/BackpressureRestHighLevelClient.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/client/BackpressureRestHighLevelClient.java index 4871950028e93..b75a0caf01f46 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/BackpressureRestHighLevelClient.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/client/BackpressureRestHighLevelClient.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch.client; +package io.trino.plugin.opensearch.client; import com.google.common.base.Stopwatch; import dev.failsafe.Failsafe; @@ -22,7 +22,7 @@ import dev.failsafe.function.CheckedSupplier; import io.airlift.log.Logger; import io.airlift.stats.TimeStat; -import io.trino.plugin.elasticsearch.OpenSearchConfig; +import io.trino.plugin.opensearch.OpenSearchConfig; import org.opensearch.OpenSearchStatusException; import org.opensearch.action.search.ClearScrollRequest; import org.opensearch.action.search.ClearScrollResponse; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/CountResponse.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/client/CountResponse.java similarity index 95% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/CountResponse.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/client/CountResponse.java index 15e5de2b0bbc2..f6db36a1012e2 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/CountResponse.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/client/CountResponse.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch.client; +package io.trino.plugin.opensearch.client; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/IndexMetadata.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/client/IndexMetadata.java similarity index 98% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/IndexMetadata.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/client/IndexMetadata.java index 2a73fafeaabf2..ff0d956595ee6 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/IndexMetadata.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/client/IndexMetadata.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch.client; +package io.trino.plugin.opensearch.client; import com.google.common.collect.ImmutableList; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/NodesResponse.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/client/NodesResponse.java similarity index 98% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/NodesResponse.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/client/NodesResponse.java index c8f2067d4168e..3ed2b84b16bc8 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/NodesResponse.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/client/NodesResponse.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch.client; +package io.trino.plugin.opensearch.client; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/OpenSearchClient.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/client/OpenSearchClient.java similarity index 93% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/OpenSearchClient.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/client/OpenSearchClient.java index e6911266fa941..0dbc9e7ac41da 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/OpenSearchClient.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/client/OpenSearchClient.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch.client; +package io.trino.plugin.opensearch.client; import com.amazonaws.auth.AWSCredentialsProvider; import com.amazonaws.auth.AWSStaticCredentialsProvider; @@ -33,9 +33,10 @@ import io.airlift.log.Logger; import io.airlift.stats.TimeStat; import io.airlift.units.Duration; -import io.trino.plugin.elasticsearch.AwsSecurityConfig; -import io.trino.plugin.elasticsearch.OpenSearchConfig; -import io.trino.plugin.elasticsearch.PasswordConfig; +import io.trino.plugin.opensearch.AwsSecurityConfig; +import io.trino.plugin.opensearch.OpenSearchConfig; +import io.trino.plugin.opensearch.OpenSearchErrorCode; +import io.trino.plugin.opensearch.PasswordConfig; import io.trino.spi.TrinoException; import jakarta.annotation.PostConstruct; import jakarta.annotation.PreDestroy; @@ -92,11 +93,6 @@ import static io.airlift.concurrent.Threads.daemonThreadsNamed; import static io.airlift.json.JsonCodec.jsonCodec; import static io.trino.plugin.base.ssl.SslUtils.createSSLContext; -import static io.trino.plugin.elasticsearch.OpenSearchErrorCode.OPENSEARCH_CONNECTION_ERROR; -import static io.trino.plugin.elasticsearch.OpenSearchErrorCode.OPENSEARCH_INVALID_METADATA; -import static io.trino.plugin.elasticsearch.OpenSearchErrorCode.OPENSEARCH_INVALID_RESPONSE; -import static io.trino.plugin.elasticsearch.OpenSearchErrorCode.OPENSEARCH_QUERY_FAILURE; -import static io.trino.plugin.elasticsearch.OpenSearchErrorCode.OPENSEARCH_SSL_INITIALIZATION_FAILURE; import static java.lang.StrictMath.toIntExact; import static java.lang.String.format; import static java.nio.charset.StandardCharsets.UTF_8; @@ -282,7 +278,7 @@ private static Optional buildSslContext( return Optional.of(createSSLContext(keyStorePath, keyStorePassword, trustStorePath, trustStorePassword)); } catch (GeneralSecurityException | IOException e) { - throw new TrinoException(OPENSEARCH_SSL_INITIALIZATION_FAILURE, e); + throw new TrinoException(OpenSearchErrorCode.OPENSEARCH_SSL_INITIALIZATION_FAILURE, e); } } @@ -370,10 +366,10 @@ public boolean indexExists(String index) if (e.getResponse().getStatusLine().getStatusCode() == 404) { return false; } - throw new TrinoException(OPENSEARCH_CONNECTION_ERROR, e); + throw new TrinoException(OpenSearchErrorCode.OPENSEARCH_CONNECTION_ERROR, e); } catch (IOException e) { - throw new TrinoException(OPENSEARCH_CONNECTION_ERROR, e); + throw new TrinoException(OpenSearchErrorCode.OPENSEARCH_CONNECTION_ERROR, e); } } @@ -399,7 +395,7 @@ public List getIndexes() return result.build(); } catch (IOException e) { - throw new TrinoException(OPENSEARCH_INVALID_RESPONSE, e); + throw new TrinoException(OpenSearchErrorCode.OPENSEARCH_INVALID_RESPONSE, e); } }); } @@ -423,7 +419,7 @@ public Map> getAliases() return result.buildOrThrow(); } catch (IOException e) { - throw new TrinoException(OPENSEARCH_INVALID_RESPONSE, e); + throw new TrinoException(OpenSearchErrorCode.OPENSEARCH_INVALID_RESPONSE, e); } }); } @@ -464,7 +460,7 @@ public IndexMetadata getIndexMetadata(String index) return new IndexMetadata(parseType(mappings.get("properties"), metaProperties)); } catch (IOException e) { - throw new TrinoException(OPENSEARCH_INVALID_RESPONSE, e); + throw new TrinoException(OpenSearchErrorCode.OPENSEARCH_INVALID_RESPONSE, e); } }); } @@ -493,7 +489,7 @@ private IndexMetadata.ObjectType parseType(JsonNode properties, JsonNode metaPro // this route, as it will likely lead to confusion in dealing with array syntax in Trino and potentially nested array and other // syntax when parsing the raw json. if (isArray && asRawJson) { - throw new TrinoException(OPENSEARCH_INVALID_METADATA, + throw new TrinoException(OpenSearchErrorCode.OPENSEARCH_INVALID_METADATA, format("A column, (%s) cannot be declared as a Trino array and also be rendered as json.", name)); } @@ -550,7 +546,7 @@ public String executeQuery(String index, String query) new BasicHeader("Accept-Encoding", "application/json")); } catch (IOException e) { - throw new TrinoException(OPENSEARCH_CONNECTION_ERROR, e); + throw new TrinoException(OpenSearchErrorCode.OPENSEARCH_CONNECTION_ERROR, e); } String body; @@ -558,7 +554,7 @@ public String executeQuery(String index, String query) body = EntityUtils.toString(response.getEntity()); } catch (IOException e) { - throw new TrinoException(OPENSEARCH_INVALID_RESPONSE, e); + throw new TrinoException(OpenSearchErrorCode.OPENSEARCH_INVALID_RESPONSE, e); } return body; @@ -602,7 +598,7 @@ public SearchResponse beginSearch(String index, int shard, QueryBuilder query, O return client.search(request); } catch (IOException e) { - throw new TrinoException(OPENSEARCH_CONNECTION_ERROR, e); + throw new TrinoException(OpenSearchErrorCode.OPENSEARCH_CONNECTION_ERROR, e); } catch (OpenSearchStatusException e) { Throwable[] suppressed = e.getSuppressed(); @@ -613,7 +609,7 @@ public SearchResponse beginSearch(String index, int shard, QueryBuilder query, O } } - throw new TrinoException(OPENSEARCH_CONNECTION_ERROR, e); + throw new TrinoException(OpenSearchErrorCode.OPENSEARCH_CONNECTION_ERROR, e); } finally { searchStats.add(Duration.nanosSince(start)); @@ -632,7 +628,7 @@ public SearchResponse nextPage(String scrollId) return client.searchScroll(request); } catch (IOException e) { - throw new TrinoException(OPENSEARCH_CONNECTION_ERROR, e); + throw new TrinoException(OpenSearchErrorCode.OPENSEARCH_CONNECTION_ERROR, e); } finally { nextPageStats.add(Duration.nanosSince(start)); @@ -662,7 +658,7 @@ public long count(String index, int shard, QueryBuilder query) throw propagate(e); } catch (IOException e) { - throw new TrinoException(OPENSEARCH_CONNECTION_ERROR, e); + throw new TrinoException(OpenSearchErrorCode.OPENSEARCH_CONNECTION_ERROR, e); } try { @@ -670,7 +666,7 @@ public long count(String index, int shard, QueryBuilder query) .getCount(); } catch (IOException e) { - throw new TrinoException(OPENSEARCH_INVALID_RESPONSE, e); + throw new TrinoException(OpenSearchErrorCode.OPENSEARCH_INVALID_RESPONSE, e); } } finally { @@ -686,7 +682,7 @@ public void clearScroll(String scrollId) client.clearScroll(request); } catch (IOException e) { - throw new TrinoException(OPENSEARCH_CONNECTION_ERROR, e); + throw new TrinoException(OpenSearchErrorCode.OPENSEARCH_CONNECTION_ERROR, e); } } @@ -728,7 +724,7 @@ private T doRequest(String path, ResponseHandler handler) .performRequest("GET", path); } catch (IOException e) { - throw new TrinoException(OPENSEARCH_CONNECTION_ERROR, e); + throw new TrinoException(OpenSearchErrorCode.OPENSEARCH_CONNECTION_ERROR, e); } String body; @@ -736,7 +732,7 @@ private T doRequest(String path, ResponseHandler handler) body = EntityUtils.toString(response.getEntity()); } catch (IOException e) { - throw new TrinoException(OPENSEARCH_INVALID_RESPONSE, e); + throw new TrinoException(OpenSearchErrorCode.OPENSEARCH_INVALID_RESPONSE, e); } return handler.process(body); @@ -754,17 +750,17 @@ private static TrinoException propagate(ResponseException exception) .path("reason"); if (!reason.isMissingNode()) { - throw new TrinoException(OPENSEARCH_QUERY_FAILURE, reason.asText(), exception); + throw new TrinoException(OpenSearchErrorCode.OPENSEARCH_QUERY_FAILURE, reason.asText(), exception); } } catch (IOException e) { - TrinoException result = new TrinoException(OPENSEARCH_QUERY_FAILURE, exception); + TrinoException result = new TrinoException(OpenSearchErrorCode.OPENSEARCH_QUERY_FAILURE, exception); result.addSuppressed(e); throw result; } } - throw new TrinoException(OPENSEARCH_QUERY_FAILURE, exception); + throw new TrinoException(OpenSearchErrorCode.OPENSEARCH_QUERY_FAILURE, exception); } @VisibleForTesting diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/OpenSearchNode.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/client/OpenSearchNode.java similarity index 96% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/OpenSearchNode.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/client/OpenSearchNode.java index ad6486feb8bbd..c41a4228b4002 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/OpenSearchNode.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/client/OpenSearchNode.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch.client; +package io.trino.plugin.opensearch.client; import java.util.Optional; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/SearchShardsResponse.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/client/SearchShardsResponse.java similarity index 98% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/SearchShardsResponse.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/client/SearchShardsResponse.java index a68d208272dc7..75b3b447520d6 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/SearchShardsResponse.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/client/SearchShardsResponse.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch.client; +package io.trino.plugin.opensearch.client; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/Shard.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/client/Shard.java similarity index 96% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/Shard.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/client/Shard.java index 2237a89a7bfac..ac0e76fd2b98c 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/client/Shard.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/client/Shard.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch.client; +package io.trino.plugin.opensearch.client; import java.util.Optional; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/ArrayDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/ArrayDecoder.java similarity index 95% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/ArrayDecoder.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/ArrayDecoder.java index 3f26319851ad0..d7b1c6a823bf6 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/ArrayDecoder.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/ArrayDecoder.java @@ -11,11 +11,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch.decoders; +package io.trino.plugin.opensearch.decoders; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import io.trino.plugin.elasticsearch.DecoderDescriptor; +import io.trino.plugin.opensearch.DecoderDescriptor; import io.trino.spi.block.ArrayBlockBuilder; import io.trino.spi.block.BlockBuilder; import org.opensearch.search.SearchHit; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/BigintDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/BigintDecoder.java similarity index 96% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/BigintDecoder.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/BigintDecoder.java index 8fcc25ef2eff0..0d08a8db2bce7 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/BigintDecoder.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/BigintDecoder.java @@ -11,11 +11,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch.decoders; +package io.trino.plugin.opensearch.decoders; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import io.trino.plugin.elasticsearch.DecoderDescriptor; +import io.trino.plugin.opensearch.DecoderDescriptor; import io.trino.spi.TrinoException; import io.trino.spi.block.BlockBuilder; import org.opensearch.search.SearchHit; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/BooleanDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/BooleanDecoder.java similarity index 96% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/BooleanDecoder.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/BooleanDecoder.java index 8be94c6d3cc8f..57fdddb4e0627 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/BooleanDecoder.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/BooleanDecoder.java @@ -11,11 +11,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch.decoders; +package io.trino.plugin.opensearch.decoders; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import io.trino.plugin.elasticsearch.DecoderDescriptor; +import io.trino.plugin.opensearch.DecoderDescriptor; import io.trino.spi.TrinoException; import io.trino.spi.block.BlockBuilder; import org.opensearch.search.SearchHit; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/Decoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/Decoder.java similarity index 94% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/Decoder.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/Decoder.java index 26fef13fff0cf..1199d5fc42f2d 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/Decoder.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/Decoder.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch.decoders; +package io.trino.plugin.opensearch.decoders; import io.trino.spi.block.BlockBuilder; import org.opensearch.search.SearchHit; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/DoubleDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/DoubleDecoder.java similarity index 96% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/DoubleDecoder.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/DoubleDecoder.java index d978e2f7e19ef..908ec9345c510 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/DoubleDecoder.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/DoubleDecoder.java @@ -11,11 +11,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch.decoders; +package io.trino.plugin.opensearch.decoders; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import io.trino.plugin.elasticsearch.DecoderDescriptor; +import io.trino.plugin.opensearch.DecoderDescriptor; import io.trino.spi.TrinoException; import io.trino.spi.block.BlockBuilder; import org.opensearch.search.SearchHit; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/IdColumnDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/IdColumnDecoder.java similarity index 92% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/IdColumnDecoder.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/IdColumnDecoder.java index e3808d9333a21..264f60f632bb0 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/IdColumnDecoder.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/IdColumnDecoder.java @@ -11,10 +11,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch.decoders; +package io.trino.plugin.opensearch.decoders; import io.airlift.slice.Slices; -import io.trino.plugin.elasticsearch.DecoderDescriptor; +import io.trino.plugin.opensearch.DecoderDescriptor; import io.trino.spi.block.BlockBuilder; import org.opensearch.search.SearchHit; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/IntegerDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/IntegerDecoder.java similarity index 96% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/IntegerDecoder.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/IntegerDecoder.java index 2cb6660569f42..fd09f8543e755 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/IntegerDecoder.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/IntegerDecoder.java @@ -11,11 +11,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch.decoders; +package io.trino.plugin.opensearch.decoders; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import io.trino.plugin.elasticsearch.DecoderDescriptor; +import io.trino.plugin.opensearch.DecoderDescriptor; import io.trino.spi.TrinoException; import io.trino.spi.block.BlockBuilder; import org.opensearch.search.SearchHit; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/IpAddressDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/IpAddressDecoder.java similarity index 97% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/IpAddressDecoder.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/IpAddressDecoder.java index 83ec1c33d8c56..9b50ae4e5bc54 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/IpAddressDecoder.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/IpAddressDecoder.java @@ -11,14 +11,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch.decoders; +package io.trino.plugin.opensearch.decoders; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.net.InetAddresses; import io.airlift.slice.Slice; import io.airlift.slice.Slices; -import io.trino.plugin.elasticsearch.DecoderDescriptor; +import io.trino.plugin.opensearch.DecoderDescriptor; import io.trino.spi.TrinoException; import io.trino.spi.block.BlockBuilder; import io.trino.spi.type.Type; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/RawJsonDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/RawJsonDecoder.java similarity index 96% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/RawJsonDecoder.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/RawJsonDecoder.java index f0c56a7d7b4f1..a1290b2ecef01 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/RawJsonDecoder.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/RawJsonDecoder.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch.decoders; +package io.trino.plugin.opensearch.decoders; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; @@ -19,7 +19,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import io.airlift.json.ObjectMapperProvider; import io.airlift.slice.Slices; -import io.trino.plugin.elasticsearch.DecoderDescriptor; +import io.trino.plugin.opensearch.DecoderDescriptor; import io.trino.spi.TrinoException; import io.trino.spi.block.BlockBuilder; import org.opensearch.search.SearchHit; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/RealDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/RealDecoder.java similarity index 96% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/RealDecoder.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/RealDecoder.java index 1c068074cd552..22276f919c65e 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/RealDecoder.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/RealDecoder.java @@ -11,11 +11,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch.decoders; +package io.trino.plugin.opensearch.decoders; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import io.trino.plugin.elasticsearch.DecoderDescriptor; +import io.trino.plugin.opensearch.DecoderDescriptor; import io.trino.spi.TrinoException; import io.trino.spi.block.BlockBuilder; import org.opensearch.search.SearchHit; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/RowDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/RowDecoder.java similarity index 92% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/RowDecoder.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/RowDecoder.java index 8b5a32db772af..522243a3d8ad4 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/RowDecoder.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/RowDecoder.java @@ -11,11 +11,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch.decoders; +package io.trino.plugin.opensearch.decoders; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import io.trino.plugin.elasticsearch.DecoderDescriptor; +import io.trino.plugin.opensearch.DecoderDescriptor; +import io.trino.plugin.opensearch.ScanQueryPageSource; import io.trino.spi.TrinoException; import io.trino.spi.block.BlockBuilder; import io.trino.spi.block.RowBlockBuilder; @@ -26,7 +27,6 @@ import java.util.function.Supplier; import static com.google.common.collect.ImmutableList.toImmutableList; -import static io.trino.plugin.elasticsearch.ScanQueryPageSource.getField; import static io.trino.spi.StandardErrorCode.TYPE_MISMATCH; import static java.lang.String.format; import static java.util.Objects.requireNonNull; @@ -57,7 +57,7 @@ else if (data instanceof Map) { ((RowBlockBuilder) output).buildEntry(fieldBuilders -> { for (int i = 0; i < decoders.size(); i++) { String field = fieldNames.get(i); - decoders.get(i).decode(hit, () -> getField((Map) data, field), fieldBuilders.get(i)); + decoders.get(i).decode(hit, () -> ScanQueryPageSource.getField((Map) data, field), fieldBuilders.get(i)); } }); } diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/ScoreColumnDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/ScoreColumnDecoder.java similarity index 91% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/ScoreColumnDecoder.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/ScoreColumnDecoder.java index 405d28e5f15e9..d7dfa71eece8c 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/ScoreColumnDecoder.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/ScoreColumnDecoder.java @@ -11,9 +11,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch.decoders; +package io.trino.plugin.opensearch.decoders; -import io.trino.plugin.elasticsearch.DecoderDescriptor; +import io.trino.plugin.opensearch.DecoderDescriptor; import io.trino.spi.block.BlockBuilder; import org.opensearch.search.SearchHit; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/SmallintDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/SmallintDecoder.java similarity index 96% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/SmallintDecoder.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/SmallintDecoder.java index ef5c69a0a89dd..61198f34e1f40 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/SmallintDecoder.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/SmallintDecoder.java @@ -11,11 +11,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch.decoders; +package io.trino.plugin.opensearch.decoders; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import io.trino.plugin.elasticsearch.DecoderDescriptor; +import io.trino.plugin.opensearch.DecoderDescriptor; import io.trino.spi.TrinoException; import io.trino.spi.block.BlockBuilder; import org.opensearch.search.SearchHit; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/SourceColumnDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/SourceColumnDecoder.java similarity index 92% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/SourceColumnDecoder.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/SourceColumnDecoder.java index 9a8e0d571ec2a..dd00f28906845 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/SourceColumnDecoder.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/SourceColumnDecoder.java @@ -11,10 +11,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch.decoders; +package io.trino.plugin.opensearch.decoders; import io.airlift.slice.Slices; -import io.trino.plugin.elasticsearch.DecoderDescriptor; +import io.trino.plugin.opensearch.DecoderDescriptor; import io.trino.spi.block.BlockBuilder; import org.opensearch.search.SearchHit; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/TimestampDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/TimestampDecoder.java similarity index 97% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/TimestampDecoder.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/TimestampDecoder.java index 524d818816eda..d13203266682b 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/TimestampDecoder.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/TimestampDecoder.java @@ -11,12 +11,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch.decoders; +package io.trino.plugin.opensearch.decoders; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.primitives.Longs; -import io.trino.plugin.elasticsearch.DecoderDescriptor; +import io.trino.plugin.opensearch.DecoderDescriptor; import io.trino.spi.TrinoException; import io.trino.spi.block.BlockBuilder; import org.opensearch.common.document.DocumentField; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/TinyintDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/TinyintDecoder.java similarity index 96% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/TinyintDecoder.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/TinyintDecoder.java index bb3296cec0e84..2e8dc67322def 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/TinyintDecoder.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/TinyintDecoder.java @@ -11,11 +11,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch.decoders; +package io.trino.plugin.opensearch.decoders; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import io.trino.plugin.elasticsearch.DecoderDescriptor; +import io.trino.plugin.opensearch.DecoderDescriptor; import io.trino.spi.TrinoException; import io.trino.spi.block.BlockBuilder; import org.opensearch.search.SearchHit; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/VarbinaryDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/VarbinaryDecoder.java similarity index 95% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/VarbinaryDecoder.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/VarbinaryDecoder.java index 2d4c5c99300c9..6af55652f199e 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/VarbinaryDecoder.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/VarbinaryDecoder.java @@ -11,12 +11,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch.decoders; +package io.trino.plugin.opensearch.decoders; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import io.airlift.slice.Slices; -import io.trino.plugin.elasticsearch.DecoderDescriptor; +import io.trino.plugin.opensearch.DecoderDescriptor; import io.trino.spi.TrinoException; import io.trino.spi.block.BlockBuilder; import org.opensearch.search.SearchHit; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/VarcharDecoder.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/VarcharDecoder.java similarity index 95% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/VarcharDecoder.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/VarcharDecoder.java index 6e4b2b6ae385f..468a9f09671a0 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/decoders/VarcharDecoder.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/decoders/VarcharDecoder.java @@ -11,12 +11,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch.decoders; +package io.trino.plugin.opensearch.decoders; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import io.airlift.slice.Slices; -import io.trino.plugin.elasticsearch.DecoderDescriptor; +import io.trino.plugin.opensearch.DecoderDescriptor; import io.trino.spi.TrinoException; import io.trino.spi.block.BlockBuilder; import org.opensearch.search.SearchHit; diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ptf/RawQuery.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/ptf/RawQuery.java similarity index 94% rename from plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ptf/RawQuery.java rename to plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/ptf/RawQuery.java index 841ab3bcf1af8..ae0b1013b0b70 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/elasticsearch/ptf/RawQuery.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/ptf/RawQuery.java @@ -11,16 +11,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch.ptf; +package io.trino.plugin.opensearch.ptf; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.inject.Inject; import com.google.inject.Provider; import io.airlift.slice.Slice; -import io.trino.plugin.elasticsearch.OpenSearchColumnHandle; -import io.trino.plugin.elasticsearch.OpenSearchMetadata; -import io.trino.plugin.elasticsearch.OpenSearchTableHandle; +import io.trino.plugin.opensearch.OpenSearchColumnHandle; +import io.trino.plugin.opensearch.OpenSearchMetadata; +import io.trino.plugin.opensearch.OpenSearchTableHandle; import io.trino.spi.connector.ColumnHandle; import io.trino.spi.connector.ColumnSchema; import io.trino.spi.connector.ConnectorAccessControl; @@ -42,7 +42,6 @@ import java.util.Optional; import static com.google.common.collect.ImmutableList.toImmutableList; -import static io.trino.plugin.elasticsearch.OpenSearchTableHandle.Type.QUERY; import static io.trino.spi.function.table.ReturnTypeSpecification.GenericTable.GENERIC_TABLE; import static io.trino.spi.type.VarcharType.VARCHAR; import static java.util.Objects.requireNonNull; @@ -106,7 +105,7 @@ public TableFunctionAnalysis analyze( String index = ((Slice) ((ScalarArgument) arguments.get("INDEX")).getValue()).toStringUtf8(); String query = ((Slice) ((ScalarArgument) arguments.get("QUERY")).getValue()).toStringUtf8(); - OpenSearchTableHandle tableHandle = new OpenSearchTableHandle(QUERY, schema, index, Optional.of(query)); + OpenSearchTableHandle tableHandle = new OpenSearchTableHandle(OpenSearchTableHandle.Type.QUERY, schema, index, Optional.of(query)); ConnectorTableSchema tableSchema = metadata.getTableSchema(session, tableHandle); Map columnsByName = metadata.getColumnHandles(session, tableHandle); List columns = tableSchema.getColumns().stream() diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/BaseOpenSearchConnectorTest.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/BaseOpenSearchConnectorTest.java similarity index 99% rename from plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/BaseOpenSearchConnectorTest.java rename to plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/BaseOpenSearchConnectorTest.java index 34bce5b7fcf1a..add3230c41329 100644 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/BaseOpenSearchConnectorTest.java +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/BaseOpenSearchConnectorTest.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch; +package io.trino.plugin.opensearch; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; @@ -41,7 +41,7 @@ import java.util.List; import java.util.Map; -import static io.trino.plugin.elasticsearch.OpenSearchQueryRunner.createOpenSearchQueryRunner; +import static io.trino.plugin.opensearch.OpenSearchQueryRunner.createOpenSearchQueryRunner; import static io.trino.spi.type.DoubleType.DOUBLE; import static io.trino.spi.type.VarcharType.VARCHAR; import static io.trino.testing.MaterializedResult.resultBuilder; diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/OpenSearchLoader.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/OpenSearchLoader.java similarity index 99% rename from plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/OpenSearchLoader.java rename to plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/OpenSearchLoader.java index 5ecf706264ecc..f51d1e3f8a3cf 100644 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/OpenSearchLoader.java +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/OpenSearchLoader.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch; +package io.trino.plugin.opensearch; import io.trino.Session; import io.trino.client.Column; diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/OpenSearchQueryRunner.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/OpenSearchQueryRunner.java similarity index 98% rename from plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/OpenSearchQueryRunner.java rename to plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/OpenSearchQueryRunner.java index 0f0c962009fd8..1d5a770931a4d 100644 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/OpenSearchQueryRunner.java +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/OpenSearchQueryRunner.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch; +package io.trino.plugin.opensearch; import com.google.common.collect.ImmutableMap; import com.google.common.net.HostAndPort; @@ -33,7 +33,7 @@ import static io.airlift.testing.Closeables.closeAllSuppress; import static io.airlift.units.Duration.nanosSince; -import static io.trino.plugin.elasticsearch.OpenSearchServer.OPENSEARCH_IMAGE; +import static io.trino.plugin.opensearch.OpenSearchServer.OPENSEARCH_IMAGE; import static io.trino.plugin.tpch.TpchMetadata.TINY_SCHEMA_NAME; import static io.trino.testing.TestingSession.testSessionBuilder; import static java.lang.String.format; diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/OpenSearchServer.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/OpenSearchServer.java similarity index 98% rename from plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/OpenSearchServer.java rename to plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/OpenSearchServer.java index 45fc8296980e1..dfd262bb7f90b 100644 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/OpenSearchServer.java +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/OpenSearchServer.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch; +package io.trino.plugin.opensearch; import com.google.common.net.HostAndPort; import io.trino.testing.ResourcePresence; diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestAwsSecurityConfig.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/TestAwsSecurityConfig.java similarity index 98% rename from plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestAwsSecurityConfig.java rename to plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/TestAwsSecurityConfig.java index f08c5b2476246..ce17f71e506b2 100644 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestAwsSecurityConfig.java +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/TestAwsSecurityConfig.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch; +package io.trino.plugin.opensearch; import com.google.common.collect.ImmutableMap; import org.junit.jupiter.api.Test; diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearch2ConnectorTest.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/TestOpenSearch2ConnectorTest.java similarity index 96% rename from plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearch2ConnectorTest.java rename to plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/TestOpenSearch2ConnectorTest.java index be0f594a6f23c..a07c25b1bb448 100644 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearch2ConnectorTest.java +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/TestOpenSearch2ConnectorTest.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch; +package io.trino.plugin.opensearch; import com.google.common.collect.ImmutableList; import org.junit.jupiter.api.parallel.Isolated; diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearchConfig.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/TestOpenSearchConfig.java similarity index 97% rename from plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearchConfig.java rename to plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/TestOpenSearchConfig.java index cf0e98a7908cc..c563d766ea84f 100644 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearchConfig.java +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/TestOpenSearchConfig.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch; +package io.trino.plugin.opensearch; import com.google.common.collect.ImmutableMap; import io.airlift.units.Duration; @@ -26,7 +26,7 @@ import static io.airlift.configuration.testing.ConfigAssertions.assertFullMapping; import static io.airlift.configuration.testing.ConfigAssertions.assertRecordedDefaults; import static io.airlift.configuration.testing.ConfigAssertions.recordDefaults; -import static io.trino.plugin.elasticsearch.OpenSearchConfig.Security.AWS; +import static io.trino.plugin.opensearch.OpenSearchConfig.Security.AWS; import static java.util.concurrent.TimeUnit.MILLISECONDS; import static java.util.concurrent.TimeUnit.MINUTES; import static java.util.concurrent.TimeUnit.SECONDS; diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearchConnectorTest.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/TestOpenSearchConnectorTest.java similarity index 95% rename from plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearchConnectorTest.java rename to plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/TestOpenSearchConnectorTest.java index 4452f2214cdf7..bdafc4822dd15 100644 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearchConnectorTest.java +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/TestOpenSearchConnectorTest.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch; +package io.trino.plugin.opensearch; public class TestOpenSearchConnectorTest extends BaseOpenSearchConnectorTest diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearchLatestConnectorTest.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/TestOpenSearchLatestConnectorTest.java similarity index 96% rename from plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearchLatestConnectorTest.java rename to plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/TestOpenSearchLatestConnectorTest.java index 096ace3be1b57..5fd7c1d0196ab 100644 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearchLatestConnectorTest.java +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/TestOpenSearchLatestConnectorTest.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch; +package io.trino.plugin.opensearch; import com.google.common.collect.ImmutableList; import org.junit.jupiter.api.parallel.Isolated; diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearchMetadata.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/TestOpenSearchMetadata.java similarity index 98% rename from plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearchMetadata.java rename to plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/TestOpenSearchMetadata.java index 1d3cdac91833a..2bccee951465d 100644 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearchMetadata.java +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/TestOpenSearchMetadata.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch; +package io.trino.plugin.opensearch; import io.airlift.slice.Slices; import org.junit.jupiter.api.Test; diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearchQueryBuilder.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/TestOpenSearchQueryBuilder.java similarity index 93% rename from plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearchQueryBuilder.java rename to plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/TestOpenSearchQueryBuilder.java index 1be2c4fd9b707..4a8407ff125dc 100644 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestOpenSearchQueryBuilder.java +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/TestOpenSearchQueryBuilder.java @@ -11,13 +11,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch; +package io.trino.plugin.opensearch; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import io.trino.plugin.elasticsearch.decoders.DoubleDecoder; -import io.trino.plugin.elasticsearch.decoders.IntegerDecoder; -import io.trino.plugin.elasticsearch.decoders.VarcharDecoder; +import io.trino.plugin.opensearch.decoders.DoubleDecoder; +import io.trino.plugin.opensearch.decoders.IntegerDecoder; +import io.trino.plugin.opensearch.decoders.VarcharDecoder; import io.trino.spi.predicate.Domain; import io.trino.spi.predicate.Range; import io.trino.spi.predicate.TupleDomain; @@ -33,7 +33,6 @@ import java.util.Map; import java.util.Optional; -import static io.trino.plugin.elasticsearch.OpenSearchQueryBuilder.buildSearchQuery; import static io.trino.spi.type.DoubleType.DOUBLE; import static io.trino.spi.type.IntegerType.INTEGER; import static io.trino.spi.type.VarcharType.VARCHAR; @@ -132,7 +131,7 @@ public void testMultiConstraint() private static void assertQueryBuilder(Map domains, QueryBuilder expected) { - QueryBuilder actual = buildSearchQuery(TupleDomain.withColumnDomains(domains), Optional.empty(), Map.of()); + QueryBuilder actual = OpenSearchQueryBuilder.buildSearchQuery(TupleDomain.withColumnDomains(domains), Optional.empty(), Map.of()); assertThat(actual).isEqualTo(expected); } } diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestPasswordAuthentication.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/TestPasswordAuthentication.java similarity index 96% rename from plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestPasswordAuthentication.java rename to plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/TestPasswordAuthentication.java index cd77dd08e84f1..9375ba1511f8e 100644 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestPasswordAuthentication.java +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/TestPasswordAuthentication.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch; +package io.trino.plugin.opensearch; import com.amazonaws.util.Base64; import com.fasterxml.jackson.databind.ObjectMapper; @@ -43,8 +43,8 @@ import static com.google.common.io.Resources.getResource; import static io.airlift.testing.Closeables.closeAll; import static io.trino.plugin.base.ssl.SslUtils.createSSLContext; -import static io.trino.plugin.elasticsearch.OpenSearchQueryRunner.createOpenSearchQueryRunner; -import static io.trino.plugin.elasticsearch.OpenSearchServer.OPENSEARCH_IMAGE; +import static io.trino.plugin.opensearch.OpenSearchQueryRunner.createOpenSearchQueryRunner; +import static io.trino.plugin.opensearch.OpenSearchServer.OPENSEARCH_IMAGE; import static java.lang.String.format; import static java.nio.charset.StandardCharsets.UTF_8; import static org.assertj.core.api.Assertions.assertThat; diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestPasswordConfig.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/TestPasswordConfig.java similarity index 97% rename from plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestPasswordConfig.java rename to plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/TestPasswordConfig.java index e905dfa9b8440..f115069b15388 100644 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/TestPasswordConfig.java +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/TestPasswordConfig.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch; +package io.trino.plugin.opensearch; import com.google.common.collect.ImmutableMap; import org.junit.jupiter.api.Test; diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/client/TestExtractAddress.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/client/TestExtractAddress.java similarity index 92% rename from plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/client/TestExtractAddress.java rename to plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/client/TestExtractAddress.java index abf69d1eb1606..28ec3f08b0a26 100644 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/elasticsearch/client/TestExtractAddress.java +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/client/TestExtractAddress.java @@ -11,13 +11,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.elasticsearch.client; +package io.trino.plugin.opensearch.client; import org.junit.jupiter.api.Test; import java.util.Optional; -import static io.trino.plugin.elasticsearch.client.OpenSearchClient.extractAddress; +import static io.trino.plugin.opensearch.client.OpenSearchClient.extractAddress; import static org.assertj.core.api.Assertions.assertThat; public class TestExtractAddress From 8da2ef384330806dc1f04e8bdf056c16ae567cd7 Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Tue, 9 Jan 2024 12:14:23 +0100 Subject: [PATCH 301/350] Fix Javadoc for largeInValuesCountData --- .../trino/plugin/opensearch/BaseOpenSearchConnectorTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/BaseOpenSearchConnectorTest.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/BaseOpenSearchConnectorTest.java index add3230c41329..4df76815cad0b 100644 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/BaseOpenSearchConnectorTest.java +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/BaseOpenSearchConnectorTest.java @@ -121,8 +121,8 @@ protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior) /** * This method overrides the default values used for the data provider - * of the test {@link AbstractTestQueries#testLargeIn(int)} by taking - * into account that by default Opensearch 2.x supports only up to `1024` + * of the test {@link AbstractTestQueries#testLargeIn()} by taking + * into account that by default OpenSearch 2.x supports only up to `1024` * clauses in query. *

* Consult `index.query.bool.max_clause_count` opensearch.yml setting From 3af00547001e5c70571ee320cda241978fe85e3d Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Tue, 9 Jan 2024 12:31:27 +0100 Subject: [PATCH 302/350] Fix remaining Elasticsearch references --- .../java/io/trino/plugin/opensearch/OpenSearchMetadata.java | 2 +- .../io/trino/plugin/opensearch/ScanQueryPageSource.java | 4 ++-- .../io/trino/plugin/opensearch/client/OpenSearchClient.java | 2 +- .../plugin/opensearch/BaseOpenSearchConnectorTest.java | 4 ++-- .../java/io/trino/plugin/opensearch/OpenSearchLoader.java | 2 +- .../io/trino/plugin/opensearch/OpenSearchQueryRunner.java | 6 +++--- 6 files changed, 10 insertions(+), 10 deletions(-) diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchMetadata.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchMetadata.java index b806093429cbb..9da43db40eca1 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchMetadata.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchMetadata.java @@ -123,7 +123,7 @@ public class OpenSearchMetadata new VarcharDecoder.Descriptor(PASSTHROUGH_QUERY_RESULT_COLUMN_NAME), false)); - // See https://www.elastic.co/guide/en/elasticsearch/reference/current/regexp-syntax.html + // See https://opensearch.org/docs/latest/query-dsl/term/regexp/ private static final Set REGEXP_RESERVED_CHARACTERS = IntStream.of('.', '?', '+', '*', '|', '{', '}', '[', ']', '(', ')', '"', '#', '@', '&', '<', '>', '~') .boxed() .collect(toImmutableSet()); diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/ScanQueryPageSource.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/ScanQueryPageSource.java index 99edcbca8cd2a..eee688f9d5b4b 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/ScanQueryPageSource.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/ScanQueryPageSource.java @@ -97,11 +97,11 @@ public ScanQueryPageSource( .filter(name -> !isBuiltinColumn(name)) .collect(toList()); - // sorting by _doc (index order) get special treatment in Elasticsearch and is more efficient + // sorting by _doc (index order) get special treatment in OpenSearch and is more efficient Optional sort = Optional.of("_doc"); if (table.getQuery().isPresent()) { - // However, if we're using a custom Elasticsearch query, use default sorting. + // However, if we're using a custom OpenSearch query, use default sorting. // Documents will be scored and returned based on relevance sort = Optional.empty(); } diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/client/OpenSearchClient.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/client/OpenSearchClient.java index 0dbc9e7ac41da..b86da88413f9d 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/client/OpenSearchClient.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/client/OpenSearchClient.java @@ -438,7 +438,7 @@ public IndexMetadata getIndexMetadata(String index) return new IndexMetadata(new IndexMetadata.ObjectType(ImmutableList.of())); } if (!mappings.has("properties")) { - // Older versions of ElasticSearch supported multiple "type" mappings + // Older versions of OpenSearch supported multiple "type" mappings // for a given index. Newer versions support only one and don't // expose it in the document. Here we skip it if it's present. mappings = mappings.elements().next(); diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/BaseOpenSearchConnectorTest.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/BaseOpenSearchConnectorTest.java index 4df76815cad0b..80c00b075e3e5 100644 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/BaseOpenSearchConnectorTest.java +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/BaseOpenSearchConnectorTest.java @@ -149,14 +149,14 @@ public void testWithoutBackpressure() @Override public void testSelectAll() { - // List columns explicitly, as there's no defined order in Elasticsearch + // List columns explicitly, as there's no defined order in OpenSearch assertQuery("SELECT orderkey, custkey, orderstatus, totalprice, orderdate, orderpriority, clerk, shippriority, comment FROM orders"); } @Override protected MaterializedResult getDescribeOrdersResult() { - // The column metadata for the Elasticsearch connector tables are provided + // The column metadata for the OpenSearch connector tables are provided // based on the column name in alphabetical order. return resultBuilder(getSession(), VARCHAR, VARCHAR, VARCHAR, VARCHAR) .row("clerk", "varchar", "", "") diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/OpenSearchLoader.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/OpenSearchLoader.java index f51d1e3f8a3cf..22cc2e980b346 100644 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/OpenSearchLoader.java +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/OpenSearchLoader.java @@ -104,7 +104,7 @@ public void addResults(QueryStatusInfo statusInfo, QueryData data) request.add(new IndexRequest(tableName).source(dataBuilder)); } catch (IOException e) { - throw new UncheckedIOException("Error loading data into Elasticsearch index: " + tableName, e); + throw new UncheckedIOException("Error loading data into OpenSearch index: " + tableName, e); } } diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/OpenSearchQueryRunner.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/OpenSearchQueryRunner.java index 1d5a770931a4d..a4ea3d90b5691 100644 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/OpenSearchQueryRunner.java +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/OpenSearchQueryRunner.java @@ -60,7 +60,7 @@ public static DistributedQueryRunner createOpenSearchQueryRunner( int nodeCount) throws Exception { - return createOpenSearchQueryRunner(address, tables, extraProperties, extraConnectorProperties, nodeCount, "elasticsearch"); + return createOpenSearchQueryRunner(address, tables, extraProperties, extraConnectorProperties, nodeCount, "opensearch"); } public static DistributedQueryRunner createOpenSearchQueryRunner( @@ -123,8 +123,8 @@ private static void installOpenSearchPlugin( Map config = ImmutableMap.builder() .put("opensearch.host", address.getHost()) .put("opensearch.port", Integer.toString(address.getPort())) - // Node discovery relies on the publish_address exposed via the Elasticseach API - // This doesn't work well within a docker environment that maps ES's port to a random public port + // Node discovery relies on the publish_address exposed via the OpenSearch API + // This doesn't work well within a docker environment that maps OpenSearch port to a random public port .put("opensearch.ignore-publish-address", "true") .put("opensearch.default-schema-name", TPCH_SCHEMA) .put("opensearch.scroll-size", "1000") From 53e4b1516cc14c3dc611ebeef071956b9f953d89 Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Tue, 9 Jan 2024 12:31:44 +0100 Subject: [PATCH 303/350] Cleanup opensearch plugin pom --- plugin/trino-opensearch/pom.xml | 34 --------------------------------- 1 file changed, 34 deletions(-) diff --git a/plugin/trino-opensearch/pom.xml b/plugin/trino-opensearch/pom.xml index 76a0af78a65e6..21b26ed635300 100644 --- a/plugin/trino-opensearch/pom.xml +++ b/plugin/trino-opensearch/pom.xml @@ -156,12 +156,6 @@ org.opensearch opensearch-common ${dep.opensearch.version} - - - org.hdrhistogram - HdrHistogram - - @@ -325,13 +319,6 @@ io.trino trino-testing-services test - - - - org.openjdk.jmh - jmh-core - - @@ -388,25 +375,4 @@ test - - - - - - org.apache.maven.plugins - maven-enforcer-plugin - - - - - - org.apache.lucene:lucene-core - - - - - - - - From d11d42fccbd6f600fa252c444a807d2bbaa8a39c Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Tue, 9 Jan 2024 13:15:37 +0100 Subject: [PATCH 304/350] Migrate to official OpenSearch testcontainers --- plugin/trino-opensearch/pom.xml | 5 +++-- .../plugin/opensearch/OpenSearchServer.java | 18 ++++++++---------- 2 files changed, 11 insertions(+), 12 deletions(-) diff --git a/plugin/trino-opensearch/pom.xml b/plugin/trino-opensearch/pom.xml index 21b26ed635300..5808a50ef4255 100644 --- a/plugin/trino-opensearch/pom.xml +++ b/plugin/trino-opensearch/pom.xml @@ -364,8 +364,9 @@ - org.testcontainers - elasticsearch + org.opensearch + opensearch-testcontainers + 2.0.1 test diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/OpenSearchServer.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/OpenSearchServer.java index dfd262bb7f90b..8139098faffb9 100644 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/OpenSearchServer.java +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/OpenSearchServer.java @@ -15,9 +15,8 @@ import com.google.common.net.HostAndPort; import io.trino.testing.ResourcePresence; +import org.opensearch.testcontainers.OpensearchContainer; import org.testcontainers.containers.Network; -import org.testcontainers.elasticsearch.ElasticsearchContainer; -import org.testcontainers.utility.DockerImageName; import java.io.IOException; import java.nio.file.Files; @@ -35,7 +34,7 @@ public class OpenSearchServer public static final String OPENSEARCH_IMAGE = "opensearchproject/opensearch:2.11.0"; private final Path configurationPath; - private final ElasticsearchContainer container; + private final OpensearchContainer container; public OpenSearchServer(String image, boolean secured, Map configurationFiles) throws IOException @@ -46,12 +45,11 @@ public OpenSearchServer(String image, boolean secured, Map confi public OpenSearchServer(Network network, String image, boolean secured, Map configurationFiles) throws IOException { - DockerImageName dockerImageName = DockerImageName.parse(image).asCompatibleSubstituteFor("docker.elastic.co/elasticsearch/elasticsearch"); - container = new ElasticsearchContainer(dockerImageName); + container = new OpensearchContainer<>(image); container.withNetwork(network); - container.withReuse(false); - container.withNetworkAliases("opensearch-server"); - container.withEnv("DISABLE_SECURITY_PLUGIN", Boolean.toString(!secured)); + if (secured) { + container.withSecurityEnabled(); + } configurationPath = createTempDirectory(null); for (Map.Entry entry : configurationFiles.entrySet()) { @@ -76,11 +74,11 @@ public void stop() @ResourcePresence public boolean isRunning() { - return container.getContainerId() != null; + return container.isRunning(); } public HostAndPort getAddress() { - return HostAndPort.fromString(container.getHttpHostAddress()); + return HostAndPort.fromParts(container.getHost(), container.getMappedPort(9200)); } } From 40da861fd27947c74011ee407555655832f17adc Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Tue, 9 Jan 2024 18:08:55 +0100 Subject: [PATCH 305/350] Drop legacy full-text search support --- docs/src/main/sphinx/connector/opensearch.md | 10 ---------- .../plugin/opensearch/OpenSearchMetadata.java | 11 ++--------- .../opensearch/BaseOpenSearchConnectorTest.java | 16 ---------------- 3 files changed, 2 insertions(+), 35 deletions(-) diff --git a/docs/src/main/sphinx/connector/opensearch.md b/docs/src/main/sphinx/connector/opensearch.md index 544ce9bb22756..4885edcb0a27f 100644 --- a/docs/src/main/sphinx/connector/opensearch.md +++ b/docs/src/main/sphinx/connector/opensearch.md @@ -317,16 +317,6 @@ The following hidden columns are available: | \_score | The document score returned by the OpenSearch query | | \_source | The source of the original document | -(opensearch-full-text-queries)= -## Full text queries - -Trino SQL queries can be combined with OpenSearch queries by providing the [full text query] -as part of the table name, separated by a colon. For example: - -```sql -SELECT * FROM "tweets: +trino SQL^2" -``` - ## Predicate push down The connector supports [predicate push down](predicate-pushdown) for the following data types: diff --git a/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchMetadata.java b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchMetadata.java index 9da43db40eca1..d013f117e7c8b 100644 --- a/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchMetadata.java +++ b/plugin/trino-opensearch/src/main/java/io/trino/plugin/opensearch/OpenSearchMetadata.java @@ -152,15 +152,8 @@ public OpenSearchTableHandle getTableHandle(ConnectorSession session, SchemaTabl requireNonNull(tableName, "tableName is null"); if (tableName.getSchemaName().equals(schemaName)) { - String[] parts = tableName.getTableName().split(":", 2); - String table = parts[0]; - Optional query = Optional.empty(); - if (parts.length == 2) { - query = Optional.of(parts[1]); - } - - if (client.indexExists(table) && !client.getIndexMetadata(table).getSchema().getFields().isEmpty()) { - return new OpenSearchTableHandle(OpenSearchTableHandle.Type.SCAN, schemaName, table, query); + if (client.indexExists(tableName.getTableName()) && !client.getIndexMetadata(tableName.getTableName()).getSchema().getFields().isEmpty()) { + return new OpenSearchTableHandle(OpenSearchTableHandle.Type.SCAN, schemaName, tableName.getTableName(), Optional.empty()); } } diff --git a/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/BaseOpenSearchConnectorTest.java b/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/BaseOpenSearchConnectorTest.java index 80c00b075e3e5..3e727a8facb06 100644 --- a/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/BaseOpenSearchConnectorTest.java +++ b/plugin/trino-opensearch/src/test/java/io/trino/plugin/opensearch/BaseOpenSearchConnectorTest.java @@ -1468,9 +1468,6 @@ public void testFilters() .put("ipv6_column", "2001:db8:0:0:1:0:0:1") .buildOrThrow()); - // _score column - assertQuery("SELECT count(*) FROM \"filter_pushdown: cool\" WHERE _score > 0", "VALUES 1"); - // boolean assertQuery("SELECT count(*) FROM filter_pushdown WHERE boolean_column = true", "VALUES 1"); assertQuery("SELECT count(*) FROM filter_pushdown WHERE boolean_column = false", "VALUES 0"); @@ -1692,12 +1689,6 @@ public void testNestedTypeDataTypesNested() assertThat(rows.getMaterializedRows()).isEqualTo(expected.getMaterializedRows()); } - @Test - public void testQueryString() - { - assertQuery("SELECT count(*) FROM \"orders: +packages -slyly\"", "VALUES 1639"); - } - @Test public void testMixedCase() throws IOException @@ -1740,13 +1731,6 @@ public void testNumericKeyword() "VALUES 20"); } - @Test - public void testQueryStringError() - { - assertQueryFails("SELECT orderkey FROM \"orders: ++foo AND\"", "\\QFailed to parse query [ ++foo and]\\E"); - assertQueryFails("SELECT count(*) FROM \"orders: ++foo AND\"", "\\QFailed to parse query [ ++foo and]\\E"); - } - @Test public void testAlias() throws IOException From a09174d1ef441bd614345f6601fe057720de1d69 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 10 Jan 2024 09:54:20 +0100 Subject: [PATCH 306/350] Fix formatting --- .../src/main/java/io/trino/server/CoordinatorModule.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/trino-main/src/main/java/io/trino/server/CoordinatorModule.java b/core/trino-main/src/main/java/io/trino/server/CoordinatorModule.java index 613edeb3335ef..7ccb787826cf0 100644 --- a/core/trino-main/src/main/java/io/trino/server/CoordinatorModule.java +++ b/core/trino-main/src/main/java/io/trino/server/CoordinatorModule.java @@ -245,7 +245,8 @@ protected void setup(Binder binder) binder.bind(ByEagerParentOutputDataSizeEstimator.Factory.class).in(Scopes.SINGLETON); // use provider method returning list to ensure ordering // OutputDataSizeEstimator factories are ordered starting from most accurate - install(new AbstractConfigurationAwareModule() { + install(new AbstractConfigurationAwareModule() + { @Override protected void setup(Binder binder) {} From 7493acc449d0a97ed2adf6b8adf3c86847bb8514 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 10 Jan 2024 10:02:54 +0100 Subject: [PATCH 307/350] Extract version embedding to a helper --- .../io/trino/dispatcher/DispatchExecutor.java | 19 ++---------- .../DecoratingListeningExecutorService.java | 2 +- .../main/java/io/trino/util/Executors.java | 30 +++++++++++++++++++ ...estDecoratingListeningExecutorService.java | 2 +- 4 files changed, 34 insertions(+), 19 deletions(-) rename core/trino-main/src/main/java/io/trino/{dispatcher => util}/DecoratingListeningExecutorService.java (99%) rename core/trino-main/src/test/java/io/trino/{dispatcher => util}/TestDecoratingListeningExecutorService.java (97%) diff --git a/core/trino-main/src/main/java/io/trino/dispatcher/DispatchExecutor.java b/core/trino-main/src/main/java/io/trino/dispatcher/DispatchExecutor.java index 7acd03cc6812b..54b979e503fc0 100644 --- a/core/trino-main/src/main/java/io/trino/dispatcher/DispatchExecutor.java +++ b/core/trino-main/src/main/java/io/trino/dispatcher/DispatchExecutor.java @@ -25,13 +25,13 @@ import org.weakref.jmx.Managed; import org.weakref.jmx.Nested; -import java.util.concurrent.Callable; import java.util.concurrent.ExecutorService; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ThreadPoolExecutor; import static com.google.common.util.concurrent.MoreExecutors.listeningDecorator; import static io.airlift.concurrent.Threads.daemonThreadsNamed; +import static io.trino.util.Executors.decorateWithVersion; import static java.util.Objects.requireNonNull; import static java.util.concurrent.Executors.newCachedThreadPool; import static java.util.concurrent.Executors.newScheduledThreadPool; @@ -51,22 +51,7 @@ public DispatchExecutor(QueryManagerConfig config, VersionEmbedder versionEmbedd { ExecutorService coreExecutor = newCachedThreadPool(daemonThreadsNamed("dispatcher-query-%s")); closer.register(coreExecutor::shutdownNow); - executor = new DecoratingListeningExecutorService( - listeningDecorator(coreExecutor), - new DecoratingListeningExecutorService.TaskDecorator() - { - @Override - public Runnable decorate(Runnable command) - { - return versionEmbedder.embedVersion(command); - } - - @Override - public Callable decorate(Callable task) - { - return versionEmbedder.embedVersion(task); - } - }); + executor = decorateWithVersion(coreExecutor, versionEmbedder); ScheduledExecutorService coreScheduledExecutor = newScheduledThreadPool(config.getQueryManagerExecutorPoolSize(), daemonThreadsNamed("dispatch-executor-%s")); closer.register(coreScheduledExecutor::shutdownNow); diff --git a/core/trino-main/src/main/java/io/trino/dispatcher/DecoratingListeningExecutorService.java b/core/trino-main/src/main/java/io/trino/util/DecoratingListeningExecutorService.java similarity index 99% rename from core/trino-main/src/main/java/io/trino/dispatcher/DecoratingListeningExecutorService.java rename to core/trino-main/src/main/java/io/trino/util/DecoratingListeningExecutorService.java index b53b4d9d43104..405037f6a14fe 100644 --- a/core/trino-main/src/main/java/io/trino/dispatcher/DecoratingListeningExecutorService.java +++ b/core/trino-main/src/main/java/io/trino/util/DecoratingListeningExecutorService.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.dispatcher; +package io.trino.util; import com.google.common.util.concurrent.ForwardingListeningExecutorService; import com.google.common.util.concurrent.ListenableFuture; diff --git a/core/trino-main/src/main/java/io/trino/util/Executors.java b/core/trino-main/src/main/java/io/trino/util/Executors.java index c98e087406a16..1062de31a53df 100644 --- a/core/trino-main/src/main/java/io/trino/util/Executors.java +++ b/core/trino-main/src/main/java/io/trino/util/Executors.java @@ -13,6 +13,9 @@ */ package io.trino.util; +import com.google.common.util.concurrent.ListeningExecutorService; +import io.trino.spi.VersionEmbedder; + import java.util.ArrayList; import java.util.Collection; import java.util.List; @@ -20,8 +23,10 @@ import java.util.concurrent.CompletionService; import java.util.concurrent.Executor; import java.util.concurrent.ExecutorCompletionService; +import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; +import static com.google.common.util.concurrent.MoreExecutors.listeningDecorator; import static io.airlift.concurrent.MoreFutures.getDone; public final class Executors @@ -66,4 +71,29 @@ private static Future take(CompletionService completionService) throw new RuntimeException("Interrupted", e); } } + + public static ListeningExecutorService decorateWithVersion(ExecutorService executorService, VersionEmbedder versionEmbedder) + { + return decorateWithVersion(listeningDecorator(executorService), versionEmbedder); + } + + public static ListeningExecutorService decorateWithVersion(ListeningExecutorService executorService, VersionEmbedder versionEmbedder) + { + return new DecoratingListeningExecutorService( + executorService, + new DecoratingListeningExecutorService.TaskDecorator() + { + @Override + public Runnable decorate(Runnable command) + { + return versionEmbedder.embedVersion(command); + } + + @Override + public Callable decorate(Callable task) + { + return versionEmbedder.embedVersion(task); + } + }); + } } diff --git a/core/trino-main/src/test/java/io/trino/dispatcher/TestDecoratingListeningExecutorService.java b/core/trino-main/src/test/java/io/trino/util/TestDecoratingListeningExecutorService.java similarity index 97% rename from core/trino-main/src/test/java/io/trino/dispatcher/TestDecoratingListeningExecutorService.java rename to core/trino-main/src/test/java/io/trino/util/TestDecoratingListeningExecutorService.java index f942a64f73109..ab45726cb3a0b 100644 --- a/core/trino-main/src/test/java/io/trino/dispatcher/TestDecoratingListeningExecutorService.java +++ b/core/trino-main/src/test/java/io/trino/util/TestDecoratingListeningExecutorService.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.dispatcher; +package io.trino.util; import com.google.common.util.concurrent.ListeningExecutorService; import org.junit.jupiter.api.Test; From 03f3315150fbcc3d3706492d6bcd2ab3f8870b68 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 10 Jan 2024 09:54:25 +0100 Subject: [PATCH 308/350] Extract query executor binding to a method --- .../io/trino/server/CoordinatorModule.java | 24 ++++++++++++------- 1 file changed, 15 insertions(+), 9 deletions(-) diff --git a/core/trino-main/src/main/java/io/trino/server/CoordinatorModule.java b/core/trino-main/src/main/java/io/trino/server/CoordinatorModule.java index 7ccb787826cf0..8ef36ecb8041d 100644 --- a/core/trino-main/src/main/java/io/trino/server/CoordinatorModule.java +++ b/core/trino-main/src/main/java/io/trino/server/CoordinatorModule.java @@ -334,15 +334,6 @@ List getCompositeOutputDataSizeEstimatorDelegate .toInstance(newSingleThreadScheduledExecutor(threadsNamed("stage-scheduler"))); // query execution - QueryManagerConfig queryManagerConfig = buildConfigObject(QueryManagerConfig.class); - ThreadPoolExecutor queryExecutor = new ThreadPoolExecutor( - queryManagerConfig.getQueryExecutorPoolSize(), - queryManagerConfig.getQueryExecutorPoolSize(), - 60, SECONDS, - new LinkedBlockingQueue<>(1000), - threadsNamed("query-execution-%s")); - queryExecutor.allowCoreThreadTimeOut(true); - binder.bind(ExecutorService.class).annotatedWith(ForQueryExecution.class).toInstance(queryExecutor); binder.bind(QueryExecutionMBean.class).in(Scopes.SINGLETON); newExporter(binder).export(QueryExecutionMBean.class) .as(generator -> generator.generatedNameOf(QueryExecution.class)); @@ -386,6 +377,21 @@ public static ResourceGroupManager getResourceGroupManager(@SuppressWarnings( return manager; } + @Provides + @Singleton + @ForQueryExecution + public static ExecutorService createQueryExecutor(QueryManagerConfig queryManagerConfig) + { + ThreadPoolExecutor queryExecutor = new ThreadPoolExecutor( + queryManagerConfig.getQueryExecutorPoolSize(), + queryManagerConfig.getQueryExecutorPoolSize(), + 60, SECONDS, + new LinkedBlockingQueue<>(1000), + threadsNamed("query-execution-%s")); + queryExecutor.allowCoreThreadTimeOut(true); + return queryExecutor; + } + @Provides @Singleton public static QueryPerformanceFetcher createQueryPerformanceFetcher(QueryManager queryManager) From 30575d063e90bc528d0e49b79440098972b8f645 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 10 Jan 2024 10:03:00 +0100 Subject: [PATCH 309/350] Embed version in scheduler stacktraces --- .../trino/execution/QueryExecutionMBean.java | 2 +- .../execution/QueryExecutorInternal.java | 29 +++++++++++++++++++ .../io/trino/server/CoordinatorModule.java | 13 ++++++++- 3 files changed, 42 insertions(+), 2 deletions(-) create mode 100644 core/trino-main/src/main/java/io/trino/execution/QueryExecutorInternal.java diff --git a/core/trino-main/src/main/java/io/trino/execution/QueryExecutionMBean.java b/core/trino-main/src/main/java/io/trino/execution/QueryExecutionMBean.java index cebb0948a5032..ccb8e18e4854d 100644 --- a/core/trino-main/src/main/java/io/trino/execution/QueryExecutionMBean.java +++ b/core/trino-main/src/main/java/io/trino/execution/QueryExecutionMBean.java @@ -26,7 +26,7 @@ public class QueryExecutionMBean private final ThreadPoolExecutorMBean executorMBean; @Inject - public QueryExecutionMBean(@ForQueryExecution ExecutorService executor) + public QueryExecutionMBean(@QueryExecutorInternal ExecutorService executor) { this.executorMBean = new ThreadPoolExecutorMBean((ThreadPoolExecutor) executor); } diff --git a/core/trino-main/src/main/java/io/trino/execution/QueryExecutorInternal.java b/core/trino-main/src/main/java/io/trino/execution/QueryExecutorInternal.java new file mode 100644 index 0000000000000..39d798b567bac --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/execution/QueryExecutorInternal.java @@ -0,0 +1,29 @@ +/* + * 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.execution; + +import com.google.inject.BindingAnnotation; + +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}) +@BindingAnnotation +public @interface QueryExecutorInternal {} diff --git a/core/trino-main/src/main/java/io/trino/server/CoordinatorModule.java b/core/trino-main/src/main/java/io/trino/server/CoordinatorModule.java index 8ef36ecb8041d..f51575bc57b3f 100644 --- a/core/trino-main/src/main/java/io/trino/server/CoordinatorModule.java +++ b/core/trino-main/src/main/java/io/trino/server/CoordinatorModule.java @@ -49,6 +49,7 @@ import io.trino.execution.ForQueryExecution; import io.trino.execution.QueryExecution; import io.trino.execution.QueryExecutionMBean; +import io.trino.execution.QueryExecutorInternal; import io.trino.execution.QueryIdGenerator; import io.trino.execution.QueryManager; import io.trino.execution.QueryManagerConfig; @@ -105,6 +106,7 @@ import io.trino.server.remotetask.RemoteTaskStats; import io.trino.server.ui.WebUiModule; import io.trino.server.ui.WorkerResource; +import io.trino.spi.VersionEmbedder; import io.trino.spi.memory.ClusterMemoryPoolManager; import io.trino.sql.PlannerContext; import io.trino.sql.analyzer.AnalyzerFactory; @@ -141,6 +143,7 @@ import static io.airlift.json.JsonCodecBinder.jsonCodecBinder; import static io.airlift.units.DataSize.Unit.MEGABYTE; import static io.trino.server.InternalCommunicationHttpClientModule.internalHttpClientModule; +import static io.trino.util.Executors.decorateWithVersion; import static java.util.concurrent.Executors.newCachedThreadPool; import static java.util.concurrent.Executors.newScheduledThreadPool; import static java.util.concurrent.Executors.newSingleThreadScheduledExecutor; @@ -379,7 +382,7 @@ public static ResourceGroupManager getResourceGroupManager(@SuppressWarnings( @Provides @Singleton - @ForQueryExecution + @QueryExecutorInternal public static ExecutorService createQueryExecutor(QueryManagerConfig queryManagerConfig) { ThreadPoolExecutor queryExecutor = new ThreadPoolExecutor( @@ -392,6 +395,14 @@ public static ExecutorService createQueryExecutor(QueryManagerConfig queryManage return queryExecutor; } + @Provides + @Singleton + @ForQueryExecution + public static ExecutorService createQueryExecutor(@QueryExecutorInternal ExecutorService queryExecutor, VersionEmbedder versionEmbedder) + { + return decorateWithVersion(queryExecutor, versionEmbedder); + } + @Provides @Singleton public static QueryPerformanceFetcher createQueryPerformanceFetcher(QueryManager queryManager) From d8f8bf2eba8933226247dcc891b35d066934ebed Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 8 Dec 2023 15:47:35 +0100 Subject: [PATCH 310/350] Fix code indentation --- .../jdbc/TestDefaultJdbcQueryBuilder.java | 98 +++++++++---------- 1 file changed, 49 insertions(+), 49 deletions(-) diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestDefaultJdbcQueryBuilder.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestDefaultJdbcQueryBuilder.java index 0c04b64d4ad7e..c58a9b5633d21 100644 --- a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestDefaultJdbcQueryBuilder.java +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestDefaultJdbcQueryBuilder.java @@ -194,41 +194,41 @@ public void testNormalBuildSql() { TupleDomain tupleDomain = TupleDomain.withColumnDomains(ImmutableMap.builder() .put(columns.get(0), Domain.create(SortedRangeSet.copyOf(BIGINT, - ImmutableList.of( - Range.equal(BIGINT, 128L), - Range.equal(BIGINT, 180L), - Range.equal(BIGINT, 233L), - Range.lessThan(BIGINT, 25L), - Range.range(BIGINT, 66L, true, 96L, true), - Range.greaterThan(BIGINT, 192L))), + ImmutableList.of( + Range.equal(BIGINT, 128L), + Range.equal(BIGINT, 180L), + Range.equal(BIGINT, 233L), + Range.lessThan(BIGINT, 25L), + Range.range(BIGINT, 66L, true, 96L, true), + Range.greaterThan(BIGINT, 192L))), false)) .put(columns.get(1), Domain.create(SortedRangeSet.copyOf(DOUBLE, - ImmutableList.of( - Range.equal(DOUBLE, 200011.0), - Range.equal(DOUBLE, 200014.0), - Range.equal(DOUBLE, 200017.0), - Range.equal(DOUBLE, 200116.5), - Range.range(DOUBLE, 200030.0, true, 200036.0, true), - Range.range(DOUBLE, 200048.0, true, 200099.0, true))), + ImmutableList.of( + Range.equal(DOUBLE, 200011.0), + Range.equal(DOUBLE, 200014.0), + Range.equal(DOUBLE, 200017.0), + Range.equal(DOUBLE, 200116.5), + Range.range(DOUBLE, 200030.0, true, 200036.0, true), + Range.range(DOUBLE, 200048.0, true, 200099.0, true))), false)) .put(columns.get(7), Domain.create(SortedRangeSet.copyOf(TINYINT, - ImmutableList.of( - Range.range(TINYINT, 60L, true, 70L, false), - Range.range(TINYINT, 52L, true, 55L, false))), + ImmutableList.of( + Range.range(TINYINT, 60L, true, 70L, false), + Range.range(TINYINT, 52L, true, 55L, false))), false)) .put(columns.get(8), Domain.create(SortedRangeSet.copyOf(SMALLINT, - ImmutableList.of( - Range.range(SMALLINT, -75L, true, -68L, true), - Range.range(SMALLINT, -200L, true, -100L, false))), + ImmutableList.of( + Range.range(SMALLINT, -75L, true, -68L, true), + Range.range(SMALLINT, -200L, true, -100L, false))), false)) .put(columns.get(9), Domain.create(SortedRangeSet.copyOf(INTEGER, - ImmutableList.of( - Range.equal(INTEGER, 80L), - Range.equal(INTEGER, 96L), - Range.lessThan(INTEGER, 0L))), + ImmutableList.of( + Range.equal(INTEGER, 80L), + Range.equal(INTEGER, 96L), + Range.lessThan(INTEGER, 0L))), false)) .put(columns.get(2), Domain.create(SortedRangeSet.copyOf(BOOLEAN, - ImmutableList.of(Range.equal(BOOLEAN, true))), + ImmutableList.of(Range.equal(BOOLEAN, true))), false)) .buildOrThrow()); @@ -309,10 +309,10 @@ public void testBuildSqlWithFloat() { TupleDomain tupleDomain = TupleDomain.withColumnDomains(ImmutableMap.of( columns.get(10), Domain.create(SortedRangeSet.copyOf(REAL, - ImmutableList.of( - Range.equal(REAL, (long) floatToRawIntBits(100.0f + 0)), - Range.equal(REAL, (long) floatToRawIntBits(100.008f + 0)), - Range.equal(REAL, (long) floatToRawIntBits(100.0f + 14)))), + ImmutableList.of( + Range.equal(REAL, (long) floatToRawIntBits(100.0f + 0)), + Range.equal(REAL, (long) floatToRawIntBits(100.008f + 0)), + Range.equal(REAL, (long) floatToRawIntBits(100.0f + 14)))), false))); Connection connection = database.getConnection(); @@ -343,10 +343,10 @@ public void testBuildSqlWithVarchar() { TupleDomain tupleDomain = TupleDomain.withColumnDomains(ImmutableMap.of( columns.get(3), Domain.create(SortedRangeSet.copyOf(VARCHAR, - ImmutableList.of( - Range.range(VARCHAR, utf8Slice("test_str_700"), true, utf8Slice("test_str_702"), false), - Range.equal(VARCHAR, utf8Slice("test_str_180")), - Range.equal(VARCHAR, utf8Slice("test_str_196")))), + ImmutableList.of( + Range.range(VARCHAR, utf8Slice("test_str_700"), true, utf8Slice("test_str_702"), false), + Range.equal(VARCHAR, utf8Slice("test_str_180")), + Range.equal(VARCHAR, utf8Slice("test_str_196")))), false))); Connection connection = database.getConnection(); @@ -379,10 +379,10 @@ public void testBuildSqlWithChar() CharType charType = CharType.createCharType(0); TupleDomain tupleDomain = TupleDomain.withColumnDomains(ImmutableMap.of( columns.get(11), Domain.create(SortedRangeSet.copyOf(charType, - ImmutableList.of( - Range.range(charType, utf8Slice("test_str_700"), true, utf8Slice("test_str_702"), false), - Range.equal(charType, utf8Slice("test_str_180")), - Range.equal(charType, utf8Slice("test_str_196")))), + ImmutableList.of( + Range.range(charType, utf8Slice("test_str_700"), true, utf8Slice("test_str_702"), false), + Range.equal(charType, utf8Slice("test_str_180")), + Range.equal(charType, utf8Slice("test_str_196")))), false))); Connection connection = database.getConnection(); @@ -419,16 +419,16 @@ public void testBuildSqlWithDateTime() { TupleDomain tupleDomain = TupleDomain.withColumnDomains(ImmutableMap.of( columns.get(4), Domain.create(SortedRangeSet.copyOf(DATE, - ImmutableList.of( - Range.range(DATE, toDays(2016, 6, 7), true, toDays(2016, 6, 17), false), - Range.equal(DATE, toDays(2016, 6, 3)), - Range.equal(DATE, toDays(2016, 10, 21)))), + ImmutableList.of( + Range.range(DATE, toDays(2016, 6, 7), true, toDays(2016, 6, 17), false), + Range.equal(DATE, toDays(2016, 6, 3)), + Range.equal(DATE, toDays(2016, 10, 21)))), false), columns.get(5), Domain.create(SortedRangeSet.copyOf(TIME_MILLIS, - ImmutableList.of( - Range.range(TIME_MILLIS, toTimeRepresentation(6, 12, 23), false, toTimeRepresentation(8, 23, 37), true), - Range.equal(TIME_MILLIS, toTimeRepresentation(2, 3, 4)), - Range.equal(TIME_MILLIS, toTimeRepresentation(20, 23, 37)))), + ImmutableList.of( + Range.range(TIME_MILLIS, toTimeRepresentation(6, 12, 23), false, toTimeRepresentation(8, 23, 37), true), + Range.equal(TIME_MILLIS, toTimeRepresentation(2, 3, 4)), + Range.equal(TIME_MILLIS, toTimeRepresentation(20, 23, 37)))), false))); Connection connection = database.getConnection(); @@ -472,10 +472,10 @@ public void testBuildSqlWithTimestamp() { TupleDomain tupleDomain = TupleDomain.withColumnDomains(ImmutableMap.of( columns.get(6), Domain.create(SortedRangeSet.copyOf(TIMESTAMP_MILLIS, - ImmutableList.of( - Range.equal(TIMESTAMP_MILLIS, toTrinoTimestamp(2016, 6, 3, 0, 23, 37)), - Range.equal(TIMESTAMP_MILLIS, toTrinoTimestamp(2016, 10, 19, 16, 23, 37)), - Range.range(TIMESTAMP_MILLIS, toTrinoTimestamp(2016, 6, 7, 8, 23, 37), false, toTrinoTimestamp(2016, 6, 9, 12, 23, 37), true))), + ImmutableList.of( + Range.equal(TIMESTAMP_MILLIS, toTrinoTimestamp(2016, 6, 3, 0, 23, 37)), + Range.equal(TIMESTAMP_MILLIS, toTrinoTimestamp(2016, 10, 19, 16, 23, 37)), + Range.range(TIMESTAMP_MILLIS, toTrinoTimestamp(2016, 6, 7, 8, 23, 37), false, toTrinoTimestamp(2016, 6, 9, 12, 23, 37), true))), false))); Connection connection = database.getConnection(); From d69f50b8aaed2c0e8927078401c776644823929c Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Tue, 19 Dec 2023 14:50:28 +0100 Subject: [PATCH 311/350] Add some traceability which option is failing --- .../java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java | 7 +++++++ .../plugin/postgresql/TestPostgreSqlConnectorTest.java | 7 +++++++ 2 files changed, 14 insertions(+) diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java index 97c158051b2b8..40eb2f8da22e0 100644 --- a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java @@ -14,6 +14,7 @@ package io.trino.plugin.jdbc; import com.google.common.collect.ImmutableList; +import io.airlift.log.Logger; import io.airlift.units.Duration; import io.trino.Session; import io.trino.spi.QueryId; @@ -124,6 +125,8 @@ public abstract class BaseJdbcConnectorTest extends BaseConnectorTest { + private static final Logger log = Logger.get(BaseJdbcConnectorTest.class); + private final ExecutorService executor = newCachedThreadPool(daemonThreadsNamed(getClass().getName())); protected abstract SqlExecutor onRemoteDatabase(); @@ -1196,6 +1199,8 @@ public void testJoinPushdown() "nation_lowercase", "AS SELECT nationkey, lower(name) name, regionkey FROM nation")) { for (JoinOperator joinOperator : JoinOperator.values()) { + log.info("Testing joinOperator=%s", joinOperator); + if (joinOperator == FULL_JOIN && !hasBehavior(SUPPORTS_JOIN_PUSHDOWN_WITH_FULL_JOIN)) { assertThat(query(session, "SELECT r.name, n.name FROM nation n FULL JOIN region r ON n.regionkey = r.regionkey")) .joinIsNotFullyPushedDown(); @@ -1255,6 +1260,7 @@ public void testJoinPushdown() // inequality along with an equality, which constitutes an equi-condition and allows filter to remain as part of the Join for (String operator : nonEqualities) { + log.info("Testing [joinOperator=%s] operator=%s on number", joinOperator, operator); assertJoinConditionallyPushedDown( session, format("SELECT n.name, c.name FROM nation n %s customer c ON n.nationkey = c.nationkey AND n.regionkey %s c.custkey", joinOperator, operator), @@ -1263,6 +1269,7 @@ public void testJoinPushdown() // varchar inequality along with an equality, which constitutes an equi-condition and allows filter to remain as part of the Join for (String operator : nonEqualities) { + log.info("Testing [joinOperator=%s] operator=%s on varchar", joinOperator, operator); assertJoinConditionallyPushedDown( session, format("SELECT n.name, nl.name FROM nation n %s %s nl ON n.regionkey = nl.regionkey AND n.name %s nl.name", joinOperator, nationLowercaseTable.getName(), operator), diff --git a/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestPostgreSqlConnectorTest.java b/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestPostgreSqlConnectorTest.java index 8170cdcff5f2e..77d01fc9de471 100644 --- a/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestPostgreSqlConnectorTest.java +++ b/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestPostgreSqlConnectorTest.java @@ -15,6 +15,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import io.airlift.log.Logger; import io.airlift.units.Duration; import io.trino.Session; import io.trino.plugin.jdbc.BaseJdbcConnectorTest; @@ -80,6 +81,8 @@ public class TestPostgreSqlConnectorTest extends BaseJdbcConnectorTest { + private static final Logger log = Logger.get(TestPostgreSqlConnectorTest.class); + protected TestingPostgreSqlServer postgreSqlServer; @Override @@ -641,6 +644,8 @@ public void testStringJoinPushdownWithCollate() // inequality for (String operator : nonEqualities) { + log.info("Testing operator=%s", operator); + // bigint inequality predicate assertThat(query(withoutDynamicFiltering, format("SELECT r.name, n.name FROM nation n JOIN region r ON n.regionkey %s r.regionkey", operator))) // Currently no pushdown as inequality predicate is removed from Join to maintain Cross Join and Filter as separate nodes @@ -654,6 +659,7 @@ public void testStringJoinPushdownWithCollate() // inequality along with an equality, which constitutes an equi-condition and allows filter to remain as part of the Join for (String operator : nonEqualities) { + log.info("Testing operator=%s", operator); assertConditionallyPushedDown( session, format("SELECT n.name, c.name FROM nation n JOIN customer c ON n.nationkey = c.nationkey AND n.regionkey %s c.custkey", operator), @@ -663,6 +669,7 @@ public void testStringJoinPushdownWithCollate() // varchar inequality along with an equality, which constitutes an equi-condition and allows filter to remain as part of the Join for (String operator : nonEqualities) { + log.info("Testing operator=%s", operator); assertConditionallyPushedDown( session, format("SELECT n.name, nl.name FROM nation n JOIN %s nl ON n.regionkey = nl.regionkey AND n.name %s nl.name", nationLowercaseTable.getName(), operator), From 30990323f369fb62c4cc664237ff43b00b0dca78 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 20 Dec 2023 10:50:03 +0100 Subject: [PATCH 312/350] Move extractVariables utility --- .../io/trino/metadata/MetadataManager.java | 4 +- .../sql/planner/ConnectorExpressions.java | 46 ------------------- .../base/expression/ConnectorExpressions.java | 22 +++++++++ 3 files changed, 24 insertions(+), 48 deletions(-) delete mode 100644 core/trino-main/src/main/java/io/trino/sql/planner/ConnectorExpressions.java diff --git a/core/trino-main/src/main/java/io/trino/metadata/MetadataManager.java b/core/trino-main/src/main/java/io/trino/metadata/MetadataManager.java index 9c9221e38ebf2..67e5666663a3b 100644 --- a/core/trino-main/src/main/java/io/trino/metadata/MetadataManager.java +++ b/core/trino-main/src/main/java/io/trino/metadata/MetadataManager.java @@ -112,7 +112,6 @@ import io.trino.spi.type.TypeOperators; import io.trino.sql.analyzer.TypeSignatureProvider; import io.trino.sql.parser.SqlParser; -import io.trino.sql.planner.ConnectorExpressions; import io.trino.sql.planner.PartitioningHandle; import io.trino.sql.tree.QualifiedName; import io.trino.transaction.TransactionManager; @@ -158,6 +157,7 @@ import static io.trino.metadata.RedirectionAwareTableHandle.noRedirection; import static io.trino.metadata.RedirectionAwareTableHandle.withRedirectionTo; import static io.trino.metadata.SignatureBinder.applyBoundVariables; +import static io.trino.plugin.base.expression.ConnectorExpressions.extractVariables; import static io.trino.spi.ErrorType.EXTERNAL; import static io.trino.spi.StandardErrorCode.FUNCTION_IMPLEMENTATION_ERROR; import static io.trino.spi.StandardErrorCode.FUNCTION_IMPLEMENTATION_MISSING; @@ -2040,7 +2040,7 @@ private void verifyProjection(TableHandle table, List proje .map(Assignment::getVariable) .collect(toImmutableSet()); projections.stream() - .flatMap(connectorExpression -> ConnectorExpressions.extractVariables(connectorExpression).stream()) + .flatMap(connectorExpression -> extractVariables(connectorExpression).stream()) .map(Variable::getName) .filter(variableName -> !assignedVariables.contains(variableName)) .findAny() diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/ConnectorExpressions.java b/core/trino-main/src/main/java/io/trino/sql/planner/ConnectorExpressions.java deleted file mode 100644 index c7ecbec765281..0000000000000 --- a/core/trino-main/src/main/java/io/trino/sql/planner/ConnectorExpressions.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * 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.sql.planner; - -import com.google.common.graph.SuccessorsFunction; -import com.google.common.graph.Traverser; -import io.trino.spi.expression.ConnectorExpression; -import io.trino.spi.expression.Variable; - -import java.util.List; -import java.util.stream.Stream; - -import static com.google.common.collect.ImmutableList.toImmutableList; -import static com.google.common.collect.Streams.stream; -import static java.util.Objects.requireNonNull; - -public final class ConnectorExpressions -{ - private ConnectorExpressions() {} - - public static List extractVariables(ConnectorExpression expression) - { - return preOrder(expression) - .filter(Variable.class::isInstance) - .map(Variable.class::cast) - .collect(toImmutableList()); - } - - public static Stream preOrder(ConnectorExpression expression) - { - return stream( - Traverser.forTree((SuccessorsFunction) ConnectorExpression::getChildren) - .depthFirstPreOrder(requireNonNull(expression, "expression is null"))); - } -} diff --git a/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/expression/ConnectorExpressions.java b/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/expression/ConnectorExpressions.java index 01fc9a38cf61e..3971a2995cc25 100644 --- a/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/expression/ConnectorExpressions.java +++ b/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/expression/ConnectorExpressions.java @@ -14,21 +14,36 @@ package io.trino.plugin.base.expression; import com.google.common.collect.ImmutableList; +import com.google.common.graph.SuccessorsFunction; +import com.google.common.graph.Traverser; import io.trino.spi.expression.Call; import io.trino.spi.expression.ConnectorExpression; +import io.trino.spi.expression.Variable; import java.util.Arrays; import java.util.List; +import java.util.stream.Stream; +import static com.google.common.collect.ImmutableList.toImmutableList; import static com.google.common.collect.Iterables.getOnlyElement; +import static com.google.common.collect.Streams.stream; import static io.trino.spi.expression.Constant.TRUE; import static io.trino.spi.expression.StandardFunctions.AND_FUNCTION_NAME; import static io.trino.spi.type.BooleanType.BOOLEAN; +import static java.util.Objects.requireNonNull; public final class ConnectorExpressions { private ConnectorExpressions() {} + public static List extractVariables(ConnectorExpression expression) + { + return preOrder(expression) + .filter(Variable.class::isInstance) + .map(Variable.class::cast) + .collect(toImmutableList()); + } + public static List extractConjuncts(ConnectorExpression expression) { ImmutableList.Builder resultBuilder = ImmutableList.builder(); @@ -64,4 +79,11 @@ public static ConnectorExpression and(List expressions) } return getOnlyElement(expressions); } + + private static Stream preOrder(ConnectorExpression expression) + { + return stream( + Traverser.forTree((SuccessorsFunction) ConnectorExpression::getChildren) + .depthFirstPreOrder(requireNonNull(expression, "expression is null"))); + } } From 18792425a4b3f917d8f36be0da79a2002f5fd100 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 22 Dec 2023 00:02:37 +0100 Subject: [PATCH 313/350] Pull RewriteComparison.ComparisonOperator as top level class --- .../jdbc/expression/ComparisonOperator.java | 70 +++++++++++++++++++ .../jdbc/expression/RewriteComparison.java | 52 -------------- .../expression/TestRewriteComparison.java | 2 +- .../io/trino/plugin/ignite/IgniteClient.java | 3 +- .../trino/plugin/phoenix5/PhoenixClient.java | 3 +- .../plugin/postgresql/PostgreSqlClient.java | 3 +- .../plugin/sqlserver/SqlServerClient.java | 3 +- 7 files changed, 79 insertions(+), 57 deletions(-) create mode 100644 plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/expression/ComparisonOperator.java diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/expression/ComparisonOperator.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/expression/ComparisonOperator.java new file mode 100644 index 0000000000000..f9fa748442890 --- /dev/null +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/expression/ComparisonOperator.java @@ -0,0 +1,70 @@ +/* + * 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.jdbc.expression; + +import io.trino.spi.expression.FunctionName; + +import java.util.Map; +import java.util.stream.Stream; + +import static com.google.common.base.Verify.verifyNotNull; +import static com.google.common.collect.ImmutableMap.toImmutableMap; +import static io.trino.spi.expression.StandardFunctions.EQUAL_OPERATOR_FUNCTION_NAME; +import static io.trino.spi.expression.StandardFunctions.GREATER_THAN_OPERATOR_FUNCTION_NAME; +import static io.trino.spi.expression.StandardFunctions.GREATER_THAN_OR_EQUAL_OPERATOR_FUNCTION_NAME; +import static io.trino.spi.expression.StandardFunctions.IS_DISTINCT_FROM_OPERATOR_FUNCTION_NAME; +import static io.trino.spi.expression.StandardFunctions.LESS_THAN_OPERATOR_FUNCTION_NAME; +import static io.trino.spi.expression.StandardFunctions.LESS_THAN_OR_EQUAL_OPERATOR_FUNCTION_NAME; +import static io.trino.spi.expression.StandardFunctions.NOT_EQUAL_OPERATOR_FUNCTION_NAME; +import static java.util.Objects.requireNonNull; +import static java.util.function.Function.identity; + +public enum ComparisonOperator +{ + EQUAL(EQUAL_OPERATOR_FUNCTION_NAME, "="), + NOT_EQUAL(NOT_EQUAL_OPERATOR_FUNCTION_NAME, "<>"), + LESS_THAN(LESS_THAN_OPERATOR_FUNCTION_NAME, "<"), + LESS_THAN_OR_EQUAL(LESS_THAN_OR_EQUAL_OPERATOR_FUNCTION_NAME, "<="), + GREATER_THAN(GREATER_THAN_OPERATOR_FUNCTION_NAME, ">"), + GREATER_THAN_OR_EQUAL(GREATER_THAN_OR_EQUAL_OPERATOR_FUNCTION_NAME, ">="), + IS_DISTINCT_FROM(IS_DISTINCT_FROM_OPERATOR_FUNCTION_NAME, "IS DISTINCT FROM"), + /**/; + + private final FunctionName functionName; + private final String operator; + + private static final Map OPERATOR_BY_FUNCTION_NAME = Stream.of(values()) + .collect(toImmutableMap(ComparisonOperator::getFunctionName, identity())); + + ComparisonOperator(FunctionName functionName, String operator) + { + this.functionName = requireNonNull(functionName, "functionName is null"); + this.operator = requireNonNull(operator, "operator is null"); + } + + public FunctionName getFunctionName() + { + return functionName; + } + + public String getOperator() + { + return operator; + } + + public static ComparisonOperator forFunctionName(FunctionName functionName) + { + return verifyNotNull(OPERATOR_BY_FUNCTION_NAME.get(functionName), "Function name not recognized: %s", functionName); + } +} diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/expression/RewriteComparison.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/expression/RewriteComparison.java index 10db5e84ab6c3..2231ca86e33b6 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/expression/RewriteComparison.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/expression/RewriteComparison.java @@ -23,14 +23,10 @@ import io.trino.spi.expression.ConnectorExpression; import io.trino.spi.expression.FunctionName; -import java.util.Map; import java.util.Optional; import java.util.Set; -import java.util.stream.Stream; import static com.google.common.base.Verify.verify; -import static com.google.common.base.Verify.verifyNotNull; -import static com.google.common.collect.ImmutableMap.toImmutableMap; import static com.google.common.collect.ImmutableSet.toImmutableSet; import static io.trino.matching.Capture.newCapture; import static io.trino.plugin.base.expression.ConnectorExpressionPatterns.argument; @@ -39,17 +35,8 @@ import static io.trino.plugin.base.expression.ConnectorExpressionPatterns.expression; import static io.trino.plugin.base.expression.ConnectorExpressionPatterns.functionName; import static io.trino.plugin.base.expression.ConnectorExpressionPatterns.type; -import static io.trino.spi.expression.StandardFunctions.EQUAL_OPERATOR_FUNCTION_NAME; -import static io.trino.spi.expression.StandardFunctions.GREATER_THAN_OPERATOR_FUNCTION_NAME; -import static io.trino.spi.expression.StandardFunctions.GREATER_THAN_OR_EQUAL_OPERATOR_FUNCTION_NAME; -import static io.trino.spi.expression.StandardFunctions.IS_DISTINCT_FROM_OPERATOR_FUNCTION_NAME; -import static io.trino.spi.expression.StandardFunctions.LESS_THAN_OPERATOR_FUNCTION_NAME; -import static io.trino.spi.expression.StandardFunctions.LESS_THAN_OR_EQUAL_OPERATOR_FUNCTION_NAME; -import static io.trino.spi.expression.StandardFunctions.NOT_EQUAL_OPERATOR_FUNCTION_NAME; import static io.trino.spi.type.BooleanType.BOOLEAN; import static java.lang.String.format; -import static java.util.Objects.requireNonNull; -import static java.util.function.Function.identity; public class RewriteComparison implements ConnectorExpressionRule @@ -57,45 +44,6 @@ public class RewriteComparison private static final Capture LEFT = newCapture(); private static final Capture RIGHT = newCapture(); - public enum ComparisonOperator - { - EQUAL(EQUAL_OPERATOR_FUNCTION_NAME, "="), - NOT_EQUAL(NOT_EQUAL_OPERATOR_FUNCTION_NAME, "<>"), - LESS_THAN(LESS_THAN_OPERATOR_FUNCTION_NAME, "<"), - LESS_THAN_OR_EQUAL(LESS_THAN_OR_EQUAL_OPERATOR_FUNCTION_NAME, "<="), - GREATER_THAN(GREATER_THAN_OPERATOR_FUNCTION_NAME, ">"), - GREATER_THAN_OR_EQUAL(GREATER_THAN_OR_EQUAL_OPERATOR_FUNCTION_NAME, ">="), - IS_DISTINCT_FROM(IS_DISTINCT_FROM_OPERATOR_FUNCTION_NAME, "IS DISTINCT FROM"), - /**/; - - private final FunctionName functionName; - private final String operator; - - private static final Map OPERATOR_BY_FUNCTION_NAME = Stream.of(values()) - .collect(toImmutableMap(ComparisonOperator::getFunctionName, identity())); - - ComparisonOperator(FunctionName functionName, String operator) - { - this.functionName = requireNonNull(functionName, "functionName is null"); - this.operator = requireNonNull(operator, "operator is null"); - } - - private FunctionName getFunctionName() - { - return functionName; - } - - private String getOperator() - { - return operator; - } - - private static ComparisonOperator forFunctionName(FunctionName functionName) - { - return verifyNotNull(OPERATOR_BY_FUNCTION_NAME.get(functionName), "Function name not recognized: %s", functionName); - } - } - private final Pattern pattern; public RewriteComparison(Set enabledOperators) diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/expression/TestRewriteComparison.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/expression/TestRewriteComparison.java index 80e70a2e60073..57504cb16b75a 100644 --- a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/expression/TestRewriteComparison.java +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/expression/TestRewriteComparison.java @@ -26,7 +26,7 @@ public class TestRewriteComparison public void testOperatorEnumsInSync() { assertThat( - Stream.of(RewriteComparison.ComparisonOperator.values()) + Stream.of(ComparisonOperator.values()) .map(Enum::name)) .containsExactlyInAnyOrder( Stream.of(ComparisonExpression.Operator.values()) diff --git a/plugin/trino-ignite/src/main/java/io/trino/plugin/ignite/IgniteClient.java b/plugin/trino-ignite/src/main/java/io/trino/plugin/ignite/IgniteClient.java index 9309cd5f259cc..2b0c5317fb0a9 100644 --- a/plugin/trino-ignite/src/main/java/io/trino/plugin/ignite/IgniteClient.java +++ b/plugin/trino-ignite/src/main/java/io/trino/plugin/ignite/IgniteClient.java @@ -45,6 +45,7 @@ import io.trino.plugin.jdbc.aggregation.ImplementCountDistinct; import io.trino.plugin.jdbc.aggregation.ImplementMinMax; import io.trino.plugin.jdbc.aggregation.ImplementSum; +import io.trino.plugin.jdbc.expression.ComparisonOperator; import io.trino.plugin.jdbc.expression.JdbcConnectorExpressionRewriterBuilder; import io.trino.plugin.jdbc.expression.ParameterizedExpression; import io.trino.plugin.jdbc.expression.RewriteComparison; @@ -163,7 +164,7 @@ public IgniteClient( JdbcTypeHandle bigintTypeHandle = new JdbcTypeHandle(Types.BIGINT, Optional.of("bigint"), Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty()); this.connectorExpressionRewriter = JdbcConnectorExpressionRewriterBuilder.newBuilder() - .add(new RewriteComparison(ImmutableSet.of(RewriteComparison.ComparisonOperator.EQUAL, RewriteComparison.ComparisonOperator.NOT_EQUAL))) + .add(new RewriteComparison(ImmutableSet.of(ComparisonOperator.EQUAL, ComparisonOperator.NOT_EQUAL))) .addStandardRules(this::quoted) .map("$like(value: varchar, pattern: varchar): boolean").to("value LIKE pattern") .map("$not($is_null(value))").to("value IS NOT NULL") diff --git a/plugin/trino-phoenix5/src/main/java/io/trino/plugin/phoenix5/PhoenixClient.java b/plugin/trino-phoenix5/src/main/java/io/trino/plugin/phoenix5/PhoenixClient.java index 9a381e6b1043e..e087630842c78 100644 --- a/plugin/trino-phoenix5/src/main/java/io/trino/plugin/phoenix5/PhoenixClient.java +++ b/plugin/trino-phoenix5/src/main/java/io/trino/plugin/phoenix5/PhoenixClient.java @@ -39,6 +39,7 @@ import io.trino.plugin.jdbc.RemoteTableName; import io.trino.plugin.jdbc.WriteFunction; import io.trino.plugin.jdbc.WriteMapping; +import io.trino.plugin.jdbc.expression.ComparisonOperator; import io.trino.plugin.jdbc.expression.JdbcConnectorExpressionRewriterBuilder; import io.trino.plugin.jdbc.expression.ParameterizedExpression; import io.trino.plugin.jdbc.expression.RewriteComparison; @@ -246,7 +247,7 @@ public PhoenixClient(PhoenixConfig config, ConnectionFactory connectionFactory, getConnectionProperties(config).forEach((k, v) -> configuration.set((String) k, (String) v)); this.connectorExpressionRewriter = JdbcConnectorExpressionRewriterBuilder.newBuilder() .addStandardRules(this::quoted) - .add(new RewriteComparison(ImmutableSet.of(RewriteComparison.ComparisonOperator.EQUAL, RewriteComparison.ComparisonOperator.NOT_EQUAL))) + .add(new RewriteComparison(ImmutableSet.of(ComparisonOperator.EQUAL, ComparisonOperator.NOT_EQUAL))) .withTypeClass("integer_type", ImmutableSet.of("tinyint", "smallint", "integer", "bigint")) .map("$add(left: integer_type, right: integer_type)").to("left + right") .map("$subtract(left: integer_type, right: integer_type)").to("left - right") diff --git a/plugin/trino-postgresql/src/main/java/io/trino/plugin/postgresql/PostgreSqlClient.java b/plugin/trino-postgresql/src/main/java/io/trino/plugin/postgresql/PostgreSqlClient.java index affd261a925bf..3b4a43827c835 100644 --- a/plugin/trino-postgresql/src/main/java/io/trino/plugin/postgresql/PostgreSqlClient.java +++ b/plugin/trino-postgresql/src/main/java/io/trino/plugin/postgresql/PostgreSqlClient.java @@ -66,6 +66,7 @@ import io.trino.plugin.jdbc.aggregation.ImplementSum; import io.trino.plugin.jdbc.aggregation.ImplementVariancePop; import io.trino.plugin.jdbc.aggregation.ImplementVarianceSamp; +import io.trino.plugin.jdbc.expression.ComparisonOperator; import io.trino.plugin.jdbc.expression.JdbcConnectorExpressionRewriterBuilder; import io.trino.plugin.jdbc.expression.ParameterizedExpression; import io.trino.plugin.jdbc.expression.RewriteComparison; @@ -306,7 +307,7 @@ public PostgreSqlClient( this.connectorExpressionRewriter = JdbcConnectorExpressionRewriterBuilder.newBuilder() .addStandardRules(this::quoted) // TODO allow all comparison operators for numeric types - .add(new RewriteComparison(ImmutableSet.of(RewriteComparison.ComparisonOperator.EQUAL, RewriteComparison.ComparisonOperator.NOT_EQUAL))) + .add(new RewriteComparison(ImmutableSet.of(ComparisonOperator.EQUAL, ComparisonOperator.NOT_EQUAL))) .add(new RewriteIn()) .withTypeClass("integer_type", ImmutableSet.of("tinyint", "smallint", "integer", "bigint")) .map("$add(left: integer_type, right: integer_type)").to("left + right") diff --git a/plugin/trino-sqlserver/src/main/java/io/trino/plugin/sqlserver/SqlServerClient.java b/plugin/trino-sqlserver/src/main/java/io/trino/plugin/sqlserver/SqlServerClient.java index c1cdd10770273..fcb974b939889 100644 --- a/plugin/trino-sqlserver/src/main/java/io/trino/plugin/sqlserver/SqlServerClient.java +++ b/plugin/trino-sqlserver/src/main/java/io/trino/plugin/sqlserver/SqlServerClient.java @@ -60,6 +60,7 @@ import io.trino.plugin.jdbc.aggregation.ImplementAvgFloatingPoint; import io.trino.plugin.jdbc.aggregation.ImplementMinMax; import io.trino.plugin.jdbc.aggregation.ImplementSum; +import io.trino.plugin.jdbc.expression.ComparisonOperator; import io.trino.plugin.jdbc.expression.JdbcConnectorExpressionRewriterBuilder; import io.trino.plugin.jdbc.expression.ParameterizedExpression; import io.trino.plugin.jdbc.expression.RewriteComparison; @@ -300,7 +301,7 @@ public SqlServerClient( this.connectorExpressionRewriter = JdbcConnectorExpressionRewriterBuilder.newBuilder() .addStandardRules(this::quoted) - .add(new RewriteComparison(ImmutableSet.of(RewriteComparison.ComparisonOperator.EQUAL, RewriteComparison.ComparisonOperator.NOT_EQUAL))) + .add(new RewriteComparison(ImmutableSet.of(ComparisonOperator.EQUAL, ComparisonOperator.NOT_EQUAL))) .add(new RewriteIn()) .withTypeClass("integer_type", ImmutableSet.of("tinyint", "smallint", "integer", "bigint")) .map("$add(left: integer_type, right: integer_type)").to("left + right") From 91a29a7d8d2ddb1d1e7ba9bb529160d06f4542b5 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 1 Dec 2023 15:12:14 +0100 Subject: [PATCH 314/350] Implement complex join pushdown in JDBC connectors Implement non-deprecated `ConnectorMetadata.applyJoin` overload in `DefaultJdbcMetadata`. Thew old implementation is retained as a safety valve. The new implementation is not limited to the `List` model, so allows pushdown of joins involving more complex expressions, such as arithmetics. The `BaseJdbcClient.implementJoin` and `QueryBuilder.prepareJoinQuery` methods logically changed, but the old implementation is left as the fallback. These methods were extension points, so the old implementations are renamed to ensure implementors are updated. For example, if an implementation was overriding `BaseJdbcClient.implementJoin` it most likely wants to override the new `implementJoin` method as well, and this is reminded about by rename of the old method. --- .../io/trino/sql/query/QueryAssertions.java | 4 + .../base/expression/ConnectorExpressions.java | 4 + .../io/trino/plugin/jdbc/BaseJdbcClient.java | 18 ++-- .../trino/plugin/jdbc/CachingJdbcClient.java | 8 +- .../plugin/jdbc/DefaultJdbcMetadata.java | 30 +++--- .../plugin/jdbc/DefaultQueryBuilder.java | 36 +++---- .../plugin/jdbc/ForwardingJdbcClient.java | 8 +- .../java/io/trino/plugin/jdbc/JdbcClient.java | 6 +- .../io/trino/plugin/jdbc/QueryBuilder.java | 6 +- .../RewriteCaseSensitiveComparison.java | 94 +++++++++++++++++++ .../jdbc/jmx/StatisticsAwareJdbcClient.java | 9 +- .../plugin/jdbc/BaseJdbcConnectorTest.java | 20 +++- .../jdbc/TestDefaultJdbcQueryBuilder.java | 19 ++-- .../io/trino/plugin/ignite/IgniteClient.java | 15 ++- .../trino/plugin/mariadb/MariaDbClient.java | 27 +++++- .../io/trino/plugin/mysql/MySqlClient.java | 17 +++- plugin/trino-oracle/pom.xml | 5 + .../io/trino/plugin/oracle/OracleClient.java | 15 +++ .../oracle/RewriteStringComparison.java | 93 ++++++++++++++++++ .../trino/plugin/phoenix5/PhoenixClient.java | 17 ++++ .../plugin/postgresql/PostgreSqlClient.java | 35 +++++-- .../postgresql/TestPostgreSqlClient.java | 7 +- .../trino/plugin/redshift/RedshiftClient.java | 8 +- .../plugin/singlestore/SingleStoreClient.java | 32 ++++++- .../plugin/sqlserver/SqlServerClient.java | 19 ++-- 25 files changed, 441 insertions(+), 111 deletions(-) create mode 100644 plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/expression/RewriteCaseSensitiveComparison.java create mode 100644 plugin/trino-oracle/src/main/java/io/trino/plugin/oracle/RewriteStringComparison.java diff --git a/core/trino-main/src/test/java/io/trino/sql/query/QueryAssertions.java b/core/trino-main/src/test/java/io/trino/sql/query/QueryAssertions.java index 75d5579809431..72705779f319f 100644 --- a/core/trino-main/src/test/java/io/trino/sql/query/QueryAssertions.java +++ b/core/trino-main/src/test/java/io/trino/sql/query/QueryAssertions.java @@ -571,7 +571,10 @@ public QueryAssert isNotFullyPushedDown(PlanMatchPattern retainedSubplan) /** * Verifies join query is not fully pushed down by containing JOIN node. + * + * @deprecated because the method is not tested in BaseQueryAssertionsTest yet */ + @Deprecated @CanIgnoreReturnValue public QueryAssert joinIsNotFullyPushedDown() { @@ -580,6 +583,7 @@ public QueryAssert joinIsNotFullyPushedDown() .whereIsInstanceOfAny(JoinNode.class) .findFirst() .isEmpty()) { + // TODO show then plan when assertions fails (like hasPlan()) and add negative test coverage in BaseQueryAssertionsTest throw new IllegalStateException("Join node should be present in explain plan, when pushdown is not applied"); } }); diff --git a/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/expression/ConnectorExpressions.java b/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/expression/ConnectorExpressions.java index 3971a2995cc25..71d587247f6ff 100644 --- a/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/expression/ConnectorExpressions.java +++ b/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/expression/ConnectorExpressions.java @@ -53,6 +53,10 @@ public static List extractConjuncts(ConnectorExpression exp private static void extractConjuncts(ConnectorExpression expression, ImmutableList.Builder resultBuilder) { + if (expression.equals(TRUE)) { + // Skip useless conjuncts. + return; + } if (expression instanceof Call call) { if (AND_FUNCTION_NAME.equals(call.getFunctionName())) { for (ConnectorExpression argument : call.getArguments()) { diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/BaseJdbcClient.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/BaseJdbcClient.java index a8150d6245634..989317dae45d5 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/BaseJdbcClient.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/BaseJdbcClient.java @@ -527,18 +527,12 @@ public Optional implementJoin( ConnectorSession session, JoinType joinType, PreparedQuery leftSource, + Map leftProjections, PreparedQuery rightSource, - List joinConditions, - Map rightAssignments, - Map leftAssignments, + Map rightProjections, + List joinConditions, JoinStatistics statistics) { - for (JdbcJoinCondition joinCondition : joinConditions) { - if (!isSupportedJoinCondition(session, joinCondition)) { - return Optional.empty(); - } - } - try (Connection connection = this.connectionFactory.openConnection(session)) { return Optional.of(queryBuilder.prepareJoinQuery( this, @@ -546,10 +540,10 @@ public Optional implementJoin( connection, joinType, leftSource, + leftProjections, rightSource, - joinConditions, - leftAssignments, - rightAssignments)); + rightProjections, + joinConditions)); } catch (SQLException e) { throw new TrinoException(JDBC_ERROR, e); diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/CachingJdbcClient.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/CachingJdbcClient.java index 032d9895ca1bf..f473c94d96b94 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/CachingJdbcClient.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/CachingJdbcClient.java @@ -284,13 +284,13 @@ public Optional implementJoin( ConnectorSession session, JoinType joinType, PreparedQuery leftSource, + Map leftProjections, PreparedQuery rightSource, - List joinConditions, - Map rightAssignments, - Map leftAssignments, + Map rightProjections, + List joinConditions, JoinStatistics statistics) { - return delegate.implementJoin(session, joinType, leftSource, rightSource, joinConditions, rightAssignments, leftAssignments, statistics); + return delegate.implementJoin(session, joinType, leftSource, leftProjections, rightSource, rightProjections, joinConditions, statistics); } @Override diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/DefaultJdbcMetadata.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/DefaultJdbcMetadata.java index cee54f86096c2..607d813b49ef3 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/DefaultJdbcMetadata.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/DefaultJdbcMetadata.java @@ -40,7 +40,6 @@ import io.trino.spi.connector.Constraint; import io.trino.spi.connector.ConstraintApplicationResult; import io.trino.spi.connector.JoinApplicationResult; -import io.trino.spi.connector.JoinCondition; import io.trino.spi.connector.JoinStatistics; import io.trino.spi.connector.JoinType; import io.trino.spi.connector.LimitApplicationResult; @@ -73,6 +72,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Map.Entry; import java.util.Optional; import java.util.OptionalInt; import java.util.OptionalLong; @@ -442,7 +442,7 @@ public Optional> applyJoin( JoinType joinType, ConnectorTableHandle left, ConnectorTableHandle right, - List joinConditions, + ConnectorExpression joinCondition, Map leftAssignments, Map rightAssignments, JoinStatistics statistics) @@ -478,26 +478,32 @@ public Optional> applyJoin( } Map newRightColumns = newRightColumnsBuilder.buildOrThrow(); - ImmutableList.Builder jdbcJoinConditions = ImmutableList.builder(); - for (JoinCondition joinCondition : joinConditions) { - Optional leftColumn = getVariableColumnHandle(leftAssignments, joinCondition.getLeftExpression()); - Optional rightColumn = getVariableColumnHandle(rightAssignments, joinCondition.getRightExpression()); - if (leftColumn.isEmpty() || rightColumn.isEmpty()) { + Map assignments = ImmutableMap.builder() + .putAll(leftAssignments.entrySet().stream() + .collect(toImmutableMap(Entry::getKey, entry -> newLeftColumns.get((JdbcColumnHandle) entry.getValue())))) + .putAll(rightAssignments.entrySet().stream() + .collect(toImmutableMap(Entry::getKey, entry -> newRightColumns.get((JdbcColumnHandle) entry.getValue())))) + .buildOrThrow(); + + ImmutableList.Builder joinConditions = ImmutableList.builder(); + for (ConnectorExpression conjunct : extractConjuncts(joinCondition)) { + Optional converted = jdbcClient.convertPredicate(session, conjunct, assignments); + if (converted.isEmpty()) { return Optional.empty(); } - jdbcJoinConditions.add(new JdbcJoinCondition(leftColumn.get(), joinCondition.getOperator(), rightColumn.get())); + joinConditions.add(converted.get()); } Optional joinQuery = jdbcClient.implementJoin( session, joinType, asPreparedQuery(leftHandle), + newLeftColumns.entrySet().stream() + .collect(toImmutableMap(Entry::getKey, entry -> entry.getValue().getColumnName())), asPreparedQuery(rightHandle), - jdbcJoinConditions.build(), newRightColumns.entrySet().stream() - .collect(toImmutableMap(Map.Entry::getKey, entry -> entry.getValue().getColumnName())), - newLeftColumns.entrySet().stream() - .collect(toImmutableMap(Map.Entry::getKey, entry -> entry.getValue().getColumnName())), + .collect(toImmutableMap(Entry::getKey, entry -> entry.getValue().getColumnName())), + joinConditions.build(), statistics); if (joinQuery.isEmpty()) { diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/DefaultQueryBuilder.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/DefaultQueryBuilder.java index dfd3b06743295..7d3a4f5f05243 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/DefaultQueryBuilder.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/DefaultQueryBuilder.java @@ -117,35 +117,32 @@ public PreparedQuery prepareJoinQuery( Connection connection, JoinType joinType, PreparedQuery leftSource, + Map leftProjections, PreparedQuery rightSource, - List joinConditions, - Map leftAssignments, - Map rightAssignments) + Map rightProjections, + List joinConditions) { - // Verify assignments are present. This is safe assumption as join conditions are not pruned, and simplifies the code here. - verify(!leftAssignments.isEmpty(), "leftAssignments is empty"); - verify(!rightAssignments.isEmpty(), "rightAssignments is empty"); // Joins wih no conditions are not pushed down, so it is a same assumption and simplifies the code here verify(!joinConditions.isEmpty(), "joinConditions is empty"); - String leftRelationAlias = "l"; - String rightRelationAlias = "r"; - String query = format( - "SELECT %s, %s FROM (%s) %s %s (%s) %s ON %s", - formatAssignments(client, leftRelationAlias, leftAssignments), - formatAssignments(client, rightRelationAlias, rightAssignments), + // The subquery aliases (`l` and `r`) are needed by some databases, but are not needed for expressions + // The joinConditions and output columns are aliased to use unique names. + "SELECT * FROM (SELECT %s FROM (%s) l) l %s (SELECT %s FROM (%s) r) r ON %s", + formatProjections(client, leftProjections), leftSource.getQuery(), - leftRelationAlias, formatJoinType(joinType), + formatProjections(client, rightProjections), rightSource.getQuery(), - rightRelationAlias, joinConditions.stream() - .map(condition -> formatJoinCondition(client, leftRelationAlias, rightRelationAlias, condition)) - .collect(joining(" AND "))); + .map(ParameterizedExpression::expression) + .collect(joining(") AND (", "(", ")"))); List parameters = ImmutableList.builder() .addAll(leftSource.getParameters()) .addAll(rightSource.getParameters()) + .addAll(joinConditions.stream() + .flatMap(expression -> expression.parameters().stream()) + .iterator()) .build(); return new PreparedQuery(query, parameters); } @@ -296,6 +293,13 @@ protected String buildJoinColumn(JdbcClient client, JdbcColumnHandle columnHandl return client.quoted(columnHandle.getColumnName()); } + protected String formatProjections(JdbcClient client, Map projections) + { + return projections.entrySet().stream() + .map(entry -> format("%s AS %s", client.quoted(entry.getKey().getColumnName()), client.quoted(entry.getValue()))) + .collect(joining(", ")); + } + protected String formatAssignments(JdbcClient client, String relationAlias, Map assignments) { return assignments.entrySet().stream() diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/ForwardingJdbcClient.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/ForwardingJdbcClient.java index 2f784634a44b0..f2ac9981f9ef7 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/ForwardingJdbcClient.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/ForwardingJdbcClient.java @@ -211,13 +211,13 @@ public Optional implementJoin( ConnectorSession session, JoinType joinType, PreparedQuery leftSource, + Map leftProjections, PreparedQuery rightSource, - List joinConditions, - Map rightAssignments, - Map leftAssignments, + Map rightProjections, + List joinConditions, JoinStatistics statistics) { - return delegate().implementJoin(session, joinType, leftSource, rightSource, joinConditions, rightAssignments, leftAssignments, statistics); + return delegate().implementJoin(session, joinType, leftSource, leftProjections, rightSource, rightProjections, joinConditions, statistics); } @Override diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcClient.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcClient.java index 38e6f7e801141..a52f100c78b14 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcClient.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcClient.java @@ -127,10 +127,10 @@ Optional implementJoin( ConnectorSession session, JoinType joinType, PreparedQuery leftSource, + Map leftProjections, PreparedQuery rightSource, - List joinConditions, - Map rightAssignments, - Map leftAssignments, + Map rightProjections, + List joinConditions, JoinStatistics statistics); boolean supportsTopN(ConnectorSession session, JdbcTableHandle handle, List sortOrder); diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/QueryBuilder.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/QueryBuilder.java index 52cb58e80cb59..a6dc6f53d9852 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/QueryBuilder.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/QueryBuilder.java @@ -47,10 +47,10 @@ PreparedQuery prepareJoinQuery( Connection connection, JoinType joinType, PreparedQuery leftSource, + Map leftProjections, PreparedQuery rightSource, - List joinConditions, - Map leftAssignments, - Map rightAssignments); + Map rightProjections, + List joinConditions); PreparedQuery prepareDeleteQuery( JdbcClient client, diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/expression/RewriteCaseSensitiveComparison.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/expression/RewriteCaseSensitiveComparison.java new file mode 100644 index 0000000000000..2ff0c1c8ad701 --- /dev/null +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/expression/RewriteCaseSensitiveComparison.java @@ -0,0 +1,94 @@ +/* + * 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.jdbc.expression; + +import com.google.common.collect.ImmutableList; +import io.trino.matching.Capture; +import io.trino.matching.Captures; +import io.trino.matching.Pattern; +import io.trino.plugin.base.expression.ConnectorExpressionRule; +import io.trino.plugin.jdbc.JdbcColumnHandle; +import io.trino.plugin.jdbc.QueryParameter; +import io.trino.spi.expression.Call; +import io.trino.spi.expression.FunctionName; +import io.trino.spi.expression.Variable; +import io.trino.spi.type.VarcharType; + +import java.util.Optional; +import java.util.Set; + +import static com.google.common.collect.ImmutableSet.toImmutableSet; +import static io.trino.matching.Capture.newCapture; +import static io.trino.plugin.base.expression.ConnectorExpressionPatterns.argument; +import static io.trino.plugin.base.expression.ConnectorExpressionPatterns.argumentCount; +import static io.trino.plugin.base.expression.ConnectorExpressionPatterns.call; +import static io.trino.plugin.base.expression.ConnectorExpressionPatterns.functionName; +import static io.trino.plugin.base.expression.ConnectorExpressionPatterns.type; +import static io.trino.plugin.base.expression.ConnectorExpressionPatterns.variable; +import static io.trino.plugin.jdbc.CaseSensitivity.CASE_SENSITIVE; +import static io.trino.spi.type.BooleanType.BOOLEAN; + +public class RewriteCaseSensitiveComparison + implements ConnectorExpressionRule +{ + private static final Capture LEFT = newCapture(); + private static final Capture RIGHT = newCapture(); + + private final Pattern pattern; + + public RewriteCaseSensitiveComparison(Set enabledOperators) + { + Set functionNames = enabledOperators.stream() + .map(ComparisonOperator::getFunctionName) + .collect(toImmutableSet()); + + pattern = call() + .with(type().equalTo(BOOLEAN)) + .with(functionName().matching(functionNames::contains)) + .with(argumentCount().equalTo(2)) + .with(argument(0).matching(variable().with(type().matching(VarcharType.class::isInstance)).capturedAs(LEFT))) + .with(argument(1).matching(variable().with(type().matching(VarcharType.class::isInstance)).capturedAs(RIGHT))); + } + + @Override + public Pattern getPattern() + { + return pattern; + } + + @Override + public Optional rewrite(Call expression, Captures captures, RewriteContext context) + { + ComparisonOperator comparison = ComparisonOperator.forFunctionName(expression.getFunctionName()); + Variable firstArgument = captures.get(LEFT); + Variable secondArgument = captures.get(RIGHT); + + if (!isCaseSensitive(firstArgument, context) || !isCaseSensitive(secondArgument, context)) { + return Optional.empty(); + } + return context.defaultRewrite(firstArgument).flatMap(first -> + context.defaultRewrite(secondArgument).map(second -> + new ParameterizedExpression( + "(%s) %s (%s)".formatted(first.expression(), comparison.getOperator(), second.expression()), + ImmutableList.builder() + .addAll(first.parameters()) + .addAll(second.parameters()) + .build()))); + } + + private static boolean isCaseSensitive(Variable variable, RewriteContext context) + { + return ((JdbcColumnHandle) context.getAssignment(variable.getName())).getJdbcTypeHandle().getCaseSensitivity().equals(Optional.of(CASE_SENSITIVE)); + } +} diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/jmx/StatisticsAwareJdbcClient.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/jmx/StatisticsAwareJdbcClient.java index d07bcfa5178c1..a190874657506 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/jmx/StatisticsAwareJdbcClient.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/jmx/StatisticsAwareJdbcClient.java @@ -17,7 +17,6 @@ import io.trino.plugin.jdbc.JdbcClient; import io.trino.plugin.jdbc.JdbcColumnHandle; import io.trino.plugin.jdbc.JdbcExpression; -import io.trino.plugin.jdbc.JdbcJoinCondition; import io.trino.plugin.jdbc.JdbcOutputTableHandle; import io.trino.plugin.jdbc.JdbcProcedureHandle; import io.trino.plugin.jdbc.JdbcProcedureHandle.ProcedureQuery; @@ -231,13 +230,13 @@ public CallableStatement buildProcedure(ConnectorSession session, Connection con public Optional implementJoin(ConnectorSession session, JoinType joinType, PreparedQuery leftSource, + Map leftProjections, PreparedQuery rightSource, - List joinConditions, - Map rightAssignments, - Map leftAssignments, + Map rightProjections, + List joinConditions, JoinStatistics statistics) { - return stats.getImplementJoin().wrap(() -> delegate().implementJoin(session, joinType, leftSource, rightSource, joinConditions, rightAssignments, leftAssignments, statistics)); + return stats.getImplementJoin().wrap(() -> delegate().implementJoin(session, joinType, leftSource, leftProjections, rightSource, rightProjections, joinConditions, statistics)); } @Override diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java index 40eb2f8da22e0..1cf08c54b3fd7 100644 --- a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java @@ -1325,6 +1325,20 @@ public void testJoinPushdown() } } + @Test + public void testComplexJoinPushdown() + { + String catalog = getSession().getCatalog().orElseThrow(); + Session session = joinPushdownEnabled(getSession()); + String query = "SELECT n.name, o.orderstatus FROM nation n JOIN orders o ON n.regionkey = o.orderkey AND n.nationkey + o.custkey - 3 = 0"; + + // The join can be pushed down + assertJoinConditionallyPushedDown( + session, + query, + hasBehavior(SUPPORTS_JOIN_PUSHDOWN) && hasBehavior(SUPPORTS_PREDICATE_ARITHMETIC_EXPRESSION_PUSHDOWN)); + } + @Test public void testExplainAnalyzePhysicalReadWallTime() { @@ -1388,8 +1402,7 @@ protected void assertConditionallyOrderedPushedDown( protected boolean expectJoinPushdown(String operator) { if ("IS NOT DISTINCT FROM".equals(operator)) { - // TODO (https://github.com/trinodb/trino/issues/6967) support join pushdown for IS NOT DISTINCT FROM - return false; + return hasBehavior(SUPPORTS_JOIN_PUSHDOWN_WITH_DISTINCT_FROM); } return switch (toJoinConditionOperator(operator)) { case EQUAL, NOT_EQUAL, LESS_THAN, LESS_THAN_OR_EQUAL, GREATER_THAN, GREATER_THAN_OR_EQUAL -> true; @@ -1406,8 +1419,7 @@ protected boolean expectJoinPushdownOnInequalityOperator(JoinOperator joinOperat private boolean expectVarcharJoinPushdown(String operator) { if ("IS NOT DISTINCT FROM".equals(operator)) { - // TODO (https://github.com/trinodb/trino/issues/6967) support join pushdown for IS NOT DISTINCT FROM - return false; + return hasBehavior(SUPPORTS_JOIN_PUSHDOWN_WITH_DISTINCT_FROM) && hasBehavior(SUPPORTS_JOIN_PUSHDOWN_WITH_VARCHAR_EQUALITY); } return switch (toJoinConditionOperator(operator)) { case EQUAL, NOT_EQUAL -> hasBehavior(SUPPORTS_JOIN_PUSHDOWN_WITH_VARCHAR_EQUALITY); diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestDefaultJdbcQueryBuilder.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestDefaultJdbcQueryBuilder.java index c58a9b5633d21..a407d624d5d86 100644 --- a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestDefaultJdbcQueryBuilder.java +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestDefaultJdbcQueryBuilder.java @@ -22,7 +22,6 @@ import io.trino.plugin.jdbc.logging.RemoteQueryModifier; import io.trino.spi.connector.ColumnHandle; import io.trino.spi.connector.ConnectorSession; -import io.trino.spi.connector.JoinCondition; import io.trino.spi.connector.JoinType; import io.trino.spi.connector.SchemaTableName; import io.trino.spi.predicate.Domain; @@ -517,17 +516,17 @@ public void testBuildJoinSql() connection, JoinType.INNER, new PreparedQuery("SELECT * FROM \"test_table\"", List.of()), + ImmutableMap.of(columns.get(2), "name1", columns.get(7), "lcol7"), new PreparedQuery("SELECT * FROM \"test_table\"", List.of()), - List.of(new JdbcJoinCondition(columns.get(7), JoinCondition.Operator.EQUAL, columns.get(8))), - Map.of(columns.get(2), "name1"), - Map.of(columns.get(3), "name2")); + ImmutableMap.of(columns.get(3), "name2", columns.get(8), "rcol8"), + List.of(new ParameterizedExpression("\"lcol7\" = \"rcol8\"", List.of()))); try (PreparedStatement preparedStatement = queryBuilder.prepareStatement(jdbcClient, SESSION, connection, preparedQuery, Optional.empty())) { - assertThat(preparedQuery.getQuery()).isEqualTo("" + - "SELECT l.\"col_2\" AS \"name1\", r.\"col_3\" AS \"name2\" FROM " + - "(SELECT * FROM \"test_table\") l " + - "INNER JOIN " + - "(SELECT * FROM \"test_table\") r " + - "ON l.\"col_7\" = r.\"col_8\""); + assertThat(preparedQuery.getQuery()).isEqualTo(""" + SELECT * FROM \ + (SELECT "col_2" AS "name1", "col_7" AS "lcol7" FROM (SELECT * FROM "test_table") l) l \ + INNER JOIN \ + (SELECT "col_3" AS "name2", "col_8" AS "rcol8" FROM (SELECT * FROM "test_table") r) r \ + ON ("lcol7" = "rcol8")"""); long count = 0; try (ResultSet resultSet = preparedStatement.executeQuery()) { while (resultSet.next()) { diff --git a/plugin/trino-ignite/src/main/java/io/trino/plugin/ignite/IgniteClient.java b/plugin/trino-ignite/src/main/java/io/trino/plugin/ignite/IgniteClient.java index 2b0c5317fb0a9..7af9b61915fc6 100644 --- a/plugin/trino-ignite/src/main/java/io/trino/plugin/ignite/IgniteClient.java +++ b/plugin/trino-ignite/src/main/java/io/trino/plugin/ignite/IgniteClient.java @@ -166,6 +166,13 @@ public IgniteClient( this.connectorExpressionRewriter = JdbcConnectorExpressionRewriterBuilder.newBuilder() .add(new RewriteComparison(ImmutableSet.of(ComparisonOperator.EQUAL, ComparisonOperator.NOT_EQUAL))) .addStandardRules(this::quoted) + .map("$equal(left, right)").to("left = right") + .map("$not_equal(left, right)").to("left <> right") + .map("$is_distinct_from(left, right)").to("left IS DISTINCT FROM right") + .map("$less_than(left, right)").to("left < right") + .map("$less_than_or_equal(left, right)").to("left <= right") + .map("$greater_than(left, right)").to("left > right") + .map("$greater_than_or_equal(left, right)").to("left >= right") .map("$like(value: varchar, pattern: varchar): boolean").to("value LIKE pattern") .map("$not($is_null(value))").to("value IS NOT NULL") .map("$not(value: boolean)").to("NOT value") @@ -574,10 +581,10 @@ public Optional implementJoin( ConnectorSession session, JoinType joinType, PreparedQuery leftSource, + Map leftProjections, PreparedQuery rightSource, - List joinConditions, - Map rightAssignments, - Map leftAssignments, + Map rightProjections, + List joinConditions, JoinStatistics statistics) { // Ignite does not support FULL JOIN @@ -585,7 +592,7 @@ public Optional implementJoin( return Optional.empty(); } - return super.implementJoin(session, joinType, leftSource, rightSource, joinConditions, rightAssignments, leftAssignments, statistics); + return super.implementJoin(session, joinType, leftSource, leftProjections, rightSource, rightProjections, joinConditions, statistics); } @Override diff --git a/plugin/trino-mariadb/src/main/java/io/trino/plugin/mariadb/MariaDbClient.java b/plugin/trino-mariadb/src/main/java/io/trino/plugin/mariadb/MariaDbClient.java index af3a6c0ecca88..2f8d2ff527de9 100644 --- a/plugin/trino-mariadb/src/main/java/io/trino/plugin/mariadb/MariaDbClient.java +++ b/plugin/trino-mariadb/src/main/java/io/trino/plugin/mariadb/MariaDbClient.java @@ -59,6 +59,7 @@ import io.trino.spi.connector.JoinStatistics; import io.trino.spi.connector.JoinType; import io.trino.spi.connector.SchemaTableName; +import io.trino.spi.expression.ConnectorExpression; import io.trino.spi.statistics.ColumnStatistics; import io.trino.spi.statistics.Estimate; import io.trino.spi.statistics.TableStatistics; @@ -173,6 +174,7 @@ public class MariaDbClient private static final int PARSE_ERROR = 1064; private final boolean statisticsEnabled; + private final ConnectorExpressionRewriter connectorExpressionRewriter; private final AggregateFunctionRewriter aggregateFunctionRewriter; @Inject @@ -187,8 +189,17 @@ public MariaDbClient( super("`", connectionFactory, queryBuilder, config.getJdbcTypesMappedToVarchar(), identifierMapping, queryModifier, false); JdbcTypeHandle bigintTypeHandle = new JdbcTypeHandle(Types.BIGINT, Optional.of("bigint"), Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty()); - ConnectorExpressionRewriter connectorExpressionRewriter = JdbcConnectorExpressionRewriterBuilder.newBuilder() + this.connectorExpressionRewriter = JdbcConnectorExpressionRewriterBuilder.newBuilder() .addStandardRules(this::quoted) + // No "real" on the list; pushdown on REAL is disabled also in toColumnMapping + .withTypeClass("numeric_type", ImmutableSet.of("tinyint", "smallint", "integer", "bigint", "decimal", "double")) + .map("$equal(left: numeric_type, right: numeric_type)").to("left = right") + .map("$not_equal(left: numeric_type, right: numeric_type)").to("left <> right") + // .map("$is_distinct_from(left: numeric_type, right: numeric_type)").to("left IS DISTINCT FROM right") + .map("$less_than(left: numeric_type, right: numeric_type)").to("left < right") + .map("$less_than_or_equal(left: numeric_type, right: numeric_type)").to("left <= right") + .map("$greater_than(left: numeric_type, right: numeric_type)").to("left > right") + .map("$greater_than_or_equal(left: numeric_type, right: numeric_type)").to("left >= right") .build(); this.statisticsEnabled = statisticsConfig.isEnabled(); this.aggregateFunctionRewriter = new AggregateFunctionRewriter<>( @@ -222,6 +233,12 @@ public boolean supportsAggregationPushdown(ConnectorSession session, JdbcTableHa return preventTextualTypeAggregationPushdown(groupingSets); } + @Override + public Optional convertPredicate(ConnectorSession session, ConnectorExpression expression, Map assignments) + { + return connectorExpressionRewriter.rewrite(session, expression, assignments); + } + private static Optional toTypeHandle(DecimalType decimalType) { return Optional.of(new JdbcTypeHandle(Types.NUMERIC, Optional.of("decimal"), Optional.of(decimalType.getPrecision()), Optional.of(decimalType.getScale()), Optional.empty(), Optional.empty())); @@ -622,17 +639,17 @@ public Optional implementJoin( ConnectorSession session, JoinType joinType, PreparedQuery leftSource, + Map leftProjections, PreparedQuery rightSource, - List joinConditions, - Map rightAssignments, - Map leftAssignments, + Map rightProjections, + List joinConditions, JoinStatistics statistics) { if (joinType == JoinType.FULL_OUTER) { // Not supported in MariaDB return Optional.empty(); } - return super.implementJoin(session, joinType, leftSource, rightSource, joinConditions, rightAssignments, leftAssignments, statistics); + return super.implementJoin(session, joinType, leftSource, leftProjections, rightSource, rightProjections, joinConditions, statistics); } @Override diff --git a/plugin/trino-mysql/src/main/java/io/trino/plugin/mysql/MySqlClient.java b/plugin/trino-mysql/src/main/java/io/trino/plugin/mysql/MySqlClient.java index a53870ed76ce8..cece995ce02b9 100644 --- a/plugin/trino-mysql/src/main/java/io/trino/plugin/mysql/MySqlClient.java +++ b/plugin/trino-mysql/src/main/java/io/trino/plugin/mysql/MySqlClient.java @@ -270,6 +270,15 @@ public MySqlClient( this.connectorExpressionRewriter = JdbcConnectorExpressionRewriterBuilder.newBuilder() .addStandardRules(this::quoted) + // No "real" on the list; pushdown on REAL is disabled also in toColumnMapping + .withTypeClass("numeric_type", ImmutableSet.of("tinyint", "smallint", "integer", "bigint", "decimal", "double")) + .map("$equal(left: numeric_type, right: numeric_type)").to("left = right") + .map("$not_equal(left: numeric_type, right: numeric_type)").to("left <> right") + // .map("$is_distinct_from(left: numeric_type, right: numeric_type)").to("left IS DISTINCT FROM right") + .map("$less_than(left: numeric_type, right: numeric_type)").to("left < right") + .map("$less_than_or_equal(left: numeric_type, right: numeric_type)").to("left <= right") + .map("$greater_than(left: numeric_type, right: numeric_type)").to("left > right") + .map("$greater_than_or_equal(left: numeric_type, right: numeric_type)").to("left >= right") .add(new RewriteLikeWithCaseSensitivity()) .add(new RewriteLikeEscapeWithCaseSensitivity()) .build(); @@ -1005,10 +1014,10 @@ public Optional implementJoin( ConnectorSession session, JoinType joinType, PreparedQuery leftSource, + Map leftProjections, PreparedQuery rightSource, - List joinConditions, - Map rightAssignments, - Map leftAssignments, + Map rightProjections, + List joinConditions, JoinStatistics statistics) { if (joinType == JoinType.FULL_OUTER) { @@ -1021,7 +1030,7 @@ public Optional implementJoin( leftSource, rightSource, statistics, - () -> super.implementJoin(session, joinType, leftSource, rightSource, joinConditions, rightAssignments, leftAssignments, statistics)); + () -> super.implementJoin(session, joinType, leftSource, leftProjections, rightSource, rightProjections, joinConditions, statistics)); } @Override diff --git a/plugin/trino-oracle/pom.xml b/plugin/trino-oracle/pom.xml index dc28a30560dd6..614570a100129 100644 --- a/plugin/trino-oracle/pom.xml +++ b/plugin/trino-oracle/pom.xml @@ -58,6 +58,11 @@ trino-base-jdbc + + io.trino + trino-matching + + io.trino trino-plugin-toolkit diff --git a/plugin/trino-oracle/src/main/java/io/trino/plugin/oracle/OracleClient.java b/plugin/trino-oracle/src/main/java/io/trino/plugin/oracle/OracleClient.java index 47837cff4e3ef..e621317714080 100644 --- a/plugin/trino-oracle/src/main/java/io/trino/plugin/oracle/OracleClient.java +++ b/plugin/trino-oracle/src/main/java/io/trino/plugin/oracle/OracleClient.java @@ -62,6 +62,7 @@ import io.trino.spi.connector.ConnectorSession; import io.trino.spi.connector.ConnectorTableMetadata; import io.trino.spi.connector.JoinCondition; +import io.trino.spi.expression.ConnectorExpression; import io.trino.spi.type.CharType; import io.trino.spi.type.DecimalType; import io.trino.spi.type.Decimals; @@ -231,6 +232,14 @@ public OracleClient( this.connectorExpressionRewriter = JdbcConnectorExpressionRewriterBuilder.newBuilder() .addStandardRules(this::quoted) + .withTypeClass("numeric_type", ImmutableSet.of("tinyint", "smallint", "integer", "bigint", "decimal", "real", "double")) + .map("$equal(left: numeric_type, right: numeric_type)").to("left = right") + .map("$not_equal(left: numeric_type, right: numeric_type)").to("left <> right") + .map("$less_than(left: numeric_type, right: numeric_type)").to("left < right") + .map("$less_than_or_equal(left: numeric_type, right: numeric_type)").to("left <= right") + .map("$greater_than(left: numeric_type, right: numeric_type)").to("left > right") + .map("$greater_than_or_equal(left: numeric_type, right: numeric_type)").to("left >= right") + .add(new RewriteStringComparison()) .build(); JdbcTypeHandle bigintTypeHandle = new JdbcTypeHandle(TRINO_BIGINT_TYPE, Optional.of("NUMBER"), Optional.of(0), Optional.of(0), Optional.empty(), Optional.empty()); @@ -538,6 +547,12 @@ public Optional implementAggregation(ConnectorSession session, A return aggregateFunctionRewriter.rewrite(session, aggregate, assignments); } + @Override + public Optional convertPredicate(ConnectorSession session, ConnectorExpression expression, Map assignments) + { + return connectorExpressionRewriter.rewrite(session, expression, assignments); + } + private static Optional toTypeHandle(DecimalType decimalType) { return Optional.of(new JdbcTypeHandle(OracleTypes.NUMBER, Optional.of("NUMBER"), Optional.of(decimalType.getPrecision()), Optional.of(decimalType.getScale()), Optional.empty(), Optional.empty())); diff --git a/plugin/trino-oracle/src/main/java/io/trino/plugin/oracle/RewriteStringComparison.java b/plugin/trino-oracle/src/main/java/io/trino/plugin/oracle/RewriteStringComparison.java new file mode 100644 index 0000000000000..1e3f8c73bf7d9 --- /dev/null +++ b/plugin/trino-oracle/src/main/java/io/trino/plugin/oracle/RewriteStringComparison.java @@ -0,0 +1,93 @@ +/* + * 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.oracle; + +import com.google.common.collect.ImmutableList; +import io.trino.matching.Capture; +import io.trino.matching.Captures; +import io.trino.matching.Pattern; +import io.trino.plugin.base.expression.ConnectorExpressionRule; +import io.trino.plugin.jdbc.JdbcColumnHandle; +import io.trino.plugin.jdbc.QueryParameter; +import io.trino.plugin.jdbc.expression.ComparisonOperator; +import io.trino.plugin.jdbc.expression.ParameterizedExpression; +import io.trino.spi.expression.Call; +import io.trino.spi.expression.Variable; +import io.trino.spi.type.CharType; +import io.trino.spi.type.VarcharType; +import oracle.jdbc.OracleTypes; + +import java.util.Optional; +import java.util.stream.Stream; + +import static com.google.common.collect.ImmutableSet.toImmutableSet; +import static io.trino.matching.Capture.newCapture; +import static io.trino.plugin.base.expression.ConnectorExpressionPatterns.argument; +import static io.trino.plugin.base.expression.ConnectorExpressionPatterns.argumentCount; +import static io.trino.plugin.base.expression.ConnectorExpressionPatterns.call; +import static io.trino.plugin.base.expression.ConnectorExpressionPatterns.functionName; +import static io.trino.plugin.base.expression.ConnectorExpressionPatterns.type; +import static io.trino.plugin.base.expression.ConnectorExpressionPatterns.variable; +import static io.trino.spi.type.BooleanType.BOOLEAN; + +public class RewriteStringComparison + implements ConnectorExpressionRule +{ + private static final Capture FIRST_ARGUMENT = newCapture(); + private static final Capture SECOND_ARGUMENT = newCapture(); + private static final Pattern PATTERN = call() + .with(type().equalTo(BOOLEAN)) + .with(functionName().matching(Stream.of(ComparisonOperator.values()) + .filter(comparison -> comparison != ComparisonOperator.IS_DISTINCT_FROM) + .map(ComparisonOperator::getFunctionName) + .collect(toImmutableSet()) + ::contains)) + .with(argumentCount().equalTo(2)) + .with(argument(0).matching(variable().with(type().matching(type -> type instanceof CharType || type instanceof VarcharType)).capturedAs(FIRST_ARGUMENT))) + .with(argument(1).matching(variable().with(type().matching(type -> type instanceof CharType || type instanceof VarcharType)).capturedAs(SECOND_ARGUMENT))); + + @Override + public Pattern getPattern() + { + return PATTERN; + } + + @Override + public Optional rewrite(Call expression, Captures captures, RewriteContext context) + { + ComparisonOperator comparison = ComparisonOperator.forFunctionName(expression.getFunctionName()); + Variable firstArgument = captures.get(FIRST_ARGUMENT); + Variable secondArgument = captures.get(SECOND_ARGUMENT); + + if (isClob(firstArgument, context) || isClob(secondArgument, context)) { + return Optional.empty(); + } + return context.defaultRewrite(firstArgument).flatMap(first -> + context.defaultRewrite(secondArgument).map(second -> + new ParameterizedExpression( + "(%s) %s (%s)".formatted(first.expression(), comparison.getOperator(), second.expression()), + ImmutableList.builder() + .addAll(first.parameters()) + .addAll(second.parameters()) + .build()))); + } + + private static boolean isClob(Variable variable, RewriteContext context) + { + return switch (((JdbcColumnHandle) context.getAssignment(variable.getName())).getJdbcTypeHandle().getJdbcType()) { + case OracleTypes.CLOB, OracleTypes.NCLOB -> true; + default -> false; + }; + } +} diff --git a/plugin/trino-phoenix5/src/main/java/io/trino/plugin/phoenix5/PhoenixClient.java b/plugin/trino-phoenix5/src/main/java/io/trino/plugin/phoenix5/PhoenixClient.java index e087630842c78..7bd91251c851a 100644 --- a/plugin/trino-phoenix5/src/main/java/io/trino/plugin/phoenix5/PhoenixClient.java +++ b/plugin/trino-phoenix5/src/main/java/io/trino/plugin/phoenix5/PhoenixClient.java @@ -51,6 +51,8 @@ import io.trino.spi.connector.ConnectorSession; import io.trino.spi.connector.ConnectorTableHandle; import io.trino.spi.connector.ConnectorTableMetadata; +import io.trino.spi.connector.JoinStatistics; +import io.trino.spi.connector.JoinType; import io.trino.spi.connector.SchemaNotFoundException; import io.trino.spi.connector.SchemaTableName; import io.trino.spi.expression.ConnectorExpression; @@ -264,6 +266,21 @@ public Optional convertPredicate(ConnectorSession sessi return connectorExpressionRewriter.rewrite(session, expression, assignments); } + @Override + public Optional implementJoin( + ConnectorSession session, + JoinType joinType, + PreparedQuery leftSource, + Map leftProjections, + PreparedQuery rightSource, + Map rightProjections, + List joinConditions, + JoinStatistics statistics) + { + // Joins are currently not supported + return Optional.empty(); + } + public Connection getConnection(ConnectorSession session) throws SQLException { diff --git a/plugin/trino-postgresql/src/main/java/io/trino/plugin/postgresql/PostgreSqlClient.java b/plugin/trino-postgresql/src/main/java/io/trino/plugin/postgresql/PostgreSqlClient.java index 3b4a43827c835..976f09c082859 100644 --- a/plugin/trino-postgresql/src/main/java/io/trino/plugin/postgresql/PostgreSqlClient.java +++ b/plugin/trino-postgresql/src/main/java/io/trino/plugin/postgresql/PostgreSqlClient.java @@ -66,10 +66,8 @@ import io.trino.plugin.jdbc.aggregation.ImplementSum; import io.trino.plugin.jdbc.aggregation.ImplementVariancePop; import io.trino.plugin.jdbc.aggregation.ImplementVarianceSamp; -import io.trino.plugin.jdbc.expression.ComparisonOperator; import io.trino.plugin.jdbc.expression.JdbcConnectorExpressionRewriterBuilder; import io.trino.plugin.jdbc.expression.ParameterizedExpression; -import io.trino.plugin.jdbc.expression.RewriteComparison; import io.trino.plugin.jdbc.expression.RewriteIn; import io.trino.plugin.jdbc.logging.RemoteQueryModifier; import io.trino.plugin.postgresql.PostgreSqlConfig.ArrayMapping; @@ -277,6 +275,7 @@ public class PostgreSqlClient private final List tableTypes; private final boolean statisticsEnabled; private final ConnectorExpressionRewriter connectorExpressionRewriter; + private final ConnectorExpressionRewriter connectorExpressionRewriterWithCollate; private final AggregateFunctionRewriter aggregateFunctionRewriter; @Inject @@ -304,12 +303,18 @@ public PostgreSqlClient( this.statisticsEnabled = statisticsConfig.isEnabled(); - this.connectorExpressionRewriter = JdbcConnectorExpressionRewriterBuilder.newBuilder() + JdbcConnectorExpressionRewriterBuilder connectorExpressionRewriterBuilder = JdbcConnectorExpressionRewriterBuilder.newBuilder() .addStandardRules(this::quoted) - // TODO allow all comparison operators for numeric types - .add(new RewriteComparison(ImmutableSet.of(ComparisonOperator.EQUAL, ComparisonOperator.NOT_EQUAL))) .add(new RewriteIn()) .withTypeClass("integer_type", ImmutableSet.of("tinyint", "smallint", "integer", "bigint")) + .withTypeClass("numeric_type", ImmutableSet.of("tinyint", "smallint", "integer", "bigint", "decimal", "real", "double")) + .map("$equal(left, right)").to("left = right") + .map("$not_equal(left, right)").to("left <> right") + .map("$is_distinct_from(left, right)").to("left IS DISTINCT FROM right") + .map("$less_than(left: numeric_type, right: numeric_type)").to("left < right") + .map("$less_than_or_equal(left: numeric_type, right: numeric_type)").to("left <= right") + .map("$greater_than(left: numeric_type, right: numeric_type)").to("left > right") + .map("$greater_than_or_equal(left: numeric_type, right: numeric_type)").to("left >= right") .map("$add(left: integer_type, right: integer_type)").to("left + right") .map("$subtract(left: integer_type, right: integer_type)").to("left - right") .map("$multiply(left: integer_type, right: integer_type)").to("left * right") @@ -321,7 +326,14 @@ public PostgreSqlClient( .map("$not($is_null(value))").to("value IS NOT NULL") .map("$not(value: boolean)").to("NOT value") .map("$is_null(value)").to("value IS NULL") - .map("$nullif(first, second)").to("NULLIF(first, second)") + .map("$nullif(first, second)").to("NULLIF(first, second)"); + this.connectorExpressionRewriter = connectorExpressionRewriterBuilder.build(); + this.connectorExpressionRewriterWithCollate = connectorExpressionRewriterBuilder + .withTypeClass("collatable_type", ImmutableSet.of("char", "varchar")) + .map("$less_than(left: collatable_type, right: collatable_type)").to("left < right COLLATE \"C\"") + .map("$less_than_or_equal(left: collatable_type, right: collatable_type)").to("left <= right COLLATE \"C\"") + .map("$greater_than(left: collatable_type, right: collatable_type)").to("left > right COLLATE \"C\"") + .map("$greater_than_or_equal(left: collatable_type, right: collatable_type)").to("left >= right COLLATE \"C\"") .build(); JdbcTypeHandle bigintTypeHandle = new JdbcTypeHandle(Types.BIGINT, Optional.of("bigint"), Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty()); @@ -786,6 +798,9 @@ public boolean supportsAggregationPushdown(ConnectorSession session, JdbcTableHa @Override public Optional convertPredicate(ConnectorSession session, ConnectorExpression expression, Map assignments) { + if (isEnableStringPushdownWithCollate(session)) { + return connectorExpressionRewriterWithCollate.rewrite(session, expression, assignments); + } return connectorExpressionRewriter.rewrite(session, expression, assignments); } @@ -1040,10 +1055,10 @@ public Optional implementJoin( ConnectorSession session, JoinType joinType, PreparedQuery leftSource, + Map leftProjections, PreparedQuery rightSource, - List joinConditions, - Map rightAssignments, - Map leftAssignments, + Map rightProjections, + List joinConditions, JoinStatistics statistics) { if (joinType == JoinType.FULL_OUTER) { @@ -1056,7 +1071,7 @@ public Optional implementJoin( leftSource, rightSource, statistics, - () -> super.implementJoin(session, joinType, leftSource, rightSource, joinConditions, rightAssignments, leftAssignments, statistics)); + () -> super.implementJoin(session, joinType, leftSource, leftProjections, rightSource, rightProjections, joinConditions, statistics)); } @Override diff --git a/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestPostgreSqlClient.java b/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestPostgreSqlClient.java index fd47c631f77d3..90692c075be72 100644 --- a/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestPostgreSqlClient.java +++ b/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestPostgreSqlClient.java @@ -13,6 +13,7 @@ */ package io.trino.plugin.postgresql; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import io.trino.plugin.base.mapping.DefaultIdentifierMapping; import io.trino.plugin.jdbc.BaseJdbcConfig; @@ -33,6 +34,7 @@ import io.trino.spi.connector.ConnectorSession; import io.trino.spi.expression.ConnectorExpression; import io.trino.spi.expression.Variable; +import io.trino.spi.session.PropertyMetadata; import io.trino.spi.type.Type; import io.trino.sql.planner.ConnectorExpressionTranslator; import io.trino.sql.planner.LiteralEncoder; @@ -118,7 +120,10 @@ public class TestPostgreSqlClient private static final ConnectorSession SESSION = TestingConnectorSession .builder() - .setPropertyMetadata(new JdbcMetadataSessionProperties(new JdbcMetadataConfig(), Optional.empty()).getSessionProperties()) + .setPropertyMetadata(ImmutableList.>builder() + .addAll(new JdbcMetadataSessionProperties(new JdbcMetadataConfig(), Optional.empty()).getSessionProperties()) + .addAll(new PostgreSqlSessionProperties(new PostgreSqlConfig()).getSessionProperties()) + .build()) .build(); @Test diff --git a/plugin/trino-redshift/src/main/java/io/trino/plugin/redshift/RedshiftClient.java b/plugin/trino-redshift/src/main/java/io/trino/plugin/redshift/RedshiftClient.java index ba04e2693f02c..d39389fc0a7cd 100644 --- a/plugin/trino-redshift/src/main/java/io/trino/plugin/redshift/RedshiftClient.java +++ b/plugin/trino-redshift/src/main/java/io/trino/plugin/redshift/RedshiftClient.java @@ -407,10 +407,10 @@ protected boolean isSupportedJoinCondition(ConnectorSession session, JdbcJoinCon public Optional implementJoin(ConnectorSession session, JoinType joinType, PreparedQuery leftSource, + Map leftProjections, PreparedQuery rightSource, - List joinConditions, - Map rightAssignments, - Map leftAssignments, + Map rightProjections, + List joinConditions, JoinStatistics statistics) { if (joinType == JoinType.FULL_OUTER) { @@ -423,7 +423,7 @@ public Optional implementJoin(ConnectorSession session, leftSource, rightSource, statistics, - () -> super.implementJoin(session, joinType, leftSource, rightSource, joinConditions, rightAssignments, leftAssignments, statistics)); + () -> super.implementJoin(session, joinType, leftSource, leftProjections, rightSource, rightProjections, joinConditions, statistics)); } @Override diff --git a/plugin/trino-singlestore/src/main/java/io/trino/plugin/singlestore/SingleStoreClient.java b/plugin/trino-singlestore/src/main/java/io/trino/plugin/singlestore/SingleStoreClient.java index b715c1b293d83..a080363ff4b76 100644 --- a/plugin/trino-singlestore/src/main/java/io/trino/plugin/singlestore/SingleStoreClient.java +++ b/plugin/trino-singlestore/src/main/java/io/trino/plugin/singlestore/SingleStoreClient.java @@ -15,6 +15,7 @@ import com.google.common.collect.ImmutableSet; import com.google.inject.Inject; +import io.trino.plugin.base.expression.ConnectorExpressionRewriter; import io.trino.plugin.base.mapping.IdentifierMapping; import io.trino.plugin.jdbc.BaseJdbcClient; import io.trino.plugin.jdbc.BaseJdbcConfig; @@ -31,6 +32,8 @@ import io.trino.plugin.jdbc.QueryBuilder; import io.trino.plugin.jdbc.RemoteTableName; import io.trino.plugin.jdbc.WriteMapping; +import io.trino.plugin.jdbc.expression.JdbcConnectorExpressionRewriterBuilder; +import io.trino.plugin.jdbc.expression.ParameterizedExpression; import io.trino.plugin.jdbc.logging.RemoteQueryModifier; import io.trino.spi.TrinoException; import io.trino.spi.connector.AggregateFunction; @@ -40,6 +43,7 @@ import io.trino.spi.connector.JoinStatistics; import io.trino.spi.connector.JoinType; import io.trino.spi.connector.SchemaTableName; +import io.trino.spi.expression.ConnectorExpression; import io.trino.spi.type.CharType; import io.trino.spi.type.DecimalType; import io.trino.spi.type.Decimals; @@ -151,6 +155,7 @@ public class SingleStoreClient private static final Pattern UNSIGNED_TYPE_REGEX = Pattern.compile("(?i).*unsigned$"); private final Type jsonType; + private final ConnectorExpressionRewriter connectorExpressionRewriter; @Inject public SingleStoreClient( @@ -183,6 +188,19 @@ protected SingleStoreClient( super("`", connectionFactory, queryBuilder, config.getJdbcTypesMappedToVarchar(), identifierMapping, queryModifier, supportsRetries); requireNonNull(typeManager, "typeManager is null"); this.jsonType = typeManager.getType(new TypeSignature(StandardTypes.JSON)); + + this.connectorExpressionRewriter = JdbcConnectorExpressionRewriterBuilder.newBuilder() + .addStandardRules(this::quoted) + // No "real" on the list; pushdown on REAL is disabled also in toColumnMapping + .withTypeClass("numeric_type", ImmutableSet.of("tinyint", "smallint", "integer", "bigint", "decimal", "double")) + .map("$equal(left: numeric_type, right: numeric_type)").to("left = right") + .map("$not_equal(left: numeric_type, right: numeric_type)").to("left <> right") + // .map("$is_distinct_from(left: numeric_type, right: numeric_type)").to("left IS DISTINCT FROM right") + .map("$less_than(left: numeric_type, right: numeric_type)").to("left < right") + .map("$less_than_or_equal(left: numeric_type, right: numeric_type)").to("left <= right") + .map("$greater_than(left: numeric_type, right: numeric_type)").to("left > right") + .map("$greater_than_or_equal(left: numeric_type, right: numeric_type)").to("left >= right") + .build(); } @Override @@ -553,22 +571,28 @@ public boolean isTopNGuaranteed(ConnectorSession session) return true; } + @Override + public Optional convertPredicate(ConnectorSession session, ConnectorExpression expression, Map assignments) + { + return connectorExpressionRewriter.rewrite(session, expression, assignments); + } + @Override public Optional implementJoin( ConnectorSession session, JoinType joinType, PreparedQuery leftSource, + Map leftProjections, PreparedQuery rightSource, - List joinConditions, - Map rightAssignments, - Map leftAssignments, + Map rightProjections, + List joinConditions, JoinStatistics statistics) { if (joinType == JoinType.FULL_OUTER) { // Not supported in SingleStore return Optional.empty(); } - return super.implementJoin(session, joinType, leftSource, rightSource, joinConditions, rightAssignments, leftAssignments, statistics); + return super.implementJoin(session, joinType, leftSource, leftProjections, rightSource, rightProjections, joinConditions, statistics); } @Override diff --git a/plugin/trino-sqlserver/src/main/java/io/trino/plugin/sqlserver/SqlServerClient.java b/plugin/trino-sqlserver/src/main/java/io/trino/plugin/sqlserver/SqlServerClient.java index fcb974b939889..2f1fac35ff771 100644 --- a/plugin/trino-sqlserver/src/main/java/io/trino/plugin/sqlserver/SqlServerClient.java +++ b/plugin/trino-sqlserver/src/main/java/io/trino/plugin/sqlserver/SqlServerClient.java @@ -63,7 +63,7 @@ import io.trino.plugin.jdbc.expression.ComparisonOperator; import io.trino.plugin.jdbc.expression.JdbcConnectorExpressionRewriterBuilder; import io.trino.plugin.jdbc.expression.ParameterizedExpression; -import io.trino.plugin.jdbc.expression.RewriteComparison; +import io.trino.plugin.jdbc.expression.RewriteCaseSensitiveComparison; import io.trino.plugin.jdbc.expression.RewriteIn; import io.trino.plugin.jdbc.logging.RemoteQueryModifier; import io.trino.spi.TrinoException; @@ -301,9 +301,16 @@ public SqlServerClient( this.connectorExpressionRewriter = JdbcConnectorExpressionRewriterBuilder.newBuilder() .addStandardRules(this::quoted) - .add(new RewriteComparison(ImmutableSet.of(ComparisonOperator.EQUAL, ComparisonOperator.NOT_EQUAL))) .add(new RewriteIn()) .withTypeClass("integer_type", ImmutableSet.of("tinyint", "smallint", "integer", "bigint")) + .withTypeClass("numeric_type", ImmutableSet.of("tinyint", "smallint", "integer", "bigint", "decimal", "real", "double")) + .map("$equal(left: numeric_type, right: numeric_type)").to("left = right") + .map("$not_equal(left: numeric_type, right: numeric_type)").to("left <> right") + .map("$less_than(left: numeric_type, right: numeric_type)").to("left < right") + .map("$less_than_or_equal(left: numeric_type, right: numeric_type)").to("left <= right") + .map("$greater_than(left: numeric_type, right: numeric_type)").to("left > right") + .map("$greater_than_or_equal(left: numeric_type, right: numeric_type)").to("left >= right") + .add(new RewriteCaseSensitiveComparison(ImmutableSet.of(ComparisonOperator.EQUAL, ComparisonOperator.NOT_EQUAL))) .map("$add(left: integer_type, right: integer_type)").to("left + right") .map("$subtract(left: integer_type, right: integer_type)").to("left - right") .map("$multiply(left: integer_type, right: integer_type)").to("left * right") @@ -882,10 +889,10 @@ public Optional implementJoin( ConnectorSession session, JoinType joinType, PreparedQuery leftSource, + Map leftProjections, PreparedQuery rightSource, - List joinConditions, - Map rightAssignments, - Map leftAssignments, + Map rightProjections, + List joinConditions, JoinStatistics statistics) { return implementJoinCostAware( @@ -894,7 +901,7 @@ public Optional implementJoin( leftSource, rightSource, statistics, - () -> super.implementJoin(session, joinType, leftSource, rightSource, joinConditions, rightAssignments, leftAssignments, statistics)); + () -> super.implementJoin(session, joinType, leftSource, leftProjections, rightSource, rightProjections, joinConditions, statistics)); } private LongWriteFunction sqlServerTimeWriteFunction(int precision) From d0f188ee9062462fd06878d938279aea6e57693e Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 8 Dec 2023 15:43:07 +0100 Subject: [PATCH 315/350] Re-add expression-less join pushdown as fallback Restore older JDBC join pushdown implementation not based on `ConnectorExpression` as a fallback. This comes as a separate commit so that the introduction of `ConnectorExpression`-based join pushdown can be seen (e.g. reviewed) as a _change_, not as an _addition_. --- .../io/trino/plugin/jdbc/BaseJdbcClient.java | 35 ++++++ .../trino/plugin/jdbc/CachingJdbcClient.java | 14 +++ .../plugin/jdbc/DefaultJdbcMetadata.java | 110 ++++++++++++++++++ .../plugin/jdbc/DefaultQueryBuilder.java | 40 +++++++ .../plugin/jdbc/ForwardingJdbcClient.java | 14 +++ .../java/io/trino/plugin/jdbc/JdbcClient.java | 11 ++ .../trino/plugin/jdbc/JdbcMetadataConfig.java | 14 +++ .../jdbc/JdbcMetadataSessionProperties.java | 11 ++ .../io/trino/plugin/jdbc/QueryBuilder.java | 11 ++ .../jdbc/jmx/StatisticsAwareJdbcClient.java | 14 +++ .../plugin/jdbc/BaseJdbcConnectorTest.java | 9 ++ .../jdbc/TestDefaultJdbcQueryBuilder.java | 34 ++++++ .../plugin/jdbc/TestJdbcMetadataConfig.java | 3 + .../io/trino/plugin/ignite/IgniteClient.java | 19 +++ .../trino/plugin/mariadb/MariaDbClient.java | 18 +++ .../io/trino/plugin/mysql/MySqlClient.java | 24 ++++ .../plugin/postgresql/PostgreSqlClient.java | 24 ++++ .../trino/plugin/redshift/RedshiftClient.java | 23 ++++ .../plugin/singlestore/SingleStoreClient.java | 18 +++ .../plugin/sqlserver/SqlServerClient.java | 20 ++++ 20 files changed, 466 insertions(+) diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/BaseJdbcClient.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/BaseJdbcClient.java index 989317dae45d5..42d6731d52cc9 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/BaseJdbcClient.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/BaseJdbcClient.java @@ -550,6 +550,41 @@ public Optional implementJoin( } } + @Deprecated + @Override + public Optional legacyImplementJoin( + ConnectorSession session, + JoinType joinType, + PreparedQuery leftSource, + PreparedQuery rightSource, + List joinConditions, + Map rightAssignments, + Map leftAssignments, + JoinStatistics statistics) + { + for (JdbcJoinCondition joinCondition : joinConditions) { + if (!isSupportedJoinCondition(session, joinCondition)) { + return Optional.empty(); + } + } + + try (Connection connection = this.connectionFactory.openConnection(session)) { + return Optional.of(queryBuilder.legacyPrepareJoinQuery( + this, + session, + connection, + joinType, + leftSource, + rightSource, + joinConditions, + leftAssignments, + rightAssignments)); + } + catch (SQLException e) { + throw new TrinoException(JDBC_ERROR, e); + } + } + protected boolean isSupportedJoinCondition(ConnectorSession session, JdbcJoinCondition joinCondition) { return false; diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/CachingJdbcClient.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/CachingJdbcClient.java index f473c94d96b94..ecea7b964a4eb 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/CachingJdbcClient.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/CachingJdbcClient.java @@ -293,6 +293,20 @@ public Optional implementJoin( return delegate.implementJoin(session, joinType, leftSource, leftProjections, rightSource, rightProjections, joinConditions, statistics); } + @Override + public Optional legacyImplementJoin( + ConnectorSession session, + JoinType joinType, + PreparedQuery leftSource, + PreparedQuery rightSource, + List joinConditions, + Map rightAssignments, + Map leftAssignments, + JoinStatistics statistics) + { + return delegate.legacyImplementJoin(session, joinType, leftSource, rightSource, joinConditions, rightAssignments, leftAssignments, statistics); + } + @Override public boolean supportsTopN(ConnectorSession session, JdbcTableHandle handle, List sortOrder) { diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/DefaultJdbcMetadata.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/DefaultJdbcMetadata.java index 607d813b49ef3..7d0d7f6d6b089 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/DefaultJdbcMetadata.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/DefaultJdbcMetadata.java @@ -40,6 +40,7 @@ import io.trino.spi.connector.Constraint; import io.trino.spi.connector.ConstraintApplicationResult; import io.trino.spi.connector.JoinApplicationResult; +import io.trino.spi.connector.JoinCondition; import io.trino.spi.connector.JoinStatistics; import io.trino.spi.connector.JoinType; import io.trino.spi.connector.LimitApplicationResult; @@ -94,6 +95,7 @@ import static io.trino.plugin.jdbc.JdbcErrorCode.JDBC_NON_TRANSIENT_ERROR; import static io.trino.plugin.jdbc.JdbcMetadataSessionProperties.isAggregationPushdownEnabled; import static io.trino.plugin.jdbc.JdbcMetadataSessionProperties.isComplexExpressionPushdown; +import static io.trino.plugin.jdbc.JdbcMetadataSessionProperties.isComplexJoinPushdownEnabled; import static io.trino.plugin.jdbc.JdbcMetadataSessionProperties.isJoinPushdownEnabled; import static io.trino.plugin.jdbc.JdbcMetadataSessionProperties.isTopNPushdownEnabled; import static io.trino.plugin.jdbc.JdbcWriteSessionProperties.isNonTransactionalInsert; @@ -447,6 +449,19 @@ public Optional> applyJoin( Map rightAssignments, JoinStatistics statistics) { + if (!isComplexJoinPushdownEnabled(session)) { + // Fallback to the old join pushdown code + return JdbcMetadata.super.applyJoin( + session, + joinType, + left, + right, + joinCondition, + leftAssignments, + rightAssignments, + statistics); + } + if (isTableHandleForProcedure(left) || isTableHandleForProcedure(right)) { return Optional.empty(); } @@ -536,6 +551,101 @@ public Optional> applyJoin( precalculateStatisticsForPushdown)); } + @Deprecated + @Override + public Optional> applyJoin( + ConnectorSession session, + JoinType joinType, + ConnectorTableHandle left, + ConnectorTableHandle right, + List joinConditions, + Map leftAssignments, + Map rightAssignments, + JoinStatistics statistics) + { + if (isTableHandleForProcedure(left) || isTableHandleForProcedure(right)) { + return Optional.empty(); + } + + if (!isJoinPushdownEnabled(session)) { + return Optional.empty(); + } + + JdbcTableHandle leftHandle = flushAttributesAsQuery(session, (JdbcTableHandle) left); + JdbcTableHandle rightHandle = flushAttributesAsQuery(session, (JdbcTableHandle) right); + + if (!leftHandle.getAuthorization().equals(rightHandle.getAuthorization())) { + return Optional.empty(); + } + int nextSyntheticColumnId = max(leftHandle.getNextSyntheticColumnId(), rightHandle.getNextSyntheticColumnId()); + + ImmutableMap.Builder newLeftColumnsBuilder = ImmutableMap.builder(); + OptionalInt maxColumnNameLength = jdbcClient.getMaxColumnNameLength(session); + for (JdbcColumnHandle column : jdbcClient.getColumns(session, leftHandle)) { + newLeftColumnsBuilder.put(column, createSyntheticJoinProjectionColumn(column, nextSyntheticColumnId, maxColumnNameLength)); + nextSyntheticColumnId++; + } + Map newLeftColumns = newLeftColumnsBuilder.buildOrThrow(); + + ImmutableMap.Builder newRightColumnsBuilder = ImmutableMap.builder(); + for (JdbcColumnHandle column : jdbcClient.getColumns(session, rightHandle)) { + newRightColumnsBuilder.put(column, createSyntheticJoinProjectionColumn(column, nextSyntheticColumnId, maxColumnNameLength)); + nextSyntheticColumnId++; + } + Map newRightColumns = newRightColumnsBuilder.buildOrThrow(); + + ImmutableList.Builder jdbcJoinConditions = ImmutableList.builder(); + for (JoinCondition joinCondition : joinConditions) { + Optional leftColumn = getVariableColumnHandle(leftAssignments, joinCondition.getLeftExpression()); + Optional rightColumn = getVariableColumnHandle(rightAssignments, joinCondition.getRightExpression()); + if (leftColumn.isEmpty() || rightColumn.isEmpty()) { + return Optional.empty(); + } + jdbcJoinConditions.add(new JdbcJoinCondition(leftColumn.get(), joinCondition.getOperator(), rightColumn.get())); + } + + Optional joinQuery = jdbcClient.legacyImplementJoin( + session, + joinType, + asPreparedQuery(leftHandle), + asPreparedQuery(rightHandle), + jdbcJoinConditions.build(), + newRightColumns.entrySet().stream() + .collect(toImmutableMap(Entry::getKey, entry -> entry.getValue().getColumnName())), + newLeftColumns.entrySet().stream() + .collect(toImmutableMap(Entry::getKey, entry -> entry.getValue().getColumnName())), + statistics); + + if (joinQuery.isEmpty()) { + return Optional.empty(); + } + + return Optional.of(new JoinApplicationResult<>( + new JdbcTableHandle( + new JdbcQueryRelationHandle(joinQuery.get()), + TupleDomain.all(), + ImmutableList.of(), + Optional.empty(), + OptionalLong.empty(), + Optional.of( + ImmutableList.builder() + .addAll(newLeftColumns.values()) + .addAll(newRightColumns.values()) + .build()), + leftHandle.getAllReferencedTables().flatMap(leftReferencedTables -> + rightHandle.getAllReferencedTables().map(rightReferencedTables -> + ImmutableSet.builder() + .addAll(leftReferencedTables) + .addAll(rightReferencedTables) + .build())), + nextSyntheticColumnId, + leftHandle.getAuthorization(), + leftHandle.getUpdateAssignments()), + ImmutableMap.copyOf(newLeftColumns), + ImmutableMap.copyOf(newRightColumns), + precalculateStatisticsForPushdown)); + } + @VisibleForTesting static JdbcColumnHandle createSyntheticJoinProjectionColumn(JdbcColumnHandle column, int nextSyntheticColumnId, OptionalInt optionalMaxColumnNameLength) { diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/DefaultQueryBuilder.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/DefaultQueryBuilder.java index 7d3a4f5f05243..bd06719f9ce1d 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/DefaultQueryBuilder.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/DefaultQueryBuilder.java @@ -147,6 +147,46 @@ public PreparedQuery prepareJoinQuery( return new PreparedQuery(query, parameters); } + @Override + public PreparedQuery legacyPrepareJoinQuery( + JdbcClient client, + ConnectorSession session, + Connection connection, + JoinType joinType, + PreparedQuery leftSource, + PreparedQuery rightSource, + List joinConditions, + Map leftAssignments, + Map rightAssignments) + { + // Verify assignments are present. This is safe assumption as join conditions are not pruned, and simplifies the code here. + verify(!leftAssignments.isEmpty(), "leftAssignments is empty"); + verify(!rightAssignments.isEmpty(), "rightAssignments is empty"); + // Joins wih no conditions are not pushed down, so it is a same assumption and simplifies the code here + verify(!joinConditions.isEmpty(), "joinConditions is empty"); + + String leftRelationAlias = "l"; + String rightRelationAlias = "r"; + + String query = format( + "SELECT %s, %s FROM (%s) %s %s (%s) %s ON %s", + formatAssignments(client, leftRelationAlias, leftAssignments), + formatAssignments(client, rightRelationAlias, rightAssignments), + leftSource.getQuery(), + leftRelationAlias, + formatJoinType(joinType), + rightSource.getQuery(), + rightRelationAlias, + joinConditions.stream() + .map(condition -> formatJoinCondition(client, leftRelationAlias, rightRelationAlias, condition)) + .collect(joining(" AND "))); + List parameters = ImmutableList.builder() + .addAll(leftSource.getParameters()) + .addAll(rightSource.getParameters()) + .build(); + return new PreparedQuery(query, parameters); + } + @Override public PreparedQuery prepareDeleteQuery( JdbcClient client, diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/ForwardingJdbcClient.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/ForwardingJdbcClient.java index f2ac9981f9ef7..3f672896d56c1 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/ForwardingJdbcClient.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/ForwardingJdbcClient.java @@ -220,6 +220,20 @@ public Optional implementJoin( return delegate().implementJoin(session, joinType, leftSource, leftProjections, rightSource, rightProjections, joinConditions, statistics); } + @Override + public Optional legacyImplementJoin( + ConnectorSession session, + JoinType joinType, + PreparedQuery leftSource, + PreparedQuery rightSource, + List joinConditions, + Map rightAssignments, + Map leftAssignments, + JoinStatistics statistics) + { + return delegate().legacyImplementJoin(session, joinType, leftSource, rightSource, joinConditions, rightAssignments, leftAssignments, statistics); + } + @Override public JdbcOutputTableHandle beginCreateTable(ConnectorSession session, ConnectorTableMetadata tableMetadata) { diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcClient.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcClient.java index a52f100c78b14..cb2ff550a8e8f 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcClient.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcClient.java @@ -133,6 +133,17 @@ Optional implementJoin( List joinConditions, JoinStatistics statistics); + @Deprecated + Optional legacyImplementJoin( + ConnectorSession session, + JoinType joinType, + PreparedQuery leftSource, + PreparedQuery rightSource, + List joinConditions, + Map rightAssignments, + Map leftAssignments, + JoinStatistics statistics); + boolean supportsTopN(ConnectorSession session, JdbcTableHandle handle, List sortOrder); /** diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcMetadataConfig.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcMetadataConfig.java index 039195481a86e..d6896c157a1ab 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcMetadataConfig.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcMetadataConfig.java @@ -30,6 +30,7 @@ public class JdbcMetadataConfig * in terms of performance and money due to an increased network traffic. */ private boolean joinPushdownEnabled; + private boolean complexJoinPushdownEnabled = true; private boolean aggregationPushdownEnabled = true; private boolean topNPushdownEnabled = true; @@ -67,6 +68,19 @@ public JdbcMetadataConfig setJoinPushdownEnabled(boolean joinPushdownEnabled) return this; } + public boolean isComplexJoinPushdownEnabled() + { + return complexJoinPushdownEnabled; + } + + @Config("join-pushdown.with-expressions") + @ConfigDescription("Enable join pushdown with complex expressions") + public JdbcMetadataConfig setComplexJoinPushdownEnabled(boolean complexJoinPushdownEnabled) + { + this.complexJoinPushdownEnabled = complexJoinPushdownEnabled; + return this; + } + public boolean isAggregationPushdownEnabled() { return aggregationPushdownEnabled; diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcMetadataSessionProperties.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcMetadataSessionProperties.java index d4ae2a0b5b12b..96476cce488e8 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcMetadataSessionProperties.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcMetadataSessionProperties.java @@ -33,6 +33,7 @@ public class JdbcMetadataSessionProperties { public static final String COMPLEX_EXPRESSION_PUSHDOWN = "complex_expression_pushdown"; public static final String JOIN_PUSHDOWN_ENABLED = "join_pushdown_enabled"; + public static final String COMPLEX_JOIN_PUSHDOWN_ENABLED = "complex_join_pushdown_enabled"; public static final String AGGREGATION_PUSHDOWN_ENABLED = "aggregation_pushdown_enabled"; public static final String TOPN_PUSHDOWN_ENABLED = "topn_pushdown_enabled"; public static final String DOMAIN_COMPACTION_THRESHOLD = "domain_compaction_threshold"; @@ -54,6 +55,11 @@ public JdbcMetadataSessionProperties(JdbcMetadataConfig jdbcMetadataConfig, @Max "Enable join pushdown", jdbcMetadataConfig.isJoinPushdownEnabled(), false)) + .add(booleanProperty( + COMPLEX_JOIN_PUSHDOWN_ENABLED, + "Enable join pushdown with non-comparison expressions", + jdbcMetadataConfig.isComplexJoinPushdownEnabled(), + false)) .add(booleanProperty( AGGREGATION_PUSHDOWN_ENABLED, "Enable aggregation pushdown", @@ -89,6 +95,11 @@ public static boolean isJoinPushdownEnabled(ConnectorSession session) return session.getProperty(JOIN_PUSHDOWN_ENABLED, Boolean.class); } + public static boolean isComplexJoinPushdownEnabled(ConnectorSession session) + { + return session.getProperty(COMPLEX_JOIN_PUSHDOWN_ENABLED, Boolean.class); + } + public static boolean isAggregationPushdownEnabled(ConnectorSession session) { return session.getProperty(AGGREGATION_PUSHDOWN_ENABLED, Boolean.class); diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/QueryBuilder.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/QueryBuilder.java index a6dc6f53d9852..9c6fbf7c4ca95 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/QueryBuilder.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/QueryBuilder.java @@ -52,6 +52,17 @@ PreparedQuery prepareJoinQuery( Map rightProjections, List joinConditions); + PreparedQuery legacyPrepareJoinQuery( + JdbcClient client, + ConnectorSession session, + Connection connection, + JoinType joinType, + PreparedQuery leftSource, + PreparedQuery rightSource, + List joinConditions, + Map leftAssignments, + Map rightAssignments); + PreparedQuery prepareDeleteQuery( JdbcClient client, ConnectorSession session, diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/jmx/StatisticsAwareJdbcClient.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/jmx/StatisticsAwareJdbcClient.java index a190874657506..339cf17df0f55 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/jmx/StatisticsAwareJdbcClient.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/jmx/StatisticsAwareJdbcClient.java @@ -17,6 +17,7 @@ import io.trino.plugin.jdbc.JdbcClient; import io.trino.plugin.jdbc.JdbcColumnHandle; import io.trino.plugin.jdbc.JdbcExpression; +import io.trino.plugin.jdbc.JdbcJoinCondition; import io.trino.plugin.jdbc.JdbcOutputTableHandle; import io.trino.plugin.jdbc.JdbcProcedureHandle; import io.trino.plugin.jdbc.JdbcProcedureHandle.ProcedureQuery; @@ -239,6 +240,19 @@ public Optional implementJoin(ConnectorSession session, return stats.getImplementJoin().wrap(() -> delegate().implementJoin(session, joinType, leftSource, leftProjections, rightSource, rightProjections, joinConditions, statistics)); } + @Override + public Optional legacyImplementJoin(ConnectorSession session, + JoinType joinType, + PreparedQuery leftSource, + PreparedQuery rightSource, + List joinConditions, + Map rightAssignments, + Map leftAssignments, + JoinStatistics statistics) + { + return stats.getImplementJoin().wrap(() -> delegate().legacyImplementJoin(session, joinType, leftSource, rightSource, joinConditions, rightAssignments, leftAssignments, statistics)); + } + @Override public Optional getTableComment(ResultSet resultSet) throws SQLException diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java index 1cf08c54b3fd7..63ab5bebabdf7 100644 --- a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java @@ -64,6 +64,7 @@ import static io.trino.SystemSessionProperties.MARK_DISTINCT_STRATEGY; import static io.trino.plugin.jdbc.JdbcDynamicFilteringSessionProperties.DYNAMIC_FILTERING_ENABLED; import static io.trino.plugin.jdbc.JdbcDynamicFilteringSessionProperties.DYNAMIC_FILTERING_WAIT_TIMEOUT; +import static io.trino.plugin.jdbc.JdbcMetadataSessionProperties.COMPLEX_JOIN_PUSHDOWN_ENABLED; import static io.trino.plugin.jdbc.JdbcMetadataSessionProperties.DOMAIN_COMPACTION_THRESHOLD; import static io.trino.plugin.jdbc.JdbcMetadataSessionProperties.JOIN_PUSHDOWN_ENABLED; import static io.trino.plugin.jdbc.JoinOperator.FULL_JOIN; @@ -1332,6 +1333,14 @@ public void testComplexJoinPushdown() Session session = joinPushdownEnabled(getSession()); String query = "SELECT n.name, o.orderstatus FROM nation n JOIN orders o ON n.regionkey = o.orderkey AND n.nationkey + o.custkey - 3 = 0"; + // The join cannot be pushed down without "complex join pushdown" + assertThat(query( + Session.builder(session) + .setCatalogSessionProperty(catalog, COMPLEX_JOIN_PUSHDOWN_ENABLED, "false") + .build(), + query)) + .joinIsNotFullyPushedDown(); + // The join can be pushed down assertJoinConditionallyPushedDown( session, diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestDefaultJdbcQueryBuilder.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestDefaultJdbcQueryBuilder.java index a407d624d5d86..170df828714ba 100644 --- a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestDefaultJdbcQueryBuilder.java +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestDefaultJdbcQueryBuilder.java @@ -22,6 +22,7 @@ import io.trino.plugin.jdbc.logging.RemoteQueryModifier; import io.trino.spi.connector.ColumnHandle; import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.JoinCondition; import io.trino.spi.connector.JoinType; import io.trino.spi.connector.SchemaTableName; import io.trino.spi.predicate.Domain; @@ -537,6 +538,39 @@ public void testBuildJoinSql() } } + @Test + public void testBuildJoinSqlLegacy() + throws SQLException + { + Connection connection = database.getConnection(); + + PreparedQuery preparedQuery = queryBuilder.legacyPrepareJoinQuery( + jdbcClient, + SESSION, + connection, + JoinType.INNER, + new PreparedQuery("SELECT * FROM \"test_table\"", List.of()), + new PreparedQuery("SELECT * FROM \"test_table\"", List.of()), + List.of(new JdbcJoinCondition(columns.get(7), JoinCondition.Operator.EQUAL, columns.get(8))), + Map.of(columns.get(2), "name1"), + Map.of(columns.get(3), "name2")); + try (PreparedStatement preparedStatement = queryBuilder.prepareStatement(jdbcClient, SESSION, connection, preparedQuery, Optional.empty())) { + assertThat(preparedQuery.getQuery()).isEqualTo("" + + "SELECT l.\"col_2\" AS \"name1\", r.\"col_3\" AS \"name2\" FROM " + + "(SELECT * FROM \"test_table\") l " + + "INNER JOIN " + + "(SELECT * FROM \"test_table\") r " + + "ON l.\"col_7\" = r.\"col_8\""); + long count = 0; + try (ResultSet resultSet = preparedStatement.executeQuery()) { + while (resultSet.next()) { + count++; + } + } + assertThat(count).isEqualTo(8); + } + } + @Test public void testBuildSqlWithLimit() throws SQLException diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcMetadataConfig.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcMetadataConfig.java index ed09ca49e4ba3..e86c88d7a3fcb 100644 --- a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcMetadataConfig.java +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcMetadataConfig.java @@ -30,6 +30,7 @@ public void testDefaults() assertRecordedDefaults(recordDefaults(JdbcMetadataConfig.class) .setComplexExpressionPushdownEnabled(true) .setJoinPushdownEnabled(false) + .setComplexJoinPushdownEnabled(true) .setAggregationPushdownEnabled(true) .setTopNPushdownEnabled(true) .setDomainCompactionThreshold(32)); @@ -41,6 +42,7 @@ public void testExplicitPropertyMappings() Map properties = ImmutableMap.builder() .put("complex-expression-pushdown.enabled", "false") .put("join-pushdown.enabled", "true") + .put("join-pushdown.with-expressions", "false") .put("aggregation-pushdown.enabled", "false") .put("domain-compaction-threshold", "42") .put("topn-pushdown.enabled", "false") @@ -49,6 +51,7 @@ public void testExplicitPropertyMappings() JdbcMetadataConfig expected = new JdbcMetadataConfig() .setComplexExpressionPushdownEnabled(false) .setJoinPushdownEnabled(true) + .setComplexJoinPushdownEnabled(false) .setAggregationPushdownEnabled(false) .setTopNPushdownEnabled(false) .setDomainCompactionThreshold(42); diff --git a/plugin/trino-ignite/src/main/java/io/trino/plugin/ignite/IgniteClient.java b/plugin/trino-ignite/src/main/java/io/trino/plugin/ignite/IgniteClient.java index 7af9b61915fc6..9fa2f946b2ade 100644 --- a/plugin/trino-ignite/src/main/java/io/trino/plugin/ignite/IgniteClient.java +++ b/plugin/trino-ignite/src/main/java/io/trino/plugin/ignite/IgniteClient.java @@ -595,6 +595,25 @@ public Optional implementJoin( return super.implementJoin(session, joinType, leftSource, leftProjections, rightSource, rightProjections, joinConditions, statistics); } + @Override + public Optional legacyImplementJoin( + ConnectorSession session, + JoinType joinType, + PreparedQuery leftSource, + PreparedQuery rightSource, + List joinConditions, + Map rightAssignments, + Map leftAssignments, + JoinStatistics statistics) + { + // Ignite does not support FULL JOIN + if (joinType == JoinType.FULL_OUTER) { + return Optional.empty(); + } + + return super.legacyImplementJoin(session, joinType, leftSource, rightSource, joinConditions, rightAssignments, leftAssignments, statistics); + } + @Override protected boolean isSupportedJoinCondition(ConnectorSession session, JdbcJoinCondition joinCondition) { diff --git a/plugin/trino-mariadb/src/main/java/io/trino/plugin/mariadb/MariaDbClient.java b/plugin/trino-mariadb/src/main/java/io/trino/plugin/mariadb/MariaDbClient.java index 2f8d2ff527de9..18b6e2ec02b51 100644 --- a/plugin/trino-mariadb/src/main/java/io/trino/plugin/mariadb/MariaDbClient.java +++ b/plugin/trino-mariadb/src/main/java/io/trino/plugin/mariadb/MariaDbClient.java @@ -652,6 +652,24 @@ public Optional implementJoin( return super.implementJoin(session, joinType, leftSource, leftProjections, rightSource, rightProjections, joinConditions, statistics); } + @Override + public Optional legacyImplementJoin( + ConnectorSession session, + JoinType joinType, + PreparedQuery leftSource, + PreparedQuery rightSource, + List joinConditions, + Map rightAssignments, + Map leftAssignments, + JoinStatistics statistics) + { + if (joinType == JoinType.FULL_OUTER) { + // Not supported in MariaDB + return Optional.empty(); + } + return super.legacyImplementJoin(session, joinType, leftSource, rightSource, joinConditions, rightAssignments, leftAssignments, statistics); + } + @Override protected boolean isSupportedJoinCondition(ConnectorSession session, JdbcJoinCondition joinCondition) { diff --git a/plugin/trino-mysql/src/main/java/io/trino/plugin/mysql/MySqlClient.java b/plugin/trino-mysql/src/main/java/io/trino/plugin/mysql/MySqlClient.java index cece995ce02b9..241ffd0cf0514 100644 --- a/plugin/trino-mysql/src/main/java/io/trino/plugin/mysql/MySqlClient.java +++ b/plugin/trino-mysql/src/main/java/io/trino/plugin/mysql/MySqlClient.java @@ -1033,6 +1033,30 @@ public Optional implementJoin( () -> super.implementJoin(session, joinType, leftSource, leftProjections, rightSource, rightProjections, joinConditions, statistics)); } + @Override + public Optional legacyImplementJoin( + ConnectorSession session, + JoinType joinType, + PreparedQuery leftSource, + PreparedQuery rightSource, + List joinConditions, + Map rightAssignments, + Map leftAssignments, + JoinStatistics statistics) + { + if (joinType == JoinType.FULL_OUTER) { + // Not supported in MySQL + return Optional.empty(); + } + return implementJoinCostAware( + session, + joinType, + leftSource, + rightSource, + statistics, + () -> super.legacyImplementJoin(session, joinType, leftSource, rightSource, joinConditions, rightAssignments, leftAssignments, statistics)); + } + @Override protected boolean isSupportedJoinCondition(ConnectorSession session, JdbcJoinCondition joinCondition) { diff --git a/plugin/trino-postgresql/src/main/java/io/trino/plugin/postgresql/PostgreSqlClient.java b/plugin/trino-postgresql/src/main/java/io/trino/plugin/postgresql/PostgreSqlClient.java index 976f09c082859..ace69134eb2da 100644 --- a/plugin/trino-postgresql/src/main/java/io/trino/plugin/postgresql/PostgreSqlClient.java +++ b/plugin/trino-postgresql/src/main/java/io/trino/plugin/postgresql/PostgreSqlClient.java @@ -1074,6 +1074,30 @@ public Optional implementJoin( () -> super.implementJoin(session, joinType, leftSource, leftProjections, rightSource, rightProjections, joinConditions, statistics)); } + @Override + public Optional legacyImplementJoin( + ConnectorSession session, + JoinType joinType, + PreparedQuery leftSource, + PreparedQuery rightSource, + List joinConditions, + Map rightAssignments, + Map leftAssignments, + JoinStatistics statistics) + { + if (joinType == JoinType.FULL_OUTER) { + // FULL JOIN is only supported with merge-joinable or hash-joinable join conditions + return Optional.empty(); + } + return implementJoinCostAware( + session, + joinType, + leftSource, + rightSource, + statistics, + () -> super.legacyImplementJoin(session, joinType, leftSource, rightSource, joinConditions, rightAssignments, leftAssignments, statistics)); + } + @Override protected boolean isSupportedJoinCondition(ConnectorSession session, JdbcJoinCondition joinCondition) { diff --git a/plugin/trino-redshift/src/main/java/io/trino/plugin/redshift/RedshiftClient.java b/plugin/trino-redshift/src/main/java/io/trino/plugin/redshift/RedshiftClient.java index d39389fc0a7cd..c1cc81b720332 100644 --- a/plugin/trino-redshift/src/main/java/io/trino/plugin/redshift/RedshiftClient.java +++ b/plugin/trino-redshift/src/main/java/io/trino/plugin/redshift/RedshiftClient.java @@ -426,6 +426,29 @@ public Optional implementJoin(ConnectorSession session, () -> super.implementJoin(session, joinType, leftSource, leftProjections, rightSource, rightProjections, joinConditions, statistics)); } + @Override + public Optional legacyImplementJoin(ConnectorSession session, + JoinType joinType, + PreparedQuery leftSource, + PreparedQuery rightSource, + List joinConditions, + Map rightAssignments, + Map leftAssignments, + JoinStatistics statistics) + { + if (joinType == JoinType.FULL_OUTER) { + // FULL JOIN is only supported with merge-joinable or hash-joinable join conditions + return Optional.empty(); + } + return implementJoinCostAware( + session, + joinType, + leftSource, + rightSource, + statistics, + () -> super.legacyImplementJoin(session, joinType, leftSource, rightSource, joinConditions, rightAssignments, leftAssignments, statistics)); + } + @Override protected void renameTable(ConnectorSession session, Connection connection, String catalogName, String remoteSchemaName, String remoteTableName, String newRemoteSchemaName, String newRemoteTableName) throws SQLException diff --git a/plugin/trino-singlestore/src/main/java/io/trino/plugin/singlestore/SingleStoreClient.java b/plugin/trino-singlestore/src/main/java/io/trino/plugin/singlestore/SingleStoreClient.java index a080363ff4b76..2f2afd7884cb4 100644 --- a/plugin/trino-singlestore/src/main/java/io/trino/plugin/singlestore/SingleStoreClient.java +++ b/plugin/trino-singlestore/src/main/java/io/trino/plugin/singlestore/SingleStoreClient.java @@ -595,6 +595,24 @@ public Optional implementJoin( return super.implementJoin(session, joinType, leftSource, leftProjections, rightSource, rightProjections, joinConditions, statistics); } + @Override + public Optional legacyImplementJoin( + ConnectorSession session, + JoinType joinType, + PreparedQuery leftSource, + PreparedQuery rightSource, + List joinConditions, + Map rightAssignments, + Map leftAssignments, + JoinStatistics statistics) + { + if (joinType == JoinType.FULL_OUTER) { + // Not supported in SingleStore + return Optional.empty(); + } + return super.legacyImplementJoin(session, joinType, leftSource, rightSource, joinConditions, rightAssignments, leftAssignments, statistics); + } + @Override protected boolean isSupportedJoinCondition(ConnectorSession session, JdbcJoinCondition joinCondition) { diff --git a/plugin/trino-sqlserver/src/main/java/io/trino/plugin/sqlserver/SqlServerClient.java b/plugin/trino-sqlserver/src/main/java/io/trino/plugin/sqlserver/SqlServerClient.java index 2f1fac35ff771..a148e184ce72d 100644 --- a/plugin/trino-sqlserver/src/main/java/io/trino/plugin/sqlserver/SqlServerClient.java +++ b/plugin/trino-sqlserver/src/main/java/io/trino/plugin/sqlserver/SqlServerClient.java @@ -904,6 +904,26 @@ public Optional implementJoin( () -> super.implementJoin(session, joinType, leftSource, leftProjections, rightSource, rightProjections, joinConditions, statistics)); } + @Override + public Optional legacyImplementJoin( + ConnectorSession session, + JoinType joinType, + PreparedQuery leftSource, + PreparedQuery rightSource, + List joinConditions, + Map rightAssignments, + Map leftAssignments, + JoinStatistics statistics) + { + return implementJoinCostAware( + session, + joinType, + leftSource, + rightSource, + statistics, + () -> super.legacyImplementJoin(session, joinType, leftSource, rightSource, joinConditions, rightAssignments, leftAssignments, statistics)); + } + private LongWriteFunction sqlServerTimeWriteFunction(int precision) { return new LongWriteFunction() From fd048d0b6bf523ac7e67de0c7a45c3a2bad4e640 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 10 Jan 2024 11:12:34 +0100 Subject: [PATCH 316/350] Fix auto formatting of throwing blocks `{ throw ...; }` lambdas read very well. However, all recent IntelliJ versions insist on removing whitespace before `throw` and after final `;`. This results in checkstyle violation when IDE automatic formatting is used. Reformat lambdas in a way that IDE automatic formatting does not violate checkstyle rules, in order to save time when working with code. --- .../io/trino/metadata/FunctionManager.java | 12 +++++++--- .../io/trino/metadata/MetadataManager.java | 16 +++++++++---- .../trino/sql/analyzer/StatementAnalyzer.java | 4 +++- .../dispatcher/TestLocalDispatchQuery.java | 12 +++++++--- .../metadata/TestGlobalFunctionCatalog.java | 24 ++++++++++++++----- .../sql/planner/TestingPlannerContext.java | 12 +++++++--- .../ExchangeSourceOutputSelector.java | 4 +++- .../io/trino/spi/predicate/TupleDomain.java | 4 +++- .../plugin/jdbc/DefaultQueryBuilder.java | 4 +++- .../deltalake/DeltaLakeSessionProperties.java | 4 +++- .../trino/plugin/ignite/TestIgniteClient.java | 4 +++- .../plugin/postgresql/PostgreSqlClient.java | 8 +++++-- .../postgresql/TestPostgreSqlClient.java | 4 +++- 13 files changed, 84 insertions(+), 28 deletions(-) diff --git a/core/trino-main/src/main/java/io/trino/metadata/FunctionManager.java b/core/trino-main/src/main/java/io/trino/metadata/FunctionManager.java index 9a7c208380ea8..9f53356708ad3 100644 --- a/core/trino-main/src/main/java/io/trino/metadata/FunctionManager.java +++ b/core/trino-main/src/main/java/io/trino/metadata/FunctionManager.java @@ -325,9 +325,15 @@ public static FunctionManager createTestingFunctionManager() { TypeOperators typeOperators = new TypeOperators(); GlobalFunctionCatalog functionCatalog = new GlobalFunctionCatalog( - () -> { throw new UnsupportedOperationException(); }, - () -> { throw new UnsupportedOperationException(); }, - () -> { throw new UnsupportedOperationException(); }); + () -> { + throw new UnsupportedOperationException(); + }, + () -> { + throw new UnsupportedOperationException(); + }, + () -> { + throw new UnsupportedOperationException(); + }); functionCatalog.addFunctions(SystemFunctionBundle.create(new FeaturesConfig(), typeOperators, new BlockTypeOperators(typeOperators), UNKNOWN)); functionCatalog.addFunctions(new InternalFunctionBundle(new LiteralFunction(new InternalBlockEncodingSerde(new BlockEncodingManager(), TESTING_TYPE_MANAGER)))); return new FunctionManager(CatalogServiceProvider.fail(), functionCatalog, LanguageFunctionProvider.DISABLED); diff --git a/core/trino-main/src/main/java/io/trino/metadata/MetadataManager.java b/core/trino-main/src/main/java/io/trino/metadata/MetadataManager.java index 67e5666663a3b..5df172e20cb95 100644 --- a/core/trino-main/src/main/java/io/trino/metadata/MetadataManager.java +++ b/core/trino-main/src/main/java/io/trino/metadata/MetadataManager.java @@ -2044,7 +2044,9 @@ private void verifyProjection(TableHandle table, List proje .map(Variable::getName) .filter(variableName -> !assignedVariables.contains(variableName)) .findAny() - .ifPresent(variableName -> { throw new IllegalStateException("Unbound variable: " + variableName); }); + .ifPresent(variableName -> { + throw new IllegalStateException("Unbound variable: " + variableName); + }); } @Override @@ -2762,9 +2764,15 @@ public MetadataManager build() GlobalFunctionCatalog globalFunctionCatalog = this.globalFunctionCatalog; if (globalFunctionCatalog == null) { globalFunctionCatalog = new GlobalFunctionCatalog( - () -> { throw new UnsupportedOperationException(); }, - () -> { throw new UnsupportedOperationException(); }, - () -> { throw new UnsupportedOperationException(); }); + () -> { + throw new UnsupportedOperationException(); + }, + () -> { + throw new UnsupportedOperationException(); + }, + () -> { + throw new UnsupportedOperationException(); + }); TypeOperators typeOperators = new TypeOperators(); globalFunctionCatalog.addFunctions(SystemFunctionBundle.create(new FeaturesConfig(), typeOperators, new BlockTypeOperators(typeOperators), UNKNOWN)); globalFunctionCatalog.addFunctions(new InternalFunctionBundle(new LiteralFunction(new InternalBlockEncodingSerde(new BlockEncodingManager(), typeManager)))); diff --git a/core/trino-main/src/main/java/io/trino/sql/analyzer/StatementAnalyzer.java b/core/trino-main/src/main/java/io/trino/sql/analyzer/StatementAnalyzer.java index 4a935a59777c2..1560b5e602446 100644 --- a/core/trino-main/src/main/java/io/trino/sql/analyzer/StatementAnalyzer.java +++ b/core/trino-main/src/main/java/io/trino/sql/analyzer/StatementAnalyzer.java @@ -2532,7 +2532,9 @@ private Scope createScopeForView( analysis.unregisterTableForView(); checkViewStaleness(columns, descriptor.getVisibleFields(), name, table) - .ifPresent(explanation -> { throw semanticException(VIEW_IS_STALE, table, "View '%s' is stale or in invalid state: %s", name, explanation); }); + .ifPresent(explanation -> { + throw semanticException(VIEW_IS_STALE, table, "View '%s' is stale or in invalid state: %s", name, explanation); + }); // Derive the type of the view from the stored definition, not from the analysis of the underlying query. // This is needed in case the underlying table(s) changed and the query in the view now produces types that diff --git a/core/trino-main/src/test/java/io/trino/dispatcher/TestLocalDispatchQuery.java b/core/trino-main/src/test/java/io/trino/dispatcher/TestLocalDispatchQuery.java index 3a0b59b5b1955..6cc0e9e990f3a 100644 --- a/core/trino-main/src/test/java/io/trino/dispatcher/TestLocalDispatchQuery.java +++ b/core/trino-main/src/test/java/io/trino/dispatcher/TestLocalDispatchQuery.java @@ -134,9 +134,15 @@ public void testSubmittedForDispatchedQuery() new FunctionManager( new ConnectorCatalogServiceProvider<>("function provider", new NoConnectorServicesProvider(), ConnectorServices::getFunctionProvider), new GlobalFunctionCatalog( - () -> { throw new UnsupportedOperationException(); }, - () -> { throw new UnsupportedOperationException(); }, - () -> { throw new UnsupportedOperationException(); }), + () -> { + throw new UnsupportedOperationException(); + }, + () -> { + throw new UnsupportedOperationException(); + }, + () -> { + throw new UnsupportedOperationException(); + }), LanguageFunctionProvider.DISABLED), new QueryMonitorConfig()); CreateTable createTable = new CreateTable(QualifiedName.of("table"), ImmutableList.of(), FAIL, ImmutableList.of(), Optional.empty()); diff --git a/core/trino-main/src/test/java/io/trino/metadata/TestGlobalFunctionCatalog.java b/core/trino-main/src/test/java/io/trino/metadata/TestGlobalFunctionCatalog.java index 2c6a42d9e975c..28746fffbd70b 100644 --- a/core/trino-main/src/test/java/io/trino/metadata/TestGlobalFunctionCatalog.java +++ b/core/trino-main/src/test/java/io/trino/metadata/TestGlobalFunctionCatalog.java @@ -101,9 +101,15 @@ public void testDuplicateFunctions() TypeOperators typeOperators = new TypeOperators(); GlobalFunctionCatalog globalFunctionCatalog = new GlobalFunctionCatalog( - () -> { throw new UnsupportedOperationException(); }, - () -> { throw new UnsupportedOperationException(); }, - () -> { throw new UnsupportedOperationException(); }); + () -> { + throw new UnsupportedOperationException(); + }, + () -> { + throw new UnsupportedOperationException(); + }, + () -> { + throw new UnsupportedOperationException(); + }); globalFunctionCatalog.addFunctions(SystemFunctionBundle.create(new FeaturesConfig(), typeOperators, new BlockTypeOperators(typeOperators), NodeVersion.UNKNOWN)); globalFunctionCatalog.addFunctions(functionBundle); assertThatThrownBy(() -> globalFunctionCatalog.addFunctions(functionBundle)) @@ -118,9 +124,15 @@ public void testConflictingScalarAggregation() TypeOperators typeOperators = new TypeOperators(); GlobalFunctionCatalog globalFunctionCatalog = new GlobalFunctionCatalog( - () -> { throw new UnsupportedOperationException(); }, - () -> { throw new UnsupportedOperationException(); }, - () -> { throw new UnsupportedOperationException(); }); + () -> { + throw new UnsupportedOperationException(); + }, + () -> { + throw new UnsupportedOperationException(); + }, + () -> { + throw new UnsupportedOperationException(); + }); globalFunctionCatalog.addFunctions(SystemFunctionBundle.create(new FeaturesConfig(), typeOperators, new BlockTypeOperators(typeOperators), NodeVersion.UNKNOWN)); assertThatThrownBy(() -> globalFunctionCatalog.addFunctions(functions)) .isInstanceOf(IllegalStateException.class) diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/TestingPlannerContext.java b/core/trino-main/src/test/java/io/trino/sql/planner/TestingPlannerContext.java index f2e8d9647efd6..1b3ce8e8c4b43 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/TestingPlannerContext.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/TestingPlannerContext.java @@ -120,9 +120,15 @@ public PlannerContext build() parametricTypes.forEach(typeRegistry::addParametricType); GlobalFunctionCatalog globalFunctionCatalog = new GlobalFunctionCatalog( - () -> { throw new UnsupportedOperationException(); }, - () -> { throw new UnsupportedOperationException(); }, - () -> { throw new UnsupportedOperationException(); }); + () -> { + throw new UnsupportedOperationException(); + }, + () -> { + throw new UnsupportedOperationException(); + }, + () -> { + throw new UnsupportedOperationException(); + }); globalFunctionCatalog.addFunctions(SystemFunctionBundle.create(featuresConfig, typeOperators, new BlockTypeOperators(typeOperators), UNKNOWN)); functionBundles.forEach(globalFunctionCatalog::addFunctions); diff --git a/core/trino-spi/src/main/java/io/trino/spi/exchange/ExchangeSourceOutputSelector.java b/core/trino-spi/src/main/java/io/trino/spi/exchange/ExchangeSourceOutputSelector.java index 14ee2c3870a0e..99c023ecc03c0 100644 --- a/core/trino-spi/src/main/java/io/trino/spi/exchange/ExchangeSourceOutputSelector.java +++ b/core/trino-spi/src/main/java/io/trino/spi/exchange/ExchangeSourceOutputSelector.java @@ -173,7 +173,9 @@ public String toString() .collect(toMap( Entry::getKey, Entry::getValue, - (a, b) -> { throw new IllegalArgumentException("got duplicate key " + a + ", " + b); }, + (a, b) -> { + throw new IllegalArgumentException("got duplicate key " + a + ", " + b); + }, TreeMap::new))) .add("finalSelector=" + finalSelector) .toString(); diff --git a/core/trino-spi/src/main/java/io/trino/spi/predicate/TupleDomain.java b/core/trino-spi/src/main/java/io/trino/spi/predicate/TupleDomain.java index 7be8cb97bb5d0..7a8786868ac6a 100644 --- a/core/trino-spi/src/main/java/io/trino/spi/predicate/TupleDomain.java +++ b/core/trino-spi/src/main/java/io/trino/spi/predicate/TupleDomain.java @@ -640,7 +640,9 @@ public Domain getDomain() return toMap( keyMapper, valueMapper, - (u, v) -> { throw new IllegalStateException(format("Duplicate values for a key: %s and %s", u, v)); }, + (u, v) -> { + throw new IllegalStateException(format("Duplicate values for a key: %s and %s", u, v)); + }, LinkedHashMap::new); } diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/DefaultQueryBuilder.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/DefaultQueryBuilder.java index bd06719f9ce1d..202f8c5a8ab24 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/DefaultQueryBuilder.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/DefaultQueryBuilder.java @@ -86,7 +86,9 @@ public PreparedQuery prepareSelectQuery( .filter(domains::containsKey) .filter(column -> columnExpressions.containsKey(column.getColumnName())) .findFirst() - .ifPresent(column -> { throw new IllegalArgumentException(format("Column %s has an expression and a constraint attached at the same time", column)); }); + .ifPresent(column -> { + throw new IllegalArgumentException(format("Column %s has an expression and a constraint attached at the same time", column)); + }); } ImmutableList.Builder conjuncts = ImmutableList.builder(); diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSessionProperties.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSessionProperties.java index 065ac2e187fad..74697fc72bfdb 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSessionProperties.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSessionProperties.java @@ -167,7 +167,9 @@ public DeltaLakeSessionProperties( "Internal Delta Lake connector property", HiveTimestampPrecision.class, MILLISECONDS, - value -> { throw new IllegalStateException("The property cannot be set"); }, + value -> { + throw new IllegalStateException("The property cannot be set"); + }, true), durationProperty( DYNAMIC_FILTERING_WAIT_TIMEOUT, diff --git a/plugin/trino-ignite/src/test/java/io/trino/plugin/ignite/TestIgniteClient.java b/plugin/trino-ignite/src/test/java/io/trino/plugin/ignite/TestIgniteClient.java index 955d0519177d1..ccb88464f350f 100644 --- a/plugin/trino-ignite/src/test/java/io/trino/plugin/ignite/TestIgniteClient.java +++ b/plugin/trino-ignite/src/test/java/io/trino/plugin/ignite/TestIgniteClient.java @@ -79,7 +79,9 @@ public class TestIgniteClient public static final JdbcClient JDBC_CLIENT = new IgniteClient( new BaseJdbcConfig(), - session -> { throw new UnsupportedOperationException(); }, + session -> { + throw new UnsupportedOperationException(); + }, new DefaultQueryBuilder(RemoteQueryModifier.NONE), new DefaultIdentifierMapping(), RemoteQueryModifier.NONE); diff --git a/plugin/trino-postgresql/src/main/java/io/trino/plugin/postgresql/PostgreSqlClient.java b/plugin/trino-postgresql/src/main/java/io/trino/plugin/postgresql/PostgreSqlClient.java index ace69134eb2da..28ddf9fd6c561 100644 --- a/plugin/trino-postgresql/src/main/java/io/trino/plugin/postgresql/PostgreSqlClient.java +++ b/plugin/trino-postgresql/src/main/java/io/trino/plugin/postgresql/PostgreSqlClient.java @@ -1430,7 +1430,9 @@ private ColumnMapping arrayAsJsonColumnMapping(ConnectorSession session, ColumnM return ColumnMapping.sliceMapping( jsonType, arrayAsJsonReadFunction(session, baseElementMapping), - (statement, index, block) -> { throw new UnsupportedOperationException(); }, + (statement, index, block) -> { + throw new UnsupportedOperationException(); + }, DISABLE_PUSHDOWN); } @@ -1558,7 +1560,9 @@ public Slice readSlice(ResultSet resultSet, int columnIndex) return utf8Slice(resultSet.getString(columnIndex)); } }, - (statement, index, value) -> { throw new TrinoException(NOT_SUPPORTED, "Money type is not supported for INSERT"); }, + (statement, index, value) -> { + throw new TrinoException(NOT_SUPPORTED, "Money type is not supported for INSERT"); + }, DISABLE_PUSHDOWN); } diff --git a/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestPostgreSqlClient.java b/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestPostgreSqlClient.java index 90692c075be72..139af68684cec 100644 --- a/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestPostgreSqlClient.java +++ b/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestPostgreSqlClient.java @@ -110,7 +110,9 @@ public class TestPostgreSqlClient new BaseJdbcConfig(), new PostgreSqlConfig(), new JdbcStatisticsConfig(), - session -> { throw new UnsupportedOperationException(); }, + session -> { + throw new UnsupportedOperationException(); + }, new DefaultQueryBuilder(RemoteQueryModifier.NONE), TESTING_TYPE_MANAGER, new DefaultIdentifierMapping(), From c7835d159ce4719b5f13ec77caa101f3088bd0e3 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 10 Jan 2024 10:28:02 +0100 Subject: [PATCH 317/350] Iterate over keys only when only keys needed --- .../trino/plugin/hive/metastore/cache/CachingHiveMetastore.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/CachingHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/CachingHiveMetastore.java index a0ea2cfea32ce..43710f68fc4da 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/CachingHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/CachingHiveMetastore.java @@ -579,7 +579,7 @@ public void updatePartitionStatistics(Table table, Map { + updates.keySet().forEach(partitionName -> { HivePartitionName hivePartitionName = hivePartitionName(hiveTableName(table.getDatabaseName(), table.getTableName()), partitionName); partitionStatisticsCache.invalidate(hivePartitionName); // basic stats are stored as partition properties From 5fbf6b03f67c9f1561b657a0442bd38a8eaed92a Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 10 Jan 2024 10:27:33 +0100 Subject: [PATCH 318/350] Remove redundant and misnamed updatePartitionStatistics "overload" Due to recent changes, `updatePartitionsStatistics` was denoting update of single partition (despite looking plural) `updatePartitionStatistics` was denoting update of multiple partitions (despite looking singular) This commit removes `updatePartitionsStatistics`, redirecting all usages to the other method. --- .../trino/plugin/hive/HiveMetastoreClosure.java | 9 --------- .../hive/metastore/ForwardingHiveMetastore.java | 9 --------- .../plugin/hive/metastore/HiveMetastore.java | 6 ------ .../metastore/SemiTransactionalHiveMetastore.java | 4 ++-- .../metastore/cache/CachingHiveMetastore.java | 14 -------------- .../metastore/tracing/TracingHiveMetastore.java | 11 ----------- .../plugin/hive/procedure/DropStatsProcedure.java | 15 +++++++++------ .../plugin/hive/metastore/MetastoreMethod.java | 1 - .../metastore/cache/TestCachingHiveMetastore.java | 2 +- .../thrift/TestHiveMetastoreAccessOperations.java | 5 ++--- 10 files changed, 14 insertions(+), 62 deletions(-) diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetastoreClosure.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetastoreClosure.java index 13d05eb634043..42555b6233f69 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetastoreClosure.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetastoreClosure.java @@ -140,15 +140,6 @@ public void updateTableStatistics(String databaseName, delegate.updateTableStatistics(databaseName, tableName, transaction, update); } - public void updatePartitionsStatistics(String databaseName, - String tableName, - String partitionName, - Function update) - { - Table table = getExistingTable(databaseName, tableName); - delegate.updatePartitionsStatistics(table, partitionName, update); - } - public void updatePartitionStatistics(String databaseName, String tableName, Map> updates) { Table table = getExistingTable(databaseName, tableName); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/ForwardingHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/ForwardingHiveMetastore.java index b6374f7138c20..702925691c174 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/ForwardingHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/ForwardingHiveMetastore.java @@ -94,15 +94,6 @@ public void updateTableStatistics( delegate.updateTableStatistics(databaseName, tableName, transaction, update); } - @Override - public void updatePartitionsStatistics( - Table table, - String partitionName, - Function update) - { - delegate.updatePartitionsStatistics(table, partitionName, update); - } - @Override public void updatePartitionStatistics( Table table, diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/HiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/HiveMetastore.java index c852b8adf77d9..5be464c024264 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/HiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/HiveMetastore.java @@ -13,7 +13,6 @@ */ package io.trino.plugin.hive.metastore; -import com.google.common.collect.ImmutableMap; import io.trino.hive.thrift.metastore.DataOperationType; import io.trino.plugin.hive.HiveColumnStatisticType; import io.trino.plugin.hive.HivePartition; @@ -56,11 +55,6 @@ public interface HiveMetastore void updateTableStatistics(String databaseName, String tableName, AcidTransaction transaction, Function update); - default void updatePartitionsStatistics(Table table, String partitionName, Function update) - { - updatePartitionStatistics(table, ImmutableMap.of(partitionName, update)); - } - void updatePartitionStatistics(Table table, Map> updates); List getTables(String databaseName); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/SemiTransactionalHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/SemiTransactionalHiveMetastore.java index 838071d7a5de0..d7943790883da 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/SemiTransactionalHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/SemiTransactionalHiveMetastore.java @@ -3365,7 +3365,7 @@ public UpdateStatisticsOperation(SchemaTableName tableName, Optional par public void run(HiveMetastoreClosure metastore, AcidTransaction transaction) { if (partitionName.isPresent()) { - metastore.updatePartitionsStatistics(tableName.getSchemaName(), tableName.getTableName(), partitionName.get(), this::updateStatistics); + metastore.updatePartitionStatistics(tableName.getSchemaName(), tableName.getTableName(), ImmutableMap.of(partitionName.get(), this::updateStatistics)); } else { metastore.updateTableStatistics(tableName.getSchemaName(), tableName.getTableName(), transaction, this::updateStatistics); @@ -3379,7 +3379,7 @@ public void undo(HiveMetastoreClosure metastore, AcidTransaction transaction) return; } if (partitionName.isPresent()) { - metastore.updatePartitionsStatistics(tableName.getSchemaName(), tableName.getTableName(), partitionName.get(), this::resetStatistics); + metastore.updatePartitionStatistics(tableName.getSchemaName(), tableName.getTableName(), ImmutableMap.of(partitionName.get(), this::resetStatistics)); } else { metastore.updateTableStatistics(tableName.getSchemaName(), tableName.getTableName(), transaction, this::resetStatistics); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/CachingHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/CachingHiveMetastore.java index 43710f68fc4da..ef964cb584d9f 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/CachingHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/cache/CachingHiveMetastore.java @@ -558,20 +558,6 @@ public void updateTableStatistics(String databaseName, } } - @Override - public void updatePartitionsStatistics(Table table, String partitionName, Function update) - { - try { - delegate.updatePartitionsStatistics(table, partitionName, update); - } - finally { - HivePartitionName hivePartitionName = hivePartitionName(hiveTableName(table.getDatabaseName(), table.getTableName()), partitionName); - partitionStatisticsCache.invalidate(hivePartitionName); - // basic stats are stored as partition properties - partitionCache.invalidate(hivePartitionName); - } - } - @Override public void updatePartitionStatistics(Table table, Map> updates) { diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/tracing/TracingHiveMetastore.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/tracing/TracingHiveMetastore.java index 18373dd1b45ec..0a6a57306f49d 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/tracing/TracingHiveMetastore.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/metastore/tracing/TracingHiveMetastore.java @@ -148,17 +148,6 @@ public void updateTableStatistics(String databaseName, String tableName, AcidTra withTracing(span, () -> delegate.updateTableStatistics(databaseName, tableName, transaction, update)); } - @Override - public void updatePartitionsStatistics(Table table, String partitionName, Function update) - { - Span span = tracer.spanBuilder("HiveMetastore.updatePartitionsStatistics") - .setAttribute(SCHEMA, table.getDatabaseName()) - .setAttribute(TABLE, table.getTableName()) - .setAttribute(PARTITION, partitionName) - .startSpan(); - withTracing(span, () -> delegate.updatePartitionsStatistics(table, partitionName, update)); - } - @Override public void updatePartitionStatistics(Table table, Map> updates) { diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/DropStatsProcedure.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/DropStatsProcedure.java index 157bde3e95b90..b430db94f02d2 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/DropStatsProcedure.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/procedure/DropStatsProcedure.java @@ -14,6 +14,7 @@ package io.trino.plugin.hive.procedure; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.inject.Inject; import com.google.inject.Provider; import io.trino.plugin.base.util.UncheckedCloseable; @@ -125,11 +126,12 @@ private void doDropStats(ConnectorSession session, ConnectorAccessControl access .collect(toImmutableList()); validatePartitions(partitionStringValues, partitionColumns); - partitionStringValues.forEach(values -> metastore.updatePartitionsStatistics( + partitionStringValues.forEach(values -> metastore.updatePartitionStatistics( schema, table, - makePartName(partitionColumns, values), - stats -> PartitionStatistics.empty())); + ImmutableMap.of( + makePartName(partitionColumns, values), + stats -> PartitionStatistics.empty()))); } else { // no partition specified, so drop stats for the entire table @@ -144,11 +146,12 @@ private void doDropStats(ConnectorSession session, ConnectorAccessControl access else { // the table is partitioned; remove stats for every partition metastore.getPartitionNamesByFilter(handle.getSchemaName(), handle.getTableName(), partitionColumns, TupleDomain.all()) - .ifPresent(partitions -> partitions.forEach(partitionName -> metastore.updatePartitionsStatistics( + .ifPresent(partitions -> partitions.forEach(partitionName -> metastore.updatePartitionStatistics( schema, table, - partitionName, - stats -> PartitionStatistics.empty()))); + ImmutableMap.of( + partitionName, + stats -> PartitionStatistics.empty())))); } } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/MetastoreMethod.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/MetastoreMethod.java index 71cfe96731bc8..30c6461a532b8 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/MetastoreMethod.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/MetastoreMethod.java @@ -39,7 +39,6 @@ public enum MetastoreMethod GET_PARTITION, GET_PARTITION_STATISTICS, UPDATE_PARTITION_STATISTICS, - UPDATE_PARTITIONS_STATISTICS, REPLACE_TABLE, DROP_TABLE, /**/; diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastore.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastore.java index ba9cf74e0f27c..89b78358317d3 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastore.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastore.java @@ -929,7 +929,7 @@ public void testUpdatePartitionStatistics() Table table = hiveMetastoreClosure.getTable(TEST_DATABASE, TEST_TABLE).orElseThrow(); assertThat(mockClient.getAccessCount()).isEqualTo(1); - hiveMetastoreClosure.updatePartitionsStatistics(table.getDatabaseName(), table.getTableName(), TEST_PARTITION1, identity()); + hiveMetastoreClosure.updatePartitionStatistics(table.getDatabaseName(), table.getTableName(), Map.of(TEST_PARTITION1, identity())); assertThat(mockClient.getAccessCount()).isEqualTo(5); } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreAccessOperations.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreAccessOperations.java index 3b816687bd400..a850c387e0b63 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreAccessOperations.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/thrift/TestHiveMetastoreAccessOperations.java @@ -31,7 +31,6 @@ import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_PARTITION_STATISTICS; import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_TABLE; import static io.trino.plugin.hive.metastore.MetastoreMethod.GET_TABLE_STATISTICS; -import static io.trino.plugin.hive.metastore.MetastoreMethod.UPDATE_PARTITIONS_STATISTICS; import static io.trino.plugin.hive.metastore.MetastoreMethod.UPDATE_PARTITION_STATISTICS; import static io.trino.plugin.hive.metastore.MetastoreMethod.UPDATE_TABLE_STATISTICS; import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; @@ -287,7 +286,7 @@ public void testDropStatsPartitionedTable() ImmutableMultiset.builder() .add(GET_TABLE) .add(GET_PARTITION_NAMES_BY_FILTER) - .add(UPDATE_PARTITIONS_STATISTICS) + .add(UPDATE_PARTITION_STATISTICS) .build()); assertUpdate("INSERT INTO drop_stats_partition SELECT 2 AS data, 20 AS part", 1); @@ -296,7 +295,7 @@ public void testDropStatsPartitionedTable() ImmutableMultiset.builder() .add(GET_TABLE) .add(GET_PARTITION_NAMES_BY_FILTER) - .addCopies(UPDATE_PARTITIONS_STATISTICS, 2) + .addCopies(UPDATE_PARTITION_STATISTICS, 2) .build()); } From c809c1f6507e798e46c6271fdea71a924c502564 Mon Sep 17 00:00:00 2001 From: yanxiangqin <54396288+everypp@users.noreply.github.com> Date: Thu, 11 Jan 2024 00:28:50 +0800 Subject: [PATCH 319/350] Fix cast time w/ tz exception message --- .../scalar/timetz/VarcharToTimeWithTimeZoneCast.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/core/trino-main/src/main/java/io/trino/operator/scalar/timetz/VarcharToTimeWithTimeZoneCast.java b/core/trino-main/src/main/java/io/trino/operator/scalar/timetz/VarcharToTimeWithTimeZoneCast.java index b778ef6135ac5..17f6a39568030 100644 --- a/core/trino-main/src/main/java/io/trino/operator/scalar/timetz/VarcharToTimeWithTimeZoneCast.java +++ b/core/trino-main/src/main/java/io/trino/operator/scalar/timetz/VarcharToTimeWithTimeZoneCast.java @@ -51,11 +51,11 @@ private VarcharToTimeWithTimeZoneCast() {} @SqlType("time(p) with time zone") public static long castToShort(@LiteralParameter("p") long precision, ConnectorSession session, @SqlType("varchar(x)") Slice value) { - checkArgument((int) precision <= MAX_SHORT_PRECISION, "precision must be less than max short timestamp precision"); + checkArgument((int) precision <= MAX_SHORT_PRECISION, "precision must be less than max short time with time zone precision"); Matcher matcher = DateTimes.TIME_PATTERN.matcher(trim(value).toStringUtf8()); if (!matcher.matches()) { - throw new TrinoException(INVALID_CAST_ARGUMENT, "Value cannot be cast to timestamp: " + value.toStringUtf8()); + throw new TrinoException(INVALID_CAST_ARGUMENT, "Value cannot be cast to time with time zone: " + value.toStringUtf8()); } try { @@ -67,7 +67,7 @@ public static long castToShort(@LiteralParameter("p") long precision, ConnectorS return packTimeWithTimeZone(nanos, offsetMinutes); } catch (IllegalArgumentException e) { - throw new TrinoException(INVALID_CAST_ARGUMENT, "Value cannot be cast to timestamp: " + value.toStringUtf8(), e); + throw new TrinoException(INVALID_CAST_ARGUMENT, "Value cannot be cast to time with time zone: " + value.toStringUtf8(), e); } } @@ -79,7 +79,7 @@ public static LongTimeWithTimeZone castToLong(@LiteralParameter("p") long precis Matcher matcher = DateTimes.TIME_PATTERN.matcher(trim(value).toStringUtf8()); if (!matcher.matches()) { - throw new TrinoException(INVALID_CAST_ARGUMENT, "Value cannot be cast to timestamp: " + value.toStringUtf8()); + throw new TrinoException(INVALID_CAST_ARGUMENT, "Value cannot be cast to time with time zone: " + value.toStringUtf8()); } try { @@ -91,7 +91,7 @@ public static LongTimeWithTimeZone castToLong(@LiteralParameter("p") long precis return new LongTimeWithTimeZone(picos, offsetMinutes); } catch (IllegalArgumentException e) { - throw new TrinoException(INVALID_CAST_ARGUMENT, "Value cannot be cast to timestamp: " + value.toStringUtf8(), e); + throw new TrinoException(INVALID_CAST_ARGUMENT, "Value cannot be cast to time with time zone: " + value.toStringUtf8(), e); } } From 931e549d137d67f6f9dcd2f7c1cf76a92d21a99f Mon Sep 17 00:00:00 2001 From: Assaf Bern Date: Sun, 24 Dec 2023 13:54:12 +0200 Subject: [PATCH 320/350] Support translation of JsonPathType to ConnectorExpression --- .../ConnectorExpressionTranslator.java | 9 ++++++++- .../TestConnectorExpressionTranslator.java | 19 +++++++++++++++++++ 2 files changed, 27 insertions(+), 1 deletion(-) diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/ConnectorExpressionTranslator.java b/core/trino-main/src/main/java/io/trino/sql/planner/ConnectorExpressionTranslator.java index e376e754c8efe..5123303360b6b 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/ConnectorExpressionTranslator.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/ConnectorExpressionTranslator.java @@ -21,6 +21,7 @@ import io.trino.Session; import io.trino.connector.system.GlobalSystemConnector; import io.trino.metadata.ResolvedFunction; +import io.trino.operator.scalar.JsonPath; import io.trino.plugin.base.expression.ConnectorExpressions; import io.trino.security.AllowAllAccessControl; import io.trino.spi.connector.CatalogSchemaName; @@ -62,6 +63,7 @@ import io.trino.sql.tree.SubscriptExpression; import io.trino.sql.tree.SymbolReference; import io.trino.type.JoniRegexp; +import io.trino.type.JsonPathType; import io.trino.type.LikePattern; import io.trino.type.Re2JRegexp; import io.trino.type.Re2JRegexpType; @@ -333,7 +335,8 @@ private Optional translateCall(String functionName, ResolvedFunction return Optional.empty(); } Expression expression = translated.get(); - if ((formalType == JONI_REGEXP || formalType instanceof Re2JRegexpType) && argumentType instanceof VarcharType) { + if ((formalType == JONI_REGEXP || formalType instanceof Re2JRegexpType || formalType instanceof JsonPathType) + && argumentType instanceof VarcharType) { // These types are not used in connector expressions, so require special handling when translating back to expressions. expression = new Cast(expression, toSqlType(formalType)); } @@ -814,6 +817,10 @@ private ConnectorExpression constantFor(Expression node) Slice pattern = Slices.utf8Slice(((Re2JRegexp) value).pattern()); return new Constant(pattern, createVarcharType(countCodePoints(pattern))); } + if (type instanceof JsonPathType) { + Slice pattern = Slices.utf8Slice(((JsonPath) value).pattern()); + return new Constant(pattern, createVarcharType(countCodePoints(pattern))); + } return new Constant(value, type); } diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/TestConnectorExpressionTranslator.java b/core/trino-main/src/test/java/io/trino/sql/planner/TestConnectorExpressionTranslator.java index 0db62980c9c59..278a67a4fb85b 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/TestConnectorExpressionTranslator.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/TestConnectorExpressionTranslator.java @@ -88,6 +88,7 @@ import static io.trino.sql.planner.TypeAnalyzer.createTestingTypeAnalyzer; import static io.trino.testing.TransactionBuilder.transaction; import static io.trino.type.JoniRegexpType.JONI_REGEXP; +import static io.trino.type.JsonPathType.JSON_PATH; import static io.trino.type.LikeFunctions.likePattern; import static io.trino.type.LikePatternType.LIKE_PATTERN; import static java.nio.charset.StandardCharsets.UTF_8; @@ -455,6 +456,24 @@ public void testTranslateRegularExpression() }); } + @Test + void testTranslateJsonPath() + { + // JSON path type is considered implementation detail of the engine and is not exposed to connectors + // within ConnectorExpression. Instead, it is replaced with a varchar pattern. + assertTranslationRoundTrips( + BuiltinFunctionCallBuilder.resolve(PLANNER_CONTEXT.getMetadata()) + .setName("json_extract_scalar") + .addArgument(VARCHAR_TYPE, new SymbolReference("varchar_symbol_1")) + .addArgument(JSON_PATH, new Cast(new StringLiteral("$.path"), toSqlType(JSON_PATH))) + .build(), + new Call( + VARCHAR_TYPE, + new FunctionName("json_extract_scalar"), + List.of(new Variable("varchar_symbol_1", VARCHAR_TYPE), + new Constant(utf8Slice("$.path"), createVarcharType(6))))); + } + @Test public void testTranslateIn() { From 074b303110cdda3a9fd713f1af6e7f2f3066f007 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jan=20Wa=C5=9B?= Date: Wed, 10 Jan 2024 11:51:50 +0100 Subject: [PATCH 321/350] Move Vale vocabulary to conform to Vale v3.0.0 --- docs/.vale/{Vocab => config/vocabularies}/Base/accept.txt | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename docs/.vale/{Vocab => config/vocabularies}/Base/accept.txt (100%) diff --git a/docs/.vale/Vocab/Base/accept.txt b/docs/.vale/config/vocabularies/Base/accept.txt similarity index 100% rename from docs/.vale/Vocab/Base/accept.txt rename to docs/.vale/config/vocabularies/Base/accept.txt From 4d51ff483d8599409554216833491bf9e28cf9c2 Mon Sep 17 00:00:00 2001 From: Will Morrison Date: Fri, 3 Nov 2023 18:10:12 -0400 Subject: [PATCH 322/350] Document single quote escape --- docs/src/main/sphinx/language/types.md | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/docs/src/main/sphinx/language/types.md b/docs/src/main/sphinx/language/types.md index e4acfe02a0ce3..08ccd5583b150 100644 --- a/docs/src/main/sphinx/language/types.md +++ b/docs/src/main/sphinx/language/types.md @@ -154,13 +154,20 @@ before any Unicode character usage with 4 digits. In the examples above with 6 digits require usage of the plus symbol before the code. For example, you need to use `\+01F600` for a grinning face emoji. +Single quotes in string literals can be escaped by using another single quote: +`'I am big, it''s the pictures that got small!'` + ### `CHAR` Fixed length character data. A `CHAR` type without length specified has a default length of 1. A `CHAR(x)` value always has `x` characters. For example, casting `dog` to `CHAR(7)` adds 4 implicit trailing spaces. Leading and trailing spaces are included in comparisons of `CHAR` values. As a result, two character values with different lengths (`CHAR(x)` and -`CHAR(y)` where `x != y`) will never be equal. +`CHAR(y)` where `x != y`) will never be equal. As with `VARCHAR`, a single quote in a `CHAR` +literal can be escaped with another single quote: +```sql +SELECT CHAR 'All right, Mr. DeMille, I''m ready for my close-up.' +``` Example type definitions: `char`, `char(20)` From 5bc6d2e88d66252e39ea35a3baa5ebc9dbe6fb14 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 10 Jan 2024 12:20:06 +0100 Subject: [PATCH 323/350] Give Hive ACID PT suite a name `suite-8-non-generic` runs only Hive ACID tests. Since these tests are very long and more flaky than others, no other tests will be added to this suite. Rename it. The `hdp3_only` test group verifies environment setup, and as such is redundant, removed. --- .github/workflows/ci.yml | 2 +- .../{Suite8NonGeneric.java => SuiteHiveTransactional.java} | 4 ++-- .../src/main/java/io/trino/tests/product/TestGroups.java | 1 - .../java/io/trino/tests/product/hive/TestHiveCreateTable.java | 4 ++-- 4 files changed, 5 insertions(+), 6 deletions(-) rename testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/suite/suites/{Suite8NonGeneric.java => SuiteHiveTransactional.java} (92%) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 1b2ddb4c9317f..48c77f7745e72 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -884,7 +884,7 @@ jobs: - suite-5 - suite-6-non-generic - suite-7-non-generic - - suite-8-non-generic + - suite-hive-transactional - suite-azure - suite-delta-lake-databricks91 - suite-delta-lake-databricks104 diff --git a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/suite/suites/Suite8NonGeneric.java b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/suite/suites/SuiteHiveTransactional.java similarity index 92% rename from testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/suite/suites/Suite8NonGeneric.java rename to testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/suite/suites/SuiteHiveTransactional.java index be6a4c7860e3c..fff324a09477d 100644 --- a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/suite/suites/Suite8NonGeneric.java +++ b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/suite/suites/SuiteHiveTransactional.java @@ -25,7 +25,7 @@ import static com.google.common.base.Verify.verify; import static io.trino.tests.product.launcher.suite.SuiteTestRun.testOnEnvironment; -public class Suite8NonGeneric +public class SuiteHiveTransactional extends Suite { @Override @@ -35,7 +35,7 @@ public List getTestRuns(EnvironmentConfig config) return ImmutableList.of( testOnEnvironment(EnvSinglenodeHiveAcid.class) - .withGroups("configured_features", "hdp3_only", "hive_transactional") + .withGroups("configured_features", "hive_transactional") .build()); } } diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/TestGroups.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/TestGroups.java index 3ab097d94fb4b..c6036a9449c32 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/TestGroups.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/TestGroups.java @@ -64,7 +64,6 @@ public final class TestGroups public static final String LDAP_CLI = "ldap_cli"; public static final String LDAP_AND_FILE_CLI = "ldap_and_file_cli"; public static final String LDAP_MULTIPLE_BINDS = "ldap_multiple_binds"; - public static final String HDP3_ONLY = "hdp3_only"; public static final String TLS = "tls"; public static final String ROLES = "roles"; public static final String CANCEL_QUERY = "cancel_query"; diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveCreateTable.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveCreateTable.java index 1e75070b1613b..286fe88e05281 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveCreateTable.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveCreateTable.java @@ -24,7 +24,7 @@ import static io.trino.plugin.hive.HiveTableProperties.TRANSACTIONAL; import static io.trino.tempto.assertions.QueryAssert.Row.row; -import static io.trino.tests.product.TestGroups.HDP3_ONLY; +import static io.trino.tests.product.TestGroups.HIVE_TRANSACTIONAL; import static io.trino.tests.product.TestGroups.PROFILE_SPECIFIC_TESTS; import static io.trino.tests.product.TestGroups.STORAGE_FORMATS; import static io.trino.tests.product.utils.HadoopTestUtils.RETRYABLE_FAILURES_ISSUES; @@ -81,7 +81,7 @@ public void testCreateTableAsSelect() onTrino().executeQuery("DROP TABLE test_create_table_as_select"); } - @Test(groups = {HDP3_ONLY, PROFILE_SPECIFIC_TESTS}) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}) public void testVerifyEnvironmentHiveTransactionalByDefault() throws SQLException { From c9df89df66af6dd3e2a13ed5f99c640b8e3952f9 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 10 Jan 2024 12:22:20 +0100 Subject: [PATCH 324/350] Remove Hive transactional tests from suite1 No need to run these tests in Suite1. They are covered in a different suite. --- .../hive/TestHiveTransactionalTable.java | 177 +++++++++--------- 1 file changed, 89 insertions(+), 88 deletions(-) diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveTransactionalTable.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveTransactionalTable.java index a4b569aa55558..44eeb86f59a41 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveTransactionalTable.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveTransactionalTable.java @@ -55,6 +55,7 @@ import static io.trino.tempto.assertions.QueryAssert.assertQueryFailure; import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.tests.product.TestGroups.HIVE_TRANSACTIONAL; +import static io.trino.tests.product.TestGroups.PROFILE_SPECIFIC_TESTS; import static io.trino.tests.product.TestGroups.STORAGE_FORMATS; import static io.trino.tests.product.hive.BucketingType.BUCKETED_V2; import static io.trino.tests.product.hive.BucketingType.NONE; @@ -97,20 +98,20 @@ public class TestHiveTransactionalTable @Inject private HdfsClient hdfsClient; - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) public void testReadFullAcid() { doTestReadFullAcid(false, BucketingType.NONE); } @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) public void testReadFullAcidBucketed() { doTestReadFullAcid(false, BucketingType.BUCKETED_DEFAULT); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testReadFullAcidPartitioned() { @@ -119,14 +120,14 @@ public void testReadFullAcidPartitioned() // This test is in STORAGE_FORMATS group to ensure test coverage of transactional tables with various // metastore and HDFS setups (kerberized or not, impersonation or not). - @Test(groups = {HIVE_TRANSACTIONAL, STORAGE_FORMATS}, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, STORAGE_FORMATS, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testReadFullAcidPartitionedBucketed() { doTestReadFullAcid(true, BucketingType.BUCKETED_DEFAULT); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testReadFullAcidBucketedV1() { @@ -134,7 +135,7 @@ public void testReadFullAcidBucketedV1() } @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) public void testReadFullAcidBucketedV2() { doTestReadFullAcid(false, BucketingType.BUCKETED_V2); @@ -195,28 +196,28 @@ private void doTestReadFullAcid(boolean isPartitioned, BucketingType bucketingTy } } - @Test(groups = HIVE_TRANSACTIONAL, dataProvider = "partitioningAndBucketingTypeDataProvider", timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, dataProvider = "partitioningAndBucketingTypeDataProvider", timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testReadInsertOnlyOrc(boolean isPartitioned, BucketingType bucketingType) { testReadInsertOnly(isPartitioned, bucketingType, "STORED AS ORC"); } - @Test(groups = HIVE_TRANSACTIONAL, dataProvider = "partitioningAndBucketingTypeSmokeDataProvider", timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, dataProvider = "partitioningAndBucketingTypeSmokeDataProvider", timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testReadInsertOnlyParquet(boolean isPartitioned, BucketingType bucketingType) { testReadInsertOnly(isPartitioned, bucketingType, "STORED AS PARQUET"); } - @Test(groups = HIVE_TRANSACTIONAL, dataProvider = "partitioningAndBucketingTypeSmokeDataProvider", timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, dataProvider = "partitioningAndBucketingTypeSmokeDataProvider", timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testReadInsertOnlyText(boolean isPartitioned, BucketingType bucketingType) { testReadInsertOnly(isPartitioned, bucketingType, "STORED AS TEXTFILE"); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testReadInsertOnlyTextWithCustomFormatProperties() { @@ -272,7 +273,7 @@ private void testReadInsertOnly(boolean isPartitioned, BucketingType bucketingTy } } - @Test(groups = {STORAGE_FORMATS, HIVE_TRANSACTIONAL}, dataProvider = "partitioningAndBucketingTypeDataProvider", timeOut = TEST_TIMEOUT) + @Test(groups = {STORAGE_FORMATS, HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, dataProvider = "partitioningAndBucketingTypeDataProvider", timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testReadFullAcidWithOriginalFiles(boolean isPartitioned, BucketingType bucketingType) { @@ -316,7 +317,7 @@ public void testReadFullAcidWithOriginalFiles(boolean isPartitioned, BucketingTy } } - @Test(groups = {STORAGE_FORMATS, HIVE_TRANSACTIONAL}, dataProvider = "partitioningAndBucketingTypeDataProvider", timeOut = TEST_TIMEOUT) + @Test(groups = {STORAGE_FORMATS, HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, dataProvider = "partitioningAndBucketingTypeDataProvider", timeOut = TEST_TIMEOUT) @Flaky(issue = RETRYABLE_FAILURES_ISSUES, match = RETRYABLE_FAILURES_MATCH) public void testUpdateFullAcidWithOriginalFilesTrinoInserting(boolean isPartitioned, BucketingType bucketingType) { @@ -366,7 +367,7 @@ public void testUpdateFullAcidWithOriginalFilesTrinoInserting(boolean isPartitio }); } - @Test(groups = {STORAGE_FORMATS, HIVE_TRANSACTIONAL}, dataProvider = "partitioningAndBucketingTypeDataProvider", timeOut = TEST_TIMEOUT) + @Test(groups = {STORAGE_FORMATS, HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, dataProvider = "partitioningAndBucketingTypeDataProvider", timeOut = TEST_TIMEOUT) @Flaky(issue = RETRYABLE_FAILURES_ISSUES, match = RETRYABLE_FAILURES_MATCH) public void testUpdateFullAcidWithOriginalFilesTrinoInsertingAndDeleting(boolean isPartitioned, BucketingType bucketingType) { @@ -418,7 +419,7 @@ String makeValues(int colStart, int colCount, int fcol, boolean isPartitioned, i .collect(Collectors.joining(", ")); } - @Test(groups = {STORAGE_FORMATS, HIVE_TRANSACTIONAL}, dataProvider = "partitioningAndBucketingTypeDataProvider", timeOut = TEST_TIMEOUT) + @Test(groups = {STORAGE_FORMATS, HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, dataProvider = "partitioningAndBucketingTypeDataProvider", timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testReadInsertOnlyWithOriginalFiles(boolean isPartitioned, BucketingType bucketingType) { @@ -473,7 +474,7 @@ public Object[][] partitioningAndBucketingTypeSmokeDataProvider() }; } - @Test(groups = HIVE_TRANSACTIONAL, dataProvider = "testCreateAcidTableDataProvider") + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, dataProvider = "testCreateAcidTableDataProvider") @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testCtasAcidTable(boolean isPartitioned, BucketingType bucketingType) { @@ -493,7 +494,7 @@ public void testCtasAcidTable(boolean isPartitioned, BucketingType bucketingType } } - @Test(groups = HIVE_TRANSACTIONAL, dataProvider = "testCreateAcidTableDataProvider") + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, dataProvider = "testCreateAcidTableDataProvider") @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testCreateAcidTable(boolean isPartitioned, BucketingType bucketingType) { @@ -506,7 +507,7 @@ public void testCreateAcidTable(boolean isPartitioned, BucketingType bucketingTy }); } - @Test(groups = HIVE_TRANSACTIONAL, dataProvider = "acidFormatColumnNames") + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, dataProvider = "acidFormatColumnNames") @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidTableColumnNameConflict(String columnName) { @@ -530,7 +531,7 @@ public Object[][] acidFormatColumnNames() }; } - @Test(groups = HIVE_TRANSACTIONAL) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testSimpleUnpartitionedTransactionalInsert() { @@ -553,7 +554,7 @@ public void testSimpleUnpartitionedTransactionalInsert() }); } - @Test(groups = HIVE_TRANSACTIONAL) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testTransactionalPartitionInsert() { @@ -587,14 +588,14 @@ public void testTransactionalPartitionInsert() }); } - @Test(groups = HIVE_TRANSACTIONAL) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testTransactionalBucketedPartitionedInsert() { testTransactionalBucketedPartitioned(false); } - @Test(groups = HIVE_TRANSACTIONAL) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testTransactionalBucketedPartitionedInsertOnly() { @@ -627,7 +628,7 @@ private void testTransactionalBucketedPartitioned(boolean insertOnly) }); } - @Test(groups = HIVE_TRANSACTIONAL, dataProvider = "inserterAndDeleterProvider", timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, dataProvider = "inserterAndDeleterProvider", timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testTransactionalUnpartitionedDelete(Engine inserter, Engine deleter) { @@ -646,7 +647,7 @@ public void testTransactionalUnpartitionedDelete(Engine inserter, Engine deleter }); } - @Test(groups = HIVE_TRANSACTIONAL, dataProvider = "inserterAndDeleterProvider", timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, dataProvider = "inserterAndDeleterProvider", timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testMultiDelete(Engine inserter, Engine deleter) { @@ -661,7 +662,7 @@ public void testMultiDelete(Engine inserter, Engine deleter) }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testReadAfterMultiInsertAndDelete() { @@ -691,7 +692,7 @@ public void testReadAfterMultiInsertAndDelete() }); } - @Test(groups = HIVE_TRANSACTIONAL, dataProvider = "inserterAndDeleterProvider", timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, dataProvider = "inserterAndDeleterProvider", timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testTransactionalMetadataDelete(Engine inserter, Engine deleter) { @@ -707,7 +708,7 @@ public void testTransactionalMetadataDelete(Engine inserter, Engine deleter) }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) @Flaky(issue = RETRYABLE_FAILURES_ISSUES, match = RETRYABLE_FAILURES_MATCH) public void testNonTransactionalMetadataDelete() { @@ -729,7 +730,7 @@ public void testNonTransactionalMetadataDelete() }); } - @Test(groups = HIVE_TRANSACTIONAL, dataProvider = "inserterAndDeleterProvider", timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, dataProvider = "inserterAndDeleterProvider", timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testUnpartitionedDeleteAll(Engine inserter, Engine deleter) { @@ -741,7 +742,7 @@ public void testUnpartitionedDeleteAll(Engine inserter, Engine deleter) }); } - @Test(groups = HIVE_TRANSACTIONAL, dataProvider = "inserterAndDeleterProvider", timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, dataProvider = "inserterAndDeleterProvider", timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testMultiColumnDelete(Engine inserter, Engine deleter) { @@ -754,7 +755,7 @@ public void testMultiColumnDelete(Engine inserter, Engine deleter) }); } - @Test(groups = HIVE_TRANSACTIONAL, dataProvider = "inserterAndDeleterProvider", timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, dataProvider = "inserterAndDeleterProvider", timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testPartitionAndRowsDelete(Engine inserter, Engine deleter) { @@ -768,7 +769,7 @@ public void testPartitionAndRowsDelete(Engine inserter, Engine deleter) }); } - @Test(groups = HIVE_TRANSACTIONAL, dataProvider = "inserterAndDeleterProvider", timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, dataProvider = "inserterAndDeleterProvider", timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testPartitionedInsertAndRowLevelDelete(Engine inserter, Engine deleter) { @@ -792,7 +793,7 @@ public void testPartitionedInsertAndRowLevelDelete(Engine inserter, Engine delet }); } - @Test(groups = HIVE_TRANSACTIONAL, dataProvider = "inserterAndDeleterProvider", timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, dataProvider = "inserterAndDeleterProvider", timeOut = TEST_TIMEOUT) @Flaky(issue = RETRYABLE_FAILURES_ISSUES, match = RETRYABLE_FAILURES_MATCH) public void testBucketedPartitionedDelete(Engine inserter, Engine deleter) { @@ -823,7 +824,7 @@ public void testBucketedPartitionedDelete(Engine inserter, Engine deleter) }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testDeleteAllRowsInPartition() { @@ -841,7 +842,7 @@ public void testDeleteAllRowsInPartition() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testDeleteAfterDelete() { @@ -860,7 +861,7 @@ public void testDeleteAfterDelete() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testDeleteAfterDeleteWithPredicate() { @@ -880,7 +881,7 @@ public void testDeleteAfterDeleteWithPredicate() }); } - @Test(groups = HIVE_TRANSACTIONAL, dataProvider = "inserterAndDeleterProvider", timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, dataProvider = "inserterAndDeleterProvider", timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testBucketedUnpartitionedDelete(Engine inserter, Engine deleter) { @@ -911,7 +912,7 @@ public void testBucketedUnpartitionedDelete(Engine inserter, Engine deleter) }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testDeleteOverManySplits() { @@ -925,7 +926,7 @@ public void testDeleteOverManySplits() }); } - @Test(groups = HIVE_TRANSACTIONAL, dataProvider = "inserterAndDeleterProvider", timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, dataProvider = "inserterAndDeleterProvider", timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testCorrectSelectCountStar(Engine inserter, Engine deleter) { @@ -938,7 +939,7 @@ public void testCorrectSelectCountStar(Engine inserter, Engine deleter) }); } - @Test(groups = HIVE_TRANSACTIONAL, dataProvider = "insertersProvider", timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, dataProvider = "insertersProvider", timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testInsertOnlyMultipleWriters(boolean bucketed, Engine inserter1, Engine inserter2) { @@ -963,7 +964,7 @@ public void testInsertOnlyMultipleWriters(boolean bucketed, Engine inserter1, En }); } - @Test(groups = HIVE_TRANSACTIONAL) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testInsertFailsInExplicitTrinoTransaction() { @@ -975,7 +976,7 @@ public void testInsertFailsInExplicitTrinoTransaction() }); } - @Test(groups = HIVE_TRANSACTIONAL) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testUpdateFailsInExplicitTrinoTransaction() { @@ -987,7 +988,7 @@ public void testUpdateFailsInExplicitTrinoTransaction() }); } - @Test(groups = HIVE_TRANSACTIONAL) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testDeleteFailsInExplicitTrinoTransaction() { @@ -999,7 +1000,7 @@ public void testDeleteFailsInExplicitTrinoTransaction() }); } - @Test(groups = HIVE_TRANSACTIONAL, dataProvider = "transactionModeProvider") + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, dataProvider = "transactionModeProvider") @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testColumnRenamesOrcPartitioned(boolean transactional) { @@ -1014,7 +1015,7 @@ public void testColumnRenamesOrcPartitioned(boolean transactional) }); } - @Test(groups = HIVE_TRANSACTIONAL, dataProvider = "transactionModeProvider") + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, dataProvider = "transactionModeProvider") @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testColumnRenamesOrcNotPartitioned(boolean transactional) { @@ -1047,7 +1048,7 @@ private void testOrcColumnRenames(String tableName) verifySelectForTrinoAndHive("SELECT * FROM " + tableName, row(111, "Katy", 57, "CA"), row(222, "Joe", 72, "WA"), row(333, "Joan", 23, "OR")); } - @Test(groups = HIVE_TRANSACTIONAL, dataProvider = "transactionModeProvider") + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, dataProvider = "transactionModeProvider") @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testOrcColumnSwap(boolean transactional) { @@ -1064,7 +1065,7 @@ public void testOrcColumnSwap(boolean transactional) }); } - @Test(groups = HIVE_TRANSACTIONAL) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testBehaviorOnParquetColumnRenames() { @@ -1087,7 +1088,7 @@ public void testBehaviorOnParquetColumnRenames() }); } - @Test(groups = HIVE_TRANSACTIONAL, dataProvider = "transactionModeProvider") + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, dataProvider = "transactionModeProvider") @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testOrcColumnDropAdd(boolean transactional) { @@ -1109,7 +1110,7 @@ public void testOrcColumnDropAdd(boolean transactional) }); } - @Test(groups = HIVE_TRANSACTIONAL, dataProvider = "transactionModeProvider") + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, dataProvider = "transactionModeProvider") @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testOrcColumnTypeChange(boolean transactional) { @@ -1128,7 +1129,7 @@ public void testOrcColumnTypeChange(boolean transactional) }); } - @Test(groups = HIVE_TRANSACTIONAL) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testParquetColumnDropAdd() { @@ -1164,7 +1165,7 @@ public Object[][] transactionModeProvider() }; } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidUpdateFailNonTransactional() { @@ -1180,7 +1181,7 @@ public void testAcidUpdateFailNonTransactional() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidUpdateFailInsertOnlyTable() { @@ -1198,7 +1199,7 @@ public void testAcidUpdateFailInsertOnlyTable() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidDeleteFailNonTransactional() { @@ -1214,7 +1215,7 @@ public void testAcidDeleteFailNonTransactional() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidDeleteFailInsertOnlyTable() { @@ -1232,7 +1233,7 @@ public void testAcidDeleteFailInsertOnlyTable() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidUpdateSucceedUpdatingPartitionKey() { @@ -1250,7 +1251,7 @@ public void testAcidUpdateSucceedUpdatingPartitionKey() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidUpdateSucceedUpdatingBucketColumn() { @@ -1268,7 +1269,7 @@ public void testAcidUpdateSucceedUpdatingBucketColumn() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidUpdateFailOnIllegalCast() { @@ -1284,7 +1285,7 @@ public void testAcidUpdateFailOnIllegalCast() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidUpdateSimple() { @@ -1299,7 +1300,7 @@ public void testAcidUpdateSimple() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidUpdateSelectedValues() { @@ -1314,7 +1315,7 @@ public void testAcidUpdateSelectedValues() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidUpdateCopyColumn() { @@ -1329,7 +1330,7 @@ public void testAcidUpdateCopyColumn() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidUpdateSomeLiteralNullColumnValues() { @@ -1348,7 +1349,7 @@ public void testAcidUpdateSomeLiteralNullColumnValues() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidUpdateSomeComputedNullColumnValues() { @@ -1368,7 +1369,7 @@ public void testAcidUpdateSomeComputedNullColumnValues() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidUpdateAllLiteralNullColumnValues() { @@ -1383,7 +1384,7 @@ public void testAcidUpdateAllLiteralNullColumnValues() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidUpdateAllComputedNullColumnValues() { @@ -1399,7 +1400,7 @@ public void testAcidUpdateAllComputedNullColumnValues() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidUpdateReversed() { @@ -1414,7 +1415,7 @@ public void testAcidUpdateReversed() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidUpdatePermuted() { @@ -1429,7 +1430,7 @@ public void testAcidUpdatePermuted() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidUpdateAllColumnsSetAndDependencies() { @@ -1444,7 +1445,7 @@ public void testAcidUpdateAllColumnsSetAndDependencies() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidUpdatePartitioned() { @@ -1461,7 +1462,7 @@ public void testAcidUpdatePartitioned() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidUpdateBucketed() { @@ -1478,7 +1479,7 @@ public void testAcidUpdateBucketed() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidUpdateMajorCompaction() { @@ -1500,7 +1501,7 @@ public void testAcidUpdateMajorCompaction() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidUpdateWithSubqueryPredicate() { @@ -1531,7 +1532,7 @@ public void testAcidUpdateWithSubqueryPredicate() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidUpdateWithSubqueryAssignment() { @@ -1562,7 +1563,7 @@ public void testAcidUpdateWithSubqueryAssignment() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidUpdateDuplicateUpdateValue() { @@ -1598,7 +1599,7 @@ public void testAcidUpdateDuplicateUpdateValue() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testAcidUpdateMultipleDuplicateValues() { @@ -1626,7 +1627,7 @@ public void testAcidUpdateMultipleDuplicateValues() } @Flaky(issue = RETRYABLE_FAILURES_ISSUES, match = RETRYABLE_FAILURES_MATCH) - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) public void testInsertDeleteUpdateWithTrinoAndHive() { withTemporaryTable("update_insert_delete_trino_hive", true, NONE, tableName -> { @@ -1662,7 +1663,7 @@ public void testInsertDeleteUpdateWithTrinoAndHive() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testDeleteFromOriginalFiles() { @@ -1678,14 +1679,14 @@ public void testDeleteFromOriginalFiles() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testDeleteWholePartition() { testDeleteWholePartition(false); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testDeleteWholePartitionWithOriginalFiles() { @@ -1725,7 +1726,7 @@ private void testDeleteWholePartition(boolean withOriginalFiles) }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testUpdateOriginalFilesPartitioned() { @@ -1739,7 +1740,7 @@ public void testUpdateOriginalFilesPartitioned() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testUpdateOriginalFilesUnpartitioned() { @@ -1753,7 +1754,7 @@ public void testUpdateOriginalFilesUnpartitioned() }); } - @Test(groups = HIVE_TRANSACTIONAL, timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testInsertRowIdCorrectness() { @@ -1833,7 +1834,7 @@ void withTemporaryTable(String rootName, boolean isPartitioned, BucketingType bu } } - @Test(groups = HIVE_TRANSACTIONAL) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}) @Flaky(issue = "https://github.com/trinodb/trino/issues/5463", match = "Expected row count to be <4>, but was <6>") public void testFilesForAbortedTransactionsIgnored() throws Exception @@ -1889,7 +1890,7 @@ public void testFilesForAbortedTransactionsIgnored() } } - @Test(groups = HIVE_TRANSACTIONAL) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testDoubleUpdateAndThenReadFromHive() { @@ -1910,7 +1911,7 @@ public void testDoubleUpdateAndThenReadFromHive() }); } - @Test(groups = HIVE_TRANSACTIONAL) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testDeleteWithOriginalFiles() { @@ -1932,7 +1933,7 @@ public void testDeleteWithOriginalFiles() }); } - @Test(groups = HIVE_TRANSACTIONAL) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testDeleteWithOriginalFilesWithWhereClause() { @@ -1970,7 +1971,7 @@ private void validateFileIsDirectlyUnderTableLocation(String tableName) "files in %s are not directly under table location", path)); } - @Test(groups = HIVE_TRANSACTIONAL) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testDeleteAfterMajorCompaction() { @@ -1982,14 +1983,14 @@ public void testDeleteAfterMajorCompaction() }); } - @Test(groups = HIVE_TRANSACTIONAL) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testUnbucketedPartitionedTransactionalTableWithTaskWriterCountGreaterThanOne() { unbucketedTransactionalTableWithTaskWriterCountGreaterThanOne(true); } - @Test(groups = HIVE_TRANSACTIONAL) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testUnbucketedTransactionalTableWithTaskWriterCountGreaterThanOne() { @@ -2038,7 +2039,7 @@ private void unbucketedTransactionalTableWithTaskWriterCountGreaterThanOne(boole }); } - @Test(groups = HIVE_TRANSACTIONAL) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testLargePartitionedDelete() { @@ -2069,7 +2070,7 @@ public void testLargePartitionedDelete() }); } - @Test(groups = HIVE_TRANSACTIONAL) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testLargePartitionedUpdate() { From d86d5ab3930b5e730efd0ee64c47c3009ffbade4 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 10 Jan 2024 12:30:48 +0100 Subject: [PATCH 325/350] Reduce Hive transactional coverage in suites Hive transactional tables are tested as part of the STORAGE_FORMATS group to ensure test coverage of transactional tables with various metastore and HDFS setups (kerberized or not, impersonation or not). Even so, this coverage can be reduced. We do not need to test e.g. various bucketing versions. --- .../hive/TestHiveTransactionalTable.java | 23 +++++++++++++++---- 1 file changed, 19 insertions(+), 4 deletions(-) diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveTransactionalTable.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveTransactionalTable.java index 44eeb86f59a41..d94bfb81b3cf9 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveTransactionalTable.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveTransactionalTable.java @@ -57,6 +57,7 @@ import static io.trino.tests.product.TestGroups.HIVE_TRANSACTIONAL; import static io.trino.tests.product.TestGroups.PROFILE_SPECIFIC_TESTS; import static io.trino.tests.product.TestGroups.STORAGE_FORMATS; +import static io.trino.tests.product.hive.BucketingType.BUCKETED_DEFAULT; import static io.trino.tests.product.hive.BucketingType.BUCKETED_V2; import static io.trino.tests.product.hive.BucketingType.NONE; import static io.trino.tests.product.hive.TestHiveTransactionalTable.CompactionMode.MAJOR; @@ -273,7 +274,14 @@ private void testReadInsertOnly(boolean isPartitioned, BucketingType bucketingTy } } - @Test(groups = {STORAGE_FORMATS, HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, dataProvider = "partitioningAndBucketingTypeDataProvider", timeOut = TEST_TIMEOUT) + @Test(groups = {STORAGE_FORMATS, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) + @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) + public void testReadFullAcidWithOriginalFilesSmoke() + { + testReadFullAcidWithOriginalFiles(true, BUCKETED_DEFAULT); + } + + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, dataProvider = "partitioningAndBucketingTypeDataProvider", timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testReadFullAcidWithOriginalFiles(boolean isPartitioned, BucketingType bucketingType) { @@ -317,7 +325,7 @@ public void testReadFullAcidWithOriginalFiles(boolean isPartitioned, BucketingTy } } - @Test(groups = {STORAGE_FORMATS, HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, dataProvider = "partitioningAndBucketingTypeDataProvider", timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, dataProvider = "partitioningAndBucketingTypeDataProvider", timeOut = TEST_TIMEOUT) @Flaky(issue = RETRYABLE_FAILURES_ISSUES, match = RETRYABLE_FAILURES_MATCH) public void testUpdateFullAcidWithOriginalFilesTrinoInserting(boolean isPartitioned, BucketingType bucketingType) { @@ -367,7 +375,14 @@ public void testUpdateFullAcidWithOriginalFilesTrinoInserting(boolean isPartitio }); } - @Test(groups = {STORAGE_FORMATS, HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, dataProvider = "partitioningAndBucketingTypeDataProvider", timeOut = TEST_TIMEOUT) + @Test(groups = {STORAGE_FORMATS, PROFILE_SPECIFIC_TESTS}, timeOut = TEST_TIMEOUT) + @Flaky(issue = RETRYABLE_FAILURES_ISSUES, match = RETRYABLE_FAILURES_MATCH) + public void testUpdateFullAcidWithOriginalFilesTrinoInsertingAndDeletingSmoke() + { + testUpdateFullAcidWithOriginalFilesTrinoInsertingAndDeleting(true, BUCKETED_DEFAULT); + } + + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, dataProvider = "partitioningAndBucketingTypeDataProvider", timeOut = TEST_TIMEOUT) @Flaky(issue = RETRYABLE_FAILURES_ISSUES, match = RETRYABLE_FAILURES_MATCH) public void testUpdateFullAcidWithOriginalFilesTrinoInsertingAndDeleting(boolean isPartitioned, BucketingType bucketingType) { @@ -419,7 +434,7 @@ String makeValues(int colStart, int colCount, int fcol, boolean isPartitioned, i .collect(Collectors.joining(", ")); } - @Test(groups = {STORAGE_FORMATS, HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, dataProvider = "partitioningAndBucketingTypeDataProvider", timeOut = TEST_TIMEOUT) + @Test(groups = {HIVE_TRANSACTIONAL, PROFILE_SPECIFIC_TESTS}, dataProvider = "partitioningAndBucketingTypeDataProvider", timeOut = TEST_TIMEOUT) @Flaky(issue = ACID_CORRUPTION_DIRECTORY_ISSUE, match = ACID_CORRUPTION_DIRECTORY_RETRY_PATTERN) public void testReadInsertOnlyWithOriginalFiles(boolean isPartitioned, BucketingType bucketingType) { From 292ce5f2d0ccd6f63689ddd33b3d8fb518656a04 Mon Sep 17 00:00:00 2001 From: Manfred Moser Date: Wed, 10 Jan 2024 10:36:08 -0800 Subject: [PATCH 326/350] Correct FTE encryption detail in docs --- docs/src/main/sphinx/admin/fault-tolerant-execution.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/src/main/sphinx/admin/fault-tolerant-execution.md b/docs/src/main/sphinx/admin/fault-tolerant-execution.md index a1624221440fc..ab085a86d79b0 100644 --- a/docs/src/main/sphinx/admin/fault-tolerant-execution.md +++ b/docs/src/main/sphinx/admin/fault-tolerant-execution.md @@ -152,10 +152,10 @@ cluster that handles short queries. (fte-encryption)= ## Encryption -Trino encrypts data before spooling it to storage. This prevents access to query data -by anyone besides the Trino cluster that wrote it, including administrators of the -storage system. A new encryption key is randomly generated for every query, and keys -are discarded once a query is completed. +Trino encrypts data before spooling it to storage. This prevents access to query +data by anyone besides the Trino cluster that wrote it, including administrators +of the storage system. A new encryption key is randomly generated for every +exchange with every query, and keys are discarded once a query is completed. ## Advanced configuration From 0d23278f1b6793c9b499801525befbd314ec711a Mon Sep 17 00:00:00 2001 From: Manfred Moser Date: Wed, 10 Jan 2024 10:08:10 -0800 Subject: [PATCH 327/350] Clean up OpenSearch connector docs --- docs/src/main/sphinx/connector/opensearch.md | 315 +++++++++++-------- 1 file changed, 190 insertions(+), 125 deletions(-) diff --git a/docs/src/main/sphinx/connector/opensearch.md b/docs/src/main/sphinx/connector/opensearch.md index 4885edcb0a27f..d50a20aad45af 100644 --- a/docs/src/main/sphinx/connector/opensearch.md +++ b/docs/src/main/sphinx/connector/opensearch.md @@ -4,12 +4,14 @@ ``` -The OpenSearch connector allows access to [OpenSearch](https://opensearch.org/) data from Trino. -This document describes how to configure a catalog with the OpenSearch connector to run SQL queries against OpenSearch. +The OpenSearch connector allows access to [OpenSearch](https://opensearch.org/) +data from Trino. This document describes how to configure a catalog with the +OpenSearch connector to run SQL queries against OpenSearch. -:::{note} -OpenSearch (1.1.0 or later) is required. -::: +## Requirements + +- OpenSearch 1.1.0 or higher. +- Network access from the Trino coordinator and workers to the OpenSearch nodes. ## Configuration @@ -19,12 +21,12 @@ properties as appropriate for your setup: ```text connector.name=opensearch -opensearch.host=localhost +opensearch.host=search.example.com opensearch.port=9200 opensearch.default-schema-name=default ``` -### Configuration properties +The following table details all general configuration properties: :::{list-table} OpenSearch configuration properties :widths: 35, 55, 10 @@ -34,8 +36,8 @@ opensearch.default-schema-name=default - Description - Default * - `opensearch.host` - - The comma-separated list of host names for the OpenSearch cluster to connect - to. This property is required. + - The comma-separated list of host names of the OpenSearch cluster. This + property is required. - * - `opensearch.port` - Port to use to connect to OpenSearch. @@ -45,43 +47,92 @@ opensearch.default-schema-name=default name. - `default` * - `opensearch.scroll-size` - - Sets the maximum number of hits that can be returned with each OpenSearch - scroll request. + - Sets the maximum number of hits that can be returned with each [OpenSearch + scroll request](https://opensearch.org/docs/latest/api-reference/scroll/). - `1000` * - `opensearch.scroll-timeout` - - Duration for OpenSearch to keep the [search context](https://opensearch.org/docs/latest/api-reference/scroll/) + - [Duration](prop-type-duration) for OpenSearch to keep the search context alive for scroll requests. - `1m` * - `opensearch.request-timeout` - - Timeout duration for all OpenSearch requests. + - Timeout [duration](prop-type-duration) for all OpenSearch requests. - `10s` * - `opensearch.connect-timeout` - - Timeout duration for all OpenSearch connection attempts. + - Timeout [duration](prop-type-duration) for all OpenSearch connection + attempts. - `1s` * - `opensearch.backoff-init-delay` - - The minimum duration between backpressure retry attempts for a single - request to OpenSearch. Setting it too low can overwhelm an already - struggling cluster. + - The minimum [duration](prop-type-duration) between backpressure retry + attempts for a single request to OpenSearch. Setting it too low can + overwhelm an already struggling cluster. - `500ms` * - `opensearch.backoff-max-delay` - - The maximum duration between backpressure retry attempts for a single - request. + - The maximum [duration](prop-type-duration) between backpressure retry + attempts for a single request. - `20s` * - `opensearch.max-retry-time` - - The maximum duration across all retry attempts for a single request. + - The maximum [duration](prop-type-duration) across all retry attempts for a + single request. - `20s` * - `opensearch.node-refresh-interval` - - Duration between requests to refresh the list of available OpenSearch nodes. + - [Duration](prop-type-duration) between requests to refresh the list of + available OpenSearch nodes. - `1m` * - `opensearch.ignore-publish-address` - - Disables using the address published by OpenSearch to connect for - queries. + - Disable using the address published by the OpenSearch API to connect for + queries. Defaults to `false`. Some deployments map OpenSearch ports to a + random public port and enabling this property can help in these cases. - ::: -### TLS security +### Authentication + +The connection to OpenSearch can use AWS or password authentication. + +To enable AWS authentication and authorization using IAM policies, the +`opensearch.security` option must be set to `AWS`. Additionally, the +following options must be configured: -The connector provides additional security options to connect to OpenSearch clusters with TLS enabled. +:::{list-table} +:widths: 40, 60 +:header-rows: 1 + +* - Property name + - Description +* - `opensearch.aws.region` + - AWS region of the OpenSearch endpoint. This option is required. +* - `opensearch.aws.access-key` + - AWS access key to use to connect to the OpenSearch domain. If not set, the + default AWS credentials provider chain is used. +* - `opensearch.aws.secret-key` + - AWS secret key to use to connect to the OpenSearch domain. If not set, the + default AWS credentials provider chain is used. +* - `opensearch.aws.iam-role` + - Optional ARN of an IAM role to assume to connect to OpenSearch. Note that + the configured IAM user must be able to assume this role. +* - `opensearch.aws.external-id` + - Optional external ID to pass while assuming an AWS IAM role. +::: + +To enable password authentication, the `opensearch.security` option must be set +to `PASSWORD`. Additionally the following options must be configured: + +:::{list-table} +:widths: 45, 55 +:header-rows: 1 + +* - Property name + - Description +* - `opensearch.auth.user` + - User name to use to connect to OpenSearch. +* - `opensearch.auth.password` + - Password to use to connect to OpenSearch. +::: + +### Connection security with TLS + +The connector provides additional security options to connect to OpenSearch +clusters with TLS enabled. If your cluster uses globally-trusted certificates, you only need to enable TLS. If you require custom configuration for certificates, the connector @@ -110,14 +161,14 @@ The available configuration values are listed in the following table: - The password for the trust store specified by `opensearch.tls.truststore-path`. * - `opensearch.tls.verify-hostnames` - - Flag to determine if the hostnames in the certificates must be verified. Defaults - to `true`. + - Flag to determine if the hostnames in the certificates must be verified. + Defaults to `true`. ::: (opensearch-type-mapping)= ## Type mapping -Because Trino and OpenSearch each support types that the other does not, this +Because Trino and OpenSearch each support types that the other does not, the connector [maps some types](type-mapping-overview) when reading data. ### OpenSearch type to Trino type mapping @@ -172,11 +223,15 @@ No other types are supported. (opensearch-array-types)= ### Array types -Fields in OpenSearch can contain [zero or more values](https://opensearch.org/docs/latest/field-types/supported-field-types/date/#custom-formats) -, but there is no dedicated array type. To indicate a field contains an array, it can be annotated in a Trino-specific structure in -the [\_meta](https://opensearch.org/docs/latest/field-types/index/#get-a-mapping) section of the index mapping. +Fields in OpenSearch can contain [zero or more +values](https://opensearch.org/docs/latest/field-types/supported-field-types/date/#custom-formats), +but there is no dedicated array type. To indicate a field contains an array, it +can be annotated in a Trino-specific structure in the +[\_meta](https://opensearch.org/docs/latest/field-types/index/#get-a-mapping) +section of the index mapping in OpenSearch. -For example, you can have an OpenSearch index that contains documents with the following structure: +For example, you can have an OpenSearch index that contains documents with the +following structure: ```json { @@ -191,12 +246,13 @@ For example, you can have an OpenSearch index that contains documents with the f } ``` -The array fields of this structure can be defined by using the following command to add the field -property definition to the `_meta.trino` property of the target index mapping. +The array fields of this structure can be defined by using the following command +to add the field property definition to the `_meta.trino` property of the target +index mapping with OpenSearch available at `search.example.com:9200`: ```shell curl --request PUT \ - --url localhost:9200/doc/_mapping \ + --url search.example.com:9200/doc/_mapping \ --header 'content-type: application/json' \ --data ' { @@ -222,23 +278,23 @@ It is not allowed to use `asRawJson` and `isArray` flags simultaneously for the (opensearch-date-types)= ### Date types -OpenSearch supports a wide array of [date] formats including -[built-in date formats] and also [custom date formats]. The OpenSearch connector supports only the default `date` type. All other -date formats including [built-in date formats] and [custom date formats] are -not supported. Dates with the [format] property are ignored. +OpenSearch [date] formats including [built-in date formats] and [custom date +formats] are not supported. Dates with the [format] property are ignored. ### Raw JSON transform -There are many occurrences where documents in OpenSearch have more complex -structures that are not represented in the mapping. For example, a single -`keyword` field can have widely different content including a single -`keyword` value, an array, or a multidimensional `keyword` array with any -level of nesting. +Documents in OpenSearch can include more complex structures that are not +represented in the mapping. For example, a single `keyword` field can have +widely different content including a single `keyword` value, an array, or a +multidimensional `keyword` array with any level of nesting + +The following command configures `array_string_field` mapping with OpenSearch +available at `search.example.com:9200`: ```shell curl --request PUT \ - --url localhost:9200/doc/_mapping \ + --url search.example.com:9200/doc/_mapping \ --header 'content-type: application/json' \ --data ' { @@ -250,9 +306,8 @@ curl --request PUT \ }' ``` -Notice for the `array_string_field` that all the following documents are legal -for OpenSearch. See the [OpenSearch array documentation](https://opensearch.org/docs/latest/field-types/supported-field-types/index/#arrays) -for more details. +All the following documents are legal for OpenSearch with `array_string_field` +mapping: ```json [ @@ -271,21 +326,26 @@ for more details. ] ``` -Further, OpenSearch supports types, such as -[k-NN vector](https://opensearch.org/docs/latest/field-types/supported-field-types/knn-vector/), -that are not supported in Trino. New types are constantly emerging which can -cause parsing exceptions for users that use of these types in OpenSearch. To -manage all of these scenarios, you can transform fields to raw JSON by -annotating it in a Trino-specific structure in the [\_meta](https://opensearch.org/docs/latest/field-types/index/) -section of the OpenSearch index mapping. This indicates to Trino that the field, and all -nested fields beneath, need to be cast to a `VARCHAR` field that contains -the raw JSON content. These fields can be defined by using the following command -to add the field property definition to the `_meta.trino` property of the -target index mapping. +See the [OpenSearch array +documentation](https://opensearch.org/docs/latest/field-types/supported-field-types/index/#arrays) +for more details. + +Further, OpenSearch supports types, such as [k-NN +vector](https://opensearch.org/docs/latest/field-types/supported-field-types/knn-vector/), +that are not supported in Trino. These and other types can cause parsing +exceptions for users that use of these types in OpenSearch. To manage all of +these scenarios, you can transform fields to raw JSON by annotating it in a +Trino-specific structure in the +[\_meta](https://opensearch.org/docs/latest/field-types/index/) section of the +OpenSearch index mapping. This indicates to Trino that the field, and all nested +fields beneath, must be cast to a `VARCHAR` field that contains the raw JSON +content. These fields can be defined by using the following command to add the +field property definition to the `_meta.trino` property of the target index +mapping. ```shell curl --request PUT \ - --url localhost:9200/doc/_mapping \ + --url search.example.com:9200/doc/_mapping \ --header 'content-type: application/json' \ --data ' { @@ -295,11 +355,11 @@ curl --request PUT \ "asRawJson":true } } - } + } }' ``` -The preceding configurations causes Trino to return the `array_string_field` +The preceding configuration causes Trino to return the `array_string_field` field as a `VARCHAR` containing raw JSON. You can parse these fields with the [built-in JSON functions](/functions/json). @@ -311,60 +371,21 @@ It is not allowed to use `asRawJson` and `isArray` flags simultaneously for the The following hidden columns are available: -| Column | Description | -|----------|-----------------------------------------------------| -| \_id | The OpenSearch document ID | -| \_score | The document score returned by the OpenSearch query | -| \_source | The source of the original document | - -## Predicate push down - -The connector supports [predicate push down](predicate-pushdown) for the following data types: - -| OpenSearch | Trino | Supported | -|--------------|---------------|---------------| -| `binary` | `VARBINARY` | `NO` | -| `boolean` | `BOOLEAN` | `YES` | -| `double` | `DOUBLE` | `YES` | -| `float` | `REAL` | `YES` | -| `byte` | `TINYINT` | `YES` | -| `short` | `SMALLINT` | `YES` | -| `integer` | `INTEGER` | `YES` | -| `long` | `BIGINT` | `YES` | -| `keyword` | `VARCHAR` | `YES` | -| `text` | `VARCHAR` | `NO` | -| `date` | `TIMESTAMP` | `YES` | -| `ip` | `IPADDRESS` | `NO` | -| (all others) | (unsupported) | (unsupported) | - -## Security -The connection to OpenSearch can use AWS or password authentication. - -### AWS authorization - -To enable AWS authorization using IAM policies, the `opensearch.security` option needs to be set to `AWS`. -Additionally, the following options must be configured: - -| Property name | Description | -|------------------------------|-------------------------------------------------------------------------------------------------------------------------------------| -| `opensearch.aws.region` | AWS region of the OpenSearch endpoint. This option is required. | -| `opensearch.aws.access-key` | AWS access key to use to connect to the OpenSearch domain. If not set, the default AWS credentials provider chain is used. | -| `opensearch.aws.secret-key` | AWS secret key to use to connect to the OpenSearch domain. If not set, the default AWS credentials provider chain is used. | -| `opensearch.aws.iam-role` | Optional ARN of an IAM role to assume to connect to OpenSearch. Note that the configured IAM user must be able to assume this role. | -| `opensearch.aws.external-id` | Optional external ID to pass while assuming an AWS IAM role. | - -### Password authentication - -To enable password authentication, the `opensearch.security` option must be set to `PASSWORD`. -Additionally the following options must be configured: +:::{list-table} +:widths: 25, 75 +:header-rows: 1 -| Property name | Description | -|----------------------------|--------------------------------------------| -| `opensearch.auth.user` | User name to use to connect to OpenSearch. | -| `opensearch.auth.password` | Password to use to connect to OpenSearch. | +* - Column + - Description +* - `_id` + - The OpenSearch document ID. +* - `_score` + - The document score returned by the OpenSearch query. +* - `_source` + - The source of the original document. +::: (opensearch-sql-support)= - ## SQL support The connector provides [globally available](sql-globally-available) and @@ -373,18 +394,18 @@ metadata in the OpenSearch catalog. ## Table functions -The connector provides specific {doc}`table functions ` to +The connector provides specific [table functions](/functions/table) to access OpenSearch. (opensearch-raw-query-function)= ### `raw_query(varchar) -> table` -The `raw_query` function allows you to query the underlying database directly. -This function requires [OpenSearch Query DSL](https://opensearch.org/docs/latest/query-dsl/index/) -syntax, because the full query is pushed down and processed in OpenSearch. -This can be useful for accessing native features which are not available in -Trino, or for improving query performance in situations where running a query -natively may be faster. +The `raw_query` function allows you to query the underlying database directly +using the [OpenSearch Query +DSL](https://opensearch.org/docs/latest/query-dsl/index/) syntax. The full DSL +query is pushed down and processed in OpenSearch. This can be useful for +accessing native features which are not available in Trino, or for improving +query performance in situations where running a query natively may be faster. ```{include} query-passthrough-warning.fragment ``` @@ -393,16 +414,17 @@ The `raw_query` function requires three parameters: - `schema`: The schema in the catalog that the query is to be executed on. - `index`: The index in OpenSearch to search. -- `query`: The query to execute, written in OpenSearch [Query DSL](https://opensearch.org/docs/latest/query-dsl). +- `query`: The query to execute, written in [OpenSearch Query DSL](https://opensearch.org/docs/latest/query-dsl). Once executed, the query returns a single row containing the resulting JSON payload returned by OpenSearch. -For example, query the `example` catalog and use the `raw_query` table -function to search for documents in the `orders` index where the country name -is `ALGERIA`: +For example, query the `example` catalog and use the `raw_query` table function +to search for documents in the `orders` index where the country name is +`ALGERIA` as defined as a JSON-formatted query matcher and passed to the +`raw_query` table function in the `query` parameter: -``` +```sql SELECT * FROM @@ -424,6 +446,49 @@ FROM ```{include} query-table-function-ordering.fragment ``` +## Performance + +The connector includes a number of performance improvements, detailed in the +following sections. + +### Parallel data access + +The connector requests data from multiple nodes of the OpenSearch cluster for +query processing in parallel. + +### Predicate push down + +The connector supports [predicate push down](predicate-pushdown) for the +following data types: + +:::{list-table} +:widths: 50, 50 +:header-rows: 1 + +* - OpenSearch + - Trino +* - `boolean` + - `BOOLEAN` +* - `double` + - `DOUBLE` +* - `float` + - `REAL` +* - `byte` + - `TINYINT` +* - `short` + - `SMALLINT` +* - `integer` + - `INTEGER` +* - `long` + - `BIGINT` +* - `keyword` + - `VARCHAR` +* - `date` + - `TIMESTAMP` +::: + +No other data types are supported for predicate push down. + [built-in date formats]: https://opensearch.org/docs/latest/field-types/supported-field-types/date/#custom-formats [custom date formats]: https://opensearch.org/docs/latest/field-types/supported-field-types/date/#custom-formats [date]: https://opensearch.org/docs/latest/field-types/supported-field-types/date/ From c57e3ebd3cf2d9440c9d608066cb8c32ff8925b2 Mon Sep 17 00:00:00 2001 From: Jaeho Yoo Date: Wed, 10 Jan 2024 17:26:13 +0900 Subject: [PATCH 328/350] Clean up Query Management doc --- .../admin/properties-query-management.md | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/docs/src/main/sphinx/admin/properties-query-management.md b/docs/src/main/sphinx/admin/properties-query-management.md index 9ca758e13e71b..83612aa93aeae 100644 --- a/docs/src/main/sphinx/admin/properties-query-management.md +++ b/docs/src/main/sphinx/admin/properties-query-management.md @@ -14,7 +14,7 @@ application, such as the CLI, before it abandons and cancels its work. - **Default value:** `phased` - **Session property:** `execution_policy` -Configures the algorithm to organize the processing of all of the +Configures the algorithm to organize the processing of all the stages of a query. You can use the following execution policies: - `phased` schedules stages in a sequence to avoid blockages because of @@ -69,7 +69,7 @@ such as joins, aggregations, partitioned window functions and others. The maximum number of tasks that will take part in writing data during `INSERT`, `CREATE TABLE AS SELECT` and `EXECUTE` queries. -The limit is only applicable when `redistribute-writes` or `scale-writers` is be enabled. +The limit is only applicable when `redistribute-writes` or `scale-writers` is enabled. ## `query.low-memory-killer.policy` @@ -97,11 +97,11 @@ Configures the behavior to handle killing running tasks in the event of low memory availability. Supports the following values: - `none` - Do not kill any tasks in the event of low memory. -- `total-reservation-on-blocked-nodes` - Kill the tasks which are part of the queries - which has task retries enabled and are currently using the most memory specifically +- `total-reservation-on-blocked-nodes` - Kill the tasks that are part of the queries + which have task retries enabled and are currently using the most memory specifically on nodes that are now out of memory. -- `least-waste` - Kill the tasks which are part of the queries - which has task retries enabled and use significant amount of memory on nodes +- `least-waste` - Kill the tasks that are part of the queries + which have task retries enabled and use significant amount of memory on nodes which are now out of memory. This policy avoids killing tasks which are already executing for a long time, so significant amount of work is not wasted. @@ -116,7 +116,7 @@ Only applies for queries with task level retries enabled (`retry-policy=TASK`) The amount of time a query is allowed to recover between running out of memory and being killed, if `query.low-memory-killer.policy` or -`task.low-memory-killer.policy` is set to value differnt than `none`. +`task.low-memory-killer.policy` is set to value different from `none`. ## `query.max-execution-time` @@ -156,7 +156,7 @@ and may not terminate immediately. The maximum allowed time for a query to be processed on the cluster, before it is terminated. The time includes time for analysis and planning, but also -time spend in a queue waiting, so essentially this is the time allowed for a +time spent in a queue waiting, so essentially this is the time allowed for a query to exist since creation. ## `query.max-scan-physical-bytes` @@ -179,7 +179,7 @@ generates more stages than this it will get killed with error `QUERY_HAS_TOO_MANY_STAGES`. :::{warning} -Setting this to a high value can cause queries with large number of +Setting this to a high value can cause queries with a large number of stages to introduce instability in the cluster causing unrelated queries to get killed with `REMOTE_TASK_ERROR` and the message `Max requests queued per destination exceeded for HttpDestination ...` From d53598809c6be52cd14f9f75474ba594ed731804 Mon Sep 17 00:00:00 2001 From: Manfred Moser Date: Wed, 10 Jan 2024 15:46:59 -0800 Subject: [PATCH 329/350] Add note about Nessie catalogs for views --- docs/src/main/sphinx/connector/metastores.md | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/docs/src/main/sphinx/connector/metastores.md b/docs/src/main/sphinx/connector/metastores.md index 950e2248055cd..2fc7612e3ba48 100644 --- a/docs/src/main/sphinx/connector/metastores.md +++ b/docs/src/main/sphinx/connector/metastores.md @@ -353,8 +353,8 @@ iceberg.catalog.type=rest iceberg.rest-catalog.uri=http://iceberg-with-rest:8181 ``` -The REST catalog does not support {doc}`views` or -{doc}`materialized views`. +The REST catalog does not support [view management](sql-view-management) or +[materialized view management](sql-materialized-view-management). (iceberg-jdbc-catalog)= @@ -390,8 +390,8 @@ iceberg.jdbc-catalog.connection-password=test iceberg.jdbc-catalog.default-warehouse-dir=s3://bucket ``` -The JDBC catalog does not support {doc}`views` or -{doc}`materialized views`. +The JDBC catalog does not support [view management](sql-view-management) or +[materialized view management](sql-materialized-view-management). (iceberg-nessie-catalog)= @@ -439,6 +439,9 @@ iceberg.nessie-catalog.uri=https://localhost:19120/api/v1 iceberg.nessie-catalog.default-warehouse-dir=/tmp ``` +The Nessie catalog does not support [view management](sql-view-management) or +[materialized view management](sql-materialized-view-management). + (partition-projection)= ## Access tables with Athena partition projection metadata From 289cbca7a9eba45b906895868e2e7c31237cad5f Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 10 Jan 2024 21:16:34 +0100 Subject: [PATCH 330/350] Fix map assertion in TestLazyMap `org.testng.Assert.assertEquals(Map, Map)` does not really check map equality when the actual map contains null values. Migrate to AssertJ and fix the test. --- .../test/java/io/trino/plugin/hive/util/TestLazyMap.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/util/TestLazyMap.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/util/TestLazyMap.java index 6fbe9fffc81ec..34bf2bd0bf20d 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/util/TestLazyMap.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/util/TestLazyMap.java @@ -28,7 +28,7 @@ import static org.apache.hadoop.hive.serde2.lazy.LazyFactory.createLazyObject; import static org.apache.hadoop.hive.serde2.lazy.objectinspector.LazyObjectInspectorFactory.getLazySimpleMapObjectInspector; import static org.apache.hadoop.hive.serde2.lazy.objectinspector.primitive.LazyPrimitiveObjectInspectorFactory.getLazyStringObjectInspector; -import static org.testng.Assert.assertEquals; +import static org.assertj.core.api.Assertions.assertThat; public class TestLazyMap { @@ -45,7 +45,7 @@ public void test() assertMapDecode("\\N\u0003ignored\u0002\u0003", ImmutableMap.of(lazyString(""), lazyString(""))); HashMap expectedMap = new HashMap<>(); - expectedMap.put("null", null); + expectedMap.put(lazyString("null"), null); assertMapDecode("\\N\u0003ignored\u0002null\u0003\\N", expectedMap); } @@ -63,7 +63,7 @@ public static void assertMapDecode(String encodedMap, Map map = lazyMap.getMap(); - assertEquals(map, expectedMap); + assertThat(map).isEqualTo(expectedMap); } private static LazyString lazyString(String string) From 2c0ae04bed38deb29ae4658e57d86ecf086cce62 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 10 Jan 2024 11:40:42 +0100 Subject: [PATCH 331/350] Remove redundant generics in expression mapping --- .../JdbcConnectorExpressionRewriterBuilder.java | 17 ++++++----------- 1 file changed, 6 insertions(+), 11 deletions(-) diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/expression/JdbcConnectorExpressionRewriterBuilder.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/expression/JdbcConnectorExpressionRewriterBuilder.java index 8256ff3fa077b..e4a726a3e71c1 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/expression/JdbcConnectorExpressionRewriterBuilder.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/expression/JdbcConnectorExpressionRewriterBuilder.java @@ -64,16 +64,11 @@ public JdbcConnectorExpressionRewriterBuilder withTypeClass(String typeClass, Se return this; } - public ExpressionMapping map(String expressionPattern) + public ExpressionMapping map(String expressionPattern) { - return new ExpressionMapping<>() - { - @Override - public JdbcConnectorExpressionRewriterBuilder to(String rewritePattern) - { - rules.add(new GenericRewrite(typeClasses, expressionPattern, rewritePattern)); - return JdbcConnectorExpressionRewriterBuilder.this; - } + return rewritePattern -> { + rules.add(new GenericRewrite(typeClasses, expressionPattern, rewritePattern)); + return JdbcConnectorExpressionRewriterBuilder.this; }; } @@ -82,8 +77,8 @@ public ConnectorExpressionRewriter build() return new ConnectorExpressionRewriter<>(rules.build()); } - public interface ExpressionMapping + public interface ExpressionMapping { - Continuation to(String rewritePattern); + JdbcConnectorExpressionRewriterBuilder to(String rewritePattern); } } From 8fb59deb276734161f712bf6997845ed05f6cf59 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 10 Jan 2024 11:43:18 +0100 Subject: [PATCH 332/350] Rename rewrite continuation interface Prepare for adding another --- .../expression/JdbcConnectorExpressionRewriterBuilder.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/expression/JdbcConnectorExpressionRewriterBuilder.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/expression/JdbcConnectorExpressionRewriterBuilder.java index e4a726a3e71c1..3a02adeb4964d 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/expression/JdbcConnectorExpressionRewriterBuilder.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/expression/JdbcConnectorExpressionRewriterBuilder.java @@ -64,7 +64,7 @@ public JdbcConnectorExpressionRewriterBuilder withTypeClass(String typeClass, Se return this; } - public ExpressionMapping map(String expressionPattern) + public ExpectRewriteTarget map(String expressionPattern) { return rewritePattern -> { rules.add(new GenericRewrite(typeClasses, expressionPattern, rewritePattern)); @@ -77,7 +77,7 @@ public ConnectorExpressionRewriter build() return new ConnectorExpressionRewriter<>(rules.build()); } - public interface ExpressionMapping + public interface ExpectRewriteTarget { JdbcConnectorExpressionRewriterBuilder to(String rewritePattern); } From 7d0c5b19c3b362c28d1e8b4d1eed7136db65347a Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 10 Jan 2024 11:52:53 +0100 Subject: [PATCH 333/350] Prevent mutable state leak in expression mapping --- .../expression/JdbcConnectorExpressionRewriterBuilder.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/expression/JdbcConnectorExpressionRewriterBuilder.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/expression/JdbcConnectorExpressionRewriterBuilder.java index 3a02adeb4964d..aee92d1785fd7 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/expression/JdbcConnectorExpressionRewriterBuilder.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/expression/JdbcConnectorExpressionRewriterBuilder.java @@ -13,6 +13,7 @@ */ package io.trino.plugin.jdbc.expression; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import io.trino.plugin.base.expression.ConnectorExpressionRewriter; import io.trino.plugin.base.expression.ConnectorExpressionRule; @@ -67,7 +68,7 @@ public JdbcConnectorExpressionRewriterBuilder withTypeClass(String typeClass, Se public ExpectRewriteTarget map(String expressionPattern) { return rewritePattern -> { - rules.add(new GenericRewrite(typeClasses, expressionPattern, rewritePattern)); + rules.add(new GenericRewrite(ImmutableMap.copyOf(typeClasses), expressionPattern, rewritePattern)); return JdbcConnectorExpressionRewriterBuilder.this; }; } From 2f3453b649af7d4ab808a94c2542ced41315cfcd Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 10 Jan 2024 11:14:45 +0100 Subject: [PATCH 334/350] Support conditional rewrites Add support for conditional rewrites in `JdbcConnectorExpressionRewriterBuilder`. This allows merging `PostgreSqlClient`'s two different rewrite objects into one. --- .../ConnectorExpressionRewriter.java | 3 ++ .../expression/ConnectorExpressionRule.java | 5 ++ .../jdbc/expression/GenericRewrite.java | 12 ++++- ...dbcConnectorExpressionRewriterBuilder.java | 46 +++++++++++++++++-- .../jdbc/expression/TestGenericRewrite.java | 4 +- .../plugin/postgresql/PostgreSqlClient.java | 20 ++++---- 6 files changed, 71 insertions(+), 19 deletions(-) diff --git a/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/expression/ConnectorExpressionRewriter.java b/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/expression/ConnectorExpressionRewriter.java index ce9f18743e43a..472eeb0e98cef 100644 --- a/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/expression/ConnectorExpressionRewriter.java +++ b/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/expression/ConnectorExpressionRewriter.java @@ -87,6 +87,9 @@ private Optional rewrite( ConnectorExpression expression, RewriteContext context) { + if (!rule.isEnabled(context.getSession())) { + return Optional.empty(); + } Capture expressionCapture = newCapture(); Pattern pattern = rule.getPattern().capturedAs(expressionCapture); Iterator matches = pattern.match(expression, context).iterator(); diff --git a/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/expression/ConnectorExpressionRule.java b/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/expression/ConnectorExpressionRule.java index 2dcbb25e6f133..c0ecc31663a5e 100644 --- a/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/expression/ConnectorExpressionRule.java +++ b/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/expression/ConnectorExpressionRule.java @@ -27,6 +27,11 @@ public interface ConnectorExpressionRule { + default boolean isEnabled(ConnectorSession session) + { + return true; + } + Pattern getPattern(); Optional rewrite(ExpressionType expression, Captures captures, RewriteContext context); diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/expression/GenericRewrite.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/expression/GenericRewrite.java index f562ff2f1491b..0917737ad9633 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/expression/GenericRewrite.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/expression/GenericRewrite.java @@ -17,11 +17,13 @@ import io.trino.matching.Captures; import io.trino.plugin.base.expression.ConnectorExpressionRule; import io.trino.plugin.jdbc.QueryParameter; +import io.trino.spi.connector.ConnectorSession; import io.trino.spi.expression.ConnectorExpression; import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.function.Predicate; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -35,16 +37,24 @@ public class GenericRewrite // Matches words in the `rewritePattern` private static final Pattern REWRITE_TOKENS = Pattern.compile("(? condition; private final ExpressionPattern expressionPattern; private final String rewritePattern; - public GenericRewrite(Map> typeClasses, String expressionPattern, String rewritePattern) + public GenericRewrite(Map> typeClasses, Predicate condition, String expressionPattern, String rewritePattern) { + this.condition = requireNonNull(condition, "condition is null"); ExpressionMappingParser parser = new ExpressionMappingParser(typeClasses); this.expressionPattern = parser.createExpressionPattern(expressionPattern); this.rewritePattern = requireNonNull(rewritePattern, "rewritePattern is null"); } + @Override + public boolean isEnabled(ConnectorSession session) + { + return condition.test(session); + } + @Override public io.trino.matching.Pattern getPattern() { diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/expression/JdbcConnectorExpressionRewriterBuilder.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/expression/JdbcConnectorExpressionRewriterBuilder.java index aee92d1785fd7..22bb050358e87 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/expression/JdbcConnectorExpressionRewriterBuilder.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/expression/JdbcConnectorExpressionRewriterBuilder.java @@ -17,11 +17,13 @@ import com.google.common.collect.ImmutableSet; import io.trino.plugin.base.expression.ConnectorExpressionRewriter; import io.trino.plugin.base.expression.ConnectorExpressionRule; +import io.trino.spi.connector.ConnectorSession; import java.util.HashMap; import java.util.Map; import java.util.Set; import java.util.function.Function; +import java.util.function.Predicate; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; @@ -65,12 +67,16 @@ public JdbcConnectorExpressionRewriterBuilder withTypeClass(String typeClass, Se return this; } + public ExpectSourceExpression when(Predicate condition) + { + return new GenericRewriteBuilder() + .when(condition); + } + public ExpectRewriteTarget map(String expressionPattern) { - return rewritePattern -> { - rules.add(new GenericRewrite(ImmutableMap.copyOf(typeClasses), expressionPattern, rewritePattern)); - return JdbcConnectorExpressionRewriterBuilder.this; - }; + return new GenericRewriteBuilder() + .map(expressionPattern); } public ConnectorExpressionRewriter build() @@ -78,8 +84,40 @@ public ConnectorExpressionRewriter build() return new ConnectorExpressionRewriter<>(rules.build()); } + public interface ExpectSourceExpression + { + ExpectRewriteTarget map(String expressionPattern); + } + public interface ExpectRewriteTarget { JdbcConnectorExpressionRewriterBuilder to(String rewritePattern); } + + private class GenericRewriteBuilder + implements ExpectSourceExpression, ExpectRewriteTarget + { + private Predicate condition = session -> true; + private String expressionPattern; + + GenericRewriteBuilder when(Predicate condition) + { + this.condition = requireNonNull(condition, "condition is null"); + return this; + } + + @Override + public ExpectRewriteTarget map(String expressionPattern) + { + this.expressionPattern = requireNonNull(expressionPattern, "expressionPattern is null"); + return this; + } + + @Override + public JdbcConnectorExpressionRewriterBuilder to(String rewritePattern) + { + rules.add(new GenericRewrite(ImmutableMap.copyOf(typeClasses), condition, expressionPattern, rewritePattern)); + return JdbcConnectorExpressionRewriterBuilder.this; + } + } } diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/expression/TestGenericRewrite.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/expression/TestGenericRewrite.java index 2c9653790656b..67f313b3069c9 100644 --- a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/expression/TestGenericRewrite.java +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/expression/TestGenericRewrite.java @@ -41,7 +41,7 @@ public class TestGenericRewrite @Test public void testRewriteCall() { - GenericRewrite rewrite = new GenericRewrite(Map.of(), "add(foo: decimal(p, s), bar: bigint): decimal(rp, rs)", "foo + bar::decimal(rp,rs)"); + GenericRewrite rewrite = new GenericRewrite(Map.of(), session -> true, "add(foo: decimal(p, s), bar: bigint): decimal(rp, rs)", "foo + bar::decimal(rp,rs)"); ConnectorExpression expression = new Call( createDecimalType(21, 2), new FunctionName("add"), @@ -58,7 +58,7 @@ public void testRewriteCall() public void testRewriteCallWithTypeClass() { Map> typeClasses = Map.of("integer_class", Set.of("integer", "bigint")); - GenericRewrite rewrite = new GenericRewrite(typeClasses, "add(foo: integer_class, bar: bigint): integer_class", "foo + bar"); + GenericRewrite rewrite = new GenericRewrite(typeClasses, session -> true, "add(foo: integer_class, bar: bigint): integer_class", "foo + bar"); assertThat(apply(rewrite, new Call( BIGINT, diff --git a/plugin/trino-postgresql/src/main/java/io/trino/plugin/postgresql/PostgreSqlClient.java b/plugin/trino-postgresql/src/main/java/io/trino/plugin/postgresql/PostgreSqlClient.java index 28ddf9fd6c561..5be46d0357166 100644 --- a/plugin/trino-postgresql/src/main/java/io/trino/plugin/postgresql/PostgreSqlClient.java +++ b/plugin/trino-postgresql/src/main/java/io/trino/plugin/postgresql/PostgreSqlClient.java @@ -134,6 +134,7 @@ import java.util.OptionalLong; import java.util.UUID; import java.util.function.BiFunction; +import java.util.function.Predicate; import java.util.stream.Stream; import static com.google.common.base.Preconditions.checkArgument; @@ -275,7 +276,6 @@ public class PostgreSqlClient private final List tableTypes; private final boolean statisticsEnabled; private final ConnectorExpressionRewriter connectorExpressionRewriter; - private final ConnectorExpressionRewriter connectorExpressionRewriterWithCollate; private final AggregateFunctionRewriter aggregateFunctionRewriter; @Inject @@ -303,7 +303,8 @@ public PostgreSqlClient( this.statisticsEnabled = statisticsConfig.isEnabled(); - JdbcConnectorExpressionRewriterBuilder connectorExpressionRewriterBuilder = JdbcConnectorExpressionRewriterBuilder.newBuilder() + Predicate pushdownWithCollateEnabled = PostgreSqlSessionProperties::isEnableStringPushdownWithCollate; + this.connectorExpressionRewriter = JdbcConnectorExpressionRewriterBuilder.newBuilder() .addStandardRules(this::quoted) .add(new RewriteIn()) .withTypeClass("integer_type", ImmutableSet.of("tinyint", "smallint", "integer", "bigint")) @@ -326,14 +327,12 @@ public PostgreSqlClient( .map("$not($is_null(value))").to("value IS NOT NULL") .map("$not(value: boolean)").to("NOT value") .map("$is_null(value)").to("value IS NULL") - .map("$nullif(first, second)").to("NULLIF(first, second)"); - this.connectorExpressionRewriter = connectorExpressionRewriterBuilder.build(); - this.connectorExpressionRewriterWithCollate = connectorExpressionRewriterBuilder + .map("$nullif(first, second)").to("NULLIF(first, second)") .withTypeClass("collatable_type", ImmutableSet.of("char", "varchar")) - .map("$less_than(left: collatable_type, right: collatable_type)").to("left < right COLLATE \"C\"") - .map("$less_than_or_equal(left: collatable_type, right: collatable_type)").to("left <= right COLLATE \"C\"") - .map("$greater_than(left: collatable_type, right: collatable_type)").to("left > right COLLATE \"C\"") - .map("$greater_than_or_equal(left: collatable_type, right: collatable_type)").to("left >= right COLLATE \"C\"") + .when(pushdownWithCollateEnabled).map("$less_than(left: collatable_type, right: collatable_type)").to("left < right COLLATE \"C\"") + .when(pushdownWithCollateEnabled).map("$less_than_or_equal(left: collatable_type, right: collatable_type)").to("left <= right COLLATE \"C\"") + .when(pushdownWithCollateEnabled).map("$greater_than(left: collatable_type, right: collatable_type)").to("left > right COLLATE \"C\"") + .when(pushdownWithCollateEnabled).map("$greater_than_or_equal(left: collatable_type, right: collatable_type)").to("left >= right COLLATE \"C\"") .build(); JdbcTypeHandle bigintTypeHandle = new JdbcTypeHandle(Types.BIGINT, Optional.of("bigint"), Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty()); @@ -798,9 +797,6 @@ public boolean supportsAggregationPushdown(ConnectorSession session, JdbcTableHa @Override public Optional convertPredicate(ConnectorSession session, ConnectorExpression expression, Map assignments) { - if (isEnableStringPushdownWithCollate(session)) { - return connectorExpressionRewriterWithCollate.rewrite(session, expression, assignments); - } return connectorExpressionRewriter.rewrite(session, expression, assignments); } From 87ea350709a25be591402bc63c2dc372a78816e8 Mon Sep 17 00:00:00 2001 From: Star Poon Date: Fri, 15 Dec 2023 07:10:52 +0900 Subject: [PATCH 335/350] Inline function calls in hot path --- .../hive/formats/line/simple/SimpleDeserializer.java | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/line/simple/SimpleDeserializer.java b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/line/simple/SimpleDeserializer.java index 16ef43157d67d..ec1943f29ba78 100644 --- a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/line/simple/SimpleDeserializer.java +++ b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/line/simple/SimpleDeserializer.java @@ -81,7 +81,8 @@ public void deserialize(LineBuffer lineBuffer, PageBuilder builder) throws IOException { builder.declarePosition(); - Slice line = Slices.wrappedBuffer(lineBuffer.getBuffer(), 0, lineBuffer.getLength()); + byte[] buffer = lineBuffer.getBuffer(); + Slice line = Slices.wrappedBuffer(buffer, 0, lineBuffer.getLength()); int offset = 0; int length = line.length(); @@ -90,7 +91,7 @@ public void deserialize(LineBuffer lineBuffer, PageBuilder builder) int elementOffset = offset; int fieldIndex = 0; while (offset < end) { - byte currentByte = line.getByte(offset); + byte currentByte = buffer[offset]; if (currentByte == separator) { decodeElementValueInto(fieldIndex, builder, line, elementOffset, offset - elementOffset); elementOffset = offset + 1; @@ -100,7 +101,7 @@ public void deserialize(LineBuffer lineBuffer, PageBuilder builder) break; } } - else if (isEscapeByte(currentByte)) { + else if (escapeByte != null && currentByte == escapeByte) { // ignore the char after escape_char offset++; } @@ -143,9 +144,4 @@ private boolean isNullSequence(Slice slice, int offset, int length) { return nullSequence.equals(0, nullSequence.length(), slice, offset, length); } - - private boolean isEscapeByte(byte currentByte) - { - return escapeByte != null && currentByte == escapeByte; - } } From e529cd705974695d14715c5bfb722e1e1e6cdc33 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 10 Jan 2024 20:15:37 +0100 Subject: [PATCH 336/350] Revert "Fix auto formatting of throwing blocks" This reverts commit fd048d0b6bf523ac7e67de0c7a45c3a2bad4e640. --- .../io/trino/metadata/FunctionManager.java | 12 +++------- .../io/trino/metadata/MetadataManager.java | 16 ++++--------- .../trino/sql/analyzer/StatementAnalyzer.java | 4 +--- .../dispatcher/TestLocalDispatchQuery.java | 12 +++------- .../metadata/TestGlobalFunctionCatalog.java | 24 +++++-------------- .../sql/planner/TestingPlannerContext.java | 12 +++------- .../ExchangeSourceOutputSelector.java | 4 +--- .../io/trino/spi/predicate/TupleDomain.java | 4 +--- .../plugin/jdbc/DefaultQueryBuilder.java | 4 +--- .../deltalake/DeltaLakeSessionProperties.java | 4 +--- .../trino/plugin/ignite/TestIgniteClient.java | 4 +--- .../plugin/postgresql/PostgreSqlClient.java | 8 ++----- .../postgresql/TestPostgreSqlClient.java | 4 +--- 13 files changed, 28 insertions(+), 84 deletions(-) diff --git a/core/trino-main/src/main/java/io/trino/metadata/FunctionManager.java b/core/trino-main/src/main/java/io/trino/metadata/FunctionManager.java index 9f53356708ad3..9a7c208380ea8 100644 --- a/core/trino-main/src/main/java/io/trino/metadata/FunctionManager.java +++ b/core/trino-main/src/main/java/io/trino/metadata/FunctionManager.java @@ -325,15 +325,9 @@ public static FunctionManager createTestingFunctionManager() { TypeOperators typeOperators = new TypeOperators(); GlobalFunctionCatalog functionCatalog = new GlobalFunctionCatalog( - () -> { - throw new UnsupportedOperationException(); - }, - () -> { - throw new UnsupportedOperationException(); - }, - () -> { - throw new UnsupportedOperationException(); - }); + () -> { throw new UnsupportedOperationException(); }, + () -> { throw new UnsupportedOperationException(); }, + () -> { throw new UnsupportedOperationException(); }); functionCatalog.addFunctions(SystemFunctionBundle.create(new FeaturesConfig(), typeOperators, new BlockTypeOperators(typeOperators), UNKNOWN)); functionCatalog.addFunctions(new InternalFunctionBundle(new LiteralFunction(new InternalBlockEncodingSerde(new BlockEncodingManager(), TESTING_TYPE_MANAGER)))); return new FunctionManager(CatalogServiceProvider.fail(), functionCatalog, LanguageFunctionProvider.DISABLED); diff --git a/core/trino-main/src/main/java/io/trino/metadata/MetadataManager.java b/core/trino-main/src/main/java/io/trino/metadata/MetadataManager.java index 5df172e20cb95..67e5666663a3b 100644 --- a/core/trino-main/src/main/java/io/trino/metadata/MetadataManager.java +++ b/core/trino-main/src/main/java/io/trino/metadata/MetadataManager.java @@ -2044,9 +2044,7 @@ private void verifyProjection(TableHandle table, List proje .map(Variable::getName) .filter(variableName -> !assignedVariables.contains(variableName)) .findAny() - .ifPresent(variableName -> { - throw new IllegalStateException("Unbound variable: " + variableName); - }); + .ifPresent(variableName -> { throw new IllegalStateException("Unbound variable: " + variableName); }); } @Override @@ -2764,15 +2762,9 @@ public MetadataManager build() GlobalFunctionCatalog globalFunctionCatalog = this.globalFunctionCatalog; if (globalFunctionCatalog == null) { globalFunctionCatalog = new GlobalFunctionCatalog( - () -> { - throw new UnsupportedOperationException(); - }, - () -> { - throw new UnsupportedOperationException(); - }, - () -> { - throw new UnsupportedOperationException(); - }); + () -> { throw new UnsupportedOperationException(); }, + () -> { throw new UnsupportedOperationException(); }, + () -> { throw new UnsupportedOperationException(); }); TypeOperators typeOperators = new TypeOperators(); globalFunctionCatalog.addFunctions(SystemFunctionBundle.create(new FeaturesConfig(), typeOperators, new BlockTypeOperators(typeOperators), UNKNOWN)); globalFunctionCatalog.addFunctions(new InternalFunctionBundle(new LiteralFunction(new InternalBlockEncodingSerde(new BlockEncodingManager(), typeManager)))); diff --git a/core/trino-main/src/main/java/io/trino/sql/analyzer/StatementAnalyzer.java b/core/trino-main/src/main/java/io/trino/sql/analyzer/StatementAnalyzer.java index 1560b5e602446..4a935a59777c2 100644 --- a/core/trino-main/src/main/java/io/trino/sql/analyzer/StatementAnalyzer.java +++ b/core/trino-main/src/main/java/io/trino/sql/analyzer/StatementAnalyzer.java @@ -2532,9 +2532,7 @@ private Scope createScopeForView( analysis.unregisterTableForView(); checkViewStaleness(columns, descriptor.getVisibleFields(), name, table) - .ifPresent(explanation -> { - throw semanticException(VIEW_IS_STALE, table, "View '%s' is stale or in invalid state: %s", name, explanation); - }); + .ifPresent(explanation -> { throw semanticException(VIEW_IS_STALE, table, "View '%s' is stale or in invalid state: %s", name, explanation); }); // Derive the type of the view from the stored definition, not from the analysis of the underlying query. // This is needed in case the underlying table(s) changed and the query in the view now produces types that diff --git a/core/trino-main/src/test/java/io/trino/dispatcher/TestLocalDispatchQuery.java b/core/trino-main/src/test/java/io/trino/dispatcher/TestLocalDispatchQuery.java index 6cc0e9e990f3a..3a0b59b5b1955 100644 --- a/core/trino-main/src/test/java/io/trino/dispatcher/TestLocalDispatchQuery.java +++ b/core/trino-main/src/test/java/io/trino/dispatcher/TestLocalDispatchQuery.java @@ -134,15 +134,9 @@ public void testSubmittedForDispatchedQuery() new FunctionManager( new ConnectorCatalogServiceProvider<>("function provider", new NoConnectorServicesProvider(), ConnectorServices::getFunctionProvider), new GlobalFunctionCatalog( - () -> { - throw new UnsupportedOperationException(); - }, - () -> { - throw new UnsupportedOperationException(); - }, - () -> { - throw new UnsupportedOperationException(); - }), + () -> { throw new UnsupportedOperationException(); }, + () -> { throw new UnsupportedOperationException(); }, + () -> { throw new UnsupportedOperationException(); }), LanguageFunctionProvider.DISABLED), new QueryMonitorConfig()); CreateTable createTable = new CreateTable(QualifiedName.of("table"), ImmutableList.of(), FAIL, ImmutableList.of(), Optional.empty()); diff --git a/core/trino-main/src/test/java/io/trino/metadata/TestGlobalFunctionCatalog.java b/core/trino-main/src/test/java/io/trino/metadata/TestGlobalFunctionCatalog.java index 28746fffbd70b..2c6a42d9e975c 100644 --- a/core/trino-main/src/test/java/io/trino/metadata/TestGlobalFunctionCatalog.java +++ b/core/trino-main/src/test/java/io/trino/metadata/TestGlobalFunctionCatalog.java @@ -101,15 +101,9 @@ public void testDuplicateFunctions() TypeOperators typeOperators = new TypeOperators(); GlobalFunctionCatalog globalFunctionCatalog = new GlobalFunctionCatalog( - () -> { - throw new UnsupportedOperationException(); - }, - () -> { - throw new UnsupportedOperationException(); - }, - () -> { - throw new UnsupportedOperationException(); - }); + () -> { throw new UnsupportedOperationException(); }, + () -> { throw new UnsupportedOperationException(); }, + () -> { throw new UnsupportedOperationException(); }); globalFunctionCatalog.addFunctions(SystemFunctionBundle.create(new FeaturesConfig(), typeOperators, new BlockTypeOperators(typeOperators), NodeVersion.UNKNOWN)); globalFunctionCatalog.addFunctions(functionBundle); assertThatThrownBy(() -> globalFunctionCatalog.addFunctions(functionBundle)) @@ -124,15 +118,9 @@ public void testConflictingScalarAggregation() TypeOperators typeOperators = new TypeOperators(); GlobalFunctionCatalog globalFunctionCatalog = new GlobalFunctionCatalog( - () -> { - throw new UnsupportedOperationException(); - }, - () -> { - throw new UnsupportedOperationException(); - }, - () -> { - throw new UnsupportedOperationException(); - }); + () -> { throw new UnsupportedOperationException(); }, + () -> { throw new UnsupportedOperationException(); }, + () -> { throw new UnsupportedOperationException(); }); globalFunctionCatalog.addFunctions(SystemFunctionBundle.create(new FeaturesConfig(), typeOperators, new BlockTypeOperators(typeOperators), NodeVersion.UNKNOWN)); assertThatThrownBy(() -> globalFunctionCatalog.addFunctions(functions)) .isInstanceOf(IllegalStateException.class) diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/TestingPlannerContext.java b/core/trino-main/src/test/java/io/trino/sql/planner/TestingPlannerContext.java index 1b3ce8e8c4b43..f2e8d9647efd6 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/TestingPlannerContext.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/TestingPlannerContext.java @@ -120,15 +120,9 @@ public PlannerContext build() parametricTypes.forEach(typeRegistry::addParametricType); GlobalFunctionCatalog globalFunctionCatalog = new GlobalFunctionCatalog( - () -> { - throw new UnsupportedOperationException(); - }, - () -> { - throw new UnsupportedOperationException(); - }, - () -> { - throw new UnsupportedOperationException(); - }); + () -> { throw new UnsupportedOperationException(); }, + () -> { throw new UnsupportedOperationException(); }, + () -> { throw new UnsupportedOperationException(); }); globalFunctionCatalog.addFunctions(SystemFunctionBundle.create(featuresConfig, typeOperators, new BlockTypeOperators(typeOperators), UNKNOWN)); functionBundles.forEach(globalFunctionCatalog::addFunctions); diff --git a/core/trino-spi/src/main/java/io/trino/spi/exchange/ExchangeSourceOutputSelector.java b/core/trino-spi/src/main/java/io/trino/spi/exchange/ExchangeSourceOutputSelector.java index 99c023ecc03c0..14ee2c3870a0e 100644 --- a/core/trino-spi/src/main/java/io/trino/spi/exchange/ExchangeSourceOutputSelector.java +++ b/core/trino-spi/src/main/java/io/trino/spi/exchange/ExchangeSourceOutputSelector.java @@ -173,9 +173,7 @@ public String toString() .collect(toMap( Entry::getKey, Entry::getValue, - (a, b) -> { - throw new IllegalArgumentException("got duplicate key " + a + ", " + b); - }, + (a, b) -> { throw new IllegalArgumentException("got duplicate key " + a + ", " + b); }, TreeMap::new))) .add("finalSelector=" + finalSelector) .toString(); diff --git a/core/trino-spi/src/main/java/io/trino/spi/predicate/TupleDomain.java b/core/trino-spi/src/main/java/io/trino/spi/predicate/TupleDomain.java index 7a8786868ac6a..7be8cb97bb5d0 100644 --- a/core/trino-spi/src/main/java/io/trino/spi/predicate/TupleDomain.java +++ b/core/trino-spi/src/main/java/io/trino/spi/predicate/TupleDomain.java @@ -640,9 +640,7 @@ public Domain getDomain() return toMap( keyMapper, valueMapper, - (u, v) -> { - throw new IllegalStateException(format("Duplicate values for a key: %s and %s", u, v)); - }, + (u, v) -> { throw new IllegalStateException(format("Duplicate values for a key: %s and %s", u, v)); }, LinkedHashMap::new); } diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/DefaultQueryBuilder.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/DefaultQueryBuilder.java index 202f8c5a8ab24..bd06719f9ce1d 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/DefaultQueryBuilder.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/DefaultQueryBuilder.java @@ -86,9 +86,7 @@ public PreparedQuery prepareSelectQuery( .filter(domains::containsKey) .filter(column -> columnExpressions.containsKey(column.getColumnName())) .findFirst() - .ifPresent(column -> { - throw new IllegalArgumentException(format("Column %s has an expression and a constraint attached at the same time", column)); - }); + .ifPresent(column -> { throw new IllegalArgumentException(format("Column %s has an expression and a constraint attached at the same time", column)); }); } ImmutableList.Builder conjuncts = ImmutableList.builder(); diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSessionProperties.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSessionProperties.java index 74697fc72bfdb..065ac2e187fad 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSessionProperties.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSessionProperties.java @@ -167,9 +167,7 @@ public DeltaLakeSessionProperties( "Internal Delta Lake connector property", HiveTimestampPrecision.class, MILLISECONDS, - value -> { - throw new IllegalStateException("The property cannot be set"); - }, + value -> { throw new IllegalStateException("The property cannot be set"); }, true), durationProperty( DYNAMIC_FILTERING_WAIT_TIMEOUT, diff --git a/plugin/trino-ignite/src/test/java/io/trino/plugin/ignite/TestIgniteClient.java b/plugin/trino-ignite/src/test/java/io/trino/plugin/ignite/TestIgniteClient.java index ccb88464f350f..955d0519177d1 100644 --- a/plugin/trino-ignite/src/test/java/io/trino/plugin/ignite/TestIgniteClient.java +++ b/plugin/trino-ignite/src/test/java/io/trino/plugin/ignite/TestIgniteClient.java @@ -79,9 +79,7 @@ public class TestIgniteClient public static final JdbcClient JDBC_CLIENT = new IgniteClient( new BaseJdbcConfig(), - session -> { - throw new UnsupportedOperationException(); - }, + session -> { throw new UnsupportedOperationException(); }, new DefaultQueryBuilder(RemoteQueryModifier.NONE), new DefaultIdentifierMapping(), RemoteQueryModifier.NONE); diff --git a/plugin/trino-postgresql/src/main/java/io/trino/plugin/postgresql/PostgreSqlClient.java b/plugin/trino-postgresql/src/main/java/io/trino/plugin/postgresql/PostgreSqlClient.java index 5be46d0357166..20482f0cfeb99 100644 --- a/plugin/trino-postgresql/src/main/java/io/trino/plugin/postgresql/PostgreSqlClient.java +++ b/plugin/trino-postgresql/src/main/java/io/trino/plugin/postgresql/PostgreSqlClient.java @@ -1426,9 +1426,7 @@ private ColumnMapping arrayAsJsonColumnMapping(ConnectorSession session, ColumnM return ColumnMapping.sliceMapping( jsonType, arrayAsJsonReadFunction(session, baseElementMapping), - (statement, index, block) -> { - throw new UnsupportedOperationException(); - }, + (statement, index, block) -> { throw new UnsupportedOperationException(); }, DISABLE_PUSHDOWN); } @@ -1556,9 +1554,7 @@ public Slice readSlice(ResultSet resultSet, int columnIndex) return utf8Slice(resultSet.getString(columnIndex)); } }, - (statement, index, value) -> { - throw new TrinoException(NOT_SUPPORTED, "Money type is not supported for INSERT"); - }, + (statement, index, value) -> { throw new TrinoException(NOT_SUPPORTED, "Money type is not supported for INSERT"); }, DISABLE_PUSHDOWN); } diff --git a/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestPostgreSqlClient.java b/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestPostgreSqlClient.java index 139af68684cec..90692c075be72 100644 --- a/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestPostgreSqlClient.java +++ b/plugin/trino-postgresql/src/test/java/io/trino/plugin/postgresql/TestPostgreSqlClient.java @@ -110,9 +110,7 @@ public class TestPostgreSqlClient new BaseJdbcConfig(), new PostgreSqlConfig(), new JdbcStatisticsConfig(), - session -> { - throw new UnsupportedOperationException(); - }, + session -> { throw new UnsupportedOperationException(); }, new DefaultQueryBuilder(RemoteQueryModifier.NONE), TESTING_TYPE_MANAGER, new DefaultIdentifierMapping(), From 5df1a2b237face731df0f614f94dd2c8745efb8e Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 10 Jan 2024 13:17:00 +0100 Subject: [PATCH 337/350] Fix TestTrinoGlueCatalog.testListTables Make it use unique namespace names and add cleanup. --- .../iceberg/catalog/BaseTrinoCatalogTest.java | 74 ++++++++++--------- 1 file changed, 41 insertions(+), 33 deletions(-) diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/BaseTrinoCatalogTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/BaseTrinoCatalogTest.java index 69830786be64e..8ed61c8af764a 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/BaseTrinoCatalogTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/BaseTrinoCatalogTest.java @@ -29,6 +29,7 @@ import io.trino.spi.security.PrincipalType; import io.trino.spi.security.TrinoPrincipal; import io.trino.spi.type.VarcharType; +import io.trino.util.AutoCloseableCloser; import org.apache.iceberg.NullOrder; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; @@ -391,39 +392,46 @@ public void testListTables() { TrinoCatalog catalog = createTrinoCatalog(false); TrinoPrincipal principal = new TrinoPrincipal(PrincipalType.USER, SESSION.getUser()); - String ns1 = "ns1"; - String ns2 = "ns2"; - - catalog.createNamespace(SESSION, ns1, defaultNamespaceProperties(ns1), principal); - catalog.createNamespace(SESSION, ns2, defaultNamespaceProperties(ns2), principal); - SchemaTableName table1 = new SchemaTableName(ns1, "t1"); - SchemaTableName table2 = new SchemaTableName(ns2, "t2"); - catalog.newCreateTableTransaction( - SESSION, - table1, - new Schema(Types.NestedField.of(1, true, "col1", Types.LongType.get())), - PartitionSpec.unpartitioned(), - SortOrder.unsorted(), - arbitraryTableLocation(catalog, SESSION, table1), - ImmutableMap.of()) - .commitTransaction(); - - catalog.newCreateTableTransaction( - SESSION, - table2, - new Schema(Types.NestedField.of(1, true, "col1", Types.LongType.get())), - PartitionSpec.unpartitioned(), - SortOrder.unsorted(), - arbitraryTableLocation(catalog, SESSION, table2), - ImmutableMap.of()) - .commitTransaction(); - - // No namespace provided, all tables across all namespaces should be returned - assertThat(catalog.listTables(SESSION, Optional.empty())).containsAll(ImmutableList.of(table1, table2)); - // Namespace is provided and exists - assertThat(catalog.listTables(SESSION, Optional.of(ns1))).isEqualTo(ImmutableList.of(table1)); - // Namespace is provided and does not exist - assertThat(catalog.listTables(SESSION, Optional.of("non_existing"))).isEmpty(); + + try (AutoCloseableCloser closer = AutoCloseableCloser.create()) { + String ns1 = "ns1" + randomNameSuffix(); + String ns2 = "ns2" + randomNameSuffix(); + catalog.createNamespace(SESSION, ns1, defaultNamespaceProperties(ns1), principal); + closer.register(() -> catalog.dropNamespace(SESSION, ns1)); + catalog.createNamespace(SESSION, ns2, defaultNamespaceProperties(ns2), principal); + closer.register(() -> catalog.dropNamespace(SESSION, ns2)); + + SchemaTableName table1 = new SchemaTableName(ns1, "t1"); + SchemaTableName table2 = new SchemaTableName(ns2, "t2"); + catalog.newCreateTableTransaction( + SESSION, + table1, + new Schema(Types.NestedField.of(1, true, "col1", Types.LongType.get())), + PartitionSpec.unpartitioned(), + SortOrder.unsorted(), + arbitraryTableLocation(catalog, SESSION, table1), + ImmutableMap.of()) + .commitTransaction(); + closer.register(() -> catalog.dropTable(SESSION, table1)); + + catalog.newCreateTableTransaction( + SESSION, + table2, + new Schema(Types.NestedField.of(1, true, "col1", Types.LongType.get())), + PartitionSpec.unpartitioned(), + SortOrder.unsorted(), + arbitraryTableLocation(catalog, SESSION, table2), + ImmutableMap.of()) + .commitTransaction(); + closer.register(() -> catalog.dropTable(SESSION, table2)); + + // No namespace provided, all tables across all namespaces should be returned + assertThat(catalog.listTables(SESSION, Optional.empty())).containsAll(ImmutableList.of(table1, table2)); + // Namespace is provided and exists + assertThat(catalog.listTables(SESSION, Optional.of(ns1))).isEqualTo(ImmutableList.of(table1)); + // Namespace is provided and does not exist + assertThat(catalog.listTables(SESSION, Optional.of("non_existing"))).isEmpty(); + } } private String arbitraryTableLocation(TrinoCatalog catalog, ConnectorSession session, SchemaTableName schemaTableName) From 3665fe6a557906dbc1ef65992d206103fa539f36 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 10 Jan 2024 21:00:15 +0100 Subject: [PATCH 338/350] Migrate off testng assertions in TestHiveFileFormats --- .../java/io/trino/plugin/hive/TestHiveFileFormats.java | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileFormats.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileFormats.java index 77c855173bce9..35d25d11ac72b 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileFormats.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveFileFormats.java @@ -212,8 +212,8 @@ import static org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory.javaTimestampObjectInspector; import static org.apache.hadoop.io.SequenceFile.CompressionType.BLOCK; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.data.Offset.offset; import static org.joda.time.DateTimeZone.UTC; -import static org.testng.Assert.assertEquals; // Failing on multiple threads because of org.apache.hadoop.hive.ql.io.parquet.write.ParquetRecordWriterWrapper // uses a single record writer across all threads. @@ -222,7 +222,7 @@ public final class TestHiveFileFormats { private static final DateTimeZone HIVE_STORAGE_TIME_ZONE = DateTimeZone.forID("America/Bahia_Banderas"); - private static final double EPSILON = 0.001; + private static final float EPSILON = 0.001f; private static final FileFormatDataSourceStats STATS = new FileFormatDataSourceStats(); private static final ConnectorSession PARQUET_SESSION = getHiveSession(createParquetHiveConfig(false)); @@ -1045,10 +1045,12 @@ private static void checkPageSource(ConnectorPageSource pageSource, List Date: Mon, 18 Sep 2023 20:07:34 +0800 Subject: [PATCH 339/350] Add support iceberg parquet predicate pushdown with column id --- .../iceberg/IcebergPageSourceProvider.java | 5 +++- .../TestIcebergParquetConnectorTest.java | 30 +++++++++++++++++++ 2 files changed, 34 insertions(+), 1 deletion(-) diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSourceProvider.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSourceProvider.java index aceaf809b65d3..c90e13d3df338 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSourceProvider.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSourceProvider.java @@ -197,6 +197,7 @@ import static java.lang.String.format; import static java.util.Locale.ENGLISH; import static java.util.Objects.requireNonNull; +import static java.util.function.Function.identity; import static java.util.function.Predicate.not; import static java.util.stream.Collectors.groupingBy; import static java.util.stream.Collectors.mapping; @@ -1471,12 +1472,14 @@ private static TupleDomain getParquetTupleDomain(Map descriptorsById = descriptorsByPath.values().stream() + .collect(toImmutableMap(descriptor -> descriptor.getPrimitiveType().getId().intValue(), identity())); ImmutableMap.Builder predicate = ImmutableMap.builder(); effectivePredicate.getDomains().orElseThrow().forEach((columnHandle, domain) -> { String baseType = columnHandle.getType().getTypeSignature().getBase(); // skip looking up predicates for complex types as Parquet only stores stats for primitives if (columnHandle.isBaseColumn() && (!baseType.equals(StandardTypes.MAP) && !baseType.equals(StandardTypes.ARRAY) && !baseType.equals(StandardTypes.ROW))) { - ColumnDescriptor descriptor = descriptorsByPath.get(ImmutableList.of(columnHandle.getName())); + ColumnDescriptor descriptor = descriptorsById.get(columnHandle.getId()); if (descriptor != null) { predicate.put(descriptor, domain); } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergParquetConnectorTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergParquetConnectorTest.java index 67fbbf6c65ccc..8af23930e067d 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergParquetConnectorTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergParquetConnectorTest.java @@ -130,6 +130,36 @@ public void testIgnoreParquetStatistics() } } + @Test + public void testPushdownPredicateToParquetAfterColumnRename() + { + try (TestTable table = new TestTable( + getQueryRunner()::execute, + "test_pushdown_predicate_statistics", + "WITH (sorted_by = ARRAY['custkey']) AS TABLE tpch.tiny.customer WITH NO DATA")) { + assertUpdate( + withSmallRowGroups(getSession()), + "INSERT INTO " + table.getName() + " TABLE tpch.tiny.customer", + "VALUES 1500"); + + assertUpdate("ALTER TABLE " + table.getName() + " RENAME COLUMN custkey TO custkey1"); + + DistributedQueryRunner queryRunner = getDistributedQueryRunner(); + MaterializedResultWithQueryId resultWithoutPredicate = queryRunner.executeWithQueryId(getSession(), "TABLE " + table.getName()); + OperatorStats queryStatsWithoutPredicate = getOperatorStats(resultWithoutPredicate.getQueryId()); + assertThat(queryStatsWithoutPredicate.getPhysicalInputPositions()).isGreaterThan(0); + assertThat(resultWithoutPredicate.getResult()).hasSize(1500); + + @Language("SQL") String selectiveQuery = "SELECT * FROM " + table.getName() + " WHERE custkey1 = 100"; + MaterializedResultWithQueryId selectiveQueryResult = queryRunner.executeWithQueryId(getSession(), selectiveQuery); + OperatorStats queryStatsSelectiveQuery = getOperatorStats(selectiveQueryResult.getQueryId()); + assertThat(queryStatsSelectiveQuery.getPhysicalInputPositions()).isGreaterThan(0); + assertThat(queryStatsSelectiveQuery.getPhysicalInputPositions()) + .isLessThan(queryStatsWithoutPredicate.getPhysicalInputPositions()); + assertThat(selectiveQueryResult.getResult()).hasSize(1); + } + } + @Override protected boolean isFileSorted(String path, String sortColumnName) { From da3ce769eb7cb6578717ea0a776eb41495f3a3ea Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Sat, 30 Dec 2023 19:23:29 +0100 Subject: [PATCH 340/350] Upgrade Elasticsearch client to 7.x This removes support for Elasticsearch 6.x and Opensearch 1.x while introducing support and test coverage for 8.x. Both http security and user authentication are enabled by default. --- .../main/sphinx/connector/elasticsearch.md | 2 +- plugin/trino-elasticsearch/pom.xml | 8 +- .../BackpressureRestHighLevelClient.java | 4 +- .../client/ElasticsearchClient.java | 5 +- .../BaseElasticsearchConnectorTest.java | 64 ++++----- .../ElasticsearchExternalQueryRunner.java | 49 ------- .../elasticsearch/ElasticsearchLoader.java | 7 +- .../ElasticsearchNginxProxy.java | 103 --------------- .../ElasticsearchQueryRunner.java | 59 +++++++-- .../elasticsearch/ElasticsearchServer.java | 59 ++++++++- .../TestElasticsearch6ConnectorTest.java | 63 --------- .../TestElasticsearch7ConnectorTest.java | 18 +-- ...a => TestElasticsearch8ConnectorTest.java} | 24 +--- .../TestElasticsearchBackpressure.java | 86 ------------ .../TestPasswordAuthentication.java | 125 ------------------ .../src/test/resources/ca.crt | 20 +++ .../src/test/resources/elasticsearch.yml | 10 ++ .../src/test/resources/server.crt | 21 +++ .../src/test/resources/server.key | 27 ++++ .../src/test/resources/truststore.jks | Bin 0 -> 1238 bytes 20 files changed, 236 insertions(+), 518 deletions(-) delete mode 100644 plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchExternalQueryRunner.java delete mode 100644 plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchNginxProxy.java delete mode 100644 plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearch6ConnectorTest.java rename plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/{TestElasticsearchOpenSearchConnectorTest.java => TestElasticsearch8ConnectorTest.java} (53%) delete mode 100644 plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchBackpressure.java delete mode 100644 plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/TestPasswordAuthentication.java create mode 100644 plugin/trino-elasticsearch/src/test/resources/ca.crt create mode 100644 plugin/trino-elasticsearch/src/test/resources/server.crt create mode 100644 plugin/trino-elasticsearch/src/test/resources/server.key create mode 100644 plugin/trino-elasticsearch/src/test/resources/truststore.jks diff --git a/docs/src/main/sphinx/connector/elasticsearch.md b/docs/src/main/sphinx/connector/elasticsearch.md index b3ae9fd0e032f..2fc3834fd29ed 100644 --- a/docs/src/main/sphinx/connector/elasticsearch.md +++ b/docs/src/main/sphinx/connector/elasticsearch.md @@ -8,7 +8,7 @@ The Elasticsearch Connector allows access to [Elasticsearch](https://www.elastic This document describes how to setup the Elasticsearch Connector to run SQL queries against Elasticsearch. :::{note} -Elasticsearch (6.6.0 or later) or OpenSearch (1.1.0 or later) is required. +Elasticsearch (7.x or 8.x) is required. OpenSearch is supported through the dedicated [OpenSearch](/connector/opensearch) connector. ::: ## Configuration diff --git a/plugin/trino-elasticsearch/pom.xml b/plugin/trino-elasticsearch/pom.xml index c742a69a3f331..9a9b35d7ebb5e 100644 --- a/plugin/trino-elasticsearch/pom.xml +++ b/plugin/trino-elasticsearch/pom.xml @@ -15,7 +15,7 @@ ${project.parent.basedir} - 6.8.23 + 7.17.16 @@ -114,7 +114,7 @@ org.apache.httpcomponents httpasyncclient - 4.1.2 + 4.1.4 @@ -158,6 +158,10 @@ org.apache.logging.log4j log4j-api + + org.apache.logging.log4j + log4j-core + org.apache.lucene diff --git a/plugin/trino-elasticsearch/src/main/java/io/trino/plugin/elasticsearch/client/BackpressureRestHighLevelClient.java b/plugin/trino-elasticsearch/src/main/java/io/trino/plugin/elasticsearch/client/BackpressureRestHighLevelClient.java index 391283144cbec..9ea38bc290192 100644 --- a/plugin/trino-elasticsearch/src/main/java/io/trino/plugin/elasticsearch/client/BackpressureRestHighLevelClient.java +++ b/plugin/trino-elasticsearch/src/main/java/io/trino/plugin/elasticsearch/client/BackpressureRestHighLevelClient.java @@ -33,6 +33,7 @@ import org.elasticsearch.client.RequestOptions; import org.elasticsearch.client.RestClientBuilder; import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.client.RestHighLevelClientBuilder; import org.elasticsearch.rest.RestStatus; import java.io.Closeable; @@ -58,7 +59,8 @@ public class BackpressureRestHighLevelClient public BackpressureRestHighLevelClient(RestClientBuilder restClientBuilder, ElasticsearchConfig config, TimeStat backpressureStats) { this.backpressureStats = requireNonNull(backpressureStats, "backpressureStats is null"); - delegate = new RestHighLevelClient(requireNonNull(restClientBuilder, "restClientBuilder is null")); + delegate = new RestHighLevelClientBuilder(requireNonNull(restClientBuilder, "restClientBuilder is null").build()) + .build(); backpressureRestClient = new BackpressureRestClient(delegate.getLowLevelClient(), config, backpressureStats); retryPolicy = RetryPolicy.builder() .withMaxAttempts(-1) diff --git a/plugin/trino-elasticsearch/src/main/java/io/trino/plugin/elasticsearch/client/ElasticsearchClient.java b/plugin/trino-elasticsearch/src/main/java/io/trino/plugin/elasticsearch/client/ElasticsearchClient.java index 3fcea05054f1a..f696192e0b2c3 100644 --- a/plugin/trino-elasticsearch/src/main/java/io/trino/plugin/elasticsearch/client/ElasticsearchClient.java +++ b/plugin/trino-elasticsearch/src/main/java/io/trino/plugin/elasticsearch/client/ElasticsearchClient.java @@ -62,7 +62,7 @@ import org.elasticsearch.client.ResponseException; import org.elasticsearch.client.RestClient; import org.elasticsearch.client.RestClientBuilder; -import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.weakref.jmx.Managed; @@ -201,8 +201,7 @@ private static BackpressureRestHighLevelClient createClient( RestClientBuilder builder = RestClient.builder( config.getHosts().stream() .map(httpHost -> new HttpHost(httpHost, config.getPort(), config.isTlsEnabled() ? "https" : "http")) - .toArray(HttpHost[]::new)) - .setMaxRetryTimeoutMillis(toIntExact(config.getMaxRetryTime().toMillis())); + .toArray(HttpHost[]::new)); builder.setHttpClientConfigCallback(ignored -> { RequestConfig requestConfig = RequestConfig.custom() diff --git a/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/BaseElasticsearchConnectorTest.java b/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/BaseElasticsearchConnectorTest.java index 60586c95bbf59..07abe0792fcd6 100644 --- a/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/BaseElasticsearchConnectorTest.java +++ b/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/BaseElasticsearchConnectorTest.java @@ -16,7 +16,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.common.net.HostAndPort; import io.trino.Session; import io.trino.spi.type.VarcharType; import io.trino.sql.planner.plan.LimitNode; @@ -26,10 +25,7 @@ import io.trino.testing.QueryRunner; import io.trino.testing.TestingConnectorBehavior; import io.trino.tpch.TpchTable; -import org.apache.http.HttpHost; -import org.apache.http.entity.ContentType; -import org.apache.http.nio.entity.NStringEntity; -import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.Request; import org.elasticsearch.client.RestHighLevelClient; import org.intellij.lang.annotations.Language; import org.junit.jupiter.api.AfterAll; @@ -49,6 +45,7 @@ import static io.trino.testing.TestingNames.randomNameSuffix; import static java.lang.String.format; import static java.nio.charset.StandardCharsets.UTF_8; +import static java.util.Objects.requireNonNull; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; @@ -57,28 +54,23 @@ public abstract class BaseElasticsearchConnectorTest extends BaseConnectorTest { - private final String image; private final String catalogName; - private ElasticsearchServer elasticsearch; - protected RestHighLevelClient client; + private ElasticsearchServer server; + private RestHighLevelClient client; - BaseElasticsearchConnectorTest(String image, String catalogName) + BaseElasticsearchConnectorTest(ElasticsearchServer server, String catalogName) { - this.image = image; + this.server = requireNonNull(server, "server is null"); this.catalogName = catalogName; + this.client = server.getClient(); } @Override protected QueryRunner createQueryRunner() throws Exception { - elasticsearch = new ElasticsearchServer(image, ImmutableMap.of()); - - HostAndPort address = elasticsearch.getAddress(); - client = new RestHighLevelClient(RestClient.builder(new HttpHost(address.getHost(), address.getPort()))); - return createElasticsearchQueryRunner( - elasticsearch.getAddress(), + server, TpchTable.getTables(), ImmutableMap.of(), ImmutableMap.of(), @@ -90,8 +82,8 @@ protected QueryRunner createQueryRunner() public final void destroy() throws IOException { - elasticsearch.stop(); - elasticsearch = null; + server.stop(); + server = null; client.close(); client = null; } @@ -123,7 +115,7 @@ protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior) /** * This method overrides the default values used for the data provider - * of the test {@link AbstractTestQueries#testLargeIn(int)} by taking + * of the test {@link AbstractTestQueries#testLargeIn()} by taking * into account that by default Elasticsearch supports only up to `1024` * clauses in query. *

@@ -1910,53 +1902,63 @@ protected void assertTableDoesNotExist(String name) assertQueryFails("SELECT * FROM " + name, ".*Table '" + catalogName + ".tpch." + name + "' does not exist"); } - protected abstract String indexEndpoint(String index, String docId); + protected String indexEndpoint(String index, String docId) + { + return format("/%s/_doc/%s", index, docId); + } private void index(String index, Map document) throws IOException { String json = new ObjectMapper().writeValueAsString(document); String endpoint = format("%s?refresh", indexEndpoint(index, String.valueOf(System.nanoTime()))); - client.getLowLevelClient() - .performRequest("PUT", endpoint, ImmutableMap.of(), new NStringEntity(json, ContentType.APPLICATION_JSON)); + + Request request = new Request("PUT", endpoint); + request.setJsonEntity(json); + + client.getLowLevelClient().performRequest(request); } private void addAlias(String index, String alias) throws IOException { client.getLowLevelClient() - .performRequest("PUT", format("/%s/_alias/%s", index, alias)); + .performRequest(new Request("PUT", format("/%s/_alias/%s", index, alias))); refreshIndex(alias); } - protected abstract String indexMapping(@Language("JSON") String properties); + protected String indexMapping(@Language("JSON") String properties) + { + return "{\"mappings\": " + properties + "}"; + } private void createIndex(String indexName) throws IOException { - client.getLowLevelClient().performRequest("PUT", "/" + indexName); + client.getLowLevelClient().performRequest(new Request("PUT", "/" + indexName)); } private void createIndex(String indexName, @Language("JSON") String properties) throws IOException { String mappings = indexMapping(properties); - client.getLowLevelClient() - .performRequest("PUT", "/" + indexName, ImmutableMap.of(), new NStringEntity(mappings, ContentType.APPLICATION_JSON)); + + Request request = new Request("PUT", "/" + indexName); + request.setJsonEntity(mappings); + + client.getLowLevelClient().performRequest(request); } private void refreshIndex(String index) throws IOException { - client.getLowLevelClient() - .performRequest("GET", format("/%s/_refresh", index)); + client.getLowLevelClient().performRequest(new Request("GET", format("/%s/_refresh", index))); } private void deleteIndex(String indexName) throws IOException { - client.getLowLevelClient() - .performRequest("DELETE", "/" + indexName); + client.getLowLevelClient().performRequest(new Request("DELETE", "/" + indexName)); } } diff --git a/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchExternalQueryRunner.java b/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchExternalQueryRunner.java deleted file mode 100644 index beef029ed9486..0000000000000 --- a/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchExternalQueryRunner.java +++ /dev/null @@ -1,49 +0,0 @@ -/* - * 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.elasticsearch; - -import com.google.common.collect.ImmutableMap; -import com.google.common.net.HostAndPort; -import io.airlift.log.Logger; -import io.trino.testing.DistributedQueryRunner; -import io.trino.tpch.TpchTable; - -import static io.trino.plugin.elasticsearch.ElasticsearchQueryRunner.createElasticsearchQueryRunner; -import static java.lang.Integer.parseInt; - -public class ElasticsearchExternalQueryRunner -{ - private static final String HOSTNAME = System.getProperty("elasticsearch.host", "localhost"); - private static final int PORT = parseInt(System.getProperty("elasticsearch.port", "9200")); - - private ElasticsearchExternalQueryRunner() {} - - public static void main(String[] args) - throws Exception - { - // Please set hostname and port via VM options. e.g. "-Delasticsearch.host=localhost -Delasticsearch.port=9200" - // To start Elasticsearch: - // docker run -p 9200:9200 -e "discovery.type=single-node" docker.elastic.co/elasticsearch/elasticsearch:7.6.2 - DistributedQueryRunner queryRunner = createElasticsearchQueryRunner( - HostAndPort.fromParts(HOSTNAME, PORT), - TpchTable.getTables(), - ImmutableMap.of("http-server.http.port", "8080"), - ImmutableMap.of(), - 3); - - Logger log = Logger.get(ElasticsearchExternalQueryRunner.class); - log.info("======== SERVER STARTED ========"); - log.info("\n====\n%s\n====", queryRunner.getCoordinator().getBaseUrl()); - } -} diff --git a/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchLoader.java b/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchLoader.java index 35fdeb39f4831..af01e19f78fca 100644 --- a/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchLoader.java +++ b/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchLoader.java @@ -25,8 +25,9 @@ import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.support.WriteRequest; +import org.elasticsearch.client.RequestOptions; import org.elasticsearch.client.RestHighLevelClient; -import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.xcontent.XContentBuilder; import java.io.IOException; import java.io.UncheckedIOException; @@ -42,7 +43,7 @@ import static io.trino.spi.type.DoubleType.DOUBLE; import static io.trino.spi.type.IntegerType.INTEGER; import static java.util.Objects.requireNonNull; -import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; +import static org.elasticsearch.xcontent.XContentFactory.jsonBuilder; public class ElasticsearchLoader extends AbstractTestingTrinoClient @@ -109,7 +110,7 @@ public void addResults(QueryStatusInfo statusInfo, QueryData data) request.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); try { - client.bulk(request); + client.bulk(request, RequestOptions.DEFAULT); } catch (IOException e) { throw new RuntimeException(e); diff --git a/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchNginxProxy.java b/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchNginxProxy.java deleted file mode 100644 index 298edeabaac77..0000000000000 --- a/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchNginxProxy.java +++ /dev/null @@ -1,103 +0,0 @@ -/* - * 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.elasticsearch; - -import com.google.common.net.HostAndPort; -import org.testcontainers.containers.Network; -import org.testcontainers.containers.NginxContainer; - -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; - -import static com.google.common.io.MoreFiles.deleteRecursively; -import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; -import static java.nio.charset.StandardCharsets.UTF_8; -import static org.testcontainers.utility.MountableFile.forHostPath; - -public class ElasticsearchNginxProxy -{ - private static final int PROXY_PORT = 9201; - private static final String NGINX_CONFIG_TEMPLATE = - "limit_req_zone $binary_remote_addr zone=mylimit:1m rate=REQUEST_PER_SECONDr/s;\n" + - "upstream elasticsearch {\n" + - " server elasticsearch-server:9200;\n" + - " keepalive 15;\n" + - "}\n" + - "server {\n" + - " access_log /var/log/nginx/access.log main;" + - " listen " + PROXY_PORT + ";\n" + - " location / {\n" + - " proxy_pass http://elasticsearch;\n" + - " proxy_redirect http://elasticsearch /;\n" + - " proxy_buffering off;\n" + - " proxy_http_version 1.1;\n" + - " proxy_set_header Connection \"Keep-Alive\";\n" + - " proxy_set_header Proxy-Connection \"Keep-Alive\";\n" + - " client_max_body_size 0;\n" + - " }\n" + - " location /_search/scroll {\n" + - " limit_req zone=mylimit;\n" + - " limit_req_status 429;\n" + - " proxy_pass http://elasticsearch;\n" + - " proxy_redirect http://elasticsearch /;\n" + - " proxy_buffering off;\n" + - " proxy_http_version 1.1;\n" + - " proxy_set_header Connection \"Keep-Alive\";\n" + - " proxy_set_header Proxy-Connection \"Keep-Alive\";\n" + - " client_max_body_size 0;\n" + - " }\n" + - " location ~ /.*/_search$ {\n" + - " limit_req zone=mylimit;\n" + - " limit_req_status 429;\n" + - " proxy_pass http://elasticsearch;\n" + - " proxy_redirect http://elasticsearch /;\n" + - " proxy_buffering off;\n" + - " proxy_http_version 1.1;\n" + - " proxy_set_header Connection \"Keep-Alive\";\n" + - " proxy_set_header Proxy-Connection \"Keep-Alive\";\n" + - " client_max_body_size 0;\n" + - " }\n" + - "}\n"; - - private final Path configurationPath; - private final NginxContainer container; - - public ElasticsearchNginxProxy(Network network, int requestsPerSecond) - throws IOException - { - container = new NginxContainer<>("nginx:1.19.8"); - container.withNetwork(network); - container.withNetworkAliases("elasticsearch-proxy"); - // Create the Nginx configuration file on host and copy it into a predefined path the container - configurationPath = Files.createTempDirectory("elasticsearchProxy"); - Path path = configurationPath.resolve("elasticsearch.conf"); - Files.writeString(path, NGINX_CONFIG_TEMPLATE.replace("REQUEST_PER_SECOND", String.valueOf(requestsPerSecond)), UTF_8); - container.withCopyFileToContainer(forHostPath(path), "/etc/nginx/conf.d/elasticsearch.conf"); - container.addExposedPort(PROXY_PORT); - container.start(); - } - - public void stop() - throws IOException - { - container.close(); - deleteRecursively(configurationPath, ALLOW_INSECURE); - } - - public HostAndPort getAddress() - { - return HostAndPort.fromString(container.getHost() + ":" + container.getMappedPort(PROXY_PORT)); - } -} diff --git a/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchQueryRunner.java b/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchQueryRunner.java index 73be8fd317517..e560cff86c180 100644 --- a/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchQueryRunner.java +++ b/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchQueryRunner.java @@ -14,7 +14,6 @@ package io.trino.plugin.elasticsearch; import com.google.common.collect.ImmutableMap; -import com.google.common.net.HostAndPort; import io.airlift.log.Level; import io.airlift.log.Logger; import io.airlift.log.Logging; @@ -25,14 +24,19 @@ import io.trino.testing.QueryRunner; import io.trino.testing.TestingTrinoClient; import io.trino.tpch.TpchTable; -import org.apache.http.HttpHost; -import org.elasticsearch.client.RestClient; import org.elasticsearch.client.RestHighLevelClient; +import javax.net.ssl.SSLContext; + +import java.io.File; +import java.net.URISyntaxException; import java.util.Map; +import java.util.Optional; +import static com.google.common.io.Resources.getResource; import static io.airlift.testing.Closeables.closeAllSuppress; import static io.airlift.units.Duration.nanosSince; +import static io.trino.plugin.base.ssl.SslUtils.createSSLContext; import static io.trino.plugin.elasticsearch.ElasticsearchServer.ELASTICSEARCH_7_IMAGE; import static io.trino.plugin.tpch.TpchMetadata.TINY_SCHEMA_NAME; import static io.trino.testing.TestingSession.testSessionBuilder; @@ -42,6 +46,9 @@ public final class ElasticsearchQueryRunner { + public static final String USER = "elastic_user"; + public static final String PASSWORD = "123456"; + static { Logging logging = Logging.initialize(); logging.setLevel("org.elasticsearch.client.RestClient", Level.OFF); @@ -53,18 +60,18 @@ private ElasticsearchQueryRunner() {} private static final String TPCH_SCHEMA = "tpch"; public static DistributedQueryRunner createElasticsearchQueryRunner( - HostAndPort address, + ElasticsearchServer server, Iterable> tables, Map extraProperties, Map extraConnectorProperties, int nodeCount) throws Exception { - return createElasticsearchQueryRunner(address, tables, extraProperties, extraConnectorProperties, nodeCount, "elasticsearch"); + return createElasticsearchQueryRunner(server, tables, extraProperties, extraConnectorProperties, nodeCount, "elasticsearch"); } public static DistributedQueryRunner createElasticsearchQueryRunner( - HostAndPort address, + ElasticsearchServer server, Iterable> tables, Map extraProperties, Map extraConnectorProperties, @@ -91,13 +98,14 @@ public static DistributedQueryRunner createElasticsearchQueryRunner( ElasticsearchConnectorFactory testFactory = new ElasticsearchConnectorFactory(); - installElasticsearchPlugin(address, queryRunner, catalogName, testFactory, extraConnectorProperties); + installElasticsearchPlugin(server, queryRunner, catalogName, testFactory, extraConnectorProperties); TestingTrinoClient trinoClient = queryRunner.getClient(); LOG.info("Loading data..."); - client = new RestHighLevelClient(RestClient.builder(HttpHost.create(address.toString()))); + client = server.getClient(); + long startTime = System.nanoTime(); for (TpchTable table : tables) { loadTpchTopic(client, trinoClient, table); @@ -112,17 +120,32 @@ public static DistributedQueryRunner createElasticsearchQueryRunner( } } + public static SSLContext getSSLContext() + { + try { + return createSSLContext( + Optional.empty(), + Optional.empty(), + Optional.of(new File(getResource("truststore.jks").toURI())), + Optional.of("123456")); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + private static void installElasticsearchPlugin( - HostAndPort address, + ElasticsearchServer server, QueryRunner queryRunner, String catalogName, ElasticsearchConnectorFactory factory, Map extraConnectorProperties) + throws URISyntaxException { queryRunner.installPlugin(new ElasticsearchPlugin(factory)); - Map config = ImmutableMap.builder() - .put("elasticsearch.host", address.getHost()) - .put("elasticsearch.port", Integer.toString(address.getPort())) + ImmutableMap.Builder builder = ImmutableMap.builder() + .put("elasticsearch.host", server.getAddress().getHost()) + .put("elasticsearch.port", Integer.toString(server.getAddress().getPort())) // Node discovery relies on the publish_address exposed via the Elasticseach API // This doesn't work well within a docker environment that maps ES's port to a random public port .put("elasticsearch.ignore-publish-address", "true") @@ -130,7 +153,15 @@ private static void installElasticsearchPlugin( .put("elasticsearch.scroll-size", "1000") .put("elasticsearch.scroll-timeout", "1m") .put("elasticsearch.request-timeout", "2m") - .putAll(extraConnectorProperties) + .put("elasticsearch.tls.enabled", "true") + .put("elasticsearch.tls.truststore-path", new File(getResource("truststore.jks").toURI()).getPath()) + .put("elasticsearch.tls.truststore-password", "123456") + .put("elasticsearch.tls.verify-hostnames", "false") + .put("elasticsearch.security", "PASSWORD") + .put("elasticsearch.auth.user", USER) + .put("elasticsearch.auth.password", PASSWORD); + + Map config = builder.putAll(extraConnectorProperties) .buildOrThrow(); queryRunner.createCatalog(catalogName, "elasticsearch", config); @@ -149,7 +180,7 @@ public static void main(String[] args) throws Exception { DistributedQueryRunner queryRunner = createElasticsearchQueryRunner( - new ElasticsearchServer(ELASTICSEARCH_7_IMAGE, ImmutableMap.of()).getAddress(), + new ElasticsearchServer(ELASTICSEARCH_7_IMAGE), TpchTable.getTables(), ImmutableMap.of("http-server.http.port", "8080"), ImmutableMap.of(), diff --git a/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchServer.java b/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchServer.java index 8f3da20229d14..5a4423901a872 100644 --- a/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchServer.java +++ b/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/ElasticsearchServer.java @@ -13,46 +13,72 @@ */ package io.trino.plugin.elasticsearch; +import com.amazonaws.util.Base64; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.io.Resources; import com.google.common.net.HostAndPort; import io.trino.testing.ResourcePresence; +import org.apache.http.HttpHost; +import org.apache.http.impl.nio.client.HttpAsyncClientBuilder; +import org.apache.http.message.BasicHeader; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.client.RestHighLevelClientBuilder; import org.testcontainers.containers.Network; import org.testcontainers.elasticsearch.ElasticsearchContainer; import org.testcontainers.utility.DockerImageName; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Path; import java.util.Map; import static com.google.common.io.MoreFiles.deleteRecursively; import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; +import static com.google.common.io.Resources.getResource; +import static io.trino.plugin.elasticsearch.ElasticsearchQueryRunner.PASSWORD; +import static io.trino.plugin.elasticsearch.ElasticsearchQueryRunner.USER; +import static io.trino.plugin.elasticsearch.ElasticsearchQueryRunner.getSSLContext; +import static java.lang.String.format; import static java.nio.charset.StandardCharsets.UTF_8; import static java.nio.file.Files.createTempDirectory; import static org.testcontainers.utility.MountableFile.forHostPath; public class ElasticsearchServer { - public static final String ELASTICSEARCH_7_IMAGE = "elasticsearch:7.0.0"; + public static final String ELASTICSEARCH_7_IMAGE = "elasticsearch:7.16.2"; + public static final String ELASTICSEARCH_8_IMAGE = "elasticsearch:8.11.3"; private final Path configurationPath; private final ElasticsearchContainer container; - public ElasticsearchServer(String image, Map configurationFiles) + public ElasticsearchServer(String image) throws IOException { - this(Network.SHARED, image, configurationFiles); + this(Network.SHARED, image); } - public ElasticsearchServer(Network network, String image, Map configurationFiles) + public ElasticsearchServer(Network network, String image) throws IOException { DockerImageName dockerImageName = DockerImageName.parse(image).asCompatibleSubstituteFor("docker.elastic.co/elasticsearch/elasticsearch"); container = new ElasticsearchContainer(dockerImageName); container.withNetwork(network); container.withNetworkAliases("elasticsearch-server"); - container.withEnv("DISABLE_SECURITY_PLUGIN", "true"); // Required for OpenSearch container configurationPath = createTempDirectory(null); + Map configurationFiles = ImmutableMap.builder() + .put("elasticsearch.yml", loadResource("elasticsearch.yml")) + .put("users", loadResource("users")) + .put("users_roles", loadResource("users_roles")) + .put("roles.yml", loadResource("roles.yml")) + .put("ca.crt", loadResource("ca.crt")) + .put("server.crt", loadResource("server.crt")) + .put("server.key", loadResource("server.key")) + .buildOrThrow(); + for (Map.Entry entry : configurationFiles.entrySet()) { String name = entry.getKey(); String contents = entry.getValue(); @@ -61,7 +87,6 @@ public ElasticsearchServer(Network network, String image, Map co Files.writeString(path, contents, UTF_8); container.withCopyFileToContainer(forHostPath(path), "/usr/share/elasticsearch/config/" + name); } - container.start(); } @@ -82,4 +107,26 @@ public HostAndPort getAddress() { return HostAndPort.fromString(container.getHttpHostAddress()); } + + public RestHighLevelClient getClient() + { + HostAndPort address = getAddress(); + return new RestHighLevelClientBuilder(RestClient.builder(new HttpHost(address.getHost(), address.getPort(), "https")) + .setStrictDeprecationMode(false) + .setHttpClientConfigCallback(ElasticsearchServer::enableSecureCommunication).build()) + .setApiCompatibilityMode(true) // Needed for 7.x client to work with 8.x server + .build(); + } + + private static HttpAsyncClientBuilder enableSecureCommunication(HttpAsyncClientBuilder clientBuilder) + { + return clientBuilder.setSSLContext(getSSLContext()) + .setDefaultHeaders(ImmutableList.of(new BasicHeader("Authorization", format("Basic %s", Base64.encodeAsString(format("%s:%s", USER, PASSWORD).getBytes(StandardCharsets.UTF_8)))))); + } + + private static String loadResource(String file) + throws IOException + { + return Resources.toString(getResource(file), UTF_8); + } } diff --git a/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearch6ConnectorTest.java b/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearch6ConnectorTest.java deleted file mode 100644 index 06b2c74992008..0000000000000 --- a/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearch6ConnectorTest.java +++ /dev/null @@ -1,63 +0,0 @@ -/* - * 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.elasticsearch; - -import com.google.common.collect.ImmutableMap; -import org.apache.http.entity.ContentType; -import org.apache.http.nio.entity.NStringEntity; -import org.intellij.lang.annotations.Language; -import org.junit.jupiter.api.Test; - -import java.io.IOException; - -import static java.lang.String.format; - -public class TestElasticsearch6ConnectorTest - extends BaseElasticsearchConnectorTest -{ - public TestElasticsearch6ConnectorTest() - { - super("docker.elastic.co/elasticsearch/elasticsearch-oss:6.6.0", "elasticsearch6"); - } - - @Test - public void testIndexWithMappingsButNoProperties() - throws IOException - { - String indexName = "test_empty_index_with_mappings_no_properties"; - - @Language("JSON") - String mappings = "{\"mappings\": " + - " {\"foo\": { \"dynamic\" : \"strict\" } }" + - "}"; - client.getLowLevelClient() - .performRequest("PUT", "/" + indexName, ImmutableMap.of(), new NStringEntity(mappings, ContentType.APPLICATION_JSON)); - - assertTableDoesNotExist(indexName); - } - - @Override - protected String indexEndpoint(String index, String docId) - { - return format("/%s/doc/%s", index, docId); - } - - @Override - protected String indexMapping(String properties) - { - return "{\"mappings\": " + - " {\"doc\": " + properties + "}" + - "}"; - } -} diff --git a/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearch7ConnectorTest.java b/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearch7ConnectorTest.java index b48d76c34cc0b..29b76f9b2d33c 100644 --- a/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearch7ConnectorTest.java +++ b/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearch7ConnectorTest.java @@ -13,26 +13,16 @@ */ package io.trino.plugin.elasticsearch; +import java.io.IOException; + import static io.trino.plugin.elasticsearch.ElasticsearchServer.ELASTICSEARCH_7_IMAGE; -import static java.lang.String.format; public class TestElasticsearch7ConnectorTest extends BaseElasticsearchConnectorTest { public TestElasticsearch7ConnectorTest() + throws IOException { - super(ELASTICSEARCH_7_IMAGE, "elasticsearch7"); - } - - @Override - protected String indexEndpoint(String index, String docId) - { - return format("/%s/_doc/%s", index, docId); - } - - @Override - protected String indexMapping(String properties) - { - return "{\"mappings\": " + properties + "}"; + super(new ElasticsearchServer(ELASTICSEARCH_7_IMAGE), "elasticsearch7"); } } diff --git a/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchOpenSearchConnectorTest.java b/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearch8ConnectorTest.java similarity index 53% rename from plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchOpenSearchConnectorTest.java rename to plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearch8ConnectorTest.java index ee0dd999729df..c2c57a5bc6e1f 100644 --- a/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchOpenSearchConnectorTest.java +++ b/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearch8ConnectorTest.java @@ -13,26 +13,16 @@ */ package io.trino.plugin.elasticsearch; -import static java.lang.String.format; +import java.io.IOException; -public class TestElasticsearchOpenSearchConnectorTest +import static io.trino.plugin.elasticsearch.ElasticsearchServer.ELASTICSEARCH_8_IMAGE; + +public class TestElasticsearch8ConnectorTest extends BaseElasticsearchConnectorTest { - public TestElasticsearchOpenSearchConnectorTest() - { - // 1.0.0 and 1.0.1 causes NotSslRecordException during the initialization - super("opensearchproject/opensearch:1.1.0", "opensearch"); - } - - @Override - protected String indexEndpoint(String index, String docId) - { - return format("/%s/_doc/%s", index, docId); - } - - @Override - protected String indexMapping(String properties) + public TestElasticsearch8ConnectorTest() + throws IOException { - return "{\"mappings\": " + properties + "}"; + super(new ElasticsearchServer(ELASTICSEARCH_8_IMAGE), "elasticsearch8"); } } diff --git a/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchBackpressure.java b/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchBackpressure.java deleted file mode 100644 index dbaf931760664..0000000000000 --- a/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/TestElasticsearchBackpressure.java +++ /dev/null @@ -1,86 +0,0 @@ -/* - * 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.elasticsearch; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import io.trino.testing.AbstractTestQueryFramework; -import io.trino.testing.QueryRunner; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; -import org.testcontainers.containers.Network; - -import java.io.IOException; - -import static io.trino.plugin.elasticsearch.ElasticsearchQueryRunner.createElasticsearchQueryRunner; -import static io.trino.tpch.TpchTable.ORDERS; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; - -@TestInstance(PER_CLASS) -public class TestElasticsearchBackpressure - extends AbstractTestQueryFramework -{ - private static final String image = "elasticsearch:7.0.0"; - - private Network network; - private ElasticsearchServer elasticsearch; - private ElasticsearchNginxProxy elasticsearchNginxProxy; - - @Override - protected QueryRunner createQueryRunner() - throws Exception - { - network = Network.newNetwork(); - elasticsearch = new ElasticsearchServer(network, image, ImmutableMap.of()); - elasticsearchNginxProxy = new ElasticsearchNginxProxy(network, 1); - - return createElasticsearchQueryRunner( - elasticsearchNginxProxy.getAddress(), - ImmutableList.of(ORDERS), - ImmutableMap.of(), - ImmutableMap.of(), - // This test can only run on a single node, otherwise each node exports its own stats beans and they override each other - // You can only bind one such bean per JVM, so this causes problems with statistics being 0 despite backpressure handling - 1, - // Use a unique catalog name to make sure JMX stats beans are unique and not affected by other tests - "elasticsearch-backpressure"); - } - - @AfterAll - public final void destroy() - throws IOException - { - elasticsearchNginxProxy.stop(); - elasticsearchNginxProxy = null; - elasticsearch.stop(); - elasticsearch = null; - network.close(); - network = null; - } - - @Test - public void testQueryWithBackpressure() - { - // Check that JMX stats show no sign of backpressure - assertQueryReturnsEmptyResult("SELECT 1 FROM jmx.current.\"trino.plugin.elasticsearch.client:*\" WHERE \"backpressurestats.alltime.count\" > 0"); - assertQueryReturnsEmptyResult("SELECT 1 FROM jmx.current.\"trino.plugin.elasticsearch.client:*\" WHERE \"backpressurestats.alltime.max\" > 0"); - - assertQuerySucceeds("SELECT * FROM orders"); - - // Check that JMX stats show requests have been retried due to backpressure - assertQuery("SELECT DISTINCT 1 FROM jmx.current.\"trino.plugin.elasticsearch.client:*\" WHERE \"backpressurestats.alltime.count\" > 0", "VALUES 1"); - assertQuery("SELECT DISTINCT 1 FROM jmx.current.\"trino.plugin.elasticsearch.client:*\" WHERE \"backpressurestats.alltime.max\" > 0", "VALUES 1"); - } -} diff --git a/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/TestPasswordAuthentication.java b/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/TestPasswordAuthentication.java deleted file mode 100644 index 14cb228e229a0..0000000000000 --- a/plugin/trino-elasticsearch/src/test/java/io/trino/plugin/elasticsearch/TestPasswordAuthentication.java +++ /dev/null @@ -1,125 +0,0 @@ -/* - * 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.elasticsearch; - -import com.amazonaws.util.Base64; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.io.Resources; -import com.google.common.net.HostAndPort; -import io.trino.sql.query.QueryAssertions; -import io.trino.testing.DistributedQueryRunner; -import org.apache.http.HttpHost; -import org.apache.http.entity.ContentType; -import org.apache.http.message.BasicHeader; -import org.apache.http.nio.entity.NStringEntity; -import org.elasticsearch.client.RestClient; -import org.elasticsearch.client.RestHighLevelClient; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInstance; -import org.junit.jupiter.api.parallel.Execution; - -import java.io.IOException; -import java.nio.charset.StandardCharsets; - -import static com.google.common.io.Resources.getResource; -import static io.airlift.testing.Closeables.closeAll; -import static io.trino.plugin.elasticsearch.ElasticsearchQueryRunner.createElasticsearchQueryRunner; -import static java.lang.String.format; -import static java.nio.charset.StandardCharsets.UTF_8; -import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; -import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; - -@TestInstance(PER_CLASS) -@Execution(CONCURRENT) -public class TestPasswordAuthentication -{ - private static final String USER = "elastic_user"; - private static final String PASSWORD = "123456"; - - private ElasticsearchServer elasticsearch; - private RestHighLevelClient client; - private QueryAssertions assertions; - - @BeforeAll - public void setUp() - throws Exception - { - // We use 7.8.0 because security became a non-commercial feature in recent versions - elasticsearch = new ElasticsearchServer("elasticsearch:7.8.0", ImmutableMap.builder() - .put("elasticsearch.yml", loadResource("elasticsearch.yml")) - .put("users", loadResource("users")) - .put("users_roles", loadResource("users_roles")) - .put("roles.yml", loadResource("roles.yml")) - .buildOrThrow()); - - HostAndPort address = elasticsearch.getAddress(); - client = new RestHighLevelClient(RestClient.builder(new HttpHost(address.getHost(), address.getPort()))); - - DistributedQueryRunner runner = createElasticsearchQueryRunner( - elasticsearch.getAddress(), - ImmutableList.of(), - ImmutableMap.of(), - ImmutableMap.builder() - .put("elasticsearch.security", "PASSWORD") - .put("elasticsearch.auth.user", USER) - .put("elasticsearch.auth.password", PASSWORD) - .buildOrThrow(), - 3); - - assertions = new QueryAssertions(runner); - } - - @AfterAll - public final void destroy() - throws IOException - { - closeAll( - () -> assertions.close(), - () -> elasticsearch.stop(), - () -> client.close()); - - assertions = null; - elasticsearch = null; - client = null; - } - - @Test - public void test() - throws IOException - { - String json = new ObjectMapper().writeValueAsString(ImmutableMap.of("value", 42L)); - - client.getLowLevelClient() - .performRequest( - "POST", - "/test/_doc?refresh", - ImmutableMap.of(), - new NStringEntity(json, ContentType.APPLICATION_JSON), - new BasicHeader("Authorization", format("Basic %s", Base64.encodeAsString(format("%s:%s", USER, PASSWORD).getBytes(StandardCharsets.UTF_8))))); - - assertThat(assertions.query("SELECT * FROM test")) - .matches("VALUES BIGINT '42'"); - } - - private static String loadResource(String file) - throws IOException - { - return Resources.toString(getResource(file), UTF_8); - } -} diff --git a/plugin/trino-elasticsearch/src/test/resources/ca.crt b/plugin/trino-elasticsearch/src/test/resources/ca.crt new file mode 100644 index 0000000000000..4cf446a6cc277 --- /dev/null +++ b/plugin/trino-elasticsearch/src/test/resources/ca.crt @@ -0,0 +1,20 @@ +-----BEGIN CERTIFICATE----- +MIIDSTCCAjGgAwIBAgIUByq1K6VbZFocXfpnEohIw8NgEW4wDQYJKoZIhvcNAQEL +BQAwNDEyMDAGA1UEAxMpRWxhc3RpYyBDZXJ0aWZpY2F0ZSBUb29sIEF1dG9nZW5l +cmF0ZWQgQ0EwHhcNMjMxMjMwMTczMjIxWhcNMjYxMjI5MTczMjIxWjA0MTIwMAYD +VQQDEylFbGFzdGljIENlcnRpZmljYXRlIFRvb2wgQXV0b2dlbmVyYXRlZCBDQTCC +ASIwDQYJKoZIhvcNAQEBBQADggEPADCCAQoCggEBAImUzmkzHoVgREpIVHcjvtkV +oT4QjNvciH0bZOcnIUBpXq5ywBoSS3CjckF9mirkAJXZHS+M13155+JKO085Ewy7 +U684jsEVRS8GmNiKrKNYKFiwdKeQKIYtwmwgHMfJCEwcFaVOmJ2PhbiHiQj+8lMr +mRMwS+Wy7deci9uJuzykSHQ7LW7MNcZBCBkWBtctI2p4h21yYZzwn4UzsDGD7i02 +GNJ/CHG4M5QjLY9P/tCHoss9kHDUn7k/rnezk8nHZgf2XAwVYdJBbVeYvUe7HgtK ++35FeSACslOtgV2kQJpULwvh8wiqgP+/oIhNoNPW/NpyoOT4luQmJfxZV5SKJ08C +AwEAAaNTMFEwHQYDVR0OBBYEFOqG2pBw+qtXIiPZ69CVSzeI43UGMB8GA1UdIwQY +MBaAFOqG2pBw+qtXIiPZ69CVSzeI43UGMA8GA1UdEwEB/wQFMAMBAf8wDQYJKoZI +hvcNAQELBQADggEBABKr0j2hk5qzWs8uWR4yqxjL4+MQTSaBtvcMHCF6w1rCCRLt +4sIz7Gy0ecnqjfXscjrrrEA4ruYP4CpAvRXyYgmJ2F1jLYrMcmYnYupCwRl88ygO +dvjVTdAjOib0NBUsE1DA8l0idFYHipCV2izpKjF/AB1HnhWm1A++pC3nZ++x7o6u +lqOuTwEMDBMnwpc+CQpQPSa9QCIl60LNpbhH+rWAL2xm5fdLV/Hs3hFQvABGorgR +78F/kBfk30c/sM2tQog7By/ic6KIEFQhFGvDz2fAqt2BAAyO22nWEQZQI+JTBAH4 +tEBodOialM5LlPFAOU0gmr6q7g8lFIkVYwxadUw= +-----END CERTIFICATE----- diff --git a/plugin/trino-elasticsearch/src/test/resources/elasticsearch.yml b/plugin/trino-elasticsearch/src/test/resources/elasticsearch.yml index eb6bb9c410c30..81d4a23f173a9 100644 --- a/plugin/trino-elasticsearch/src/test/resources/elasticsearch.yml +++ b/plugin/trino-elasticsearch/src/test/resources/elasticsearch.yml @@ -1,4 +1,14 @@ cluster.name: "docker-cluster" +node.name: "elasticsearch-server" network.host: 0.0.0.0 xpack.security.enabled: true +xpack.security.http.ssl.enabled: true +xpack.security.http.ssl.certificate_authorities: /usr/share/elasticsearch/config/ca.crt +xpack.security.http.ssl.certificate: /usr/share/elasticsearch/config/server.crt +xpack.security.http.ssl.key: /usr/share/elasticsearch/config/server.key +xpack.security.transport.ssl.enabled: true +xpack.security.transport.ssl.verification_mode: none +xpack.security.transport.ssl.certificate_authorities: /usr/share/elasticsearch/config/ca.crt +xpack.security.transport.ssl.certificate: /usr/share/elasticsearch/config/server.crt +xpack.security.transport.ssl.key: /usr/share/elasticsearch/config/server.key diff --git a/plugin/trino-elasticsearch/src/test/resources/server.crt b/plugin/trino-elasticsearch/src/test/resources/server.crt new file mode 100644 index 0000000000000..6b033c7f8de27 --- /dev/null +++ b/plugin/trino-elasticsearch/src/test/resources/server.crt @@ -0,0 +1,21 @@ +-----BEGIN CERTIFICATE----- +MIIDYTCCAkmgAwIBAgIVALANPI8Y3YDiH0EsyazPC0a305zjMA0GCSqGSIb3DQEB +CwUAMDQxMjAwBgNVBAMTKUVsYXN0aWMgQ2VydGlmaWNhdGUgVG9vbCBBdXRvZ2Vu +ZXJhdGVkIENBMB4XDTIzMTIzMDE3MzIyM1oXDTI2MTIyOTE3MzIyM1owHzEdMBsG +A1UEAxMUZWxhc3RpY3NlYXJjaC1zZXJ2ZXIwggEiMA0GCSqGSIb3DQEBAQUAA4IB +DwAwggEKAoIBAQC3lnDqYCsnVWLxlokBBhM4kJsOc+Dwp7FQWvcEEDQYIPHBUDoE +1b1uPO10iwDy41+vCd+L3tkt5vgzhkU+W+AMYOLVJSDUTU1oaFT3gQdAPBLssMZJ ++mrVVysxK/BOkB1r4GRCj3+DaL0/YiIm+VT4fmt1bQ46C9AqPtGBNjjAheOfBVvG +sp8j9zs296OtUnlDLYhtEPF5bSpjyTn0rlKuW9Vc1zYNFW+oshLMJxmiLcFCFzLK +RKdNLGNXs0JGy7BfjOGJ7b8zgzBe8phua7VvDoYw4Hda8z8uNSBWd0bOOE0DT4mb +biC+M0G3c76TNceeZMGlkw6g0Vt4G5jFYTIXAgMBAAGjfzB9MB0GA1UdDgQWBBSA +l2wp8zAFBrBMH+Zf+tS0dznNnjAfBgNVHSMEGDAWgBTqhtqQcPqrVyIj2evQlUs3 +iON1BjAwBgNVHREEKTAngglsb2NhbGhvc3SHBH8AAAGCFGVsYXN0aWNzZWFyY2gt +c2VydmVyMAkGA1UdEwQCMAAwDQYJKoZIhvcNAQELBQADggEBACYQKkoSJrQsFWpR +zTD+5visP/W/wUeHdQxU5iEesLsd43r7jKkZLRJzZgDtPSfYw3LLL/dZHVCtzdUg +NtarP+EaMp5YlGyEvMRiaIy2pFajHN9+wElImQhi5iw5CY6R/YL6m4ap2mVmIeMi +nvWKYaibMzNx2wouahIrEvANb3N93lr8T5LDfLPPLXmHAa0ebk2GbWt90mVdk6UZ +QQqI0VK8YlzR9ybp8jJ5ZRgXIegtn91Tts164+61wQQ2M6fV1le41m/1NENAzkIW +Q7LJvvqe+Q3YO8yBvxYP7ru/kKgBWHPyof+6mocqirwvrVLb+MPL+sKolcuXP0sS +c52vk0Q= +-----END CERTIFICATE----- diff --git a/plugin/trino-elasticsearch/src/test/resources/server.key b/plugin/trino-elasticsearch/src/test/resources/server.key new file mode 100644 index 0000000000000..eaee8c711e3a5 --- /dev/null +++ b/plugin/trino-elasticsearch/src/test/resources/server.key @@ -0,0 +1,27 @@ +-----BEGIN RSA PRIVATE KEY----- +MIIEpAIBAAKCAQEAt5Zw6mArJ1Vi8ZaJAQYTOJCbDnPg8KexUFr3BBA0GCDxwVA6 +BNW9bjztdIsA8uNfrwnfi97ZLeb4M4ZFPlvgDGDi1SUg1E1NaGhU94EHQDwS7LDG +Sfpq1VcrMSvwTpAda+BkQo9/g2i9P2IiJvlU+H5rdW0OOgvQKj7RgTY4wIXjnwVb +xrKfI/c7NvejrVJ5Qy2IbRDxeW0qY8k59K5SrlvVXNc2DRVvqLISzCcZoi3BQhcy +ykSnTSxjV7NCRsuwX4zhie2/M4MwXvKYbmu1bw6GMOB3WvM/LjUgVndGzjhNA0+J +m24gvjNBt3O+kzXHnmTBpZMOoNFbeBuYxWEyFwIDAQABAoIBACRPxrys0cxBWMUk ++6suqu2UuH6Gz/Z5jJoyq8nrdIpTNc4BEsCzD/k2ZtcWyVTBQUMrRkJxjO5F0VDL +nXWEkY+RODxlUBV9xrJAdJHRvP/pAes72bR3KdtRlvLuWGUJnzAfnWUG4fc5Z7ka +LVwdE7/krVd3/wjfHkjEzQZsFZSKcDf/3gAt/xfVW2Oi9xIbUlsHKjjTyE7H3rb2 +DG91TEDYHa5tt8WBzdyyKiXgy8WDugHWrH8C6rNShAocuMtPmkk/W2LyAtioKY5u +qE+XCDpjFFJq99feJQwhkbVNyfpEDX6s6jUzJ67ppr3dkEsxEkVpQF8pXnbo+oB0 +flvsGvUCgYEAuy0YDx6GCc28rO9YvaML92RhvSgwQN8J27taApJRi9Vf8en7sPzk +wbe2oz/80SSnM3IOz6NGf/3y+MtDwy+Z6Xo/foQGngER4ojYu0nqOUG/w7Vo/k3M +JCFla90opMxv5ndvZHzCB4hKXNpB2KmDMO2lxwS1JMdQ50t5lQEF2XUCgYEA+xeP +++5LRLGrOxsWxcnEO2DTLOFIMXI0WnsxipkBCYSyVO/fVloNC2KR0da9Us2ea6er +6iKNCQ8I7LGbvx36+ITLbI5VN1y+0NTXvQuOQa5kif1ShE1E3peRRVPStd+nzJss +VIQMSiXgaJxzZLfLOgHlNUF+Kp8p3WZZqDN/H9sCgYAjhUI8lElRkPaORvmrO0bp +Euatc/P8S5deWKaTOFs7UynbCdFW3Bc5X/jL8De1JOSKEcoXw198qJ+0cTaR0vW9 +ZhKomrt6vq87M0cV+8f0R7Lxm/qpz+++p4vMtXKiG53cuautATP9I/MgvdajHEAm +R+Me6pjJuv1yd1NCYKhQhQKBgQC75/07xzAS33dKS1J2gDFXUnWW750MS/A8Clef +5jLtgZ3HjJ+xysBaVdDUnYSfc/tri1ZPaL3tGpBOOfhZRUFtPEvUGIxTWxf9xlsR +QZWbN7xLvAVFt2Gjxv/TRxC4RQEZUMMTGKLt1nyjLPn6VMVEyuhSO4oEsYouAwtz +od4v4wKBgQCcb3OVsDRMqILwie7ksnVU9j1HhpjA7rhH9wtRZZP0oc3rabr5irgv +A0LGxKKSBPdrQFsH94C1SHwJh+m0MscrjI41XZei9+7B1+I1M3X/IcOTheOS4Iin +04gVZdj7A9lE/USRIrJO2eSL71uOVkcczpOAMFtrkoCQ/DRkw5sC1A== +-----END RSA PRIVATE KEY----- diff --git a/plugin/trino-elasticsearch/src/test/resources/truststore.jks b/plugin/trino-elasticsearch/src/test/resources/truststore.jks new file mode 100644 index 0000000000000000000000000000000000000000..12391c7e447bbf60f062af9a7cca7c1c0cd4631f GIT binary patch literal 1238 zcmV;{1S$J4f&|h60Ru3C1bhYwDuzgg_YDCD0ic2eZ3Kb@X)uBWWiWySVFn2*hDe6@ z4FLxRpn?QaFoFb50s#Opf&@nf2`Yw2hW8Bt2LUi<1_>&LNQU+thDZTr0|Wso1Q6P&9b0uTq#z~Il>@4hTugw11JGH^^VfwMnjH`_IZUY->TB3C zQo5L1=O)w|P;>c2_03>$FKfI~W^W*TDNIVl%Tub~3c2PjjSeekiXH61 z;8NFiDV z&pgR$hUY1x1ZKU4w1)8BfI?ldl*pDy7G{%Lm72c#O>%Hy1LV9QgwENXE+p7{BOvrUK7JIpVqMBS3^88y#eA)VxZiiMC>g3`~WU z_F_Y^29oCBp%_bBu?h!ljLxRxx$Ef(&?YfC3ALZMV>s|L#tY>OD1AU%E{~kF8iF2@ zl-ga#QEa^Hq1<vP&K(gj`DC*V})w$WArd)We_`*I4xV|72U-*;srxiWflVB*UPaJUvcYhbGW46 z%gxZx<4vmzTjRh*u^1J9lUqke*ULC-h`ih%41WETYk0`_f-Pq!1RLs!f&(!B!!-(y850Ob&-?N#W z0f3p{g#c&1I^c3($@(jIIrC7LJKJ!0VGmd>OmG6m%@oAIiZ6_1jQXw-cD!z9Sx9AA z^s=?A>KcDj#tq#tT>ia#Bqei8)#aj)7gvG{$wdBFP>S1&$4ANMDqbWl)Inilg%N-@ zlGWz=LZ%o(FflL<1_@w>NC9O71OfpC00bcGNS%lVjoWf2 z!GBfa&#`pm2hcQMx)1az<3or_y+mmQ6bDm=vMUc5$fG*ebk&q0_H&g-fdT?25W@I5 AbpQYW literal 0 HcmV?d00001 From b2c7cd402f0b747b292d412054f4c49554ce95bb Mon Sep 17 00:00:00 2001 From: "Mateusz \"Serafin\" Gajewski" Date: Wed, 10 Jan 2024 12:34:07 +0100 Subject: [PATCH 341/350] Cleanup elasticsearch pom --- plugin/trino-elasticsearch/pom.xml | 38 +----------------------------- 1 file changed, 1 insertion(+), 37 deletions(-) diff --git a/plugin/trino-elasticsearch/pom.xml b/plugin/trino-elasticsearch/pom.xml index 9a9b35d7ebb5e..e4601e8391cec 100644 --- a/plugin/trino-elasticsearch/pom.xml +++ b/plugin/trino-elasticsearch/pom.xml @@ -22,13 +22,6 @@ com.amazonaws aws-java-sdk-core - - - - org.apache.httpcomponents - httpclient - - @@ -154,24 +147,15 @@ elasticsearch ${dep.elasticsearch.version} - - org.apache.logging.log4j - log4j-api - org.apache.logging.log4j log4j-core - + org.apache.lucene lucene-analyzers-common - - - org.elasticsearch - jna - @@ -185,12 +169,6 @@ org.elasticsearch elasticsearch-x-content ${dep.elasticsearch.version} - - - org.yaml - snakeyaml - - @@ -409,20 +387,6 @@ - - org.apache.maven.plugins - maven-enforcer-plugin - - - - - - org.apache.lucene:lucene-core - - - - - org.apache.maven.plugins maven-dependency-plugin From b32b287c738301cc32dcbf3cde2e13fcef1f5990 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Thu, 11 Jan 2024 11:52:15 +0100 Subject: [PATCH 342/350] Fix impacted features inclusion in product tests In few suites `configured_features` was misspelled as `configured-features`. --- .../trino/tests/product/launcher/suite/suites/SuiteClients.java | 2 +- .../tests/product/launcher/suite/suites/SuiteFunctions.java | 2 +- .../io/trino/tests/product/launcher/suite/suites/SuiteGcs.java | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/suite/suites/SuiteClients.java b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/suite/suites/SuiteClients.java index c7d63bb5c449a..deb49293cf44f 100644 --- a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/suite/suites/SuiteClients.java +++ b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/suite/suites/SuiteClients.java @@ -32,7 +32,7 @@ public List getTestRuns(EnvironmentConfig config) return ImmutableList.of( testOnEnvironment(EnvMultinode.class) .withGroups( - "configured-features", + "configured_features", "cli", "jdbc", "trino_jdbc") diff --git a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/suite/suites/SuiteFunctions.java b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/suite/suites/SuiteFunctions.java index 4513988a1ad50..ad3fc9e5543d4 100644 --- a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/suite/suites/SuiteFunctions.java +++ b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/suite/suites/SuiteFunctions.java @@ -31,7 +31,7 @@ public List getTestRuns(EnvironmentConfig config) { return ImmutableList.of( testOnEnvironment(EnvMultinode.class) - .withGroups("configured-features", "functions") + .withGroups("configured_features", "functions") .build()); } } diff --git a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/suite/suites/SuiteGcs.java b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/suite/suites/SuiteGcs.java index 08b5b635276c4..325a1a0aec9ca 100644 --- a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/suite/suites/SuiteGcs.java +++ b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/suite/suites/SuiteGcs.java @@ -31,7 +31,7 @@ public List getTestRuns(EnvironmentConfig config) { return ImmutableList.of( testOnEnvironment(EnvMultinodeGcs.class) - .withGroups("delta-lake-gcs", "configured-features") + .withGroups("delta-lake-gcs", "configured_features") .build()); } } From ecb0c78bd0691198e5ec5c7f19bf77d4c1c70a3e Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 10 Jan 2024 15:54:25 +0100 Subject: [PATCH 343/350] Reformat array initializers Prepare for upcoming airbase change requiring exactly one space in array initializers. This is to help ensure that the code that passes CI does not change upon automatic formatting. --- .../main/java/io/trino/operator/FlatHash.java | 2 +- .../main/java/io/trino/operator/FlatSet.java | 2 +- .../AbstractMapAggregationState.java | 2 +- .../aggregation/histogram/TypedHistogram.java | 2 +- .../AbstractMultimapAggregationState.java | 2 +- .../sql/rewrite/DescribeInputRewrite.java | 2 +- .../sql/rewrite/DescribeOutputRewrite.java | 2 +- .../java/io/trino/block/TestRowBlock.java | 8 ++-- .../operator/TestSimplePagesHashStrategy.java | 40 +++++++++---------- .../operator/exchange/TestLocalExchange.java | 2 +- .../TestPositionsAppenderPageBuilder.java | 10 ++--- .../TestDeleteAndInsertMergeProcessor.java | 14 +++---- .../spi/block/VariableWidthBlockBuilder.java | 2 +- .../java/io/trino/spi/type/BooleanType.java | 2 +- .../spi/block/BenchmarkCopyPositions.java | 2 +- .../io/trino/spi/block/TestColumnarMap.java | 2 +- .../io/trino/spi/block/TestLazyBlock.java | 2 +- .../trino/hive/formats/avro/TestAvroBase.java | 10 ++--- .../TestTupleDomainParquetPredicate.java | 4 +- .../plugin/deltalake/DeltaLakeWriter.java | 2 +- .../checkpoint/TestCheckpointWriter.java | 8 ++-- .../plugin/hive/coercions/CoercionUtils.java | 2 +- .../iceberg/procedure/MigrateProcedure.java | 2 +- .../BaseIcebergConnectorSmokeTest.java | 2 +- .../trino/plugin/iceberg/TestIcebergV2.java | 20 +++++----- .../kafka/protobuf/TestProtobufEncoder.java | 6 +-- .../plugin/postgresql/PostgreSqlClient.java | 2 +- .../TestHiveRequireQueryPartitionsFilter.java | 2 +- 28 files changed, 80 insertions(+), 78 deletions(-) diff --git a/core/trino-main/src/main/java/io/trino/operator/FlatHash.java b/core/trino-main/src/main/java/io/trino/operator/FlatHash.java index 3086d95940322..36119075a6823 100644 --- a/core/trino-main/src/main/java/io/trino/operator/FlatHash.java +++ b/core/trino-main/src/main/java/io/trino/operator/FlatHash.java @@ -342,7 +342,7 @@ private void rehash(int minimumRequiredCapacity) // we incrementally allocate the record groups to smooth out memory allocation if (capacity <= RECORDS_PER_GROUP) { - recordGroups = new byte[][]{new byte[multiplyExact(capacity, recordSize)]}; + recordGroups = new byte[][] {new byte[multiplyExact(capacity, recordSize)]}; } else { recordGroups = new byte[(capacity + 1) >> RECORDS_PER_GROUP_SHIFT][]; diff --git a/core/trino-main/src/main/java/io/trino/operator/FlatSet.java b/core/trino-main/src/main/java/io/trino/operator/FlatSet.java index 5b5c298fdd284..fc6b968f4481f 100644 --- a/core/trino-main/src/main/java/io/trino/operator/FlatSet.java +++ b/core/trino-main/src/main/java/io/trino/operator/FlatSet.java @@ -99,7 +99,7 @@ public FlatSet( private static byte[][] createRecordGroups(int capacity, int recordSize) { if (capacity < RECORDS_PER_GROUP) { - return new byte[][]{new byte[multiplyExact(capacity, recordSize)]}; + return new byte[][] {new byte[multiplyExact(capacity, recordSize)]}; } byte[][] groups = new byte[(capacity + 1) >> RECORDS_PER_GROUP_SHIFT][]; diff --git a/core/trino-main/src/main/java/io/trino/operator/aggregation/AbstractMapAggregationState.java b/core/trino-main/src/main/java/io/trino/operator/aggregation/AbstractMapAggregationState.java index 4d4bfc76934fe..a18380534ad5f 100644 --- a/core/trino-main/src/main/java/io/trino/operator/aggregation/AbstractMapAggregationState.java +++ b/core/trino-main/src/main/java/io/trino/operator/aggregation/AbstractMapAggregationState.java @@ -186,7 +186,7 @@ public AbstractMapAggregationState(AbstractMapAggregationState state) private static byte[][] createRecordGroups(int capacity, int recordSize) { if (capacity < RECORDS_PER_GROUP) { - return new byte[][]{new byte[multiplyExact(capacity, recordSize)]}; + return new byte[][] {new byte[multiplyExact(capacity, recordSize)]}; } byte[][] groups = new byte[(capacity + 1) >> RECORDS_PER_GROUP_SHIFT][]; diff --git a/core/trino-main/src/main/java/io/trino/operator/aggregation/histogram/TypedHistogram.java b/core/trino-main/src/main/java/io/trino/operator/aggregation/histogram/TypedHistogram.java index e40f503047a0f..f4763b6c0037b 100644 --- a/core/trino-main/src/main/java/io/trino/operator/aggregation/histogram/TypedHistogram.java +++ b/core/trino-main/src/main/java/io/trino/operator/aggregation/histogram/TypedHistogram.java @@ -139,7 +139,7 @@ public TypedHistogram( private static byte[][] createRecordGroups(int capacity, int recordSize) { if (capacity < RECORDS_PER_GROUP) { - return new byte[][]{new byte[multiplyExact(capacity, recordSize)]}; + return new byte[][] {new byte[multiplyExact(capacity, recordSize)]}; } byte[][] groups = new byte[(capacity + 1) >> RECORDS_PER_GROUP_SHIFT][]; diff --git a/core/trino-main/src/main/java/io/trino/operator/aggregation/multimapagg/AbstractMultimapAggregationState.java b/core/trino-main/src/main/java/io/trino/operator/aggregation/multimapagg/AbstractMultimapAggregationState.java index 5a69677e9168b..0d2d57fffd9a1 100644 --- a/core/trino-main/src/main/java/io/trino/operator/aggregation/multimapagg/AbstractMultimapAggregationState.java +++ b/core/trino-main/src/main/java/io/trino/operator/aggregation/multimapagg/AbstractMultimapAggregationState.java @@ -191,7 +191,7 @@ public AbstractMultimapAggregationState(AbstractMultimapAggregationState state) private static byte[][] createRecordGroups(int capacity, int recordSize) { if (capacity < RECORDS_PER_GROUP) { - return new byte[][]{new byte[multiplyExact(capacity, recordSize)]}; + return new byte[][] {new byte[multiplyExact(capacity, recordSize)]}; } byte[][] groups = new byte[(capacity + 1) >> RECORDS_PER_GROUP_SHIFT][]; diff --git a/core/trino-main/src/main/java/io/trino/sql/rewrite/DescribeInputRewrite.java b/core/trino-main/src/main/java/io/trino/sql/rewrite/DescribeInputRewrite.java index 9f917d2e6d445..fe524be5f298f 100644 --- a/core/trino-main/src/main/java/io/trino/sql/rewrite/DescribeInputRewrite.java +++ b/core/trino-main/src/main/java/io/trino/sql/rewrite/DescribeInputRewrite.java @@ -88,7 +88,7 @@ private static final class Visitor extends AstVisitor { private static final Query EMPTY_INPUT = createDesctibeInputQuery( - new Row[]{row( + new Row[] {row( new Cast(new NullLiteral(), toSqlType(BIGINT)), new Cast(new NullLiteral(), toSqlType(VARCHAR)))}, Optional.of(new Limit(new LongLiteral("0")))); diff --git a/core/trino-main/src/main/java/io/trino/sql/rewrite/DescribeOutputRewrite.java b/core/trino-main/src/main/java/io/trino/sql/rewrite/DescribeOutputRewrite.java index b8a78d502e299..9acab16abe275 100644 --- a/core/trino-main/src/main/java/io/trino/sql/rewrite/DescribeOutputRewrite.java +++ b/core/trino-main/src/main/java/io/trino/sql/rewrite/DescribeOutputRewrite.java @@ -88,7 +88,7 @@ private static final class Visitor extends AstVisitor { private static final Query EMPTY_OUTPUT = createDesctibeOutputQuery( - new Row[]{row( + new Row[] {row( new Cast(new NullLiteral(), toSqlType(VARCHAR)), new Cast(new NullLiteral(), toSqlType(VARCHAR)), new Cast(new NullLiteral(), toSqlType(VARCHAR)), diff --git a/core/trino-main/src/test/java/io/trino/block/TestRowBlock.java b/core/trino-main/src/test/java/io/trino/block/TestRowBlock.java index 399d57adc4857..a980da71f20cf 100644 --- a/core/trino-main/src/test/java/io/trino/block/TestRowBlock.java +++ b/core/trino-main/src/test/java/io/trino/block/TestRowBlock.java @@ -69,9 +69,11 @@ public void testFromFieldBlocksNoNullsDetection() { // Blocks does not discard the null mask during creation if no values are null boolean[] rowIsNull = new boolean[5]; - assertThat(fromNotNullSuppressedFieldBlocks(5, Optional.of(rowIsNull), new Block[] {new ByteArrayBlock(5, Optional.empty(), createExpectedValue(5).getBytes())}).mayHaveNull()).isTrue(); + assertThat(fromNotNullSuppressedFieldBlocks(5, Optional.of(rowIsNull), new Block[] { + new ByteArrayBlock(5, Optional.empty(), createExpectedValue(5).getBytes())}).mayHaveNull()).isTrue(); rowIsNull[rowIsNull.length - 1] = true; - assertThat(fromNotNullSuppressedFieldBlocks(5, Optional.of(rowIsNull), new Block[] {new ByteArrayBlock(5, Optional.of(rowIsNull), createExpectedValue(5).getBytes())}).mayHaveNull()).isTrue(); + assertThat(fromNotNullSuppressedFieldBlocks(5, Optional.of(rowIsNull), new Block[] { + new ByteArrayBlock(5, Optional.of(rowIsNull), createExpectedValue(5).getBytes())}).mayHaveNull()).isTrue(); // Empty blocks have no nulls and can also discard their null mask assertThat(fromNotNullSuppressedFieldBlocks(0, Optional.of(new boolean[0]), new Block[] {new ByteArrayBlock(0, Optional.empty(), new byte[0])}).mayHaveNull()).isFalse(); @@ -101,7 +103,7 @@ public void testCompactBlock() // NOTE: nested row blocks are required to have the exact same size so they are always compact assertCompact(fromFieldBlocks(0, new Block[] {emptyBlock, emptyBlock})); - assertCompact(fromNotNullSuppressedFieldBlocks(rowIsNull.length, Optional.of(rowIsNull), new Block[]{ + assertCompact(fromNotNullSuppressedFieldBlocks(rowIsNull.length, Optional.of(rowIsNull), new Block[] { new ByteArrayBlock(6, Optional.of(rowIsNull), createExpectedValue(6).getBytes()), new ByteArrayBlock(6, Optional.of(rowIsNull), createExpectedValue(6).getBytes())})); } diff --git a/core/trino-main/src/test/java/io/trino/operator/TestSimplePagesHashStrategy.java b/core/trino-main/src/test/java/io/trino/operator/TestSimplePagesHashStrategy.java index 379dcd0a1fcda..bb6957e05a7e1 100644 --- a/core/trino-main/src/test/java/io/trino/operator/TestSimplePagesHashStrategy.java +++ b/core/trino-main/src/test/java/io/trino/operator/TestSimplePagesHashStrategy.java @@ -37,7 +37,7 @@ public class TestSimplePagesHashStrategy @Test public void testHashRowWithIntegerType() { - Block block = new IntArrayBlock(1, Optional.empty(), new int[]{1234}); + Block block = new IntArrayBlock(1, Optional.empty(), new int[] {1234}); SimplePagesHashStrategy strategy = createSimplePagesHashStrategy(INTEGER, ImmutableList.of(block)); Page page = new Page(block); @@ -51,9 +51,9 @@ public void testHashRowWithMapType() MapType mapType = new MapType(INTEGER, INTEGER, new TypeOperators()); Block block = mapType.createBlockFromKeyValue( Optional.empty(), - new int[]{0, 1}, - new IntArrayBlock(1, Optional.empty(), new int[]{1234}), - new IntArrayBlock(1, Optional.empty(), new int[]{5678})); + new int[] {0, 1}, + new IntArrayBlock(1, Optional.empty(), new int[] {1234}), + new IntArrayBlock(1, Optional.empty(), new int[] {5678})); SimplePagesHashStrategy strategy = createSimplePagesHashStrategy(mapType, ImmutableList.of(block)); Page page = new Page(block); @@ -67,9 +67,9 @@ public void testRowEqualsRowWithIntegerType() { SimplePagesHashStrategy strategy = createSimplePagesHashStrategy(INTEGER, ImmutableList.of()); - Page leftPage = new Page(new IntArrayBlock(1, Optional.empty(), new int[]{1234})); - Page rightPage1 = new Page(new IntArrayBlock(1, Optional.empty(), new int[]{1234})); - Page rightPage2 = new Page(new IntArrayBlock(1, Optional.empty(), new int[]{5678})); + Page leftPage = new Page(new IntArrayBlock(1, Optional.empty(), new int[] {1234})); + Page rightPage1 = new Page(new IntArrayBlock(1, Optional.empty(), new int[] {1234})); + Page rightPage2 = new Page(new IntArrayBlock(1, Optional.empty(), new int[] {5678})); // This works because IntegerType is comparable. assertThat(strategy.rowEqualsRow(0, leftPage, 0, rightPage1)).isTrue(); @@ -84,21 +84,21 @@ public void testRowEqualsRowWithMapType() Page leftPage = new Page(mapType.createBlockFromKeyValue( Optional.empty(), - new int[]{0, 1}, - new IntArrayBlock(1, Optional.empty(), new int[]{1234}), - new IntArrayBlock(1, Optional.empty(), new int[]{5678}))); + new int[] {0, 1}, + new IntArrayBlock(1, Optional.empty(), new int[] {1234}), + new IntArrayBlock(1, Optional.empty(), new int[] {5678}))); Page rightPage1 = new Page(mapType.createBlockFromKeyValue( Optional.empty(), - new int[]{0, 1}, - new IntArrayBlock(1, Optional.empty(), new int[]{1234}), - new IntArrayBlock(1, Optional.empty(), new int[]{5678}))); + new int[] {0, 1}, + new IntArrayBlock(1, Optional.empty(), new int[] {1234}), + new IntArrayBlock(1, Optional.empty(), new int[] {5678}))); Page rightPage2 = new Page(mapType.createBlockFromKeyValue( Optional.empty(), - new int[]{0, 1}, - new IntArrayBlock(1, Optional.empty(), new int[]{1234}), - new IntArrayBlock(1, Optional.empty(), new int[]{1234}))); + new int[] {0, 1}, + new IntArrayBlock(1, Optional.empty(), new int[] {1234}), + new IntArrayBlock(1, Optional.empty(), new int[] {1234}))); // This works because MapType is comparable. assertThat(strategy.rowEqualsRow(0, leftPage, 0, rightPage1)).isTrue(); @@ -108,7 +108,7 @@ public void testRowEqualsRowWithMapType() @Test public void testCompareSortChannelPositionsWithIntegerType() { - Block block = new IntArrayBlock(3, Optional.empty(), new int[]{1234, 5678, 1234}); + Block block = new IntArrayBlock(3, Optional.empty(), new int[] {1234, 5678, 1234}); SimplePagesHashStrategy strategy = createSimplePagesHashStrategy(INTEGER, ImmutableList.of(block)); // This works because IntegerType is orderable. @@ -123,9 +123,9 @@ public void testCompareSortChannelPositionsWithMapType() MapType mapType = new MapType(INTEGER, INTEGER, new TypeOperators()); Block block = mapType.createBlockFromKeyValue( Optional.empty(), - new int[]{0, 1}, - new IntArrayBlock(1, Optional.empty(), new int[]{1234}), - new IntArrayBlock(1, Optional.empty(), new int[]{5678})); + new int[] {0, 1}, + new IntArrayBlock(1, Optional.empty(), new int[] {1234}), + new IntArrayBlock(1, Optional.empty(), new int[] {5678})); SimplePagesHashStrategy strategy = createSimplePagesHashStrategy(mapType, ImmutableList.of(block)); diff --git a/core/trino-main/src/test/java/io/trino/operator/exchange/TestLocalExchange.java b/core/trino-main/src/test/java/io/trino/operator/exchange/TestLocalExchange.java index 9f8cdedef0c0c..fa35534e9ae42 100644 --- a/core/trino-main/src/test/java/io/trino/operator/exchange/TestLocalExchange.java +++ b/core/trino-main/src/test/java/io/trino/operator/exchange/TestLocalExchange.java @@ -1448,7 +1448,7 @@ private static void assertPartitionedRemovePage(LocalExchangeSource source, int Page page = source.removePage(); assertThat(page).isNotNull(); - LocalPartitionGenerator partitionGenerator = new LocalPartitionGenerator(createChannelsHashGenerator(TYPES, new int[]{0}, TYPE_OPERATORS), partitionCount); + LocalPartitionGenerator partitionGenerator = new LocalPartitionGenerator(createChannelsHashGenerator(TYPES, new int[] {0}, TYPE_OPERATORS), partitionCount); for (int position = 0; position < page.getPositionCount(); position++) { assertThat(partitionGenerator.getPartition(page, position)).isEqualTo(partition); } diff --git a/core/trino-main/src/test/java/io/trino/operator/output/TestPositionsAppenderPageBuilder.java b/core/trino-main/src/test/java/io/trino/operator/output/TestPositionsAppenderPageBuilder.java index 542cea8944be0..197e59f951a19 100644 --- a/core/trino-main/src/test/java/io/trino/operator/output/TestPositionsAppenderPageBuilder.java +++ b/core/trino-main/src/test/java/io/trino/operator/output/TestPositionsAppenderPageBuilder.java @@ -50,7 +50,7 @@ public void testFullOnPositionCountLimit() Block rleBlock = RunLengthEncodedBlock.create(VARCHAR, Slices.utf8Slice("test"), 10); Page inputPage = new Page(rleBlock); - IntArrayList positions = IntArrayList.wrap(new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}); + IntArrayList positions = IntArrayList.wrap(new int[] {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}); // Append 32760 positions, just less than MAX_POSITION_COUNT assertEquals(32768, PositionsAppenderPageBuilder.MAX_POSITION_COUNT, "expected MAX_POSITION_COUNT to be 32768"); for (int i = 0; i < 3276; i++) { @@ -85,7 +85,7 @@ public void testFullOnDirectSizeInBytes() Block rleBlock = RunLengthEncodedBlock.create(VARCHAR, Slices.utf8Slice("test"), 10); Page inputPage = new Page(rleBlock); - IntArrayList positions = IntArrayList.wrap(new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}); + IntArrayList positions = IntArrayList.wrap(new int[] {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}); pageBuilder.appendToOutputPartition(inputPage, positions); // 10 positions inserted, size in bytes is still the same since we're in RLE mode but direct size is 10x sizeAccumulator = pageBuilder.computeAppenderSizes(); @@ -124,7 +124,7 @@ public void testFlushUsefulDictionariesOnRelease() Block dictionaryBlock = DictionaryBlock.create(10, valueBlock, new int[10]); Page inputPage = new Page(dictionaryBlock); - pageBuilder.appendToOutputPartition(inputPage, IntArrayList.wrap(new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})); + pageBuilder.appendToOutputPartition(inputPage, IntArrayList.wrap(new int[] {0, 1, 2, 3, 4, 5, 6, 7, 8, 9})); // Dictionary mode appender should report the size of the ID's, but doesn't currently track // the per-position size at all because it would be inefficient assertEquals(Integer.BYTES * 10, pageBuilder.getSizeInBytes()); @@ -140,7 +140,7 @@ public void testFlattenUnhelpfulDictionariesOnRelease() { // Create unhelpful dictionary wrapping Block valueBlock = createRandomBlockForType(VARCHAR, 10, 0.25f); - Block dictionaryBlock = DictionaryBlock.create(10, valueBlock, new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}); + Block dictionaryBlock = DictionaryBlock.create(10, valueBlock, new int[] {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}); Page inputPage = new Page(dictionaryBlock); // Ensure the builder allows the entire value block to be inserted without being full @@ -152,7 +152,7 @@ public void testFlattenUnhelpfulDictionariesOnRelease() List.of(VARCHAR), new PositionsAppenderFactory(new BlockTypeOperators())); - pageBuilder.appendToOutputPartition(inputPage, IntArrayList.wrap(new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})); + pageBuilder.appendToOutputPartition(inputPage, IntArrayList.wrap(new int[] {0, 1, 2, 3, 4, 5, 6, 7, 8, 9})); assertEquals(Integer.BYTES * 10, pageBuilder.getSizeInBytes()); assertFalse(pageBuilder.isFull()); diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/TestDeleteAndInsertMergeProcessor.java b/core/trino-main/src/test/java/io/trino/sql/planner/TestDeleteAndInsertMergeProcessor.java index 3a144d52b8b74..ad0ac8eed07db 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/TestDeleteAndInsertMergeProcessor.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/TestDeleteAndInsertMergeProcessor.java @@ -104,10 +104,10 @@ public void testUpdateAndDeletedMerge() Page inputPage = makePageFromBlocks( 5, Optional.of(rowIdNulls), - new Block[]{ - new LongArrayBlock(5, Optional.of(rowIdNulls), new long[]{2, 0, 1, 2, 2}), // TransactionId - new LongArrayBlock(5, Optional.of(rowIdNulls), new long[]{0, 0, 3, 1, 2}), // rowId - new IntArrayBlock(5, Optional.of(rowIdNulls), new int[]{536870912, 0, 536870912, 536870912, 536870912})}, // bucket + new Block[] { + new LongArrayBlock(5, Optional.of(rowIdNulls), new long[] {2, 0, 1, 2, 2}), // TransactionId + new LongArrayBlock(5, Optional.of(rowIdNulls), new long[] {0, 0, 3, 1, 2}), // rowId + new IntArrayBlock(5, Optional.of(rowIdNulls), new int[] {536870912, 0, 536870912, 536870912, 536870912})}, // bucket new Block[] { // customer makeVarcharArrayBlock("Aaron", "Carol", "Dave", "Dave", "Ed"), @@ -145,9 +145,9 @@ public void testAnotherMergeCase() 5, Optional.of(rowIdNulls), new Block[] { - new LongArrayBlock(5, Optional.of(rowIdNulls), new long[]{2, 0, 1, 2, 2}), // TransactionId - new LongArrayBlock(5, Optional.of(rowIdNulls), new long[]{0, 0, 3, 1, 2}), // rowId - new IntArrayBlock(5, Optional.of(rowIdNulls), new int[]{536870912, 0, 536870912, 536870912, 536870912})}, // bucket + new LongArrayBlock(5, Optional.of(rowIdNulls), new long[] {2, 0, 1, 2, 2}), // TransactionId + new LongArrayBlock(5, Optional.of(rowIdNulls), new long[] {0, 0, 3, 1, 2}), // rowId + new IntArrayBlock(5, Optional.of(rowIdNulls), new int[] {536870912, 0, 536870912, 536870912, 536870912})}, // bucket new Block[] { // customer makeVarcharArrayBlock("Aaron", "Carol", "Dave", "Dave", "Ed"), diff --git a/core/trino-spi/src/main/java/io/trino/spi/block/VariableWidthBlockBuilder.java b/core/trino-spi/src/main/java/io/trino/spi/block/VariableWidthBlockBuilder.java index 59aca4f3b550f..58d819995e589 100644 --- a/core/trino-spi/src/main/java/io/trino/spi/block/VariableWidthBlockBuilder.java +++ b/core/trino-spi/src/main/java/io/trino/spi/block/VariableWidthBlockBuilder.java @@ -32,7 +32,7 @@ public class VariableWidthBlockBuilder implements BlockBuilder { private static final int INSTANCE_SIZE = instanceSize(VariableWidthBlockBuilder.class); - private static final Block NULL_VALUE_BLOCK = new VariableWidthBlock(0, 1, EMPTY_SLICE, new int[]{0, 0}, new boolean[]{true}); + private static final Block NULL_VALUE_BLOCK = new VariableWidthBlock(0, 1, EMPTY_SLICE, new int[] {0, 0}, new boolean[] {true}); private static final int SIZE_IN_BYTES_PER_POSITION = Integer.BYTES + Byte.BYTES; private final BlockBuilderStatus blockBuilderStatus; diff --git a/core/trino-spi/src/main/java/io/trino/spi/type/BooleanType.java b/core/trino-spi/src/main/java/io/trino/spi/type/BooleanType.java index d2195f2c16190..2c8442f26ec1c 100644 --- a/core/trino-spi/src/main/java/io/trino/spi/type/BooleanType.java +++ b/core/trino-spi/src/main/java/io/trino/spi/type/BooleanType.java @@ -64,7 +64,7 @@ public static Block wrapByteArrayAsBooleanBlockWithoutNulls(byte[] booleansAsByt public static Block createBlockForSingleNonNullValue(boolean value) { byte byteValue = value ? (byte) 1 : 0; - return new ByteArrayBlock(1, Optional.empty(), new byte[]{byteValue}); + return new ByteArrayBlock(1, Optional.empty(), new byte[] {byteValue}); } private BooleanType() diff --git a/core/trino-spi/src/test/java/io/trino/spi/block/BenchmarkCopyPositions.java b/core/trino-spi/src/test/java/io/trino/spi/block/BenchmarkCopyPositions.java index c66eade9807be..10166979629ff 100644 --- a/core/trino-spi/src/test/java/io/trino/spi/block/BenchmarkCopyPositions.java +++ b/core/trino-spi/src/test/java/io/trino/spi/block/BenchmarkCopyPositions.java @@ -102,7 +102,7 @@ public void setup() else if (type.equals("ROW(BIGINT)")) { Optional rowIsNull = nullsAllowed ? Optional.of(generateIsNull(POSITIONS)) : Optional.empty(); LongArrayBlock randomLongArrayBlock = new LongArrayBlock(POSITIONS, rowIsNull, new Random(SEED).longs().limit(POSITIONS).toArray()); - block = RowBlock.fromNotNullSuppressedFieldBlocks(POSITIONS, rowIsNull, new Block[]{randomLongArrayBlock}); + block = RowBlock.fromNotNullSuppressedFieldBlocks(POSITIONS, rowIsNull, new Block[] {randomLongArrayBlock}); } } diff --git a/core/trino-spi/src/test/java/io/trino/spi/block/TestColumnarMap.java b/core/trino-spi/src/test/java/io/trino/spi/block/TestColumnarMap.java index 7a625a7f2c316..2b4db2558b214 100644 --- a/core/trino-spi/src/test/java/io/trino/spi/block/TestColumnarMap.java +++ b/core/trino-spi/src/test/java/io/trino/spi/block/TestColumnarMap.java @@ -37,7 +37,7 @@ public class TestColumnarMap { private static final TypeOperators TYPE_OPERATORS = new TypeOperators(); private static final MapType MAP_TYPE = new MapType(VARCHAR, VARCHAR, TYPE_OPERATORS); - private static final int[] MAP_SIZES = new int[]{16, 0, 13, 1, 2, 11, 4, 7}; + private static final int[] MAP_SIZES = new int[] {16, 0, 13, 1, 2, 11, 4, 7}; @Test public void test() diff --git a/core/trino-spi/src/test/java/io/trino/spi/block/TestLazyBlock.java b/core/trino-spi/src/test/java/io/trino/spi/block/TestLazyBlock.java index 30d27f8a34ee9..f0d7293a01ea7 100644 --- a/core/trino-spi/src/test/java/io/trino/spi/block/TestLazyBlock.java +++ b/core/trino-spi/src/test/java/io/trino/spi/block/TestLazyBlock.java @@ -66,7 +66,7 @@ public void testNestedGetLoadedBlock() List actualNotifications = new ArrayList<>(); Block arrayBlock = new IntArrayBlock(2, Optional.empty(), new int[] {0, 1}); LazyBlock lazyArrayBlock = new LazyBlock(2, () -> arrayBlock); - Block rowBlock = RowBlock.fromFieldBlocks(2, new Block[]{lazyArrayBlock}); + Block rowBlock = RowBlock.fromFieldBlocks(2, new Block[] {lazyArrayBlock}); LazyBlock lazyBlock = new LazyBlock(2, () -> rowBlock); LazyBlock.listenForLoads(lazyBlock, actualNotifications::add); diff --git a/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/avro/TestAvroBase.java b/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/avro/TestAvroBase.java index 81426283e0126..988dfdff7c599 100644 --- a/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/avro/TestAvroBase.java +++ b/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/avro/TestAvroBase.java @@ -160,15 +160,15 @@ public abstract class TestAvroBase ALL_TYPES_GENERIC_RECORD = new GenericData.Record(ALL_TYPES_RECORD_SCHEMA); ALL_TYPES_GENERIC_RECORD.put("aBoolean", true); - allTypeBlocks.add(new ByteArrayBlock(1, Optional.empty(), new byte[]{1})); + allTypeBlocks.add(new ByteArrayBlock(1, Optional.empty(), new byte[] {1})); ALL_TYPES_GENERIC_RECORD.put("aInt", 42); - allTypeBlocks.add(new IntArrayBlock(1, Optional.empty(), new int[]{42})); + allTypeBlocks.add(new IntArrayBlock(1, Optional.empty(), new int[] {42})); ALL_TYPES_GENERIC_RECORD.put("aLong", 3400L); - allTypeBlocks.add(new LongArrayBlock(1, Optional.empty(), new long[]{3400L})); + allTypeBlocks.add(new LongArrayBlock(1, Optional.empty(), new long[] {3400L})); ALL_TYPES_GENERIC_RECORD.put("aFloat", 3.14f); - allTypeBlocks.add(new IntArrayBlock(1, Optional.empty(), new int[]{floatToIntBits(3.14f)})); + allTypeBlocks.add(new IntArrayBlock(1, Optional.empty(), new int[] {floatToIntBits(3.14f)})); ALL_TYPES_GENERIC_RECORD.put("aDouble", 9.81); - allTypeBlocks.add(new LongArrayBlock(1, Optional.empty(), new long[]{doubleToLongBits(9.81)})); + allTypeBlocks.add(new LongArrayBlock(1, Optional.empty(), new long[] {doubleToLongBits(9.81)})); ALL_TYPES_GENERIC_RECORD.put("aString", A_STRING_VALUE); allTypeBlocks.add(new VariableWidthBlock(1, Slices.utf8Slice(A_STRING_VALUE), new int[] {0, Slices.utf8Slice(A_STRING_VALUE).length()}, Optional.empty())); ALL_TYPES_GENERIC_RECORD.put("aBytes", A_BYTES_VALUE); diff --git a/lib/trino-parquet/src/test/java/io/trino/parquet/TestTupleDomainParquetPredicate.java b/lib/trino-parquet/src/test/java/io/trino/parquet/TestTupleDomainParquetPredicate.java index 04cefc95771c9..0e6e7ada80179 100644 --- a/lib/trino-parquet/src/test/java/io/trino/parquet/TestTupleDomainParquetPredicate.java +++ b/lib/trino-parquet/src/test/java/io/trino/parquet/TestTupleDomainParquetPredicate.java @@ -474,7 +474,7 @@ private void testTimestampInt64(TimeUnit timeUnit, int precision, LocalDateTime .as(LogicalTypeAnnotation.timestampType(false, timeUnit)) .named("TimestampColumn"); - ColumnDescriptor columnDescriptor = new ColumnDescriptor(new String[]{}, type, 0, 0); + ColumnDescriptor columnDescriptor = new ColumnDescriptor(new String[] {}, type, 0, 0); TimestampType timestampType = createTimestampType(precision); assertThat(getDomain(columnDescriptor, timestampType, 0, null, ID, UTC)).isEqualTo(all(timestampType)); LocalDateTime maxTime = baseTime.plus(Duration.ofMillis(50)); @@ -719,7 +719,7 @@ public void testColumnIndexWithNoNullsCount() private ColumnDescriptor createColumnDescriptor(PrimitiveTypeName typeName, String columnName) { - return new ColumnDescriptor(new String[]{}, new PrimitiveType(REQUIRED, typeName, columnName), 0, 0); + return new ColumnDescriptor(new String[] {}, new PrimitiveType(REQUIRED, typeName, columnName), 0, 0); } private TupleDomain getEffectivePredicate(ColumnDescriptor column, VarcharType type, Slice value) diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeWriter.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeWriter.java index 7c27b8d151cd3..9fa930f2297f2 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeWriter.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeWriter.java @@ -350,7 +350,7 @@ public Block apply(Block block) RowBlock rowBlock = (RowBlock) runLengthEncodedBlock.getValue(); RowBlock newRowBlock = RowBlock.fromNotNullSuppressedFieldBlocks( 1, - rowBlock.isNull(0) ? Optional.of(new boolean[]{true}) : Optional.empty(), + rowBlock.isNull(0) ? Optional.of(new boolean[] {true}) : Optional.empty(), coerceFields(rowBlock.getFieldBlocks())); return RunLengthEncodedBlock.create(newRowBlock, runLengthEncodedBlock.getPositionCount()); } diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/transactionlog/checkpoint/TestCheckpointWriter.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/transactionlog/checkpoint/TestCheckpointWriter.java index b928ae46be54d..0ad6e8b06f355 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/transactionlog/checkpoint/TestCheckpointWriter.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/transactionlog/checkpoint/TestCheckpointWriter.java @@ -242,11 +242,11 @@ public void testCheckpointWriteReadParquetStatisticsRoundtrip() ProtocolEntry protocolEntry = new ProtocolEntry(10, 20, Optional.empty(), Optional.empty()); TransactionEntry transactionEntry = new TransactionEntry("appId", 1, 1001); - Block[] minMaxRowFieldBlocks = new Block[]{ + Block[] minMaxRowFieldBlocks = new Block[] { nativeValueToBlock(IntegerType.INTEGER, 1L), nativeValueToBlock(createUnboundedVarcharType(), utf8Slice("a")) }; - Block[] nullCountRowFieldBlocks = new Block[]{ + Block[] nullCountRowFieldBlocks = new Block[] { nativeValueToBlock(BigintType.BIGINT, 0L), nativeValueToBlock(BigintType.BIGINT, 15L) }; @@ -359,11 +359,11 @@ public void testDisablingRowStatistics() ImmutableMap.of(), 1000); ProtocolEntry protocolEntry = new ProtocolEntry(10, 20, Optional.empty(), Optional.empty()); - Block[] minMaxRowFieldBlocks = new Block[]{ + Block[] minMaxRowFieldBlocks = new Block[] { nativeValueToBlock(IntegerType.INTEGER, 1L), nativeValueToBlock(createUnboundedVarcharType(), utf8Slice("a")) }; - Block[] nullCountRowFieldBlocks = new Block[]{ + Block[] nullCountRowFieldBlocks = new Block[] { nativeValueToBlock(BigintType.BIGINT, 0L), nativeValueToBlock(BigintType.BIGINT, 15L) }; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/coercions/CoercionUtils.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/coercions/CoercionUtils.java index fcbe2aa6fc257..24c39905dcb25 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/coercions/CoercionUtils.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/coercions/CoercionUtils.java @@ -440,7 +440,7 @@ public Block apply(Block block) RowBlock rowBlock = (RowBlock) runLengthEncodedBlock.getValue(); RowBlock newRowBlock = RowBlock.fromNotNullSuppressedFieldBlocks( 1, - rowBlock.isNull(0) ? Optional.of(new boolean[]{true}) : Optional.empty(), + rowBlock.isNull(0) ? Optional.of(new boolean[] {true}) : Optional.empty(), coerceFields(rowBlock.getFieldBlocks())); return RunLengthEncodedBlock.create(newRowBlock, runLengthEncodedBlock.getPositionCount()); } diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/procedure/MigrateProcedure.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/procedure/MigrateProcedure.java index 91283376cbb68..7ff3a65bf742e 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/procedure/MigrateProcedure.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/procedure/MigrateProcedure.java @@ -232,7 +232,7 @@ public void doMigrate(ConnectorSession session, String schemaName, String tableN ImmutableList.Builder dataFilesBuilder = ImmutableList.builder(); if (hiveTable.getPartitionColumns().isEmpty()) { log.debug("Building data files from %s", location); - dataFilesBuilder.addAll(buildDataFiles(session, recursive, storageFormat, location, partitionSpec, new PartitionData(new Object[]{}), schema)); + dataFilesBuilder.addAll(buildDataFiles(session, recursive, storageFormat, location, partitionSpec, new PartitionData(new Object[0]), schema)); } else { Map> partitions = listAllPartitions(metastore, hiveTable); diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorSmokeTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorSmokeTest.java index df10f40055a19..d196f209612e9 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorSmokeTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorSmokeTest.java @@ -132,7 +132,7 @@ public void testDeleteRowsConcurrently() ExecutorService executor = newFixedThreadPool(threads); List rows = ImmutableList.of("(1, 0, 0, 0)", "(0, 1, 0, 0)", "(0, 0, 1, 0)", "(0, 0, 0, 1)"); - String[] expectedErrors = new String[]{"Failed to commit Iceberg update to table:", "Failed to replace table due to concurrent updates:"}; + String[] expectedErrors = new String[] {"Failed to commit Iceberg update to table:", "Failed to replace table due to concurrent updates:"}; try (TestTable table = new TestTable( getQueryRunner()::execute, "test_concurrent_delete", diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergV2.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergV2.java index a7e9b8f42650a..b75eef99f21dc 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergV2.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergV2.java @@ -206,7 +206,7 @@ public void testV2TableWithEqualityDelete() String tableName = "test_v2_equality_delete" + randomNameSuffix(); assertUpdate("CREATE TABLE " + tableName + " AS SELECT * FROM tpch.tiny.nation", 25); Table icebergTable = loadTable(tableName); - writeEqualityDeleteToNationTable(icebergTable, Optional.of(icebergTable.spec()), Optional.of(new PartitionData(new Long[]{1L}))); + writeEqualityDeleteToNationTable(icebergTable, Optional.of(icebergTable.spec()), Optional.of(new PartitionData(new Long[] {1L}))); assertQuery("SELECT * FROM " + tableName, "SELECT * FROM nation WHERE regionkey != 1"); // nationkey is before the equality delete column in the table schema, comment is after assertQuery("SELECT nationkey, comment FROM " + tableName, "SELECT nationkey, comment FROM nation WHERE regionkey != 1"); @@ -235,7 +235,7 @@ public void testV2TableWithEqualityDeleteWhenColumnIsNested() "SELECT regionkey, ARRAY[1,2] array_column, MAP(ARRAY[1], ARRAY[2]) map_column, " + "CAST(ROW(1, 2e0) AS ROW(x BIGINT, y DOUBLE)) row_column FROM tpch.tiny.nation", 25); Table icebergTable = loadTable(tableName); - writeEqualityDeleteToNationTable(icebergTable, Optional.of(icebergTable.spec()), Optional.of(new PartitionData(new Long[]{1L}))); + writeEqualityDeleteToNationTable(icebergTable, Optional.of(icebergTable.spec()), Optional.of(new PartitionData(new Long[] {1L}))); assertQuery("SELECT array_column[1], map_column[1], row_column.x FROM " + tableName, "SELECT 1, 2, 1 FROM nation WHERE regionkey != 1"); } @@ -247,7 +247,7 @@ public void testOptimizingV2TableRemovesEqualityDeletesWhenWholeTableIsScanned() assertUpdate("CREATE TABLE " + tableName + " WITH (partitioning = ARRAY['regionkey']) AS SELECT * FROM tpch.tiny.nation", 25); Table icebergTable = loadTable(tableName); assertThat(icebergTable.currentSnapshot().summary()).containsEntry("total-equality-deletes", "0"); - writeEqualityDeleteToNationTable(icebergTable, Optional.of(icebergTable.spec()), Optional.of(new PartitionData(new Long[]{1L}))); + writeEqualityDeleteToNationTable(icebergTable, Optional.of(icebergTable.spec()), Optional.of(new PartitionData(new Long[] {1L}))); List initialActiveFiles = getActiveFiles(tableName); query("ALTER TABLE " + tableName + " EXECUTE OPTIMIZE"); assertQuery("SELECT * FROM " + tableName, "SELECT * FROM nation WHERE regionkey != 1"); @@ -267,7 +267,7 @@ public void testOptimizingV2TableDoesntRemoveEqualityDeletesWhenOnlyPartOfTheTab Table icebergTable = loadTable(tableName); assertThat(icebergTable.currentSnapshot().summary()).containsEntry("total-equality-deletes", "0"); List initialActiveFiles = getActiveFiles(tableName); - writeEqualityDeleteToNationTable(icebergTable, Optional.of(icebergTable.spec()), Optional.of(new PartitionData(new Long[]{1L}))); + writeEqualityDeleteToNationTable(icebergTable, Optional.of(icebergTable.spec()), Optional.of(new PartitionData(new Long[] {1L}))); query("ALTER TABLE " + tableName + " EXECUTE OPTIMIZE WHERE regionkey != 1"); assertQuery("SELECT * FROM " + tableName, "SELECT * FROM nation WHERE regionkey != 1"); // nationkey is before the equality delete column in the table schema, comment is after @@ -284,7 +284,7 @@ public void testSelectivelyOptimizingLeavesEqualityDeletes() String tableName = "test_selectively_optimizing_leaves_eq_deletes_" + randomNameSuffix(); assertUpdate("CREATE TABLE " + tableName + " WITH (partitioning = ARRAY['nationkey']) AS SELECT * FROM tpch.tiny.nation", 25); Table icebergTable = loadTable(tableName); - writeEqualityDeleteToNationTable(icebergTable, Optional.of(icebergTable.spec()), Optional.of(new PartitionData(new Long[]{1L}))); + writeEqualityDeleteToNationTable(icebergTable, Optional.of(icebergTable.spec()), Optional.of(new PartitionData(new Long[] {1L}))); query("ALTER TABLE " + tableName + " EXECUTE OPTIMIZE WHERE nationkey < 5"); assertQuery("SELECT * FROM " + tableName, "SELECT * FROM nation WHERE regionkey != 1 OR nationkey != 1"); assertThat(loadTable(tableName).currentSnapshot().summary()).containsEntry("total-equality-deletes", "1"); @@ -420,7 +420,7 @@ public void testOptimizingWholeTableRemovesEqualityDeletes() String tableName = "test_optimizing_whole_table_removes_eq_deletes_" + randomNameSuffix(); assertUpdate("CREATE TABLE " + tableName + " WITH (partitioning = ARRAY['nationkey']) AS SELECT * FROM tpch.tiny.nation", 25); Table icebergTable = loadTable(tableName); - writeEqualityDeleteToNationTable(icebergTable, Optional.of(icebergTable.spec()), Optional.of(new PartitionData(new Long[]{1L}))); + writeEqualityDeleteToNationTable(icebergTable, Optional.of(icebergTable.spec()), Optional.of(new PartitionData(new Long[] {1L}))); query("ALTER TABLE " + tableName + " EXECUTE OPTIMIZE"); assertQuery("SELECT * FROM " + tableName, "SELECT * FROM nation WHERE regionkey != 1 OR nationkey != 1"); assertThat(loadTable(tableName).currentSnapshot().summary()).containsEntry("total-equality-deletes", "0"); @@ -453,7 +453,7 @@ public void testOptimizingPartitionsOfV2TableWithGlobalEqualityDeleteFile() assertUpdate("CREATE TABLE " + tableName + " WITH (partitioning = ARRAY['regionkey']) AS SELECT * FROM tpch.tiny.nation", 25); Table icebergTable = loadTable(tableName); assertThat(icebergTable.currentSnapshot().summary()).containsEntry("total-equality-deletes", "0"); - writeEqualityDeleteToNationTable(icebergTable, Optional.of(icebergTable.spec()), Optional.of(new PartitionData(new Long[]{1L}))); + writeEqualityDeleteToNationTable(icebergTable, Optional.of(icebergTable.spec()), Optional.of(new PartitionData(new Long[] {1L}))); List initialActiveFiles = getActiveFiles(tableName); assertQuery("SELECT * FROM " + tableName, "SELECT * FROM nation WHERE regionkey != 1"); query("ALTER TABLE " + tableName + " EXECUTE OPTIMIZE WHERE regionkey != 1"); @@ -464,8 +464,8 @@ public void testOptimizingPartitionsOfV2TableWithGlobalEqualityDeleteFile() List updatedFiles = getActiveFiles(tableName); assertThat(updatedFiles) .doesNotContain(initialActiveFiles.stream() - .filter(path -> !path.contains("regionkey=1")) - .toArray(String[]::new)); + .filter(path -> !path.contains("regionkey=1")) + .toArray(String[]::new)); } @Test @@ -955,7 +955,7 @@ private void writeEqualityDeleteToNationTableWithDeleteColumns( List equalityDeleteFieldIds = deleteColumns.stream() .map(name -> deleteRowSchema.findField(name).fieldId()) .collect(toImmutableList()); - writeEqualityDeleteToNationTableWithDeleteColumns(icebergTable, partitionSpec, partitionData, overwriteValues, deleteRowSchema, equalityDeleteFieldIds); + writeEqualityDeleteToNationTableWithDeleteColumns(icebergTable, partitionSpec, partitionData, overwriteValues, deleteRowSchema, equalityDeleteFieldIds); } private void writeEqualityDeleteToNationTableWithDeleteColumns( diff --git a/plugin/trino-kafka/src/test/java/io/trino/plugin/kafka/protobuf/TestProtobufEncoder.java b/plugin/trino-kafka/src/test/java/io/trino/plugin/kafka/protobuf/TestProtobufEncoder.java index 41dc2fae1c8ec..da9b8bae14ca9 100644 --- a/plugin/trino-kafka/src/test/java/io/trino/plugin/kafka/protobuf/TestProtobufEncoder.java +++ b/plugin/trino-kafka/src/test/java/io/trino/plugin/kafka/protobuf/TestProtobufEncoder.java @@ -396,7 +396,7 @@ private void testNestedStructuralDataTypes(String stringData, Integer integerDat BlockBuilder mapBlockBuilder = mapType.createBlockBuilder(null, 1); Block mapBlock = mapType.createBlockFromKeyValue( Optional.empty(), - new int[]{0, 1}, + new int[] {0, 1}, nativeValueToBlock(VARCHAR, utf8Slice("Key")), rowBlockBuilder.build()); mapType.appendTo( @@ -409,12 +409,12 @@ private void testNestedStructuralDataTypes(String stringData, Integer integerDat Block arrayBlock = fromElementBlock( 1, Optional.empty(), - new int[]{0, rowBlockBuilder.getPositionCount()}, + new int[] {0, rowBlockBuilder.getPositionCount()}, rowBlockBuilder.build()); listType.appendTo(arrayBlock, 0, listBlockBuilder); BlockBuilder nestedBlockBuilder = nestedRowType.createBlockBuilder(null, 1); - Block rowBlock = fromFieldBlocks(1, new Block[]{listBlockBuilder.build(), mapBlockBuilder.build(), rowBlockBuilder.build()}); + Block rowBlock = fromFieldBlocks(1, new Block[] {listBlockBuilder.build(), mapBlockBuilder.build(), rowBlockBuilder.build()}); nestedRowType.appendTo(rowBlock, 0, nestedBlockBuilder); rowEncoder.appendColumnValue(nestedBlockBuilder.build(), 0); diff --git a/plugin/trino-postgresql/src/main/java/io/trino/plugin/postgresql/PostgreSqlClient.java b/plugin/trino-postgresql/src/main/java/io/trino/plugin/postgresql/PostgreSqlClient.java index 20482f0cfeb99..d6247d866ed82 100644 --- a/plugin/trino-postgresql/src/main/java/io/trino/plugin/postgresql/PostgreSqlClient.java +++ b/plugin/trino-postgresql/src/main/java/io/trino/plugin/postgresql/PostgreSqlClient.java @@ -1350,7 +1350,7 @@ private ObjectReadFunction varcharMapReadFunction() varcharMapType.getValueType().writeSlice(valueBlockBuilder, utf8Slice(entry.getValue())); } } - MapBlock mapBlock = varcharMapType.createBlockFromKeyValue(Optional.empty(), new int[]{0, map.size()}, keyBlockBuilder.build(), valueBlockBuilder.build()); + MapBlock mapBlock = varcharMapType.createBlockFromKeyValue(Optional.empty(), new int[] {0, map.size()}, keyBlockBuilder.build(), valueBlockBuilder.build()); return varcharMapType.getObject(mapBlock, 0); }); } diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveRequireQueryPartitionsFilter.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveRequireQueryPartitionsFilter.java index 31e6ebefa986c..8d996b1900799 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveRequireQueryPartitionsFilter.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveRequireQueryPartitionsFilter.java @@ -71,7 +71,7 @@ public void testRequiresQueryPartitionFilterOnSpecificSchema(String queryPartiti @DataProvider public Object[][] queryPartitionFilterRequiredSchemasDataProvider() { - return new Object[][]{ + return new Object[][] { {"ARRAY['default']"}, {"ARRAY['DEFAULT']"}, {"ARRAY['deFAUlt']"} From 082c8a69c9791fb7d7c049789fbad09a63e48d96 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 10 Jan 2024 15:55:53 +0100 Subject: [PATCH 344/350] Remove redundant lambda braces --- .../trino/spiller/TestFileSingleStreamSpillerFactory.java | 2 +- .../trino/sql/planner/assertions/BasePushdownPlanTest.java | 2 +- .../java/io/trino/plugin/jdbc/TestJdbcTableProperties.java | 4 ++-- .../plugin/iceberg/TestIcebergOrcMetricsCollection.java | 6 ++---- 4 files changed, 6 insertions(+), 8 deletions(-) diff --git a/core/trino-main/src/test/java/io/trino/spiller/TestFileSingleStreamSpillerFactory.java b/core/trino-main/src/test/java/io/trino/spiller/TestFileSingleStreamSpillerFactory.java index c53d8bfcf75da..1d4d283bf0e24 100644 --- a/core/trino-main/src/test/java/io/trino/spiller/TestFileSingleStreamSpillerFactory.java +++ b/core/trino-main/src/test/java/io/trino/spiller/TestFileSingleStreamSpillerFactory.java @@ -218,7 +218,7 @@ public void testCacheInvalidatedOnBadDisk() // Set second spiller path to read-only after initialization to emulate a disk failing during runtime setPosixFilePermissions(spillPath2.toPath(), ImmutableSet.of(PosixFilePermission.OWNER_READ)); - assertThatThrownBy(() -> { getUnchecked(singleStreamSpiller2.spill(page)); }) + assertThatThrownBy(() -> getUnchecked(singleStreamSpiller2.spill(page))) .isInstanceOf(com.google.common.util.concurrent.UncheckedExecutionException.class) .hasMessageContaining("Failed to spill pages"); spillers.add(singleStreamSpiller2); diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/assertions/BasePushdownPlanTest.java b/core/trino-main/src/test/java/io/trino/sql/planner/assertions/BasePushdownPlanTest.java index 325b4a5d26317..9ba223af6a669 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/assertions/BasePushdownPlanTest.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/assertions/BasePushdownPlanTest.java @@ -29,7 +29,7 @@ public abstract class BasePushdownPlanTest { protected Optional getTableHandle(Session session, QualifiedObjectName objectName) { - return getQueryRunner().inTransaction(session, transactionSession -> { return getQueryRunner().getMetadata().getTableHandle(transactionSession, objectName); }); + return getQueryRunner().inTransaction(session, transactionSession -> getQueryRunner().getMetadata().getTableHandle(transactionSession, objectName)); } protected Map getColumnHandles(Session session, QualifiedObjectName tableName) diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcTableProperties.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcTableProperties.java index 8c69032467c7b..8dc5ab420eaf0 100644 --- a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcTableProperties.java +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcTableProperties.java @@ -56,7 +56,7 @@ public Map getTableProperties(ConnectorSession session, JdbcTabl @Test public void testGetTablePropertiesIsNotCalledForSelect() { - onGetTableProperties = () -> { fail("Unexpected call of: getTableProperties"); }; + onGetTableProperties = () -> fail("Unexpected call of: getTableProperties"); assertUpdate("CREATE TABLE copy_of_nation AS SELECT * FROM nation", 25); assertQuerySucceeds("SELECT * FROM copy_of_nation"); assertQuerySucceeds("SELECT nationkey FROM copy_of_nation"); @@ -66,7 +66,7 @@ public void testGetTablePropertiesIsNotCalledForSelect() public void testGetTablePropertiesIsCalled() { AtomicInteger counter = new AtomicInteger(); - onGetTableProperties = () -> { counter.incrementAndGet(); }; + onGetTableProperties = () -> counter.incrementAndGet(); assertQuerySucceeds("SHOW CREATE TABLE nation"); assertThat(counter.get()).isOne(); } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergOrcMetricsCollection.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergOrcMetricsCollection.java index fd003829577a5..03f440472ac52 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergOrcMetricsCollection.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergOrcMetricsCollection.java @@ -180,10 +180,8 @@ public void testMetrics() assertThat(datafile.getRecordCount()).isEqualTo(1); assertThat(datafile.getValueCounts().size()).isEqualTo(1); assertThat(datafile.getNullValueCounts().size()).isEqualTo(1); - datafile.getUpperBounds().forEach((k, v) -> { - assertThat(v.length()).isEqualTo(10); }); - datafile.getLowerBounds().forEach((k, v) -> { - assertThat(v.length()).isEqualTo(10); }); + datafile.getUpperBounds().forEach((k, v) -> assertThat(v.length()).isEqualTo(10)); + datafile.getLowerBounds().forEach((k, v) -> assertThat(v.length()).isEqualTo(10)); // keep both c1 and c2 metrics assertUpdate("create table c_metrics (c1 varchar, c2 varchar)"); From 7e463340e89169218da116653e394d715d75ea23 Mon Sep 17 00:00:00 2001 From: sheajamba Date: Thu, 11 Jan 2024 09:37:54 -0500 Subject: [PATCH 345/350] Fix sentence in SQL routines docs --- docs/src/main/sphinx/routines/introduction.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/src/main/sphinx/routines/introduction.md b/docs/src/main/sphinx/routines/introduction.md index 0174a3d2f2121..f444c29374a2f 100644 --- a/docs/src/main/sphinx/routines/introduction.md +++ b/docs/src/main/sphinx/routines/introduction.md @@ -158,8 +158,8 @@ terms of memory and processing. Take the following considerations into account when writing and running SQL routines: * Some checks for the runtime behavior of routines are in place. For example, - routines that use take longer to process than a hardcoded threshold are automatically - terminated. + routines that take longer to process than a hardcoded threshold are + automatically terminated. * Avoid creation of arrays in a looping construct. Each iteration creates a separate new array with all items and copies the data for each modification, leaving the prior array in memory for automated clean up later. Use a [lambda From 2e17f45ec398fb028c36e53e737e4c8b60a6ef58 Mon Sep 17 00:00:00 2001 From: yuuteng Date: Tue, 5 Dec 2023 21:22:20 +0100 Subject: [PATCH 346/350] Add Snowflake JDBC Connector (#11) Had to redo the connector because all the rebases caused havoc --- core/trino-server/src/main/provisio/trino.xml | 6 + docs/src/main/sphinx/connector.md | 1 + docs/src/main/sphinx/connector/snowflake.md | 96 +++ docs/src/main/sphinx/static/img/snowflake.png | Bin 0 -> 93500 bytes plugin/trino-snowflake/pom.xml | 247 +++++++ .../plugin/snowflake/SnowflakeClient.java | 651 ++++++++++++++++++ .../snowflake/SnowflakeClientModule.java | 95 +++ .../plugin/snowflake/SnowflakeConfig.java | 100 +++ .../plugin/snowflake/SnowflakePlugin.java | 25 + .../snowflake/BaseSnowflakeConnectorTest.java | 610 ++++++++++++++++ .../snowflake/SnowflakeQueryRunner.java | 97 +++ .../plugin/snowflake/TestSnowflakeClient.java | 153 ++++ .../plugin/snowflake/TestSnowflakeConfig.java | 61 ++ .../snowflake/TestSnowflakeConnectorTest.java | 38 + .../plugin/snowflake/TestSnowflakePlugin.java | 33 + .../snowflake/TestSnowflakeTypeMapping.java | 391 +++++++++++ .../snowflake/TestingSnowflakeServer.java | 77 +++ pom.xml | 7 + .../EnvMultinodeAllConnectors.java | 1 + .../environment/EnvMultinodeSnowflake.java | 77 +++ .../launcher/suite/suites/SuiteSnowflake.java | 37 + .../multinode-all/snowflake.properties | 4 + .../multinode-snowflake/snowflake.properties | 4 + .../io/trino/tests/product/TestGroups.java | 1 + .../product/snowflake/TestSnowflake.java | 46 ++ 25 files changed, 2858 insertions(+) create mode 100644 docs/src/main/sphinx/connector/snowflake.md create mode 100644 docs/src/main/sphinx/static/img/snowflake.png create mode 100644 plugin/trino-snowflake/pom.xml create mode 100644 plugin/trino-snowflake/src/main/java/io/trino/plugin/snowflake/SnowflakeClient.java create mode 100644 plugin/trino-snowflake/src/main/java/io/trino/plugin/snowflake/SnowflakeClientModule.java create mode 100644 plugin/trino-snowflake/src/main/java/io/trino/plugin/snowflake/SnowflakeConfig.java create mode 100644 plugin/trino-snowflake/src/main/java/io/trino/plugin/snowflake/SnowflakePlugin.java create mode 100644 plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/BaseSnowflakeConnectorTest.java create mode 100644 plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/SnowflakeQueryRunner.java create mode 100644 plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/TestSnowflakeClient.java create mode 100644 plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/TestSnowflakeConfig.java create mode 100644 plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/TestSnowflakeConnectorTest.java create mode 100644 plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/TestSnowflakePlugin.java create mode 100644 plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/TestSnowflakeTypeMapping.java create mode 100644 plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/TestingSnowflakeServer.java create mode 100644 testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/EnvMultinodeSnowflake.java create mode 100644 testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/suite/suites/SuiteSnowflake.java create mode 100644 testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-all/snowflake.properties create mode 100644 testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-snowflake/snowflake.properties create mode 100644 testing/trino-product-tests/src/main/java/io/trino/tests/product/snowflake/TestSnowflake.java diff --git a/core/trino-server/src/main/provisio/trino.xml b/core/trino-server/src/main/provisio/trino.xml index 2987eb5d00076..9041aba020074 100644 --- a/core/trino-server/src/main/provisio/trino.xml +++ b/core/trino-server/src/main/provisio/trino.xml @@ -296,6 +296,12 @@ + + + + + + diff --git a/docs/src/main/sphinx/connector.md b/docs/src/main/sphinx/connector.md index c741fdedb5479..3c8dc201e7f09 100644 --- a/docs/src/main/sphinx/connector.md +++ b/docs/src/main/sphinx/connector.md @@ -38,6 +38,7 @@ Prometheus Redis Redshift SingleStore +Snowflake SQL Server System Thrift diff --git a/docs/src/main/sphinx/connector/snowflake.md b/docs/src/main/sphinx/connector/snowflake.md new file mode 100644 index 0000000000000..579764534e230 --- /dev/null +++ b/docs/src/main/sphinx/connector/snowflake.md @@ -0,0 +1,96 @@ +# Snowflake connector + +```{raw} html + +``` + +The Snowflake connector allows querying and creating tables in an +external [Snowflake](https://www.snowflake.com/) account. This can be used to join data between +different systems like Snowflake and Hive, or between two different +Snowflake accounts. + +## Configuration + +To configure the Snowflake connector, create a catalog properties file +in `etc/catalog` named, for example, `example.properties`, to +mount the Snowflake connector as the `snowflake` catalog. +Create the file with the following contents, replacing the +connection properties as appropriate for your setup: + +```none +connector.name=snowflake +connection-url=jdbc:snowflake://.snowflakecomputing.com +connection-user=root +connection-password=secret +snowflake.account=account +snowflake.database=database +snowflake.role=role +snowflake.warehouse=warehouse +``` + +### Arrow serialization support + +This is an experimental feature which introduces support for using Apache Arrow +as the serialization format when reading from Snowflake. Please note there are +a few caveats: + +- Using Apache Arrow serialization is disabled by default. In order to enable + it, add `--add-opens=java.base/java.nio=ALL-UNNAMED` to the Trino + {ref}`jvm-config`. + +### Multiple Snowflake databases or accounts + +The Snowflake connector can only access a single database within +a Snowflake account. Thus, if you have multiple Snowflake databases, +or want to connect to multiple Snowflake accounts, you must configure +multiple instances of the Snowflake connector. + +% snowflake-type-mapping: + +## Type mapping + +Trino supports the following Snowflake data types: + +| Snowflake Type | Trino Type | +| -------------- | -------------- | +| `boolean` | `boolean` | +| `tinyint` | `bigint` | +| `smallint` | `bigint` | +| `byteint` | `bigint` | +| `int` | `bigint` | +| `integer` | `bigint` | +| `bigint` | `bigint` | +| `float` | `real` | +| `real` | `real` | +| `double` | `double` | +| `decimal` | `decimal(P,S)` | +| `varchar(n)` | `varchar(n)` | +| `char(n)` | `varchar(n)` | +| `binary(n)` | `varbinary` | +| `varbinary` | `varbinary` | +| `date` | `date` | +| `time` | `time` | +| `timestampntz` | `timestamp` | +| `timestamptz` | `timestampTZ` | +| `timestampltz` | `timestampTZ` | + +Complete list of [Snowflake data types](https://docs.snowflake.com/en/sql-reference/intro-summary-data-types.html). + +(snowflake-sql-support)= + +## SQL support + +The connector provides read access and write access to data and metadata in +a Snowflake database. In addition to the {ref}`globally available +` and {ref}`read operation ` +statements, the connector supports the following features: + +- {doc}`/sql/insert` +- {doc}`/sql/delete` +- {doc}`/sql/truncate` +- {doc}`/sql/create-table` +- {doc}`/sql/create-table-as` +- {doc}`/sql/drop-table` +- {doc}`/sql/alter-table` +- {doc}`/sql/create-schema` +- {doc}`/sql/drop-schema` diff --git a/docs/src/main/sphinx/static/img/snowflake.png b/docs/src/main/sphinx/static/img/snowflake.png new file mode 100644 index 0000000000000000000000000000000000000000..b337bc4d5a779c77e0c63cdf302ba793d235cc2c GIT binary patch literal 93500 zcmeFZ^tXCBVez@@pDk}xL!2C;V%7FtD~dH0D@o zOKvL`@2(zXY>}Njyyxij+)`5`zM28f!&9;}KRZ7gguC!@=iG1UPa!wxl4F%Y1&s^TwtEn$#QA*wHqmw=7fS`SY_# zDr4g~uSM2~gYHa)|4&y)-G)i_YmTH`u8+UX-z12D4-{d14Xby%MsAOPd1^C3Fe>kL zbs|H-SmJ5#7hQJWPf>+8F*>HMp!6ZQ5Q++Xe^$<;B75DtaQjtTn`Qc`W!^_gsi>dh z(?+Y>5L~(Zi7qj9*op-I6jkk%HY*N)v8b6mAsXvS=Nf!?pf@jv>uW0>Qt>>?`qUrG zYlO;uDbVwX@r-_Rb6ayw-5F^(MxY?pDBC5|GWN?t&TFr9vY>Ze<2RyHe!p8Y=W{rc z;7X4wp3I=@AzeL@od@~&H$>OV-67)@`kk(5H^|KLG?h_d@QVyEt1PtofaqXT7~=4K zljNOfssub9e`@+y7eztTWNc-Mo9wQ$%0nX$?jEONJ_muS`$eMHO`O_M3QH zzvc#CWt87t5oyHf5_a~5_QY^y3Mu8waA7%`7pET-QW3mQGA!z(&5a&RjCx#SlePq@6eTciLfNrh^F1qwn=|n%xD|0GoE1@Y>bkLh{`}9KB5_nGn1JtfGA=GmRr-5CSBFwTKxPG#qMi%6YKSx9F7 zshVV6a;#01k#Yg{l^$Yg3w*C}ewnJ~6F#{&B)&CM-91F6TM;~Ob!fnE({+5y+$}i2 z<*R)r@lq9XCnxDB$$Ar|Y%7{^X!swI zBkl=sc_WB0a6_I#D@n`2qDOvd35D-ASqNTX`iIn~;zh}fK3UJ##hmZ=u8alx7K$bQ z)3eePFQ>LLE#y(D_5!`LmdF z$cG?%xcT{(g5YOpQj7tAZ?C94v6p*yxewcus{_U20tcwYQY6DMQGw(M1cq*j)SGp^ zzBRYpxGn~M8vPFzCmR0wY6}$_Kmc6O@BZ{rJ8aM*Iev#9HXGc#`a8iElx{n8A2>g2)ldFEbcA88rdxH}eiS|%C z3^mQRpoN^oe8wNI`1N+LgyHJb0kgoa(XovE@D4yWLFR(%co8J$t&csZSsH6qlzuS( z+bi(-i3^x!H1b1BB9S5LhtwwFdPaZa^_@*-z##d*i(C}gAXtwG1dr27erITf@oxLR z=%XJNhJsn<-2e9&2=wQ{N#1Z0H*w!Hh3BKw7E-BW*LW}-@SiQ`5(ZQe0}Dy8osdiC zO%^ZsVEyV35wId-7-F#0Z{^rG?7vF|$U`|;UKo)ZbMEyAa(F9WU*g^gp<%*UMgLv_ zvl7?nN_#!%N3Rv%`iHKL^4V&dVENVkA?G6jWh_>4tlLHpiW36^r+u)|BOrOdPIo}; zdvyVgKLFz?e3W}K5;@JK-D{T=jrEE8S9uun-?1>Cm?=p0hU@rJv#gRM;BiP583LiN z?+dR!6M~NuaP>O~q(O%Vx;ixwC|4X<-2XhJ@&;I(|J+H#0yyhGcf@gkjr-4?06eK{ zbpN>ny8jkS{J(ctiT;0ruCB=cr&s8GE+FC0f7+x>CNqFQoQr9{f?qoedVPUiWv;56 z$eS`U+yfO+k3HhL3Gww$>TK(gO<*uXUWs-r1=2KI#mOoNLaF}^stX=qg|Ab8|I#pA zO0?R=8;F-A`C~F|c^s$>l=HgwE>z z>^`Xtk8~f{%4!{z?oya_<3IDzXF2Vl#?Ri55xv59aiLhBUVaBm`_@iHzZ#2ow5}7) zSOc3pb&<|TuT74>_yIPWkAucVuEV77GL7;NdJxec>s6l_J>cZPKA@_Iy1OWt!FiKI z&itJiOK@5E@Z%4s)j-oA#RG-Pm1iz}U$oug zw6E4ivC%u}-dpgS&OJXL>T;QF(p{+J2T6$o=9UsP+cCVS1il*m_T%vATL1eK7`n~- zmwhG4m1X!LBQe_T0+`y+?42c+oAyu9n{QLh^|NW22ssNC%)KJhhd{=ryhcs-YR=h? zF*=)Mh{(v_R!TAG&Mw>6WA9h%ttzkVBIt+owV&NwJO-03qgsuc=45i#Z_gi0JXCX3 z2W(bNjGa|U2a1(lnd=KN<-34TY90hYXyGIU0Rw)%O(Qb{A30KF@UJW(ks=9bJwpfM z!5kIA2ntHo`5_JjL1{t>?1o$ztFV$0bUW3w-s8D|J<7nTK?U{so1phhdryhHc{o`q z7;w!0cm0sO_LIZ1i*)*4?Tn?G369Yh^14*A!8pMrp5Pj_(Q6>d(?acd2Sszd7Wk1X zOqcagO<{crj36x;b?*quw9@%sPasf65Z0&U!H?{XGX`QWq-~EHB(sxTq@Ak|5sQ&? z1y(OX9-fzOA-7)RC#YKv@?ODWo4&(?0e)LF>BIoMjdg|KgFTbxPmf_p82{mX23Clc51R zV@_aCVHt?h2=4Ije+hVNtorbtMKQ=5T+r%=_^jX&Fd^r;<~l+?QLT5xy4-UwaTCh! z^yR7|d(Y+9ICG6Xp)JO#MQX_4^g#dKg4@d`<=?iDCFLAk(5D@nwclex30+2u>BR`% z26K){SL^M~xh3>fwfg=s9>}O|72M&$+#s*UUL10O$FDaDW0@4NvD%{Z!Uowof8TDa zjaTqfCBI$(7Rr<{IdpjU`AQe$IwYXx57IQT%ax-G1J=)OAQo?=&&2eQv`qNy)k9ov zgQ`yTE&CQ`o^s`$=T~lwb@FK>?*u-Jr5jm;d zm4+Q^HHY?a-0W0rW~$?zk)Z+QPXYFS4kOP|ciZ=K`jG<2!VJ{}Y)vHTsuj#E2CRz8 z>IsZ7x*dN@2akQ=Bl}}3gJGc{VUR|WRX8m*yoXfm-eBCYoLdOPi#qRgrohv-1nQ^_ z>8p-*tQPg%(+13Tdj8&l%w;94ER>S;IrNb<&&+(^9K%(b+_6%-rh-na2*z2gms>xH z`9hFDemy8PX7^gg-<3H}ca`2abSzN3+14r|pB{BIAr{r~m^n^3mI0x7o8&V?mk3oM z52q;QdoNqN38Y27Xqq$)FYWLND^HWedg!-726g>nKL{bzug2SPD!chSYlT#4=6MvB z%25A%vnHN4HMB-oWDKvO+Vcr(K`#c^#MJ{vJye4A<7iUQ&pPei-<}LSM`CjOy0B67 zbfrY`MVDQqzZQOx21}0b_wDENpVHp({vF8EC#}epO0MgWb0>B(aip@77O?Lwuff2vZrUW8kg^0kyAWte^wDj|t-5S^ zKGp51y=}j6?@3duF8>JuzLYwv;yW)CJutzEA14HL{t_rrp-m%U6VyRlQP9k9QM>UQ zUbF$$s7Y0yDxE{kgeG&`^F3IDfys9N()hgsIYJPfD(KJwhW3#+w}N>*Yx=fcS?Xf1 zbuyU5HtWIynq(mXy#wTh_g4*>YsMJs-AZH1Lh-m1NzPtlqkpX9&bmbd`$M#NRl?ZT zq2UuspQ0Xum{TH1s9d6c@qE(Yn)yYXP54DMs^ViZdvgW@d;Cdy1JUOj!N+pNB3o=; z-~f64oJTP;6-_JrLkzlmB)yNPRhkb59&J@M-og0D+282&+YovGCPJOBa& zhzf)lU$QtzN)scH0hJ+%J|(f`^itOhIZrnh<%o_hnDK$Tnl z_j{b671QIah16J=jBnbfOc0Kz-DioW*_$iRs6|ZJo{lU=p1uYj8M);hI_MF>O^?l> zf)}prPJT<*Rm#lWgqyXDh)&u$NG~92Jy|;|zH=M82$6cGFf4I96XVlKh^+aU0>t)2 zfLEcoCiaF&0UXelMQmQ3_ihC+L$bq}{xM;ivRXGnR8ofB4ihY3Yl8nhX)0mnrW=%T ztIR3mA2|#k><6zXE{~qiIYR$Ya(-#_*lkCm8$Tj0-T@Pnz=@BsL3N+$+3(YKwR>}B zk6`8YEcJ$jl^Jik!}>js{ClHrwTG;jv+IbxF};@N25dA{_GZ&7y@hl>nf0Y$-NVOHo{;=4O>aRg-dZ*Jwtc_6A3B#J9gqEX*ufVKOoKv}|_HiPzDddze$=B7q zzE+HyLOe^3X8Ob}VLW73IThAJn&%D*pa8V|3{B;Hcqmjp`ZJgECNi@(Z~~riYF3nB zxP61`l+-i2hqRxnRCNm07o(xf++Ljg*Lfm>tPjKbQm36f%A8L~TROE?t3FKgvkf?F z4=PuHC%oS#ObaUb`{_UaZh;;Jv9>m|(NM7;k%ewOIOveqr6vV&xd2n2pz89aVTpY$ z&nomtqqZ6hH>$Kt!v;x~)IIXo-_zw^$v$snC?q>jn z)U`AuMkPE#JHp5xW#`J{nh&bS*UHVi^RQZju%jaQ!{ z|7>zV3?Ex^YD3ja8^oo7^)0;8Q9?u6&e>s>x$A&@AG?kg>VQjRwZ8akOaiT4NBIU@ z+%6rYf;ucKi_cHq)r`N_Wa?gHiZAvtrdRnP0Z53)${NuBgePrEJihY8df`zXNtS#M z|BPmgtOR^|%dUl%yB`cV^Xh9t_cs6enxdEM+YP@qBa8L6IstvkknjaDt8&lWmE-{Y z8kEaU0+NTlhe)^#)~ytJ5x!s=INH%AbLS(v6+p&=VF>gXkfjweacbARHZmixPp zt=Vg(P5_;%1>PGdpM=5vTF>C>=29 zl^>Zc)(>Z$L6FaHJaEPKhW#~oLiB88x8j_-f#`MSU151nOmEg?z+x`+v8vr4GE8JZ zPjtQR5bOU&8pTMI=7k9sPJU!LO*3t6T6ZH3(}?GKTlR_#(+mU((FD9(W%5I(u7lh$ zq8^7dCd}PS;uz!P(&?#5YiD3RB!Y>E#RD)wnpckwdaoBfr&C7vHNmBt;l*j8xGN38 z|48e8zZw!&Y#fI4U|%BAt^H>j)7u~i9td1FsEl{NXzKY2#`>=;{!Oh+C2j@r)VolWs&*WyHS9vLUQ?LTGGZ-=an3Okb}|Gj_#6LK zDKEnyZU%ANr2(WHS@)?RqymE64$f-5_BU4d$ zQs@Hn%Mg>Ciss?|O6K?_^aOOr`z`BXQ@Y$o4$L1N$|Y|jFxP46z_TkN-hWrpz)WcLnA+{QY= zL-*vLmNAV~D|&@cVVVJa&LmiMF*RxCYXyC;988(8Ga%g5VUz2HVk68r6jRAlhWU%@UWrL(MQW+Q90PY$k z5K6!f!)8v!SQ>iVN78E1@DntH8us%kiNA`5oO_+IqnV}4n)gj-&+hbkZ`gi=^3*)b z`bdP`;v}oHtC|oD?IzvFwVEuP1vTcKUttEG6smuzNfq0ohw* z-*-|4Y1`<-87{CSK<)bvGY?DnQuEzK9OqJ$;eSK;A%RiJF4agasnyCG3>Ot7r(-}k zy|rn64o=%R&qbNmfnkhTAP+ZzBeA+%N@6Gncpr_z92a-~Wl!eShPl=sFH=SBeh4HN zaW@Pbw9r)6uMA^&juf+NTqxiG6+0WwkDF#YmLRbq*q{IdvXI&TDX4TV^X}1L7=PrZ zRiQgtZ(Ps=a$q&U_nj3pcE&$gpOzeJ3VJ?biGx6HQ%yFc3TupiHR|nK2ZT{Sf1H4Q ztD2R_CY{w@@kC%&pY9#t=Mfa(`ytiPEkK?Fc)Vfraoic_|A_ASwoAo`etTEYN2xft zZvQi@E{0U`SRAR%j|r2FES~UU`dh$wUFxo`e$W0URg9TG1QJGkPal{yUX^=y0W<(< zVBvu-raGj6@O>Z8L2Bv!zq%=;4U+@?fr80m(9(S{ievK2iF#bJ? zx2e4)b)w9Vw-uf3q9-qjqX*nzDJ|AADO5kub%6hpnuD*xQuL+x0Q3bcRIvGdsP`zJ z-On023|;rbu|6TVjQ42c3*f*AhfA`^hL&F`aN;JZf0xe*@EJ1|N|;Bs6!zo#f{V@= z2!V!aYJkN~jA6?!Ce!Q0I6yYu)}lPB$lgI$;@|a?$3AnbWMCVZP4TTxKi9g<*r&l% zLABWROZ*}v9*2w&J%D{xRus;Ai#1Mh8U7t|v1~KfiMo3s;g7?OIfW`+0NYC#tf@N? zkTgo5Dv;MexjMzPu2U(8%7K`f04(FkRkM8`l&F1%AyB*{6XeL`l1+gCT=`$)QWT&7 z-FXzDGuu4vcY1ta*vm^2V@L!cCA(c}Ba|H|dZGn^+X3>j*M2H38-90DXBN{z-NTIa zz?{sjrbA3_i}p{$5U79~N&%7yG+eC@9;o@hh8)^c)0Ol$+a(zqf_On7&2=4jf30G6 zzviBg0)N_ay*Ed!h}+k{slD7YY&?*=>QZ1{)ZoZb8H^Nu1C&OQuq#e9UQj=g!~wqZ zr_pU8b(=Z@O{RvJPyezm%%*t)ImH z;zG%{VMK%qJ2Qxi9*`KmfacsHAoMMU7RFx(HuoFgN6e`xAUHo)&w1o=y z$pf4K3>En|Z~`O?y(I<|r)oM0lJ3k*$N zJz;-OnJUrOoH(?~fmM1OU?~HfxYv`T!dJI{xMy;2OOhd(4b4J)4HO&Jy(-UQi#hEMPSYklXtSkek99(ytlI{&SRuR4+?k~?_N{k zT*FWE<5|lV=^SPljHnJyU(<=l);YXsb-CJ%u2Nj(FRGM#A&DbED2;1G8n*xFKIl_7 z*H6>|ZqYf9jXFF9a9=Jj4J@9SCaTWYHL?&&OlMDD;(U7h+aBt2J&B(eDyW5$0Vcc} z=+g6e3?QYVI$}=jZ$sjV%*Ua(M3jdy-!Zeji>fQF5<|%AlMsAK>~#r4Mr!~vwgb5? z-g>IfT4t=Ncj^aphPlLlZH4_3kXI?$GsT>dv_usUxdgzysDYt)0U-?!BvLHqCg>-Q z@I&5=aXf|)<#Y`7Sndjid-X97)lZ{%ddMlJdlhK9Tr!*!yizb{s=b{vDiv`{qLsZ;?dk z(`0mdzbm=~U~SF5s;-4p+R_MIH-Ih(-5=6(`6R4tBy|WDyh185i>NX6@!d^U^@G6d zqro((foPapn zm|);cnPr~H_QJ-3VaLGK^V^I}1qBh-g3d!MTlD{mp~O4P%*xl&tnmME7?84l@djf$ z=IQI5`e?0+Lmm)c3;4EB?#k_LQOth6tgkoM56lE}1?Z}X)wd|HfsQMXmwP>~0@9YQ zS63fiPz!UbOJ>$$3IinmZ}?a+;AZz!b#THH?)|0NTH5OSw;gXy!I$n8E(z|)L^qOD>8MOK)k6u(Oc%xLURt;<%x5Wjm(C+CmnCT1`GJbd|gvm96h z6AZ$NlAo~h+nEo965yI|^wc~+VuX09&@O5B_TQzyd_Evu{F?C{jI4>*_Z|qIH7*=e zK*-@ia6lm`kSX)&cwi!h*5NbC^j8KkW(JJj_7|z#Svdvluic0a3mDGAqyR5!6b|dN zqoNcmEhw^L`43`fR{R>>;IkI=KCqlm7n@a3nZWus*nU?)+}0Lg15UQffIcO|UqOF4 z6qi0T=rf=B^w6}6nQt=e-vi|K5z1fhVyUG!w_0VVxB^V>|E}Boc%Uui^=B+)b!TfWkFNLH3uLqJ>*F1^$2bU4|GZVD#ld^~R1V zU?cdE#}dF|0X_6|;YmUh&j_i%W-izo3j^i8 z^5t&eBB$hC`j0BQY$1gM3}JSuf9AJ>z<~a-BEF_#hNAJX!3?=H`9<+RBxQtPi8nrW zr!vO|1Rwp3;2|D@BVL34KIne$s?yHp2S5y`dHW4WUjK78|4^0j4om!Y*!n=W-ESpu z-xGWS#3~^}V1Dy!udhuNg7*PWandd#)V)f%DdDeeaoJ9MaOm!}5H&5mc>`FaT^}jw zhZTAN{DYSt(ai{ZuKBHIBB9Fe|CG&TI~5k~4!rh^2KFCiuXn@?Ci@YUF;Xgu zleYY7xfCOv!aip1tgi_={QnSl`8N$!ibWUFi!s*Z>K*sDJByj`eBp@QC56av$Mr~Z zM=OZNldv|i6{wdxW5AiW{;G<@&TOm}o^Ez;;-M4BYBtgpBGQMs?h05LCS z_w4mq1j4!}YuuMc>!x>kJiH<5-QK%0f8e za!rdg_K^#zQBmRk8`*Vyax{Oa&GlR#Zmc}`{Auq6qD83~{qiTCDXdW;MWv+&IzS{IJD7jXoDUl20!-qugP3GEo?k``zjwRUm%k`uk& zI?Cf-Y$s&Q#&Rt#0!utrap^RB;+LO`lhwuprxq2pKn>RLE+C;2-%b~<^Wvif&sn9> zY|cpa)1v;EzotbR07;vq(`%GNy-}k2Ri0`}dlUOYoLl{?c2ECG;>aN6F($YI9yp5a z5oI2MI1Sl(x6IRKisB=^Qa7cb^^U>bj<4+{=6#VjG#K$fs)pt%a1J3$&sqMUpz~A? zA!U?hU|#G#;2Xmh#ud(tr0b?%`EeL!T+7wOHQHHN&pNi~fo4)3n@}9o_#19ywSKz6 zH$72vz@uvk1|nD*;n)L&8xJArBg*8Z+R1T&7L>1i^%&68gVV}g?~eg34@9Iq^|z83 zeKi`_gkfri<@XXRj+9+$p86sL0=6eE>Y3aqYg|pZh-^mLALB@n-`~O`;ad_B4vBCo#RT&0Wod1dYp&|SDhryRpRU9kH zcjMbipn}1?tg|66X;eM(;*H)ZV zT2bC?_M-CXg*P}*5BY!e^e9d0g+0#xp31#LLm1Q&scChD0d@>oi*^=m*1t0HE1}Cx zqN!JY@f$wL3wBH`{KRNW`N}fGw9KEz{gB6;H5m1x&WcF>KcYNw<<>I<#w7a zKN$#m*+rXcb92MUb!D)#^6PWmG{IgVa@k*Oaa^;s|SZ zP3c5tQTHa6h@)zKlDmc>S)tIE?Pq{b%3Bds#rQUr3b{43$}*8!(ewn1J;7B;x57JYv)@u5`+GK4#|ubt5d zU7mD5gny};_bl!{Hk&d4oKeAPWnxIqC%c%-=||4!K0ef6YWz|)l-T(O9eO~%8KQN= z9Jd}#huG)05`zpd3=wudF<5?zmH=3@P#x7Yn|9&47kDVy`1F=-&8SI{iI+v_lP0y| zG4ojo4k4A|yhkoFdWv4(v%K}|*|LAyL)>?8Jj^giJ66hChMOQYI$k9;}(qDHPnGC znl_?OEX)$if-Z*Coi3Don+&wsIW^TlZ=K}K@s{L{;|=}Tzdw3XF|SqCvB5O-+r}_) zhX3YTT>1oCZ=069TN=_=<9r5wo^6DpNch#$&H*vVqWeuFmi5FP@cP8V?-1gF?o$6R zgKjYKqD12Jb-cvGbjd_aKD*4=ky?b_OI@zZpN4rKa%|=5ugX=BSzFRENFjI9?E~}X zf^2tj<1aK1Dfp(Z=p={`|14mHE>M+-L|M$ksf2d% zHe)G?0M#V0v)0Flk|WtH2P#xX52{N79b??=tG=q3BXA`yI)#=L{R2tfj1R&%n%jF@ zI9QtRUFblHH{S1Uwq#a*bAwhNy18FGE27XjbL#D?VtS$3yA%poBQXioBJh>KpO$qk z!@ZI5yt3`P#4|*A;G+0bH~t1N>2M%A({1UkodNn4Jick{Xwt^F3Rfi)q(d6{CLyvK>hFKFpc1KpC0( zRfM+o?(O-p2`$}Dz6+>Sj9#(va7*jJa(KFfUav+dcz=63OREgt>)!pRj%;dGmA`wa zu~%N7pjW*0@3%kTM2$d}!7o8^pSaQ(JNmN2!dhg)uI^X_iN*P!@E!!0Ng zg*x65q?T1@^>PF~!lfaJy*T_1hj}3ldl9vWcc{eobVS*=>Tbl5@l=~UTSiJ%rH6_N z-V$4>rxP`cm4dWyvAzNKnNoQc_I?lTXdv+rZ*|hs<^#(2P zq}{15u}k8c<0R?-YX3od1KXz3Iru$u%ZhW}=dd9sAPE0TYYh_|H{px&OIBEeQM2Xg zXqMJSbPp%A-7o)ntdekBTlC9^&cCXVqCKxHqjlppTD+Z5f6ZIfYwe8pfO5&gNry+$D{le?c{3=67 zWYNF!_?%I%*D$+7f!XNPNW z9Mn@1h7+V!3d0SVI3E!yo5($X12>yy99hvwE}(9J_&Ze?*%p6ICGSpL?@4+PDd0y(DDUR>Yji`8hdF7`u(wN=%u89_q zptFfP=6Dy6&;#xKkIM+%0ALawdRvt61lOk-M?!wR1W~CMq3@N7oatIkr2DPBV{oX{ zFBIDOnys<8G)rJ+at1u+bEp3u{yIuN!RSlWf#&Hac{1hbBkA!M0`KlN=#5$c!JdnU zZ|wjxuq0uEN$%BW+nBMAR`QcqmoAYiHKzJ)4L|SzDp=5Sa4K5o_Ygg2J#L8eXaj>h zZfcnwQwIwTWsN{7G$j!;Oh=lF=kvw;dHe&E-wG}&jufMq6g{jpRHZ zQTANm8%v;bsev!4I^dL+`dXCMd^njf@Tzk2y_;a|SW|aKpG$k{nzd2+WVio^oyk^& zFE(zGo`5e@dKz<5O5bn1Ba=Vxnx&4>z)f` z@HMPbpQ5Tt>L{pJ3u}q~%m?6~8X(jRPv}uJhq4VodO(U~=%vhlv+5nuSzE+8RP7b= z1krJND(@Fg^vE^_x9bu94#~I>IQg-aOOfmx$_RU)!`Gx*f9o=D2fy7CuW(MuadH@4 zlXdiZ%1*I`7QK?xbXtaf$>NdP&{Wgo+?R`nkHyg}6E}aC)~Gc1BIIE4sX^9!7?h*G zU}QqS(>6e(Ta0`2gErGpRVtwRx`{{4>sl3lx zGNx2bNOI00PeaX8U-Jls?2*Osd$<~%RB>41!ciSGHqUD@19MqIZv%^acV4_MFUFg^&=*p4hMoC)(4zjKwtwC z%iEN+!7L1bFuj48HZS&sQONxY&Q_~$o#ZTQW{Q8m0M%6GGriY@$>E;#h_Y7ffY(Gb z?l`z<5{>2vEEzuQ83-3Nv%RVZP+j=g=6; zm2c@(UU_i4&R-R^dox)Fg6by;_vPbvkWUF|BM>92-fz*%j*|1j2@@{aGe19Th51|b z0vU~aotvZ0dmFy6<Y5b59@b_h`TP;E?Y7DT{9r>w*t|op%5XpvwD!Ad}gn zH)$U+7_%XwNU^crK9~zu*lM&2D#E_L*4s~*oreiF>%~DZA%8PSP2dqC@tBJRdno5G z6vP*{PZRQbBKz2@X%3UXGHYSV-eF0|vYLITI$rB({5-4JX-^-B0i~{&0qV}1$mdfQ zkLCpD-rD<|Nf%F|LqfJ+9eaxketBJscLq{tsDJ=8>(&WifIW%5^RZ{&PivbnZY7%0 z;76Q%WAbL`Z_nC7cNy~IlZA5{h`O9icnx%2{f~&kTgahugvvFboc$ z)%7x~_e{a+q?l!D==c1!5=KV#Araj=>SN{b-}<*Bfp8KIpH|~U}z>d%Q0fI z|6YfG1B&v%aYVya%oKo=j2H(vf4v)*LbjUG0)>{Y ziq7>O@uWgx#F{bd`+4p_db#BFKo^XLK4!K#G~Q4CW`DR@Wavl$l+2;X!9yB|4Zo!y zT;{5n-)e#-v;hG^0Q+=*YhbFk$A1*oW%RTn(}KGw)PK1fJ$KHYFxM|1RCZz}sFl5i zEDutcH`w1wE8Z1{GG0Usn?2{R8wfx%W2 z)a4D6{rJoBgw>|+#B4AyXF0Axa|6B6l4t6*xf)A^_7}5}uTk9%4*lbXpBjoWcb^U< z(ZL^5^h6bjsYZa-C86)p(7O!L;UZ7hZd~nG$)=Szsqm;ZooDN>O{3uElIH~tzPmfs( z*fc*@(8AdV?2W|Xj^sOCwp%^%?cvGh=adj8>znR2MZfZrAf4TN12^UJH$~mkJ#!O#S_>(!tVdu)(PmNlR%VZ!dJ`MBe(rbW%pm zou$+nOvwTq-B$O~=hB<-2Op(oLR zFM#_+k*hsWyQg@p{d?B8Fjhz^>hVB9h-U^@m8Z|fiX265^HQ?UhYi~72amQgWSIR! zNQ)nzxn`y?;Akw0_Y{GX#Y7U=i(xwBS#|$#FX&G0!M#!)jiA|wZ!l|nm*Oqq#ldgJ z<^tK?XZAoPLMNF&$5_H67re3{H@3)Z@Sh4=2A*K&w%2f)=#wmtr|@s3V|u8X&Zq)^ z=7he~ON29fNa+rMOqm`QP=0eEqXsS-eqCBO6${|U%(KH!1}+LZF|pHy@g$Dh!nSd{ zHV`FN$!E$tP*a)g{!C0z$#mZaMvl^%u*OMQmIo{Kv0FAq3Fk`!=sG$o{ybnfu;%ix zxp!CKObt-1`#qz|3MfJy1%SE(KgJ20qBj;+z*O%p+x7>y?AW?@dQomff zRqla_4!+5DVes*Zd>wFD@Wq5JJ>RTRxbKxQQ$IV!-Qs~z)dz0;dWExokh?J6sb4dMor=0ZfkC4fy&hd3NBa}v zo;VdK#uc7!ZYx>v>XuWg*r0i^(mMrdl>&@NPjc%?uCT4Cf^r={YkS8PD$Hbh9T|mMY(pq zMk^)46H-hPJMV8q?>>SISarSpVO$(Fo_h>bKAn2T0Hy9RuExi+B0wEf4ItO5)&-%q zLz?^jMGbuvj^BWb7j86TnVXgW@1f_=S=RSNV_9qy&y8>_}|SG)ta1eM4Us{)VV) zRF2%7w2Yu#gnL3|AcNajn(53=5jHO7NUZ9Qjw>^0a(yz`|DDpC&?cSU76*$gsxE^k zT%>{3g_Gio{V=KqFdOxo&LYg-!M^chCEWWk^#Oo9_J3W}!Lhga5NAgDOM;filli<4 zQvn8OLp+w+E~FMJ!0gCp?O_EzaDLb20oY|UZc0Dk=x`+T`CD?4yQV5E$!8e5AjN9z z$&|>o`rWmKN~omaCMK>y4)oS(f|aJ*C8`OA#x86Z-DWluiNnzP(D>r$y8%ZVa@XS0 z0GYJ`n9Hnn=bvA+s2cMa9NTg0X~(SoevoJ9)U9TEw?e{DO>!tHrpZCW>J-U#{|yB- zya*|6X!&z-7{=Z#Ld9vzMo6`t;9R*wJ%BmV3)_Jquz~ z_&2n=U0JQ(k0OtD1#2_`n~43NvY?o*Z)zmO!*{N8t`I)nP6HbpTKZ}u;JYt0*O#~B z3Z)kTnDG&GjFh~8Xfj5u7$~)an$N()oA0s@)cA%Dd8=by3GjL~+Tt6ph(J*88|w$k zxMnB9Nn-<#Tw=nl(BF&~yXCH_VoL5YsOvVm-{Z}!_nl8HMFu0)i6&DN$T+Mp{Ba#i zSESI#h!2D5}*-*(tw1~Pxi?Jej2Jj2OYm)RaL$B7mfwEB7qA?$6mEi*P z+>r<9If_=xy-L4R^R)@B`C<)F>I1a^iYVfmjYR%UhswQlOapwWa&>snjuytMmT0W( z!%?iM`_baocE&AK#TF;vw|6SwK&=&qDe*H(_dp;W^7-N`2_{c1dZ~}*Ezg#@G2v5S zk>gBf;&>yaWa1$a+-KYg9H!=+{_i!(3qBRv;Ga}c88%KUl6CW@NXeY4yJgD^xq<<_ zY*$A0rC8&)z$4Q!=9f!Uq>o7{q@4#hGys;!+ae1iRFIj~T{Iy3LphJ$;MHur!+~RfvoFo;sPm5@quaf_lG+xrukKIa?2#6B`os! zG?WE7x5`W9Wb80I|EWuxrb2GX%SKP~?h52kGJ+{KF92>w!P zL=nS2Sg)K^Es^aoFc(9M^p^;K%y3hx7n2g?hPNo-*%NaTXu?E#PeXw>dfdUhe1Zs< zXwQ2sBrGUMmnLiKS?}o_CS8^8T8()t37=)=u-y4e%?*3Yp~BWlQPmCXf_yu@pJZpR zYM$wNv@Fo(SJgFli}F`HC|CdnGlg$Uy-PLN{(V#g?XtZER9Bh4zku^*;6e9$c02dy z2dCMgldH}QTaEhL-Pkxh^=gXYG>Hu}6;f(PmoRF8+4&W-G-2a5PjTTC(4evcj58=* zl4}brOgqP!1j-M2SEnK2h!CFeDv>_xTlpGp;459gu!sQ1d=(v1s!(-JW*mFA|c z3D~(5@r4U#)0CtT24GXcGo(!y!&>?^$h6vHvEq)N7WCL3wtP!kb};y!DD-xm)y#du zNTqCiv9ab~4%lHDZQ9ayeEI9em~z@%kB)WYNe#6&_O%kmlz^RoZk2Qx@ z$4W7DaduaAv?voaSjGewk2ZisdW!~z>vs3os}Ex^L3Px#eADe8fcQV!oJ)3UiPd~{ zZsuZnY>G*8!EzsCO8a~jWFEh60@E#2KUfJ(?I>4MkI&UlMh7jl1j2Q__&aTj9Q=xd{^SJ-V8Or`a69hKgp z%Iu7pgw>wQeUGvY2t-#qx#mhbLGAN9VTO6-S9!{LKO%*!Ng1VN(CnT6%J#S^@#Yu9 z{~3;ei|~4(OSThz|4RL#+uwnv*>YriMHIiQADF4%?WYmD2kAX6LYl~Ezjoh(}7h`XUe@*8={0mMtauC%z>^q=5v0x@>0Q8 zdiI01IU-~13yUoaLTDCV3?0oQI4M8>sO=EScBmLS)DIu$V@IC$YlCe76N0?|kjA$^ z#fx052M7fTo?lB0nBhu7=NVnv*nmf5kBA;>ECZ8aJ>^Jj%>CzL2TkZHmuEV3XB%2} z=_Z9czp7RJfy6$hV*T~QUIzGsn|Z-8?t{Hh@37v}?B$Wr)auF+)BhH3b|aUgKwM2M zCGQefM^!ip=}+{&ZF_vckH~CnJPN3|{GPKbbG9O-r5c=)g(YJ(($E9Z_Z>`(y8J#k zgTMfsTTcrn3~WZOng4nYUG%(fFPI&Yc1jv+0;1BiRcyj#q8QzNc76=Lvpwtuj*SjT z1;7SFeGmGDaP^lns_goJ#(PixkX%fs*7lB+S2(GX8+PlV+)_#5UHM~Z z@h@O@&I))3991$)l~Tl|{d)Ab0D?q2;e7{MEUKbjMe4FBAp1(>;FIP?boEICl=kebm$RG-*&m=JZLOe#{wg0%#`lgFGt<4H(@7h~ue8=owx^2T zuR$DD%XJ}!jA3)93{e@Ex}*0$6n|g4+L!Fm3=eTbi*Iz^h{o3GRQ z7p=~>X;c3-(uWv)BbT&N?Xaeex?G4nFasfu|5>>$nh@Uc^ zr|CsO7YEgCjJM6lt^x*ewH(am%ovqVnmv3xr`UFZRw3Kk`hGtEdSYCQ-RzQ=&v+TV zc7PVIZ?Lwf9ZV$|!XTEAz*y^7e z_hB|)!k&KofCwVwD|iyFsY!c042lkM`BqEU`-ZgqTnKcvLI88If(KySLhWiy=ReHA zH@W&c%K$IE?y?;)k;n^mbkp3eAdPTl zy@wmH0pY&m{_*hHP|@l?cRL8oykhUqK+c{35T~Let0>-IznH) z>}UcoF!(3XEhk7I?0xeamw?;f#RJtEkOB z0xk-abIJSFyqvBD`FTwWK1|8=_5`|m8n1s`@Da)SeorQ)He2rK7fR&Mb+2Ds(G=5| zKG`O^nTfe44}n095yH&i)@}?+@6bBv$x!+?88|1Pkj9=<7tl(Z>`B1Cw^tqM@h^Pk zz|-B4YxKr zNHbQ>gYDKX8G#;+o2{b=Gqeob=$cS!rCcTS^K`0HENdzuBd={$$;!pKTmi8lBV|J?8ftZ^`m}a{ZY> z0A7^iXfU=SzHqdkp%mq$8_(Z<`_a^EbfBQ{DNwak!d*Mp7=W=}`I?a>9Zs@ERh`P* zJJVtkZF*4n-ZIv*>^vQt6vigZrCjlAS84n@K)G;X}%;FwI?lWWE%U9Cl7tk!5f zFpvr@c1muGX*uW{nR}c)Zlw0}Av=a>6rSqGgDL$ei$D6wW@78C03-)bgcjfn>39M{ z5_o4%b)3vYH$*6SI zO8imBzXEq?Kb3u4z5Ik1w}4#^sfNKpQ`FiY^CfsAtOZJ2{c?en@GY}S-Xh6X8iR_? z#OB-n{R*JGIbpBM+rd|Fdj4ayydY*onRwY?LJtU|3hEA?lyK$}sMHsL`$6TN$qe4` zQ?Ip!e7#VtL>g#hE?qNL73i!v;z)Qq0=u8Kd5RouEG*0Q$^stfJK{@s>0Z)l=9O;c z(+A+uV9DHnV+3pJRvZHwFNKK^0HuL@4nGHz&+*Y+d|XYjU*z}TvOh|^2r~c}i#8+M zC)i6&YGPoXRqM@rkID{OkznM4Nm$4SqqwAI<* z+G#RGuzw#f&5gZ7WS?2m81ey&I4sf7%fq3OKX+T_==hj zhgb}aW`o7F*h>IwO#hfH8*!g+Yn!J9rEV`X)M+|33?`~>b^g0rgxKW`S6Kti;bNwA zzV%*RoV9f;4*=*e?-7`OUc?!2F?6dGs0nYweXaW+lWJh)Tdzp%SDpY#OWkokaMf(v zxAQwScSj)0EK$v6R!U=k5J9e9eh`LZ4GVAAR ziJM9_T~D9+v~9EzIwplzr{@TvwTz)daDL&7O!aIN`(5dtsG(+`ybj zDD5@7Rc7jzJus6R)Q6u;TGPe#{mW8t8w3CllB{EZ2Jg3v^EM z9khUI=|YRUO|n*mZZk1E?%+_zdtIY``Neg*>76=u@X)58MaXbsHpg8q>)Gwi$>-tc z7Tx=)Cf@^nhnD2YqYO7#>rifuOibhuw2YS<`|#hkJyrxWp_EHjb~Bh>vpl zP14vA|9^{2%C3*Ai776ji}o*#YJ{vd?iU*E#lET6C6{Nv{`ZGs;a#JCpoNW_`=&IT3h$;%HbdGCmI@<}rcU4N)3Eq)v=X9s< z1gM^3{z(l36UQJZHfbr#XZ7vAv`Z@~V>$ty6fff(=3M~7U5H+SU|>tm+c!h#S?nw7 zmr5I9jqegoiN3h=wNh*N#}rS?o^p}r1`#x%bh&{pqtKQ|7TX8kV_~ry?tEQWrQ_?{zR-Q{ z?>XuFl2*EDLj0IVT!z1%i#IZ{WUg>_o}s7mgsH{>y<*I+Z|Gtt9WCja{I6I1j|K<{ z;Ftx)Ni%_-#-h$GcB=0eWf4&Y_QXe`E7$8VK!NtNgJ%p7shv_KQo<8F9V*%w2Y`7GbNt30aIY1dCB(4L(8e>za&*L_ zm2F=SR9@fCSZ_Pma&%UOTn7wZP~RH^j04~LG3mve01K;u?Uba09?1k=fDV!%1uNJ- z1=K-DJAlMrGBuUym@I)g#Zr7L+N5D%IHe7BruTV0e<^bT!tjzNYVM-=!x*wVUrR1~ z3YYe8mDcS~b{=~VJg5Q89zyw`YOcP|z_gE!@KL@}3b;Lr1nTb)eg3pbIs?Lc*`P3p z(94T8I36~DUrE#A*V$r!3Q<~k>GWTK0OeOhppPAmccxTTd3dzRy_%2gV`R>3rpb$) zEM@1`TTOO2f78kuY0N&LLmK+qcz@`>gx40ePj+iRAXI8`w=k9%brN>AsGGxNFUFFh~EISgaz4q zv5*Wqn#p+Kd%K;-+i+<=!+eAz%j;bVChuT!H47UBx_XK{Ek!mL7}y5TWPN^f5_ced zamuahE9W1s^`&<>I%}ZjTe;jZ232P9E|~H>$a(2YnUEpEfNIxfYbKu7XRZKV+NHuQ zrL}!t_GfJJZ%YU4)WxT1TlQ(}Lv_N^(GeYFy7uo{1Rx4w=VZr*;QKS*v6^~bKI1i% z@cV$O=!XEk3V8KDl`w>SW%5>Wr+Xra_b>70Kn+mi*DC!T5 ziU-1-hIY;E8D^x#MM^l+?OzhKE0KalYEubjpF;*=azK|i9I*nS1^IM8d<5iC=IVtz z)xzCQQr_Y_<%a_wFn5l2OS7P64?}_6$s2;4ky-KWB4(*d+7gVN4^skJrNujjfHR&U zL$VXL0@vJu>NwxlEwp?4(=snEUkX25)qElNHYPrLolqx>^g~@)kn$d0d9S*!8Ne~H zR$MY)ygJ4KD-n!bQ5`It{w-d#IWcu`LF;F&xz z0>%}2hxfrX-TD9N$a04h_5r7-=dJ}zV*U}}5xaZK%)?x(1*TTS$c5#n@3(Vo>9p5z zVG2!Audu4NA(W^A@NLb3s!K(oBQV8TXfb0S{42unu1}a{Uv3*F1U4xQpEsK_Js7^k@Z5W#XyBYVtrvr2e0VYHG{Z+3RZ(8oRJE;qc44)~ zJ{iy0-lr=>t5EJ{325w?j#Y)9ouKZme_CwUp}!VlH6SVaPbV_a`5BPC-Lgv{8Bh+( z65Yh4uH+BWJvpbS|C%f4CT?p@15Zi;ejD)%Z{j=UCXNYQ_F;i{3qT)K>9zJ>=+Q#6L8}HXj@<7m}ArXT!>N`c6uExKdqh4Ld&+62xDZbWN_S#QO<1d?wr%F4*=(fej#0b)M?Y&YJn%}|v*mtV5>Em1Taa}2 zY<5a_mRHZBZ9VBFxi&C%4i!Y8*-kR=X%NiWM?1#&P7SXp_If*Yw&~Jicg}SD-lNva z34dNALzYgEXta`862&3z@Y{5G8aiE{NIB>uGy>R=z%E~88)c`%?L~&c>PHLZWC&tz zeAK-{%4iY?HS&tXBMl(VEYrFu1UWo(Xn1w$uDslT&Qx}3YpKl_UWFNaiV~k$GDt(C zm0=yxSrD(mR#AQ@vbGRES`s_?YUybI!?w_F8c}CW+Cx8rBu`^G#kY8=B!#9r+A%4# zl4PZ&dX$}xFeCaqVyIVm22@wu1~##)As;@(L&iF#d5dM;XW=8jSd9AWy(JOwxBv+u zg}rz8U%B|}%Aa^{w(*bWAI9A2E4r*TdW)8KPEKw4A{$iuEwNi&%m8b&AtH}aWzg-{ z8xeGnwWW@8B!7RR<#9_MI^hLg>o%t=a1W{;5Bj@>Zl<8s1P;-|+KyA$__Te*q`W#8 zSE?i{(prq2BMoG$4e~otYLDm@YTozsd#ajcj{^Ejitzn9#8d}xRCG>DU8bDjfAtr8 zg-M%;Mn@+e?u4M|D_8lj*B%r?y$>tmGpY9^ZS^Z$E!v`%wk8OI(V4Xo*7j3neF&4M zhiN6)&JavTDo@hndy8PEwSrIBhe4hirYxGf6LK%FocwHB!UtIZ0`g&O7(x_C%&(s7 zCmz0kJICFu6WFw~&20*BPchL!_Ut!R?pIWm(es3z%rl6r7}z6dvP~ApK{fHo+RYG{ zMS@w{vu}VI(e`y1`e&%0R7#tX{t+PkR#Azn2Z+>vG3@8Bjho&pzY)>Up2`gO2jBNa zgbY!2PSpf!LFCgz53HTTD z2frN^(_Gj`&+`Ip!NTb27IyDaeCqN{%bkC!iw|nX+3#U_NJEeR1OJ9C;OemJCy8WW zyXX$|>BEn2#nwV@{XHmuqE;Pb%rrMr_oV{8wZ|%VnvYgU#(7=5AI@_(N5;zO+naTd zo0$G7E!rL1?IGJqyiYNkSees6x*slxXeNkpFppJFMCNN!PtmSG{g2iDYS)!XsP{j< zd|RWO!d5nC3*QG+kGyc+T;Or@=MFg`t((>j=tcGd=F z37uuv1VvU8CiHbA4La+RF}goBLlnuP4Vg4ivi zBzGosDft!$yKoIa9+(p!6XFD=W~kXaO<2m|+i9o?WMx-efR)cE20<3{-Yoj(rI;P% zFnhHta~fU#FWWOodk!ybcg#06@gAe z`aS|h49vsK5-gVk6Q9_9kvT0w)ihIK86%_^VLLHDUgu8ytVuFW52@(k`#`6PhBV?7uAv@1#Ci?-I$q{n7G6H+5m^b~;Y$_iy<^^%@??Pr&Hx^}Q}o?jb6}mx5!^ z9f0wePNmxh^XCN^Km^pCr8Y$D|7}cEsCnt0?!(15va0WH|{fYL2p(R5#e9hO}z$?nvin z$e*syX8?|$5g`8*YiN|PGu?;+V8NJTe(lxwLWq%s211bOgB$ST+h(A4jScf&%c1#q zEQ5SR$xnPMEFQ=7RQGA*YU^n=_2iVKSI8z?ris%XU*9&+Uw1+YM*ym{?I$PGYOaT{ zp|s?gY6YNU-JDA0e7BuiC<>RupZTeJ*M?bYc0Mm^@7D|(QxR}kl3`feD_CBqt~32A zc#6HI!!jhF0O*t(ksrzzSMq6ykXy5}-mNU8IQ!p7KAIU0*cgWB^o_fcbFC&ZTfHNJ zuORFP0F`x?J6jDNqfi*2yx-NLm3^bMpbl?SF!gA2`Lsy#{2EK$!_RL( z79v=?nzEqL!q{X^5OJh$tIawyh8}=o=ci;Ndw8*cv2HbunBr?1=tu2-`%v}GM!v$< zQ?Nbz?2oqGeT%po{T0!6>&NO%AtqHQH(4Qy4R1=KAGgt*hl>Ogyh)l7=&0=c83QvkCZsHZDOlkxo%hLV*wH zfwVq;O@ZmOH9o9341@&N;QgmfCtNTAUY>tmWbI1lJUD_QBW6#pNbnxCD*PcG8_um+ zJb|eC&a~?ro)1<`1VUAd60bI`^3z4_IGRFbhzK6gGY4D*#YjN`J86^S>vL^%Ks%P2yVAI273YsVHID0Zc^dB7)Mz*n_zD|Fmy_}~;=lfH= z_+Lr<%KAjcabqETQwGQKY9I`)ZDFG*-7`Hul)}wR4qc`bflCYb`Q8EStP9yz)p3TW zHvPaNr=u`Dnx_S%2nK|I0ZH|}X1uT3gzz&+t>ES9AcFW?+VEyL7@ja#_bFV;eBvyy zACnqFv(at$`xd+F#Yj5iJE1#VvBq1-)9YVk@iJ6`@@Fg2|4f(Xrd^=3qU~Z6E%96u zrwr~1mjC%na}Zl8vFUn54Eg0q62R|t3t0FI)g$27x@W7qUauI!9M1HmD#>UG`F@vG z>^b`Y2idw%v<#4+-;p)T$Jw}kM>EL=*l9b755<8p347FW@1Y}!n!KsBIx}e_O|fX8 zsn71#fV1HVMi5hqulxk@*D&zxY8Rz4O<%G4j`*hRbKPq)Ak(J)uj}}B4+45>Qwghj zhOS4o2WBIC^3~czs!(Kg&ikbMbgVRm5qf3fM=BO0Jw;ap#E@nR;&uE zPtFKhmQunw>FrQ_N+))}=p;p#`j~F~_w@JTV<7`$gEPXbh&m-a+nn$n?cVY3k{hR> zqQlXDD}#C;oiW~?BjB4`7p9TAkf1bNy8+o?lkWi%GtH1;0mL0BdZf5WH6 zqeC6-4acNLD{cHmU>6X8Q$$GHp|$Xb)4QIiRozn|rkImj#Z&+=?rm}(l2211+{X6v zRaP~^2gJHoY0>s_n^h?Kn5oz;R7wJ{x;3wH9;`gDQPAPzTK4$82+_DB`ZcSxVgu;m z6E6Y+rM$-k$WQ(+cjZ50b3m})P}D%-FUqF% zU{L;ZQIAIT(TRj;$h;*2HWaz0xwvSQ?g=5l(wO9VS6zS%*VEUF z9kWZ6ELUUj8-az-S`9C^`}FV>u3~!evDDt9!CX=Y)e=vKe<|<1RYp^^V@!Q)8@Vib zku`6$!uz=n5MRE$xKA+$rrm+Zs*^q`o!S%Z<7uDnbC!wpKGKTIsTmL`SB{S$qF@9?bx@LCadn!7JVC_(`dhO`$!m}=H9Wbi5R zHIMD>+KY@{LU+D`Isr#U##-zDnE7q<>qV0Hk@?jOD#zb$wc=yy<7Ki3mq`7O_7m{1 zar>eN8v3fpV#(g+5Fp9s040C~b{qC#J_7P*t+U22VLT>iJs1K^CwP1`d0gy(y#a`c zrZFigxut3S=qT16MiAN1QYqt&Imy01m8FcPGDk-y<9HxWXLeH6>ls%9)N|0Fpg%xC zI9yN-0^Uiwvu(tz`*+e0dH2#icn}N!SU!OgFLMf^=*kSxprIM+m$~tncurB$i z1C(B^zm_$9s=c-udcvKkT4PJ3`trf;i;xJwf%?|y_XjG=+ZAO@+vhnF(;I}l`2ZwB z??YRAF%7s6oPY)5;ve8NZ?%!RdxBQ`>W4>Zqtab!hV}50E*wiVYCV^Z!r*!fh~-fh zyA@X1X(ZKARZ5qB&A!&ojUIjtTd=M$Z@~(u3c4yd&TY!vo64p#gW6!b|RU*V?h{Hk% zszln9ap=Btmn;||tbQKZ1bMa+WaNB51Ewb(fS-qI_@HNt%Vo0BT99G(gY>*Ccd^CM z7g+?oYi9=#=22mRigoIe39+pP@a;He8kmk0$e|wzX9<;wd!?*Ub4l(YhJ6_ z&+i~zs`p&|*5bz-$=Dp0Pq705yE@n*@YdHoW4l1!CFwo1kcx?Zlw0U@1?=u9D$dsd zo{CK>Q*{4c^Dt9njNCiKeB$w!syMdZ+XlG>}_6ajFt{@Cs0j@c<4h?fAOWcYmkS zjZxf+Ww_&?v&h~oxIFQv>b|a-yuX_Dc&txbE-{QlBc0aG*~R}Ob3Syi@A3`MNXiTH zP_QC#Kma`$aceeB33004#|QtB)OC{33*tb^WpJ^N@#0aP@0V(7#NGmT3S3a}yNV&w zr861HB_CeeQ^^K?$=0da-fy%<%TDR7;#Moz=wZZ?ITIdv$O!QSSht3sCatwl6a| z0m>Lir)6bL=G2RR02;2!k(ZFs!i`gr{_grD!+8A-`n`Qj+U_ZW?R`bJ_zZKtIm#9% z;7|N3e{SpT(NwCRJn)uLA#^|Ys-p5cPs+sL*8g%aJDYkYE;Ds+uR{DIT0Roy9qbxI ziT;UwFZ_1)md~nVwe{|F8PdEUg~O9+PL-Js*kXA?q+Svlkr*O;4=`s}4cV67CdE_? z_;XqB8Q1+d0J&28&zgpgKdKorx9HDjF#3tU?{RJ=`e#pts~V25f7!?P4i0gPm3@$G z#IWyqc+M`cnFDXwyps8EImrO|;D7qzk&cs>GEY<_bBwaAKH&}RhxZPEMZqM0P?@Qq z68gb%&mh)T96$h=o{u}hLTQtpxcqou2LsGwWLo3S5*D{_=KwR4Tw>Bpj6y-ccJYrR zO0X8w-fA2#cFOW}{0O`qVYXCp&d*)=#kVg%8Ak|b@_9;UO&jBxx0b2N5@6U_x2RwX z5SeyrAyJoi%^ci3mACpldlArh$Je`nFPTCjZ8tlmZ+6zJ#EUq>45i+@rS?)rKcNy7 znoz_wZ+dMkMbSy~b8q_o*K#bMr4=(Hdu?N9`&BoJXYpavfix8QE^xiyoT%(>Va*jj;60`Cb?GrChduh+KHeY-uZ~1lARRiI5nn#9UfQrESP^7RY51~Eze;H4;0UO%vMqp0|We^Q^n$~1U0zeWY zJLovZaej!FJxRIr8rV5}gkO5OXHwOj^aOQFB|v_!~VGRp}jo(v=ohQAL)2wF zvs;KSk}X4S(-X+B_d>Zf2h;|<3ClXY3s(jCSig0QYF2TjDBvW&_cT!EKi>d7JMdAG z?%t2_U033-hN9mJM>#YB>jDu<4_tYPT;%ud$={AoGa!c z4Q-*Y=&y18^^f>FgS=bDLeG7ON58v}c2_$zC;a+cc_l31(TSPJDlBq-#aL9ARdIRo z-L-#An`PRC)lncx1O0pCu|tEl7zpN@GuOsmv|jG|=-1=gIu3G7Cpa($cV?#|kK%P| z;ZfAp%QHdJnnNKuI}65AAxZQ8P+Dl}V&qOoAp;Yo)Wlk-RvmitnW| z(0XV`XGHCzI(GTIT`hEow0nFT-}l)lpq-NDeVLkBlcOTGedqwCwAftT4}F_;f0257V7XCG+ zJl&a<*-Bx>-R?1u(#x*t`1Ge64@vau?eRA_QjF;mW*?c>KD%y@Sl?M-L0iusu=j6V ztm;@Uu@EW(MR|Ux-S~tO%)kkwt7PhYL_dil0CGnhKB3k?zzE{1F1c7H zE*`ClrpzwaN=)l_h*d!z-@HR!#P~gYnEyiLceBv!uiLlUfstuN>MUt#*6#CV1WvC4 zaGw`Ayx2q)|HwzpJV_2ph-BXR9g%KLpKXZonORTcEK$wsJ$g4}>|sMT3+<4$Mhh-2 zKIRzLEx6L+qqLf#=1MWGnfnnf8|~1T!ugdpaV zjZnDguiBm0hqT+Tdc*W)QF_>4rW75~o>PAkf@H}i>Kf-A+@LC&+rXIrZ1{)KJZzV! zJv$KqUAxzlk_&~0_-I_9%)LuJm{w;ZijhZ60EJzb>j`w%Zf2qj%$RaM7&2_OrM;@B zz*)YJn!Hk6?fus8e|%0m@mGC-QSKu=f$MMs-@>_>4ox#_IN;h8gS<4zd~P`8OJGTI z#ZYkElw6?1G?uw&f4(q>Cz!?#b#*G`7&$Ke*E7k&XA~9L*rZuHKWzJ??ug@p>MTw6 zB&pw+4os_hzY6$LICNF~7&z`e{?*p%Zu?t597RD3>-nk3p%XW|B~B7gO^Q!;R0#cw z(dN4wpEx4()3u)%l{t!Hb*2ZHp!ZmEkWMx*tE=S4rtW`K15N}ja|66DPO;0TN&!0C z+&~$*`@cU)3c#*`u?Naqn`+Um$}pLD)4SIyO2jhDIB`6OJ)(14VvskW=OlD?y5)I~ zQsS9EYZ*UNUFp(9JGxDL&+hpOhWAbPGmcEsr!i>DgSb)2=L?LJOSu)((y)J~%d4r< zfN?M${09k>lqta=Str*@xDtGkIyMm@kE?frQACw=# zSH5yNemO(-3QNTAAheOyQBzq_9gbCOZ253^5^R_#6_99ZoM$b8MiJJAbU-VNWh{!G z_~Cp?Rf_%foYszMZ>qlK(0rUJm?rN;kly@WxuzOk1pN6BQiLAPTsjy+_nFpsq5In` zkwxXz($LcOA`1ukF|=|l-0}%a{O|j;eLB5*d~+m+^?RE0Z?{Sx`#L{W_yeZk-4W;6 zOUn3GXsqa%g_r84>a*WMI9yf4>4?{Rf{}UdHYkTkWTZt7t1< zk$+wggDMMml9pZRLkQ=oiKTos1B~BS_F@|w!cS)3wp5w+78A}+JF;9cSP%_KkzCuN zTsc;tg0818zk+#iuwkTwqB#{^2|U`ln^#O;r1;V+d5TU!CkZ> z?~`9RP=j`G6;wMl@#Iu@ycn7w7o;qzi{SUkfEg)^k{b{VXvj|WH4VA*?(F-biMDT` zB&%JP#AHqq42N)PZSuYu_}Y02>I{{2R8%FEsLw&}wX$6N8Q;yS7Do z7ln1Gh(kq!tKzEoaqlmEBiF4e%|=<2)97I6=ii@#Pq6yu0fT}%@v6}E6Px9)1!~n) zxId|hrWx!=hCIdsmLr!^9-B}SK$f6TrKJ7de|wq9(DGPogX5oXoT@5PBi&j5#zGH9 zJ&=D_OmubOUBOF3`WLtN2;?wB_&7Uv;jAAb-qFk(hp9R6Nk&%cvw!>D>FB^YWi{t^ zdxiI-V`w=WZN`4KQ&FtSp&DI`YtyWF`w>l%eA>skfHnq+dT`!p+!<@wH}P(LaDI>& z@ye)9{^EZrJljpjaWCylR7Z7g20IO+^1{mCc^S~JM$KKHOt>fbD);k%7eaG~#+_-E5 zvD3m64uM7FKPl}$Z`QneU?s0_(#d$d0TvA6emHU!vbV{67fG3|9_Ix@VE!=})h)A!y z!8J9tb4+eIQ`li&7fqDQJD9<2M4jQ|9{DRHrk63G!g?;FQoR20r_P|H;>Yx6O3i87 z={%6UgHl7Q;n3bl1K4}+E7=QiEul`>CTG#{kC=a4DgLPBgVX&AXRYBysw^ataB}Z^%zESGgtI!36)V`T-HN@_h3hsF2Le zlRp>rV*O(^jbrTBN|#eouP@HCko;?Y)qVO-WQXJL*aR@sk%`8SEu<15zofTo!ly38 zu61tGfZvStC#7{6#lTQdR9IP1+&`*7$$izV-|KeW7Uwj=Wtaprrk<17EIuRe>Np)ki21zW&i4MHGxI#&Hk0Q3rseM8BwBKzf@dA!xZJ>^Ix?9zv`; zB_;9dtJpT`^V3Z%@4BwKYa5BI2pd>yLhenx&sS_6{Lfb&ZtQ?#vzOB=6zi5ecQRpf zRMUwSfn^MPR$7Yp$Y6>z?$G^+0TH$j|Hiu-nvS%E_&M))YrhF!e2wt{cn#iQ;lrR( z7+lBtFIWjlI+0jr;}(i`tBeg-1P=G^Ebs95#w#yNklT+H3Fz1|?<;g-^0=cO3=Sjg z*I2vkv@d{2c}@`~6BIX9o5i7w%t7OTB3g;MNgXI?VtzOi6t}4jjr(Vh9Cv*dnYJ(; zzFkQl$i*c`Ru}jMD7R99v#*S`=WWPpVi2(H;qWKXCimxo;o!vKVR_p8^%Jn<$Le8a zDpmiaYL#vUi>VEH(kNy3{21_Cq^Hx;ek7yAC;YOS3S9)KCrD!TdDOrlAuF*celZ0efAfcjkcldYvSgjrDbnCa-W8)H+K-p8dKDc_IlkLw9KJ%D5%?`4T0Ru zPKze=A+rtTtdm99U+a&MX00GHA(6Hjl7mSKp%TnZE`KC@h*gJYG6lJMR?#t3U19;t z7~&ebFS*_>KXi}%fg6d}6Wjk06yl>2Yl$VU;y-xhm(6_xK)XgivkdL-nE9Vle%6-q zMP-9#`%*nKGaxY;7b`%$v>`S0W|cNj^#hizkF|xA`^^gI-By}3E!RS6LygwThD{D! zt)beNOxSN%wM|%wD4WwuKs&ekpB^2&@+XsH@ojMXY}3l8tw#l914P;S?3O1z(Jfcq z2JtOsw*NXZCBEkY64p~%vQ$7l`v!>?N}!|>4uh{~pa?O}eAJF{!*kJ}sEQ=TYYe2Yn;nxRMVhr&;7`1yp@jYNWBVLR2E^G?<8T1vvvN zfKNwBkAy@n@-QyG02B?RvE{%JJS{8w`_rxQg?|eGlf9CMhLQ z5flOyfm^X=yVUW~kuySB<-*Obg~pREKne?g$y9rKk2#y|^8NMUpK%@Svk&;(pX*AK zFUklj$_BS?T@9U4Kd>3bT=f}}M*MV{GOr4u^vnmvu(+XqeJ{XN4SNHhr@lK86-yGnl#882l zvdIOE){H)%C9!OUD=7lw)k3`&9QlGH4vM5e(d4gfj%#3;-Yy$a3VgFe*)uC7reJn# zEt;Ff72I=Juwg&=OQR`2lA^7R*=E>1M`{{bRhdV4T)(YA{1aDze1hqGc-6Kk%T9@D z8M{RC&p~W0vs?9Q38Ga_8{5{mU1eDKM@f;Niu6elqd_wku7BzZJ>7Y$Lf7u$8$g6m1Tr;w@VveDR zxO7XMLf8uqWQ;UvkhhaV>V}bk_rlcMbK1B ztMkMLR8X25<7JsN>Y~insjQ_TuM~f8huL?F(^H>)BMy3HyAd?5+A;6(uIrij z_j`_Svrgl{&CDhKClb?UBch6Yi1Z(DKu)r%k0xrmRAhuZFUy86WKd<7H>4GHT&(m^ zYxD#NSO3{roRpwB-;UK*bA}GWU-c*e6(@>G<>=mQ2#gu@^9|i`KH{2$kXp;jad?)4 zc1#cS3QOUko>r_A)Snp7se%iZ1_JPu8WB}scjU$FY4} zhro;NhdQM9QY~74>gFY}gLe67Q$QR`Uzyu{-~y66AN77_Q>*EuEvCqm1<-zJ{_f=B zN6OCBur8|Ie20qMyF*i&(qpdLKAo!8&>aDY{>EY4ToWgn!^qdJUD=N``; zR#5l#+kEQX-(1C^6*Oj{oE_uB3eUqNNV(46f&fUAsf>8}i6Qrieck2~sg(Y|T|Ce~ zD;V~ROGEJ>r@6+4;s*S#PdfGJnpoa2bvI8> z&~%rN&VgLRBzCKzY8Tj^z3|dRr8rkp6!NUAs;9*LnXy{F;A<&Cr3M<#IHUmu)I=bZ zAZEA*PpSWbDB2TCOXPGbKU$29XsUMEE$3W#HVo9oJ9_azwj^jkF78Ov?Br@x7XCV8`1jfVPR#p6hEXyq1=gcNcfRTUybr=v!mZ%!u8k=jp zAtm-=qO11Atp3Ou*C>VV!}TyE<`X>5#p&KZ9N4mx7@;wGSwQSUKKSEEf;4NdOY4cG zOeAln7IimRaIFQu>p5^BO3++awPZ91=-9K6S$I+2yQngQ%j%W3#SNRlw7cPL=FxSY zf6D{ykuv|LJxI~M-2fVa5>x(uc|i!=$1Is9LJ$FhF{LNod*Ja+*SdPT8lT^AM$BLu zi=JB`X?XyVqlLo$l0{!tB3|M-wgL>L^aAOY7-zqT+v`~o+jtK@hUEsU2J2~1EX_^D zo5PEhA(qYTE!U(>^4pGXa)CE7nM8dcG%SS>&>Q>aSw`U%Fjm3i3`d7g!wbLTF6H|S zFn_U2Is#}y^7tcLytyl!@5}FPeLZKD?p5P#GLrjmEz+Zr$eNDasj=p&m}7mVx@jliXC zZ^OZgOTLrJlp+CFw#WB8a|PL_XTCG~iSO;zfg@Md7#kfz?}s{GHk&^sp+}%&Wgf19 zMuMjSo#J+MuXv}?1sn!ujV$u8Ww+!(AW%Isqw`}E^S%9XKFK>f&B62{V|U%D@~i?)6bTMjUE_ zmDUA>icahUYD~y5_UdL;>JG+wn|^#9J}S>|yy6rWXbT$uw}b!rpQ0m4YjsN?0)Cvm z9{-*u@3o7xY4npuJ3x>QT>glKs%Jz=qcJ5qPi^||o-Y8MsN!FWwZ^>%SY|(y{2LJx zu}SkcQkYCftx;O1T;R|!HigsK7nzk^sH@@C^+{w1ogXdTiea7d%AVp+9qW{+Z#X)M zFIY)=T5kbIzXuV91duai^T+grf$=(BaRY9bjrjNLiMK<>@A1m${iAXmdiUS_cI?o?rg8`4xsZW+@&FHn1pvB2&k*&UEO|9*jrFvY6t$=eewij zr&&xvre&J$BxAstxi_*7uV*UZmenGc`y~VDG&-yn-w8`Kq=VCYLite*R8TmmvP<-d zNh+EBe9q(zzo0hhDlJGfkux}a6qRLmmtXFx_gjTORj>jdr-DtfFDod$`$H23SH0gL z;e#WHVoCWE($hG_vhfB`ftzC-%{7kXUy|pA&~a&3R~lTFG_4!LVd%Ox&9gYcyw5Yo z%bPt)64D^+S+UqMSYTKj!3v{1Flfp?*bGh(5ajkG0rz@IO*j`byr#ED1E~ZIAh-Cf8)&*tWZCMaS@G(nn)49I`(Tu zkn9nr(!^w5-{z$%2V+XQo=8c5({hH_srt40eA0Hce+~*ZorJBe%=TermbH1z9CFu` z9S9A|V7D_mImf2J7^q{?JQbfsy3xi%5P8)0A;YYcf2h&(kr~I-lDvw%l;|xJXVQR& z|M2SM@jq+&lyqR(La|UOd4-CBX3Q7XMS+I+B6dqiBs6F3q$b46lb+!{D@nL?-N^TY zv_D%?;@S7Jj+M`1S+#O6j}zaNRrx!%PrJ93%A&>^{8bhjsb_a{25Q+g#J$qz&`gp5 zcVnkA!2Ml@N}?r10ALt|tAc^iW}+4!2yL?JeRu`)*5!Kt5ad;#xhd9w`7OJ9VQ+kL zV``4-mwQ<@JJnp196@jn?rCjyC%5-k{qt+DeFhwAV*0*6ph5l*K%`_Wud>}r@I!pVZVF35B*C>>`{;zERSqp5mi`*02xiD!e~f> z#v3<~BH6Wy_Z76vq8I<|PLj4s&tIT`9(NOTO3aiN#tF+mfiRs%I8k)lOY*#V`(_1u z<)$v3gI&e+9;4%mz~P~=s;`@Rf3XR%FvJrij`kEZVfSAez&mV>Hx3|w^9yAmf|m}O zQts~Uyz!s0FQ$wn7efd9y-6)opX~~E_Z$&G$2V=O0SLp1gW@ZSQw@e4^HmaAr&A$w zk41B@HWtH|P&XoX-GMPY-^+=v_2h&{dzRNB6N~r$+vt;)JbdmnXFhMtVos#Bb5rU< zmF>&RD(#01%aszPF+w9;#_&W zOov%@R#D_~!p*J~_bVdjg-{GGzcfiw7r8g8G#s<5PJtgVX?epsTnvElA)$}O%cNT& z)cK=M{LMHoEDj{=%V6N$k@`Rkw2MCwbyWay#|`}{tZO+dm8?UDDl%bV{SNbf%sFT8z1DSouXV6VuiXQ(9%E02UdxFmC6UuSLb^ z{^I$!2q}9BQx<@#hB-T1kzo#A{83IPGiJBhn}i$1A(lHp<`e5r`wIw zyI02@rm$MQQHSTAqUG79?R znQjgWOI@)cZt1^?AJQ^eg-X=@OFng6k>b|~8-8*oEofZv-IV^=Br4$N%&{l6L`a2h zj+&EwBlZJwWx=u^jb2je=m>g*ZxzQZPs|=~xl@60EFl(F6w1`Z*bPmfz+Bc&nk}27 z?_Fb(t5?AM-*fCJxbqmC{ojL*6Ce@%0JdGxtxqijxg`EKtMvLz_yKSNPF zlzx`J-=0~^TSzbH)k92h;G-(LL}eYvBEfcq#Ui26laBZ6AO~==U>Z^iQEbxFB>~9b z#ST9S{D&_~%gbJ8bIy8)iJ=u>|J4YSlOzGw>s0xR}#QW3QcUsJ84T4hiBUQwoUWd`TtBd5~fRjrr_I0BGY0 zdaxt8o$zrIuE(;tDyEN2EYgM2Em+Wtpp9cmw0O}JI~P& z7S;ZlJ;6F(rUXUEiM|m^>t)m%Vys)&b$z3D8RE$o;&3z4sIu<<^g;-%bg1Ocs0rvY z@?Vz+xGC5VQU^Kaa7Qp zLiaQWhgdj6;(g&wp6NzP)!v`uW!q-Ci+s-X;oXvMdxvWqgmwUGKJtfHuyrHJ+fQ9a zQqnfxPj*>STr-ZPd?ycb)T+2mV=q zH`2{lE990OgYOx>Qa);*gxgFR9m7Ju46!)99rsdpRyfBN$Ns0ELC6HwaCWAPE<&$xeXGLW)6K9Toc zPi^w0TpTlU^s>jI%wUn}UrRa(|rO#2G2HpeH`$28WgXvlM!=9+Saa z{G#$x^)2mcX*j0RfH{P$z2W}&7hk5>1uH8dfbL5Do&(CKz4`3yagoTL&gqeEOU{FN zL`m>mwO>RGJ@2-?7sR16 zyU11S5ZzzG7tNOA5tG#SgW?ZA`!B00``$10oUJ^VA5}x_c8oE*P!D_)Ym3{&NiH4} zYWaCMj>kgV^QeW>-s619(<}$&`?N<&Xk)68)6lS3S6%Ma@v&ja!r-@uQMsCLNWR%W zLvAGcP!fpO%@AcOqAZG2bf+DjkkqZisc|W{BMemJ|KRpT6?~@bE|0T>mM3oKjSb zI~-#EgzjzF=DjcI@SywC;}~S@9E+Q~t1(*$y!Gleq;Y(Jl}1Dxb0s1jBo2>$R-`xc zv_J3cKgyld+v(t>0h81hyaHMh%?ND>Vcz1KtePUteTy4EM+vr?wWsq;_mo6grEvU| zYo8;82B&8u!kC5wY)MgdJUX*|E{SsI#TQDzKxW62bKDDQ3Pn)fa0y-K-g^C6edwRa z-?NiBXqM&V(-+o1LgR0nGcHtZjU7m!nss@~l_p7}3U9>*O6d*MB{jS9 z(=vc)e66ALwE7wg5^;}?dc8Z z*#?<|^#rDDioSo%8N2jWgX<4qiJ@5PSJ<36=sS6rtJE~8cHjJD2*2bCo^m?JF;_>mPcm5hz8PMqvETHpzi+W7OE z-0}9N_NPeta-;&6=wIL1rlhM9Sc0&V9pC7bx`gKvU|k>B>|!5XHBKH)BC+x7n0O#| z_Y$=wn5e*=1b~G@Vt6|pY~IdHhG@$P>(B3*Ed2+O{qV4X_ixvwCa54^nTwCMzQ&Y=J_dRiL{K9K#oy|?GGCLLkL^Cny~Pn#_Ksg>;8*3 z%{lRJgvmv?GXA8&I`MjEBgY^#$g3IKIPt?6(13~k^UqM_MWetr>>n3PfVjL%ON@f4 z)Z4`lgckelLc3;cJZ#n`g~ZzSp6A=608|)!Jf4IMz$bM!6}YOjGJxhkZ)v44hev9a z)C5|S)+c^hxfj)6QgG}yTb9(P-KihD#12+{A=tI}9n)UrE&v>`zx02%JvGBGxBS#e zG4Cs)UNtFeNCkA6$&`YS)6c1(7<;$^Jo;&O%Cirzz|z0DvhJh@Tk&^e?N}o4>5b|5 z%i=4)WIug<1ubucqR^jSLN(XuH&Ee;jqI!rQzw5m*2MmkB69lN5lx17Y1T=!L=J1| zmMxZsS2$8ZjTX5RakHM#2*+KbKC$hJFnz1K%7ro_2 zwnOg9SNi_bBd@d&1-nZFsZ7GnkyOJfMW#>fVf&(THzVqeHGttoyt9^+@VntG8Pw9! zvv0~aUST7pKCX5MuzzRj4NLFlWg}DliL>4cyk5kyJFI7_L3fjL^0+1#Z%Lcvp$o4@ z@JDP1Bgs4IY-c70^C#fPWbG(gTwK)mM7vw3bj1u-&qQ|Gm#zGBYeh33GCxcP;%bD6 z4PaF+C`4}0965|Bu}Qo+bUbbh!%NdEndpxVUgUtUG-TmiY?DH<-5g8NtBGB7@P|3; zE2s$QtbQ?2xK8~~q91_8?Q2yRueQfDiDfj3W09fmJkphFQ7ip?vhGegSM`l>7Gxa< z7p0eY-zhaw^|Hpl=QTGBknhil6U}M@by0wV7qrf(Zj&GPK}N*|SASI(-v1)xfvi{0 zi+pDdTZaoa8o|x^nK2~mZu=qMP*Dj=L{b(ICWr+1Px0dp?i-EB$v1zQBlk#fa?! zhbey?XKVRwLB8XWsXtx;FTQ5_pn%8V>XCR_m zw@SO&9x2SC@P<;?inF&$cSVb7)4Z6d5sigimCDOEhLsK@zRLY(oV#Cb$ zQ1hKJh=6CKVVxCE7`9@X6G#5}j1w6{rE)Mx2GA&05-eDM?)z{+x0Vz9!uY+8eVsag zzOi+v9Lg$-X`b!Q0l|#DU)4v3GNq*(qay{jQ_ogT-vaU2DCbIDexcUGVyJG@;s!!S zuza%m)B9AMkNcpH2&-%9-|!INt7}ut0R9TSm!lSWr*v|kNLOouIXjYcDAkx0oNn-f zrnh=ebQZ4w-#8yvc6o67G|}qB>|osUsh@Ho?>jZghqm>N3rxz3X8hTPN_*~p9cw>u zyC{05Gd<=ur6#5{bg95r+#a9~z&#-CqbD}Yp~+SiO1Lu1+lxP%KT7k(5%gGRkcQiB zgNprlw<`NdR@<|`t*E2$jA*>$t^aGSP5Pm!_K0TzfrUt$2R;nM*!^r|Tw%vh2ZFD^srw%}N6;Ultow zy&>A3n}*)$!_J)E;EmF;(Hh}EnAh4H+5Ncf0QjDrjt6`6=ukjatKK`3zw zJ;{GEvWzg~Lw+6NHN?lYR*5dg_ef1=Wuo*BCDgKmXvx^(>hviJ6qmo0-0!GB4xy}} zn)03fU5{&=UUc2pQCX;Ka1agAq!Za*F0lHyx5bz!n6g9}Tb3sUujV0s&!4VotUHAV zV8uKlC%JMLoU&saWtQ$x&;INJdYN!VX4T0n9$t3%4fH!auX6-_hw&s0*Uw{AOPI=& zFC`^>w6M|P48*(tq(ATXfZP2|ww$wqM>)Q`&45L{E&g8_buyuxXNo3SM+`gHA1j;0 z9psjN4}EB99&|!#n4Qvn|2Hy`eOMQwtt2D*8AWDiQCZ66HnO`tM>b=43Ep2*Ww`JDC^LPt)3AaG<&ZJ6v@PrrbVY=@JN1YNUhzJn)mEx^nx+R-IOl7O-IK?HV7mv3fX7=6rCW0XLPJ(%fs%Uw~qyX-mbZV|HiW5 zzLuA1Az^qNDOt-ER)SVe%yJWFj!Z|~$UsYB%mp!XdI6l0jsirg6_XbH?8QLc@Z_$H z0&?3bY~MCCvRtcAapOQ*z%YZ`gJJ{;=3)b0w);Mv(C+7nuUZQuvyHxzc#=v?;wlVm zQHBOnu#e=ZS-zU!hD-drMS7pb_8d9u8_k#0ms)TwAfGB9KKTUP37VOueyoY{m%rrs zu1Krp`~E-ZAFNkA+w@fgJd9rkc-oyA$>)x#4z&GY+F$;ABN(>d^cx4T`BGuzYNJyo zq{?kJPQo`h5zwG*m|m(GV@v+d{k5_}e-DxqpTJ>G)1SwQ9BP@3-^tY$Z-Z+gp3oBx z*PYrw+Q!R4*}GB_L9pssFFML{ZBrjeY@+20;4+c!n`YYm1j`H>!Sx1~H#=saYP?GQ)SVrE;Z zpXSMn+3H9MXp^w^F&|QrWZ@6%GCJ=Apd!8KJG0#BXgPLZBy|Gz#At!Afq`n>U^PNM zgE$Kia>?0&_H$y_88SlSCmHf>4_Y$J?uOQ0H#TG{`w-JSD$B~FQ8HpcL1m$lC6z;$ zWqC>skq`MCOu2RTKMfm~BGa?8;(d)&J+xc; zTn28@a_m#cF=s@r`b&>mb&L!|_CDl=plN}jqpxgJ=$5t4Ht5p2`Uk)h!H;cLF2X_# z0CX^n6!?VmW=-%KpWpbq&Pme3#SZyiDeYqU=f50m5!*M18p-wWZ^?_u)&bvEN>Hvs zvsOfwM*_N;5e1I6s?07D6c;QIvA~_`jnvZ+vH6)n6_XT>6$`_7Ai-O+#lmY@G`-`K zsH_SzFtSGO=aQ{T zV#M3*JuXwgEs4iB_M$G>SC<@>fv&Bs3z9XaD8tXvtwLXdphQp@hFOp#?ag2E(9>FM z(e4?4+)-ghx@xB7A476u^xsJsjyA!D_n%ok<_mN`S;rM(H#hv(&hx&e8Mt%~E|&&v zSMt$pyCXSVB4GIJrU=xNyOGw-<->q=gblbbGt>2?xBRE}^Ly`sc_RJsvz~Dhcc*lz zzOiN1a)jzOkV8fNP6v-iH$6&%pqJ2968${_@q}VIFgUj^PElh1E9E|4LZVT>p#at< z(lF5}v`OK=CxUJF_9x81y%4$2POGSm)HemC?f!*1{1u@&0=%is9H$TW6Ff#T4ILpq zZa5T?4{NHl-WpN`(*d?l$3u)&e6@f~Lp{Xy<#|ou>sTT5l3{sWUpo&4?~lV!Sy_L2 zOG+@$Sr4wm+#_*pCIB)NOAQocv;PBoeJQ9b8Jo*%x)gB%nu{Kzzj*>vnK#&lsi$^% z0najlw*6n|xz<%2qa=uhw?w<3<47lOy6qI>d1DdxLd5Mf2x$aG8iFXdgY~@^EC2;? zdm80X_cP)Uw?Q@@BL74Aiw5i!2Y)GBQArB2d!3al9NZ&Q0~6jcXCUuj2K5$`U~#)Mz9UP1eRH>U0G{2akvS}4fgY# zoq+ly2ZrJr(61U}lCSIzT}pZTeK0)EscndL}5%umMavP+V#S3!tq<;D2an)3b|;3!hJ3HH>?5(cKVbDSeT9daYrIE`$0>HnCGM7 zCl8JK{8tAh2q_FRH`{w5e@kD>^6?;Wo+_;WCk6up|EVA2E;_Nj#pmHHIlgT24_}4A zK1iICIT(GtEN|h#CC%5S1{WtdkrRFtsIk`=2hoJN^lABEKzm`}th9#5h`4>6stw6Z z5vyGH?U5J0x1?ot-{C#o_PF}9{1{V7z78001u9uPVg6h6xgS-aMC}U_xhh{-2gL<- ze}INp-IH!SsF_T#IW6jh)_Q%I7<#L&0wEqMWvcBwhco5ZDSU!`3qNEk^I|GWgMPbI zqA&a7)vcZ{iAX=vWGYyOWtGShr-1~yC02VPN#BTBpJ%E_VEQBu9Wgv7h4AvmO%>{hX5;0g9ZrJ!Slp*(cma1xCJZ1saJTJK#VL*) z{v!9)e!(DzZaq8K&`g0o6U}XP_%dvmvyvPLh=?&>eafZo{vB+4)V*|oINqk9hD<v?7IW$Y^X|Z6LqSC0fZUwJUh<`yp_u;|2`tIcB|-&Y8rdRFe@0ivm@q_S9OZOFZz+q1p<0K%rqBk+*M|>4|^zoHHQkSK)SNM z^i}bdXe91Ul$UNh&;0KkLOk8Oyx(-fU{Z+YF!}5)kZ&vCWzR#`T1_NtqAsc&z&zp0|Em=jl~w6`VXIeYO$c=1si|**A7(b$Ec&b z_DM=eRlu|Ws>mK&w^bE~@_WgsC~UH|wr;{+IwDX^7k?+8OJ&6PoZ~m{1u6@fy#!Ax z0=}A%o4Zj4Ee6M11!p)7;&(GpUVZ52amq;ofs?Hao4t3CdtPWQ_Uo6YHs3w!jM@3W z1dW@s?I1nfRtkt`dEB4+4_)?1l2U?(&A|>|cKPMj4I!>A@FGD5eZOshxUKP1hgASb zo`~6CB^og68&gws!4!n3OG4L>HRP>Y(U2;_cLF-TsiBk%--h8-YU3K+4A4bHL}98T zk{ue@!kXy{a8L02ml!w~lZmY*K(yLdG?c72lqT7~*F7uFpx~$zovk$ zC#o}K1%fZqZn{4s@(1B2+jJ37w%AkEwAU;^(9p@3k%=w0e(^+MewrzNvgEJKQz9#^ zR^F?U)^enMQ}ah?XVO-%fAAh^(8)sQ7%s^JPzAx2*_>M^FS;*62^_Mb0Ew{*c(IZ9 z89fNQm<)G>xLW4Vmks(yYj!?r8?&9w6A8N~Pi?>cP(bs>b-@x~Im7`oX$gi^rf88- zm|txKFWkiPNDctS1kKDPBn=MgH1&6m12>(2bAd|AeJ_f0I>6%iX)Xt6bB;8oU> zeY88fA!`3LwG3-3_ZNcp*+876df&W_8hYBO;bM0JLp%L$^L@DQS9b(28@6ujylyn- zB;>`SuA@ORv$|@Ufbr0q0eQ3MNZ3EIqA@?l@FBU`B0vHblDbUCY8otaR7S0+Q|&k$ zAr>kr)0VRCO0r&EU&1-(;b8hTv%DHvMAj%IKfZGbLnHWasMVZdA%vEXfOJ{|AprRL z$q_NWYG$Ry^zHtok^l{$wd z&m$v=EKg-NSBdNehm?pDWLe0>)zEUVm?fY7vUqZ0tSYmUN;B$55%W4j?;ZYAtO5Cj z1RKZpNFV-hxYVKqF;+zHE9DhRM$>%=Pj+ZBOHVK$TWg5gl?Qx(9|#+dI! zmD@;rbtDxX{=5LK;p=)mm#XscYN&z0lLa>D0s`p z%N*srAdx_G+`-8?5Vtj%@l8Uhc(K@6jH=*P!A9n4R9rsz>Y-BAvs<4Gmz`;Jmt8DJ zhIqkF|Hd4l9e2dN?%6Mo!P40eMrEwDMz9`48BC*OBn%^hBn<^wmE9+(G$IZNUlCf% zGV(g({r1lUz$@|66d>o{^~g#+yyqoz7BdZjR-WkWNpaOz@7*m*>GO7A6>j_pSfM{H zf#6iCBO6)&ANJJ$=2!I4k%r6G6Z4kL06DcwPwrcDnkujWer_RwL06vACVTdIN3Qa) z3H8l133$8tpcv5mr#YGgoFLs-#ZC(G$srilrpa5^nOe6CdcIsM)h#PM&&KA zHvBt($J>Uv93m%(9@Dw-Y>L+P*-t_zOkMHsZW{EllN4RuA||}*9X|>d^PgPNlr-rJ zrwlXr8LLz5A|jEHWOz{=2EisKC67m3oP|V*>O}qFTR5|6BHf`RCW7nJm8BKtGkG#W z0d~|(liP_InH(y7`77QR+#>=W+)x~}`o6{x#^r>`-M8|4B3mOV*HnwqLwB6mGGiP% zA9m5*brSEvcJEZh({gv4V@_^p+Izn0${ti$-@`tAI}J>`2bXruN-s)pR|9zTeRm9O zHO|jxM|Js7#O*gsH7azm`EQVQxL+AD zp->PGvemFb1KqD|qh1}r2e-rD(K4|+HAP@RkqZ&p47fnu!F7r+-X8^@9#^#DD49xZ zSUUN(N{J_jdWDoze;Lzx(HHmXmmfjsX$C`5x~QxeN1}`=q}5ICXho9`OFG;B%%snMQC;)^$-#{n{;ZbOiJC)V z7+?*1k#acGnDpoy5mu4m(F<~wh-Po#>jNWa-@B?4{HsDoSJe+Ycd!W^WuELWR|sg( zLNRJcS7$sa+Bf-TZ} zWv@$gy@-HEB&+B?QNwfvu8~m!d2E)tm~n6{#Zcw+#Mp!ac`W!`zx3Yrm^LAGQu&~} zNkI16R0v+kK$$o@D@@!RPEAfm=3VS$X6?Xf4lyVBF7psE@q<&4;Y#x9T`^C_(s~9E zNNebRONKLt>h%5Bz{ujl(#Jq!)T?k^hxkM=zBDM_NKRP%V2=R#7H-rI!56|YtfAps z>RfE;g~=59je^5bT=f)EGLDgO2->5vNPS=@{TS_Ui(JcS{c1Y9CRfrMEupaz>AA_S z$EV9O6Hha574_vFrm8&~mxU>D`>zp`4^Atc9ZBlR(aa-?HhzIvfBx@GHQ5QC_h&^` zZ9#OT@4Z=ot+P9h>=!r;SV?BX!7GD`BBI(tcY3fUqCcT0?UI^aEhQ!QN-TiQFk;0m zQ-KZlgin>Mn;13?#!F-of?jj9qI&vo)gls)_?=(%_V&dMeFNc<197iaK~r3UCQL@a zO)rvt`49b?E2yHjaJmaX#X$OZ=qyJ)gmUl4Z&ktR@AewR5g(#*-@GVXRN-AkN-~-L zJ%090Jcxb@4E7->k;qh^-+?9!oG`4^xoU<0FXs|&SuuWrS#5l6@)KwkJyKKK| zXmv?%%gBn95hcra^dh1A;e`yT< zHMVLG=`>n_@w$ID5cBjFQ1=IarkUM0NiAqF4}*wT5&Rg1)scwxoXP#m1Gzu7abw0p zWK@Rc&_6k~)lfl3l8c4fs`KMTzhb8ebeZm8(2P8_q~2sKa)%UNx@G(g!ZWpWPD2ly zo;?Lwj%{*$26v|ygjOmaE&DEbmf@kd^~o&-=-|t!^`|c>*#8cJPi;s*v=IUbfKa(K z3`q@wvuoByb3$T9_9GTfGB%fR=5)SR8Cm@NhLjgYMRhb*L;HhVvYA!bTx#rPh~fzc z+W}|XMe1%fDRZJ_b&loII{LlrlF=T7yI*Yd@d=;;2nfzJN=U4e;N&v2H09sW2+TVBN?cpmQ>y# zU}$V~0`daoXeU9fBg0fSsy$%=m06jeafoMIljg7!f;o`|uX<;Fx=n?LyqKOg-$q z5^T9})%xX%dc#$r^jcR3@{+Ms`jG@rQ`aq>v0oRBpoQ67g8)LnYk;=$V-r)wTGo9+ zFQU-ZDOnxve$P2X@6>NP@H%Y}0}lI-hE$zH2ghCTQOVkX0{0pf-tD0)u6{18^!?}) z;#{WC6{OGnqFupGz}yU{Ynccr_(KOFM9G>}Z)`>yuQxudx{Y%3YR9E5aHD`hg2(D! zZTVNvUZ9_6pNd5#fCV|y+<)vJf=UT=SO5_@;Hvqz|J1^XrQL9*vW{-nWMgg=70p>& zP~oeuL76gDjFnhOs}WyEDap&oI}phUH%l`9%a<$wAx@{TxI;mM`*MY#*aH&yL#+@5 zjSx$H1HatkQjh?0;{LFz3b41kPw8&KwO~d+vlP>1S$%Xf6sfFt{=U}%{J%ZH|LI#y zvRhpu=j*k{Nj*g(@E%Y;sBYlc^}uh(P1=l8tKVZSZbdC|R(D>L`w+#JI3Kgx7iQ9^ zB{YLKlhqkaPiMh_VXrfuG)w9(VG0)Bc)g3Fx*<8Yy6VO*e5$$r1+$aif1X`(N3JaZ-TY}cq$oYe4N{CLGi_MvYvF_P3x_$e#08(t z|2#mfA?Al&I zfMQ{%FXpGK;$3b_IU#cBy;9ioEsI<~>f!FEXeH5;&)lN29FY?L^WsH55+#>Zqw{*8 zG@x=?e;)oPLX`<*Ql2&{ZiwNxlS{vpQqWDa2b=hUW(6yW331&7JG!lpy;3@ES{(3# z^OI*XYuA5!F8O4UeM+3{DA5xioK{?3e`M9yh)FwRxubK!toWQh`H{w+ZomSv>#LEc0C%0ncR%@-At`YCxE` zBHY9L1FUMv)`S;(IJEB+cx{^6m9?|cNz1>yz1AJr7DHqEV=#_sML*p+Z)X&!*J zB|GKcgyQ%cf}}hn=gqgdBFuw`*Ki2Jg$KmH+&>mMXgU<9HQGvBp~1SG;HwYFtji4< z;IT3T4*fa>_dpy0P2T3zq;XIHHR$q#eoQ7E?uXf0(tiM7u61oO+>RE*K7%Vk-rFTY z&Niovot_@D{Mf4BcX3U#qf#1#YW}fEG*QXfr5d?`ZXx>uQQ*8{Rpb93a6>f~J#>9b zAw;%hsqFb8Of{h{txJ?j=+C+hg2JM^%HNe&D)(l1x~fC1ROf_yYi7~j`y;})l$gt1)ZA?&I z+r~w5K|7;sBK``b=dKu0YU6ML^QTXP>k~e4S+Pa2VO7iVN55tW2vJC&5MvU-wlV$w zH!5&+f%KKipO%LntfMc<$*ziHSce@lg zl)Td#bhB?MF$mo7Y?LEPL{TkkfCA&kD!NOeWhiTCFtu0?Kfott|W+!WhC~aR+ z&&fXZnt{8<$!y-9%2<5OW&aONY$0Mk7U1F%jokh|;OGd>S22OU4hCOS2mhEr)-M>v z=ePDkNuE5dyz655vE}U{F(^;O=>NlqD(Sdg%sxe!nq8kh%Hz|J|E&ysnc?SU+Iq2U z)NeZX@&$FW3m5S{{}&_|!Ga_wi6a#OkQJI-aooZVsEl^alreHL94E8f1hEI5hR&)*e!%Ce68 zcbmDT`zzdEvhmc9j+KGJ3hoT2&<>3+9@ttKLM*}P{g-cA1VSH# zz^~IRve!c}W&B2iIXKymCU14hXhe;_*J))8Js6V((0aQTK_1#o4vw`G1>*uZ2I@%e zWQ-{berTmLkM8d!(Z-FIJ@iDQbW#zWr(>9n zm@rvzmW$` zuj&RD2?J-_`>FE(XlS}6EX3W3ktscRA<$fYD%pce#7=KtSL?$=a}g{u|C^GTBmvzm zveNnT4L>VEu5$*CYmL!jH=aK#!M>oF_L{+QpA?% z9W*MJxeA>JlZ#Ki)||;R5|hhQSL#~vgcd-oIU4oJe;PGjok7Lry4W3HImwreVMMbY=^)?Zgs~8Mbe}d>z9Q zmoxs2s;|P#s*4VaP}>?NWBYdHi@Ea0P#OYCEPY8F^YFf?eeF*_J=`n)Q^S(sGwfP;Y6h5x@}G9D4*VOz+!RDS ziIqO@17ZCiP{W-8uLY_0^eKBnX!$o)6v4lAmnM-QA90FW*5n!Y{tMGJ^)Z$Z5?g@>NMw zf;m8v+35{gxQo|-k!3~2y@$Q6!H5*Z4PCmm2LAmEM%8v(P2`<}KA3f`sj7~XId=lc z%b!;>1Y^slL^cWUcOgO!fR_AC_ohfDLHVnGl-|52TrTxs>2N^UX!>DjI!^JWUo*2# z<0>(cz)ChcT^=jZqq6Y#N1=ybeB-NF&4Whh)uiPK zp8mJXXhfRM9%4>}1m`g=rT3g?OIybb#vo# z%#%Ww7opy?NE8F|MB-XPt@HOpWDe|$BCIk;Az)a6jZDcjL4~Dm%7eV%ResTq1pd4J z#%PWbZk#drIT^^9BUZkR9LS4*~+L&}$oR z{pR7+Kp}Nb)Cr-PpVNAJ2gYMBbE8&2feuOQE2H?;Ki{>-qf&`jz$`v8S)bz6-~jX7 zn!tiyKnM4uDu>DObGC;`)H$h!p#jGBqnEn4Jk1CC;Cb`gB6P(IELPzYI_zlce!wbC zN5c}iL9n}a_(ongEk=VtVwqp$l2>CRQ!hx;Y)VM@5MHMpKQ8!G+d$W)fegc6Dm7nt&zV`~@i$@rox$LoFEFK1P zVmrU(LOSx^4*o!u_lz>=xv@t*vi+7M+j`@;PVebBM7X<15I^oyvBn!vl(My2yqt$SJ+0OsmY zW0<7H$CzJ4ab>310%ellz}=(U#|K^o&MgfCeWkEMcL>xU={@|;5`CRDBx1IF>8tE4 zUKwHf6=8U8yMhh&g~X_K5(YGr6$V^p8MZ3Fffe+f`6sQwat5x^SfRDn-6smohTjo@ zE|i0pOdgNVz1z6sOVt>!Q0^jP>X?Zr3B)y%ZEy>B1CFoU`;+xwaQRRF+{!a??$kHFqhzOwp}(AOp{4kctk0cbi`me%Bws_X{Yjrj`3yMYu{Kiw4USS?rrA3Qk(=B`acm_2|++(!XW z*ufL}9B=vjG7r^vZq?07)#7_4Pkl3lFz%C@qUR@5lIay7&gyhVVX=W<%FUnoR2XCG zbWwWWp${%)do>bp`u3tvXft0c+(dg>r0p4aboPS}e?}Sk=HvhKfvI3j1m}9AWg~L{ER(#1FCh6<*+L;?T~S8(cUfp+E177M&hfu0x!tPOvl$S% zPc4i{QEm0D0J$+enfTfm+kl#^i?#3UB5rBARf5&`;ZF$S z9uS9Q!k8bywCWl-{aTphDr?b-^HG?=PFOnrozzZvNJ zaIsh#A|rbSaxU$k%jo+pk)lq<{$uIg`upWfso17yk$qSrKif7lBlCJ- zH#oBs#M*V3TizM~jM$v`n370Sj;?`#bVhU&$I2<4w@eqb{8iRKn zG&Twa((TuMF(JKW(dL|G-tCK-#Baybn)mgFi<=1aIkE+0nqcUSiD!DSxic_0owmfk3O%7ED0uZxwt1KKi zCD2c2i6=kcfzuJ$P;9e}LpRdWlMZQFg8v|pTXg93n}4&g?ydCh{o_U3J`Fx7vnZ{s zEY}4%_ZZm$j?7RIlt94Re2P~hCYt(uA!$kVVWrz^w$|FX83;%m? zJkB3~1$No%ETDoN(U&J*d;V^y9bP%513S6A!d!Yf|$z}2=}JC$Ix4^ zvMX1j^cu>UjK?a_ykE>0NM|TWd4%?m5)Yr5fHf+;vp+XlLx>F4c@P(ahu;r7um!BE zs&w6u$YGo8M11O2!(+#h&%mp5Jm3dc{Yk3>Xn~oJ5@;UkrNSz|m)=*8YH5?X@xk~= zMSiV@N}F1LN6J(-73JX(x%Sl;mrB~Ghp-8fTCVPWKm6>}KK!>9<3_HvbC1vauHI?y zt4&bSYgrMuo>o0F_J3?Fl1WS5&Q617?(~eH$MB!{NK%tZ+%%8cb2nmcdGdS@DbT{< zo?AlVMf&fuSE=3@P~GWS+puX^#(!B-RTY1#dtzZQj5=Ua`-r#?wi@2U(bs7x6{-B> zlRaVa!}W+5*A{X+{hYv3U5vi(oG{ElslQjjS8jA{u0))Gx_kLyW;Y$POX5NXK}4 ztOd=BJe{|NLkqc*qNe@%mIvFIiI2?1x$BK^Igd*S4)2E5*Z;@WUxj70cJKf2O)DWt zh|<#CA&qoOcPiaTcgUhaq&oybLO{Apq`Nz$yF>baO`q?^`+N7+;l?_+=bTrJG0yXI zYV_ZFz^Ed{U&k5(6B$DWZ9T8rgMH8dQuZ^H_dq#v<5{8q)ym_mH7SQkn)p?D6zj*U$J`Gdl>Bim&v5Y^sM}3!6S-}du8>!joBtP)r zc`4)SjOS*nw&L;J8?!CIV~WK|2%O1wHVx>%l6WWVyR9W3@BkRk<)6?0P3WnjYRTg$ z30>+>!S{NL8p68!#WS+vRWwjZ5!k~J{6Z>g_Cy(1StBg@PWAx_`A#gUzE2;RRckvc z-nssrHu%6!$uB(P{G&@*JX(yrULG zqvDi5%J~O7|54=xXcw8iGr~o6{Ijj~e*%GmhY6(iWAm{o(eJ_)eNWGR2iIdn-AzW! z(Xef+Njvhvv0^Dqx#(!biEP2G_{;i?+Uch3KzP&Z^Z(KD^gBCg$XiLXw~neyZ+@s- zO>T;T!XNrl`%L!Z5JR1MzWUO?NU%>jcYdS+h0EdjA(D7}lvn5RRjSwn+E*Qvx5bTz z{coZS?XZ=P@=tDl6xJ)pEOzJ%(C&%5BoRyk|gaCXW!|PKPiUaiYwTYu2V1lDz z#5}4d{`dmeoiz(&SNz{> zvH9AXGc8$mwt2o3HrhIFH-YP;doHyZKMeoYIx^}Imr!#dxAeWigWd%vST)CY))nv= z*a4$>CpstW{Q(TPBlTTGaXo%C3?qGN3T67Il23)0Kz3P%a>C0y;H%8j0dAnUx!)#>Y*EAZs&wEYw73q(^MHFoenlx=y zxP)3L3gisGWRPz|Ns3p82t+XXX4<82CPD+V3E@lc6+1C1VqxwE*D4(0(V-c`!1k@N znm&b63!Q#127i)3iA<$x3;hI~)=BIP>mrZlrGaLmH=(q(b{8LP?z3wKF+*vKtq4@I z)K;)0bH3EkT-)oqi1WCdr+?pcO_)H7`-S8laDc0?mKPc zd76%qGf||>f0n9jKT2l(9WS`#&tC!<{W$?z!epc0uUGD`SnW=i+;jg?c&_5 zaSZ@Bv+MnQ6-Mu?vzYuQ!^f0oO)~mQJNcHi$WzqV`it^+(HI<-N|1q(14R4v@$Qcl zZKg1HM6!!+)HWe{NV@+-_)<*9f4Fi-pKtKdug_!6Nd z?&9pn=pJ0MjFu{!FS`MVBmE(gP?9QuoC_4OFx zdh1NxYfgVW8ISo1KT3gNf-eN{DGI`>6yP0b zDU_cqWZcz?$<;Z&BE#n+Wkv^-4QEgd^WF|I;|I&hgub`n0DjQ55-nkH74(jNEr4DLiZJ6&9O-dJ~LqoX&pfxe#ys%OK z=7Ndm{`>B zB9xbTt^4Pn!w!OxfAZ%0YC_^D*vt8+Vq{lk$(#00i|iVm+syu`_YH;7n_WCjg0ca_ zgg2HKL6>?_p{Joyq3!(byg+`=68H{5xVvZIJud27J4zq`M7uo<=y72#=nD`Oym5ef z=C>p7rkWC&*=An^^!5&O?5bf+C(hG5Tde7d0nI^m;(<0C*#LD;^Re4}9}z%G+0&kj zfcI=$obyZm!D|`VDBnLSP9_c`xJMCxLhdnV)M5cXAbtoh92`h13>r8B1w~u@dQl$( zq+<;^a7OK1jL+g=+`zCnW8C{Hk`IFuZlJkMRUiPkC3g9zfyY7@3G<~TYP)t^l4VRv z#Sgj2OUwS2&Zq<3{1h)Z8&)k77o(UIe1pV=3V%Cof9x? z#z3)K{H5|`Sd$^urW!cy%JKVKzkia-z|8tz1LbxU{A`xLNeJ%q3v)^>zY0{}4zTH~ z#sPP2Zehf@Mz6I}Gb*g|YoQ@-g+@Rn_>pW(9FolU1@`Us%H48p|1PlS}$w+HmwLz{k6NQ_)sLg*u-wBS-TM{4_JQ zruD9TDEsAUo5g8roBU}WEFT;0fASoWx8n6hbeyMnK|i?MDOPR*kku+UxQJoGzlY7q zr#C);quD5PCbNBN7G~fg5ZUGdsvd0Dg6P^9dig{6H(s}?@w8?2;pc$Necp>zIf?)l{zuz?c;!h_rmVT@fG;>WYmv70}%TLOzwtGwQxX30H6oI$-kaP zKO|bfj?CW#+KBpL&E4A?m>h9q5JJ{_2CUJ{P(b_g|9Vq+@KjeEVcd`DzM*~9Qy}v^ zH}dAN6Obf3M$j0Z_ue$fupGu~J zRZQL&1CDvOQ!09^)4(T%|5Rlwmf65g#E$AAD5;MRruRb|StC%7T*osO-j?~v_63Bw zE;sg}q^HLuWGsE@8XT4!|9E9igO&wY(#b}YA8V;jDgDE2p3R{}xgv7`N%3xC3Iz`v zQDYaSNC@k-{T0s1&PFi+huO$E5a|Ka+C~&YHeV}WtapW#a`+i~4iQwGf@+<1q^@0$0FRu**v2Xl~;K&tq2?wsMkg9 zZOzT54(82ix~`@@SEM5Hhwguvg17E|EF~GG@JZ@1l^_23D)r zcrx#1#=xyXvi@AM*-gN5&kkmZAc4^bbVIW@{rvMPm9R+m>28N^NISOs&{<^boFuAV zpPk=A$1r^Z7C1xOt6-hKDYj4J#l`DOa&7vTVNJ$Ad+bg2ghY^P{p+0P<0DSXY|p?I zCj{!_(4UjztC=QiUFLds-31(2*A(<}OMl;xixl83*&SVPpNyFZ3a~is0_dG~khF#s zNx2E|iKK{xs@iYh$pxjEJ}gr}upCn3M!eh6er57oqhL z-t>k9tBa*^4Ybv6wn2yGGtli_S!pJ~^ZXj~Hju7>(jT5!ptUlFucT*cHB z^_KSynK*bcSJ^Y}N5i`h8pQxY&GC=@%an!KRJb;ClMl>%S-A+$Hv%Dfqj~^Rejm=Z z1rwnwH-(~?F;v+_tP)9*V{W{M>H?5FlU=v8<4KFP%{+Sp1JKiFwNX?zU;b#+!57IJ zQA;5bs8#}Ha6}?X8N<1TvbVmtB7zcHj~?t1GbbA3kSP6$cr4%nl%QN*r?OudLy$;t z^jCsG)(VW9KIt2Kpiz~I(G&w@Ib;vP`SsvoWOnyT(KIC(2FS!8K zBXks{?-p}+XOr1~f(1HTQCmza=cy%Dm2^hN$D2x`c9=apF4 zSM?yk4qML$NhLhZ_x=>tPyPS(WWZFnKP{|y6+T(WF8S1Qw;UDZjr@H44~lGx1U;2f z0N?9j?q6@U2?YbaJRw6Yh)ZWIZsAX4U)%)cA*~$pit~z!e|4!lkInXcBgx}~*dVn~LEf_IN}+JWxZ`UeD-nNC z>S=PHxmpvwZib5eO4|uI_xY2)Cw8a+_*;$-PtxBo?hqq2$tauI*P~j4MBXdbIYtE^ zf3%@>Ke*eWDts`QH7KhBmm|HZVg#`c`t5$sOPoII!tz;vjf?NRfo^eCT4a_T0z4yG zw81v|=^R{u70Nhz7$5mm3W`Di+WiQL6I%bk*&h?jZ1^Ky;Y|gAFrf!sEz@4J8S7$` z{W8Kf@(Woq|DmE7+UqxqpC-gU3aNHyZ!BTM9ORrL3`>7trI?A;%~%%lKEZIW9m@_H zMO#f>L%+yMW`=qi&l6}od~l9b(c^9T>VlZ7*vw@iw?oC+cuShSP+#1->eAweip8Uv z%Ej8NR|LN;f!r~%Gg;c`RPFSvxT7wzo=YYl6504eKQoYaNMDoP|p?X+)Jwz1rgkJ;bD#3_-LpJ?`{G84kFf6Ld z=KmH-zs}r|_jymAie<$@WVWKZZm;&`%$6N|X9*O{vAjr1@TGV0Ym_|(2H2p@etTAD znd9Tw9#v~u={jmL>an>&Zh49iByq4?L4C8+ECdK}4KSct&nIVD)1QcX_y{V#7&R{j z&JbYZb)qpQzw+$b7f?^p5)*IEQV_A{&|CKKpZ@)CbJ+3eC^_!mg(RP6zjez}V>7prhGcRgB8aG_y*x>n1B0sf(< zm)tCW1OH%S77Gb@Jfl!-W&;oWzfj+Zm>$faOtAMtiE(O=E8Yt0KbaC2nhy|);1825 zhZkX<=!R&*tNbE;!MhT@Ylv@n(EKhQR95f**i8?5>576?v={T10kIAX7Cn&rpo0+O z1YtFFL8=QY(a;BMqpa^w&3QR=Y98 z?V4`Gn{4WO-U^VXhk7!ARsd=zj+r@2ox8X3E|z50<+tLx+eTP$%0ZT=95cz_F7-~N zn-C1AG3#KSZc4^WP*b)3_x8d_tgd6dPrr9d27kNx?`jvbP^|WH&2=4x^zRv!UW{<9k#&Av3SDd4GQd?~|> z)tL)VJedE@cWIQH;kcP8Pq(XERLHk;ha3PsU1-3!5piP!T=Q?EzjecTV!}+-EMP>m zIT7HTuNHU_*kfwb^uD1d>^2XX?O1$bFX2-VAv(VHhcF)w3dKeoNRPU`5gf6Yqs!rE za(*=g!QEhtTq?;u3(dRckP%Rx2cOu3Z(5kvKuE$9(KBRJl*23O+baucAA%8j4j$mC z01J#C}u756Q*g^#-KZh1Bb*A%G{vNdS$w0QH2tUuO>-#YidN#vvC^zun&xz>kQv-7~yvdys|UHt}f z{o!p)5k_@%D&*}jsQ-8WP$t0~)E@T%@jB-22g#8wtH*wVktEreP|M?VjbQf#rvO0z zJO_(KzpKVR=uh;V#IXXk(8wWw{vCeyd&h;qul|YpG z;cJZq#2-X_rh`0-;dsTW2*UBI{ehXy=y(CCU4l3*E;5O z{w%RX3gm|VpUG(9;GR;XT{W`n5xoQ~nR*Z&d{q8y88<)=F+1b=*JeZC*Awas7tzfg z&n3kXJY;sg^82eIklEnAgx;_!g@qTLe8W{<-~_u^JcbCVI%vS@lZ`7Rr9J;M6pygp zJaiZW*JPbd8$IXR`q`COzt!xg$PmZN#NS3)es6`U?4i~lEUs08QNeH|>+vUK^i3S# zR+dr!AJm5kE|hk;1QHpQ_kfTL@e)Jup>P0^e*{qYQ#yiVfy*Bd1`m%HL1b<;{7JA@ zjSwFsOY~3c2-Al}gQzXTKM5jU#y02G*z<|PzuLOLz^fhYE?>~Wu>w&)?9(mm_$~Xz zcbXayOA5ug+7=bNgE$yss8JLyP+*F^ayxDc_*&t!v6jzn?HG2LUtz}V!JOG>tCEUh zWQ4guGnZ5B?K&goe&or{j}ifc%+MRkxRbl5X0a|p`~L5ux1=J4d}g4XTV1Y{r`5X) zr{0&dIPC`z?V#ae5~f@gWX@+j+AY79F#PXBUHSE^^GoR5_tTl(w+p5}O#qWkQ=gXB zgwF+BQ$tXIZh^y-ee1#D>GN}lEtIQBs!{1a$O`A%A-&EGfEWe>>$hp5rH(*mv%fPM zVhtehXywG-fj*cB?)?Edeu6E34wRd)3tGbuUJD~Q6vUJhzX4Fr#ht^Z=6B#8MBPH2 zQ=603jjZ~1Ph&;0u^R^pU4mRou?roPZC4>`;BjOpDZaV^=#54`;U0{9XLFVTd^y?#ZRGY!fv$xlTy}gz z=Y{XkHs-nGZ|($iuCLT%+=WLSn9SU>)#I+C0Kt0J1u}9GSW`F~{S*ld_nmlfxg77x z;B8HXmaz@~imTD4fk_9Vpw+Y=eR_4sja+k^_jF9>i{)z=A;BhKv%t}fpx7VoOF8$xCU z8~)Njb3;|qY4p=CIy!jZPH&&W>xe@UJD-f$v)-y#3#`UK+$yTX3l~le!cczEuv-qV zmXBWCvg3*m9lJLLMkUcnEXX8=-V7YdJ`(uv821ap9ylS~K@ALC6m>*430Sm_hx5NY z%G9*Ag#)h^+%YMRDIL3y>?tY`@*o(6w>t<_6xsc5@U}Kquk1EsaIiFd@P@8wumM(% z$d#76xj!+X1)Ay7-Y-*5=y6E5?=J$M44F|n`o8NOWWD)1an#s@an!E<@hyOURbkDG z@GZ6AwD}R!^64!m-(Z))??yla^u!(?*2b93MaQwwjLqWxa%^CsNbAKHn+V$@qgCYB5|9gpyu%OD@uTvSJGowF(=g$1$@F1ksUVJYLN&Ejz?q zhP(LyuT|&`BZMa`An+>%&W|1}mexxa2apjc^~9kO$5hv<{LwTp6!qMIB962t37`p< z3~KAQse-?o^K2R8Bw216zz8E<=vLyO@vs;1MdVckOg1u-qUQLoAvyh<@}I(}m5uF( ztQ>$8-uZ&o2D&u4S>21QBLnGCw4Fj0vKXoUfoo=|qmK(GTkgnQHstRA9itEXn!^Jn z-=4~5FS)!-$OKyBO}BwX7yx<^c)C5R|77ERE4~NkKI=IlcTnW`{zr$KMN?uo$9nNE zIe5S$Y*f^Hv{iMXO4QoO31;1vjh&`?V-K)LYvyiFIxsiFUT=)>VE;DL{+ zA%(XRkkRx98QXKSldW$cA39kbDJ;4scWGOBr7QL@_5{!b54?F(A5m$Ze(CkQwsR@S zX$FR(k=)C#^>)!34bD*qW~%L;h%;}^q3zd`~}Ppo!M>+ zEgc>=GQSkYI-W9LWl&jM7Grs2%Wo?-K-oIm%rDenG5_qAD*pr7m*p)d(zDP7^fqp- zb<1X%zcUY~Cr`v&8saqhpW_v^Z){vRm|YdV2R9^q0AJke z5Sc2g^8X%O81K&q8Qz+&@d@s()7UNdTDw@S>}AZv#Z& zG1w7TD}Ypp%J-#WompBw?z`VX`k)H2Ex&w#Av?A>DIf_ms;q>(n-&WYaqFEzK|QE{ z=V5tG3dhM`xzcqFu--`?cq2nl0fy?pa_O*Ymw4_jxO1b=JnMFe2^3RLbL!$1q#8-D zu>OA$8!E}CJka|O0;CNdo89fg_8J)<6bjL1=bsxghTW`KDat@$i{4|B>txuO7Af%j zAPPzpdV9hVGIi9A+&|~r4iYz^t|gNeCIYSN2R5b=hFMhP(!F*KL0sxgZS*;vF_)Xe z2a(&v^l+aGaoZos8kPj^J9eUKJ)NjDk+P4)Drr6l|-(bdgALJlT|j8If_I zC7-DnU95i?X2DiVZH0*5z%_3bd?1d9yP;e%^C&GYtuT=fc|HIs0o4VmIyekzW=Re^Upziz5C5!Y@ABU(|y+fD0uz@NdMg~Z|?};eVvh8$%I5aP@ z=iY(hl9+_5Ri?#0=du>JFlGvdV>%ZwQIcT0o0}uaF(LrzGi5H;ZMpu zbav)uSOdg}%0AWd7Op3WKO%)_<0~n%3Gp~@Bhh%<8Uo?quFL$iSMKBlPforZ*<&#} z6#Q*J*24G)g%7*K3H{dcG$ta0-cZeN83gKa>HbQesW_#9yPFn2%f~aE+ac=h%42np zjfC;ws}SN};V#2vcNoPO7JyW?zoUNckRcTA&7cy4a2w`uSg4SZ_EZ9_hEMJ8E$C(0 z&+mMUw+Bv_vM?x$ax@cY?&5%O3w?Od%?xO8{xwAXKRmK$*Yaxx4Lj>(hJb}gZtygS zVKU_g!9R}phv*N&G{2T^kF}6TDH+Xa37BNX^|siQ$a?w=`f~aI5FT@J3ku*2B|QKs zE^B|nh4DPt-en61J7MbgY}j3R3lDbZ-m2qfdcp{pl24@ z@5C>S-#aR3;`jf$fLDagQnQ!c*kLI4uLN+~9)kO}PFlsqNyor5trp1;a9RJYW`6n( zG>=PS6!$_Vda>$bV#szo+=-a|wAD9%rR2^E4`j*cqM=!8HpCvjT}JxiT#C$REJuGq zvxJ7G3Mf06ZJOY5L}Pf_*$c(rW(Q6cqmbj5Ut2W33wUss+ew7L_jaMQVxr9nUjM#~ zC0!t59EC)TCuUs$9#0|4|M3I3hXlMQ>w5;r?3_K5#Hwf`nH7pGtE#`@_w|Ml1!cUv zlO9cSy6n}!RF6+U@NYJyfo>_GbFR5OH5PzudLYs52%FLM`;>yO1O>%C?uR@Ebf@uR zwzcH@7Q@$ms4+n;iDH6`yv3U&(8d>yxIA*w+E7$jx+-BXD=z-H*Zle11w?zVL>j+4 zKCW4=q&gK-5LpGS=Vbl3H#QLuJA<++~M~XQ=7w zf>bUDWuJaw+sNzXeX(A#*}DA*hWXlMnWRmAm87u(NNsOzetr7>|DZlL0r8HdJ`P;9 zg?F~nQ)C^O`|`cm>IQcM5Z8cV?5WI@SmDC8jo z##(>4bDF;~Bxc?qI--mr7) z$(Z%wS%Uf!P{QG;&Wm)E~%D#E?{3W=O8KFL{J}uD@zAfRqHO z)!gxLgN-;pQh_wDzp{SRQ#*Su#7l~>D)k|AoEJ^_h<}0FByCYm^V%>t@G|i>4wiB! ziN4LR?WckxI&0|_wX6y~X#i>Juo1DS&aG6gzYKw8Vg z5FpPo=0gI+p!8-D>zTIvZboeAhix@Dx?F45e7YUlIUm-0)YQ|73~9&G4HkLj=9WK`HV8rAuWA965fK0?O~d<*&eP<&Be zpjT%J>~ndhxh7c6TFXN7S_{Ss^@`z$I?Na+(r|UuH+})J#xQ5d{lemZ$njqPo&^FV zfd<)~c9(PdYh;eZ{&tdiD+jtJn7onM=m7ACT zSWV2CcI>4fChJ0DE*rT-*v#-~LUj-32bYbqa*6cn>bh+>m@YbQ^>T~pQo})sUs&%l z?UyvP`l$cw^}RfX068@!mP~BUcPyD&ytWer2BYHQjZ+j5;-6-Z41}GG{?4EX{h+zW%*iP3 zM$)duYBy@#N}YK%Kk= zLvqvPOFb7x2@0%$>kZDG+gHD;Ccyi7a{jD?<52rNpet;ofleF@4^j0Z!njsVJf8jIze!p%ZiQ$w1_wFP9%TV)$BMs))g)&X zi4u-w+Q0t(#qN70M_m_AmzpAZCyUiPO)%dz9p4?PV}9<9^o#_l$)|~b=&&m5i9kgn zzi6G-PqydCKRO-Uw}fFCKTDi$9l@Kp#(r@h_cbTql+l_57G1wg%~Ku}AV25D?@wa< zQS(3B9!w0ptEClrGdSIpP>bQ!*UQVeb&DtA1qOIF@((9~S#DZfrKmegWg;}eeTaTmyZ}`pApYd9Wv_;sR>cc_fckkiTrDf%MU3CDy3T% z{6K+hxD^Rup`PDoBJAr|JDl2+=K10s~9C4a@=LB!Q;DRPBF%$i7361dFA8OoYF5FxBpN^YHf{_w!D5gybp5 zL5-HFHRBiPnkW3VEy}^u9G|@pn>SXOm0~7Ypi;W|W2jzJT)i!wKTBxekznG525W4C z$8EXkx+7F00XGI2;yYrQYb`%~9dU7=$-t{pIN$^dHjGXHyd02RQnHF&gl!@JATR!a z*mj+3iaUBBb@(N1ATct0DJi6N|Cvz#HS6uB_nmqUH;`SE?*>A7Zbj_)%@m@wzI{Mp0@P-9TK~LuiTXe|1pT-I^&xAz4 zaDLwU2Gnr|&uZqr;;U`CS@;9oio51HRYF;}wa@UyRcq~jq;KkA>a?f0gI8A#K&n4t z2&4w-{duf)+dBC;dGXM_oW&uN6fa?smh#jzggXuwu|MaN>~kCbIHk8c&CLD=cx6ir zE2zRr*7~KN7#t*m_I{E7VnP*VAJ>S6@aqjT^YWd$3SfN@Pwk zNt`}A*yh(C|>c1$OdBs|dHqMS3EDkR|>Nk5_)e@OU-^g&ilGg?8AF|mA7>hd0k^O?%@W>gdCZ zjv8h|KQ))~qcy8anZBX~;|jznxz(Nf$|%N2=PieGWnhIbyd7Rw{C^Utk&ms$g#c{e|$uEyKOc8j^k!WM?fFpSn+27WA)1<}-3 znZ@%^t=hH9R+aG9jd2)G&5e*E2dPq(u*4`7#ogkCT`rOGux!8V&{odb^ctDTd}Ill zqAuh#V|8s6-1fTM&$v$($-lSw6Mpuyez-qBr#GPlL;rVxN^MTw>okSVmK24~<@ke$ zDWvyxWnS*TFCW4ta4&nAqdpJk=SZ38et29IG%vFolaOIQD^=T1ZD6&Dzx8mmC5sQY z-S_FucF(Wb!u)cp=vo?9717@^M~bK)wX<-$VMocoi$E+_wdHF;umS&ae3);O3v+}F zCaF_HhkPZo6TmP5d6oo z9s*2N?Ec(`&^N!z} zC|Dn5Xzy{!ZG6}eHo`Sk#Zc@yFp)`<@=(33B5T_DNto#+mS?^M{j@m0sw(8o%P@yu z-}njeFAux5m38uKF66BQ*ps-=$_ky$`VKJAxE|SwTBRoctMK`oQ29ZCzAFt+$Orf80kMMI*4r+ zvud>`=^yXcmQBWD@_q_Z6B`y>+~!wb`TlIAYWeR4eP$p#W^~Dgek;>8Zyp!-&!Aba zy;cRZgcb**?P>0s&Bi(QH5Xp5Z^k?8%i}p^cZ{!NhV)0oIlsWgSjr8 zYL|0G*JRX{eg-_HMh~lsZdAG_9KMiMO5W;dP>UFtwb}&|e|0q*htT8YbVLAsFnc8iB*h+^Te3VdCW5V4xiIu#VZZUlC(D&(k z*SM@i9e(~ak;Y!*QZbW+V4m9Dw4kfez%AFZs7~RzO9s1XhJ1?SGG@|!vhg7oa;jO9 zH2tmj0KIvaEbQgvt)KPzyZR)J_jP+r)do&}If7SZRZ1V_ZTBn(F1C&wY9CcB@g8|g z0*hLiYk!xqUL8f>^lo!Au)SAMZyK1M#lQXZ%{k0=iZaf3F@9)~^PsC5BmGtGn!m6> zqZjy}Il1-6Yn^2E@2?R%t;&V$As{65rFP9I`=R0zjiyBUc&oFwB))-VNAiDX!l*Sp zDil^RGnej*kv4rv)beWg82dTeQts+Ks~xjfbm33(n4!n*f2RgZ>1NFxVY5=B+E}Bk zkrS$244MTVEVm@L`j5=nv0$vMCGV*S*mg6N8h?1aKZ>auJAPvqHcdI_WQdBl#ANAd zX`3Oqh5F8OWnD1ydtP5ieP45dl`$o|125C{Zp}cdc|?D62G8W@Yo?N?unvlO?8owT z&BPfqh1i``alUD8pObNUX%4+juP@SDL!S}bee&3?@8s^}$JRKp$y}BQ!E?DBUB-U& z1`i=79Md}!@7LF!TS_LDY6Yi4t^zt|?wYOPcz%ZVsBCdQztx$5fDQJ0-KQk@hJDnz z9dmAIl#^ro8{MYGrmm9QuQFz1iv0Y=Vc9CkM=#9cMj5QFoxk+9Qio~l_oq*acYP66 zZgN3v$<;W0d`UFc5q7zdWL9;dG{C!`p;IgFbx84k#jVa+#L_0V^gvd@KxPd_iQe(Rh?xRYgJ<^d%YZH-;epI#JP`A2Z9`@_t*TUH~F$Ay){X+v`>V4e9G@8rX|K> zPfeT9G~EW4ZnbqzFZfFTP;Cg}j$A}%)@`A2Qo_))qFmC1n6KvL9L9A}#bQduy^iX2 zR-Wrvb#v8jEGNIA_iOxyMwKuzuuskO^k(WFr3BH2Yx$Q0?WwN}gX{|8y)t>6ms6h9 zlAznEC#H9)oVBdOnJdqk98SvJ(f7sp`7E4obp&OGf^4Iijq<0Rv6po13U}`{>ACkS zoseU)*u z#jFIm^4~G2GQaiZlIGGve#_VD#RA1nXzadSy--F5&hUuN#@18{H|5TpB3yVvpQT`cDu1Mla!yeygdd?lZ-w!d8Ja*W=mWhsw$|u7XT!6nD$l65Hb_yB6xNMYeD(&u zFIetI{q>KOL~z2^doC>zh9Sb}4WV1atci1G<6vSuRm`a|bNEHXG!KELSMPbHxc)FWz4NQd)l?W;H0Q;{1g?%I6-|qDAl&1R+ix?*n}570qg(@Qez4*$XkjoAgIibQ z`lVgMA-(n5hVS8*C@;#)X0v)T%iTIY~#0! zU3;2}-zFclRrz>$KAQLjEtGx1Wf>jxp3<+>9GKgq(?NHSI?~FhA#D zXw+U9i!A5P(VaHGWn(i)0d2w0^wZNsP==UddRzHphL-onjwq>Q-K`DL;j!Madg6cY-P6D}G^@b1I1v;17|ektQm89ve-vem(lr$0Y#(NCRbuiT z>HW7{x{SsKy2{OCE9>I>L;>zny2aZ+`4nfwn+-CD_VgFF+&e7Q)%3e*JukJ5$$b-K zZ>1jnoOg=W+tV!T4{-jw%w?ZnZ`7>eb{=(vDR~nY-biddz$H9HtT*DBmsylx<&pn z0`}pW-+qi|PW>N@6$b5m)%}E8>0^d`ZlJpgzxklBwfxpInQvZYk%;Z6GDqWy3HEZy z%fcqUtGP6X*P`XEg_8MFRK)%7RoLgy?99t-!h~dT6IjYEY!^^E?ip8a>uhAnNcSoh zYxGdOl}_;&1MwNLOP?`TIG)b*MxT+jG!WPRwsuT&RQ>en=-|byBC4??VQ!o2{rtx@ z9hyoPvp*bS??;;1)gND0@~Gt>@{VrAezqT2*Fl&Lq7qJ%RXK`?!;g zJ@^J;7H_A0%(p}5c%`tVrae7gb?o_P6?qVr*Y^A6EL8s1`9 zd<4q7rM2r0JNLuigT}~q%z>mq6BrU=*Lo9JSs|`0d)``UL0~6+i!X&KMG+y4+ zUCqRUcjX!;vuyPd5&nPb_rm$Z7nX7Gg&^xS|Hg5gl zEB?@v!H14-&HW~oC){BRxkVbl61UC*t6^ZUh;;h`a>!083JgIb>Mp}uuj z=zM`Nbd`O*U2X4zsYpmP)|xo((zBkWHOseSIYbNFSPzK1a`&~;9p0BVYEAkfZ#I4Q z%29tbYdC8;d8b*&PE(^#MfsM=!)D=c)Y?{`ODby6EOSKs_+{F8(k~^GXKTOvOb#{c zw2RK8#achAJN8-W*o5FTiEo?$c!oWJo`@G*ThFd^mtUO4@MO<9gUrT=CI)4OQ7I01 z3sZAW!Y8Rc_7#;hR~n*TM9?iE>S*f2>9}Z+OiIGIN_5)#!xAXGUYo!8X-r}@dsPp1 zWg)sJ)`#)l(aJ>s9bo_GJN`AoNs0-4aH>1?Zh1MvAVmsEMp*4B3*IeeSMU9V2p?BW z?=sDmVUSqN<6b6z{db+ABuk!B+b34RLS!4lbe65B1o?YYM&2WHF{$hy7&<5GcHS~q z=I(0C7?zbq82lQr!?OsGBA{8{O7_lV%!;2EpMJNfVnwCkcJ16NWMU>ulrl`0VeK}q zfU}z$LGM8wWc5+{7RL+Sa-WX9S(iS2W$ewtz`0DmzC54GDTV%xiIVC|c@sy!labO_ zs8*%=3F&{y3zJ$udkiAPGRrdRG`9Wv zZoQ%m`Tcq^_=o$$8l`|op^87V{I!{l8@-DJEoftTvuf$2 zWN`6$pjk3ZH1_bDKg4FATt+HkG{1ZBz$sQ}I(bOY;n>px%&v^>Lny=HU&KvO(&L3+@o4 z+lepEsoxsNebr0!x>K${Rtx#O z!+J=`f>u#Otc36;aF-e`e6^lsJah~?*DbWXe3v{koqX6lC@)8O>*VZxzNF&!dL8Qf zbZnL*boVw(qT4OXE>Ib#4R&}Yv6FKaiyNKAwCiJ7ASlpPM zjM(N%R^00>P%mc<;3JJO`K<9Qo%|iR=SeVblpkKlWRyN)zv2}eYov@~T88It724Ll zYMIPcQnQYYm%duk6q4&wgH4lCWyINrm5@dDjk(|NhH;l(o;g0ohwHXoc#5XnPYM2Y ztRKu#iUOCQXAA|$wsC`F+b@j2;{`t^*xJ@bYZS)cM(L{ZT^aD)W`6T|V)7fqmW0D> z47wx0E8}XadJWGnd=gIFF*?Y*QD_j3PwcG{Mbc$sEv<3ljQ-jc)O?#VKl)-FzC z%vrr{KZm&_?7u%T8<#bI_i8lx9$&@zwQds+W{+r<_iGzBQq6g(vRjjeeRKwWNcgbU z3NidGYV>D2p0*w$8c(@XoXu%(d67b(L4^;Gn0+&7__D7$av*EuO2Y)r?r? ziT>T|GT~Eq#Dhe>Ze{O`JT~d*?MSUYpFLtr(HDhID`{`SOwy_?^OQRt_5+pIjpvcQ zkLwll3${DK4*|ED!-TW1+^X6r#`iqG2Zs+hT+cJu+NBk*5M%%Xtudrg_^Mj)KKR04D z0*WzK5FO`V?R(!Id|vvNkM)~CvZ%5BzC!q<=aQlYI%9JMb=r|rOx35VkUV!9**)7 zuN6^#&MFiSxhC|m+(jCTJq_x0PT*0#<|f82W3|Zk8$BWpU#8f)NoPEJf<`O>Q|W@BEQo9Joe zxj#QGCFIqJ3S#Y)B}EUBV$X@&F55+NW+rWucJ$-?X% z5_oy}s`k@G*jJvrRA)ch{R2NJygd#H`HRp5p!g__b;u4{fSd-Tt-hs(xQ9)GthW7oSg|7Nx4MO=9`{6I)lL_={+UA!Zl&f>MSl6sKPA;HxI2#f9 zl&(q@&C`7BziQLdPggTKM@P!uK45(!zbUxnrTy#>l1ISxQi3S8S-%)3?$V z_~$2vEakgT8(_>e zKVS90nrw0eof`6X&hsXd#vPNf#P$72P4F6XLps0aXE}OVmVM`-*XnrUPM_ouOQ|8* z@r6_-MAYKR?zI~eyb2x0vW!NZ*7;)*;5cVJu!m_FWNluiPB@83<2`z3cpOAl^YTP) z{F|a%^^f^fJI%LJOvWlg)mmsiUQvdzq~w{$LDM14Gw8{HH}@!BXXF!Ni~~2 zJsMG?Q*3qV4GX3gY>%?|@(CiC(TPj}^?Ox4#Rib?!@;j;>+2n+El#M7LG9DbjAgXswW%$U26}0Y6g0Ysw{Hr ze_IG@ZTw-P|h`Nd|||r4GJKL5j=M8*1iZ8&-<>DYetAwf?J9g<{-bvyj7!;wqk? zg;M`CqeY*lX9+t5>@yu%xUK){QjX(aEKy!MWh7!@>ye2Mde28@%>A3@4&rx>-wsVA z!>+R2c(TG;DFI`%5HG2J;MYl;%m-`==6?OJK8L)V^iW)JF50}i($W@pl++P@MC2+G zli8MA`P6a;%dY;R?SADjs=WRBb^klS8Zu0fpT#XwmP;noYId>~GG;s~YCK4OHWr4w z&X_pI>V2rMMxmQgu$%|LE8qCyLwm7mc}16%;N;jeMb{ajW#6M9E$KdsifB>#v8A%k z<)JqmFt*_&%IxORgg&vAotK)q-Qw5FU!YP@qe=QX*k46kmIv+4XNkY-$(>wR7yxcD z@`h!I^{Ra_$O_C%`t~KLTsQKN_G_#pY{^HY52A~khGRW#Po(f_V#Jr zL7U?aawhKsbL;e-ty&0NP%t&G>WSfV@WYl-u ztC`Te{xyiptwkl83&hql(h4yR`TLpG1uEVuY)5m=$uT=j@^#!icdIK38kZa^%q*9k zBDny(NXuEy=ii-9LxG83s)8WNd2WL9;9?wMYSc#iqCa={$&XamsSVJ*z7TBpJv9o$ ze>vpW%BpFhQGCp*{8!*vo3ZONi*l^Q=2a*$adcL-X?+)Jo!q?UVY}j(ka9)@lm7wX zS=6h7O_H2}%;6Q5!&@aDd1hFdZ948UtqOjN^Y~ac+XQBdZQJwa_YDj2X$z9p>X)7v zWhu!h$DR0m_%JT8hPo8O?eA7VibZhg(sJx|^vhz+)Xp{1kJ+VW#3{+lpi`r!l;{)R zf+#aZD*u2GU~Q`LeQE`bJdM$cAMPJ0(#K;j6bZRfP3&6N;Rhw~*NPl`zV1bcc_X(p z4xx>*FSUY?{z4GnuD!}&Dkp?HG9{R<5!D1fA$a?akF1{ODuJy0uoBg%FWrfgw!)gi zJS;oSy{DkLF+bnsdCvwz^oMUQdhb5~W{qq)T_B)=TJrhQ}glXbu~xQWj#B&7D|r3L1zDxGj$ue_qp?)nFU znH~syv+&JAJ&KWOh{5+T^;VR%Z;j8ZPz_>-89?E%SN${3ank_alagxz=he$QmlJv4 z^45gF$9I{Z-(;HRswRZ5Yn`hnjXQ}2$luF(rwW&+0syt_@8Qz4yw^NrmgB*n=eN{v zvs&=hqk9HQ1^Nq8ik1+EN7VY?h`jpJnz_bU_UhY0zdoa>CZ8<|AKPBn5Y_Eo2exB; zGnh&=K=f7%Od1*`nFzQ(#H|L699(Ov{6Q;Yw&$=+-5`P9W&FgGpLu2AINc_3w)$gb z&dhvra@T%JMJa=3)>}(h_MQjhFEN6UpWXO$u&m*y4|8Ev*>{W6%0wjd=orbqtg z0teO~Wd0kEcVf?s+gceIIhJhN*eP7f`z_-Kf(!v5Z)`_R-mn(*L?J_=%uC#nyPU{M-^_aF4FOl>AMZn@$lLG4 zbUmN*w5r5ixOJ+-xbsFV;S&f!5(W=VpPa4Fs#t_8dLd;zUeQkOiG^0pa*PchM?{`y ziNlU-z$X8=vEa^YB42xiVAk3M_DFdndzq@Ef^DsJ>vcw2T4St&pKY8y+jRU4m5LKW@vuj-f2RcRg29BL7YQ!qjnx|{3Ds@PEMlVA(Fs1 zFwASn_#%e6L=<{t7M?INxVl4Rojgo8#Pbg0IrsiH1geAdxmb9}C!+2%1BCg9`6!Pk zAnj)m6FfWkBl9G|!1x1lN>sjBm}F&|9ZKz01Rm*oKEGzA{^_Ov?)PW4a+V2Xs^Qi`Ksbs>GP4HYo@<0a_3vkirhBgn{vdFS237_WP7r4pw$vr4DoallLk?MCTt_k^(uRnxh+rMcew3crYVf z2mk~dN%lOxo5k^vxYVF@<%9v67?V$Jv=}a@O|pZNK!IJ#*O%B=tC2E zU*;rW-b_301l=q@rxjn{5^|mWou*)j_Hpd${i~R2RD^ARGWV2po)@0msn z1f&YbZ8Y+7*Gr>wG;d4aDK{42(sLu0{Th=FOnUg|Ga=3HwKzbN>2A`a-l)qhXgtt} zdC~P9aG}J~qncWkm=V6Oq>NN4F6<8g3PgX8t7SHHvx2S#~w|$sA*D8SSIVTTZvJ~{@>nCuQSYUnQRTP`(oPM0Oz>ar^xCTT9NlDU$K z(I+WiNGKs^47exEe2lRyx^~uf4mRNgSQdhSf96)8iM|O})6z%P4~3idtfrrca!vKoyuIzTCPGVhY5FLS)y(O(>DEa?zGRPcudy5wsa5pqw( zv8YM$>VoajM;&89$`e=O@Mxxb1GQ%wU0rk`+#>Mws53-BeJXH!FzuVxmeSh&kezw#I0fDeWXn91s zt-l@%sc#=i*vNLeGe;&IE=vSzI!5PPg4-_T4-t#GRnybcb|Qs3zrFU?`!ZB>n^jNJ z&=r*|nL}T?Ft^o}O#`J?z|(ZVgQm6Q{Q3JMIzsMJ2bK@wgELS$Ce9SidUxv^qb|of}bl zO8zQC#)x>_rl-z(H%iw}RQedE^5s{=@N($b!vS*(&u~;FT+KB!!cyPrYHU`Mzm~%_ zAK1bs?G`b~xrsETZnMJk`O$vmgX^)Y}*HyFXBuDq~T$0lJdyp%T zQ80?g8T;gV7AGaY?V0=PQlO1Bn=flQ=3JhO{6&Uc^qa_30vWj~#wz;ss78hVI6jQC zoUHz`fU@(KgZ9+q0YJ)D^bYty|ZK-J#IL@AQrW?aNE573c5)4OV(T+%+ouSNy+e~QQ! zh)=mOA3Ue+ORRWG+7yV#vBhRyJocxs4*97Tj2l_5CwX)zNiq7WFX?cJZNc*E&p()! zzn7edm7-diC7H;vj^)NK@WnpHSTw}{YH;NTsd|9w^Qy&>qrehSNTmAvlVR+d)!`1%mWtfhf7PF6QCE5 zCrWw#d_?E5U(#m9Ydh*Ayum^0}=E7b(#|~`1PHi7`trGo> zJ?@JY@{w8YN3lV~MTYdKO17)9uU9TcQofgwC^4Yx-Ipc;k>p?-&#~e;qMhygq2}$P zA8&a7wAqoB^PA@oZn+4SKzaD!8tqzHe^g$h%!m3~^a5QUXa^!GCpZzQ!nZfeB#7AP z9#a~l$8&Grao##Bk||C(sROp-^yjN}K~IxYLI&Ql;PABe_BgX>u)}7 z+xnefeE{znk|z;%_^7ZHb~fA9zJHpUwn?g{6-?H-%mBDeMq!-POH3JqiLbCPl8VdD z>D#vkK-T&WiK5$z8RFDL#ecQ;bQSm!4<<-XY8NRLjvB7>&6F>bON`lBnY1a0`p#u? z$@$t{xVTLex4*0lks=HWmWe{@e!>u0l#zlZGMu6;gUwdS?IeZ;hYOP@+$0&75ZbYy z?~BH*$8ykb4U_|Xh5x8uX1;Xi^+M1mN!kZKu}gV3Qoi&C2-wDc>`2M+6TrjgdUkpa z&3TO-IUrl7XGZtR`+;FGHZeuu!)5ab0jp#5=(A5JNl#{Hj8E`9H!oi(t++kmsJZjA zm+ILPHB-^OZ+HlYEZ`q%c9zi9&c5Yf{IkwI(-LEGdVfq~@5Pc^>dr4rgU9jPSH&lr=#1*-+Bz(?9uEK^POQB!U<@LIQd#Z=Q^A>AEUC5fcJ{T3^DPJ@H)A=;NH1gtyNe|d8wgyNV9}3;|O{ots zNaWG)Vd4`m0(pa!;UXg%kmv>W*qTUhofNSoH>||gI(4KH+tWWKV%0O9(Rrwy+#mJl zMwj5e0>Nwv{0~{R&l|eUZGvxMQ)=@iml<+q9-|pu=-xmvWz<hkV8*ynKBW}V#D#QX-=ir5_$3EF=MeNBpX0QXKaaY zZ`SWyo6+jBTdn^|K+W|B{=x{OIX_i7+jc-P>hQ3QF(0PXS-i|m%7XI(3-)uXwFxMte>RE-hOTG#)=q%H!O&Lk7f|WRQteLqIw>< zH%#fOYiezAeO$PAy#g@DegaR`vbwJ#r)y?`=ErEJ*kDf(OF^;X{vSkY)9Z|zhNXYj zwKQHp96IB`-gk-^GKBM;I8*J{E$7RDL-%5CA%CjIb0j-|t3t=j_b{RqJrq{D*pOHh zpUD){~jHn?l+lEYgea4+`Y0${H3>@EMqCfK02tW<|2?dhIyXOnS1Ae11_$;vSduX;dt5 zo3i5gJr5~*=fwcqj(nS4@KW5D@R%__F@BLJL}cQUB$Y z&b@dwIY?gZW~4*8O0o2txgr0FkNSSxREbo039SF^hN z$xS>|9o)aob%W?#iH&WC>u-&S#j4DJmOC}FY85&&vt>hu0e?=m%|^B!tc5R4m-_T9 z@+k^+=be|c@IE==t?wi}Hr2M@>$96U%1EoU^Yv5CHHj+(Or(~R(M>0Ehh@6M4anTi zdzX|-0Gi{4POgN4RQ{-rvjzu+Q@jxG+Wh^~OH-$8NS5h86Z10xpPZNRif0ntr}~tV zS0a#4ShDQ1c|Sz@R~XQ-n9Q$L&wqM)Q!vJ!wc`oj$UTHPV!`8Djjd{1XqG#inQKux zBV#0GB_N`fL-nZ@i%}*B+2pzonRi9yZr3hm=895Z7D)umRAP5qDEzuucty-=hX^qq zD`hk!%5)MoG&Xdv^Es08UEVIUJmzH1zb=3|Jt|Y7D=&vNLwGz-_~j4no{Isi*MC{k zuDuH7ZH%{{qK)Uzku%=j1^0?7lGf)xEE3GNA#$GseYUvgz}8iM>{`!Un{~JRAd=k9 zIIdf(`>ZSNbKoa>$^g{IQR@o@%jVBA;LhDk?7jQJC##TuLxtIVz-7eDwj8MA*)Ut}X%dsjJ|Z!7{K z;zLxeeMEcY-f*6~*eqmI4mu6Lo6%6%M$#DahSM0DTAa*QD%F+3HWlpdM&Edtn&g>h zCL7l#c1^|EwIC55CDtJ@m|W)eyCjzzb&;Q@N(KhxtV>3c}mHrmD5-_Hr5m~%FUfI8u#kcRdyqv zS5*rVzl`~N@bQ@{J@U<%<6Kf{hydx@;>!wCqcIvzG7!&O&s33P?Abq7nEpT|?JQz) zlQH8#4_!JDeW+4Tg{IxF?P1t@578-+1Kv9^@lGIyf%V zFR1=z$amZ1TN#n|Zr@s3Lm#L1V6-#$^>;>VwgZ06tS?%ptVto_*kV!2Xr&ELY3-o1 z-UX2fE&2@m%mOef(Rnp{+xs?7t7FU+*^>E&;YV+?0LAJ0MBSdyha#fH&MeLY z_Nc9a*rH8p2dlCqdSmjWk7Hj?PNx77W1d<5!`(#3I_v!2(;e5;yVaxN>t9ZR00I#w zH}B>urjM>soi1qBA$~IgWtU=(y2?C+eqfP%S!T95ar0_5V%;$86yX2dyckjOq-TR4 zNq(Ro8j>*xvJ|hm*;=6-%5Zr78-{W$v+V)Nc*S?aksm*9Vd&0VUX6hGhfb&YPrubq zIJo&G&kufZ|2e>Tg!Q4Q6@tK9S($+oRw8*J7Ye`7hK0xxkmGE+yVBiy7^i%ZAZ9~u zXi2WxfAiVRhz4bE zO}e{k^A5|@<>mQwrWv+iww$hQqF-ip=9`JZ4X;K3R??Kf3(}4FmF(HZ`zIq%+z1yL zHxHi;Fzw9~05K(lceplX6=wKN!Vmg`F}9*e>!<=_GzV`zCx!Efc9-HTWN`Uze`xfh zC|L@13N+K^#k`BEp|#rCOgeEE@K-n&!IS}|`8$SCX`AJ>baKag+<~Y|1W1TWi4u0b z-nI9n_}V%gU%vAXCz-S7n(Gf{R+<#!`^7&~p@f0d4w|RmWu|AscYphHpXb;&nHuw3 zbp@iATQ@xbo-H$|l>)fOF?zcawL!Ad2F9y(w$?0Cg5A3bKV+iL3Xb_a$jY=oV(lA*I^@FAMmS^xl~)gTjXUg7L})-*ejC zHjdxoVHa-FQpbEr(q5D4VFmeu z5Yq&SzCSKScTimNs5^@w!|xXo&cVRMv>Y;9`N#zT6Q zQWat2kE>S4HrBlRglZfEO=fkVTA9mBg-dJhCB{O{AYu+eMa}-Gcwaphu~uyQhH|C2 z&Sr1DJEl|uO=Ae9LH&a;UR0U7UxvFpM9jmx`2U)svAq2YZW$cV4A) zjPo~P`(yhjgVeto^%GV}31g9+uWJKpKqOr#-}R0s9{aI2$cv)AJ*Qg?ofu`@H}tkn z00d(`cC$gTqQNA1-I>UrXrE()M>Aj34kFoL=QCRj4l8GF#=(>n$4JT zlB6Iqd0Vu<`|R|KNo!ak@{%*~5x6UhtkvL``;t~>=ItoHluX|{3c2$g>)sQ|mB+|E z7rFziC4q#?$IfZCFz3XNy66<%$wX`T)RI%sr_VuIQ^baPeccHfA-4Ay_G6{ocb}Y< z+Ivcu&ZhDS19jbNma0y?SRh`+enyph&Z*N9p7--Qc(d$^S7Z>u**8RVbtMCkIY#s? z+I^wZ0Va_Qg= zF%Y(iA#xc#dvz)Vi%YjWYa5kzyyWb(EE`SO5$Q^G+B@kSFD6BNCg_I{?V&H|GP#pt#guQ|fbyD1*OK#d{W$!7H*r*2ezFD4?W zi=t#yl}{sgf3y;&Dn;bsOUePPr16h~# zVhJVhrunSV;zyOX83^O$H#g51`LL!D$KCo#WwmnXM-$uq1W3*q;LKaoi9YQom-|$A zdn>})i>};fs}~s~NI1h%Eomtvo~KGW%C&Mgnf9{9LH$u^s9um57ekI&t-dOU0bxqCp_xv21AA||M|Mf6lfTcMTru@n!WUOW*X!zc?2 zJq#Z6NVm5albI=#`gYavD4y&PA6d2J74az5ur#ps^MO|XgdYQ|R>|UCTFOTkhxb1( zyE!9&tf2!Lc>|Xd-Lq1;;wJ)VBPZiCL~E-*OxQ!$do;$fHmk@nXnK1ms7(c-r+&Wc$YNkA`z$HlPv3HszaRFWn(Rx`s5fkt82kO=B*?v@&56Df?KOI> zztL2^H^22>X)B|cVp%Jkv8sAXR+VMtMylvf^b~=|lFHT`Z)WX!iYn`ew9HOA9hPO4 zm)&)ttdG9;z6|3qM&XA5tr@EbfQ;r^Yy2IHlnA_&MQ}Y|kXzCu^H+7L)LL7he zJaoK7+6uG+)1O{Igfd=Is=7co_QR3DZkpEjfkOUGxY9)A zlU0U|x^BD^g&g8qTgz3{I|RkTJVcs5IaEr27bd~odrySLePROD2v-_NvNVdYzp~zo z%=BWlM*kY$S>lPOt^Q6s^PqVD>GAo6cb`PDTIE~i$P9SFq9!)CP;ycG6%EBf^Ixb(q~>$Q7*4`@`@fNB4^W#q{ z$t|fgMuE-%uK4{Oj_ym!leS1!oIkEM7po;RmSk>EwVH>vJU?Twx|BBdENA5op+~mjiB0!z z1p-OWP=hGuh?f`PIAYk)BfH=$RUDH>RiyKXeOR{e8j+Qz62MG?j zRSh?47J6sCkJOLF?`%Cy&U8PB(D{v@xB=vxSI)w6MC}@^h}3H$SVvne>uc>B@Ybnn zh{BzM=xWlPUrz`To`yZX>)#~!-lwUgc+=@&@2A;E4@qroLdO4jDMOXW_epcLB7*pacIL30#=BeG zuWggRhmL>te^Vc;P28}YLvYI`K5VvKMM6TNhV$~CV(CL~tFMd4)+}4+jlO}%BS!B; z#w&~SC=)44ePLf`rl!enMNBJ6%Fn3|U#LqmYwgz6vKgJkGQqQp?mN3vE}Sdt*w}t z;c#ZeOxW8vPxvy8r47>7- zzl93y1-fRo;-Kg9y_$|ZbGf(AxhOVvh4kk}H2tnCt&2Ll5n_67sOARTvW=c>LJt{Z z-2&Ur!~TBQ`k^@BG1>lYg3yleKaxIy6&JxTRC?`Rz}~U~@Wa6L(6KgaQB9mauNMUA z5=h-Q#PT&hDBo1$w3OFk-zIjmHUL49iR?em%XW1zWP7#H0{x#jsex%iPtAxuU_Fen zk=D%zUi@a}<1*%ix)YY~tHwfjh0i#lW1bw27QM%Q{Xgyh{*&#xv^K@l=u`vyphsDW z?m2LRuC7R1^2e{|tJv=*sQVoisFjw1^q37$hS&<@G*sbXfwjFz9UmS{kz3*e>L~Ox zxLRNjD=Y!dXMVJ|!`F@~4;g=~4e|zxy5-1mC8Ma3=Kva2+YN_&S z5>LD#Kffof_2$a7So$gx;w0`e?EQb}g(66R?+oL`{xcPdYoxF_G6ADzUo-H?IKLW= zvTD%e^!$hvf}F`y*G*a7k7xve(+T$+*uLHW=fnUS*^T%w8we*qxQfi2C@;Fo)z@ghdSCUClID^l}2c`oEv3Xgu-v zJdK`qP(tE}4lV!&wN@%rn?=1J0f;sj_gd<>|9%@5YzFk9Q8b>dZq9NRJ4@*VL*dC+}E5dRx-0VXqNWdDxL?ldmnee)D5Zox=`(+&4+ z7T8CUL2|K1FZwAzOB2tRC`qW%5!u zN$4~b0?{1FGn@v&>(b{;@YA4R80e=qtN{B>k{+f~W(O4R@XK}3RrrnLsnsGrPMe8W zc%KP)p)wB;0{<+ep0P0z)Fp6OV_j(xI6I1fw4*n1_x$Ls9)68;P#|%IS@HC_#$kB9 zC*utmi!!G`T}v1M5gUJc?sp|EzeVBUbYpal50+F5WVu7kiaqjmD$q>Wq!ijjCQ|wg zL4-5q^v`k`UBM5w{F{Jf9jxy#xL@lIi-d`Pr8GzDRz4aX0=aYb-3&iLoDRGxqpox* z?&iMRcTim7WYgJtUEfQ81NPqg|G*XbfU!f2JCQ?n0$6w3Q-y~7PohCqI`7v7SMDKh zsGSS8o8!Yfm;ltT5GZ@xfxB72?z%_qhc19y_s<2K_aW{9>Lp0==efx;$MgUVJUd((?BCL}19xFZ7r7~*7;*!G zGQ}GYaRx>fgZCK}SOUQ)xESZ1buhPt=3O9Zw%+6lZu76}ds zDH?FgokCf|MBICeg^Lt+y&Tf7C)xDpK`hNHU?K@M+pc2w-?LgkluXzAxF#`ML$L@#^?UIMvdK+*`K3h?+fct%oW?D3juhH{m(~ ztXdM%YvCP)Aw3zc!NiB2idBQ#)9(U~4HCD_hPhBHDIDCMy>$;+!EOtKeMn3XsA3&F z5kTTR&SXy&R&$Ny<#4-F95s9>4vcTSc0xN1X!qe)aK0x)jvriN4wddHR5{G(d=xOGbVi7))`fM76&m*6=j zv_~;0D$vw~It&vAvMLl3IA7s*Wr3Yu0+JoR9uE=(3|2~`SWo(KOE(1PJ0|lgutTDr zH3v2DkV_jmPIRK~Po406nAX8%; z)Wt)AxSyn8NdNnRzHwUk+6?T%@X-3F$~e2xLU!~7w!)w#=Xq1RG--+IPUz@KrbB7??hJ3VNUPu;AXrZdBppzhQGeiSar+FoZ|d)K zZ#}>dCi|w{)ySA0l`;BIcv^y1eC03g0^$J9s-RSN3gHh6G!mx%V*vhHV@a^aUg>fK zWWRiBv&i>E{Q!F_lj%C()%Ryi#w&eQCc+cb%NItVTz@#-9*xJnFIY0E=wHL5?}n6- z%LjTmMh7?@eLSR^gLNr?dvmB(OE;L%*ErJK#AWNO#_CpyzfpoSUD6wTFnM0!KQx|m zzV_HpU64#H;OHa8*C27-to}u)LV}%r0{77f+>t(kcf*2|z7mDgT6|jtyr+y*;_e^j z8~d3Kyn^MR(88Ukutal+-)PPDNX$&o5WOjfQWU1V&KnGz$H;cKSUbc zi9?%^!f-aFiaRMxew#9M>r*QCg< zWdc?h&Q+Pc4Bk%#5PBG+d65(X;G{E8kc;L5t?SLFuu)Y2*%?$Z?%xQTjYH1J(7$CS z39pXF=^PsMPc5sa7o;+i*$U*bG!%X&f@Jk_yRUR9&Ru=!0FjwAPS>4;5a?fU-Z>#* zK=_3Jvs~b&E^6V{Q6yma4U)jKh6STze>1Ud=dvjPkFe@bK&X>EI%g+A z1}PnWFqi6!=U8^Da4IK&It~rnfP#&IUm&r2Bq&7q+a`J@9_XIf%K6|6auzA6XODL| z;JEG@G=}8m%yR8&1b{}kd8VkI@8>(P1s?C{@+4_rRCzqYyz9X0yO2HfBz~PpJNqCtg zfvL(N6A#V-&J{BMyEhU zpExqaOxXMc_hq~SDHLOAir?nQW4Sra6yM49UqA+1?U`?OFZF8noI5B*XedZ_7Wv0O>yVUvn=D+@a;Q5^N5tUe1ir>qJRM#T?Y6^@J1?hzsHZEAs8*7aar2 z6|l-ABcC}0_1eU8eYEc2hSD;$U>ug2WTcUA=nJa5;ydal*1#YN0K1>|*JqeYkrpG- z^dzv3C2|&Nshsf|Lt)OGSPN+D%i_-GEpRPH;s9VYGBwR^n@!+_)kQ2>oa9EhS4xq) z@xwmPY+eI0<8M=~Zx;gn1guPU=-l(>lEH8O`{DKl=jjwAQMDJQCGP5|>5E~F+xMmRZ-zR`@1vs4(K(BgMV31WDV%P|RDQ`it zlOBl7F2X?sD{1W!jvpvoZE^VkstVegOK+B(09O1~roQz!aWsz0@Oc?r>%xH1OI|m& z#O8iL>sbIm&;OWV`}3dob%DHW22i)@Dsl>t#*CfpF84PtGf6GF0~)CKOWfPz7|iO4 zVdwGzV+KQkTGH15BeeDJ+=0bnjgJ|(E&?eQDB1$vOpj4tP&zT$o?AT;{l&)0O7(7y(_U6bh6tD{O{;sFnO>QT2X}p zf%X-}fptBPSPL|ghNc{f=~zh~LKN3csMu&^5?=7YNa`!D+jP;ic4a}ydm51r!`a}k z@7V;Z0KMV)-s;y$ICdzzLKXJl6?91e+s-_zGH;UHv|Kjywt_Xi*Ik!5TwPe-EUOFX zh!j>~;8jhB>+`w1$}a6AYNw0m$PGN@k9F@}Ezuhr-@3|i4wmN(kzSFUnNLfwsG#1o zgh?@a+ZdEwT=TJMD)!#x@W5e+8g6?9XHB|@@Pnn@R~4y|?ir|6kis(K%v#-1hl3Iw zHKvDAAt=9I|4(EI8^D-I{)VY!LRK9a$h_jEK9M=bPQv;IexQqtNv>Gs_LpX{t?dqE zKKcVJsU!*~zaj4P7UuFC+^}v)&P4Z>SALD2gX5E?_#*?)F7X)%iSGo!*BJ};$uK@F zE4m0#UXMEaT2w@Z&vIX}dj3ZSL!iU?kARpN8CbTE`DuUo`x&v1_j`ju%c^FSC}VU` zSkMOVA9>%$#mv1G2Kn`u8azQ+wKj4D`G02`+y!DGmQkVZL3B|N#_5^t^#VWF`v|Jb z-vn;kWM5T<;ran!K$HOB_~+68|A79# zxk4X|g1%&s0%zzp-vkRBH);*e1)EOOdF`-6Mr~A(^G{p|>4m%BdIYV%^57UJn94m; zSnh9++xKgg)#OMfKKqN_m9x6xIy(IEaSSES7J&c&{=le875>7d2j@{RQjo(AhR4(14RgHGMaOi3wVof7 zfKq~QA73P$oy56pVJswI!y`C1nZicylU|&MBX&6w)cKF=6h#Y>H;w=CZiDBtVZ^WD z_X#G9W)>6A{wNc-YRL>Nugb!C|9K)k^hDajGt^ccEn0+~xYd~QThD5SHu^jYcZ@0` z;9w!cD`q(n)QCBOl(RTuBiHS#A^M2f%360q+z1~=3NF=PlxAn>{cev-hz22|%IBmQ zr=6NkFNB^lNr|0JhaFZC*MEe4l{|N+G%-D8c4NqqR(-YlX6ww9&|vs{T{PTisPXUI zaGk4Q5G4|ivXNXVj-Q8b%ZoHZYR20}eLX>v1SNLoZr~1_6j;5_T9Cc~Q5O$4dWNrk zqlLXqjlA{=h-|i%!}0J(h;cW7_)~B@G(7KQaX+!{zyAAcSld>=kkzoW%Zd#0@KW)N z7r4i9{b!hp0azKXPqvEd0w73?Z00*KE|#ZlNQE1%(^DDA6JT9>QaBlKgP-8Jw=iPw zNJv10?`8OyB&fVV<^9n)o`$-H}1XL`ybL|^E2a{d3MPoJ^Zk`Vu z9+2vI%w9&F+kE$*l?5rYJ6i-TIX}~ZNBNuBU`CI}zU18ls;cc2o)a&Gc^9^fI znDGJCe>(zh=0&i|?2x3pu~SU_>kSbRijP=Y*%r^+d~UhjQ+x5hl|Bazfy_Y*F6IG) z;+G3XKu_Bwu2x6jC3;m6+XQLXV?x&XRD>7VLACs_dW@n4%b|!YZg-zMz(MoA7P?5M zRmp+xaO^3|>q%JDd`1g#%X&^V5TTkC6SOst%7D(h|DWx^T?X~o`oY4#h_uNVt>Urv zY33FUU50Y^W)0x=CM#ey1%w^;Gh&iPl`3$P+@Cw3mqr66$)?sCDg<-lZAe*ja50v(| ziZ`kKn=OKhe>VEtOnx3cP~panNVt7pT@H4fcT!3fXm&Y@U~}jfyeP3%4AsaS>}Xxh z?x~xy3)oUdUN0=A{`NNiX9_5Y6{CgwQYl1xV3!@(`RM41#MF1#*=3m)7H@e+VR6@4hK>)0H_<%>YJ`^nRME1D*(c1D7gW8ai`FyfRTy5I zqb6*ATCHmtu&kgE(W737+^(!(!+FBzW%G;TCVr_tAXPPkDrdZwv|{w}MGB4N_Dshz zr~vUrP{a?}$;mbI%jo{Af&s;UyO+NNx)(Opt6O^UA_VKU61+2J*ZiTX{n01rz~8#< z^A`xl>Y$&43ZQpugC3?w1;|=GA;didP_qA7#t$fWPAF;4DX@Exbhv_fQ`-k>fnlPw zs12ct7mX&(NOzq`I=(3h;>=+n3Qf?$y%ShKJ|3?X^)Oy|+Az81hmKkLA`%OYqZ*R3 za%j?nzh~jymKK|K6*3M~r zzx3HUaWFpPNVCRA7T;E%@TFq7A>3)mr?fAKxaqmW08rcrgaRJ^I>>@ulEXE-+5X@X ze09;wUhj=P0imaj$@H?*Abg$K7Kt*f!W6`>`E-sbjGL%UFH9keu98}ijrl7>1&|qg zesljaDp@y<&&pBh@>UV%{iF!~Modt@2Fr3SUp@R3Byj zjfePg9Za@X?4=*1hNVtfMv%p0beGm~w=i|L6f$?U1pk8Za`Wu9rUYv`=17m&K9;-KL76xt~a?UK?{nIej-);*d*lt0xL(fMgRZ+ literal 0 HcmV?d00001 diff --git a/plugin/trino-snowflake/pom.xml b/plugin/trino-snowflake/pom.xml new file mode 100644 index 0000000000000..2a61433fe13b5 --- /dev/null +++ b/plugin/trino-snowflake/pom.xml @@ -0,0 +1,247 @@ + + + 4.0.0 + + + io.trino + trino-root + 435-SNAPSHOT + ../../pom.xml + + + trino-snowflake + trino-plugin + Trino - Snowflake Connector + + + ${project.parent.basedir} + + + + + com.google.guava + guava + + + + com.google.inject + guice + + + + io.airlift + configuration + + + + io.airlift + log + + + + io.trino + trino-base-jdbc + + + + io.trino + trino-plugin-toolkit + + + + net.snowflake + snowflake-jdbc + 3.13.32 + + + + + com.fasterxml.jackson.core + jackson-annotations + provided + + + + io.airlift + slice + provided + + + + io.opentelemetry + opentelemetry-api + provided + + + + io.opentelemetry + opentelemetry-context + provided + + + + io.trino + trino-spi + provided + + + + org.openjdk.jol + jol-core + provided + + + + + io.airlift + testing + test + + + + io.trino + trino-base-jdbc + test-jar + test + + + + io.trino + trino-main + test + + + + io.trino + trino-main + test-jar + test + + + + io.trino + trino-testing + test + + + + io.trino + trino-testing-services + test + + + + io.trino + trino-tpch + test + + + + io.trino.tpch + tpch + test + + + + org.assertj + assertj-core + test + + + + org.jetbrains + annotations + test + + + + org.junit.jupiter + junit-jupiter-api + test + + + + org.junit.jupiter + junit-jupiter-engine + test + + + + org.testcontainers + jdbc + test + + + + org.testcontainers + testcontainers + test + + + + org.testng + testng + test + + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + --add-opens=java.base/java.nio=ALL-UNNAMED + + + + + + + + default + + true + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + **/TestSnowflakeClient.java + **/TestSnowflakeConfig.java + **/TestSnowflakeConnectorTest.java + **/TestSnowflakePlugin.java + **/TestSnowflakeTypeMapping.java + + + + + + + + + + cloud-tests + + + + org.apache.maven.plugins + maven-surefire-plugin + + + **/TestSnowflakeClient.java + **/TestSnowflakeConfig.java + **/TestSnowflakeConnectorTest.java + **/TestSnowflakePlugin.java + **/TestSnowflakeTypeMapping.java + + + + + + + + diff --git a/plugin/trino-snowflake/src/main/java/io/trino/plugin/snowflake/SnowflakeClient.java b/plugin/trino-snowflake/src/main/java/io/trino/plugin/snowflake/SnowflakeClient.java new file mode 100644 index 0000000000000..56d41e6d2464a --- /dev/null +++ b/plugin/trino-snowflake/src/main/java/io/trino/plugin/snowflake/SnowflakeClient.java @@ -0,0 +1,651 @@ +/* + * 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.snowflake; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.inject.Inject; +import io.airlift.log.Logger; +import io.airlift.slice.Slices; +import io.trino.plugin.base.aggregation.AggregateFunctionRewriter; +import io.trino.plugin.base.aggregation.AggregateFunctionRule; +import io.trino.plugin.base.expression.ConnectorExpressionRewriter; +import io.trino.plugin.base.mapping.IdentifierMapping; +import io.trino.plugin.jdbc.BaseJdbcClient; +import io.trino.plugin.jdbc.BaseJdbcConfig; +import io.trino.plugin.jdbc.ColumnMapping; +import io.trino.plugin.jdbc.ConnectionFactory; +import io.trino.plugin.jdbc.JdbcColumnHandle; +import io.trino.plugin.jdbc.JdbcExpression; +import io.trino.plugin.jdbc.JdbcTableHandle; +import io.trino.plugin.jdbc.JdbcTypeHandle; +import io.trino.plugin.jdbc.LongWriteFunction; +import io.trino.plugin.jdbc.ObjectReadFunction; +import io.trino.plugin.jdbc.ObjectWriteFunction; +import io.trino.plugin.jdbc.PredicatePushdownController; +import io.trino.plugin.jdbc.QueryBuilder; +import io.trino.plugin.jdbc.SliceReadFunction; +import io.trino.plugin.jdbc.SliceWriteFunction; +import io.trino.plugin.jdbc.StandardColumnMappings; +import io.trino.plugin.jdbc.WriteMapping; +import io.trino.plugin.jdbc.aggregation.ImplementAvgDecimal; +import io.trino.plugin.jdbc.aggregation.ImplementAvgFloatingPoint; +import io.trino.plugin.jdbc.aggregation.ImplementCount; +import io.trino.plugin.jdbc.aggregation.ImplementCountAll; +import io.trino.plugin.jdbc.aggregation.ImplementMinMax; +import io.trino.plugin.jdbc.aggregation.ImplementSum; +import io.trino.plugin.jdbc.expression.JdbcConnectorExpressionRewriterBuilder; +import io.trino.plugin.jdbc.expression.ParameterizedExpression; +import io.trino.plugin.jdbc.logging.RemoteQueryModifier; +import io.trino.spi.TrinoException; +import io.trino.spi.connector.AggregateFunction; +import io.trino.spi.connector.ColumnHandle; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.type.CharType; +import io.trino.spi.type.Chars; +import io.trino.spi.type.DateTimeEncoding; +import io.trino.spi.type.DateType; +import io.trino.spi.type.DecimalType; +import io.trino.spi.type.LongTimestamp; +import io.trino.spi.type.LongTimestampWithTimeZone; +import io.trino.spi.type.StandardTypes; +import io.trino.spi.type.TimeType; +import io.trino.spi.type.TimeZoneKey; +import io.trino.spi.type.TimestampType; +import io.trino.spi.type.TimestampWithTimeZoneType; +import io.trino.spi.type.Timestamps; +import io.trino.spi.type.Type; +import io.trino.spi.type.TypeManager; +import io.trino.spi.type.TypeSignature; +import io.trino.spi.type.VarcharType; + +import java.math.RoundingMode; +import java.sql.Connection; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Time; +import java.sql.Timestamp; +import java.sql.Types; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.ZoneId; +import java.time.ZoneOffset; +import java.time.ZonedDateTime; +import java.time.format.DateTimeFormatter; +import java.util.Calendar; +import java.util.Date; +import java.util.GregorianCalendar; +import java.util.Locale; +import java.util.Map; +import java.util.Optional; +import java.util.TimeZone; +import java.util.function.BiFunction; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.util.concurrent.MoreExecutors.directExecutor; +import static io.airlift.slice.Slices.utf8Slice; +import static io.trino.plugin.base.util.JsonTypeUtil.jsonParse; +import static io.trino.plugin.jdbc.JdbcErrorCode.JDBC_ERROR; +import static io.trino.plugin.jdbc.PredicatePushdownController.DISABLE_PUSHDOWN; +import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; + +public class SnowflakeClient + extends BaseJdbcClient +{ + /* TIME supports an optional precision parameter for fractional seconds, e.g. TIME(3). Time precision can range from 0 (seconds) to 9 (nanoseconds). The default precision is 9. + All TIME values must be between 00:00:00 and 23:59:59.999999999. TIME internally stores “wallclock” time, and all operations on TIME values are performed without taking any time zone into consideration. + */ + private static final int SNOWFLAKE_MAX_SUPPORTED_TIMESTAMP_PRECISION = 9; + private static final Logger log = Logger.get(SnowflakeClient.class); + private static final DateTimeFormatter SNOWFLAKE_DATETIME_FORMATTER = DateTimeFormatter.ofPattern("y-MM-dd'T'HH:mm:ss.SSSSSSSSSXXX"); + private static final DateTimeFormatter SNOWFLAKE_DATE_FORMATTER = DateTimeFormatter.ofPattern("uuuu-MM-dd"); + private static final DateTimeFormatter SNOWFLAKE_TIMESTAMP_FORMATTER = DateTimeFormatter.ofPattern("y-MM-dd'T'HH:mm:ss.SSSSSSSSS"); + private static final DateTimeFormatter SNOWFLAKE_TIME_FORMATTER = DateTimeFormatter.ofPattern("HH:mm:ss.SSSSSSSSS"); + private final Type jsonType; + private final AggregateFunctionRewriter aggregateFunctionRewriter; + + private interface WriteMappingFunction + { + WriteMapping convert(Type type); + } + + private interface ColumnMappingFunction + { + Optional convert(JdbcTypeHandle typeHandle); + } + + private static final TimeZone UTC_TZ = TimeZone.getTimeZone(ZoneId.of("UTC")); + // Mappings for JDBC column types to internal Trino types + private static final Map STANDARD_COLUMN_MAPPINGS = ImmutableMap.builder() + .put(Types.BOOLEAN, StandardColumnMappings.booleanColumnMapping()) + .put(Types.TINYINT, StandardColumnMappings.tinyintColumnMapping()) + .put(Types.SMALLINT, StandardColumnMappings.smallintColumnMapping()) + .put(Types.INTEGER, StandardColumnMappings.integerColumnMapping()) + .put(Types.BIGINT, StandardColumnMappings.bigintColumnMapping()) + .put(Types.REAL, StandardColumnMappings.realColumnMapping()) + .put(Types.DOUBLE, StandardColumnMappings.doubleColumnMapping()) + .put(Types.FLOAT, StandardColumnMappings.doubleColumnMapping()) + .put(Types.BINARY, StandardColumnMappings.varbinaryColumnMapping()) + .put(Types.VARBINARY, StandardColumnMappings.varbinaryColumnMapping()) + .put(Types.LONGVARBINARY, StandardColumnMappings.varbinaryColumnMapping()) + .buildOrThrow(); + + private static final Map SHOWFLAKE_COLUMN_MAPPINGS = ImmutableMap.builder() + .put("time", typeHandle -> { + //return Optional.of(columnMappingPushdown(timeColumnMapping(typeHandle))); + return Optional.of(timeColumnMapping(typeHandle)); + }) + .put("timestampntz", typeHandle -> { + return Optional.of(timestampColumnMapping(typeHandle)); + }) + .put("timestamptz", typeHandle -> { + return Optional.of(timestampTZColumnMapping(typeHandle)); + }) + .put("timestampltz", typeHandle -> { + return Optional.of(timestampTZColumnMapping(typeHandle)); + }) + .put("date", typeHandle -> { + return Optional.of(ColumnMapping.longMapping( + DateType.DATE, (resultSet, columnIndex) -> + LocalDate.ofEpochDay(resultSet.getLong(columnIndex)).toEpochDay(), + snowFlakeDateWriter())); + }) + .put("object", typeHandle -> { + return Optional.of(ColumnMapping.sliceMapping( + VarcharType.createUnboundedVarcharType(), + StandardColumnMappings.varcharReadFunction(VarcharType.createUnboundedVarcharType()), + StandardColumnMappings.varcharWriteFunction(), + PredicatePushdownController.DISABLE_PUSHDOWN)); + }) + .put("array", typeHandle -> { + return Optional.of(ColumnMapping.sliceMapping( + VarcharType.createUnboundedVarcharType(), + StandardColumnMappings.varcharReadFunction(VarcharType.createUnboundedVarcharType()), + StandardColumnMappings.varcharWriteFunction(), + PredicatePushdownController.DISABLE_PUSHDOWN)); + }) + .put("variant", typeHandle -> { + return Optional.of(ColumnMapping.sliceMapping( + VarcharType.createUnboundedVarcharType(), variantReadFunction(), StandardColumnMappings.varcharWriteFunction(), + PredicatePushdownController.FULL_PUSHDOWN)); + }) + .put("varchar", typeHandle -> { + return Optional.of(varcharColumnMapping(typeHandle.getRequiredColumnSize())); + }) + .put("number", typeHandle -> { + int decimalDigits = typeHandle.getRequiredDecimalDigits(); + int precision = typeHandle.getRequiredColumnSize() + Math.max(-decimalDigits, 0); + if (precision > 38) { + return Optional.empty(); + } + return Optional.of(columnMappingPushdown( + StandardColumnMappings.decimalColumnMapping(DecimalType.createDecimalType( + precision, Math.max(decimalDigits, 0)), RoundingMode.UNNECESSARY))); + }) + .buildOrThrow(); + + // Mappings for internal Trino types to JDBC column types + private static final Map STANDARD_WRITE_MAPPINGS = ImmutableMap.builder() + .put("BooleanType", WriteMapping.booleanMapping("boolean", StandardColumnMappings.booleanWriteFunction())) + .put("BigintType", WriteMapping.longMapping("number(19)", StandardColumnMappings.bigintWriteFunction())) + .put("IntegerType", WriteMapping.longMapping("number(10)", StandardColumnMappings.integerWriteFunction())) + .put("SmallintType", WriteMapping.longMapping("number(5)", StandardColumnMappings.smallintWriteFunction())) + .put("TinyintType", WriteMapping.longMapping("number(3)", StandardColumnMappings.tinyintWriteFunction())) + .put("DoubleType", WriteMapping.doubleMapping("double precision", StandardColumnMappings.doubleWriteFunction())) + .put("RealType", WriteMapping.longMapping("real", StandardColumnMappings.realWriteFunction())) + .put("VarbinaryType", WriteMapping.sliceMapping("varbinary", StandardColumnMappings.varbinaryWriteFunction())) + .put("DateType", WriteMapping.longMapping("date", snowFlakeDateWriter())) + .buildOrThrow(); + + private static final Map SNOWFLAKE_WRITE_MAPPINGS = ImmutableMap.builder() + .put("TimeType", type -> { + return WriteMapping.longMapping("time", SnowflakeClient.snowFlaketimeWriter(type)); + }) + .put("ShortTimestampType", type -> { + WriteMapping myMap = SnowflakeClient.snowFlakeTimestampWriter(type); + return myMap; + }) + .put("ShortTimestampWithTimeZoneType", type -> { + WriteMapping myMap = SnowflakeClient.snowFlakeTimestampWithTZWriter(type); + return myMap; + }) + .put("LongTimestampType", type -> { + WriteMapping myMap = SnowflakeClient.snowFlakeTimestampWithTZWriter(type); + return myMap; + }) + .put("LongTimestampWithTimeZoneType", type -> { + WriteMapping myMap = SnowflakeClient.snowFlakeTimestampWithTZWriter(type); + return myMap; + }) + .put("VarcharType", type -> { + WriteMapping myMap = SnowflakeClient.snowFlakeVarCharWriter(type); + return myMap; + }) + .put("CharType", type -> { + WriteMapping myMap = SnowflakeClient.snowFlakeCharWriter(type); + return myMap; + }) + .put("LongDecimalType", type -> { + WriteMapping myMap = SnowflakeClient.snowFlakeDecimalWriter(type); + return myMap; + }) + .put("ShortDecimalType", type -> { + WriteMapping myMap = SnowflakeClient.snowFlakeDecimalWriter(type); + return myMap; + }) + .buildOrThrow(); + + @Inject + public SnowflakeClient(BaseJdbcConfig config, ConnectionFactory connectionFactory, QueryBuilder queryBuilder, + TypeManager typeManager, IdentifierMapping identifierMapping, + RemoteQueryModifier remoteQueryModifier) + { + super("\"", connectionFactory, queryBuilder, config.getJdbcTypesMappedToVarchar(), identifierMapping, remoteQueryModifier, false); + this.jsonType = typeManager.getType(new TypeSignature(StandardTypes.JSON)); + + JdbcTypeHandle bigintTypeHandle = new JdbcTypeHandle(Types.BIGINT, Optional.of("bigint"), Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty()); + ConnectorExpressionRewriter connectorExpressionRewriter = JdbcConnectorExpressionRewriterBuilder.newBuilder() + .addStandardRules(this::quoted) + .build(); + + this.aggregateFunctionRewriter = new AggregateFunctionRewriter<>( + connectorExpressionRewriter, + ImmutableSet.>builder() + .add(new ImplementCountAll(bigintTypeHandle)) + .add(new ImplementCount(bigintTypeHandle)) + .add(new ImplementMinMax(false)) + .add(new ImplementSum(SnowflakeClient::toTypeHandle)) + .add(new ImplementAvgFloatingPoint()) + .add(new ImplementAvgDecimal()) + .build()); + } + + @Override + public void abortReadConnection(Connection connection, ResultSet resultSet) + throws SQLException + { + // Abort connection before closing. Without this, the Snowflake driver + // attempts to drain the connection by reading all the results. + connection.abort(directExecutor()); + } + + @Override + public Optional toColumnMapping(ConnectorSession session, Connection connection, JdbcTypeHandle typeHandle) + { + String jdbcTypeName = typeHandle.getJdbcTypeName() + .orElseThrow(() -> new TrinoException(JDBC_ERROR, "Type name is missing: " + typeHandle)); + jdbcTypeName = jdbcTypeName.toLowerCase(Locale.ENGLISH); + int type = typeHandle.getJdbcType(); + + ColumnMapping columnMap = STANDARD_COLUMN_MAPPINGS.get(type); + if (columnMap != null) { + return Optional.of(columnMap); + } + + ColumnMappingFunction columnMappingFunction = SHOWFLAKE_COLUMN_MAPPINGS.get(jdbcTypeName); + if (columnMappingFunction != null) { + return columnMappingFunction.convert(typeHandle); + } + + // Code should never reach here so throw an error. + throw new TrinoException(NOT_SUPPORTED, "SNOWFLAKE_CONNECTOR_COLUMN_TYPE_NOT_SUPPORTED: Unsupported column type(" + type + + "):" + jdbcTypeName); + } + + @Override + public WriteMapping toWriteMapping(ConnectorSession session, Type type) + { + Class myClass = type.getClass(); + String simple = myClass.getSimpleName(); + + WriteMapping writeMapping = STANDARD_WRITE_MAPPINGS.get(simple); + if (writeMapping != null) { + return writeMapping; + } + + WriteMappingFunction writeMappingFunction = SNOWFLAKE_WRITE_MAPPINGS.get(simple); + if (writeMappingFunction != null) { + return writeMappingFunction.convert(type); + } + + log.debug("SnowflakeClient.toWriteMapping: SNOWFLAKE_CONNECTOR_COLUMN_TYPE_NOT_SUPPORTED: Unsupported column type: " + type.getDisplayName() + ", simple:" + simple); + + throw new TrinoException(NOT_SUPPORTED, "SNOWFLAKE_CONNECTOR_COLUMN_TYPE_NOT_SUPPORTED: Unsupported column type: " + type.getDisplayName() + ", simple:" + simple); + } + + @Override + public Optional implementAggregation(ConnectorSession session, AggregateFunction aggregate, Map assignments) + { + // TODO support complex ConnectorExpressions + return aggregateFunctionRewriter.rewrite(session, aggregate, assignments); + } + + private static Optional toTypeHandle(DecimalType decimalType) + { + return Optional.of(new JdbcTypeHandle(Types.NUMERIC, Optional.of("decimal"), Optional.of(decimalType.getPrecision()), Optional.of(decimalType.getScale()), Optional.empty(), Optional.empty())); + } + + @Override + protected Optional> limitFunction() + { + return Optional.of((sql, limit) -> sql + " LIMIT " + limit); + } + + @Override + public boolean isLimitGuaranteed(ConnectorSession session) + { + return true; + } + + private ColumnMapping jsonColumnMapping() + { + return ColumnMapping.sliceMapping( + jsonType, + (resultSet, columnIndex) -> jsonParse(utf8Slice(resultSet.getString(columnIndex))), + StandardColumnMappings.varcharWriteFunction(), + DISABLE_PUSHDOWN); + } + + @Override + public void setColumnType(ConnectorSession session, JdbcTableHandle handle, JdbcColumnHandle column, Type type) + { + throw new TrinoException(NOT_SUPPORTED, "This connector does not support setting column types"); + } + + private static SliceReadFunction variantReadFunction() + { + return (resultSet, columnIndex) -> Slices.utf8Slice(resultSet.getString(columnIndex).replaceAll("^\"|\"$", "")); + } + + private static ColumnMapping columnMappingPushdown(ColumnMapping mapping) + { + if (mapping.getPredicatePushdownController() == PredicatePushdownController.DISABLE_PUSHDOWN) { + log.debug("SnowflakeClient.columnMappingPushdown: NOT_SUPPORTED mapping.getPredicatePushdownController() is DISABLE_PUSHDOWN. Type was " + mapping.getType()); + throw new TrinoException(NOT_SUPPORTED, "mapping.getPredicatePushdownController() is DISABLE_PUSHDOWN. Type was " + mapping.getType()); + } + + return new ColumnMapping(mapping.getType(), mapping.getReadFunction(), mapping.getWriteFunction(), + PredicatePushdownController.FULL_PUSHDOWN); + } + + private static ColumnMapping timeColumnMapping(JdbcTypeHandle typeHandle) + { + int precision = typeHandle.getRequiredDecimalDigits(); + checkArgument((precision <= SNOWFLAKE_MAX_SUPPORTED_TIMESTAMP_PRECISION), + "The max timestamp precision in Snowflake is " + SNOWFLAKE_MAX_SUPPORTED_TIMESTAMP_PRECISION); + return ColumnMapping.longMapping( + TimeType.createTimeType(precision), + (resultSet, columnIndex) -> { + LocalTime time = SNOWFLAKE_TIME_FORMATTER.parse(resultSet.getString(columnIndex), LocalTime::from); + long nanosOfDay = time.toNanoOfDay(); + long picosOfDay = nanosOfDay * Timestamps.PICOSECONDS_PER_NANOSECOND; + return Timestamps.round(picosOfDay, 12 - precision); + }, + timeWriteFunction(precision), + PredicatePushdownController.FULL_PUSHDOWN); + } + + private static LongWriteFunction snowFlaketimeWriter(Type type) + { + TimeType timeType = (TimeType) type; + int precision = timeType.getPrecision(); + return timeWriteFunction(precision); + } + + private static LongWriteFunction timeWriteFunction(int precision) + { + checkArgument(precision <= SNOWFLAKE_MAX_SUPPORTED_TIMESTAMP_PRECISION, "Unsupported precision: %s", precision); + String bindExpression = String.format("CAST(? AS time(%s))", precision); + return new LongWriteFunction() + { + @Override + public String getBindExpression() + { + return bindExpression; + } + + @Override + public void set(PreparedStatement statement, int index, long picosOfDay) + throws SQLException + { + picosOfDay = Timestamps.round(picosOfDay, 12 - precision); + if (picosOfDay == Timestamps.PICOSECONDS_PER_DAY) { + picosOfDay = 0; + } + LocalTime localTime = LocalTime.ofNanoOfDay(picosOfDay / Timestamps.PICOSECONDS_PER_NANOSECOND); + // statement.setObject(.., localTime) would yield incorrect end result for 23:59:59.999000 + statement.setString(index, SNOWFLAKE_TIME_FORMATTER.format(localTime)); + } + }; + } + + private static long toTrinoTime(Time sqlTime) + { + return Timestamps.PICOSECONDS_PER_SECOND * sqlTime.getTime(); + } + + private static ColumnMapping timestampTZColumnMapping(JdbcTypeHandle typeHandle) + { + int precision = typeHandle.getRequiredDecimalDigits(); + String jdbcTypeName = typeHandle.getJdbcTypeName() + .orElseThrow(() -> new TrinoException(JDBC_ERROR, "Type name is missing: " + typeHandle)); + int type = typeHandle.getJdbcType(); + log.debug("timestampTZColumnMapping: jdbcTypeName(%s):%s precision:%s", type, jdbcTypeName, precision); + + if (precision <= 3) { + return ColumnMapping.longMapping(TimestampWithTimeZoneType.createTimestampWithTimeZoneType(precision), + (resultSet, columnIndex) -> { + ZonedDateTime timestamp = (ZonedDateTime) SNOWFLAKE_DATETIME_FORMATTER.parse(resultSet.getString(columnIndex), ZonedDateTime::from); + return DateTimeEncoding.packDateTimeWithZone(timestamp.toInstant().toEpochMilli(), timestamp.getZone().getId()); + }, + timestampWithTZWriter(), PredicatePushdownController.FULL_PUSHDOWN); + } + else { + return ColumnMapping.objectMapping(TimestampWithTimeZoneType.createTimestampWithTimeZoneType(precision), longTimestampWithTimezoneReadFunction(), longTimestampWithTZWriteFunction()); + } + } + + private static ColumnMapping varcharColumnMapping(int varcharLength) + { + VarcharType varcharType = varcharLength <= VarcharType.MAX_LENGTH + ? VarcharType.createVarcharType(varcharLength) + : VarcharType.createUnboundedVarcharType(); + return ColumnMapping.sliceMapping( + varcharType, + StandardColumnMappings.varcharReadFunction(varcharType), + StandardColumnMappings.varcharWriteFunction()); + } + + private static ObjectReadFunction longTimestampWithTimezoneReadFunction() + { + return ObjectReadFunction.of(LongTimestampWithTimeZone.class, (resultSet, columnIndex) -> { + ZonedDateTime timestamp = (ZonedDateTime) SNOWFLAKE_DATETIME_FORMATTER.parse(resultSet.getString(columnIndex), ZonedDateTime::from); + return LongTimestampWithTimeZone.fromEpochSecondsAndFraction(timestamp.toEpochSecond(), + (long) timestamp.getNano() * Timestamps.PICOSECONDS_PER_NANOSECOND, + TimeZoneKey.getTimeZoneKey(timestamp.getZone().getId())); + }); + } + + private static ObjectWriteFunction longTimestampWithTZWriteFunction() + { + return ObjectWriteFunction.of(LongTimestampWithTimeZone.class, (statement, index, value) -> { + long epoMilli = value.getEpochMillis(); + long epoSeconds = Math.floorDiv(epoMilli, Timestamps.MILLISECONDS_PER_SECOND); + long adjNano = Math.floorMod(epoMilli, Timestamps.MILLISECONDS_PER_SECOND) * + Timestamps.NANOSECONDS_PER_MILLISECOND + value.getPicosOfMilli() / Timestamps.PICOSECONDS_PER_NANOSECOND; + ZoneId zone = TimeZoneKey.getTimeZoneKey(value.getTimeZoneKey()).getZoneId(); + Instant timeI = Instant.ofEpochSecond(epoSeconds, adjNano); + statement.setString(index, SNOWFLAKE_DATETIME_FORMATTER.format(ZonedDateTime.ofInstant(timeI, zone))); + }); + } + + private static LongWriteFunction snowFlakeDateTimeWriter() + { + return (statement, index, encodedTimeWithZone) -> { + Instant time = Instant.ofEpochMilli(DateTimeEncoding.unpackMillisUtc(encodedTimeWithZone)); + ZoneId zone = ZoneId.of(DateTimeEncoding.unpackZoneKey(encodedTimeWithZone).getId()); + statement.setString(index, SNOWFLAKE_DATETIME_FORMATTER.format(time.atZone(zone))); + }; + } + + private static WriteMapping snowFlakeDecimalWriter(Type type) + { + DecimalType decimalType = (DecimalType) type; + String dataType = String.format("decimal(%s, %s)", new Object[] { + Integer.valueOf(decimalType.getPrecision()), Integer.valueOf(decimalType.getScale()) + }); + + if (decimalType.isShort()) { + return WriteMapping.longMapping(dataType, StandardColumnMappings.shortDecimalWriteFunction(decimalType)); + } + return WriteMapping.objectMapping(dataType, StandardColumnMappings.longDecimalWriteFunction(decimalType)); + } + + private static LongWriteFunction snowFlakeDateWriter() + { + return (statement, index, day) -> statement.setString(index, SNOWFLAKE_DATE_FORMATTER.format(LocalDate.ofEpochDay(day))); + } + + private static WriteMapping snowFlakeCharWriter(Type type) + { + CharType charType = (CharType) type; + return WriteMapping.sliceMapping("char(" + charType.getLength() + ")", + charWriteFunction(charType)); + } + + private static WriteMapping snowFlakeVarCharWriter(Type type) + { + String dataType; + VarcharType varcharType = (VarcharType) type; + + if (varcharType.isUnbounded()) { + dataType = "varchar"; + } + else { + dataType = "varchar(" + varcharType.getBoundedLength() + ")"; + } + return WriteMapping.sliceMapping(dataType, StandardColumnMappings.varcharWriteFunction()); + } + + private static SliceWriteFunction charWriteFunction(CharType charType) + { + return (statement, index, value) -> statement.setString(index, Chars.padSpaces(value, charType).toStringUtf8()); + } + + private static WriteMapping snowFlakeTimestampWriter(Type type) + { + TimestampType timestampType = (TimestampType) type; + checkArgument((timestampType.getPrecision() <= SNOWFLAKE_MAX_SUPPORTED_TIMESTAMP_PRECISION), + "The max timestamp precision in Snowflake is " + SNOWFLAKE_MAX_SUPPORTED_TIMESTAMP_PRECISION); + + if (timestampType.isShort()) { + return WriteMapping.longMapping( + String.format("timestamp_ntz(%d)", new Object[] {Integer.valueOf(timestampType.getPrecision()) }), + timestampWriteFunction()); + } + return WriteMapping.objectMapping( + String.format("timestamp_ntz(%d)", new Object[] {Integer.valueOf(timestampType.getPrecision()) }), + longTimestampWriter(timestampType.getPrecision())); + } + + private static LongWriteFunction timestampWriteFunction() + { + return (statement, index, value) -> statement.setString(index, + StandardColumnMappings.fromTrinoTimestamp(value).toString()); + } + + private static ObjectWriteFunction longTimestampWriter(int precision) + { + return ObjectWriteFunction.of(LongTimestamp.class, + (statement, index, value) -> statement.setString(index, + SNOWFLAKE_TIMESTAMP_FORMATTER.format(StandardColumnMappings.fromLongTrinoTimestamp(value, + precision)))); + } + + private static WriteMapping snowFlakeTimestampWithTZWriter(Type type) + { + TimestampWithTimeZoneType timeTZType = (TimestampWithTimeZoneType) type; + + checkArgument((timeTZType.getPrecision() <= SNOWFLAKE_MAX_SUPPORTED_TIMESTAMP_PRECISION), + "Max Snowflake precision is is " + SNOWFLAKE_MAX_SUPPORTED_TIMESTAMP_PRECISION); + if (timeTZType.isShort()) { + return WriteMapping.longMapping(String.format("timestamp_tz(%d)", + new Object[] {Integer.valueOf(timeTZType.getPrecision()) }), + timestampWithTZWriter()); + } + return WriteMapping.objectMapping( + String.format("timestamp_tz(%d)", new Object[] {Integer.valueOf(timeTZType.getPrecision()) }), + longTimestampWithTZWriteFunction()); + } + + private static LongWriteFunction timestampWithTZWriter() + { + return (statement, index, encodedTimeWithZone) -> { + Instant timeI = Instant.ofEpochMilli(DateTimeEncoding.unpackMillisUtc(encodedTimeWithZone)); + ZoneId zone = ZoneId.of(DateTimeEncoding.unpackZoneKey(encodedTimeWithZone).getId()); + statement.setString(index, SNOWFLAKE_DATETIME_FORMATTER.format(timeI.atZone(zone))); + }; + } + + private static ObjectReadFunction longTimestampReader() + { + return ObjectReadFunction.of(LongTimestamp.class, (resultSet, columnIndex) -> { + Calendar calendar = new GregorianCalendar(UTC_TZ, Locale.ENGLISH); + calendar.setTime(new Date(0)); + Timestamp ts = resultSet.getTimestamp(columnIndex, calendar); + long epochMillis = ts.getTime(); + int nanosInTheSecond = ts.getNanos(); + int nanosInTheMilli = nanosInTheSecond % Timestamps.NANOSECONDS_PER_MILLISECOND; + long micro = epochMillis * Timestamps.MICROSECONDS_PER_MILLISECOND + (nanosInTheMilli / Timestamps.NANOSECONDS_PER_MICROSECOND); + int picosOfMicro = nanosInTheMilli % 1000 * 1000; + return new LongTimestamp(micro, picosOfMicro); + }); + } + + private static ColumnMapping timestampColumnMapping(JdbcTypeHandle typeHandle) + { + int precision = typeHandle.getRequiredDecimalDigits(); + String jdbcTypeName = typeHandle.getJdbcTypeName() + .orElseThrow(() -> new TrinoException(JDBC_ERROR, "Type name is missing: " + typeHandle)); + int type = typeHandle.getJdbcType(); + log.debug("timestampColumnMapping: jdbcTypeName(%s):%s precision:%s", type, jdbcTypeName, precision); + + // <= 6 fits into a long + if (precision <= 6) { + return ColumnMapping.longMapping( + (Type) TimestampType.createTimestampType(precision), (resultSet, columnIndex) -> + StandardColumnMappings.toTrinoTimestamp(TimestampType.createTimestampType(precision), + toLocalDateTime(resultSet, columnIndex)), + timestampWriteFunction()); + } + + // Too big. Put it in an object + return ColumnMapping.objectMapping( + (Type) TimestampType.createTimestampType(precision), + longTimestampReader(), + longTimestampWriter(precision)); + } + + private static LocalDateTime toLocalDateTime(ResultSet resultSet, int columnIndex) + throws SQLException + { + Calendar calendar = new GregorianCalendar(UTC_TZ, Locale.ENGLISH); + calendar.setTime(new Date(0)); + Timestamp ts = resultSet.getTimestamp(columnIndex, calendar); + return LocalDateTime.ofInstant(Instant.ofEpochMilli(ts.getTime()), ZoneOffset.UTC); + } +} diff --git a/plugin/trino-snowflake/src/main/java/io/trino/plugin/snowflake/SnowflakeClientModule.java b/plugin/trino-snowflake/src/main/java/io/trino/plugin/snowflake/SnowflakeClientModule.java new file mode 100644 index 0000000000000..19fc358471915 --- /dev/null +++ b/plugin/trino-snowflake/src/main/java/io/trino/plugin/snowflake/SnowflakeClientModule.java @@ -0,0 +1,95 @@ +/* + * 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.snowflake; + +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.google.inject.Scopes; +import com.google.inject.Singleton; +import io.trino.plugin.jdbc.BaseJdbcConfig; +import io.trino.plugin.jdbc.ConnectionFactory; +import io.trino.plugin.jdbc.DriverConnectionFactory; +import io.trino.plugin.jdbc.ForBaseJdbc; +import io.trino.plugin.jdbc.JdbcClient; +import io.trino.plugin.jdbc.TypeHandlingJdbcConfig; +import io.trino.plugin.jdbc.credential.CredentialProvider; +import io.trino.spi.TrinoException; +import net.snowflake.client.jdbc.SnowflakeDriver; + +import java.net.MalformedURLException; +import java.net.URL; +import java.util.Properties; + +import static io.airlift.configuration.ConfigBinder.configBinder; +import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; + +public class SnowflakeClientModule + implements Module +{ + @Override + public void configure(Binder binder) + { + binder.bind(JdbcClient.class).annotatedWith(ForBaseJdbc.class).to(SnowflakeClient.class).in(Scopes.SINGLETON); + configBinder(binder).bindConfig(SnowflakeConfig.class); + configBinder(binder).bindConfig(TypeHandlingJdbcConfig.class); + } + + @Singleton + @Provides + @ForBaseJdbc + public ConnectionFactory getConnectionFactory(BaseJdbcConfig baseJdbcConfig, SnowflakeConfig snowflakeConfig, CredentialProvider credentialProvider) + throws MalformedURLException + { + Properties properties = new Properties(); + snowflakeConfig.getAccount().ifPresent(account -> properties.setProperty("account", account)); + snowflakeConfig.getDatabase().ifPresent(database -> properties.setProperty("db", database)); + snowflakeConfig.getRole().ifPresent(role -> properties.setProperty("role", role)); + snowflakeConfig.getWarehouse().ifPresent(warehouse -> properties.setProperty("warehouse", warehouse)); + + // Set the expected date/time formatting we expect for our plugin to parse + properties.setProperty("TIMESTAMP_OUTPUT_FORMAT", "YYYY-MM-DD\"T\"HH24:MI:SS.FF9TZH:TZM"); + properties.setProperty("TIMESTAMP_NTZ_OUTPUT_FORMAT", "YYYY-MM-DD\"T\"HH24:MI:SS.FF9TZH:TZM"); + properties.setProperty("TIMESTAMP_TZ_OUTPUT_FORMAT", "YYYY-MM-DD\"T\"HH24:MI:SS.FF9TZH:TZM"); + properties.setProperty("TIMESTAMP_LTZ_OUTPUT_FORMAT", "YYYY-MM-DD\"T\"HH24:MI:SS.FF9TZH:TZM"); + properties.setProperty("TIME_OUTPUT_FORMAT", "HH24:MI:SS.FF9"); + snowflakeConfig.getTimestampNoTimezoneAsUTC().ifPresent(as_utc -> properties.setProperty("JDBC_TREAT_TIMESTAMP_NTZ_AS_UTC", as_utc ? "true" : "false")); + + // Support for Corporate proxies + if (snowflakeConfig.getHTTPProxy().isPresent()) { + String proxy = snowflakeConfig.getHTTPProxy().get(); + + URL url = new URL(proxy); + + properties.setProperty("useProxy", "true"); + properties.setProperty("proxyHost", url.getHost()); + properties.setProperty("proxyPort", Integer.toString(url.getPort())); + properties.setProperty("proxyProtocol", url.getProtocol()); + + String userInfo = url.getUserInfo(); + if (userInfo != null) { + String[] usernamePassword = userInfo.split(":", 2); + + if (usernamePassword.length != 2) { + throw new TrinoException(NOT_SUPPORTED, "Improper snowflake.http_proxy. username:password@ is optional but what was entered was not correct"); + } + + properties.setProperty("proxyUser", usernamePassword[0]); + properties.setProperty("proxyPassword", usernamePassword[1]); + } + } + + return new DriverConnectionFactory(new SnowflakeDriver(), baseJdbcConfig.getConnectionUrl(), properties, credentialProvider); + } +} diff --git a/plugin/trino-snowflake/src/main/java/io/trino/plugin/snowflake/SnowflakeConfig.java b/plugin/trino-snowflake/src/main/java/io/trino/plugin/snowflake/SnowflakeConfig.java new file mode 100644 index 0000000000000..6dbf125201774 --- /dev/null +++ b/plugin/trino-snowflake/src/main/java/io/trino/plugin/snowflake/SnowflakeConfig.java @@ -0,0 +1,100 @@ +/* + * 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.snowflake; + +import io.airlift.configuration.Config; + +import java.util.Optional; + +public class SnowflakeConfig +{ + private String account; + private String database; + private String role; + private String warehouse; + private Boolean timestampNoTimezoneAsUTC; + private String httpProxy; + + public Optional getAccount() + { + return Optional.ofNullable(account); + } + + @Config("snowflake.account") + public SnowflakeConfig setAccount(String account) + { + this.account = account; + return this; + } + + public Optional getDatabase() + { + return Optional.ofNullable(database); + } + + @Config("snowflake.database") + public SnowflakeConfig setDatabase(String database) + { + this.database = database; + return this; + } + + public Optional getRole() + { + return Optional.ofNullable(role); + } + + @Config("snowflake.role") + public SnowflakeConfig setRole(String role) + { + this.role = role; + return this; + } + + public Optional getWarehouse() + { + return Optional.ofNullable(warehouse); + } + + @Config("snowflake.warehouse") + public SnowflakeConfig setWarehouse(String warehouse) + { + this.warehouse = warehouse; + return this; + } + + public Optional getTimestampNoTimezoneAsUTC() + { + return Optional.ofNullable(timestampNoTimezoneAsUTC); + } + + @Config("snowflake.timestamp-no-timezone-as-utc") + public SnowflakeConfig setTimestampNoTimezoneAsUTC(Boolean timestampNoTimezoneAsUTC) + { + this.timestampNoTimezoneAsUTC = timestampNoTimezoneAsUTC; + return this; + } + + public Optional getHTTPProxy() + { + return Optional.ofNullable(httpProxy); + } + + @Config("snowflake.http-proxy") + public SnowflakeConfig setHTTPProxy(String httpProxy) + { + this.httpProxy = httpProxy; + return this; + } +} diff --git a/plugin/trino-snowflake/src/main/java/io/trino/plugin/snowflake/SnowflakePlugin.java b/plugin/trino-snowflake/src/main/java/io/trino/plugin/snowflake/SnowflakePlugin.java new file mode 100644 index 0000000000000..728264d297784 --- /dev/null +++ b/plugin/trino-snowflake/src/main/java/io/trino/plugin/snowflake/SnowflakePlugin.java @@ -0,0 +1,25 @@ +/* + * 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.snowflake; + +import io.trino.plugin.jdbc.JdbcPlugin; + +public class SnowflakePlugin + extends JdbcPlugin +{ + public SnowflakePlugin() + { + super("snowflake", new SnowflakeClientModule()); + } +} diff --git a/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/BaseSnowflakeConnectorTest.java b/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/BaseSnowflakeConnectorTest.java new file mode 100644 index 0000000000000..b547624cebeac --- /dev/null +++ b/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/BaseSnowflakeConnectorTest.java @@ -0,0 +1,610 @@ +/* + * 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.snowflake; + +import io.trino.Session; +import io.trino.plugin.jdbc.BaseJdbcConnectorTest; +import io.trino.testing.MaterializedResult; +import io.trino.testing.TestingConnectorBehavior; +import io.trino.testing.sql.TestTable; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.parallel.Execution; + +import java.util.Optional; +import java.util.OptionalInt; + +import static com.google.common.base.Strings.nullToEmpty; +import static io.trino.plugin.snowflake.TestingSnowflakeServer.TEST_SCHEMA; +import static io.trino.spi.connector.ConnectorMetadata.MODIFYING_ROWS_MESSAGE; +import static io.trino.spi.type.VarcharType.VARCHAR; +import static io.trino.testing.MaterializedResult.resultBuilder; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_TABLE; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_TABLE_WITH_DATA; +import static io.trino.testing.TestingNames.randomNameSuffix; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assumptions.abort; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; + +@TestInstance(PER_CLASS) +@Execution(CONCURRENT) +public abstract class BaseSnowflakeConnectorTest + extends BaseJdbcConnectorTest +{ + protected TestingSnowflakeServer server; + + @Override + protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior) + { + switch (connectorBehavior) { + case SUPPORTS_AGGREGATION_PUSHDOWN: + case SUPPORTS_TOPN_PUSHDOWN: + case SUPPORTS_LIMIT_PUSHDOWN: + return false; + case SUPPORTS_COMMENT_ON_COLUMN: + case SUPPORTS_ADD_COLUMN_WITH_COMMENT: + case SUPPORTS_COMMENT_ON_TABLE: + case SUPPORTS_CREATE_TABLE_WITH_TABLE_COMMENT: + case SUPPORTS_CREATE_TABLE_WITH_COLUMN_COMMENT: + case SUPPORTS_SET_COLUMN_TYPE: + return false; + case SUPPORTS_DROP_FIELD: + case SUPPORTS_ROW_TYPE: + case SUPPORTS_ARRAY: + return false; + default: + return super.hasBehavior(connectorBehavior); + } + } + + @Override + protected TestTable createTableWithDefaultColumns() + { + return new TestTable( + onRemoteDatabase(), + TEST_SCHEMA, + "(col_required BIGINT NOT NULL," + + "col_nullable BIGINT," + + "col_default BIGINT DEFAULT 43," + + "col_nonnull_default BIGINT NOT NULL DEFAULT 42," + + "col_required2 BIGINT NOT NULL)"); + } + + @Override + protected TestTable createTableWithUnsupportedColumn() + { + return new TestTable( + onRemoteDatabase(), + TEST_SCHEMA, + "(one bigint, two decimal(38,0), three varchar(10))"); + } + + @Override + protected Optional filterDataMappingSmokeTestData(DataMappingTestSetup dataMappingTestSetup) + { + String typeName = dataMappingTestSetup.getTrinoTypeName(); + // TODO: Test fails with these types + // Error: No result for query: SELECT row_id FROM test_data_mapping_smoke_real_3u8xo6hp59 WHERE rand() = 42 OR value = REAL '567.123' + // In the testDataMappingSmokeTestDataProvider(), the type sampleValueLiteral of type real should be "DOUBLE" rather than "REAL". + if (typeName.equals("real")) { + return Optional.empty(); + } + // Error: Failed to insert data: SQL compilation error: error line 1 at position 130 + if (typeName.equals("time") + || typeName.equals("time(6)") + || typeName.equals("timestamp(6)")) { + return Optional.empty(); + } + // Error: not equal + if (typeName.equals("char(3)")) { + return Optional.empty(); + } + return Optional.of(dataMappingTestSetup); + } + + @Override + protected boolean isColumnNameRejected(Exception exception, String columnName, boolean delimited) + { + return nullToEmpty(exception.getMessage()).matches(".*(Incorrect column name).*"); + } + + @Override + protected MaterializedResult getDescribeOrdersResult() + { + // Override this test because the type of row "shippriority" should be bigint rather than integer for snowflake case + return resultBuilder(getSession(), VARCHAR, VARCHAR, VARCHAR, VARCHAR) + .row("orderkey", "bigint", "", "") + .row("custkey", "bigint", "", "") + .row("orderstatus", "varchar(1)", "", "") + .row("totalprice", "double", "", "") + .row("orderdate", "date", "", "") + .row("orderpriority", "varchar(15)", "", "") + .row("clerk", "varchar(15)", "", "") + .row("shippriority", "bigint", "", "") + .row("comment", "varchar(79)", "", "") + .build(); + } + + @Test + @Override + public void testShowColumns() + { + assertThat(query("SHOW COLUMNS FROM orders")).matches(getDescribeOrdersResult()); + } + + @Test + public void testViews() + { + String tableName = "test_view_" + randomNameSuffix(); + onRemoteDatabase().execute("CREATE OR REPLACE VIEW tpch." + tableName + " AS SELECT * FROM tpch.orders"); + assertQuery("SELECT orderkey FROM " + tableName, "SELECT orderkey FROM orders"); + onRemoteDatabase().execute("DROP VIEW IF EXISTS tpch." + tableName); + } + + @Test + @Override + public void testShowCreateTable() + { + // Override this test because the type of row "shippriority" should be bigint rather than integer for snowflake case + assertThat(computeActual("SHOW CREATE TABLE orders").getOnlyValue()) + .isEqualTo("CREATE TABLE snowflake.tpch.orders (\n" + + " orderkey bigint,\n" + + " custkey bigint,\n" + + " orderstatus varchar(1),\n" + + " totalprice double,\n" + + " orderdate date,\n" + + " orderpriority varchar(15),\n" + + " clerk varchar(15),\n" + + " shippriority bigint,\n" + + " comment varchar(79)\n" + + ")\n" + + "COMMENT ''"); + } + + @Test + @Override + public void testAddNotNullColumn() + { + assertThatThrownBy(super::testAddNotNullColumn) + .isInstanceOf(AssertionError.class) + .hasMessage("Unexpected failure when adding not null column"); + } + + @Test + @Override + public void testCharVarcharComparison() + { + assertThatThrownBy(super::testCharVarcharComparison) + .hasMessageContaining("For query") + .hasMessageContaining("Actual rows") + .hasMessageContaining("Expected rows"); + } + + @Test + @Override + public void testCountDistinctWithStringTypes() + { + abort("TODO"); + } + + @Test + @Override + public void testInsertInPresenceOfNotSupportedColumn() + { + abort("TODO"); + } + + @Test + @Override + public void testAggregationPushdown() + { + abort("TODO"); + } + + @Test + @Override + public void testDistinctAggregationPushdown() + { + abort("TODO"); + } + + @Test + @Override + public void testNumericAggregationPushdown() + { + abort("TODO"); + } + + @Test + @Override + public void testLimitPushdown() + { + abort("TODO"); + } + + @Test + @Override + public void testInsertIntoNotNullColumn() + { + // TODO: java.lang.UnsupportedOperationException: This method should be overridden + assertThatThrownBy(super::testInsertIntoNotNullColumn); + } + + @Test + @Override + public void testDeleteWithLike() + { + assertThatThrownBy(super::testDeleteWithLike) + .hasStackTraceContaining("TrinoException: " + MODIFYING_ROWS_MESSAGE); + } + + @Test + @Override + public void testCreateTableAsSelect() + { + String tableName = "test_ctas" + randomNameSuffix(); + if (!hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA)) { + assertQueryFails("CREATE TABLE IF NOT EXISTS " + tableName + " AS SELECT name, regionkey FROM nation", "This connector does not support creating tables with data"); + return; + } + assertUpdate("CREATE TABLE IF NOT EXISTS " + tableName + " AS SELECT name, regionkey FROM nation", "SELECT count(*) FROM nation"); + assertTableColumnNames(tableName, "name", "regionkey"); + + assertEquals(getTableComment(getSession().getCatalog().orElseThrow(), getSession().getSchema().orElseThrow(), tableName), ""); + assertUpdate("DROP TABLE " + tableName); + + // Some connectors support CREATE TABLE AS but not the ordinary CREATE TABLE. Let's test CTAS IF NOT EXISTS with a table that is guaranteed to exist. + assertUpdate("CREATE TABLE IF NOT EXISTS nation AS SELECT nationkey, regionkey FROM nation", 0); + assertTableColumnNames("nation", "nationkey", "name", "regionkey", "comment"); + + assertCreateTableAsSelect( + "SELECT nationkey, name, regionkey FROM nation", + "SELECT count(*) FROM nation"); + + assertCreateTableAsSelect( + "SELECT mktsegment, sum(acctbal) x FROM customer GROUP BY mktsegment", + "SELECT count(DISTINCT mktsegment) FROM customer"); + + assertCreateTableAsSelect( + "SELECT count(*) x FROM nation JOIN region ON nation.regionkey = region.regionkey", + "SELECT 1"); + + assertCreateTableAsSelect( + "SELECT nationkey FROM nation ORDER BY nationkey LIMIT 10", + "SELECT 10"); + + assertCreateTableAsSelect( + "SELECT * FROM nation WITH DATA", + "SELECT * FROM nation", + "SELECT count(*) FROM nation"); + + assertCreateTableAsSelect( + "SELECT * FROM nation WITH NO DATA", + "SELECT * FROM nation LIMIT 0", + "SELECT 0"); + + // Tests for CREATE TABLE with UNION ALL: exercises PushTableWriteThroughUnion optimizer + + assertCreateTableAsSelect( + "SELECT name, nationkey, regionkey FROM nation WHERE nationkey % 2 = 0 UNION ALL " + + "SELECT name, nationkey, regionkey FROM nation WHERE nationkey % 2 = 1", + "SELECT name, nationkey, regionkey FROM nation", + "SELECT count(*) FROM nation"); + + assertCreateTableAsSelect( + Session.builder(getSession()).setSystemProperty("redistribute_writes", "true").build(), + "SELECT CAST(nationkey AS BIGINT) nationkey, regionkey FROM nation UNION ALL " + + "SELECT 1234567890, 123", + "SELECT nationkey, regionkey FROM nation UNION ALL " + + "SELECT 1234567890, 123", + "SELECT count(*) + 1 FROM nation"); + + assertCreateTableAsSelect( + Session.builder(getSession()).setSystemProperty("redistribute_writes", "false").build(), + "SELECT CAST(nationkey AS BIGINT) nationkey, regionkey FROM nation UNION ALL " + + "SELECT 1234567890, 123", + "SELECT nationkey, regionkey FROM nation UNION ALL " + + "SELECT 1234567890, 123", + "SELECT count(*) + 1 FROM nation"); + + // TODO: BigQuery throws table not found at BigQueryClient.insert if we reuse the same table name + tableName = "test_ctas" + randomNameSuffix(); + assertExplainAnalyze("EXPLAIN ANALYZE CREATE TABLE " + tableName + " AS SELECT name FROM nation"); + assertQuery("SELECT * from " + tableName, "SELECT name FROM nation"); + assertUpdate("DROP TABLE " + tableName); + } + + @Test + @Override + public void testCreateTable() + { + String tableName = "test_create_" + randomNameSuffix(); + if (!hasBehavior(SUPPORTS_CREATE_TABLE)) { + assertQueryFails("CREATE TABLE " + tableName + " (a bigint, b double, c varchar(50))", "This connector does not support creating tables"); + return; + } + + assertThat(computeActual("SHOW TABLES").getOnlyColumnAsSet()) // prime the cache, if any + .doesNotContain(tableName); + assertUpdate("CREATE TABLE " + tableName + " (a bigint, b double, c varchar(50))"); + assertTrue(getQueryRunner().tableExists(getSession(), tableName)); + assertThat(computeActual("SHOW TABLES").getOnlyColumnAsSet()) + .contains(tableName); + assertTableColumnNames(tableName, "a", "b", "c"); + assertEquals(getTableComment(getSession().getCatalog().orElseThrow(), getSession().getSchema().orElseThrow(), tableName), ""); + + assertUpdate("DROP TABLE " + tableName); + assertFalse(getQueryRunner().tableExists(getSession(), tableName)); + assertThat(computeActual("SHOW TABLES").getOnlyColumnAsSet()) + .doesNotContain(tableName); + + assertQueryFails("CREATE TABLE " + tableName + " (a bad_type)", ".* Unknown type 'bad_type' for column 'a'"); + assertFalse(getQueryRunner().tableExists(getSession(), tableName)); + + // TODO (https://github.com/trinodb/trino/issues/5901) revert to longer name when Oracle version is updated + tableName = "test_cr_not_exists_" + randomNameSuffix(); + assertUpdate("CREATE TABLE " + tableName + " (a bigint, b varchar(50), c double)"); + assertTrue(getQueryRunner().tableExists(getSession(), tableName)); + assertTableColumnNames(tableName, "a", "b", "c"); + + assertUpdate("CREATE TABLE IF NOT EXISTS " + tableName + " (d bigint, e varchar(50))"); + assertTrue(getQueryRunner().tableExists(getSession(), tableName)); + assertTableColumnNames(tableName, "a", "b", "c"); + + assertUpdate("DROP TABLE " + tableName); + assertFalse(getQueryRunner().tableExists(getSession(), tableName)); + + // Test CREATE TABLE LIKE + tableName = "test_create_orig_" + randomNameSuffix(); + assertUpdate("CREATE TABLE " + tableName + " (a bigint, b double, c varchar(50))"); + assertTrue(getQueryRunner().tableExists(getSession(), tableName)); + assertTableColumnNames(tableName, "a", "b", "c"); + + String tableNameLike = "test_create_like_" + randomNameSuffix(); + assertUpdate("CREATE TABLE " + tableNameLike + " (LIKE " + tableName + ", d bigint, e varchar(50))"); + assertTrue(getQueryRunner().tableExists(getSession(), tableNameLike)); + assertTableColumnNames(tableNameLike, "a", "b", "c", "d", "e"); + + assertUpdate("DROP TABLE " + tableName); + assertFalse(getQueryRunner().tableExists(getSession(), tableName)); + + assertUpdate("DROP TABLE " + tableNameLike); + assertFalse(getQueryRunner().tableExists(getSession(), tableNameLike)); + } + + @Test + @Override + public void testNativeQueryCreateStatement() + { + abort("TODO"); + } + + @Test + @Override + public void testNativeQueryInsertStatementTableExists() + { + abort("TODO"); + } + + @Test + @Override + public void testNativeQuerySelectUnsupportedType() + { + abort("TODO"); + } + + @Test + @Override + public void testCreateTableWithLongColumnName() + { + String tableName = "test_long_column" + randomNameSuffix(); + String baseColumnName = "col"; + + int maxLength = maxColumnNameLength() + // Assume 2^16 is enough for most use cases. Add a bit more to ensure 2^16 isn't actual limit. + .orElse(65536 + 5); + + String validColumnName = baseColumnName + "z".repeat(maxLength - baseColumnName.length()); + assertUpdate("CREATE TABLE " + tableName + " (" + validColumnName + " bigint)"); + assertTrue(columnExists(tableName, validColumnName)); + assertUpdate("DROP TABLE " + tableName); + + if (maxColumnNameLength().isEmpty()) { + return; + } + assertFalse(getQueryRunner().tableExists(getSession(), tableName)); + } + + @Test + @Override + public void testCreateTableWithLongTableName() + { + // TODO: Find the maximum table name length in Snowflake and enable this test. + abort("TODO"); + } + + @Override + protected OptionalInt maxColumnNameLength() + { + return OptionalInt.of(251); + } + + @Test + @Override + public void testAlterTableAddLongColumnName() + { + String tableName = "test_long_column" + randomNameSuffix(); + assertUpdate("CREATE TABLE " + tableName + " AS SELECT 123 x", 1); + + String baseColumnName = "col"; + int maxLength = maxColumnNameLength() + // Assume 2^16 is enough for most use cases. Add a bit more to ensure 2^16 isn't actual limit. + .orElse(65536 + 5); + + String validTargetColumnName = baseColumnName + "z".repeat(maxLength - baseColumnName.length()); + assertUpdate("ALTER TABLE " + tableName + " ADD COLUMN " + validTargetColumnName + " int"); + assertTrue(getQueryRunner().tableExists(getSession(), tableName)); + assertQuery("SELECT x FROM " + tableName, "VALUES 123"); + assertUpdate("DROP TABLE " + tableName); + + if (maxColumnNameLength().isEmpty()) { + return; + } + + assertUpdate("CREATE TABLE " + tableName + " AS SELECT 123 x", 1); + assertQuery("SELECT x FROM " + tableName, "VALUES 123"); + } + + @Test + @Override + public void testAlterTableRenameColumnToLongName() + { + String tableName = "test_long_column" + randomNameSuffix(); + assertUpdate("CREATE TABLE " + tableName + " AS SELECT 123 x", 1); + + String baseColumnName = "col"; + int maxLength = maxColumnNameLength() + // Assume 2^16 is enough for most use cases. Add a bit more to ensure 2^16 isn't actual limit. + .orElse(65536 + 5); + + String validTargetColumnName = baseColumnName + "z".repeat(maxLength - baseColumnName.length()); + assertUpdate("ALTER TABLE " + tableName + " RENAME COLUMN x TO " + validTargetColumnName); + assertQuery("SELECT " + validTargetColumnName + " FROM " + tableName, "VALUES 123"); + assertUpdate("DROP TABLE " + tableName); + + if (maxColumnNameLength().isEmpty()) { + return; + } + + assertUpdate("CREATE TABLE " + tableName + " AS SELECT 123 x", 1); + assertQuery("SELECT x FROM " + tableName, "VALUES 123"); + } + + @Test + @Override + public void testCreateSchemaWithLongName() + { + // TODO: Find the maximum table schema length in Snowflake and enable this test. + abort("TODO"); + } + + @Test + @Override + public void testInsertArray() + { + // Snowflake does not support this feature. + abort("Not supported"); + } + + @Test + @Override + public void testInsertRowConcurrently() + { + abort("TODO: Connection is already closed"); + } + + @Test + @Override + public void testNativeQueryColumnAlias() + { + abort("TODO: Table function system.query not registered"); + } + + @Test + @Override + public void testNativeQueryColumnAliasNotFound() + { + abort("TODO: Table function system.query not registered"); + } + + @Test + @Override + public void testNativeQueryIncorrectSyntax() + { + abort("TODO"); + } + + @Test + @Override + public void testNativeQueryInsertStatementTableDoesNotExist() + { + abort("TODO"); + } + + @Test + @Override + public void testNativeQueryParameters() + { + abort("TODO"); + } + + @Test + @Override + public void testNativeQuerySelectFromNation() + { + abort("TODO"); + } + + @Test + @Override + public void testNativeQuerySelectFromTestTable() + { + abort("TODO"); + } + + @Test + @Override + public void testNativeQuerySimple() + { + abort("TODO"); + } + + @Test + @Override + public void testRenameSchemaToLongName() + { + // TODO: Find the maximum table schema length in Snowflake and enable this test. + abort("TODO"); + } + + @Test + @Override + public void testRenameTableToLongTableName() + { + // TODO: Find the maximum table length in Snowflake and enable this test. + abort("TODO"); + } + + @Test + @Override + public void testCharTrailingSpace() + { + assertThatThrownBy(super::testCharVarcharComparison) + .hasMessageContaining("For query") + .hasMessageContaining("Actual rows") + .hasMessageContaining("Expected rows"); + } + + @Test + @Override + public void testDescribeTable() + { + assertThat(query("DESCRIBE orders")).matches(getDescribeOrdersResult()); + } +} diff --git a/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/SnowflakeQueryRunner.java b/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/SnowflakeQueryRunner.java new file mode 100644 index 0000000000000..a50debaf003b6 --- /dev/null +++ b/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/SnowflakeQueryRunner.java @@ -0,0 +1,97 @@ +/* + * 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.snowflake; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.airlift.log.Logger; +import io.trino.Session; +import io.trino.plugin.tpch.TpchPlugin; +import io.trino.testing.DistributedQueryRunner; +import io.trino.tpch.TpchTable; + +import java.util.HashMap; +import java.util.Map; + +import static io.airlift.testing.Closeables.closeAllSuppress; +import static io.trino.plugin.tpch.TpchMetadata.TINY_SCHEMA_NAME; +import static io.trino.testing.QueryAssertions.copyTpchTables; +import static io.trino.testing.TestingSession.testSessionBuilder; + +public final class SnowflakeQueryRunner +{ + public static final String TPCH_SCHEMA = "tpch"; + + private SnowflakeQueryRunner() {} + + public static DistributedQueryRunner createSnowflakeQueryRunner( + TestingSnowflakeServer server, + Map extraProperties, + Map connectorProperties, + Iterable> tables) + throws Exception + { + DistributedQueryRunner queryRunner = DistributedQueryRunner.builder(createSession()) + .setExtraProperties(extraProperties) + .build(); + try { + queryRunner.installPlugin(new TpchPlugin()); + queryRunner.createCatalog("tpch", "tpch"); + + connectorProperties = new HashMap<>(ImmutableMap.copyOf(connectorProperties)); + connectorProperties.putIfAbsent("connection-url", TestingSnowflakeServer.TEST_URL); + connectorProperties.putIfAbsent("connection-user", TestingSnowflakeServer.TEST_USER); + connectorProperties.putIfAbsent("connection-password", TestingSnowflakeServer.TEST_PASSWORD); + connectorProperties.putIfAbsent("snowflake.database", TestingSnowflakeServer.TEST_DATABASE); + connectorProperties.putIfAbsent("snowflake.role", TestingSnowflakeServer.TEST_ROLE); + connectorProperties.putIfAbsent("snowflake.warehouse", TestingSnowflakeServer.TEST_WAREHOUSE); + if (TestingSnowflakeServer.TEST_PROXY != null) { + connectorProperties.putIfAbsent("snowflake.httpProxy", TestingSnowflakeServer.TEST_PROXY); + } + + queryRunner.installPlugin(new SnowflakePlugin()); + queryRunner.createCatalog("snowflake", "snowflake", connectorProperties); + + copyTpchTables(queryRunner, "tpch", TINY_SCHEMA_NAME, createSession(), tables); + + return queryRunner; + } + catch (Throwable e) { + closeAllSuppress(e, queryRunner); + throw e; + } + } + + public static Session createSession() + { + return testSessionBuilder() + .setCatalog("snowflake") + .setSchema(TPCH_SCHEMA) + .build(); + } + + public static void main(String[] args) + throws Exception + { + DistributedQueryRunner queryRunner = createSnowflakeQueryRunner( + new TestingSnowflakeServer(), + ImmutableMap.of("http-server.http.port", "8080"), + ImmutableMap.of(), + ImmutableList.of()); + + Logger log = Logger.get(SnowflakeQueryRunner.class); + log.info("======== SERVER STARTED ========"); + log.info("\n====\n%s\n====", queryRunner.getCoordinator().getBaseUrl()); + } +} diff --git a/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/TestSnowflakeClient.java b/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/TestSnowflakeClient.java new file mode 100644 index 0000000000000..b743314af7633 --- /dev/null +++ b/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/TestSnowflakeClient.java @@ -0,0 +1,153 @@ +/* + * 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.snowflake; + +import io.trino.plugin.base.mapping.DefaultIdentifierMapping; +import io.trino.plugin.jdbc.BaseJdbcConfig; +import io.trino.plugin.jdbc.ColumnMapping; +import io.trino.plugin.jdbc.DefaultQueryBuilder; +import io.trino.plugin.jdbc.JdbcClient; +import io.trino.plugin.jdbc.JdbcColumnHandle; +import io.trino.plugin.jdbc.JdbcExpression; +import io.trino.plugin.jdbc.JdbcTypeHandle; +import io.trino.plugin.jdbc.logging.RemoteQueryModifier; +import io.trino.spi.connector.AggregateFunction; +import io.trino.spi.connector.ColumnHandle; +import io.trino.spi.expression.ConnectorExpression; +import io.trino.spi.expression.Variable; +import org.testng.annotations.Test; + +import java.sql.Types; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static io.trino.spi.type.BigintType.BIGINT; +import static io.trino.spi.type.BooleanType.BOOLEAN; +import static io.trino.spi.type.DoubleType.DOUBLE; +import static io.trino.testing.TestingConnectorSession.SESSION; +import static io.trino.type.InternalTypeManager.TESTING_TYPE_MANAGER; +import static org.assertj.core.api.Assertions.assertThat; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; + +public class TestSnowflakeClient +{ + private static final JdbcColumnHandle BIGINT_COLUMN = + JdbcColumnHandle.builder() + .setColumnName("c_bigint") + .setColumnType(BIGINT) + .setJdbcTypeHandle(new JdbcTypeHandle(Types.BIGINT, Optional.of("int8"), Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty())) + .build(); + + private static final JdbcColumnHandle DOUBLE_COLUMN = + JdbcColumnHandle.builder() + .setColumnName("c_double") + .setColumnType(DOUBLE) + .setJdbcTypeHandle(new JdbcTypeHandle(Types.DOUBLE, Optional.of("double"), Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty())) + .build(); + + private static final JdbcClient JDBC_CLIENT = new SnowflakeClient( + new BaseJdbcConfig(), + session -> { throw new UnsupportedOperationException(); }, + new DefaultQueryBuilder(RemoteQueryModifier.NONE), + TESTING_TYPE_MANAGER, + new DefaultIdentifierMapping(), + RemoteQueryModifier.NONE); + + @Test + public void testImplementCount() + { + Variable bigintVariable = new Variable("v_bigint", BIGINT); + Variable doubleVariable = new Variable("v_double", BIGINT); + Optional filter = Optional.of(new Variable("a_filter", BOOLEAN)); + + // count(*) + testImplementAggregation( + new AggregateFunction("count", BIGINT, List.of(), List.of(), false, Optional.empty()), + Map.of(), + Optional.of("count(*)")); + + // count(bigint) + testImplementAggregation( + new AggregateFunction("count", BIGINT, List.of(bigintVariable), List.of(), false, Optional.empty()), + Map.of(bigintVariable.getName(), BIGINT_COLUMN), + Optional.of("count(\"c_bigint\")")); + + // count(double) + testImplementAggregation( + new AggregateFunction("count", BIGINT, List.of(doubleVariable), List.of(), false, Optional.empty()), + Map.of(doubleVariable.getName(), DOUBLE_COLUMN), + Optional.of("count(\"c_double\")")); + + // count() FILTER (WHERE ...) + testImplementAggregation( + new AggregateFunction("count", BIGINT, List.of(), List.of(), false, filter), + Map.of(), + Optional.empty()); + + // count(bigint) FILTER (WHERE ...) + testImplementAggregation( + new AggregateFunction("count", BIGINT, List.of(bigintVariable), List.of(), false, filter), + Map.of(bigintVariable.getName(), BIGINT_COLUMN), + Optional.empty()); + } + + @Test + public void testImplementSum() + { + Variable bigintVariable = new Variable("v_bigint", BIGINT); + Variable doubleVariable = new Variable("v_double", DOUBLE); + Optional filter = Optional.of(new Variable("a_filter", BOOLEAN)); + + // sum(bigint) + testImplementAggregation( + new AggregateFunction("sum", BIGINT, List.of(bigintVariable), List.of(), false, Optional.empty()), + Map.of(bigintVariable.getName(), BIGINT_COLUMN), + Optional.of("sum(\"c_bigint\")")); + + // sum(double) + testImplementAggregation( + new AggregateFunction("sum", DOUBLE, List.of(doubleVariable), List.of(), false, Optional.empty()), + Map.of(doubleVariable.getName(), DOUBLE_COLUMN), + Optional.of("sum(\"c_double\")")); + + // sum(DISTINCT bigint) + testImplementAggregation( + new AggregateFunction("sum", BIGINT, List.of(bigintVariable), List.of(), true, Optional.empty()), + Map.of(bigintVariable.getName(), BIGINT_COLUMN), + Optional.of("sum(DISTINCT \"c_bigint\")")); + + // sum(bigint) FILTER (WHERE ...) + testImplementAggregation( + new AggregateFunction("sum", BIGINT, List.of(bigintVariable), List.of(), false, filter), + Map.of(bigintVariable.getName(), BIGINT_COLUMN), + Optional.empty()); // filter not supported + } + + private static void testImplementAggregation(AggregateFunction aggregateFunction, Map assignments, Optional expectedExpression) + { + Optional result = JDBC_CLIENT.implementAggregation(SESSION, aggregateFunction, assignments); + if (expectedExpression.isEmpty()) { + assertThat(result).isEmpty(); + } + else { + assertThat(result).isPresent(); + assertEquals(result.get().getExpression(), expectedExpression.get()); + Optional columnMapping = JDBC_CLIENT.toColumnMapping(SESSION, null, result.get().getJdbcTypeHandle()); + assertTrue(columnMapping.isPresent(), "No mapping for: " + result.get().getJdbcTypeHandle()); + assertEquals(columnMapping.get().getType(), aggregateFunction.getOutputType()); + } + } +} diff --git a/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/TestSnowflakeConfig.java b/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/TestSnowflakeConfig.java new file mode 100644 index 0000000000000..eb5c32a3d063c --- /dev/null +++ b/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/TestSnowflakeConfig.java @@ -0,0 +1,61 @@ +/* + * 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.snowflake; + +import com.google.common.collect.ImmutableMap; +import org.testng.annotations.Test; + +import java.util.Map; + +import static io.airlift.configuration.testing.ConfigAssertions.assertFullMapping; +import static io.airlift.configuration.testing.ConfigAssertions.assertRecordedDefaults; +import static io.airlift.configuration.testing.ConfigAssertions.recordDefaults; + +public class TestSnowflakeConfig +{ + @Test + public void testDefaults() + { + assertRecordedDefaults(recordDefaults(SnowflakeConfig.class) + .setAccount(null) + .setDatabase(null) + .setRole(null) + .setWarehouse(null) + .setHTTPProxy(null) + .setTimestampNoTimezoneAsUTC(null)); + } + + @Test + public void testExplicitPropertyMappings() + { + Map properties = ImmutableMap.builder() + .put("snowflake.account", "MYACCOUNT") + .put("snowflake.database", "MYDATABASE") + .put("snowflake.role", "MYROLE") + .put("snowflake.warehouse", "MYWAREHOUSE") + .put("snowflake.http-proxy", "MYPROXY") + .put("snowflake.timestamp-no-timezone-as-utc", "true") + .buildOrThrow(); + + SnowflakeConfig expected = new SnowflakeConfig() + .setAccount("MYACCOUNT") + .setDatabase("MYDATABASE") + .setRole("MYROLE") + .setWarehouse("MYWAREHOUSE") + .setHTTPProxy("MYPROXY") + .setTimestampNoTimezoneAsUTC(true); + + assertFullMapping(properties, expected); + } +} diff --git a/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/TestSnowflakeConnectorTest.java b/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/TestSnowflakeConnectorTest.java new file mode 100644 index 0000000000000..8b9b0c78c73b5 --- /dev/null +++ b/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/TestSnowflakeConnectorTest.java @@ -0,0 +1,38 @@ +/* + * 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.snowflake; + +import com.google.common.collect.ImmutableMap; +import io.trino.testing.QueryRunner; +import io.trino.testing.sql.SqlExecutor; + +import static io.trino.plugin.snowflake.SnowflakeQueryRunner.createSnowflakeQueryRunner; + +public class TestSnowflakeConnectorTest + extends BaseSnowflakeConnectorTest +{ + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + server = closeAfterClass(new TestingSnowflakeServer()); + return createSnowflakeQueryRunner(server, ImmutableMap.of(), ImmutableMap.of(), REQUIRED_TPCH_TABLES); + } + + @Override + protected SqlExecutor onRemoteDatabase() + { + return server::execute; + } +} diff --git a/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/TestSnowflakePlugin.java b/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/TestSnowflakePlugin.java new file mode 100644 index 0000000000000..26165c3f018cb --- /dev/null +++ b/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/TestSnowflakePlugin.java @@ -0,0 +1,33 @@ +/* + * 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.snowflake; + +import com.google.common.collect.ImmutableMap; +import io.trino.spi.Plugin; +import io.trino.spi.connector.ConnectorFactory; +import io.trino.testing.TestingConnectorContext; +import org.testng.annotations.Test; + +import static com.google.common.collect.Iterables.getOnlyElement; + +public class TestSnowflakePlugin +{ + @Test + public void testCreateConnector() + { + Plugin plugin = new SnowflakePlugin(); + ConnectorFactory factory = getOnlyElement(plugin.getConnectorFactories()); + factory.create("test", ImmutableMap.of("connection-url", "jdbc:snowflake://test"), new TestingConnectorContext()).shutdown(); + } +} diff --git a/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/TestSnowflakeTypeMapping.java b/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/TestSnowflakeTypeMapping.java new file mode 100644 index 0000000000000..5377e013a6cd8 --- /dev/null +++ b/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/TestSnowflakeTypeMapping.java @@ -0,0 +1,391 @@ +/* + * 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.snowflake; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.trino.Session; +import io.trino.spi.type.TimeZoneKey; +import io.trino.testing.AbstractTestQueryFramework; +import io.trino.testing.QueryRunner; +import io.trino.testing.TestingSession; +import io.trino.testing.datatype.CreateAndInsertDataSetup; +import io.trino.testing.datatype.CreateAsSelectDataSetup; +import io.trino.testing.datatype.DataSetup; +import io.trino.testing.datatype.SqlDataTypeTest; +import io.trino.testing.sql.TrinoSqlExecutor; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.parallel.Execution; + +import java.time.LocalDate; +import java.time.ZoneId; + +import static com.google.common.base.Preconditions.checkState; +import static com.google.common.base.Verify.verify; +import static io.trino.plugin.snowflake.SnowflakeQueryRunner.createSnowflakeQueryRunner; +import static io.trino.spi.type.BigintType.BIGINT; +import static io.trino.spi.type.BooleanType.BOOLEAN; +import static io.trino.spi.type.DateType.DATE; +import static io.trino.spi.type.DecimalType.createDecimalType; +import static io.trino.spi.type.DoubleType.DOUBLE; +import static io.trino.spi.type.TimeZoneKey.getTimeZoneKey; +import static io.trino.spi.type.TimestampType.createTimestampType; +import static io.trino.spi.type.VarbinaryType.VARBINARY; +import static io.trino.spi.type.VarcharType.createVarcharType; +import static java.time.ZoneOffset.UTC; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; + +@TestInstance(PER_CLASS) +@Execution(CONCURRENT) +public class TestSnowflakeTypeMapping + extends AbstractTestQueryFramework +{ + protected TestingSnowflakeServer snowflakeServer; + + private final ZoneId jvmZone = ZoneId.systemDefault(); + // no DST in 1970, but has DST in later years (e.g. 2018) + private final ZoneId vilnius = ZoneId.of("Europe/Vilnius"); + // minutes offset change since 1970-01-01, no DST + private final ZoneId kathmandu = ZoneId.of("Asia/Kathmandu"); + + @BeforeAll + public void setUp() + { + String zone = jvmZone.getId(); + checkState(jvmZone.getId().equals("America/Bahia_Banderas"), "Timezone not configured correctly. Add -Duser.timezone=America/Bahia_Banderas to your JVM arguments"); + checkIsGap(jvmZone, LocalDate.of(1970, 1, 1)); + checkIsGap(vilnius, LocalDate.of(1983, 4, 1)); + verify(vilnius.getRules().getValidOffsets(LocalDate.of(1983, 10, 1).atStartOfDay().minusMinutes(1)).size() == 2); + } + + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + snowflakeServer = new TestingSnowflakeServer(); + return createSnowflakeQueryRunner( + snowflakeServer, + ImmutableMap.of(), + ImmutableMap.of(), + ImmutableList.of()); + } + + @Test + public void testBoolean() + { + SqlDataTypeTest.create() + .addRoundTrip("boolean", "true", BOOLEAN, "BOOLEAN '1'") + .addRoundTrip("boolean", "false", BOOLEAN, "BOOLEAN '0'") + .addRoundTrip("boolean", "NULL", BOOLEAN, "CAST(NULL AS BOOLEAN)") + .execute(getQueryRunner(), snowflakeCreateAndInsert("tpch.test_boolean")) + .execute(getQueryRunner(), trinoCreateAsSelect("tpch.test_boolean")) + .execute(getQueryRunner(), trinoCreateAndInsert("tpch.test_boolean")); + } + + @Test + public void testInteger() + { + // INT , INTEGER , BIGINT , SMALLINT , TINYINT , BYTEINT, DECIMAL , NUMERIC are aliases for NUMBER(38, 0) in snowflake + // https://docs.snowflake.com/en/sql-reference/data-types-numeric.html#int-integer-bigint-smallint-tinyint-byteint + testInteger("INT"); + testInteger("INTEGER"); + testInteger("BIGINT"); + testInteger("SMALLINT"); + testInteger("TINYINT"); + testInteger("BYTEINT"); + } + + private void testInteger(String inputType) + { + SqlDataTypeTest.create() + .addRoundTrip(inputType, "-9223372036854775808", BIGINT, "-9223372036854775808") + .addRoundTrip(inputType, "9223372036854775807", BIGINT, "9223372036854775807") + .addRoundTrip(inputType, "0", BIGINT, "CAST(0 AS BIGINT)") + .addRoundTrip(inputType, "NULL", BIGINT, "CAST(NULL AS BIGINT)") + .execute(getQueryRunner(), snowflakeCreateAndInsert("tpch.integer")); + } + + @Test + public void testDecimal() + { + SqlDataTypeTest.create() + .addRoundTrip("decimal(3, 0)", "NULL", BIGINT, "CAST(NULL AS BIGINT)") + .addRoundTrip("decimal(3, 0)", "CAST('193' AS decimal(3, 0))", BIGINT, "CAST('193' AS BIGINT)") + .addRoundTrip("decimal(3, 0)", "CAST('19' AS decimal(3, 0))", BIGINT, "CAST('19' AS BIGINT)") + .addRoundTrip("decimal(3, 0)", "CAST('-193' AS decimal(3, 0))", BIGINT, "CAST('-193' AS BIGINT)") + .addRoundTrip("decimal(3, 1)", "CAST('10.0' AS decimal(3, 1))", createDecimalType(3, 1), "CAST('10.0' AS decimal(3, 1))") + .addRoundTrip("decimal(3, 1)", "CAST('10.1' AS decimal(3, 1))", createDecimalType(3, 1), "CAST('10.1' AS decimal(3, 1))") + .addRoundTrip("decimal(3, 1)", "CAST('-10.1' AS decimal(3, 1))", createDecimalType(3, 1), "CAST('-10.1' AS decimal(3, 1))") + .addRoundTrip("decimal(4, 2)", "CAST('2' AS decimal(4, 2))", createDecimalType(4, 2), "CAST('2' AS decimal(4, 2))") + .addRoundTrip("decimal(4, 2)", "CAST('2.3' AS decimal(4, 2))", createDecimalType(4, 2), "CAST('2.3' AS decimal(4, 2))") + .addRoundTrip("decimal(24, 2)", "CAST('2' AS decimal(24, 2))", createDecimalType(24, 2), "CAST('2' AS decimal(24, 2))") + .addRoundTrip("decimal(24, 2)", "CAST('2.3' AS decimal(24, 2))", createDecimalType(24, 2), "CAST('2.3' AS decimal(24, 2))") + .addRoundTrip("decimal(24, 2)", "CAST('123456789.3' AS decimal(24, 2))", createDecimalType(24, 2), "CAST('123456789.3' AS decimal(24, 2))") + .addRoundTrip("decimal(24, 4)", "CAST('12345678901234567890.31' AS decimal(24, 4))", createDecimalType(24, 4), "CAST('12345678901234567890.31' AS decimal(24, 4))") + .addRoundTrip("decimal(30, 5)", "CAST('3141592653589793238462643.38327' AS decimal(30, 5))", createDecimalType(30, 5), "CAST('3141592653589793238462643.38327' AS decimal(30, 5))") + .addRoundTrip("decimal(30, 5)", "CAST('-3141592653589793238462643.38327' AS decimal(30, 5))", createDecimalType(30, 5), "CAST('-3141592653589793238462643.38327' AS decimal(30, 5))") + .addRoundTrip("decimal(38, 0)", "CAST(NULL AS decimal(38, 0))", BIGINT, "CAST(NULL AS BIGINT)") + .execute(getQueryRunner(), snowflakeCreateAndInsert("tpch.test_decimal")) + .execute(getQueryRunner(), trinoCreateAsSelect("test_decimal")) + .execute(getQueryRunner(), trinoCreateAndInsert("test_decimal")); + } + + @Test + public void testFloat() + { + // https://docs.snowflake.com/en/sql-reference/data-types-numeric.html#float-float4-float8 + SqlDataTypeTest.create() + .addRoundTrip("real", "3.14", DOUBLE, "DOUBLE '3.14'") + .addRoundTrip("real", "10.3e0", DOUBLE, "DOUBLE '10.3e0'") + .addRoundTrip("real", "NULL", DOUBLE, "CAST(NULL AS DOUBLE)") + .addRoundTrip("real", "CAST('NaN' AS DOUBLE)", DOUBLE, "nan()") + .addRoundTrip("real", "CAST('Infinity' AS DOUBLE)", DOUBLE, "+infinity()") + .addRoundTrip("real", "CAST('-Infinity' AS DOUBLE)", DOUBLE, "-infinity()") + .execute(getQueryRunner(), trinoCreateAsSelect("tpch.test_real")) + .execute(getQueryRunner(), trinoCreateAndInsert("tpch.test_real")); + + SqlDataTypeTest.create() + .addRoundTrip("float", "3.14", DOUBLE, "DOUBLE '3.14'") + .addRoundTrip("float", "10.3e0", DOUBLE, "DOUBLE '10.3e0'") + .addRoundTrip("float", "NULL", DOUBLE, "CAST(NULL AS DOUBLE)") + .addRoundTrip("float", "CAST('NaN' AS float)", DOUBLE, "nan()") + .addRoundTrip("float", "CAST('Infinity' AS float)", DOUBLE, "+infinity()") + .addRoundTrip("float", "CAST('-Infinity' AS float)", DOUBLE, "-infinity()") + .execute(getQueryRunner(), snowflakeCreateAndInsert("tpch.test_float")); + } + + @Test + public void testDouble() + { + SqlDataTypeTest.create() + .addRoundTrip("double", "3.14", DOUBLE, "CAST(3.14 AS DOUBLE)") + .addRoundTrip("double", "1.0E100", DOUBLE, "1.0E100") + .addRoundTrip("double", "1.23456E12", DOUBLE, "1.23456E12") + .addRoundTrip("double", "NULL", DOUBLE, "CAST(NULL AS DOUBLE)") + .addRoundTrip("double", "CAST('NaN' AS DOUBLE)", DOUBLE, "nan()") + .addRoundTrip("double", "CAST('Infinity' AS DOUBLE)", DOUBLE, "+infinity()") + .addRoundTrip("double", "CAST('-Infinity' AS DOUBLE)", DOUBLE, "-infinity()") + .execute(getQueryRunner(), trinoCreateAsSelect("trino_test_double")) + .execute(getQueryRunner(), trinoCreateAndInsert("trino_test_double")) + .execute(getQueryRunner(), snowflakeCreateAndInsert("tpch.test_double")); + } + + @Test + public void testSnowflakeCreatedParameterizedVarchar() + { + SqlDataTypeTest.create() + .addRoundTrip("text", "'b'", createVarcharType(16777216), "CAST('b' AS VARCHAR(16777216))") + .addRoundTrip("varchar(32)", "'e'", createVarcharType(32), "CAST('e' AS VARCHAR(32))") + .addRoundTrip("varchar(15000)", "'f'", createVarcharType(15000), "CAST('f' AS VARCHAR(15000))") + .execute(getQueryRunner(), snowflakeCreateAndInsert("tpch.snowflake_test_parameterized_varchar")); + } + + @Test + public void testSnowflakeCreatedParameterizedVarcharUnicode() + { + SqlDataTypeTest.create() + .addRoundTrip("text collate \'utf8\'", "'攻殻機動隊'", createVarcharType(16777216), "CAST('攻殻機動隊' AS VARCHAR(16777216))") + .addRoundTrip("varchar(5) collate \'utf8\'", "'攻殻機動隊'", createVarcharType(5), "CAST('攻殻機動隊' AS VARCHAR(5))") + .addRoundTrip("varchar(32) collate \'utf8\'", "'攻殻機動隊'", createVarcharType(32), "CAST('攻殻機動隊' AS VARCHAR(32))") + .addRoundTrip("varchar(20000) collate \'utf8\'", "'攻殻機動隊'", createVarcharType(20000), "CAST('攻殻機動隊' AS VARCHAR(20000))") + .addRoundTrip("varchar(1) collate \'utf8mb4\'", "'😂'", createVarcharType(1), "CAST('😂' AS VARCHAR(1))") + .addRoundTrip("varchar(77) collate \'utf8mb4\'", "'Ну, погоди!'", createVarcharType(77), "CAST('Ну, погоди!' AS VARCHAR(77))") + .execute(getQueryRunner(), snowflakeCreateAndInsert("tpch.snowflake_test_parameterized_varchar_unicode")); + } + + @Test + public void testParameterizedChar() + { + SqlDataTypeTest.create() + .addRoundTrip("char", "''", createVarcharType(1), "CAST(' ' AS varchar(1))") + .addRoundTrip("char", "'a'", createVarcharType(1), "CAST('a' AS varchar(1))") + .addRoundTrip("char(1)", "''", createVarcharType(1), "CAST(' ' AS varchar(1))") + .addRoundTrip("char(1)", "'a'", createVarcharType(1), "CAST('a' AS varchar(1))") + .addRoundTrip("char(8)", "'abc'", createVarcharType(8), "CAST('abc ' AS varchar(8))") + .addRoundTrip("char(8)", "'12345678'", createVarcharType(8), "CAST('12345678' AS varchar(8))") + .execute(getQueryRunner(), trinoCreateAsSelect("snowflake_test_parameterized_char")); + + SqlDataTypeTest.create() + .addRoundTrip("char", "''", createVarcharType(1), "CAST('' AS varchar(1))") + .addRoundTrip("char", "'a'", createVarcharType(1), "CAST('a' AS varchar(1))") + .addRoundTrip("char(1)", "''", createVarcharType(1), "CAST('' AS varchar(1))") + .addRoundTrip("char(1)", "'a'", createVarcharType(1), "CAST('a' AS varchar(1))") + .addRoundTrip("char(8)", "'abc'", createVarcharType(8), "CAST('abc' AS varchar(8))") + .addRoundTrip("char(8)", "'12345678'", createVarcharType(8), "CAST('12345678' AS varchar(8))") + .execute(getQueryRunner(), trinoCreateAndInsert("snowflake_test_parameterized_char")) + .execute(getQueryRunner(), snowflakeCreateAndInsert("tpch.snowflake_test_parameterized_char")); + } + + @Test + public void testSnowflakeParameterizedCharUnicode() + { + SqlDataTypeTest.create() + .addRoundTrip("char(1) collate \'utf8\'", "'攻'", createVarcharType(1), "CAST('攻' AS VARCHAR(1))") + .addRoundTrip("char(5) collate \'utf8\'", "'攻殻'", createVarcharType(5), "CAST('攻殻' AS VARCHAR(5))") + .addRoundTrip("char(5) collate \'utf8\'", "'攻殻機動隊'", createVarcharType(5), "CAST('攻殻機動隊' AS VARCHAR(5))") + .addRoundTrip("char(1)", "'😂'", createVarcharType(1), "CAST('😂' AS VARCHAR(1))") + .addRoundTrip("char(77)", "'Ну, погоди!'", createVarcharType(77), "CAST('Ну, погоди!' AS VARCHAR(77))") + .execute(getQueryRunner(), snowflakeCreateAndInsert("tpch.snowflake_test_parameterized_char")); + } + + @Test + public void testBinary() + { + SqlDataTypeTest.create() + .addRoundTrip("binary(18)", "NULL", VARBINARY, "CAST(NULL AS varbinary)") + .addRoundTrip("binary(18)", "X''", VARBINARY, "X''") + .addRoundTrip("binary(18)", "X'68656C6C6F'", VARBINARY, "to_utf8('hello')") + .addRoundTrip("binary(18)", "X'C582C4856B61207720E69DB1E4BAACE983BD'", VARBINARY, "to_utf8('łąka w 東京都')") // no trailing zeros + .addRoundTrip("binary(18)", "X'4261672066756C6C206F6620F09F92B0'", VARBINARY, "to_utf8('Bag full of 💰')") + .addRoundTrip("binary(18)", "X'0001020304050607080DF9367AA7000000'", VARBINARY, "X'0001020304050607080DF9367AA7000000'") // non-text prefix + .addRoundTrip("binary(18)", "X'000000000000'", VARBINARY, "X'000000000000'") + .execute(getQueryRunner(), snowflakeCreateAndInsert("tpch.test_binary")); + } + + @Test + public void testVarbinary() + { + SqlDataTypeTest.create() + .addRoundTrip("varbinary", "NULL", VARBINARY, "CAST(NULL AS varbinary)") + .addRoundTrip("varbinary", "X''", VARBINARY, "X''") + .addRoundTrip("varbinary", "X'68656C6C6F'", VARBINARY, "to_utf8('hello')") + .addRoundTrip("varbinary", "X'5069C4996B6E6120C582C4856B61207720E69DB1E4BAACE983BD'", VARBINARY, "to_utf8('Piękna łąka w 東京都')") + .addRoundTrip("varbinary", "X'4261672066756C6C206F6620F09F92B0'", VARBINARY, "to_utf8('Bag full of 💰')") + .addRoundTrip("varbinary", "X'0001020304050607080DF9367AA7000000'", VARBINARY, "X'0001020304050607080DF9367AA7000000'") // non-text + .addRoundTrip("varbinary", "X'000000000000'", VARBINARY, "X'000000000000'") + .execute(getQueryRunner(), trinoCreateAsSelect("test_varbinary")) + .execute(getQueryRunner(), trinoCreateAndInsert("test_varbinary")) + .execute(getQueryRunner(), snowflakeCreateAndInsert("tpch.test_varbinary")); + } + + @Test + public void testDate() + { + testDate(UTC); + testDate(jvmZone); + testDate(vilnius); + testDate(kathmandu); + testDate(TestingSession.DEFAULT_TIME_ZONE_KEY.getZoneId()); + } + + private void testDate(ZoneId sessionZone) + { + Session session = Session.builder(getSession()) + .setTimeZoneKey(getTimeZoneKey(sessionZone.getId())) + .build(); + + SqlDataTypeTest.create() + .addRoundTrip("date", "NULL", DATE, "CAST(NULL AS DATE)") + .addRoundTrip("date", "'-5877641-06-23'", DATE, "DATE '-5877641-06-23'") // min value in Trino + .addRoundTrip("date", "'0000-01-01'", DATE, "DATE '0000-01-01'") + .addRoundTrip("date", "DATE '0001-01-01'", DATE, "DATE '0001-01-01'") // Min value for the function Date. + .addRoundTrip("date", "DATE '1582-10-05'", DATE, "DATE '1582-10-05'") // begin julian->gregorian switch + .addRoundTrip("date", "DATE '1582-10-14'", DATE, "DATE '1582-10-14'") // end julian->gregorian switch + .addRoundTrip("date", "DATE '1983-04-01'", DATE, "DATE '1983-04-01'") + .addRoundTrip("date", "DATE '1983-10-01'", DATE, "DATE '1983-10-01'") + .addRoundTrip("date", "DATE '2017-07-01'", DATE, "DATE '2017-07-01'") // summer on northern hemisphere (possible DST) + .addRoundTrip("date", "DATE '2017-01-01'", DATE, "DATE '2017-01-01'") // winter on northern hemisphere (possible DST on southern hemisphere) + .addRoundTrip("date", "DATE '99999-12-31'", DATE, "DATE '99999-12-31'") + .addRoundTrip("date", "'5881580-07-11'", DATE, "DATE '5881580-07-11'") // max value in Trino + .execute(getQueryRunner(), session, trinoCreateAsSelect("test_date")) + .execute(getQueryRunner(), session, snowflakeCreateAndInsert("tpch.test_date")); + } + + @Test + public void testTimestamp() + { + testTimestamp(UTC); + testTimestamp(jvmZone); + testTimestamp(vilnius); + testTimestamp(kathmandu); + testTimestamp(TestingSession.DEFAULT_TIME_ZONE_KEY.getZoneId()); + } + + private void testTimestamp(ZoneId sessionZone) + { + Session session = Session.builder(getSession()) + .setTimeZoneKey(TimeZoneKey.getTimeZoneKey(sessionZone.getId())) + .build(); + + SqlDataTypeTest.create() + // after epoch (MariaDb's timestamp type doesn't support values <= epoch) + .addRoundTrip("timestamp(3)", "TIMESTAMP '2019-03-18 10:01:17.987'", createTimestampType(3), "TIMESTAMP '2019-03-18 10:01:17.987'") + // time doubled in JVM zone + .addRoundTrip("timestamp(3)", "TIMESTAMP '2018-10-28 01:33:17.456'", createTimestampType(3), "TIMESTAMP '2018-10-28 01:33:17.456'") + // time double in Vilnius + .addRoundTrip("timestamp(3)", "TIMESTAMP '2018-10-28 03:33:33.333'", createTimestampType(3), "TIMESTAMP '2018-10-28 03:33:33.333'") + .addRoundTrip("timestamp(3)", "TIMESTAMP '1970-01-01 00:13:42.000'", createTimestampType(3), "TIMESTAMP '1970-01-01 00:13:42.000'") + .addRoundTrip("timestamp(3)", "TIMESTAMP '2018-04-01 02:13:55.123'", createTimestampType(3), "TIMESTAMP '2018-04-01 02:13:55.123'") + // time gap in Vilnius + .addRoundTrip("timestamp(3)", "TIMESTAMP '2018-03-25 03:17:17.000'", createTimestampType(3), "TIMESTAMP '2018-03-25 03:17:17.000'") + // time gap in Kathmandu + .addRoundTrip("timestamp(3)", "TIMESTAMP '1986-01-01 00:13:07.000'", createTimestampType(3), "TIMESTAMP '1986-01-01 00:13:07.000'") + // max value 2038-01-19 03:14:07 + .addRoundTrip("timestamp(3)", "TIMESTAMP '2038-01-19 03:14:07.000'", createTimestampType(3), "TIMESTAMP '2038-01-19 03:14:07.000'") + // test arbitrary time for all supported precisions + .addRoundTrip("timestamp(0)", "TIMESTAMP '1970-01-01 00:00:01'", createTimestampType(0), "TIMESTAMP '1970-01-01 00:00:01'") + .addRoundTrip("timestamp(1)", "TIMESTAMP '1970-01-01 00:00:01.1'", createTimestampType(1), "TIMESTAMP '1970-01-01 00:00:01.1'") + .addRoundTrip("timestamp(1)", "TIMESTAMP '1970-01-01 00:00:01.9'", createTimestampType(1), "TIMESTAMP '1970-01-01 00:00:01.9'") + .addRoundTrip("timestamp(2)", "TIMESTAMP '1970-01-01 00:00:01.12'", createTimestampType(2), "TIMESTAMP '1970-01-01 00:00:01.12'") + .addRoundTrip("timestamp(3)", "TIMESTAMP '1970-01-01 00:00:01.123'", createTimestampType(3), "TIMESTAMP '1970-01-01 00:00:01.123'") + .addRoundTrip("timestamp(3)", "TIMESTAMP '1970-01-01 00:00:01.999'", createTimestampType(3), "TIMESTAMP '1970-01-01 00:00:01.999'") + .addRoundTrip("timestamp(1)", "TIMESTAMP '2020-09-27 12:34:56.1'", createTimestampType(1), "TIMESTAMP '2020-09-27 12:34:56.1'") + .addRoundTrip("timestamp(1)", "TIMESTAMP '2020-09-27 12:34:56.9'", createTimestampType(1), "TIMESTAMP '2020-09-27 12:34:56.9'") + .addRoundTrip("timestamp(3)", "TIMESTAMP '2020-09-27 12:34:56.123'", createTimestampType(3), "TIMESTAMP '2020-09-27 12:34:56.123'") + .addRoundTrip("timestamp(3)", "TIMESTAMP '2020-09-27 12:34:56.999'", createTimestampType(3), "TIMESTAMP '2020-09-27 12:34:56.999'") + .execute(getQueryRunner(), session, snowflakeCreateAndInsert("tpch.test_timestamp")) + .execute(getQueryRunner(), session, trinoCreateAsSelect(session, "test_timestamp")) + .execute(getQueryRunner(), session, trinoCreateAsSelect("test_timestamp")) + .execute(getQueryRunner(), session, trinoCreateAndInsert(session, "test_timestamp")) + .execute(getQueryRunner(), session, trinoCreateAndInsert("test_timestamp")); + } + + private DataSetup trinoCreateAsSelect(String tableNamePrefix) + { + return trinoCreateAsSelect(getSession(), tableNamePrefix); + } + + private DataSetup trinoCreateAsSelect(Session session, String tableNamePrefix) + { + return new CreateAsSelectDataSetup(new TrinoSqlExecutor(getQueryRunner(), session), tableNamePrefix); + } + + private DataSetup trinoCreateAndInsert(String tableNamePrefix) + { + return trinoCreateAndInsert(getSession(), tableNamePrefix); + } + + private DataSetup trinoCreateAndInsert(Session session, String tableNamePrefix) + { + return new CreateAndInsertDataSetup(new TrinoSqlExecutor(getQueryRunner(), session), tableNamePrefix); + } + + private DataSetup snowflakeCreateAndInsert(String tableNamePrefix) + { + return new CreateAndInsertDataSetup(snowflakeServer::execute, tableNamePrefix); + } + + private static void checkIsGap(ZoneId zone, LocalDate date) + { + verify(isGap(zone, date), "Expected %s to be a gap in %s", date, zone); + } + + private static boolean isGap(ZoneId zone, LocalDate date) + { + return zone.getRules().getValidOffsets(date.atStartOfDay()).isEmpty(); + } +} diff --git a/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/TestingSnowflakeServer.java b/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/TestingSnowflakeServer.java new file mode 100644 index 0000000000000..bd64f30307597 --- /dev/null +++ b/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/TestingSnowflakeServer.java @@ -0,0 +1,77 @@ +/* + * 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.snowflake; + +import org.intellij.lang.annotations.Language; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.Properties; + +import static java.util.Objects.requireNonNull; + +public class TestingSnowflakeServer + implements AutoCloseable +{ + public static final String TEST_URL = requireNonNull(System.getProperty("snowflake.test.server.url"), "snowflake.test.server.url is not set"); + public static final String TEST_USER = requireNonNull(System.getProperty("snowflake.test.server.user"), "snowflake.test.server.user is not set"); + public static final String TEST_PASSWORD = requireNonNull(System.getProperty("snowflake.test.server.password"), "snowflake.test.server.password is not set"); + public static final String TEST_DATABASE = requireNonNull(System.getProperty("snowflake.test.server.database"), "snowflake.test.server.database is not set"); + public static final String TEST_WAREHOUSE = requireNonNull(System.getProperty("snowflake.test.server.warehouse"), "snowflake.test.server.warehouse is not set"); + public static final String TEST_ROLE = requireNonNull(System.getProperty("snowflake.test.server.role"), "snowflake.test.server.role is not set"); + public static final String TEST_PROXY = System.getProperty("snowflake.test.http_proxy"); + public static final String TEST_SCHEMA = "tpch"; + + public TestingSnowflakeServer() + { + execute("CREATE SCHEMA IF NOT EXISTS tpch"); + } + + public void execute(@Language("SQL") String sql) + { + execute(TEST_URL, getProperties(), sql); + } + + private static void execute(String url, Properties properties, String sql) + { + try (Connection connection = DriverManager.getConnection(url, properties); + Statement statement = connection.createStatement()) { + statement.execute(sql); + } + catch (SQLException e) { + throw new RuntimeException(e); + } + } + + public Properties getProperties() + { + Properties properties = new Properties(); + properties.setProperty("user", TEST_USER); + properties.setProperty("password", TEST_PASSWORD); + properties.setProperty("db", TEST_DATABASE); + properties.setProperty("schema", TEST_SCHEMA); + properties.setProperty("warehouse", TEST_WAREHOUSE); + properties.setProperty("role", TEST_ROLE); + return properties; + } + + @Override + public void close() + throws Exception + { + execute("DROP SCHEMA IF EXISTS tpch"); + } +} diff --git a/pom.xml b/pom.xml index e0e1574ff03d4..3e1c58b500535 100644 --- a/pom.xml +++ b/pom.xml @@ -105,6 +105,7 @@ plugin/trino-resource-group-managers plugin/trino-session-property-managers plugin/trino-singlestore + plugin/trino-snowflake plugin/trino-sqlserver plugin/trino-teradata-functions plugin/trino-thrift @@ -1399,6 +1400,12 @@ test-jar + + io.trino + trino-snowflake + ${project.version} + + io.trino trino-spi diff --git a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/EnvMultinodeAllConnectors.java b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/EnvMultinodeAllConnectors.java index 12c958581ee87..c44f85b98437a 100644 --- a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/EnvMultinodeAllConnectors.java +++ b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/EnvMultinodeAllConnectors.java @@ -78,6 +78,7 @@ public void extendEnvironment(Environment.Builder builder) "raptor_legacy", "redis", "redshift", + "snowflake", "sqlserver", "trino_thrift", "tpcds") diff --git a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/EnvMultinodeSnowflake.java b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/EnvMultinodeSnowflake.java new file mode 100644 index 0000000000000..7f4ab574084a1 --- /dev/null +++ b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/EnvMultinodeSnowflake.java @@ -0,0 +1,77 @@ +/* + * 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.tests.product.launcher.env.environment; + +import com.google.inject.Inject; +import io.trino.tests.product.launcher.docker.DockerFiles; +import io.trino.tests.product.launcher.env.Environment; +import io.trino.tests.product.launcher.env.EnvironmentProvider; +import io.trino.tests.product.launcher.env.common.Standard; +import io.trino.tests.product.launcher.env.common.TestsEnvironment; + +import java.io.File; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.attribute.PosixFilePermissions; + +import static java.nio.file.attribute.PosixFilePermissions.fromString; +import static java.util.Objects.requireNonNull; +import static org.testcontainers.utility.MountableFile.forHostPath; + +@TestsEnvironment +public class EnvMultinodeSnowflake + extends EnvironmentProvider +{ + private final DockerFiles.ResourceProvider configDir; + + @Inject + public EnvMultinodeSnowflake(DockerFiles dockerFiles, Standard standard) + { + super(standard); + configDir = requireNonNull(dockerFiles, "dockerFiles is null").getDockerFilesHostDirectory("conf/environment/multinode-snowflake"); + } + + @Override + public void extendEnvironment(Environment.Builder builder) + { + builder.addConnector("snowflake", forHostPath(getEnvProperties())); + } + + private Path getEnvProperties() + { + try { + String properties = Files.readString(configDir.getPath("snowflake.properties")) + .replace("${ENV:SNOWFLAKE_URL}", requireEnv("SNOWFLAKE_URL")) + .replace("${ENV:SNOWFLAKE_USER}", requireEnv("SNOWFLAKE_USER")) + .replace("${ENV:SNOWFLAKE_PASSWORD}", requireEnv("SNOWFLAKE_PASSWORD")) + .replace("${ENV:SNOWFLAKE_DATABASE}", requireEnv("SNOWFLAKE_DATABASE")) + .replace("${ENV:SNOWFLAKE_ROLE}", requireEnv("SNOWFLAKE_ROLE")) + .replace("${ENV:SNOWFLAKE_WAREHOUSE}", requireEnv("SNOWFLAKE_WAREHOUSE")); + File newProperties = Files.createTempFile("snowflake-replaced", ".properties", PosixFilePermissions.asFileAttribute(fromString("rwxrwxrwx"))).toFile(); + newProperties.deleteOnExit(); + Files.writeString(newProperties.toPath(), properties); + return newProperties.toPath(); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + private static String requireEnv(String variable) + { + return requireNonNull(System.getenv(variable), () -> "environment variable not set: " + variable); + } +} diff --git a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/suite/suites/SuiteSnowflake.java b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/suite/suites/SuiteSnowflake.java new file mode 100644 index 0000000000000..317d348172365 --- /dev/null +++ b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/suite/suites/SuiteSnowflake.java @@ -0,0 +1,37 @@ +/* + * 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.tests.product.launcher.suite.suites; + +import com.google.common.collect.ImmutableList; +import io.trino.tests.product.launcher.env.EnvironmentConfig; +import io.trino.tests.product.launcher.env.environment.EnvMultinodeSnowflake; +import io.trino.tests.product.launcher.suite.Suite; +import io.trino.tests.product.launcher.suite.SuiteTestRun; + +import java.util.List; + +import static io.trino.tests.product.launcher.suite.SuiteTestRun.testOnEnvironment; + +public class SuiteSnowflake + extends Suite +{ + @Override + public List getTestRuns(EnvironmentConfig config) + { + return ImmutableList.of( + testOnEnvironment(EnvMultinodeSnowflake.class) + .withGroups("configured_features", "snowflake") + .build()); + } +} diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-all/snowflake.properties b/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-all/snowflake.properties new file mode 100644 index 0000000000000..669489ea4363e --- /dev/null +++ b/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-all/snowflake.properties @@ -0,0 +1,4 @@ +connector.name=snowflake +connection-url=${ENV:SNOWFLAKE_URL} +connection-user=${ENV:SNOWFLAKE_USER} +connection-password=${ENV:SNOWFLAKE_PASSWORD} diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-snowflake/snowflake.properties b/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-snowflake/snowflake.properties new file mode 100644 index 0000000000000..669489ea4363e --- /dev/null +++ b/testing/trino-product-tests-launcher/src/main/resources/docker/presto-product-tests/conf/environment/multinode-snowflake/snowflake.properties @@ -0,0 +1,4 @@ +connector.name=snowflake +connection-url=${ENV:SNOWFLAKE_URL} +connection-user=${ENV:SNOWFLAKE_USER} +connection-password=${ENV:SNOWFLAKE_PASSWORD} diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/TestGroups.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/TestGroups.java index c6036a9449c32..980ec71c3addf 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/TestGroups.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/TestGroups.java @@ -81,6 +81,7 @@ public final class TestGroups public static final String CLICKHOUSE = "clickhouse"; public static final String KUDU = "kudu"; public static final String MARIADB = "mariadb"; + public static final String SNOWFLAKE = "snowflake"; public static final String DELTA_LAKE_OSS = "delta-lake-oss"; public static final String DELTA_LAKE_HDFS = "delta-lake-hdfs"; public static final String DELTA_LAKE_MINIO = "delta-lake-minio"; diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/snowflake/TestSnowflake.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/snowflake/TestSnowflake.java new file mode 100644 index 0000000000000..8850ca2165508 --- /dev/null +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/snowflake/TestSnowflake.java @@ -0,0 +1,46 @@ +/* + * 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.tests.product.snowflake; + +import io.trino.tempto.ProductTest; +import io.trino.tempto.query.QueryResult; +import org.testng.annotations.Test; + +import static io.trino.tempto.assertions.QueryAssert.Row.row; +import static io.trino.tempto.assertions.QueryAssert.assertThat; +import static io.trino.testing.TestingNames.randomNameSuffix; +import static io.trino.tests.product.TestGroups.PROFILE_SPECIFIC_TESTS; +import static io.trino.tests.product.TestGroups.SNOWFLAKE; +import static io.trino.tests.product.utils.QueryExecutors.onTrino; + +public class TestSnowflake + extends ProductTest +{ + @Test(groups = {SNOWFLAKE, PROFILE_SPECIFIC_TESTS}) + public void testCreateTableAsSelect() + { + String tableName = "snowflake.tpch.nation_" + randomNameSuffix(); + + onTrino().executeQuery("DROP TABLE IF EXISTS " + tableName); + QueryResult result = onTrino().executeQuery("CREATE TABLE " + tableName + " AS SELECT * FROM tpch.tiny.nation"); + try { + assertThat(result).updatedRowsCountIsEqualTo(25); + assertThat(onTrino().executeQuery("SELECT COUNT(*) FROM " + tableName)) + .containsOnly(row(25)); + } + finally { + onTrino().executeQuery("DROP TABLE " + tableName); + } + } +} From 934555bfe8a18a8562f5aba7ab4772bbfc588cec Mon Sep 17 00:00:00 2001 From: yuuteng Date: Wed, 6 Dec 2023 13:25:14 +0100 Subject: [PATCH 347/350] Update the github CI (#12) * Add Snowflake JDBC Connector * Add snowflake in the ci --- .github/workflows/ci.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 48c77f7745e72..e934d6a0be96e 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -421,6 +421,7 @@ jobs: !:trino-server, !:trino-server-rpm, !:trino-singlestore, + !:trino-snowflake, !:trino-sqlserver, !:trino-test-jdbc-compatibility-old-server, !:trino-tests, @@ -546,6 +547,7 @@ jobs: - { modules: plugin/trino-redshift, profile: fte-tests } - { modules: plugin/trino-resource-group-managers } - { modules: plugin/trino-singlestore } + - { modules: plugin/trino-snowflake } - { modules: plugin/trino-sqlserver } - { modules: testing/trino-faulttolerant-tests, profile: default } - { modules: testing/trino-faulttolerant-tests, profile: test-fault-tolerant-delta } From 89e72452090ee6afc88342116cbee21ffafaf7e1 Mon Sep 17 00:00:00 2001 From: Martin Traverso Date: Thu, 21 Dec 2023 07:26:33 -0800 Subject: [PATCH 348/350] Various style fixes and cleanup (#15) --- .../plugin/snowflake/SnowflakeClient.java | 274 ++++++------------ .../plugin/snowflake/TestSnowflakeClient.java | 2 - 2 files changed, 94 insertions(+), 182 deletions(-) diff --git a/plugin/trino-snowflake/src/main/java/io/trino/plugin/snowflake/SnowflakeClient.java b/plugin/trino-snowflake/src/main/java/io/trino/plugin/snowflake/SnowflakeClient.java index 56d41e6d2464a..d9436766052dd 100644 --- a/plugin/trino-snowflake/src/main/java/io/trino/plugin/snowflake/SnowflakeClient.java +++ b/plugin/trino-snowflake/src/main/java/io/trino/plugin/snowflake/SnowflakeClient.java @@ -59,15 +59,12 @@ import io.trino.spi.type.DecimalType; import io.trino.spi.type.LongTimestamp; import io.trino.spi.type.LongTimestampWithTimeZone; -import io.trino.spi.type.StandardTypes; import io.trino.spi.type.TimeType; import io.trino.spi.type.TimeZoneKey; import io.trino.spi.type.TimestampType; import io.trino.spi.type.TimestampWithTimeZoneType; import io.trino.spi.type.Timestamps; import io.trino.spi.type.Type; -import io.trino.spi.type.TypeManager; -import io.trino.spi.type.TypeSignature; import io.trino.spi.type.VarcharType; import java.math.RoundingMode; @@ -75,7 +72,6 @@ import java.sql.PreparedStatement; import java.sql.ResultSet; import java.sql.SQLException; -import java.sql.Time; import java.sql.Timestamp; import java.sql.Types; import java.time.Instant; @@ -97,11 +93,16 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.util.concurrent.MoreExecutors.directExecutor; -import static io.airlift.slice.Slices.utf8Slice; -import static io.trino.plugin.base.util.JsonTypeUtil.jsonParse; import static io.trino.plugin.jdbc.JdbcErrorCode.JDBC_ERROR; -import static io.trino.plugin.jdbc.PredicatePushdownController.DISABLE_PUSHDOWN; import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; +import static io.trino.spi.type.DecimalType.createDecimalType; +import static io.trino.spi.type.TimestampWithTimeZoneType.createTimestampWithTimeZoneType; +import static io.trino.spi.type.Timestamps.MILLISECONDS_PER_SECOND; +import static io.trino.spi.type.Timestamps.NANOSECONDS_PER_MILLISECOND; +import static io.trino.spi.type.Timestamps.PICOSECONDS_PER_NANOSECOND; +import static io.trino.spi.type.VarcharType.createUnboundedVarcharType; +import static io.trino.spi.type.VarcharType.createVarcharType; +import static java.lang.String.format; public class SnowflakeClient extends BaseJdbcClient @@ -115,8 +116,7 @@ public class SnowflakeClient private static final DateTimeFormatter SNOWFLAKE_DATE_FORMATTER = DateTimeFormatter.ofPattern("uuuu-MM-dd"); private static final DateTimeFormatter SNOWFLAKE_TIMESTAMP_FORMATTER = DateTimeFormatter.ofPattern("y-MM-dd'T'HH:mm:ss.SSSSSSSSS"); private static final DateTimeFormatter SNOWFLAKE_TIME_FORMATTER = DateTimeFormatter.ofPattern("HH:mm:ss.SSSSSSSSS"); - private final Type jsonType; - private final AggregateFunctionRewriter aggregateFunctionRewriter; + private final AggregateFunctionRewriter aggregateFunctionRewriter; private interface WriteMappingFunction { @@ -145,47 +145,30 @@ private interface ColumnMappingFunction .buildOrThrow(); private static final Map SHOWFLAKE_COLUMN_MAPPINGS = ImmutableMap.builder() - .put("time", typeHandle -> { - //return Optional.of(columnMappingPushdown(timeColumnMapping(typeHandle))); - return Optional.of(timeColumnMapping(typeHandle)); - }) - .put("timestampntz", typeHandle -> { - return Optional.of(timestampColumnMapping(typeHandle)); - }) - .put("timestamptz", typeHandle -> { - return Optional.of(timestampTZColumnMapping(typeHandle)); - }) - .put("timestampltz", typeHandle -> { - return Optional.of(timestampTZColumnMapping(typeHandle)); - }) - .put("date", typeHandle -> { - return Optional.of(ColumnMapping.longMapping( - DateType.DATE, (resultSet, columnIndex) -> - LocalDate.ofEpochDay(resultSet.getLong(columnIndex)).toEpochDay(), - snowFlakeDateWriter())); - }) - .put("object", typeHandle -> { - return Optional.of(ColumnMapping.sliceMapping( - VarcharType.createUnboundedVarcharType(), - StandardColumnMappings.varcharReadFunction(VarcharType.createUnboundedVarcharType()), - StandardColumnMappings.varcharWriteFunction(), - PredicatePushdownController.DISABLE_PUSHDOWN)); - }) - .put("array", typeHandle -> { - return Optional.of(ColumnMapping.sliceMapping( - VarcharType.createUnboundedVarcharType(), - StandardColumnMappings.varcharReadFunction(VarcharType.createUnboundedVarcharType()), - StandardColumnMappings.varcharWriteFunction(), - PredicatePushdownController.DISABLE_PUSHDOWN)); - }) - .put("variant", typeHandle -> { - return Optional.of(ColumnMapping.sliceMapping( - VarcharType.createUnboundedVarcharType(), variantReadFunction(), StandardColumnMappings.varcharWriteFunction(), - PredicatePushdownController.FULL_PUSHDOWN)); - }) - .put("varchar", typeHandle -> { - return Optional.of(varcharColumnMapping(typeHandle.getRequiredColumnSize())); - }) + .put("time", typeHandle -> Optional.of(timeColumnMapping(typeHandle))) + .put("timestampntz", typeHandle -> Optional.of(timestampColumnMapping(typeHandle))) + .put("timestamptz", typeHandle -> Optional.of(timestampTzColumnMapping(typeHandle))) + .put("timestampltz", typeHandle -> Optional.of(timestampTzColumnMapping(typeHandle))) + .put("date", typeHandle -> Optional.of(ColumnMapping.longMapping( + DateType.DATE, + (resultSet, columnIndex) -> LocalDate.ofEpochDay(resultSet.getLong(columnIndex)).toEpochDay(), + snowFlakeDateWriter()))) + .put("object", typeHandle -> Optional.of(ColumnMapping.sliceMapping( + createUnboundedVarcharType(), + StandardColumnMappings.varcharReadFunction(createUnboundedVarcharType()), + StandardColumnMappings.varcharWriteFunction(), + PredicatePushdownController.DISABLE_PUSHDOWN))) + .put("array", typeHandle -> Optional.of(ColumnMapping.sliceMapping( + createUnboundedVarcharType(), + StandardColumnMappings.varcharReadFunction(createUnboundedVarcharType()), + StandardColumnMappings.varcharWriteFunction(), + PredicatePushdownController.DISABLE_PUSHDOWN))) + .put("variant", typeHandle -> Optional.of(ColumnMapping.sliceMapping( + createUnboundedVarcharType(), + variantReadFunction(), + StandardColumnMappings.varcharWriteFunction(), + PredicatePushdownController.FULL_PUSHDOWN))) + .put("varchar", typeHandle -> Optional.of(varcharColumnMapping(typeHandle.getRequiredColumnSize()))) .put("number", typeHandle -> { int decimalDigits = typeHandle.getRequiredDecimalDigits(); int precision = typeHandle.getRequiredColumnSize() + Math.max(-decimalDigits, 0); @@ -193,8 +176,7 @@ private interface ColumnMappingFunction return Optional.empty(); } return Optional.of(columnMappingPushdown( - StandardColumnMappings.decimalColumnMapping(DecimalType.createDecimalType( - precision, Math.max(decimalDigits, 0)), RoundingMode.UNNECESSARY))); + StandardColumnMappings.decimalColumnMapping(createDecimalType(precision, Math.max(decimalDigits, 0)), RoundingMode.UNNECESSARY))); }) .buildOrThrow(); @@ -212,50 +194,26 @@ private interface ColumnMappingFunction .buildOrThrow(); private static final Map SNOWFLAKE_WRITE_MAPPINGS = ImmutableMap.builder() - .put("TimeType", type -> { - return WriteMapping.longMapping("time", SnowflakeClient.snowFlaketimeWriter(type)); - }) - .put("ShortTimestampType", type -> { - WriteMapping myMap = SnowflakeClient.snowFlakeTimestampWriter(type); - return myMap; - }) - .put("ShortTimestampWithTimeZoneType", type -> { - WriteMapping myMap = SnowflakeClient.snowFlakeTimestampWithTZWriter(type); - return myMap; - }) - .put("LongTimestampType", type -> { - WriteMapping myMap = SnowflakeClient.snowFlakeTimestampWithTZWriter(type); - return myMap; - }) - .put("LongTimestampWithTimeZoneType", type -> { - WriteMapping myMap = SnowflakeClient.snowFlakeTimestampWithTZWriter(type); - return myMap; - }) - .put("VarcharType", type -> { - WriteMapping myMap = SnowflakeClient.snowFlakeVarCharWriter(type); - return myMap; - }) - .put("CharType", type -> { - WriteMapping myMap = SnowflakeClient.snowFlakeCharWriter(type); - return myMap; - }) - .put("LongDecimalType", type -> { - WriteMapping myMap = SnowflakeClient.snowFlakeDecimalWriter(type); - return myMap; - }) - .put("ShortDecimalType", type -> { - WriteMapping myMap = SnowflakeClient.snowFlakeDecimalWriter(type); - return myMap; - }) + .put("TimeType", type -> WriteMapping.longMapping("time", SnowflakeClient.snowFlaketimeWriter(type))) + .put("ShortTimestampType", SnowflakeClient::snowFlakeTimestampWriter) + .put("ShortTimestampWithTimeZoneType", SnowflakeClient::snowFlakeTimestampWithTZWriter) + .put("LongTimestampType", SnowflakeClient::snowFlakeTimestampWithTZWriter) + .put("LongTimestampWithTimeZoneType", SnowflakeClient::snowFlakeTimestampWithTZWriter) + .put("VarcharType", SnowflakeClient::snowFlakeVarCharWriter) + .put("CharType", SnowflakeClient::snowFlakeCharWriter) + .put("LongDecimalType", SnowflakeClient::snowFlakeDecimalWriter) + .put("ShortDecimalType", SnowflakeClient::snowFlakeDecimalWriter) .buildOrThrow(); @Inject - public SnowflakeClient(BaseJdbcConfig config, ConnectionFactory connectionFactory, QueryBuilder queryBuilder, - TypeManager typeManager, IdentifierMapping identifierMapping, - RemoteQueryModifier remoteQueryModifier) + public SnowflakeClient( + BaseJdbcConfig config, + ConnectionFactory connectionFactory, + QueryBuilder queryBuilder, + IdentifierMapping identifierMapping, + RemoteQueryModifier remoteQueryModifier) { super("\"", connectionFactory, queryBuilder, config.getJdbcTypesMappedToVarchar(), identifierMapping, remoteQueryModifier, false); - this.jsonType = typeManager.getType(new TypeSignature(StandardTypes.JSON)); JdbcTypeHandle bigintTypeHandle = new JdbcTypeHandle(Types.BIGINT, Optional.of("bigint"), Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty()); ConnectorExpressionRewriter connectorExpressionRewriter = JdbcConnectorExpressionRewriterBuilder.newBuilder() @@ -302,14 +260,13 @@ public Optional toColumnMapping(ConnectorSession session, Connect } // Code should never reach here so throw an error. - throw new TrinoException(NOT_SUPPORTED, "SNOWFLAKE_CONNECTOR_COLUMN_TYPE_NOT_SUPPORTED: Unsupported column type(" + type + - "):" + jdbcTypeName); + throw new TrinoException(NOT_SUPPORTED, "SNOWFLAKE_CONNECTOR_COLUMN_TYPE_NOT_SUPPORTED: Unsupported column type(" + type + "):" + jdbcTypeName); } @Override public WriteMapping toWriteMapping(ConnectorSession session, Type type) { - Class myClass = type.getClass(); + Class myClass = type.getClass(); String simple = myClass.getSimpleName(); WriteMapping writeMapping = STANDARD_WRITE_MAPPINGS.get(simple); @@ -351,15 +308,6 @@ public boolean isLimitGuaranteed(ConnectorSession session) return true; } - private ColumnMapping jsonColumnMapping() - { - return ColumnMapping.sliceMapping( - jsonType, - (resultSet, columnIndex) -> jsonParse(utf8Slice(resultSet.getString(columnIndex))), - StandardColumnMappings.varcharWriteFunction(), - DISABLE_PUSHDOWN); - } - @Override public void setColumnType(ConnectorSession session, JdbcTableHandle handle, JdbcColumnHandle column, Type type) { @@ -378,22 +326,18 @@ private static ColumnMapping columnMappingPushdown(ColumnMapping mapping) throw new TrinoException(NOT_SUPPORTED, "mapping.getPredicatePushdownController() is DISABLE_PUSHDOWN. Type was " + mapping.getType()); } - return new ColumnMapping(mapping.getType(), mapping.getReadFunction(), mapping.getWriteFunction(), - PredicatePushdownController.FULL_PUSHDOWN); + return new ColumnMapping(mapping.getType(), mapping.getReadFunction(), mapping.getWriteFunction(), PredicatePushdownController.FULL_PUSHDOWN); } private static ColumnMapping timeColumnMapping(JdbcTypeHandle typeHandle) { int precision = typeHandle.getRequiredDecimalDigits(); - checkArgument((precision <= SNOWFLAKE_MAX_SUPPORTED_TIMESTAMP_PRECISION), - "The max timestamp precision in Snowflake is " + SNOWFLAKE_MAX_SUPPORTED_TIMESTAMP_PRECISION); + checkArgument(precision <= SNOWFLAKE_MAX_SUPPORTED_TIMESTAMP_PRECISION, "The max timestamp precision in Snowflake is " + SNOWFLAKE_MAX_SUPPORTED_TIMESTAMP_PRECISION); return ColumnMapping.longMapping( TimeType.createTimeType(precision), (resultSet, columnIndex) -> { LocalTime time = SNOWFLAKE_TIME_FORMATTER.parse(resultSet.getString(columnIndex), LocalTime::from); - long nanosOfDay = time.toNanoOfDay(); - long picosOfDay = nanosOfDay * Timestamps.PICOSECONDS_PER_NANOSECOND; - return Timestamps.round(picosOfDay, 12 - precision); + return Timestamps.round(time.toNanoOfDay() * PICOSECONDS_PER_NANOSECOND, 12 - precision); }, timeWriteFunction(precision), PredicatePushdownController.FULL_PUSHDOWN); @@ -401,15 +345,13 @@ private static ColumnMapping timeColumnMapping(JdbcTypeHandle typeHandle) private static LongWriteFunction snowFlaketimeWriter(Type type) { - TimeType timeType = (TimeType) type; - int precision = timeType.getPrecision(); - return timeWriteFunction(precision); + return timeWriteFunction(((TimeType) type).getPrecision()); } private static LongWriteFunction timeWriteFunction(int precision) { checkArgument(precision <= SNOWFLAKE_MAX_SUPPORTED_TIMESTAMP_PRECISION, "Unsupported precision: %s", precision); - String bindExpression = String.format("CAST(? AS time(%s))", precision); + String bindExpression = format("CAST(? AS time(%s))", precision); return new LongWriteFunction() { @Override @@ -426,19 +368,14 @@ public void set(PreparedStatement statement, int index, long picosOfDay) if (picosOfDay == Timestamps.PICOSECONDS_PER_DAY) { picosOfDay = 0; } - LocalTime localTime = LocalTime.ofNanoOfDay(picosOfDay / Timestamps.PICOSECONDS_PER_NANOSECOND); + LocalTime localTime = LocalTime.ofNanoOfDay(picosOfDay / PICOSECONDS_PER_NANOSECOND); // statement.setObject(.., localTime) would yield incorrect end result for 23:59:59.999000 statement.setString(index, SNOWFLAKE_TIME_FORMATTER.format(localTime)); } }; } - private static long toTrinoTime(Time sqlTime) - { - return Timestamps.PICOSECONDS_PER_SECOND * sqlTime.getTime(); - } - - private static ColumnMapping timestampTZColumnMapping(JdbcTypeHandle typeHandle) + private static ColumnMapping timestampTzColumnMapping(JdbcTypeHandle typeHandle) { int precision = typeHandle.getRequiredDecimalDigits(); String jdbcTypeName = typeHandle.getJdbcTypeName() @@ -447,23 +384,23 @@ private static ColumnMapping timestampTZColumnMapping(JdbcTypeHandle typeHandle) log.debug("timestampTZColumnMapping: jdbcTypeName(%s):%s precision:%s", type, jdbcTypeName, precision); if (precision <= 3) { - return ColumnMapping.longMapping(TimestampWithTimeZoneType.createTimestampWithTimeZoneType(precision), + return ColumnMapping.longMapping( + createTimestampWithTimeZoneType(precision), (resultSet, columnIndex) -> { - ZonedDateTime timestamp = (ZonedDateTime) SNOWFLAKE_DATETIME_FORMATTER.parse(resultSet.getString(columnIndex), ZonedDateTime::from); + ZonedDateTime timestamp = SNOWFLAKE_DATETIME_FORMATTER.parse(resultSet.getString(columnIndex), ZonedDateTime::from); return DateTimeEncoding.packDateTimeWithZone(timestamp.toInstant().toEpochMilli(), timestamp.getZone().getId()); }, - timestampWithTZWriter(), PredicatePushdownController.FULL_PUSHDOWN); + timestampWithTZWriter(), + PredicatePushdownController.FULL_PUSHDOWN); } else { - return ColumnMapping.objectMapping(TimestampWithTimeZoneType.createTimestampWithTimeZoneType(precision), longTimestampWithTimezoneReadFunction(), longTimestampWithTZWriteFunction()); + return ColumnMapping.objectMapping(createTimestampWithTimeZoneType(precision), longTimestampWithTimezoneReadFunction(), longTimestampWithTzWriteFunction()); } } private static ColumnMapping varcharColumnMapping(int varcharLength) { - VarcharType varcharType = varcharLength <= VarcharType.MAX_LENGTH - ? VarcharType.createVarcharType(varcharLength) - : VarcharType.createUnboundedVarcharType(); + VarcharType varcharType = varcharLength <= VarcharType.MAX_LENGTH ? createVarcharType(varcharLength) : createUnboundedVarcharType(); return ColumnMapping.sliceMapping( varcharType, StandardColumnMappings.varcharReadFunction(varcharType), @@ -473,41 +410,30 @@ private static ColumnMapping varcharColumnMapping(int varcharLength) private static ObjectReadFunction longTimestampWithTimezoneReadFunction() { return ObjectReadFunction.of(LongTimestampWithTimeZone.class, (resultSet, columnIndex) -> { - ZonedDateTime timestamp = (ZonedDateTime) SNOWFLAKE_DATETIME_FORMATTER.parse(resultSet.getString(columnIndex), ZonedDateTime::from); - return LongTimestampWithTimeZone.fromEpochSecondsAndFraction(timestamp.toEpochSecond(), - (long) timestamp.getNano() * Timestamps.PICOSECONDS_PER_NANOSECOND, + ZonedDateTime timestamp = SNOWFLAKE_DATETIME_FORMATTER.parse(resultSet.getString(columnIndex), ZonedDateTime::from); + return LongTimestampWithTimeZone.fromEpochSecondsAndFraction( + timestamp.toEpochSecond(), + (long) timestamp.getNano() * PICOSECONDS_PER_NANOSECOND, TimeZoneKey.getTimeZoneKey(timestamp.getZone().getId())); }); } - private static ObjectWriteFunction longTimestampWithTZWriteFunction() + private static ObjectWriteFunction longTimestampWithTzWriteFunction() { return ObjectWriteFunction.of(LongTimestampWithTimeZone.class, (statement, index, value) -> { - long epoMilli = value.getEpochMillis(); - long epoSeconds = Math.floorDiv(epoMilli, Timestamps.MILLISECONDS_PER_SECOND); - long adjNano = Math.floorMod(epoMilli, Timestamps.MILLISECONDS_PER_SECOND) * - Timestamps.NANOSECONDS_PER_MILLISECOND + value.getPicosOfMilli() / Timestamps.PICOSECONDS_PER_NANOSECOND; + long epochMilli = value.getEpochMillis(); + long epochSecond = Math.floorDiv(epochMilli, MILLISECONDS_PER_SECOND); + int nanosOfSecond = Math.floorMod(epochMilli, MILLISECONDS_PER_SECOND) * NANOSECONDS_PER_MILLISECOND + value.getPicosOfMilli() / PICOSECONDS_PER_NANOSECOND; ZoneId zone = TimeZoneKey.getTimeZoneKey(value.getTimeZoneKey()).getZoneId(); - Instant timeI = Instant.ofEpochSecond(epoSeconds, adjNano); - statement.setString(index, SNOWFLAKE_DATETIME_FORMATTER.format(ZonedDateTime.ofInstant(timeI, zone))); + Instant instant = Instant.ofEpochSecond(epochSecond, nanosOfSecond); + statement.setString(index, SNOWFLAKE_DATETIME_FORMATTER.format(ZonedDateTime.ofInstant(instant, zone))); }); } - private static LongWriteFunction snowFlakeDateTimeWriter() - { - return (statement, index, encodedTimeWithZone) -> { - Instant time = Instant.ofEpochMilli(DateTimeEncoding.unpackMillisUtc(encodedTimeWithZone)); - ZoneId zone = ZoneId.of(DateTimeEncoding.unpackZoneKey(encodedTimeWithZone).getId()); - statement.setString(index, SNOWFLAKE_DATETIME_FORMATTER.format(time.atZone(zone))); - }; - } - private static WriteMapping snowFlakeDecimalWriter(Type type) { DecimalType decimalType = (DecimalType) type; - String dataType = String.format("decimal(%s, %s)", new Object[] { - Integer.valueOf(decimalType.getPrecision()), Integer.valueOf(decimalType.getScale()) - }); + String dataType = format("decimal(%s, %s)", decimalType.getPrecision(), decimalType.getScale()); if (decimalType.isShort()) { return WriteMapping.longMapping(dataType, StandardColumnMappings.shortDecimalWriteFunction(decimalType)); @@ -523,8 +449,7 @@ private static LongWriteFunction snowFlakeDateWriter() private static WriteMapping snowFlakeCharWriter(Type type) { CharType charType = (CharType) type; - return WriteMapping.sliceMapping("char(" + charType.getLength() + ")", - charWriteFunction(charType)); + return WriteMapping.sliceMapping("char(" + charType.getLength() + ")", charWriteFunction(charType)); } private static WriteMapping snowFlakeVarCharWriter(Type type) @@ -549,55 +474,45 @@ private static SliceWriteFunction charWriteFunction(CharType charType) private static WriteMapping snowFlakeTimestampWriter(Type type) { TimestampType timestampType = (TimestampType) type; - checkArgument((timestampType.getPrecision() <= SNOWFLAKE_MAX_SUPPORTED_TIMESTAMP_PRECISION), + checkArgument( + timestampType.getPrecision() <= SNOWFLAKE_MAX_SUPPORTED_TIMESTAMP_PRECISION, "The max timestamp precision in Snowflake is " + SNOWFLAKE_MAX_SUPPORTED_TIMESTAMP_PRECISION); if (timestampType.isShort()) { - return WriteMapping.longMapping( - String.format("timestamp_ntz(%d)", new Object[] {Integer.valueOf(timestampType.getPrecision()) }), - timestampWriteFunction()); + return WriteMapping.longMapping(format("timestamp_ntz(%d)", timestampType.getPrecision()), timestampWriteFunction()); } - return WriteMapping.objectMapping( - String.format("timestamp_ntz(%d)", new Object[] {Integer.valueOf(timestampType.getPrecision()) }), - longTimestampWriter(timestampType.getPrecision())); + return WriteMapping.objectMapping(format("timestamp_ntz(%d)", timestampType.getPrecision()), longTimestampWriter(timestampType.getPrecision())); } private static LongWriteFunction timestampWriteFunction() { - return (statement, index, value) -> statement.setString(index, - StandardColumnMappings.fromTrinoTimestamp(value).toString()); + return (statement, index, value) -> statement.setString(index, StandardColumnMappings.fromTrinoTimestamp(value).toString()); } private static ObjectWriteFunction longTimestampWriter(int precision) { - return ObjectWriteFunction.of(LongTimestamp.class, - (statement, index, value) -> statement.setString(index, - SNOWFLAKE_TIMESTAMP_FORMATTER.format(StandardColumnMappings.fromLongTrinoTimestamp(value, - precision)))); + return ObjectWriteFunction.of( + LongTimestamp.class, + (statement, index, value) -> statement.setString(index, SNOWFLAKE_TIMESTAMP_FORMATTER.format(StandardColumnMappings.fromLongTrinoTimestamp(value, precision)))); } private static WriteMapping snowFlakeTimestampWithTZWriter(Type type) { TimestampWithTimeZoneType timeTZType = (TimestampWithTimeZoneType) type; - checkArgument((timeTZType.getPrecision() <= SNOWFLAKE_MAX_SUPPORTED_TIMESTAMP_PRECISION), - "Max Snowflake precision is is " + SNOWFLAKE_MAX_SUPPORTED_TIMESTAMP_PRECISION); + checkArgument(timeTZType.getPrecision() <= SNOWFLAKE_MAX_SUPPORTED_TIMESTAMP_PRECISION, "Max Snowflake precision is is " + SNOWFLAKE_MAX_SUPPORTED_TIMESTAMP_PRECISION); if (timeTZType.isShort()) { - return WriteMapping.longMapping(String.format("timestamp_tz(%d)", - new Object[] {Integer.valueOf(timeTZType.getPrecision()) }), - timestampWithTZWriter()); + return WriteMapping.longMapping(format("timestamp_tz(%d)", timeTZType.getPrecision()), timestampWithTZWriter()); } - return WriteMapping.objectMapping( - String.format("timestamp_tz(%d)", new Object[] {Integer.valueOf(timeTZType.getPrecision()) }), - longTimestampWithTZWriteFunction()); + return WriteMapping.objectMapping(format("timestamp_tz(%d)", timeTZType.getPrecision()), longTimestampWithTzWriteFunction()); } private static LongWriteFunction timestampWithTZWriter() { return (statement, index, encodedTimeWithZone) -> { - Instant timeI = Instant.ofEpochMilli(DateTimeEncoding.unpackMillisUtc(encodedTimeWithZone)); + Instant instant = Instant.ofEpochMilli(DateTimeEncoding.unpackMillisUtc(encodedTimeWithZone)); ZoneId zone = ZoneId.of(DateTimeEncoding.unpackZoneKey(encodedTimeWithZone).getId()); - statement.setString(index, SNOWFLAKE_DATETIME_FORMATTER.format(timeI.atZone(zone))); + statement.setString(index, SNOWFLAKE_DATETIME_FORMATTER.format(instant.atZone(zone))); }; } @@ -609,7 +524,7 @@ private static ObjectReadFunction longTimestampReader() Timestamp ts = resultSet.getTimestamp(columnIndex, calendar); long epochMillis = ts.getTime(); int nanosInTheSecond = ts.getNanos(); - int nanosInTheMilli = nanosInTheSecond % Timestamps.NANOSECONDS_PER_MILLISECOND; + int nanosInTheMilli = nanosInTheSecond % NANOSECONDS_PER_MILLISECOND; long micro = epochMillis * Timestamps.MICROSECONDS_PER_MILLISECOND + (nanosInTheMilli / Timestamps.NANOSECONDS_PER_MICROSECOND); int picosOfMicro = nanosInTheMilli % 1000 * 1000; return new LongTimestamp(micro, picosOfMicro); @@ -627,15 +542,14 @@ private static ColumnMapping timestampColumnMapping(JdbcTypeHandle typeHandle) // <= 6 fits into a long if (precision <= 6) { return ColumnMapping.longMapping( - (Type) TimestampType.createTimestampType(precision), (resultSet, columnIndex) -> - StandardColumnMappings.toTrinoTimestamp(TimestampType.createTimestampType(precision), - toLocalDateTime(resultSet, columnIndex)), + TimestampType.createTimestampType(precision), + (resultSet, columnIndex) -> StandardColumnMappings.toTrinoTimestamp(TimestampType.createTimestampType(precision), toLocalDateTime(resultSet, columnIndex)), timestampWriteFunction()); } // Too big. Put it in an object return ColumnMapping.objectMapping( - (Type) TimestampType.createTimestampType(precision), + TimestampType.createTimestampType(precision), longTimestampReader(), longTimestampWriter(precision)); } diff --git a/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/TestSnowflakeClient.java b/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/TestSnowflakeClient.java index b743314af7633..3e92cfcb6f6ce 100644 --- a/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/TestSnowflakeClient.java +++ b/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/TestSnowflakeClient.java @@ -37,7 +37,6 @@ import static io.trino.spi.type.BooleanType.BOOLEAN; import static io.trino.spi.type.DoubleType.DOUBLE; import static io.trino.testing.TestingConnectorSession.SESSION; -import static io.trino.type.InternalTypeManager.TESTING_TYPE_MANAGER; import static org.assertj.core.api.Assertions.assertThat; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertTrue; @@ -62,7 +61,6 @@ public class TestSnowflakeClient new BaseJdbcConfig(), session -> { throw new UnsupportedOperationException(); }, new DefaultQueryBuilder(RemoteQueryModifier.NONE), - TESTING_TYPE_MANAGER, new DefaultIdentifierMapping(), RemoteQueryModifier.NONE); From 69d4ab23c235cd65c8640f5cd1eee76dc8a1fe11 Mon Sep 17 00:00:00 2001 From: Teng Date: Wed, 10 Jan 2024 20:22:58 +0100 Subject: [PATCH 349/350] Various style fixes and cleanup (#15) (#17) Co-authored-by: Martin Traverso --- .github/workflows/ci.yml | 20 ++++ plugin/trino-snowflake/pom.xml | 28 ++--- .../plugin/snowflake/SnowflakeClient.java | 107 ++++++++++++++++-- 3 files changed, 129 insertions(+), 26 deletions(-) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index e934d6a0be96e..71fcf9249f7e3 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -548,6 +548,7 @@ jobs: - { modules: plugin/trino-resource-group-managers } - { modules: plugin/trino-singlestore } - { modules: plugin/trino-snowflake } + - { modules: plugin/trino-snowflake, profile: cloud-tests } - { modules: plugin/trino-sqlserver } - { modules: testing/trino-faulttolerant-tests, profile: default } - { modules: testing/trino-faulttolerant-tests, profile: test-fault-tolerant-delta } @@ -715,6 +716,24 @@ jobs: if: matrix.modules == 'plugin/trino-bigquery' && !contains(matrix.profile, 'cloud-tests-2') && (env.CI_SKIP_SECRETS_PRESENCE_CHECKS != '' || env.BIGQUERY_CASE_INSENSITIVE_CREDENTIALS_KEY != '') run: | $MAVEN test ${MAVEN_TEST} -pl :trino-bigquery -Pcloud-tests-case-insensitive-mapping -Dbigquery.credentials-key="${BIGQUERY_CASE_INSENSITIVE_CREDENTIALS_KEY}" + - name: Cloud Snowflake Tests + env: + SNOWFLAKE_TEST_SERVER_URL: ${{ secrets.SNOWFLAKE_TEST_SERVER_URL }} + SNOWFLAKE_TEST_SERVER_USER: ${{ secrets.SNOWFLAKE_TEST_SERVER_USER }} + SNOWFLAKE_TEST_SERVER_PASSWORD: ${{ secrets.SNOWFLAKE_TEST_SERVER_PASSWORD }} + SNOWFLAKE_TEST_SERVER_DATABASE: ${{ secrets.SNOWFLAKE_TEST_SERVER_DATABASE }} + SNOWFLAKE_TEST_SERVER_ROLE: ${{ secrets.SNOWFLAKE_TEST_SERVER_ROLE }} + SNOWFLAKE_TEST_SERVER_WAREHOUSE: ${{ secrets.SNOWFLAKE_TEST_SERVER_WAREHOUSE }} + if: matrix.modules == 'plugin/trino-snowflake' && !contains(matrix.profile, 'cloud-tests') && (env.SNOWFLAKE_TEST_SERVER_URL != '' && env.SNOWFLAKE_TEST_SERVER_USER != '' && env.SNOWFLAKE_TEST_SERVER_PASSWORD != '') + run: | + $MAVEN test ${MAVEN_TEST} -pl :trino-snowflake -Pcloud-tests \ + -Dconnector.name="snowflake" \ + -Dsnowflake.test.server.url="${SNOWFLAKE_TEST_SERVER_URL}" \ + -Dsnowflake.test.server.user="${SNOWFLAKE_TEST_SERVER_USER}" \ + -Dsnowflake.test.server.password="${SNOWFLAKE_TEST_SERVER_PASSWORD}" \ + -Dsnowflake.test.server.database="${SNOWFLAKE_TEST_SERVER_DATABASE}" \ + -Dsnowflake.test.server.role="${SNOWFLAKE_TEST_SERVER_ROLE}" \ + -Dsnowflake.test.server.warehouse="${SNOWFLAKE_TEST_SERVER_WAREHOUSE}" - name: Iceberg Cloud Tests env: AWS_ACCESS_KEY_ID: ${{ secrets.TRINO_AWS_ACCESS_KEY_ID }} @@ -910,6 +929,7 @@ jobs: - suite-clickhouse - suite-mysql - suite-iceberg + - suite-snowflake - suite-hudi - suite-ignite exclude: diff --git a/plugin/trino-snowflake/pom.xml b/plugin/trino-snowflake/pom.xml index 2a61433fe13b5..33f14d2a93990 100644 --- a/plugin/trino-snowflake/pom.xml +++ b/plugin/trino-snowflake/pom.xml @@ -15,6 +15,7 @@ ${project.parent.basedir} + --add-opens=java.base/java.nio=ALL-UNNAMED @@ -185,18 +186,6 @@ - - - - org.apache.maven.plugins - maven-surefire-plugin - - --add-opens=java.base/java.nio=ALL-UNNAMED - - - - - default @@ -210,8 +199,6 @@ maven-surefire-plugin - **/TestSnowflakeClient.java - **/TestSnowflakeConfig.java **/TestSnowflakeConnectorTest.java **/TestSnowflakePlugin.java **/TestSnowflakeTypeMapping.java @@ -225,6 +212,9 @@ cloud-tests + + false + @@ -232,11 +222,11 @@ maven-surefire-plugin - **/TestSnowflakeClient.java - **/TestSnowflakeConfig.java - **/TestSnowflakeConnectorTest.java - **/TestSnowflakePlugin.java - **/TestSnowflakeTypeMapping.java + **/TestSnowflakeClient.java + **/TestSnowflakeConfig.java + **/TestSnowflakeConnectorTest.java + **/TestSnowflakePlugin.java + **/TestSnowflakeTypeMapping.java diff --git a/plugin/trino-snowflake/src/main/java/io/trino/plugin/snowflake/SnowflakeClient.java b/plugin/trino-snowflake/src/main/java/io/trino/plugin/snowflake/SnowflakeClient.java index d9436766052dd..c68e786e2efeb 100644 --- a/plugin/trino-snowflake/src/main/java/io/trino/plugin/snowflake/SnowflakeClient.java +++ b/plugin/trino-snowflake/src/main/java/io/trino/plugin/snowflake/SnowflakeClient.java @@ -244,17 +244,62 @@ public void abortReadConnection(Connection connection, ResultSet resultSet) @Override public Optional toColumnMapping(ConnectorSession session, Connection connection, JdbcTypeHandle typeHandle) { + Optional mapping = getForcedMappingToVarchar(typeHandle); + if (mapping.isPresent()) { + return mapping; + } + String jdbcTypeName = typeHandle.getJdbcTypeName() .orElseThrow(() -> new TrinoException(JDBC_ERROR, "Type name is missing: " + typeHandle)); jdbcTypeName = jdbcTypeName.toLowerCase(Locale.ENGLISH); int type = typeHandle.getJdbcType(); - ColumnMapping columnMap = STANDARD_COLUMN_MAPPINGS.get(type); + // Mappings for JDBC column types to internal Trino types + final Map standardColumnMappings = ImmutableMap.builder() + .put(Types.BOOLEAN, StandardColumnMappings.booleanColumnMapping()) + .put(Types.TINYINT, StandardColumnMappings.tinyintColumnMapping()) + .put(Types.SMALLINT, StandardColumnMappings.smallintColumnMapping()) + .put(Types.INTEGER, StandardColumnMappings.integerColumnMapping()) + .put(Types.BIGINT, StandardColumnMappings.bigintColumnMapping()) + .put(Types.REAL, StandardColumnMappings.realColumnMapping()) + .put(Types.DOUBLE, StandardColumnMappings.doubleColumnMapping()) + .put(Types.FLOAT, StandardColumnMappings.doubleColumnMapping()) + .put(Types.BINARY, StandardColumnMappings.varbinaryColumnMapping()) + .put(Types.VARBINARY, StandardColumnMappings.varbinaryColumnMapping()) + .put(Types.LONGVARBINARY, StandardColumnMappings.varbinaryColumnMapping()) + .buildOrThrow(); + + ColumnMapping columnMap = standardColumnMappings.get(type); if (columnMap != null) { return Optional.of(columnMap); } - ColumnMappingFunction columnMappingFunction = SHOWFLAKE_COLUMN_MAPPINGS.get(jdbcTypeName); + final Map snowflakeColumnMappings = ImmutableMap.builder() + .put("time", handle -> { + return Optional.of(timeColumnMapping(handle)); + }) + .put("date", handle -> { + return Optional.of(ColumnMapping.longMapping( + DateType.DATE, (resultSet, columnIndex) -> + LocalDate.ofEpochDay(resultSet.getLong(columnIndex)).toEpochDay(), + snowFlakeDateWriter())); + }) + .put("varchar", handle -> { + return Optional.of(varcharColumnMapping(handle.getRequiredColumnSize())); + }) + .put("number", handle -> { + int decimalDigits = handle.getRequiredDecimalDigits(); + int precision = handle.getRequiredColumnSize() + Math.max(-decimalDigits, 0); + if (precision > 38) { + return Optional.empty(); + } + return Optional.of(columnMappingPushdown( + StandardColumnMappings.decimalColumnMapping(DecimalType.createDecimalType( + precision, Math.max(decimalDigits, 0)), RoundingMode.UNNECESSARY))); + }) + .buildOrThrow(); + + ColumnMappingFunction columnMappingFunction = snowflakeColumnMappings.get(jdbcTypeName); if (columnMappingFunction != null) { return columnMappingFunction.convert(typeHandle); } @@ -269,18 +314,67 @@ public WriteMapping toWriteMapping(ConnectorSession session, Type type) Class myClass = type.getClass(); String simple = myClass.getSimpleName(); - WriteMapping writeMapping = STANDARD_WRITE_MAPPINGS.get(simple); + // Mappings for internal Trino types to JDBC column types + final Map standardWriteMappings = ImmutableMap.builder() + .put("BooleanType", WriteMapping.booleanMapping("boolean", StandardColumnMappings.booleanWriteFunction())) + .put("BigintType", WriteMapping.longMapping("number(19)", StandardColumnMappings.bigintWriteFunction())) + .put("IntegerType", WriteMapping.longMapping("number(10)", StandardColumnMappings.integerWriteFunction())) + .put("SmallintType", WriteMapping.longMapping("number(5)", StandardColumnMappings.smallintWriteFunction())) + .put("TinyintType", WriteMapping.longMapping("number(3)", StandardColumnMappings.tinyintWriteFunction())) + .put("DoubleType", WriteMapping.doubleMapping("double precision", StandardColumnMappings.doubleWriteFunction())) + .put("RealType", WriteMapping.longMapping("real", StandardColumnMappings.realWriteFunction())) + .put("VarbinaryType", WriteMapping.sliceMapping("varbinary", StandardColumnMappings.varbinaryWriteFunction())) + .put("DateType", WriteMapping.longMapping("date", snowFlakeDateWriter())) + .buildOrThrow(); + + WriteMapping writeMapping = standardWriteMappings.get(simple); if (writeMapping != null) { return writeMapping; } - WriteMappingFunction writeMappingFunction = SNOWFLAKE_WRITE_MAPPINGS.get(simple); + final Map snowflakeWriteMappings = ImmutableMap.builder() + .put("TimeType", writeType -> { + return WriteMapping.longMapping("time", SnowflakeClient.snowFlaketimeWriter(writeType)); + }) + .put("ShortTimestampType", writeType -> { + WriteMapping myMap = SnowflakeClient.snowFlakeTimestampWriter(writeType); + return myMap; + }) + .put("ShortTimestampWithTimeZoneType", writeType -> { + WriteMapping myMap = SnowflakeClient.snowFlakeTimestampWithTZWriter(writeType); + return myMap; + }) + .put("LongTimestampType", writeType -> { + WriteMapping myMap = SnowflakeClient.snowFlakeTimestampWithTZWriter(writeType); + return myMap; + }) + .put("LongTimestampWithTimeZoneType", writeType -> { + WriteMapping myMap = SnowflakeClient.snowFlakeTimestampWithTZWriter(writeType); + return myMap; + }) + .put("VarcharType", writeType -> { + WriteMapping myMap = SnowflakeClient.snowFlakeVarCharWriter(writeType); + return myMap; + }) + .put("CharType", writeType -> { + WriteMapping myMap = SnowflakeClient.snowFlakeCharWriter(writeType); + return myMap; + }) + .put("LongDecimalType", writeType -> { + WriteMapping myMap = SnowflakeClient.snowFlakeDecimalWriter(writeType); + return myMap; + }) + .put("ShortDecimalType", writeType -> { + WriteMapping myMap = SnowflakeClient.snowFlakeDecimalWriter(writeType); + return myMap; + }) + .buildOrThrow(); + + WriteMappingFunction writeMappingFunction = snowflakeWriteMappings.get(simple); if (writeMappingFunction != null) { return writeMappingFunction.convert(type); } - log.debug("SnowflakeClient.toWriteMapping: SNOWFLAKE_CONNECTOR_COLUMN_TYPE_NOT_SUPPORTED: Unsupported column type: " + type.getDisplayName() + ", simple:" + simple); - throw new TrinoException(NOT_SUPPORTED, "SNOWFLAKE_CONNECTOR_COLUMN_TYPE_NOT_SUPPORTED: Unsupported column type: " + type.getDisplayName() + ", simple:" + simple); } @@ -322,7 +416,6 @@ private static SliceReadFunction variantReadFunction() private static ColumnMapping columnMappingPushdown(ColumnMapping mapping) { if (mapping.getPredicatePushdownController() == PredicatePushdownController.DISABLE_PUSHDOWN) { - log.debug("SnowflakeClient.columnMappingPushdown: NOT_SUPPORTED mapping.getPredicatePushdownController() is DISABLE_PUSHDOWN. Type was " + mapping.getType()); throw new TrinoException(NOT_SUPPORTED, "mapping.getPredicatePushdownController() is DISABLE_PUSHDOWN. Type was " + mapping.getType()); } From a185e04c11dfd9473e5f49d8203e62126ea63f7d Mon Sep 17 00:00:00 2001 From: Teng YU Date: Thu, 11 Jan 2024 17:42:33 +0100 Subject: [PATCH 350/350] Update according to reviews 11/01/2024 --- docs/src/main/sphinx/connector/snowflake.md | 2 - plugin/trino-snowflake/pom.xml | 8 +- .../plugin/snowflake/SnowflakeClient.java | 157 +++--------------- .../snowflake/SnowflakeClientModule.java | 5 +- .../plugin/snowflake/SnowflakeConfig.java | 7 - .../snowflake/BaseSnowflakeConnectorTest.java | 4 - .../snowflake/SnowflakeQueryRunner.java | 2 - .../plugin/snowflake/TestSnowflakeConfig.java | 6 +- .../snowflake/TestSnowflakeConnectorTest.java | 3 +- .../snowflake/TestSnowflakeTypeMapping.java | 4 - 10 files changed, 37 insertions(+), 161 deletions(-) diff --git a/docs/src/main/sphinx/connector/snowflake.md b/docs/src/main/sphinx/connector/snowflake.md index 579764534e230..1ca16df602f2b 100644 --- a/docs/src/main/sphinx/connector/snowflake.md +++ b/docs/src/main/sphinx/connector/snowflake.md @@ -71,8 +71,6 @@ Trino supports the following Snowflake data types: | `date` | `date` | | `time` | `time` | | `timestampntz` | `timestamp` | -| `timestamptz` | `timestampTZ` | -| `timestampltz` | `timestampTZ` | Complete list of [Snowflake data types](https://docs.snowflake.com/en/sql-reference/intro-summary-data-types.html). diff --git a/plugin/trino-snowflake/pom.xml b/plugin/trino-snowflake/pom.xml index 33f14d2a93990..b99a536a724e6 100644 --- a/plugin/trino-snowflake/pom.xml +++ b/plugin/trino-snowflake/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 435-SNAPSHOT + 436-SNAPSHOT ../../pom.xml @@ -93,6 +93,12 @@ + + io.airlift + junit-extensions + test + + io.airlift testing diff --git a/plugin/trino-snowflake/src/main/java/io/trino/plugin/snowflake/SnowflakeClient.java b/plugin/trino-snowflake/src/main/java/io/trino/plugin/snowflake/SnowflakeClient.java index c68e786e2efeb..35fcb6dfc4074 100644 --- a/plugin/trino-snowflake/src/main/java/io/trino/plugin/snowflake/SnowflakeClient.java +++ b/plugin/trino-snowflake/src/main/java/io/trino/plugin/snowflake/SnowflakeClient.java @@ -17,7 +17,6 @@ import com.google.common.collect.ImmutableSet; import com.google.inject.Inject; import io.airlift.log.Logger; -import io.airlift.slice.Slices; import io.trino.plugin.base.aggregation.AggregateFunctionRewriter; import io.trino.plugin.base.aggregation.AggregateFunctionRule; import io.trino.plugin.base.expression.ConnectorExpressionRewriter; @@ -92,10 +91,10 @@ import java.util.function.BiFunction; import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.util.concurrent.MoreExecutors.directExecutor; +import static io.airlift.slice.Slices.utf8Slice; import static io.trino.plugin.jdbc.JdbcErrorCode.JDBC_ERROR; import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; -import static io.trino.spi.type.DecimalType.createDecimalType; +import static io.trino.spi.type.TimestampWithTimeZoneType.MAX_SHORT_PRECISION; import static io.trino.spi.type.TimestampWithTimeZoneType.createTimestampWithTimeZoneType; import static io.trino.spi.type.Timestamps.MILLISECONDS_PER_SECOND; import static io.trino.spi.type.Timestamps.NANOSECONDS_PER_MILLISECOND; @@ -110,11 +109,11 @@ public class SnowflakeClient /* TIME supports an optional precision parameter for fractional seconds, e.g. TIME(3). Time precision can range from 0 (seconds) to 9 (nanoseconds). The default precision is 9. All TIME values must be between 00:00:00 and 23:59:59.999999999. TIME internally stores “wallclock” time, and all operations on TIME values are performed without taking any time zone into consideration. */ - private static final int SNOWFLAKE_MAX_SUPPORTED_TIMESTAMP_PRECISION = 9; + private static final int MAX_SUPPORTED_TEMPORAL_PRECISION = 9; private static final Logger log = Logger.get(SnowflakeClient.class); - private static final DateTimeFormatter SNOWFLAKE_DATETIME_FORMATTER = DateTimeFormatter.ofPattern("y-MM-dd'T'HH:mm:ss.SSSSSSSSSXXX"); + private static final DateTimeFormatter SNOWFLAKE_DATETIME_FORMATTER = DateTimeFormatter.ofPattern("u-MM-dd'T'HH:mm:ss.SSSSSSSSSXXX"); private static final DateTimeFormatter SNOWFLAKE_DATE_FORMATTER = DateTimeFormatter.ofPattern("uuuu-MM-dd"); - private static final DateTimeFormatter SNOWFLAKE_TIMESTAMP_FORMATTER = DateTimeFormatter.ofPattern("y-MM-dd'T'HH:mm:ss.SSSSSSSSS"); + private static final DateTimeFormatter SNOWFLAKE_TIMESTAMP_FORMATTER = DateTimeFormatter.ofPattern("u-MM-dd'T'HH:mm:ss.SSSSSSSSS"); private static final DateTimeFormatter SNOWFLAKE_TIME_FORMATTER = DateTimeFormatter.ofPattern("HH:mm:ss.SSSSSSSSS"); private final AggregateFunctionRewriter aggregateFunctionRewriter; @@ -129,81 +128,6 @@ private interface ColumnMappingFunction } private static final TimeZone UTC_TZ = TimeZone.getTimeZone(ZoneId.of("UTC")); - // Mappings for JDBC column types to internal Trino types - private static final Map STANDARD_COLUMN_MAPPINGS = ImmutableMap.builder() - .put(Types.BOOLEAN, StandardColumnMappings.booleanColumnMapping()) - .put(Types.TINYINT, StandardColumnMappings.tinyintColumnMapping()) - .put(Types.SMALLINT, StandardColumnMappings.smallintColumnMapping()) - .put(Types.INTEGER, StandardColumnMappings.integerColumnMapping()) - .put(Types.BIGINT, StandardColumnMappings.bigintColumnMapping()) - .put(Types.REAL, StandardColumnMappings.realColumnMapping()) - .put(Types.DOUBLE, StandardColumnMappings.doubleColumnMapping()) - .put(Types.FLOAT, StandardColumnMappings.doubleColumnMapping()) - .put(Types.BINARY, StandardColumnMappings.varbinaryColumnMapping()) - .put(Types.VARBINARY, StandardColumnMappings.varbinaryColumnMapping()) - .put(Types.LONGVARBINARY, StandardColumnMappings.varbinaryColumnMapping()) - .buildOrThrow(); - - private static final Map SHOWFLAKE_COLUMN_MAPPINGS = ImmutableMap.builder() - .put("time", typeHandle -> Optional.of(timeColumnMapping(typeHandle))) - .put("timestampntz", typeHandle -> Optional.of(timestampColumnMapping(typeHandle))) - .put("timestamptz", typeHandle -> Optional.of(timestampTzColumnMapping(typeHandle))) - .put("timestampltz", typeHandle -> Optional.of(timestampTzColumnMapping(typeHandle))) - .put("date", typeHandle -> Optional.of(ColumnMapping.longMapping( - DateType.DATE, - (resultSet, columnIndex) -> LocalDate.ofEpochDay(resultSet.getLong(columnIndex)).toEpochDay(), - snowFlakeDateWriter()))) - .put("object", typeHandle -> Optional.of(ColumnMapping.sliceMapping( - createUnboundedVarcharType(), - StandardColumnMappings.varcharReadFunction(createUnboundedVarcharType()), - StandardColumnMappings.varcharWriteFunction(), - PredicatePushdownController.DISABLE_PUSHDOWN))) - .put("array", typeHandle -> Optional.of(ColumnMapping.sliceMapping( - createUnboundedVarcharType(), - StandardColumnMappings.varcharReadFunction(createUnboundedVarcharType()), - StandardColumnMappings.varcharWriteFunction(), - PredicatePushdownController.DISABLE_PUSHDOWN))) - .put("variant", typeHandle -> Optional.of(ColumnMapping.sliceMapping( - createUnboundedVarcharType(), - variantReadFunction(), - StandardColumnMappings.varcharWriteFunction(), - PredicatePushdownController.FULL_PUSHDOWN))) - .put("varchar", typeHandle -> Optional.of(varcharColumnMapping(typeHandle.getRequiredColumnSize()))) - .put("number", typeHandle -> { - int decimalDigits = typeHandle.getRequiredDecimalDigits(); - int precision = typeHandle.getRequiredColumnSize() + Math.max(-decimalDigits, 0); - if (precision > 38) { - return Optional.empty(); - } - return Optional.of(columnMappingPushdown( - StandardColumnMappings.decimalColumnMapping(createDecimalType(precision, Math.max(decimalDigits, 0)), RoundingMode.UNNECESSARY))); - }) - .buildOrThrow(); - - // Mappings for internal Trino types to JDBC column types - private static final Map STANDARD_WRITE_MAPPINGS = ImmutableMap.builder() - .put("BooleanType", WriteMapping.booleanMapping("boolean", StandardColumnMappings.booleanWriteFunction())) - .put("BigintType", WriteMapping.longMapping("number(19)", StandardColumnMappings.bigintWriteFunction())) - .put("IntegerType", WriteMapping.longMapping("number(10)", StandardColumnMappings.integerWriteFunction())) - .put("SmallintType", WriteMapping.longMapping("number(5)", StandardColumnMappings.smallintWriteFunction())) - .put("TinyintType", WriteMapping.longMapping("number(3)", StandardColumnMappings.tinyintWriteFunction())) - .put("DoubleType", WriteMapping.doubleMapping("double precision", StandardColumnMappings.doubleWriteFunction())) - .put("RealType", WriteMapping.longMapping("real", StandardColumnMappings.realWriteFunction())) - .put("VarbinaryType", WriteMapping.sliceMapping("varbinary", StandardColumnMappings.varbinaryWriteFunction())) - .put("DateType", WriteMapping.longMapping("date", snowFlakeDateWriter())) - .buildOrThrow(); - - private static final Map SNOWFLAKE_WRITE_MAPPINGS = ImmutableMap.builder() - .put("TimeType", type -> WriteMapping.longMapping("time", SnowflakeClient.snowFlaketimeWriter(type))) - .put("ShortTimestampType", SnowflakeClient::snowFlakeTimestampWriter) - .put("ShortTimestampWithTimeZoneType", SnowflakeClient::snowFlakeTimestampWithTZWriter) - .put("LongTimestampType", SnowflakeClient::snowFlakeTimestampWithTZWriter) - .put("LongTimestampWithTimeZoneType", SnowflakeClient::snowFlakeTimestampWithTZWriter) - .put("VarcharType", SnowflakeClient::snowFlakeVarCharWriter) - .put("CharType", SnowflakeClient::snowFlakeCharWriter) - .put("LongDecimalType", SnowflakeClient::snowFlakeDecimalWriter) - .put("ShortDecimalType", SnowflakeClient::snowFlakeDecimalWriter) - .buildOrThrow(); @Inject public SnowflakeClient( @@ -232,23 +156,9 @@ public SnowflakeClient( .build()); } - @Override - public void abortReadConnection(Connection connection, ResultSet resultSet) - throws SQLException - { - // Abort connection before closing. Without this, the Snowflake driver - // attempts to drain the connection by reading all the results. - connection.abort(directExecutor()); - } - @Override public Optional toColumnMapping(ConnectorSession session, Connection connection, JdbcTypeHandle typeHandle) { - Optional mapping = getForcedMappingToVarchar(typeHandle); - if (mapping.isPresent()) { - return mapping; - } - String jdbcTypeName = typeHandle.getJdbcTypeName() .orElseThrow(() -> new TrinoException(JDBC_ERROR, "Type name is missing: " + typeHandle)); jdbcTypeName = jdbcTypeName.toLowerCase(Locale.ENGLISH); @@ -275,18 +185,9 @@ public Optional toColumnMapping(ConnectorSession session, Connect } final Map snowflakeColumnMappings = ImmutableMap.builder() - .put("time", handle -> { - return Optional.of(timeColumnMapping(handle)); - }) - .put("date", handle -> { - return Optional.of(ColumnMapping.longMapping( - DateType.DATE, (resultSet, columnIndex) -> - LocalDate.ofEpochDay(resultSet.getLong(columnIndex)).toEpochDay(), - snowFlakeDateWriter())); - }) - .put("varchar", handle -> { - return Optional.of(varcharColumnMapping(handle.getRequiredColumnSize())); - }) + .put("time", handle -> { return Optional.of(timeColumnMapping(handle.getRequiredDecimalDigits())); }) + .put("date", handle -> { return Optional.of(ColumnMapping.longMapping(DateType.DATE, (resultSet, columnIndex) -> LocalDate.ofEpochDay(resultSet.getLong(columnIndex)).toEpochDay(), snowFlakeDateWriter())); }) + .put("varchar", handle -> { return Optional.of(varcharColumnMapping(handle.getRequiredColumnSize())); }) .put("number", handle -> { int decimalDigits = handle.getRequiredDecimalDigits(); int precision = handle.getRequiredColumnSize() + Math.max(-decimalDigits, 0); @@ -294,8 +195,7 @@ public Optional toColumnMapping(ConnectorSession session, Connect return Optional.empty(); } return Optional.of(columnMappingPushdown( - StandardColumnMappings.decimalColumnMapping(DecimalType.createDecimalType( - precision, Math.max(decimalDigits, 0)), RoundingMode.UNNECESSARY))); + StandardColumnMappings.decimalColumnMapping(DecimalType.createDecimalType(precision, Math.max(decimalDigits, 0)), RoundingMode.UNNECESSARY))); }) .buildOrThrow(); @@ -305,7 +205,7 @@ public Optional toColumnMapping(ConnectorSession session, Connect } // Code should never reach here so throw an error. - throw new TrinoException(NOT_SUPPORTED, "SNOWFLAKE_CONNECTOR_COLUMN_TYPE_NOT_SUPPORTED: Unsupported column type(" + type + "):" + jdbcTypeName); + throw new TrinoException(NOT_SUPPORTED, "Unsupported column type(" + type + "):" + jdbcTypeName); } @Override @@ -334,7 +234,7 @@ public WriteMapping toWriteMapping(ConnectorSession session, Type type) final Map snowflakeWriteMappings = ImmutableMap.builder() .put("TimeType", writeType -> { - return WriteMapping.longMapping("time", SnowflakeClient.snowFlaketimeWriter(writeType)); + return WriteMapping.longMapping("time", timeWriteFunction(((TimeType) writeType).getPrecision())); }) .put("ShortTimestampType", writeType -> { WriteMapping myMap = SnowflakeClient.snowFlakeTimestampWriter(writeType); @@ -375,7 +275,7 @@ public WriteMapping toWriteMapping(ConnectorSession session, Type type) return writeMappingFunction.convert(type); } - throw new TrinoException(NOT_SUPPORTED, "SNOWFLAKE_CONNECTOR_COLUMN_TYPE_NOT_SUPPORTED: Unsupported column type: " + type.getDisplayName() + ", simple:" + simple); + throw new TrinoException(NOT_SUPPORTED, "Unsupported column type: " + type.getDisplayName() + ", simple:" + simple); } @Override @@ -410,7 +310,7 @@ public void setColumnType(ConnectorSession session, JdbcTableHandle handle, Jdbc private static SliceReadFunction variantReadFunction() { - return (resultSet, columnIndex) -> Slices.utf8Slice(resultSet.getString(columnIndex).replaceAll("^\"|\"$", "")); + return (resultSet, columnIndex) -> utf8Slice(resultSet.getString(columnIndex).replaceAll("^\"|\"$", "")); } private static ColumnMapping columnMappingPushdown(ColumnMapping mapping) @@ -422,10 +322,9 @@ private static ColumnMapping columnMappingPushdown(ColumnMapping mapping) return new ColumnMapping(mapping.getType(), mapping.getReadFunction(), mapping.getWriteFunction(), PredicatePushdownController.FULL_PUSHDOWN); } - private static ColumnMapping timeColumnMapping(JdbcTypeHandle typeHandle) + private static ColumnMapping timeColumnMapping(int precision) { - int precision = typeHandle.getRequiredDecimalDigits(); - checkArgument(precision <= SNOWFLAKE_MAX_SUPPORTED_TIMESTAMP_PRECISION, "The max timestamp precision in Snowflake is " + SNOWFLAKE_MAX_SUPPORTED_TIMESTAMP_PRECISION); + checkArgument(precision <= MAX_SUPPORTED_TEMPORAL_PRECISION, "The max timestamp precision in Snowflake is " + MAX_SUPPORTED_TEMPORAL_PRECISION); return ColumnMapping.longMapping( TimeType.createTimeType(precision), (resultSet, columnIndex) -> { @@ -436,21 +335,15 @@ private static ColumnMapping timeColumnMapping(JdbcTypeHandle typeHandle) PredicatePushdownController.FULL_PUSHDOWN); } - private static LongWriteFunction snowFlaketimeWriter(Type type) - { - return timeWriteFunction(((TimeType) type).getPrecision()); - } - private static LongWriteFunction timeWriteFunction(int precision) { - checkArgument(precision <= SNOWFLAKE_MAX_SUPPORTED_TIMESTAMP_PRECISION, "Unsupported precision: %s", precision); - String bindExpression = format("CAST(? AS time(%s))", precision); + checkArgument(precision <= MAX_SUPPORTED_TEMPORAL_PRECISION, "Unsupported precision: %s", precision); return new LongWriteFunction() { @Override public String getBindExpression() { - return bindExpression; + return format("CAST(? AS time(%s))", precision); } @Override @@ -474,16 +367,14 @@ private static ColumnMapping timestampTzColumnMapping(JdbcTypeHandle typeHandle) String jdbcTypeName = typeHandle.getJdbcTypeName() .orElseThrow(() -> new TrinoException(JDBC_ERROR, "Type name is missing: " + typeHandle)); int type = typeHandle.getJdbcType(); - log.debug("timestampTZColumnMapping: jdbcTypeName(%s):%s precision:%s", type, jdbcTypeName, precision); - - if (precision <= 3) { + if (precision <= MAX_SHORT_PRECISION) { return ColumnMapping.longMapping( createTimestampWithTimeZoneType(precision), (resultSet, columnIndex) -> { ZonedDateTime timestamp = SNOWFLAKE_DATETIME_FORMATTER.parse(resultSet.getString(columnIndex), ZonedDateTime::from); return DateTimeEncoding.packDateTimeWithZone(timestamp.toInstant().toEpochMilli(), timestamp.getZone().getId()); }, - timestampWithTZWriter(), + timestampWithTimezoneWriteFunction(), PredicatePushdownController.FULL_PUSHDOWN); } else { @@ -568,8 +459,8 @@ private static WriteMapping snowFlakeTimestampWriter(Type type) { TimestampType timestampType = (TimestampType) type; checkArgument( - timestampType.getPrecision() <= SNOWFLAKE_MAX_SUPPORTED_TIMESTAMP_PRECISION, - "The max timestamp precision in Snowflake is " + SNOWFLAKE_MAX_SUPPORTED_TIMESTAMP_PRECISION); + timestampType.getPrecision() <= MAX_SUPPORTED_TEMPORAL_PRECISION, + "The max timestamp precision in Snowflake is " + MAX_SUPPORTED_TEMPORAL_PRECISION); if (timestampType.isShort()) { return WriteMapping.longMapping(format("timestamp_ntz(%d)", timestampType.getPrecision()), timestampWriteFunction()); @@ -593,14 +484,14 @@ private static WriteMapping snowFlakeTimestampWithTZWriter(Type type) { TimestampWithTimeZoneType timeTZType = (TimestampWithTimeZoneType) type; - checkArgument(timeTZType.getPrecision() <= SNOWFLAKE_MAX_SUPPORTED_TIMESTAMP_PRECISION, "Max Snowflake precision is is " + SNOWFLAKE_MAX_SUPPORTED_TIMESTAMP_PRECISION); + checkArgument(timeTZType.getPrecision() <= MAX_SUPPORTED_TEMPORAL_PRECISION, "Max Snowflake precision is is " + MAX_SUPPORTED_TEMPORAL_PRECISION); if (timeTZType.isShort()) { - return WriteMapping.longMapping(format("timestamp_tz(%d)", timeTZType.getPrecision()), timestampWithTZWriter()); + return WriteMapping.longMapping(format("timestamp_tz(%d)", timeTZType.getPrecision()), timestampWithTimezoneWriteFunction()); } return WriteMapping.objectMapping(format("timestamp_tz(%d)", timeTZType.getPrecision()), longTimestampWithTzWriteFunction()); } - private static LongWriteFunction timestampWithTZWriter() + private static LongWriteFunction timestampWithTimezoneWriteFunction() { return (statement, index, encodedTimeWithZone) -> { Instant instant = Instant.ofEpochMilli(DateTimeEncoding.unpackMillisUtc(encodedTimeWithZone)); diff --git a/plugin/trino-snowflake/src/main/java/io/trino/plugin/snowflake/SnowflakeClientModule.java b/plugin/trino-snowflake/src/main/java/io/trino/plugin/snowflake/SnowflakeClientModule.java index 19fc358471915..587ca8d11faab 100644 --- a/plugin/trino-snowflake/src/main/java/io/trino/plugin/snowflake/SnowflakeClientModule.java +++ b/plugin/trino-snowflake/src/main/java/io/trino/plugin/snowflake/SnowflakeClientModule.java @@ -18,6 +18,7 @@ import com.google.inject.Provides; import com.google.inject.Scopes; import com.google.inject.Singleton; +import io.opentelemetry.api.OpenTelemetry; import io.trino.plugin.jdbc.BaseJdbcConfig; import io.trino.plugin.jdbc.ConnectionFactory; import io.trino.plugin.jdbc.DriverConnectionFactory; @@ -49,7 +50,7 @@ public void configure(Binder binder) @Singleton @Provides @ForBaseJdbc - public ConnectionFactory getConnectionFactory(BaseJdbcConfig baseJdbcConfig, SnowflakeConfig snowflakeConfig, CredentialProvider credentialProvider) + public ConnectionFactory getConnectionFactory(BaseJdbcConfig baseJdbcConfig, SnowflakeConfig snowflakeConfig, CredentialProvider credentialProvider, OpenTelemetry openTelemetry) throws MalformedURLException { Properties properties = new Properties(); @@ -90,6 +91,6 @@ public ConnectionFactory getConnectionFactory(BaseJdbcConfig baseJdbcConfig, Sno } } - return new DriverConnectionFactory(new SnowflakeDriver(), baseJdbcConfig.getConnectionUrl(), properties, credentialProvider); + return new DriverConnectionFactory(new SnowflakeDriver(), baseJdbcConfig.getConnectionUrl(), properties, credentialProvider, openTelemetry); } } diff --git a/plugin/trino-snowflake/src/main/java/io/trino/plugin/snowflake/SnowflakeConfig.java b/plugin/trino-snowflake/src/main/java/io/trino/plugin/snowflake/SnowflakeConfig.java index 6dbf125201774..c002728f85b76 100644 --- a/plugin/trino-snowflake/src/main/java/io/trino/plugin/snowflake/SnowflakeConfig.java +++ b/plugin/trino-snowflake/src/main/java/io/trino/plugin/snowflake/SnowflakeConfig.java @@ -79,13 +79,6 @@ public Optional getTimestampNoTimezoneAsUTC() return Optional.ofNullable(timestampNoTimezoneAsUTC); } - @Config("snowflake.timestamp-no-timezone-as-utc") - public SnowflakeConfig setTimestampNoTimezoneAsUTC(Boolean timestampNoTimezoneAsUTC) - { - this.timestampNoTimezoneAsUTC = timestampNoTimezoneAsUTC; - return this; - } - public Optional getHTTPProxy() { return Optional.ofNullable(httpProxy); diff --git a/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/BaseSnowflakeConnectorTest.java b/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/BaseSnowflakeConnectorTest.java index b547624cebeac..0b64ddd61ee1c 100644 --- a/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/BaseSnowflakeConnectorTest.java +++ b/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/BaseSnowflakeConnectorTest.java @@ -59,12 +59,10 @@ protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior) return false; case SUPPORTS_COMMENT_ON_COLUMN: case SUPPORTS_ADD_COLUMN_WITH_COMMENT: - case SUPPORTS_COMMENT_ON_TABLE: case SUPPORTS_CREATE_TABLE_WITH_TABLE_COMMENT: case SUPPORTS_CREATE_TABLE_WITH_COLUMN_COMMENT: case SUPPORTS_SET_COLUMN_TYPE: return false; - case SUPPORTS_DROP_FIELD: case SUPPORTS_ROW_TYPE: case SUPPORTS_ARRAY: return false; @@ -323,7 +321,6 @@ public void testCreateTableAsSelect() "SELECT 1234567890, 123", "SELECT count(*) + 1 FROM nation"); - // TODO: BigQuery throws table not found at BigQueryClient.insert if we reuse the same table name tableName = "test_ctas" + randomNameSuffix(); assertExplainAnalyze("EXPLAIN ANALYZE CREATE TABLE " + tableName + " AS SELECT name FROM nation"); assertQuery("SELECT * from " + tableName, "SELECT name FROM nation"); @@ -357,7 +354,6 @@ public void testCreateTable() assertQueryFails("CREATE TABLE " + tableName + " (a bad_type)", ".* Unknown type 'bad_type' for column 'a'"); assertFalse(getQueryRunner().tableExists(getSession(), tableName)); - // TODO (https://github.com/trinodb/trino/issues/5901) revert to longer name when Oracle version is updated tableName = "test_cr_not_exists_" + randomNameSuffix(); assertUpdate("CREATE TABLE " + tableName + " (a bigint, b varchar(50), c double)"); assertTrue(getQueryRunner().tableExists(getSession(), tableName)); diff --git a/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/SnowflakeQueryRunner.java b/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/SnowflakeQueryRunner.java index a50debaf003b6..2f877068f88af 100644 --- a/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/SnowflakeQueryRunner.java +++ b/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/SnowflakeQueryRunner.java @@ -36,7 +36,6 @@ public final class SnowflakeQueryRunner private SnowflakeQueryRunner() {} public static DistributedQueryRunner createSnowflakeQueryRunner( - TestingSnowflakeServer server, Map extraProperties, Map connectorProperties, Iterable> tables) @@ -85,7 +84,6 @@ public static void main(String[] args) throws Exception { DistributedQueryRunner queryRunner = createSnowflakeQueryRunner( - new TestingSnowflakeServer(), ImmutableMap.of("http-server.http.port", "8080"), ImmutableMap.of(), ImmutableList.of()); diff --git a/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/TestSnowflakeConfig.java b/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/TestSnowflakeConfig.java index eb5c32a3d063c..93b4dc8dff9b0 100644 --- a/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/TestSnowflakeConfig.java +++ b/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/TestSnowflakeConfig.java @@ -32,8 +32,7 @@ public void testDefaults() .setDatabase(null) .setRole(null) .setWarehouse(null) - .setHTTPProxy(null) - .setTimestampNoTimezoneAsUTC(null)); + .setHTTPProxy(null)); } @Test @@ -53,8 +52,7 @@ public void testExplicitPropertyMappings() .setDatabase("MYDATABASE") .setRole("MYROLE") .setWarehouse("MYWAREHOUSE") - .setHTTPProxy("MYPROXY") - .setTimestampNoTimezoneAsUTC(true); + .setHTTPProxy("MYPROXY"); assertFullMapping(properties, expected); } diff --git a/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/TestSnowflakeConnectorTest.java b/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/TestSnowflakeConnectorTest.java index 8b9b0c78c73b5..b448e5756c0b0 100644 --- a/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/TestSnowflakeConnectorTest.java +++ b/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/TestSnowflakeConnectorTest.java @@ -26,8 +26,7 @@ public class TestSnowflakeConnectorTest protected QueryRunner createQueryRunner() throws Exception { - server = closeAfterClass(new TestingSnowflakeServer()); - return createSnowflakeQueryRunner(server, ImmutableMap.of(), ImmutableMap.of(), REQUIRED_TPCH_TABLES); + return createSnowflakeQueryRunner(ImmutableMap.of(), ImmutableMap.of(), REQUIRED_TPCH_TABLES); } @Override diff --git a/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/TestSnowflakeTypeMapping.java b/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/TestSnowflakeTypeMapping.java index 5377e013a6cd8..1e7a28572b6e8 100644 --- a/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/TestSnowflakeTypeMapping.java +++ b/plugin/trino-snowflake/src/test/java/io/trino/plugin/snowflake/TestSnowflakeTypeMapping.java @@ -65,7 +65,6 @@ public class TestSnowflakeTypeMapping @BeforeAll public void setUp() { - String zone = jvmZone.getId(); checkState(jvmZone.getId().equals("America/Bahia_Banderas"), "Timezone not configured correctly. Add -Duser.timezone=America/Bahia_Banderas to your JVM arguments"); checkIsGap(jvmZone, LocalDate.of(1970, 1, 1)); checkIsGap(vilnius, LocalDate.of(1983, 4, 1)); @@ -76,9 +75,7 @@ public void setUp() protected QueryRunner createQueryRunner() throws Exception { - snowflakeServer = new TestingSnowflakeServer(); return createSnowflakeQueryRunner( - snowflakeServer, ImmutableMap.of(), ImmutableMap.of(), ImmutableList.of()); @@ -322,7 +319,6 @@ private void testTimestamp(ZoneId sessionZone) .build(); SqlDataTypeTest.create() - // after epoch (MariaDb's timestamp type doesn't support values <= epoch) .addRoundTrip("timestamp(3)", "TIMESTAMP '2019-03-18 10:01:17.987'", createTimestampType(3), "TIMESTAMP '2019-03-18 10:01:17.987'") // time doubled in JVM zone .addRoundTrip("timestamp(3)", "TIMESTAMP '2018-10-28 01:33:17.456'", createTimestampType(3), "TIMESTAMP '2018-10-28 01:33:17.456'")